Skip to content

Commit

Permalink
fix: exclude deleted values from proof (#12390)
Browse files Browse the repository at this point in the history
TLDR: the change is very small, majority is the test.

Before recording resharding proof, I want to slightly improve
`MemTrieUpdate::to_trie_changes` API. It returns `TrieAccesses` which
includes `values` which were read during processing state changes. But
in fact, we don't need the removed values at all, to generate proof.
Because we have hash uniqueness, it's enough to find that node has a
value with arbitrary hash.

Currently it adds weird dependency on `DBCol::State`, as memtries don't
have long values, so I completely remove values from the proof.

If we don't consider migration, then for correctness it is enough to
maintain the same behaviour on memtrie, disk and partial storage
processing. I achieve that by removing value accesses simultaneously,
with removing `values.insert` and `internal_retrieve_trie_node`.

Migration could be annoying - it's possible that chunk producers may
have this adopted and chunk validators may not, then CV would receive
insufficient proof. But I checked all `TrieUpdate::remove` calls, and,
surprisingly, each remove is prepended with getting value. For
`VMLogic`' storage remove API requires to get previous value, for
account removal we iterate over all keys **and values because that's how
iterator works**. So with current runtime this change is even no-op.

I still find it worthy to write a test triggering
`test_create_delete_account` because we don't have any integration test
exercising this path. I tried to write it in intuitive way and manually
checked that if runtime removes value without reading, and my change is
inconsistent, then test fails.
  • Loading branch information
Longarithm authored Nov 8, 2024
1 parent fc801ad commit 5d3008b
Show file tree
Hide file tree
Showing 6 changed files with 251 additions and 51 deletions.
16 changes: 6 additions & 10 deletions core/store/src/trie/mem/mem_trie_update.rs
Original file line number Diff line number Diff line change
Expand Up @@ -67,13 +67,10 @@ impl UpdatedMemTrieNodeWithSize {
}
}

/// Keeps values and internal nodes accessed on updating memtrie.
/// Keeps hashes and encoded trie nodes accessed on updating memtrie.
pub struct TrieAccesses {
/// Hashes and encoded trie nodes.
pub nodes: HashMap<CryptoHash, Arc<[u8]>>,
/// Hashes of accessed values - because values themselves are not
/// necessarily present in memtrie.
pub values: HashMap<CryptoHash, FlatStateValue>,
}

/// Tracks intermediate trie changes, final version of which is to be committed
Expand All @@ -86,11 +83,11 @@ struct TrieChangesTracker {
/// Separated from `refcount_deleted_hashes` to postpone hash computation
/// as far as possible.
refcount_inserted_values: BTreeMap<Vec<u8>, u32>,
/// All observed values and internal nodes.
/// All observed internal nodes.
/// Needed to prepare recorded storage.
/// Note that negative `refcount_changes` does not fully cover it, as node
/// or value of the same hash can be removed and inserted for the same
/// update in different parts of trie!
/// Note that negative `refcount_deleted_hashes` does not fully cover it,
/// as node or value of the same hash can be removed and inserted for the
/// same update in different parts of trie!
accesses: TrieAccesses,
}

Expand Down Expand Up @@ -181,7 +178,6 @@ impl<'a, M: ArenaMemory> GenericTrieUpdate<'a, MemTrieNodeId, FlatStateValue>
fn delete_value(&mut self, value: FlatStateValue) -> Result<(), StorageError> {
if let Some(tracked_node_changes) = self.tracked_trie_changes.as_mut() {
let hash = value.to_value_ref().hash;
tracked_node_changes.accesses.values.insert(hash, value);
tracked_node_changes
.refcount_deleted_hashes
.entry(hash)
Expand Down Expand Up @@ -209,7 +205,7 @@ impl<'a, M: ArenaMemory> MemTrieUpdate<'a, M> {
Some(TrieChangesTracker {
refcount_inserted_values: BTreeMap::new(),
refcount_deleted_hashes: BTreeMap::new(),
accesses: TrieAccesses { nodes: HashMap::new(), values: HashMap::new() },
accesses: TrieAccesses { nodes: HashMap::new() },
})
} else {
None
Expand Down
29 changes: 3 additions & 26 deletions core/store/src/trie/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1650,21 +1650,6 @@ impl Trie {
}
let (trie_changes, trie_accesses) = trie_update.to_trie_changes();

// Sanity check for tests: all modified trie items must be
// present in ever accessed trie items.
#[cfg(test)]
{
for t in trie_changes.deletions.iter() {
let hash = t.trie_node_or_value_hash;
assert!(
trie_accesses.values.contains_key(&hash)
|| trie_accesses.nodes.contains_key(&hash),
"Hash {} is not present in trie accesses",
hash
);
}
}

// Retroactively record all accessed trie items which are
// required to process trie update but were not recorded at
// processing lookups.
Expand All @@ -1675,15 +1660,6 @@ impl Trie {
for (node_hash, serialized_node) in trie_accesses.nodes {
recorder.borrow_mut().record(&node_hash, serialized_node);
}
for (value_hash, value) in trie_accesses.values {
let value = match value {
FlatStateValue::Ref(_) => {
self.storage.retrieve_raw_bytes(&value_hash)?
}
FlatStateValue::Inlined(value) => value.into(),
};
recorder.borrow_mut().record(&value_hash, value);
}
}
Ok(trie_changes)
}
Expand All @@ -1708,6 +1684,7 @@ impl Trie {
GenericNodeOrIndex::Updated(root_node.0),
);
}

trie_update.flatten_nodes(&self.root, root_node.0)
}
}
Expand Down Expand Up @@ -2251,8 +2228,8 @@ mod tests {
let trie2 = tries.get_trie_for_shard(ShardUId::single_shard(), root).recording_reads();
let updates = vec![(b"doge".to_vec(), None)];
trie2.update(updates).unwrap();
// record extension, branch and both leaves (one with value)
assert_eq!(trie2.recorded_storage().unwrap().nodes.len(), 5);
// record extension, branch and both leaves, but not the value.
assert_eq!(trie2.recorded_storage().unwrap().nodes.len(), 4);
}

{
Expand Down
2 changes: 1 addition & 1 deletion core/store/src/trie/trie_storage_update.rs
Original file line number Diff line number Diff line change
Expand Up @@ -141,7 +141,7 @@ impl<'a> GenericTrieUpdate<'a, TrieStorageNodePtr, ValueHandle> for TrieStorageU
fn delete_value(&mut self, value: ValueHandle) -> Result<(), StorageError> {
match value {
ValueHandle::HashAndSize(value) => {
self.trie.internal_retrieve_trie_node(&value.hash, true, true)?;
// Note that we don't need to read the actual value to remove it.
self.refcount_changes.subtract(value.hash, 1);
}
ValueHandle::InMemory(_) => {
Expand Down
115 changes: 115 additions & 0 deletions integration-tests/src/test_loop/tests/create_delete_account.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,115 @@
use itertools::Itertools;
use near_async::futures::{DelayedActionRunner, DelayedActionRunnerExt};
use near_async::time::Duration;
use near_chain_configs::test_genesis::TestGenesisBuilder;
use near_client::client_actor::ClientActorInner;
use near_o11y::testonly::init_test_logger;
use near_primitives::types::AccountId;

use crate::test_loop::builder::TestLoopBuilder;
use crate::test_loop::env::TestLoopEnv;
use crate::test_loop::utils::transactions::{
call_contract, check_txs, do_create_account, do_delete_account, do_deploy_contract,
get_next_nonce,
};
use crate::test_loop::utils::ONE_NEAR;

/// Write block height to contract storage.
fn do_call_contract(env: &mut TestLoopEnv, rpc_id: &AccountId, contract_id: &AccountId) {
tracing::info!(target: "test", "Calling contract.");
let nonce = get_next_nonce(env, contract_id);
let tx = call_contract(
&mut env.test_loop,
&env.datas,
rpc_id,
contract_id,
contract_id,
"write_block_height".to_string(),
vec![],
nonce,
);
env.test_loop.run_for(Duration::seconds(5));
check_txs(&env.test_loop, &env.datas, rpc_id, &[tx]);
}

/// Tracks latest block heights and checks that all chunks are produced.
fn check_chunks(
actor: &mut ClientActorInner,
runner: &mut dyn DelayedActionRunner<ClientActorInner>,
latest_block_height: std::cell::Cell<u64>,
) {
let client = &actor.client;
let tip = client.chain.head().unwrap().height;
if tip > latest_block_height.get() {
latest_block_height.set(tip);
let block = client.chain.get_block_by_height(tip).unwrap();
let num_shards = block.header().chunk_mask().len();
println!("Chain tip: {} Chunks: {:?}", tip, block.header().chunk_mask());
assert_eq!(block.header().chunk_mask(), vec![true; num_shards]);
}

runner.run_later("check_chunks", Duration::milliseconds(500), move |this, runner| {
check_chunks(this, runner, latest_block_height);
});
}

/// Tests account existence flow, from creation to deletion.
#[test]
fn test_create_delete_account() {
init_test_logger();
let builder = TestLoopBuilder::new();

let initial_balance = 1_000_000 * ONE_NEAR;
let epoch_length = 5;
let accounts =
(0..5).map(|i| format!("account{}", i).parse().unwrap()).collect::<Vec<AccountId>>();
let clients = accounts.clone();

// Split the clients into producers, validators, and rpc nodes.
let tmp = accounts.iter().map(|t| t.as_str()).collect_vec();
let (producers, tmp) = tmp.split_at(2);
let (validators, tmp) = tmp.split_at(2);
let (rpcs, tmp) = tmp.split_at(1);
let rpc_id = rpcs[0].parse().unwrap();
assert!(tmp.is_empty());

// Build test environment.
let mut genesis_builder = TestGenesisBuilder::new();
genesis_builder
.genesis_time_from_clock(&builder.clock())
.epoch_length(epoch_length)
.validators_desired_roles(&producers, &validators);
for account in &accounts {
genesis_builder.add_user_account_simple(account.clone(), initial_balance);
}
let (genesis, epoch_config_store) = genesis_builder.build();

let mut env =
builder.genesis(genesis).epoch_config_store(epoch_config_store).clients(clients).build();

// Launch a task to check that all chunks are produced.
// Needed to make sure that chunks are valid. Currently, if chunk
// validation fails, it only prevents production of new chunks, but the
// last chunk containing problematic tx or receipt is still executed on the
// rpc nodes, and result will be considered final based on block
// endorsements.
let mut client_sender = env.datas[0].client_sender.clone();
client_sender.run_later("check_chunks", Duration::seconds(0), move |actor, runner| {
check_chunks(actor, runner, std::cell::Cell::new(0));
});

let new_account: AccountId = format!("alice.{}", accounts[0]).parse().unwrap();
let contract_code = near_test_contracts::rs_contract().to_vec();

// Create account.
do_create_account(&mut env, &rpc_id, &accounts[0], &new_account, 100 * ONE_NEAR);
// Deploy contract.
do_deploy_contract(&mut env, &rpc_id, &new_account, contract_code);
// Write a key-value pair to the contract storage.
do_call_contract(&mut env, &rpc_id, &new_account);
// Delete account. Should remove everything - account, contract code and
// storage.
do_delete_account(&mut env, &rpc_id, &new_account, &accounts[1]);

env.shutdown_and_drain_remaining_events(Duration::seconds(20));
}
1 change: 1 addition & 0 deletions integration-tests/src/test_loop/tests/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ pub mod congestion_control;
pub mod congestion_control_genesis_bootstrap;
pub mod contract_distribution_cross_shard;
pub mod contract_distribution_simple;
mod create_delete_account;
pub mod epoch_sync;
pub mod fix_min_stake_ratio;
pub mod in_memory_tries;
Expand Down
Loading

0 comments on commit 5d3008b

Please sign in to comment.