diff --git a/chain/chain/src/chain.rs b/chain/chain/src/chain.rs index b8029caa135..bf51dffe90c 100644 --- a/chain/chain/src/chain.rs +++ b/chain/chain/src/chain.rs @@ -3170,48 +3170,59 @@ impl Chain { block: &Block, prev_block_header: &BlockHeader, chunk: &ShardChunk, - ) -> Result<(), Error> { + ) -> Result, Error> { let protocol_version = self.epoch_manager.get_epoch_protocol_version(block.header().epoch_id())?; - - if checked_feature!( + let relaxed_chunk_validation = checked_feature!( "protocol_feature_relaxed_chunk_validation", RelaxedChunkValidation, protocol_version - ) { - return Ok(()); - } + ); - if !validate_transactions_order(chunk.transactions()) { - let merkle_paths = - Block::compute_chunk_headers_root(block.chunks().iter_deprecated()).1; - let epoch_id = block.header().epoch_id(); - let shard_layout = self.epoch_manager.get_shard_layout(&epoch_id)?; - let shard_id = chunk.shard_id(); - let shard_index = shard_layout.get_shard_index(shard_id)?; - - let chunk_proof = ChunkProofs { - block_header: borsh::to_vec(&block.header()).expect("Failed to serialize"), - merkle_proof: merkle_paths[shard_index].clone(), - chunk: MaybeEncodedShardChunk::Decoded(chunk.clone()).into(), - }; - return Err(Error::InvalidChunkProofs(Box::new(chunk_proof))); + if !relaxed_chunk_validation { + if !validate_transactions_order(chunk.transactions()) { + let merkle_paths = + Block::compute_chunk_headers_root(block.chunks().iter_deprecated()).1; + let epoch_id = block.header().epoch_id(); + let shard_layout = self.epoch_manager.get_shard_layout(&epoch_id)?; + let shard_id = chunk.shard_id(); + let shard_index = shard_layout.get_shard_index(shard_id)?; + + let chunk_proof = ChunkProofs { + block_header: borsh::to_vec(&block.header()).expect("Failed to serialize"), + merkle_proof: merkle_paths[shard_index].clone(), + chunk: MaybeEncodedShardChunk::Decoded(chunk.clone()).into(), + }; + return Err(Error::InvalidChunkProofs(Box::new(chunk_proof))); + } } if checked_feature!("stable", AccessKeyNonceRange, protocol_version) { let transaction_validity_period = self.transaction_validity_period; - for transaction in chunk.transactions() { - self.chain_store() - .check_transaction_validity_period( - prev_block_header, - transaction.transaction.block_hash(), - transaction_validity_period, - ) - .map_err(|_| Error::from(Error::InvalidTransactions))?; - } + return chunk + .transactions() + .into_iter() + .map(|transaction| { + let tx_valid = self + .chain_store() + .check_transaction_validity_period( + prev_block_header, + transaction.transaction.block_hash(), + transaction_validity_period, + ) + .is_ok(); + if relaxed_chunk_validation { + Ok(tx_valid) + } else if !tx_valid { + Err(Error::from(Error::InvalidTransactions)) + } else { + Ok(true) + } + }) + .collect::>(); }; - Ok(()) + Ok(vec![true; chunk.transactions().len()]) } pub fn transaction_validity_check<'a>( @@ -3773,7 +3784,8 @@ impl Chain { } })?; - self.validate_chunk_transactions(&block, prev_block.header(), &chunk)?; + let tx_valid_list = + self.validate_chunk_transactions(&block, prev_block.header(), &chunk)?; // we can't use hash from the current block here yet because the incoming receipts // for this block is not stored yet @@ -3806,6 +3818,7 @@ impl Chain { ShardUpdateReason::NewChunk(NewChunkData { chunk_header: chunk_header.clone(), transactions: chunk.transactions().to_vec(), + transaction_validity_check_results: tx_valid_list, receipts, block: block_context, is_first_block_with_chunk_of_version, diff --git a/chain/chain/src/chain_update.rs b/chain/chain/src/chain_update.rs index 89208c1c8c2..7badfc94a6e 100644 --- a/chain/chain/src/chain_update.rs +++ b/chain/chain/src/chain_update.rs @@ -23,6 +23,7 @@ use near_primitives::state_sync::{ReceiptProofResponse, ShardStateSyncResponseHe use near_primitives::types::chunk_extra::ChunkExtra; use near_primitives::types::{BlockExtra, BlockHeight, BlockHeightDelta, ShardId}; use near_primitives::views::LightClientBlockView; +use node_runtime::SignedValidPeriodTransactions; use std::sync::Arc; use tracing::{debug, info, warn}; @@ -528,7 +529,10 @@ impl<'a> ChainUpdate<'a> { let is_first_block_with_chunk_of_version = false; let block = self.chain_store_update.get_block(block_header.hash())?; - + let transactions = chunk.transactions(); + // All transactions in state sync are assumed to have had correct validity period. + let transaction_validity = vec![true; transactions.len()]; + let transactions = SignedValidPeriodTransactions::new(transactions, &transaction_validity); let apply_result = self.runtime_adapter.apply_chunk( RuntimeStorageConfig::new(chunk_header.prev_state_root(), true), ApplyChunkReason::UpdateTrackedShard, @@ -551,7 +555,7 @@ impl<'a> ChainUpdate<'a> { bandwidth_requests: block.block_bandwidth_requests(), }, &receipts, - chunk.transactions(), + transactions, )?; let (outcome_root, outcome_proofs) = @@ -659,7 +663,7 @@ impl<'a> ChainUpdate<'a> { block.block_bandwidth_requests(), ), &[], - &[], + SignedValidPeriodTransactions::new(&[], &[]), )?; let flat_storage_manager = self.runtime_adapter.get_flat_storage_manager(); let store_update = flat_storage_manager.save_flat_state_changes( diff --git a/chain/chain/src/runtime/mod.rs b/chain/chain/src/runtime/mod.rs index 671acbdb3fa..f1fa2c701de 100644 --- a/chain/chain/src/runtime/mod.rs +++ b/chain/chain/src/runtime/mod.rs @@ -172,7 +172,7 @@ impl NightshadeRuntime { chunk: ApplyChunkShardContext, block: ApplyChunkBlockContext, receipts: &[Receipt], - transactions: &[SignedTransaction], + transactions: node_runtime::SignedValidPeriodTransactions<'_>, state_patch: SandboxStatePatch, ) -> Result { let ApplyChunkBlockContext { @@ -838,7 +838,7 @@ impl RuntimeAdapter for NightshadeRuntime { chunk: ApplyChunkShardContext, block: ApplyChunkBlockContext, receipts: &[Receipt], - transactions: &[SignedTransaction], + transactions: node_runtime::SignedValidPeriodTransactions<'_>, ) -> Result { let shard_id = chunk.shard_id; let _timer = metrics::APPLYING_CHUNKS_TIME diff --git a/chain/chain/src/runtime/tests.rs b/chain/chain/src/runtime/tests.rs index a5e75ca546c..d68f99e6bec 100644 --- a/chain/chain/src/runtime/tests.rs +++ b/chain/chain/src/runtime/tests.rs @@ -26,6 +26,7 @@ use near_store::genesis::initialize_genesis_state; use near_vm_runner::{ get_contract_cache_key, CompiledContract, CompiledContractInfo, FilesystemContractRuntimeCache, }; +use node_runtime::SignedValidPeriodTransactions; use num_rational::Ratio; use rand::{rngs::StdRng, seq::SliceRandom, SeedableRng}; @@ -244,6 +245,8 @@ impl TestEnv { .collect(); BlockCongestionInfo::new(shards_congestion_info) }; + let transaction_validity = vec![true; transactions.len()]; + let transactions = SignedValidPeriodTransactions::new(transactions, &transaction_validity); self.runtime .apply_chunk( RuntimeStorageConfig::new(state_root, true), diff --git a/chain/chain/src/stateless_validation/chunk_validation.rs b/chain/chain/src/stateless_validation/chunk_validation.rs index ba517efae31..d0627b79ec7 100644 --- a/chain/chain/src/stateless_validation/chunk_validation.rs +++ b/chain/chain/src/stateless_validation/chunk_validation.rs @@ -67,18 +67,6 @@ impl MainTransition { pub struct PreValidationOutput { pub main_transition_params: MainTransition, pub implicit_transition_params: Vec, - /// List of the transactions that are valid and should be processed by e.g. - /// `validate_chunk_state_witness`. - /// - /// This list is exactly the length of the corresponding `ChunkStateWitness::transactions` - /// field. Element at the index N in this array corresponds to an element at index N in the - /// transactions list. - /// - /// Transactions for which a `false` is stored here ought to be ignored/dropped/skipped. - /// - /// All elements will be true for protocol versions where `RelaxedChunkValidation` is not - /// enabled. - pub transaction_validity_check_passed: Vec, } #[derive(Clone)] @@ -381,7 +369,7 @@ pub fn pre_validate_chunk_state_witness( let current_protocol_version = epoch_manager.get_epoch_protocol_version(&state_witness.epoch_id)?; - let transaction_validity_check_passed = if checked_feature!( + let transaction_validity_check_results = if checked_feature!( "protocol_feature_relaxed_chunk_validation", RelaxedChunkValidation, current_protocol_version @@ -467,6 +455,7 @@ pub fn pre_validate_chunk_state_witness( MainTransition::NewChunk(NewChunkData { chunk_header: last_chunk_block.chunks().get(last_chunk_shard_index).unwrap().clone(), transactions: state_witness.transactions.clone(), + transaction_validity_check_results, receipts: receipts_to_apply, block: Chain::get_apply_chunk_block_context( epoch_manager, @@ -484,11 +473,7 @@ pub fn pre_validate_chunk_state_witness( }) }; - Ok(PreValidationOutput { - main_transition_params, - implicit_transition_params, - transaction_validity_check_passed, - }) + Ok(PreValidationOutput { main_transition_params, implicit_transition_params }) } /// Validate that receipt proofs contain the receipts that should be applied during the @@ -637,21 +622,7 @@ pub fn validate_chunk_state_witness( let (mut chunk_extra, mut outgoing_receipts) = match (pre_validation_output.main_transition_params, cache_result) { (MainTransition::Genesis { chunk_extra, .. }, _) => (chunk_extra, vec![]), - (MainTransition::NewChunk(mut new_chunk_data), None) => { - let mut validity_iterator = - pre_validation_output.transaction_validity_check_passed.iter(); - new_chunk_data.transactions.retain(|t| { - let valid = *validity_iterator.next().unwrap(); - if !valid { - tracing::debug!( - target: "chain", - message="discarding invalid transaction", - tx=%t.get_hash() - ); - } - valid - }); - + (MainTransition::NewChunk(new_chunk_data), None) => { let chunk_header = new_chunk_data.chunk_header.clone(); let NewChunkResult { apply_result: mut main_apply_result, .. } = apply_new_chunk( ApplyChunkReason::ValidateChunkStateWitness, diff --git a/chain/chain/src/test_utils/kv_runtime.rs b/chain/chain/src/test_utils/kv_runtime.rs index 1a02d269d63..128c2d22f32 100644 --- a/chain/chain/src/test_utils/kv_runtime.rs +++ b/chain/chain/src/test_utils/kv_runtime.rs @@ -54,6 +54,7 @@ use near_store::{ TrieChanges, WrappedTrieChanges, }; use near_vm_runner::{ContractCode, ContractRuntimeCache, NoContractRuntimeCache}; +use node_runtime::SignedValidPeriodTransactions; use std::cmp::Ordering; use std::collections::{BTreeMap, HashMap, HashSet}; use std::sync::{Arc, RwLock}; @@ -1091,7 +1092,7 @@ impl RuntimeAdapter for KeyValueRuntime { chunk: ApplyChunkShardContext, block: ApplyChunkBlockContext, receipts: &[Receipt], - transactions: &[SignedTransaction], + transactions: SignedValidPeriodTransactions<'_>, ) -> Result { let mut tx_results = vec![]; let shard_id = chunk.shard_id; @@ -1128,7 +1129,7 @@ impl RuntimeAdapter for KeyValueRuntime { } } - for transaction in transactions { + for transaction in transactions.iter_nonexpired_transactions() { assert_eq!( account_id_to_shard_id(transaction.transaction.signer_id(), self.num_shards), shard_id diff --git a/chain/chain/src/types.rs b/chain/chain/src/types.rs index 221d11f832e..e04af0a661e 100644 --- a/chain/chain/src/types.rs +++ b/chain/chain/src/types.rs @@ -42,6 +42,7 @@ use near_store::flat::FlatStorageManager; use near_store::{PartialStorage, ShardTries, Store, Trie, WrappedTrieChanges}; use near_vm_runner::ContractCode; use near_vm_runner::ContractRuntimeCache; +use node_runtime::SignedValidPeriodTransactions; use num_rational::Rational32; use tracing::instrument; @@ -472,7 +473,7 @@ pub trait RuntimeAdapter: Send + Sync { chunk: ApplyChunkShardContext, block: ApplyChunkBlockContext, receipts: &[Receipt], - transactions: &[SignedTransaction], + transactions: SignedValidPeriodTransactions<'_>, ) -> Result; /// Query runtime with given `path` and `data`. diff --git a/chain/chain/src/update_shard.rs b/chain/chain/src/update_shard.rs index 9f1c9dad937..05550deedcd 100644 --- a/chain/chain/src/update_shard.rs +++ b/chain/chain/src/update_shard.rs @@ -13,6 +13,7 @@ use near_primitives::sharding::ShardChunkHeader; use near_primitives::transaction::SignedTransaction; use near_primitives::types::chunk_extra::ChunkExtra; use near_primitives::types::Gas; +use node_runtime::SignedValidPeriodTransactions; /// Result of updating a shard for some block when it has a new chunk for this /// shard. @@ -43,6 +44,7 @@ pub enum ShardUpdateResult { pub struct NewChunkData { pub chunk_header: ShardChunkHeader, pub transactions: Vec, + pub transaction_validity_check_results: Vec, pub receipts: Vec, pub block: ApplyChunkBlockContext, pub is_first_block_with_chunk_of_version: bool, @@ -119,6 +121,7 @@ pub fn apply_new_chunk( let NewChunkData { chunk_header, transactions, + transaction_validity_check_results, block, receipts, is_first_block_with_chunk_of_version, @@ -153,7 +156,7 @@ pub fn apply_new_chunk( }, block, &receipts, - &transactions, + SignedValidPeriodTransactions::new(&transactions, &transaction_validity_check_results), ) { Ok(apply_result) => { Ok(NewChunkResult { gas_limit, shard_uid: shard_context.shard_uid, apply_result }) @@ -200,7 +203,7 @@ pub fn apply_old_chunk( }, block, &[], - &[], + SignedValidPeriodTransactions::new(&[], &[]), ) { Ok(apply_result) => Ok(OldChunkResult { shard_uid: shard_context.shard_uid, apply_result }), Err(err) => Err(err), diff --git a/integration-tests/src/user/runtime_user.rs b/integration-tests/src/user/runtime_user.rs index e94f6eb7905..49f7524c768 100644 --- a/integration-tests/src/user/runtime_user.rs +++ b/integration-tests/src/user/runtime_user.rs @@ -26,6 +26,7 @@ use near_primitives::views::{ use near_store::adapter::StoreUpdateAdapter; use near_store::{ShardTries, TrieUpdate}; use node_runtime::state_viewer::TrieViewer; +use node_runtime::SignedValidPeriodTransactions; use node_runtime::{state_viewer::ViewApplyState, ApplyState, Runtime}; use crate::user::{User, POISONED_LOCK_ERR}; @@ -111,7 +112,7 @@ impl RuntimeUser { &None, &apply_state, &receipts, - &txs, + SignedValidPeriodTransactions::new(&txs, &vec![true; txs.len()]), &self.epoch_info_provider, Default::default(), ) diff --git a/runtime/runtime-params-estimator/src/estimator_context.rs b/runtime/runtime-params-estimator/src/estimator_context.rs index 97cd226903c..ac593c773a4 100644 --- a/runtime/runtime-params-estimator/src/estimator_context.rs +++ b/runtime/runtime-params-estimator/src/estimator_context.rs @@ -26,7 +26,7 @@ use near_store::{ShardTries, ShardUId, StateSnapshotConfig, TrieUpdate}; use near_store::{TrieCache, TrieCachingStorage, TrieConfig}; use near_vm_runner::logic::LimitConfig; use near_vm_runner::FilesystemContractRuntimeCache; -use node_runtime::{ApplyState, Runtime}; +use node_runtime::{ApplyState, Runtime, SignedValidPeriodTransactions}; use std::collections::HashMap; use std::iter; use std::sync::Arc; @@ -355,7 +355,7 @@ impl Testbed<'_> { &None, &self.apply_state, &self.prev_receipts, - transactions, + SignedValidPeriodTransactions::new(transactions, &vec![true; transactions.len()]), &self.epoch_info_provider, Default::default(), ) diff --git a/runtime/runtime/src/balance_checker.rs b/runtime/runtime/src/balance_checker.rs index 3a622df947b..aeec8117a1a 100644 --- a/runtime/runtime/src/balance_checker.rs +++ b/runtime/runtime/src/balance_checker.rs @@ -2,7 +2,7 @@ use crate::config::{ safe_add_balance, safe_add_gas, safe_gas_to_balance, total_deposit, total_prepaid_exec_fees, total_prepaid_gas, total_prepaid_send_fees, }; -use crate::safe_add_balance_apply; +use crate::{safe_add_balance_apply, SignedValidPeriodTransactions}; use crate::{ApplyStats, DelayedReceiptIndices, ValidatorAccountsUpdate}; use near_parameters::{ActionCosts, RuntimeConfig}; use near_primitives::errors::{ @@ -10,7 +10,6 @@ use near_primitives::errors::{ }; use near_primitives::hash::CryptoHash; use near_primitives::receipt::{Receipt, ReceiptEnum, ReceiptOrStateStoredReceipt}; -use near_primitives::transaction::SignedTransaction; use near_primitives::trie_key::TrieKey; use near_primitives::types::{AccountId, Balance, ShardId}; use near_store::trie::receipts_column_helper::{ShardsOutgoingReceiptBuffer, TrieQueue}; @@ -180,11 +179,11 @@ fn all_touched_accounts( incoming_receipts: &[Receipt], yield_timeout_receipts: &[Receipt], processed_delayed_receipts: &[Receipt], - transactions: &[SignedTransaction], + transactions: SignedValidPeriodTransactions<'_>, validator_accounts_update: &Option, ) -> Result, RuntimeError> { let mut all_accounts_ids: HashSet = transactions - .iter() + .iter_nonexpired_transactions() .map(|tx| tx.transaction.signer_id().clone()) .chain(incoming_receipts.iter().map(|r| r.receiver_id().clone())) .chain(yield_timeout_receipts.iter().map(|r| r.receiver_id().clone())) @@ -277,7 +276,7 @@ pub(crate) fn check_balance( incoming_receipts: &[Receipt], processed_delayed_receipts: &[Receipt], yield_timeout_receipts: &[Receipt], - transactions: &[SignedTransaction], + transactions: SignedValidPeriodTransactions<'_>, outgoing_receipts: &[Receipt], stats: &ApplyStats, ) -> Result<(), RuntimeError> { @@ -395,7 +394,7 @@ mod tests { ActionReceipt, BufferedReceiptIndices, ReceiptPriority, ReceiptV0, TrieQueueIndices, }; use near_primitives::test_utils::account_new; - use near_primitives::transaction::{Action, TransferAction}; + use near_primitives::transaction::{Action, SignedTransaction, TransferAction}; use near_primitives::types::{MerkleHash, StateChangeCause}; use near_store::test_utils::TestTriesBuilder; use near_store::{set, set_account, Trie}; @@ -422,7 +421,7 @@ mod tests { &[], &[], &[], - &[], + SignedValidPeriodTransactions::empty(), &[], &ApplyStats::default(), ) @@ -441,7 +440,7 @@ mod tests { &[Receipt::new_balance_refund(&alice_account(), 1000, ReceiptPriority::NoPriority)], &[], &[], - &[], + SignedValidPeriodTransactions::empty(), &[], &ApplyStats::default(), ) @@ -506,7 +505,7 @@ mod tests { )], &[], &[], - &[], + SignedValidPeriodTransactions::empty(), &[], &ApplyStats::default(), ) @@ -555,7 +554,7 @@ mod tests { &[], &[], &[], - &[tx], + SignedValidPeriodTransactions::new(&[tx], &[true]), &[receipt], &ApplyStats { tx_burnt_amount: total_validator_reward, @@ -628,7 +627,7 @@ mod tests { &[receipt], &[], &[], - &[tx], + SignedValidPeriodTransactions::new(&[tx], &[true]), &[], &ApplyStats::default(), ), @@ -671,7 +670,7 @@ mod tests { &[receipt], &[], &[], - &[tx], + SignedValidPeriodTransactions::new(&[tx], &[true]), &[], &ApplyStats::default(), ), @@ -751,7 +750,7 @@ mod tests { &[], &[], &[], - &[tx], + SignedValidPeriodTransactions::new(&[tx], &[true]), &[], &ApplyStats { // send gas was burnt on this shard, exec gas is part of the receipt value @@ -822,7 +821,7 @@ mod tests { &[], &[], &[], - &[], + SignedValidPeriodTransactions::empty(), &outgoing_receipts, &ApplyStats::default(), ) @@ -886,7 +885,7 @@ mod tests { &[], &[], &[], - &[], + SignedValidPeriodTransactions::empty(), &outgoing_receipts, &ApplyStats::default(), ); diff --git a/runtime/runtime/src/lib.rs b/runtime/runtime/src/lib.rs index 1ca1a43b3de..d86ff495361 100644 --- a/runtime/runtime/src/lib.rs +++ b/runtime/runtime/src/lib.rs @@ -1419,7 +1419,7 @@ impl Runtime { validator_accounts_update: &Option, apply_state: &ApplyState, incoming_receipts: &[Receipt], - transactions: &[SignedTransaction], + transactions: SignedValidPeriodTransactions<'_>, epoch_info_provider: &dyn EpochInfoProvider, state_patch: SandboxStatePatch, ) -> Result { @@ -1586,7 +1586,8 @@ impl Runtime { let total = &mut processing_state.total; let apply_state = &mut processing_state.apply_state; let state_update = &mut processing_state.state_update; - for signed_transaction in processing_state.transactions { + + for signed_transaction in processing_state.transactions.iter_nonexpired_transactions() { let tx_result = self.process_transaction( state_update, apply_state, @@ -2486,7 +2487,7 @@ struct ApplyProcessingState<'a> { prefetcher: Option, state_update: TrieUpdate, epoch_info_provider: &'a dyn EpochInfoProvider, - transactions: &'a [SignedTransaction], + transactions: SignedValidPeriodTransactions<'a>, total: TotalResourceGuard, stats: ApplyStats, } @@ -2496,7 +2497,7 @@ impl<'a> ApplyProcessingState<'a> { apply_state: &'a ApplyState, trie: Trie, epoch_info_provider: &'a dyn EpochInfoProvider, - transactions: &'a [SignedTransaction], + transactions: SignedValidPeriodTransactions<'a>, ) -> Self { let protocol_version = apply_state.current_protocol_version; let prefetcher = TriePrefetcher::new_if_enabled(&trie); @@ -2552,6 +2553,43 @@ impl<'a> ApplyProcessingState<'a> { } } +#[derive(Clone, Copy)] +pub struct SignedValidPeriodTransactions<'a> { + transactions: &'a [SignedTransaction], + /// List of the transactions that are valid and should be processed by `apply`. + /// + /// This list is exactly the length of the corresponding `Self::transactions` field. Element at + /// the index N in this array corresponds to an element at index N in the transactions list. + /// + /// Transactions for which a `false` is stored here must be ignored/dropped/skipped. + /// + /// All elements will be true for protocol versions where `RelaxedChunkValidation` is not + /// enabled. + transaction_validity_check_passed: &'a [bool], +} + +impl<'a> SignedValidPeriodTransactions<'a> { + pub fn new(transactions: &'a [SignedTransaction], validity_check_results: &'a [bool]) -> Self { + assert_eq!(transactions.len(), validity_check_results.len()); + Self { transactions, transaction_validity_check_passed: validity_check_results } + } + + pub fn empty() -> Self { + Self::new(&[], &[]) + } + + pub fn iter_nonexpired_transactions(&self) -> impl Iterator { + self.transactions + .into_iter() + .zip(self.transaction_validity_check_passed.into_iter()) + .filter_map(|(t, v)| v.then_some(t)) + } + + pub fn len(&self) -> usize { + self.transactions.len() + } +} + /// Similar to [ApplyProcessingState], with the difference that this contains extra state used /// by receipt processing. struct ApplyProcessingReceiptState<'a> { @@ -2560,7 +2598,7 @@ struct ApplyProcessingReceiptState<'a> { prefetcher: Option, state_update: TrieUpdate, epoch_info_provider: &'a dyn EpochInfoProvider, - transactions: &'a [SignedTransaction], + transactions: SignedValidPeriodTransactions<'a>, total: TotalResourceGuard, stats: ApplyStats, outcomes: Vec, diff --git a/runtime/runtime/src/prefetch.rs b/runtime/runtime/src/prefetch.rs index ef44616fa8e..a4164ba0856 100644 --- a/runtime/runtime/src/prefetch.rs +++ b/runtime/runtime/src/prefetch.rs @@ -45,7 +45,7 @@ use borsh::BorshSerialize as _; use near_o11y::metrics::prometheus; use near_o11y::metrics::prometheus::core::GenericCounter; use near_primitives::receipt::{Receipt, ReceiptEnum}; -use near_primitives::transaction::{Action, SignedTransaction}; +use near_primitives::transaction::Action; use near_primitives::trie_key::TrieKey; use near_primitives::types::AccountId; use near_primitives::types::StateRoot; @@ -54,7 +54,7 @@ use sha2::Digest; use std::str::FromStr; use tracing::{debug, warn}; -use crate::metrics; +use crate::{metrics, SignedValidPeriodTransactions}; /// Transaction runtime view of the prefetching subsystem. pub(crate) struct TriePrefetcher { prefetch_api: PrefetchApi, @@ -195,10 +195,10 @@ impl TriePrefetcher { /// for some transactions may have been initiated. pub(crate) fn prefetch_transactions_data( &mut self, - transactions: &[SignedTransaction], + transactions: SignedValidPeriodTransactions<'_>, ) -> Result<(), PrefetchError> { if self.prefetch_api.enable_receipt_prefetching { - for t in transactions { + for t in transactions.iter_nonexpired_transactions() { let account_id = t.transaction.signer_id().clone(); let trie_key = TrieKey::Account { account_id }; self.prefetch_trie_key(trie_key)?; diff --git a/runtime/runtime/src/tests/apply.rs b/runtime/runtime/src/tests/apply.rs index f48a8fa7db6..ffcb64b5431 100644 --- a/runtime/runtime/src/tests/apply.rs +++ b/runtime/runtime/src/tests/apply.rs @@ -5,7 +5,7 @@ use crate::tests::{ create_receipt_for_create_account, create_receipt_with_actions, set_sha256_cost, MAX_ATTACHED_GAS, }; -use crate::total_prepaid_exec_fees; +use crate::{total_prepaid_exec_fees, SignedValidPeriodTransactions}; use crate::{ApplyResult, ApplyState, Runtime, ValidatorAccountsUpdate}; use assert_matches::assert_matches; use near_crypto::{InMemorySigner, KeyType, PublicKey, Signer}; @@ -149,7 +149,7 @@ fn test_apply_no_op() { &None, &apply_state, &[], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -182,7 +182,7 @@ fn test_apply_check_balance_validation_rewards() { small_refund, ReceiptPriority::NoPriority, )], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -211,7 +211,7 @@ fn test_apply_refund_receipts() { &None, &apply_state, prev_receipts, - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -255,7 +255,7 @@ fn test_apply_delayed_receipts_feed_all_at_once() { &None, &apply_state, prev_receipts, - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -300,7 +300,7 @@ fn test_apply_delayed_receipts_add_more_using_chunks() { &None, &apply_state, prev_receipts, - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -353,7 +353,7 @@ fn test_apply_delayed_receipts_adjustable_gas_limit() { &None, &apply_state, prev_receipts, - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -514,7 +514,7 @@ fn test_apply_delayed_receipts_local_tx() { &None, &apply_state, &receipts[0..2], - &local_transactions[0..4], + SignedValidPeriodTransactions::new(&local_transactions[0..4], &[true; 4]), &epoch_info_provider, Default::default(), ) @@ -563,7 +563,7 @@ fn test_apply_delayed_receipts_local_tx() { &None, &apply_state, &receipts[2..3], - &local_transactions[4..5], + SignedValidPeriodTransactions::new(&local_transactions[4..5], &[true]), &epoch_info_provider, Default::default(), ) @@ -605,7 +605,7 @@ fn test_apply_delayed_receipts_local_tx() { &None, &apply_state, &receipts[3..4], - &local_transactions[5..9], + SignedValidPeriodTransactions::new(&local_transactions[5..9], &[true; 4]), &epoch_info_provider, Default::default(), ) @@ -656,7 +656,7 @@ fn test_apply_delayed_receipts_local_tx() { &None, &apply_state, &receipts[4..5], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -690,7 +690,7 @@ fn test_apply_delayed_receipts_local_tx() { &None, &apply_state, &receipts[5..6], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -732,7 +732,7 @@ fn test_apply_deficit_gas_for_transfer() { &None, &apply_state, &receipts, - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -789,7 +789,7 @@ fn test_apply_deficit_gas_for_function_call_covered() { &None, &apply_state, &receipts, - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -856,7 +856,7 @@ fn test_apply_deficit_gas_for_function_call_partial() { &None, &apply_state, &receipts, - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -892,7 +892,7 @@ fn test_delete_key_add_key() { &None, &apply_state, &receipts, - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -935,7 +935,7 @@ fn test_delete_key_underflow() { &None, &apply_state, &receipts, - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -974,7 +974,7 @@ fn test_contract_precompilation() { &None, &apply_state, &receipts, - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -1047,7 +1047,7 @@ fn test_compute_usage_limit() { first_call_receipt.clone(), second_call_receipt.clone(), ], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -1071,7 +1071,7 @@ fn test_compute_usage_limit() { &None, &apply_state, &[], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -1114,7 +1114,7 @@ fn test_compute_usage_limit_with_failed_receipt() { &None, &apply_state, &[deploy_contract_receipt.clone(), first_call_receipt.clone()], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -1160,7 +1160,7 @@ fn test_main_storage_proof_size_soft_limit() { create_acc_fn(alice_account(), signers[0].clone()), create_acc_fn(bob_account(), signers[1].clone()), ], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -1202,7 +1202,7 @@ fn test_main_storage_proof_size_soft_limit() { function_call_fn(alice_account(), signers[0].clone()), function_call_fn(bob_account(), signers[1].clone()), ], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -1265,7 +1265,7 @@ fn test_exclude_contract_code_from_witness() { create_acc_fn(alice_account(), signers[0].clone()), create_acc_fn(bob_account(), signers[1].clone()), ], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -1307,7 +1307,7 @@ fn test_exclude_contract_code_from_witness() { function_call_fn(alice_account(), signers[0].clone()), function_call_fn(bob_account(), signers[1].clone()), ], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -1379,7 +1379,7 @@ fn test_exclude_contract_code_from_witness_with_failed_call() { create_acc_fn(alice_account(), signers[0].clone()), create_acc_fn(bob_account(), signers[1].clone()), ], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -1420,7 +1420,7 @@ fn test_exclude_contract_code_from_witness_with_failed_call() { function_call_fn(alice_account(), signers[0].clone()), function_call_fn(bob_account(), signers[1].clone()), ], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -1510,7 +1510,7 @@ fn test_deploy_and_call_different_contracts() { &None, &apply_state, &[first_deploy_receipt, second_deploy_receipt], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -1537,7 +1537,7 @@ fn test_deploy_and_call_different_contracts() { &None, &apply_state, &[first_call_receipt, second_call_receipt], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -1616,7 +1616,7 @@ fn test_deploy_and_call_different_contracts_with_failed_call() { &None, &apply_state, &[first_deploy_receipt, second_deploy_receipt], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -1643,7 +1643,7 @@ fn test_deploy_and_call_different_contracts_with_failed_call() { &None, &apply_state, &[first_call_receipt, second_call_receipt], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -1720,7 +1720,7 @@ fn test_deploy_and_call_in_apply() { &None, &apply_state, &[first_deploy_receipt, second_deploy_receipt, first_call_receipt, second_call_receipt], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -1799,7 +1799,7 @@ fn test_deploy_and_call_in_apply_with_failed_call() { &None, &apply_state, &[first_deploy_receipt, second_deploy_receipt, first_call_receipt, second_call_receipt], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -1854,7 +1854,7 @@ fn test_deploy_existing_contract_to_different_account() { &None, &apply_state, &[first_deploy_receipt, first_call_receipt], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -1896,7 +1896,7 @@ fn test_deploy_existing_contract_to_different_account() { &None, &apply_state, &[second_deploy_receipt, second_call_receipt], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -1945,7 +1945,7 @@ fn test_deploy_and_call_in_same_receipt() { &None, &apply_state, &[receipt], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -1994,7 +1994,7 @@ fn test_deploy_and_call_in_same_receipt_with_failed_call() { &None, &apply_state, &[receipt], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -2030,7 +2030,7 @@ fn test_call_account_without_contract() { &None, &apply_state, &[receipt], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -2074,7 +2074,7 @@ fn test_contract_accesses_when_validating_chunk() { &None, &apply_state, &[deploy_receipt], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -2100,7 +2100,7 @@ fn test_contract_accesses_when_validating_chunk() { &None, &apply_state, &[call_receipt.clone()], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -2121,7 +2121,7 @@ fn test_contract_accesses_when_validating_chunk() { &None, &apply_state, &[call_receipt], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -2168,7 +2168,7 @@ fn test_exclude_existing_contract_code_for_deploy_action() { &None, &apply_state, &[deploy_receipt1], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -2192,7 +2192,7 @@ fn test_exclude_existing_contract_code_for_deploy_action() { &None, &apply_state, &[deploy_receipt2], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -2269,7 +2269,7 @@ fn test_exclude_existing_contract_code_for_delete_account_action() { &None, &apply_state, &[create_account_receipt, deploy_receipt], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -2293,7 +2293,7 @@ fn test_exclude_existing_contract_code_for_delete_account_action() { &None, &apply_state, &[delete_account_receipt], - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -2337,7 +2337,6 @@ fn test_empty_apply() { setup_runtime(vec![alice_account()], initial_balance, initial_locked, gas_limit); let receipts = []; - let transactions = []; let apply_result = runtime .apply( @@ -2345,7 +2344,7 @@ fn test_empty_apply() { &None, &apply_state, &receipts, - &transactions, + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -2387,7 +2386,7 @@ fn test_congestion_delayed_receipts_accounting() { &None, &apply_state, &receipts, - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -2490,7 +2489,7 @@ fn test_congestion_buffering() { &None, &apply_state, prev_receipts, - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -2551,7 +2550,7 @@ fn test_congestion_buffering() { &None, &apply_state, prev_receipts, - &[], + SignedValidPeriodTransactions::empty(), &epoch_info_provider, Default::default(), ) @@ -2633,7 +2632,7 @@ fn check_congestion_info_bootstrapping(is_new_chunk: bool, want: Option