diff --git a/chain/chain/src/flat_storage_creator.rs b/chain/chain/src/flat_storage_creator.rs
index 11e7a2e99f1..a00410a8496 100644
--- a/chain/chain/src/flat_storage_creator.rs
+++ b/chain/chain/src/flat_storage_creator.rs
@@ -18,9 +18,9 @@ use near_primitives::state::ValueRef;
use near_primitives::state_part::PartId;
use near_primitives::types::{AccountId, BlockHeight, StateRoot};
use near_store::flat::{
- store_helper, BlockInfo, FetchingStateStatus, FlatStateChanges, FlatStorageCreationMetrics,
- FlatStorageCreationStatus, FlatStorageReadyStatus, FlatStorageStatus, NUM_PARTS_IN_ONE_STEP,
- STATE_PART_MEMORY_LIMIT,
+ store_helper, BlockInfo, FetchingStateStatus, FlatStateChanges, FlatStateValue,
+ FlatStorageCreationMetrics, FlatStorageCreationStatus, FlatStorageReadyStatus,
+ FlatStorageStatus, NUM_PARTS_IN_ONE_STEP, STATE_PART_MEMORY_LIMIT,
};
use near_store::Store;
use near_store::{Trie, TrieDBStorage, TrieTraversalItem};
@@ -104,9 +104,13 @@ impl FlatStorageShardCreator {
{
if let Some(key) = key {
let value = trie.storage.retrieve_raw_bytes(&hash).unwrap();
- let value_ref = ValueRef::new(&value);
- store_helper::set_ref(&mut store_update, shard_uid, key, Some(value_ref))
- .expect("Failed to put value in FlatState");
+ store_helper::set_flat_state_value(
+ &mut store_update,
+ shard_uid,
+ key,
+ Some(FlatStateValue::value_ref(&value)),
+ )
+ .expect("Failed to put value in FlatState");
num_items += 1;
}
}
diff --git a/core/store/src/flat/chunk_view.rs b/core/store/src/flat/chunk_view.rs
index 4a92e6f190d..d1b523cffb0 100644
--- a/core/store/src/flat/chunk_view.rs
+++ b/core/store/src/flat/chunk_view.rs
@@ -1,9 +1,8 @@
use near_primitives::hash::CryptoHash;
-use near_primitives::state::ValueRef;
use crate::Store;
-use super::FlatStorage;
+use super::{FlatStateValue, FlatStorage};
/// Struct for getting value references from the flat storage, corresponding
/// to some block defined in `blocks_to_head`.
@@ -39,7 +38,7 @@ impl FlatStorageChunkView {
/// they are stored in `DBCol::State`. Also the separation is done so we
/// could charge users for the value length before loading the value.
// TODO (#7327): consider inlining small values, so we could use only one db access.
- pub fn get_ref(&self, key: &[u8]) -> Result, crate::StorageError> {
- self.flat_storage.get_ref(&self.block_hash, key)
+ pub fn get_value(&self, key: &[u8]) -> Result , crate::StorageError> {
+ self.flat_storage.get_value(&self.block_hash, key)
}
}
diff --git a/core/store/src/flat/delta.rs b/core/store/src/flat/delta.rs
index fe8e03180af..1eb2ed08900 100644
--- a/core/store/src/flat/delta.rs
+++ b/core/store/src/flat/delta.rs
@@ -7,7 +7,8 @@ use near_primitives::types::RawStateChangesWithTrieKey;
use std::collections::HashMap;
use std::sync::Arc;
-use super::{store_helper, BlockInfo};
+use super::types::INLINE_DISK_VALUE_THRESHOLD;
+use super::{store_helper, BlockInfo, FlatStateValue};
use crate::{CryptoHash, StoreUpdate};
#[derive(Debug)]
@@ -34,14 +35,14 @@ impl KeyForFlatStateDelta {
res
}
}
-/// Delta of the state for some shard and block, stores mapping from keys to value refs or None, if key was removed in
-/// this block.
+/// Delta of the state for some shard and block, stores mapping from keys to values
+/// or None, if key was removed in this block.
#[derive(BorshSerialize, BorshDeserialize, Clone, Default, PartialEq, Eq)]
-pub struct FlatStateChanges(pub(crate) HashMap, Option>);
+pub struct FlatStateChanges(pub(crate) HashMap, Option>);
impl From for FlatStateChanges
where
- T: IntoIterator- , Option
)>,
+ T: IntoIterator- , Option
)>,
{
fn from(iter: T) -> Self {
Self(HashMap::from_iter(iter))
@@ -57,13 +58,17 @@ impl std::fmt::Debug for FlatStateChanges {
}
impl FlatStateChanges {
- /// Returns `Some(Option)` from delta for the given key. If key is not present, returns None.
- pub fn get(&self, key: &[u8]) -> Option> {
+ /// Returns `Some(Option)` from delta for the given key. If key is not present, returns None.
+ pub fn get(&self, key: &[u8]) -> Option> {
self.0.get(key).cloned()
}
/// Inserts a key-value pair to delta.
- pub fn insert(&mut self, key: Vec, value: Option) -> Option> {
+ pub fn insert(
+ &mut self,
+ key: Vec,
+ value: Option,
+ ) -> Option> {
self.0.insert(key, value)
}
@@ -88,12 +93,14 @@ impl FlatStateChanges {
.last()
.expect("Committed entry should have at least one change")
.data;
- match last_change {
- Some(value) => {
- delta.insert(key, Some(near_primitives::state::ValueRef::new(value)))
+ let flat_state_value = last_change.map(|value| {
+ if value.len() <= INLINE_DISK_VALUE_THRESHOLD {
+ FlatStateValue::inlined(&value)
+ } else {
+ FlatStateValue::value_ref(&value)
}
- None => delta.insert(key, None),
- };
+ });
+ delta.insert(key, flat_state_value);
}
Self(delta)
}
@@ -101,7 +108,8 @@ impl FlatStateChanges {
/// Applies delta to the flat state.
pub fn apply_to_flat_state(self, store_update: &mut StoreUpdate, shard_uid: ShardUId) {
for (key, value) in self.0.into_iter() {
- store_helper::set_ref(store_update, shard_uid, key, value).expect("Borsh cannot fail");
+ store_helper::set_flat_state_value(store_update, shard_uid, key, value)
+ .expect("Borsh cannot fail");
}
}
}
@@ -117,7 +125,13 @@ pub struct CachedFlatStateDelta {
impl From for CachedFlatStateChanges {
fn from(delta: FlatStateChanges) -> Self {
- Self(delta.0.into_iter().map(|(key, value)| (hash(&key), value)).collect())
+ Self(
+ delta
+ .0
+ .into_iter()
+ .map(|(key, value)| (hash(&key), value.map(|v| v.to_value_ref())))
+ .collect(),
+ )
}
}
@@ -144,8 +158,9 @@ impl CachedFlatStateChanges {
#[cfg(test)]
mod tests {
+ use crate::flat::FlatStateValue;
+
use super::FlatStateChanges;
- use near_primitives::state::ValueRef;
use near_primitives::trie_key::TrieKey;
use near_primitives::types::{RawStateChange, RawStateChangesWithTrieKey, StateChangeCause};
@@ -208,17 +223,17 @@ mod tests {
let flat_state_changes = FlatStateChanges::from_state_changes(&state_changes);
assert_eq!(
flat_state_changes.get(&alice_trie_key.to_vec()),
- Some(Some(ValueRef::new(&[3, 4])))
+ Some(Some(FlatStateValue::inlined(&[3, 4])))
);
assert_eq!(flat_state_changes.get(&bob_trie_key.to_vec()), Some(None));
assert_eq!(flat_state_changes.get(&carol_trie_key.to_vec()), None);
assert_eq!(
flat_state_changes.get(&delayed_trie_key.to_vec()),
- Some(Some(ValueRef::new(&[1])))
+ Some(Some(FlatStateValue::inlined(&[1])))
);
assert_eq!(
flat_state_changes.get(&delayed_receipt_trie_key.to_vec()),
- Some(Some(ValueRef::new(&[2])))
+ Some(Some(FlatStateValue::inlined(&[2])))
);
}
@@ -227,23 +242,23 @@ mod tests {
#[test]
fn flat_state_changes_merge() {
let mut changes = FlatStateChanges::from([
- (vec![1], Some(ValueRef::new(&[4]))),
- (vec![2], Some(ValueRef::new(&[5]))),
+ (vec![1], Some(FlatStateValue::value_ref(&[4]))),
+ (vec![2], Some(FlatStateValue::value_ref(&[5]))),
(vec![3], None),
- (vec![4], Some(ValueRef::new(&[6]))),
+ (vec![4], Some(FlatStateValue::value_ref(&[6]))),
]);
let changes_new = FlatStateChanges::from([
- (vec![2], Some(ValueRef::new(&[7]))),
- (vec![3], Some(ValueRef::new(&[8]))),
+ (vec![2], Some(FlatStateValue::value_ref(&[7]))),
+ (vec![3], Some(FlatStateValue::value_ref(&[8]))),
(vec![4], None),
- (vec![5], Some(ValueRef::new(&[9]))),
+ (vec![5], Some(FlatStateValue::value_ref(&[9]))),
]);
changes.merge(changes_new);
- assert_eq!(changes.get(&[1]), Some(Some(ValueRef::new(&[4]))));
- assert_eq!(changes.get(&[2]), Some(Some(ValueRef::new(&[7]))));
- assert_eq!(changes.get(&[3]), Some(Some(ValueRef::new(&[8]))));
+ assert_eq!(changes.get(&[1]), Some(Some(FlatStateValue::value_ref(&[4]))));
+ assert_eq!(changes.get(&[2]), Some(Some(FlatStateValue::value_ref(&[7]))));
+ assert_eq!(changes.get(&[3]), Some(Some(FlatStateValue::value_ref(&[8]))));
assert_eq!(changes.get(&[4]), Some(None));
- assert_eq!(changes.get(&[5]), Some(Some(ValueRef::new(&[9]))));
+ assert_eq!(changes.get(&[5]), Some(Some(FlatStateValue::value_ref(&[9]))));
}
}
diff --git a/core/store/src/flat/mod.rs b/core/store/src/flat/mod.rs
index 328df2db5e4..0eb0564c36d 100644
--- a/core/store/src/flat/mod.rs
+++ b/core/store/src/flat/mod.rs
@@ -39,7 +39,7 @@ pub use manager::FlatStorageManager;
pub use metrics::FlatStorageCreationMetrics;
pub use storage::FlatStorage;
pub use types::{
- BlockInfo, FetchingStateStatus, FlatStorageCreationStatus, FlatStorageError,
+ BlockInfo, FetchingStateStatus, FlatStateValue, FlatStorageCreationStatus, FlatStorageError,
FlatStorageReadyStatus, FlatStorageStatus,
};
diff --git a/core/store/src/flat/storage.rs b/core/store/src/flat/storage.rs
index eb3d132a2a3..065821988d6 100644
--- a/core/store/src/flat/storage.rs
+++ b/core/store/src/flat/storage.rs
@@ -4,7 +4,6 @@ use std::sync::{Arc, RwLock};
use near_primitives::errors::StorageError;
use near_primitives::hash::CryptoHash;
use near_primitives::shard_layout::{ShardLayout, ShardUId};
-use near_primitives::state::ValueRef;
use tracing::{debug, info, warn};
use crate::flat::delta::CachedFlatStateChanges;
@@ -14,7 +13,7 @@ use crate::{Store, StoreUpdate};
use super::delta::{CachedFlatStateDelta, FlatStateDelta};
use super::metrics::FlatStorageMetrics;
-use super::types::FlatStorageError;
+use super::types::{FlatStateValue, FlatStorageError};
use super::{store_helper, BlockInfo};
/// FlatStorage stores information on which blocks flat storage current supports key lookups on.
@@ -177,11 +176,11 @@ impl FlatStorage {
guard.get_blocks_to_head(target_block_hash)
}
- pub fn get_ref(
+ pub fn get_value(
&self,
block_hash: &CryptoHash,
key: &[u8],
- ) -> Result, crate::StorageError> {
+ ) -> Result , crate::StorageError> {
let guard = self.0.read().expect(super::POISONED_LOCK_ERR);
let blocks_to_head =
guard.get_blocks_to_head(block_hash).map_err(|e| StorageError::from(e))?;
@@ -190,14 +189,14 @@ impl FlatStorage {
let changes = guard.get_block_changes(block_hash)?;
match changes.get(key) {
Some(value_ref) => {
- return Ok(value_ref);
+ return Ok(value_ref.map(|value_ref| FlatStateValue::Ref(value_ref)));
}
None => {}
};
}
- let value_ref = store_helper::get_ref(&guard.store, guard.shard_uid, key)?;
- Ok(value_ref)
+ let value = store_helper::get_flat_state_value(&guard.store, guard.shard_uid, key)?;
+ Ok(value)
}
/// Update the head of the flat storage, including updating the flat state in memory and on disk
@@ -331,13 +330,12 @@ mod tests {
use crate::flat::delta::{FlatStateChanges, FlatStateDelta, FlatStateDeltaMetadata};
use crate::flat::manager::FlatStorageManager;
use crate::flat::storage::FlatStorage;
- use crate::flat::types::{BlockInfo, FlatStorageError};
+ use crate::flat::types::{BlockInfo, FlatStateValue, FlatStorageError};
use crate::flat::{store_helper, FlatStorageReadyStatus, FlatStorageStatus};
use crate::test_utils::create_test_store;
use crate::StorageError;
use borsh::BorshSerialize;
use near_primitives::hash::{hash, CryptoHash};
- use near_primitives::state::ValueRef;
use near_primitives::types::BlockHeight;
use assert_matches::assert_matches;
@@ -551,11 +549,19 @@ mod tests {
shard_uid,
FlatStorageStatus::Ready(FlatStorageReadyStatus { flat_head: chain.get_block(0) }),
);
- store_helper::set_ref(&mut store_update, shard_uid, vec![1], Some(ValueRef::new(&[0])))
- .unwrap();
+ store_helper::set_flat_state_value(
+ &mut store_update,
+ shard_uid,
+ vec![1],
+ Some(FlatStateValue::value_ref(&[0])),
+ )
+ .unwrap();
for i in 1..10 {
let delta = FlatStateDelta {
- changes: FlatStateChanges::from([(vec![1], Some(ValueRef::new(&[i as u8])))]),
+ changes: FlatStateChanges::from([(
+ vec![1],
+ Some(FlatStateValue::value_ref(&[i as u8])),
+ )]),
metadata: FlatStateDeltaMetadata { block: chain.get_block(i) },
};
store_helper::set_delta(&mut store_update, shard_uid, &delta).unwrap();
@@ -574,7 +580,10 @@ mod tests {
assert_eq!(blocks.len(), i as usize);
let chunk_view =
flat_storage_manager.chunk_view(shard_uid, Some(block_hash), false).unwrap();
- assert_eq!(chunk_view.get_ref(&[1]).unwrap(), Some(ValueRef::new(&[i as u8])));
+ assert_eq!(
+ chunk_view.get_value(&[1]).unwrap(),
+ Some(FlatStateValue::value_ref(&[i as u8]))
+ );
}
// 3. Create a new block that deletes &[1] and add a new value &[2]
@@ -584,7 +593,7 @@ mod tests {
.add_delta(FlatStateDelta {
changes: FlatStateChanges::from([
(vec![1], None),
- (vec![2], Some(ValueRef::new(&[1]))),
+ (vec![2], Some(FlatStateValue::value_ref(&[1]))),
]),
metadata: FlatStateDeltaMetadata { block: chain.get_block_info(&hash) },
})
@@ -601,10 +610,10 @@ mod tests {
let chunk_view1 = flat_storage_manager
.chunk_view(shard_uid, Some(chain.get_block_hash(4)), false)
.unwrap();
- assert_eq!(chunk_view0.get_ref(&[1]).unwrap(), None);
- assert_eq!(chunk_view0.get_ref(&[2]).unwrap(), Some(ValueRef::new(&[1])));
- assert_eq!(chunk_view1.get_ref(&[1]).unwrap(), Some(ValueRef::new(&[4])));
- assert_eq!(chunk_view1.get_ref(&[2]).unwrap(), None);
+ assert_eq!(chunk_view0.get_value(&[1]).unwrap(), None);
+ assert_eq!(chunk_view0.get_value(&[2]).unwrap(), Some(FlatStateValue::value_ref(&[1])));
+ assert_eq!(chunk_view1.get_value(&[1]).unwrap(), Some(FlatStateValue::value_ref(&[4])));
+ assert_eq!(chunk_view1.get_value(&[2]).unwrap(), None);
assert_matches!(
store_helper::get_delta_changes(&store, shard_uid, chain.get_block_hash(5)).unwrap(),
Some(_)
@@ -618,15 +627,15 @@ mod tests {
// and chunk_view1 returns an error. Also check that DBCol::FlatState is updated correctly
flat_storage.update_flat_head(&chain.get_block_hash(5)).unwrap();
assert_eq!(
- store_helper::get_ref(&store, shard_uid, &[1]).unwrap(),
- Some(ValueRef::new(&[5]))
+ store_helper::get_flat_state_value(&store, shard_uid, &[1]).unwrap(),
+ Some(FlatStateValue::value_ref(&[5]))
);
let blocks = flat_storage.get_blocks_to_head(&chain.get_block_hash(10)).unwrap();
assert_eq!(blocks.len(), 5);
- assert_eq!(chunk_view0.get_ref(&[1]).unwrap(), None);
- assert_eq!(chunk_view0.get_ref(&[2]).unwrap(), Some(ValueRef::new(&[1])));
+ assert_eq!(chunk_view0.get_value(&[1]).unwrap(), None);
+ assert_eq!(chunk_view0.get_value(&[2]).unwrap(), Some(FlatStateValue::value_ref(&[1])));
assert_matches!(
- chunk_view1.get_ref(&[1]),
+ chunk_view1.get_value(&[1]),
Err(StorageError::FlatStorageBlockNotSupported(_))
);
assert_matches!(
@@ -643,13 +652,13 @@ mod tests {
flat_storage.update_flat_head(&chain.get_block_hash(10)).unwrap();
let blocks = flat_storage.get_blocks_to_head(&chain.get_block_hash(10)).unwrap();
assert_eq!(blocks.len(), 0);
- assert_eq!(store_helper::get_ref(&store, shard_uid, &[1]).unwrap(), None);
+ assert_eq!(store_helper::get_flat_state_value(&store, shard_uid, &[1]).unwrap(), None);
assert_eq!(
- store_helper::get_ref(&store, shard_uid, &[2]).unwrap(),
- Some(ValueRef::new(&[1]))
+ store_helper::get_flat_state_value(&store, shard_uid, &[2]).unwrap(),
+ Some(FlatStateValue::value_ref(&[1]))
);
- assert_eq!(chunk_view0.get_ref(&[1]).unwrap(), None);
- assert_eq!(chunk_view0.get_ref(&[2]).unwrap(), Some(ValueRef::new(&[1])));
+ assert_eq!(chunk_view0.get_value(&[1]).unwrap(), None);
+ assert_eq!(chunk_view0.get_value(&[2]).unwrap(), Some(FlatStateValue::value_ref(&[1])));
assert_matches!(
store_helper::get_delta_changes(&store, shard_uid, chain.get_block_hash(10)).unwrap(),
None
diff --git a/core/store/src/flat/store_helper.rs b/core/store/src/flat/store_helper.rs
index e78334dac0e..bd61d894e4d 100644
--- a/core/store/src/flat/store_helper.rs
+++ b/core/store/src/flat/store_helper.rs
@@ -7,7 +7,6 @@ use crate::{Store, StoreUpdate};
use near_primitives::errors::StorageError;
use near_primitives::hash::CryptoHash;
use near_primitives::shard_layout::{ShardLayout, ShardUId};
-use near_primitives::state::ValueRef;
use super::delta::{FlatStateDelta, FlatStateDeltaMetadata};
use super::types::{FlatStateValue, FlatStorageStatus};
@@ -108,27 +107,25 @@ fn decode_flat_state_db_key(key: &Box<[u8]>) -> Result<(ShardUId, Vec), Stor
Ok((shard_uid, trie_key.to_vec()))
}
-pub(crate) fn get_ref(
+pub(crate) fn get_flat_state_value(
store: &Store,
shard_uid: ShardUId,
key: &[u8],
-) -> Result, FlatStorageError> {
+) -> Result , FlatStorageError> {
let db_key = encode_flat_state_db_key(shard_uid, key);
store
.get_ser(FlatStateColumn::State.to_db_col(), &db_key)
.map_err(|_| FlatStorageError::StorageInternalError)
- .map(|maybe_value| maybe_value.map(|FlatStateValue::Ref(v)| v))
}
// TODO(#8577): make pub(crate) once flat storage creator is moved inside `flat` module.
-pub fn set_ref(
+pub fn set_flat_state_value(
store_update: &mut StoreUpdate,
shard_uid: ShardUId,
key: Vec,
- value_ref: Option,
+ value: Option,
) -> Result<(), FlatStorageError> {
let db_key = encode_flat_state_db_key(shard_uid, &key);
- let value = value_ref.map(|v| FlatStateValue::Ref(v));
match value {
Some(value) => store_update
.set_ser(FlatStateColumn::State.to_db_col(), &db_key, &value)
diff --git a/core/store/src/flat/types.rs b/core/store/src/flat/types.rs
index a6d9b738210..ba6b2be1065 100644
--- a/core/store/src/flat/types.rs
+++ b/core/store/src/flat/types.rs
@@ -4,10 +4,35 @@ use near_primitives::hash::CryptoHash;
use near_primitives::state::ValueRef;
use near_primitives::types::BlockHeight;
+/// Defines value size threshold for flat state inlining.
+/// It means that values having size greater than the threshold will be stored
+/// in FlatState as `FlatStateValue::Ref`, otherwise the whole value will be
+/// stored as `FlatStateValue::Inlined`.
+/// See the following comment for reasoning behind the threshold value:
+/// https://github.com/near/nearcore/issues/8243#issuecomment-1523049994
+pub const INLINE_DISK_VALUE_THRESHOLD: usize = 4000;
+
#[derive(BorshSerialize, BorshDeserialize, Debug, Clone, PartialEq, Eq)]
pub enum FlatStateValue {
Ref(ValueRef),
- // TODO(8243): add variant here for the inlined value
+ Inlined(Vec),
+}
+
+impl FlatStateValue {
+ pub fn value_ref(value: &[u8]) -> Self {
+ Self::Ref(ValueRef::new(value))
+ }
+
+ pub fn inlined(value: &[u8]) -> Self {
+ Self::Inlined(value.to_vec())
+ }
+
+ pub fn to_value_ref(&self) -> ValueRef {
+ match self {
+ Self::Ref(value_ref) => value_ref.clone(),
+ Self::Inlined(value) => ValueRef::new(value),
+ }
+ }
}
#[derive(BorshSerialize, BorshDeserialize, Debug, Copy, Clone, PartialEq, Eq)]
diff --git a/core/store/src/trie/mod.rs b/core/store/src/trie/mod.rs
index 20d6464e6c2..17cb9c6c8db 100644
--- a/core/store/src/trie/mod.rs
+++ b/core/store/src/trie/mod.rs
@@ -795,7 +795,9 @@ impl Trie {
matches!(mode, KeyLookupMode::FlatStorage) && self.flat_storage_chunk_view.is_some();
if use_flat_storage {
- self.flat_storage_chunk_view.as_ref().unwrap().get_ref(&key)
+ let flat_state_value =
+ self.flat_storage_chunk_view.as_ref().unwrap().get_value(&key)?;
+ Ok(flat_state_value.map(|value| value.to_value_ref()))
} else {
let key_nibbles = NibbleSlice::new(key);
self.lookup(key_nibbles)
diff --git a/core/store/src/trie/state_parts.rs b/core/store/src/trie/state_parts.rs
index 081251ccc9c..26c10881315 100644
--- a/core/store/src/trie/state_parts.rs
+++ b/core/store/src/trie/state_parts.rs
@@ -7,7 +7,7 @@ use near_primitives::state_part::PartId;
use near_primitives::types::StateRoot;
use tracing::error;
-use crate::flat::FlatStateChanges;
+use crate::flat::{FlatStateChanges, FlatStateValue};
use crate::trie::iterator::TrieTraversalItem;
use crate::trie::nibble_slice::NibbleSlice;
use crate::trie::{
@@ -216,7 +216,7 @@ impl Trie {
map.entry(hash).or_insert_with(|| (value.to_vec(), 0)).1 += 1;
if let Some(trie_key) = key {
let value_ref = ValueRef::new(&value);
- flat_state_delta.insert(trie_key.clone(), Some(value_ref));
+ flat_state_delta.insert(trie_key.clone(), Some(FlatStateValue::Ref(value_ref)));
if is_contract_code_key(&trie_key) {
contract_codes.push(ContractCode::new(value.to_vec(), None));
}
diff --git a/integration-tests/src/tests/client/flat_storage.rs b/integration-tests/src/tests/client/flat_storage.rs
index 537c53e89cc..76e13fb467d 100644
--- a/integration-tests/src/tests/client/flat_storage.rs
+++ b/integration-tests/src/tests/client/flat_storage.rs
@@ -333,7 +333,8 @@ fn test_flat_storage_creation_start_from_state_part() {
};
let mut store_update = store.store_update();
for key in trie_keys[1].iter() {
- store_helper::set_ref(&mut store_update, shard_uid, key.clone(), None).unwrap();
+ store_helper::set_flat_state_value(&mut store_update, shard_uid, key.clone(), None)
+ .unwrap();
}
store_helper::set_flat_storage_status(
&mut store_update,
@@ -368,7 +369,7 @@ fn test_flat_storage_creation_start_from_state_part() {
let chunk_view = trie.flat_storage_chunk_view.unwrap();
for part_trie_keys in trie_keys.iter() {
for trie_key in part_trie_keys.iter() {
- assert_matches!(chunk_view.get_ref(trie_key), Ok(Some(_)));
+ assert_matches!(chunk_view.get_value(trie_key), Ok(Some(_)));
}
}
}
diff --git a/runtime/runtime-params-estimator/src/estimator_context.rs b/runtime/runtime-params-estimator/src/estimator_context.rs
index 9328fa6ffde..924c93b4346 100644
--- a/runtime/runtime-params-estimator/src/estimator_context.rs
+++ b/runtime/runtime-params-estimator/src/estimator_context.rs
@@ -7,14 +7,13 @@ use near_primitives::hash::CryptoHash;
use near_primitives::receipt::Receipt;
use near_primitives::runtime::config_store::RuntimeConfigStore;
use near_primitives::runtime::migration_data::{MigrationData, MigrationFlags};
-use near_primitives::state::ValueRef;
use near_primitives::test_utils::MockEpochInfoProvider;
use near_primitives::transaction::{ExecutionStatus, SignedTransaction};
use near_primitives::types::{Gas, MerkleHash};
use near_primitives::version::PROTOCOL_VERSION;
use near_store::flat::{
- BlockInfo, FlatStateChanges, FlatStateDelta, FlatStateDeltaMetadata, FlatStorage,
- FlatStorageManager,
+ BlockInfo, FlatStateChanges, FlatStateDelta, FlatStateDeltaMetadata, FlatStateValue,
+ FlatStorage, FlatStorageManager,
};
use near_store::{ShardTries, ShardUId, Store, StoreCompiledContractCache, TrieUpdate};
use near_store::{TrieCache, TrieCachingStorage, TrieConfig};
@@ -165,7 +164,7 @@ impl<'c> EstimatorContext<'c> {
let random_data = iter::repeat_with(|| {
(
crate::utils::random_vec(delta_key_len),
- Some(ValueRef::new(b"this is never stored or accessed, we only need it to blow up in-memory deltas")),
+ Some(FlatStateValue::value_ref(b"this is never stored or accessed, we only need it to blow up in-memory deltas")),
)
})
.take(num_changes_per_delta);