Skip to content

Commit

Permalink
runtime: implement a receipt preparation pipeline (near#11839)
Browse files Browse the repository at this point in the history
This PR implements a very basic form of pipelining of the contract
preparation. In particular with this PR only actions from one upcoming
"interesting" local receipt is handled, with other receipt kinds
(incoming, delayed) being left for a future change. I don't anticipate
any trouble integrating them as they largely have similar data and
structure as the local receipts seen here.

The primary reason I want this reviewed separately, however, is due to
the changes to storage. You will notice that I have largely changed the
code to read contracts without going through The State (i.e. it now
reads the contract code directly via the code hash.) However at the same
time writing of the contract continues involving The State; largely
because I don't want to deal with migrations and similar such
complexities… but also because `TrieStorage` does not provide write
access. This also means that writing is transactional and I had to add
some work-arounds to the `store::contract::Storage` in order to make
deployed contracts visible before the commit.
  • Loading branch information
nagisa authored Aug 7, 2024
1 parent e0895b6 commit 08cec9d
Show file tree
Hide file tree
Showing 9 changed files with 545 additions and 161 deletions.
52 changes: 52 additions & 0 deletions core/store/src/contract.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
use crate::TrieStorage;
use near_primitives::hash::CryptoHash;
use near_vm_runner::ContractCode;
use std::collections::BTreeMap;
use std::sync::{Arc, Mutex};

/// Reads contract code from the trie by its hash.
///
/// Cloning is cheap.
#[derive(Clone)]
pub struct ContractStorage {
storage: Arc<dyn TrieStorage>,

/// During an apply of a single chunk contracts may be deployed through the
/// `Action::DeployContract`.
///
/// Unfortunately `TrieStorage` does not have a way to write to the underlying storage, and the
/// `TrieUpdate` will only write the contract once the whole transaction is committed at the
/// end of the chunk's apply.
///
/// As a temporary work-around while we're still involving `Trie` for `ContractCode` storage,
/// we'll keep a list of such deployed contracts here. Once the contracts are no longer part of
/// The State this field should be removed, and the `Storage::store` function should be
/// adjusted to write out the contract into the relevant part of the database immediately
/// (without going through transactional storage operations and such).
uncommitted_deploys: Arc<Mutex<BTreeMap<CryptoHash, ContractCode>>>,
}

impl ContractStorage {
pub fn new(storage: Arc<dyn TrieStorage>) -> Self {
Self { storage, uncommitted_deploys: Default::default() }
}

pub fn get(&self, code_hash: CryptoHash) -> Option<ContractCode> {
{
let guard = self.uncommitted_deploys.lock().expect("no panics");
if let Some(v) = guard.get(&code_hash) {
return Some(ContractCode::new(v.code().to_vec(), Some(code_hash)));
}
}

match self.storage.retrieve_raw_bytes(&code_hash) {
Ok(raw_code) => Some(ContractCode::new(raw_code.to_vec(), Some(code_hash))),
Err(_) => None,
}
}

pub fn store(&self, code: ContractCode) {
let mut guard = self.uncommitted_deploys.lock().expect("no panics");
guard.insert(*code.hash(), code);
}
}
10 changes: 1 addition & 9 deletions core/store/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ use strum;
pub mod cold_storage;
mod columns;
pub mod config;
pub mod contract;
pub mod db;
pub mod flat;
pub mod genesis;
Expand Down Expand Up @@ -970,15 +971,6 @@ pub fn set_code(state_update: &mut TrieUpdate, account_id: AccountId, code: &Con
state_update.set(TrieKey::ContractCode { account_id }, code.code().to_vec());
}

pub fn get_code(
trie: &dyn TrieAccess,
account_id: &AccountId,
code_hash: Option<CryptoHash>,
) -> Result<Option<ContractCode>, StorageError> {
let key = TrieKey::ContractCode { account_id: account_id.clone() };
trie.get(&key).map(|opt| opt.map(|code| ContractCode::new(code, code_hash)))
}

/// Removes account, code and all access keys associated to it.
pub fn remove_account(
state_update: &mut TrieUpdate,
Expand Down
28 changes: 3 additions & 25 deletions core/store/src/trie/update.rs
Original file line number Diff line number Diff line change
@@ -1,41 +1,19 @@
pub use self::iterator::TrieUpdateIterator;
use super::accounting_cache::TrieAccountingCacheSwitch;
use super::{OptimizedValueRef, Trie, TrieWithReadLock};
use crate::contract::ContractStorage;
use crate::trie::{KeyLookupMode, TrieChanges};
use crate::{StorageError, TrieStorage};
use crate::StorageError;
use near_primitives::hash::CryptoHash;
use near_primitives::trie_key::TrieKey;
use near_primitives::types::{
AccountId, RawStateChange, RawStateChanges, RawStateChangesWithTrieKey, StateChangeCause,
StateRoot, TrieCacheMode,
};
use near_vm_runner::ContractCode;
use std::collections::BTreeMap;
use std::sync::Arc;

mod iterator;

/// Reads contract code from the trie by its hash.
/// Currently, uses `TrieStorage`. Consider implementing separate logic for
/// requesting and compiling contracts, as any contract code read and
/// compilation is a major bottleneck during chunk execution.
struct ContractStorage {
storage: Arc<dyn TrieStorage>,
}

impl ContractStorage {
fn new(storage: Arc<dyn TrieStorage>) -> Self {
Self { storage }
}

pub fn get(&self, code_hash: CryptoHash) -> Option<ContractCode> {
match self.storage.retrieve_raw_bytes(&code_hash) {
Ok(raw_code) => Some(ContractCode::new(raw_code.to_vec(), Some(code_hash))),
Err(_) => None,
}
}
}

/// Key-value update. Contains a TrieKey and a value.
pub struct TrieKeyValueUpdate {
pub trie_key: TrieKey,
Expand All @@ -49,7 +27,7 @@ pub type TrieUpdates = BTreeMap<Vec<u8>, TrieKeyValueUpdate>;
/// TODO (#7327): rename to StateUpdate
pub struct TrieUpdate {
pub trie: Trie,
contract_storage: ContractStorage,
pub contract_storage: ContractStorage,
committed: RawStateChanges,
prospective: TrieUpdates,
}
Expand Down
9 changes: 6 additions & 3 deletions integration-tests/src/tests/runtime/state_viewer.rs
Original file line number Diff line number Diff line change
@@ -1,11 +1,12 @@
use std::{collections::HashMap, io, sync::Arc};

use borsh::BorshDeserialize;
use near_vm_runner::ContractCode;

use crate::runtime_utils::{get_runtime_and_trie, get_test_trie_viewer, TEST_SHARD_UID};
use near_primitives::{
account::Account,
hash::{hash as sha256, CryptoHash},
hash::CryptoHash,
serialize::to_base64,
trie_key::trie_key_parsers,
types::{AccountId, StateRoot},
Expand Down Expand Up @@ -374,12 +375,14 @@ fn test_view_state_with_large_contract() {
let (_, tries, root) = get_runtime_and_trie();
let mut state_update = tries.new_trie_update(TEST_SHARD_UID, root);
let contract_code = [0; Account::MAX_ACCOUNT_DELETION_STORAGE_USAGE as usize].to_vec();
let code = ContractCode::new(contract_code, None);
set_account(
&mut state_update,
alice_account(),
&Account::new(0, 0, 0, sha256(&contract_code), 50_001, PROTOCOL_VERSION),
&Account::new(0, 0, 0, *code.hash(), 50_001, PROTOCOL_VERSION),
);
state_update.set(TrieKey::ContractCode { account_id: alice_account() }, contract_code);
// FIXME: this really should use the deploy action.
state_update.contract_storage.store(code);
let trie_viewer = TrieViewer::new(Some(50_000), None);
let result = trie_viewer.view_state(&state_update, &alice_account(), b"", false);
assert!(result.is_ok());
Expand Down
102 changes: 40 additions & 62 deletions runtime/runtime/src/actions.rs
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@ use crate::config::{
safe_add_compute, safe_add_gas, total_prepaid_exec_fees, total_prepaid_gas,
total_prepaid_send_fees,
};
use crate::ext::{ExternalError, RuntimeContractExt, RuntimeExt};
use crate::ext::{ExternalError, RuntimeExt};
use crate::receipt_manager::ReceiptManager;
use crate::{metrics, ActionResult, ApplyState};
use near_crypto::PublicKey;
Expand Down Expand Up @@ -30,16 +30,15 @@ use near_primitives::version::{
};
use near_primitives_core::account::id::AccountType;
use near_store::{
enqueue_promise_yield_timeout, get_access_key, get_code, get_promise_yield_indices,
remove_access_key, remove_account, set_access_key, set_code, set_promise_yield_indices,
StorageError, TrieUpdate,
enqueue_promise_yield_timeout, get_access_key, get_promise_yield_indices, remove_access_key,
remove_account, set_access_key, set_code, set_promise_yield_indices, StorageError, TrieUpdate,
};
use near_vm_runner::logic::errors::{
CompilationError, FunctionCallError, InconsistentStateError, VMRunnerError,
};
use near_vm_runner::logic::{VMContext, VMOutcome};
use near_vm_runner::ContractCode;
use near_vm_runner::{precompile_contract, PreparedContract};
use near_vm_runner::{ContractCode, ContractRuntimeCache};
use near_wallet_contract::{wallet_contract, wallet_contract_magic_bytes};
use std::sync::Arc;

Expand Down Expand Up @@ -161,44 +160,6 @@ pub(crate) fn execute_function_call(
Ok(outcome)
}

pub(crate) fn prepare_function_call(
state_update: &TrieUpdate,
apply_state: &ApplyState,
account: &Account,
account_id: &AccountId,
function_call: &FunctionCallAction,
config: &RuntimeConfig,
epoch_info_provider: &(dyn EpochInfoProvider),
view_config: Option<ViewConfig>,
) -> Box<dyn PreparedContract> {
let max_gas_burnt = match view_config {
Some(ViewConfig { max_gas_burnt }) => max_gas_burnt,
None => config.wasm_config.limit_config.max_gas_burnt,
};
let gas_counter = near_vm_runner::logic::GasCounter::new(
config.wasm_config.ext_costs.clone(),
max_gas_burnt,
config.wasm_config.regular_op_cost,
function_call.gas,
view_config.is_some(),
);
let code_ext = RuntimeContractExt {
trie_update: state_update,
account_id,
account,
chain_id: &epoch_info_provider.chain_id(),
current_protocol_version: apply_state.current_protocol_version,
};
let contract = near_vm_runner::prepare(
&code_ext,
Arc::clone(&config.wasm_config),
apply_state.cache.as_deref(),
gas_counter,
&function_call.method_name,
);
contract
}

pub(crate) fn action_function_call(
state_update: &mut TrieUpdate,
apply_state: &ApplyState,
Expand Down Expand Up @@ -645,11 +606,12 @@ pub(crate) fn action_deploy_contract(
account: &mut Account,
account_id: &AccountId,
deploy_contract: &DeployContractAction,
apply_state: &ApplyState,
config: Arc<near_parameters::vm::Config>,
cache: Option<&dyn ContractRuntimeCache>,
) -> Result<(), StorageError> {
let _span = tracing::debug_span!(target: "runtime", "action_deploy_contract").entered();
let code = ContractCode::new(deploy_contract.code.clone(), None);
let prev_code = get_code(state_update, account_id, Some(account.code_hash()))?;
let prev_code = state_update.contract_storage.get(account.code_hash());
let prev_code_length = prev_code.map(|code| code.code().len() as u64).unwrap_or_default();
account.set_storage_usage(account.storage_usage().saturating_sub(prev_code_length));
account.set_storage_usage(
Expand All @@ -661,16 +623,20 @@ pub(crate) fn action_deploy_contract(
})?,
);
account.set_code_hash(*code.hash());
// Legacy: populate the mapping from `AccountId => sha256(code)` thus making contracts part of
// The State. For the time being we are also relying on the `TrieUpdate` to actually write the
// contracts into the storage as part of the commit routine, however no code should be relying
// that the contracts are written to The State.
set_code(state_update, account_id.clone(), &code);
// Precompile the contract and store result (compiled code or error) in the database.
// Note, that contract compilation costs are already accounted in deploy cost using
// special logic in estimator (see get_runtime_config() function).
precompile_contract(
&code,
Arc::clone(&apply_state.config.wasm_config),
apply_state.cache.as_deref(),
)
.ok();
// Precompile the contract and store result (compiled code or error) in the contract runtime
// cache.
// Note, that contract compilation costs are already accounted in deploy cost using special
// logic in estimator (see get_runtime_config() function).
precompile_contract(&code, config, cache).ok();
// Inform the `store::contract::Storage` about the new deploy (so that the `get` method can
// return the contract before the contract is written out to the underlying storage as part of
// the `TrieUpdate` commit.)
state_update.contract_storage.store(code);
Ok(())
}

Expand All @@ -687,7 +653,7 @@ pub(crate) fn action_delete_account(
if current_protocol_version >= ProtocolFeature::DeleteActionRestriction.protocol_version() {
let account = account.as_ref().unwrap();
let mut account_storage_usage = account.storage_usage();
let contract_code = get_code(state_update, account_id, Some(account.code_hash()))?;
let contract_code = state_update.contract_storage.get(account.code_hash());
if let Some(code) = contract_code {
// account storage usage should be larger than code size
let code_len = code.code().len() as u64;
Expand Down Expand Up @@ -1190,10 +1156,8 @@ mod tests {
use near_primitives::action::delegate::NonDelegateAction;
use near_primitives::congestion_info::BlockCongestionInfo;
use near_primitives::errors::InvalidAccessKeyError;
use near_primitives::hash::hash;
use near_primitives::runtime::migration_data::MigrationFlags;
use near_primitives::transaction::CreateAccountAction;
use near_primitives::trie_key::TrieKey;
use near_primitives::types::{EpochId, StateChangeCause};
use near_primitives_core::version::PROTOCOL_VERSION;
use near_store::set_account;
Expand Down Expand Up @@ -1355,11 +1319,25 @@ mod tests {
let mut state_update =
tries.new_trie_update(ShardUId::single_shard(), CryptoHash::default());
let account_id = "alice".parse::<AccountId>().unwrap();
let trie_key = TrieKey::ContractCode { account_id: account_id.clone() };
let empty_contract = [0; 10_000].to_vec();
let contract_hash = hash(&empty_contract);
state_update.set(trie_key, empty_contract);
test_delete_large_account(&account_id, &contract_hash, storage_usage, &mut state_update)
let deploy_action = DeployContractAction { code: [0; 10_000].to_vec() };
let mut account =
Account::new(100, 0, 0, CryptoHash::default(), storage_usage, PROTOCOL_VERSION);
let apply_state = create_apply_state(0);
let res = action_deploy_contract(
&mut state_update,
&mut account,
&account_id,
&deploy_action,
Arc::clone(&apply_state.config.wasm_config),
None,
);
assert!(res.is_ok());
test_delete_large_account(
&account_id,
&account.code_hash(),
storage_usage,
&mut state_update,
)
}

#[test]
Expand Down
17 changes: 6 additions & 11 deletions runtime/runtime/src/ext.rs
Original file line number Diff line number Diff line change
Expand Up @@ -6,9 +6,10 @@ use near_primitives::checked_feature;
use near_primitives::errors::{EpochError, StorageError};
use near_primitives::hash::CryptoHash;
use near_primitives::trie_key::{trie_key_parsers, TrieKey};
use near_primitives::types::{AccountId, Balance, EpochId, EpochInfoProvider, Gas, TrieCacheMode};
use near_primitives::types::{AccountId, Balance, EpochId, EpochInfoProvider, Gas};
use near_primitives::utils::create_receipt_id_from_action_hash;
use near_primitives::version::ProtocolVersion;
use near_store::contract::ContractStorage;
use near_store::{has_promise_yield_receipt, KeyLookupMode, TrieUpdate, TrieUpdateValuePtr};
use near_vm_runner::logic::errors::{AnyError, VMLogicError};
use near_vm_runner::logic::types::ReceiptIndex;
Expand Down Expand Up @@ -362,34 +363,28 @@ impl<'a> External for RuntimeExt<'a> {
}

pub(crate) struct RuntimeContractExt<'a> {
pub(crate) trie_update: &'a TrieUpdate,
pub(crate) storage: ContractStorage,
pub(crate) account_id: &'a AccountId,
pub(crate) account: &'a Account,
pub(crate) code_hash: CryptoHash,
pub(crate) chain_id: &'a str,
pub(crate) current_protocol_version: ProtocolVersion,
}

impl<'a> Contract for RuntimeContractExt<'a> {
fn hash(&self) -> CryptoHash {
self.account.code_hash()
self.code_hash
}

fn get_code(&self) -> Option<Arc<ContractCode>> {
let account_id = self.account_id;
let code_hash = self.hash();
let version = self.current_protocol_version;
let chain_id = self.chain_id;
if checked_feature!("stable", EthImplicitAccounts, self.current_protocol_version)
&& account_id.get_account_type() == AccountType::EthImplicitAccount
&& &code_hash == wallet_contract_magic_bytes(&chain_id).hash()
{
return Some(wallet_contract(&chain_id));
}
let mode = match checked_feature!("stable", ChunkNodesCache, version) {
true => Some(TrieCacheMode::CachingShard),
false => None,
};
let _guard = self.trie_update.with_trie_cache_mode(mode);
self.trie_update.get_code(self.account_id.clone(), code_hash).map(Arc::new)
self.storage.get(code_hash).map(Arc::new)
}
}
Loading

0 comments on commit 08cec9d

Please sign in to comment.