diff --git a/tools/state-viewer/src/apply_chunk.rs b/tools/state-viewer/src/apply_chunk.rs index f876db0639a..33dc867f379 100644 --- a/tools/state-viewer/src/apply_chunk.rs +++ b/tools/state-viewer/src/apply_chunk.rs @@ -1,4 +1,5 @@ use anyhow::{anyhow, Context}; +use borsh::BorshDeserialize; use near_chain::chain::collect_receipts_from_response; use near_chain::migrations::check_if_block_is_first_with_chunk_of_version; use near_chain::types::ApplyTransactionResult; @@ -6,16 +7,20 @@ use near_chain::{ChainStore, ChainStoreAccess, RuntimeAdapter}; use near_primitives::hash::CryptoHash; use near_primitives::merkle::combine_hash; use near_primitives::receipt::Receipt; +use near_primitives::shard_layout; use near_primitives::sharding::{ChunkHash, ReceiptProof}; use near_primitives::syncing::ReceiptProofResponse; +use near_primitives::types::{BlockHeight, ShardId}; use near_primitives_core::hash::hash; use near_primitives_core::types::Gas; +use near_store::db::DBCol; +use near_store::Store; use nearcore::NightshadeRuntime; use rand::rngs::StdRng; use rand::seq::SliceRandom; use std::cmp::Ord; -use std::collections::HashSet; -use std::sync::Arc; +use std::collections::{HashMap, HashSet}; +use tracing::warn; // like ChainStoreUpdate::get_incoming_receipts_for_shard(), but for the case when we don't // know of a block containing the target chunk @@ -65,33 +70,12 @@ fn get_incoming_receipts( Ok(collect_receipts_from_response(&responses)) } -fn check_hashes_exist CryptoHash>( - hashes: &Option>, - items: &[T], - item_hash: F, -) -> Result<(), CryptoHash> { - match hashes { - Some(hashes) => { - let hashes_seen = items.iter().map(item_hash).collect::>(); - for hash in hashes.iter() { - if !hashes_seen.contains(hash) { - return Err(*hash); - } - } - Ok(()) - } - None => Ok(()), - } -} - // returns (apply_result, gas limit) pub(crate) fn apply_chunk( - runtime: Arc, + runtime: &NightshadeRuntime, chain_store: &mut ChainStore, chunk_hash: ChunkHash, target_height: Option, - tx_hashes: &Option>, - receipt_hashes: &Option>, rng: Option, ) -> anyhow::Result<(ApplyTransactionResult, Gas)> { let chunk = chain_store.get_chunk(&chunk_hash)?; @@ -102,10 +86,6 @@ pub(crate) fn apply_chunk( let prev_state_root = chunk.prev_state_root(); let transactions = chunk.transactions().clone(); - check_hashes_exist(&tx_hashes, &transactions, |tx| tx.get_hash()).map_err(|hash| { - anyhow!("transaction with hash {} not found in chunk {:?}", hash, &chunk_hash) - })?; - let prev_block = chain_store.get_block(&prev_block_hash).context("Failed getting chunk's prev block")?; let prev_height_included = prev_block.chunks()[shard_id as usize].height_included(); @@ -126,12 +106,10 @@ pub(crate) fn apply_chunk( rng, ) .context("Failed collecting incoming receipts")?; - check_hashes_exist(&receipt_hashes, &receipts, |r| r.receipt_id).map_err(|hash| { - anyhow!("receipt with ID {} not found in any incoming receipt for shard {}", hash, shard_id) - })?; + let is_first_block_with_chunk_of_version = check_if_block_is_first_with_chunk_of_version( chain_store, - runtime.as_ref(), + runtime, &prev_block_hash, shard_id, )?; @@ -162,6 +140,267 @@ pub(crate) fn apply_chunk( )) } +enum HashType { + Tx, + Receipt, +} + +fn find_tx_or_receipt( + hash: &CryptoHash, + block_hash: &CryptoHash, + runtime: &NightshadeRuntime, + chain_store: &mut ChainStore, +) -> anyhow::Result> { + let block = chain_store.get_block(&block_hash)?; + let chunk_hashes = block.chunks().iter().map(|c| c.chunk_hash()).collect::>(); + + for (shard_id, chunk_hash) in chunk_hashes.iter().enumerate() { + let chunk = + chain_store.get_chunk(chunk_hash).context("Failed looking up canditate chunk")?; + for tx in chunk.transactions() { + if &tx.get_hash() == hash { + return Ok(Some((HashType::Tx, shard_id as ShardId))); + } + } + for receipt in chunk.receipts() { + if &receipt.get_hash() == hash { + let shard_layout = runtime.get_shard_layout_from_prev_block(chunk.prev_block())?; + let to_shard = + shard_layout::account_id_to_shard_id(&receipt.receiver_id, &shard_layout); + return Ok(Some((HashType::Receipt, to_shard))); + } + } + } + Ok(None) +} + +fn apply_tx_in_block( + runtime: &NightshadeRuntime, + chain_store: &mut ChainStore, + tx_hash: &CryptoHash, + block_hash: CryptoHash, +) -> anyhow::Result { + match find_tx_or_receipt(tx_hash, &block_hash, &runtime, chain_store)? { + Some((hash_type, shard_id)) => { + match hash_type { + HashType::Tx => { + println!("Found tx in block {} shard {}. equivalent command:\nview_state apply --height {} --shard-id {}\n", + &block_hash, shard_id, chain_store.get_block_header(&block_hash)?.height(), shard_id); + let (block, apply_result) = crate::commands::apply_block(block_hash, shard_id, runtime, chain_store); + crate::commands::print_apply_block_result(&block, &apply_result, runtime, chain_store, shard_id); + Ok(apply_result) + }, + HashType::Receipt => { + Err(anyhow!("{} appears to be a Receipt ID, not a tx hash. Try running:\nview_state apply_receipt --hash {}", tx_hash, tx_hash)) + }, + } + }, + None => { + Err(anyhow!("Could not find tx with hash {} in block {}, even though `ColTransactionResult` says it should be there", tx_hash, block_hash)) + } + } +} + +fn apply_tx_in_chunk( + runtime: &NightshadeRuntime, + store: Store, + chain_store: &mut ChainStore, + tx_hash: &CryptoHash, +) -> anyhow::Result> { + if chain_store.get_transaction(tx_hash)?.is_none() { + return Err(anyhow!("tx with hash {} not known", tx_hash)); + } + + println!("Transaction is known but doesn't seem to have been applied. Searching in chunks that haven't been applied..."); + + let head = chain_store.head()?.height; + let mut chunk_hashes = vec![]; + + for (k, v) in store.iter(DBCol::ColChunkHashesByHeight) { + let height = BlockHeight::from_le_bytes(k[..].try_into().unwrap()); + if height > head { + let hashes = HashSet::::try_from_slice(&v).unwrap(); + for chunk_hash in hashes { + let chunk = match chain_store.get_chunk(&chunk_hash) { + Ok(c) => c, + Err(_) => { + warn!(target: "state-viewer", "chunk hash {:?} appears in ColChunkHashesByHeight but the chunk is not saved", &chunk_hash); + continue; + } + }; + for hash in chunk.transactions().iter().map(|tx| tx.get_hash()) { + if hash == *tx_hash { + chunk_hashes.push(chunk_hash); + break; + } + } + } + } + } + + if chunk_hashes.len() == 0 { + return Err(anyhow!( + "Could not find tx with hash {} in any chunk that hasn't been applied yet", + tx_hash + )); + } + + let mut results = Vec::new(); + for chunk_hash in chunk_hashes { + println!("found tx in chunk {}. Equivalent command (which will run faster than apply_tx):\nview_state apply_chunk --chunk_hash {}\n", &chunk_hash.0, &chunk_hash.0); + let (apply_result, gas_limit) = + apply_chunk(runtime.clone(), chain_store, chunk_hash, None, None)?; + println!( + "resulting chunk extra:\n{:?}", + crate::commands::resulting_chunk_extra(&apply_result, gas_limit) + ); + results.push(apply_result); + } + Ok(results) +} + +pub(crate) fn apply_tx( + genesis_height: BlockHeight, + runtime: &NightshadeRuntime, + store: Store, + tx_hash: CryptoHash, +) -> anyhow::Result> { + let mut chain_store = ChainStore::new(store.clone(), genesis_height); + let outcomes = chain_store.get_outcomes_by_id(&tx_hash)?; + + if let Some(outcome) = outcomes.first() { + Ok(vec![apply_tx_in_block(runtime, &mut chain_store, &tx_hash, outcome.block_hash)?]) + } else { + apply_tx_in_chunk(runtime, store, &mut chain_store, &tx_hash) + } +} + +fn apply_receipt_in_block( + runtime: &NightshadeRuntime, + chain_store: &mut ChainStore, + id: &CryptoHash, + block_hash: CryptoHash, +) -> anyhow::Result { + match find_tx_or_receipt(id, &block_hash, &runtime, chain_store)? { + Some((hash_type, shard_id)) => { + match hash_type { + HashType::Tx => { + Err(anyhow!("{} appears to be a tx hash, not a Receipt ID. Try running:\nview_state apply_tx --hash {}", id, id)) + }, + HashType::Receipt => { + println!("Found receipt in block {}. Receiver is in shard {}. equivalent command:\nview_state apply --height {} --shard-id {}\n", + &block_hash, shard_id, chain_store.get_block_header(&block_hash)?.height(), shard_id); + let (block, apply_result) = crate::commands::apply_block(block_hash, shard_id, runtime, chain_store); + crate::commands::print_apply_block_result(&block, &apply_result, runtime, chain_store, shard_id); + Ok(apply_result) + }, + } + }, + None => { + // TODO: handle local/delayed receipts + Err(anyhow!("Could not find receipt with ID {} in block {}. Is it a local or delayed receipt?", id, block_hash)) + } + } +} + +fn apply_receipt_in_chunk( + runtime: &NightshadeRuntime, + store: Store, + chain_store: &mut ChainStore, + id: &CryptoHash, +) -> anyhow::Result> { + if chain_store.get_receipt(id)?.is_none() { + // TODO: handle local/delayed receipts + return Err(anyhow!("receipt with ID {} not known. Is it a local or delayed receipt?", id)); + } + + println!( + "Receipt is known but doesn't seem to have been applied. Searching in chunks that haven't been applied..." + ); + + let head = chain_store.head()?.height; + let mut to_apply = HashSet::new(); + let mut non_applied_chunks = HashMap::new(); + + for (k, v) in store.iter(DBCol::ColChunkHashesByHeight) { + let height = BlockHeight::from_le_bytes(k[..].try_into().unwrap()); + if height > head { + let hashes = HashSet::::try_from_slice(&v).unwrap(); + for chunk_hash in hashes { + let chunk = match chain_store.get_chunk(&chunk_hash) { + Ok(c) => c, + Err(_) => { + warn!(target: "state-viewer", "chunk hash {:?} appears in ColChunkHashesByHeight but the chunk is not saved", &chunk_hash); + continue; + } + }; + non_applied_chunks.insert((height, chunk.shard_id()), chunk_hash.clone()); + + for receipt in chunk.receipts().iter() { + if receipt.get_hash() == *id { + let shard_layout = + runtime.get_shard_layout_from_prev_block(chunk.prev_block())?; + let to_shard = shard_layout::account_id_to_shard_id( + &receipt.receiver_id, + &shard_layout, + ); + to_apply.insert((height, to_shard)); + println!( + "found receipt in chunk {}. Receiver is in shard {}", + &chunk_hash.0, to_shard + ); + break; + } + } + } + } + } + + if to_apply.len() == 0 { + return Err(anyhow!( + "Could not find receipt with hash {} in any chunk that hasn't been applied yet", + id + )); + } + + let mut results = Vec::new(); + for (height, shard_id) in to_apply { + let chunk_hash = match non_applied_chunks.get(&(height, shard_id)) { + Some(h) => h, + None => { + eprintln!( + "Wanted to apply chunk in shard {} at height {}, but no such chunk was found.", + shard_id, height, + ); + continue; + } + }; + println!("Applying chunk at height {} in shard {}. Equivalent command (which will run faster than apply_receipt):\nview_state apply_chunk --chunk_hash {}\n", + height, shard_id, chunk_hash.0); + let (apply_result, gas_limit) = + apply_chunk(runtime.clone(), chain_store, chunk_hash.clone(), None, None)?; + let chunk_extra = crate::commands::resulting_chunk_extra(&apply_result, gas_limit); + println!("resulting chunk extra:\n{:?}", chunk_extra); + results.push(apply_result); + } + Ok(results) +} + +pub(crate) fn apply_receipt( + genesis_height: BlockHeight, + runtime: &NightshadeRuntime, + store: Store, + id: CryptoHash, +) -> anyhow::Result> { + let mut chain_store = ChainStore::new(store.clone(), genesis_height); + let outcomes = chain_store.get_outcomes_by_id(&id)?; + if let Some(outcome) = outcomes.first() { + Ok(vec![apply_receipt_in_block(runtime, &mut chain_store, &id, outcome.block_hash)?]) + } else { + apply_receipt_in_chunk(runtime, store, &mut chain_store, &id) + } +} + #[cfg(test)] mod test { use near_chain::{ChainGenesis, ChainStore, ChainStoreAccess, Provenance, RuntimeAdapter}; @@ -171,6 +410,7 @@ mod test { use near_network::types::NetworkClientResponses; use near_primitives::hash::CryptoHash; use near_primitives::runtime::config_store::RuntimeConfigStore; + use near_primitives::shard_layout; use near_primitives::transaction::SignedTransaction; use near_primitives::utils::get_num_seats_per_shard; use near_store::test_utils::create_test_store; @@ -263,12 +503,10 @@ mod test { let new_root = new_roots[shard]; let (apply_result, _) = crate::apply_chunk::apply_chunk( - runtime.clone(), + runtime.as_ref(), &mut chain_store, chunk_hash.clone(), None, - &None, - &None, Some(rng), ) .unwrap(); @@ -277,4 +515,153 @@ mod test { } } } + + #[test] + fn test_apply_tx_apply_receipt() { + let genesis = Genesis::test_sharded( + vec![ + "test0".parse().unwrap(), + "test1".parse().unwrap(), + "test2".parse().unwrap(), + "test3".parse().unwrap(), + ], + 1, + get_num_seats_per_shard(4, 1), + ); + + let store = create_test_store(); + let mut chain_store = ChainStore::new(store.clone(), genesis.config.genesis_height); + let runtime = Arc::new(NightshadeRuntime::test_with_runtime_config_store( + Path::new("."), + store.clone(), + &genesis, + TrackedConfig::AllShards, + RuntimeConfigStore::test(), + )); + let mut chain_genesis = ChainGenesis::test(); + // receipts get delayed with the small ChainGenesis::test() limit + chain_genesis.gas_limit = genesis.config.gas_limit; + + let signers = (0..4) + .map(|i| { + let acc = format!("test{}", i); + InMemorySigner::from_seed(acc.parse().unwrap(), KeyType::ED25519, &acc) + }) + .collect::>(); + + let mut env = + TestEnv::builder(chain_genesis).runtime_adapters(vec![runtime.clone()]).build(); + let genesis_hash = *env.clients[0].chain.genesis().hash(); + + // first check that applying txs and receipts works when the block exists + + for height in 1..5 { + send_txs(&mut env, &signers, height, genesis_hash); + + let block = env.clients[0].produce_block(height).unwrap().unwrap(); + + let hash = *block.hash(); + let prev_hash = *block.header().prev_hash(); + let chunk_hashes = block.chunks().iter().map(|c| c.chunk_hash()).collect::>(); + let epoch_id = block.header().epoch_id().clone(); + + env.process_block(0, block, Provenance::PRODUCED); + + let new_roots = (0..4) + .map(|i| { + let shard_uid = runtime.shard_id_to_uid(i, &epoch_id).unwrap(); + chain_store.get_chunk_extra(&hash, &shard_uid).unwrap().state_root().clone() + }) + .collect::>(); + let shard_layout = runtime.get_shard_layout_from_prev_block(&prev_hash).unwrap(); + + if height >= 2 { + for shard_id in 0..4 { + let chunk = chain_store.get_chunk(&chunk_hashes[shard_id]).unwrap(); + + for tx in chunk.transactions() { + let results = crate::apply_chunk::apply_tx( + genesis.config.genesis_height, + runtime.as_ref(), + store.clone(), + tx.get_hash(), + ) + .unwrap(); + assert_eq!(results.len(), 1); + assert_eq!(results[0].new_root, new_roots[shard_id as usize]); + } + + for receipt in chunk.receipts() { + let to_shard = shard_layout::account_id_to_shard_id( + &receipt.receiver_id, + &shard_layout, + ); + + let results = crate::apply_chunk::apply_receipt( + genesis.config.genesis_height, + runtime.as_ref(), + store.clone(), + receipt.get_hash(), + ) + .unwrap(); + assert_eq!(results.len(), 1); + assert_eq!(results[0].new_root, new_roots[to_shard as usize]); + } + } + } + } + + // then check what happens when the block doesn't exist + // it won't exist because the chunks for the last height + // in the loop above are produced by env.process_block() but + // there was no corresponding env.clients[0].produce_block() after + + let chunks = chain_store.get_all_chunk_hashes_by_height(5).unwrap(); + let blocks = chain_store.get_all_header_hashes_by_height(5).unwrap(); + assert_ne!(chunks.len(), 0); + assert_eq!(blocks.len(), 0); + + for chunk_hash in chunks { + let chunk = chain_store.get_chunk(&chunk_hash).unwrap(); + + for tx in chunk.transactions() { + let results = crate::apply_chunk::apply_tx( + genesis.config.genesis_height, + runtime.as_ref(), + store.clone(), + tx.get_hash(), + ) + .unwrap(); + for result in results { + let mut applied = false; + for outcome in result.outcomes { + if outcome.id == tx.get_hash() { + applied = true; + break; + } + } + assert!(applied); + } + } + for receipt in chunk.receipts() { + let results = crate::apply_chunk::apply_receipt( + genesis.config.genesis_height, + runtime.as_ref(), + store.clone(), + receipt.get_hash(), + ) + .unwrap(); + for result in results { + let mut applied = false; + for outcome in result.outcomes { + if outcome.id == receipt.get_hash() { + applied = true; + break; + } + } + assert!(applied); + } + } + } + } } diff --git a/tools/state-viewer/src/cli.rs b/tools/state-viewer/src/cli.rs index d847d5c6f01..f3d41493c24 100644 --- a/tools/state-viewer/src/cli.rs +++ b/tools/state-viewer/src/cli.rs @@ -109,6 +109,14 @@ pub enum StateViewerSubCommand { /// Apply a chunk, even if it's not included in any block on disk #[clap(name = "apply_chunk")] ApplyChunk(ApplyChunkCmd), + /// Apply a transaction if it occurs in some chunk we know about, + /// even if it's not included in any block on disk + #[clap(name = "apply_tx")] + ApplyTx(ApplyTxCmd), + /// Apply a receipt if it occurs in some chunk we know about, + /// even if it's not included in any block on disk + #[clap(name = "apply_receipt")] + ApplyReceipt(ApplyReceiptCmd), } impl StateViewerSubCommand { @@ -137,6 +145,8 @@ impl StateViewerSubCommand { StateViewerSubCommand::Chunks(cmd) => cmd.run(near_config, store), StateViewerSubCommand::PartialChunks(cmd) => cmd.run(near_config, store), StateViewerSubCommand::ApplyChunk(cmd) => cmd.run(home_dir, near_config, store), + StateViewerSubCommand::ApplyTx(cmd) => cmd.run(home_dir, near_config, store), + StateViewerSubCommand::ApplyReceipt(cmd) => cmd.run(home_dir, near_config, store), } } } @@ -388,21 +398,37 @@ pub struct ApplyChunkCmd { chunk_hash: String, #[clap(long)] target_height: Option, - #[clap(long)] - txs: Option>, - #[clap(long)] - receipts: Option>, } impl ApplyChunkCmd { pub fn run(self, home_dir: &Path, near_config: NearConfig, store: Store) { let hash = ChunkHash::from(CryptoHash::from_str(&self.chunk_hash).unwrap()); - let receipts = self - .receipts - .map(|v| v.iter().map(|h| CryptoHash::from_str(h).unwrap()).collect::>()); - let txs = self - .txs - .map(|v| v.iter().map(|h| CryptoHash::from_str(h).unwrap()).collect::>()); - apply_chunk(home_dir, near_config, store, hash, self.target_height, txs, receipts).unwrap() + apply_chunk(home_dir, near_config, store, hash, self.target_height).unwrap() + } +} + +#[derive(Parser)] +pub struct ApplyTxCmd { + #[clap(long)] + hash: String, +} + +impl ApplyTxCmd { + pub fn run(self, home_dir: &Path, near_config: NearConfig, store: Store) { + let hash = CryptoHash::from_str(&self.hash).unwrap(); + apply_tx(home_dir, near_config, store, hash).unwrap(); + } +} + +#[derive(Parser)] +pub struct ApplyReceiptCmd { + #[clap(long)] + hash: String, +} + +impl ApplyReceiptCmd { + pub fn run(self, home_dir: &Path, near_config: NearConfig, store: Store) { + let hash = CryptoHash::from_str(&self.hash).unwrap(); + apply_receipt(home_dir, near_config, store, hash).unwrap(); } } diff --git a/tools/state-viewer/src/commands.rs b/tools/state-viewer/src/commands.rs index 65a8636d826..5462e61630b 100644 --- a/tools/state-viewer/src/commands.rs +++ b/tools/state-viewer/src/commands.rs @@ -10,7 +10,7 @@ use near_chain::{ChainStore, ChainStoreAccess, ChainStoreUpdate, RuntimeAdapter} use near_epoch_manager::EpochManager; use near_network::iter_peers_from_store; use near_primitives::account::id::AccountId; -use near_primitives::block::BlockHeader; +use near_primitives::block::{Block, BlockHeader}; use near_primitives::hash::CryptoHash; use near_primitives::serialize::to_base; use near_primitives::shard_layout::ShardUId; @@ -24,7 +24,7 @@ use near_store::test_utils::create_test_store; use near_store::{Store, TrieIterator}; use nearcore::{NearConfig, NightshadeRuntime}; use node_runtime::adapter::ViewRuntimeAdapter; -use std::collections::{HashMap, HashSet}; +use std::collections::HashMap; use std::fs::{self, File}; use std::io::Write; use std::path::{Path, PathBuf}; @@ -333,41 +333,32 @@ pub(crate) fn replay_chain( } } -fn resulting_chunk_extra(result: ApplyTransactionResult, gas_limit: Gas) -> ChunkExtra { +pub(crate) fn resulting_chunk_extra(result: &ApplyTransactionResult, gas_limit: Gas) -> ChunkExtra { let (outcome_root, _) = ApplyTransactionResult::compute_outcomes_proof(&result.outcomes); ChunkExtra::new( &result.new_root, outcome_root, - result.validator_proposals, + result.validator_proposals.clone(), result.total_gas_burnt, gas_limit, result.total_balance_burnt, ) } -pub(crate) fn apply_block_at_height( - height: BlockHeight, +pub(crate) fn apply_block( + block_hash: CryptoHash, shard_id: ShardId, - home_dir: &Path, - near_config: NearConfig, - store: Store, -) { - let mut chain_store = ChainStore::new(store.clone(), near_config.genesis.config.genesis_height); - let runtime_adapter: Arc = Arc::new(NightshadeRuntime::with_config( - home_dir, - store, - &near_config, - None, - near_config.client_config.max_gas_burnt_view, - )); - let block_hash = chain_store.get_block_hash_by_height(height).unwrap(); + runtime_adapter: &dyn RuntimeAdapter, + chain_store: &mut ChainStore, +) -> (Block, ApplyTransactionResult) { let block = chain_store.get_block(&block_hash).unwrap().clone(); + let height = block.header().height(); let shard_uid = runtime_adapter.shard_id_to_uid(shard_id, block.header().epoch_id()).unwrap(); let apply_result = if block.chunks()[shard_id as usize].height_included() == height { let chunk = chain_store.get_chunk(&block.chunks()[shard_id as usize].chunk_hash()).unwrap().clone(); let prev_block = chain_store.get_block(block.header().prev_hash()).unwrap().clone(); - let mut chain_store_update = ChainStoreUpdate::new(&mut chain_store); + let mut chain_store_update = ChainStoreUpdate::new(chain_store); let receipt_proof_response = chain_store_update .get_incoming_receipts_for_shard( shard_id, @@ -379,12 +370,13 @@ pub(crate) fn apply_block_at_height( let chunk_inner = chunk.cloned_header().take_inner(); let is_first_block_with_chunk_of_version = check_if_block_is_first_with_chunk_of_version( - &mut chain_store, - runtime_adapter.as_ref(), + chain_store, + runtime_adapter, block.header().prev_hash(), shard_id, ) .unwrap(); + runtime_adapter .apply_transactions( shard_id, @@ -430,12 +422,25 @@ pub(crate) fn apply_block_at_height( ) .unwrap() }; + (block, apply_result) +} + +pub(crate) fn print_apply_block_result( + block: &Block, + apply_result: &ApplyTransactionResult, + runtime_adapter: &dyn RuntimeAdapter, + chain_store: &mut ChainStore, + shard_id: ShardId, +) { + let height = block.header().height(); + let block_hash = block.header().hash(); println!( "apply chunk for shard {} at height {}, resulting chunk extra {:?}", shard_id, height, resulting_chunk_extra(apply_result, block.chunks()[shard_id as usize].gas_limit()) ); + let shard_uid = runtime_adapter.shard_id_to_uid(shard_id, block.header().epoch_id()).unwrap(); if block.chunks()[shard_id as usize].height_included() == height { if let Ok(chunk_extra) = chain_store.get_chunk_extra(&block_hash, &shard_uid) { println!("Existing chunk extra: {:?}", chunk_extra); @@ -447,6 +452,33 @@ pub(crate) fn apply_block_at_height( } } +pub(crate) fn apply_block_at_height( + height: BlockHeight, + shard_id: ShardId, + home_dir: &Path, + near_config: NearConfig, + store: Store, +) { + let mut chain_store = ChainStore::new(store.clone(), near_config.genesis.config.genesis_height); + let runtime_adapter: Arc = Arc::new(NightshadeRuntime::with_config( + home_dir, + store, + &near_config, + None, + near_config.client_config.max_gas_burnt_view, + )); + let block_hash = chain_store.get_block_hash_by_height(height).unwrap(); + let (block, apply_result) = + apply_block(block_hash, shard_id, runtime_adapter.as_ref(), &mut chain_store); + print_apply_block_result( + &block, + &apply_result, + runtime_adapter.as_ref(), + &mut chain_store, + shard_id, + ); +} + pub(crate) fn view_chain( height: Option, view_block: bool, @@ -673,57 +705,57 @@ pub fn chunk_mask_to_str(mask: &[bool]) -> String { mask.iter().map(|f| if *f { '.' } else { 'X' }).collect() } -fn print_apply_chunk_result( - result: ApplyTransactionResult, - gas_limit: Gas, - tx_hashes: Option>, - receipt_hashes: Option>, -) { - if tx_hashes.is_some() || receipt_hashes.is_some() { - let mut hashes = HashSet::new(); - if let Some(tx_hashes) = tx_hashes { - hashes.extend(tx_hashes); - } - if let Some(receipt_hashes) = receipt_hashes { - hashes.extend(receipt_hashes); - } - - println!("outcomes:"); - for outcome in result.outcomes.iter() { - if hashes.contains(&outcome.id) { - println!("{:?}", outcome); - } - } - } - println!("resulting chunk extra:\n{:?}", resulting_chunk_extra(result, gas_limit)); -} - pub(crate) fn apply_chunk( home_dir: &Path, near_config: NearConfig, store: Store, chunk_hash: ChunkHash, target_height: Option, - tx_hashes: Option>, - receipt_hashes: Option>, ) -> anyhow::Result<()> { - let runtime = Arc::new(NightshadeRuntime::with_config( + let runtime = NightshadeRuntime::with_config( home_dir, store.clone(), &near_config, None, near_config.client_config.max_gas_burnt_view, - )); + ); let mut chain_store = ChainStore::new(store, near_config.genesis.config.genesis_height); - let (apply_result, gas_limit) = apply_chunk::apply_chunk( - runtime, - &mut chain_store, - chunk_hash, - target_height, - &tx_hashes, - &receipt_hashes, - None, - )?; - print_apply_chunk_result(apply_result, gas_limit, tx_hashes, receipt_hashes); + let (apply_result, gas_limit) = + apply_chunk::apply_chunk(&runtime, &mut chain_store, chunk_hash, target_height, None)?; + println!("resulting chunk extra:\n{:?}", resulting_chunk_extra(&apply_result, gas_limit)); Ok(()) } + +pub(crate) fn apply_tx( + home_dir: &Path, + near_config: NearConfig, + store: Store, + hash: CryptoHash, +) -> anyhow::Result<()> { + let runtime = NightshadeRuntime::with_config( + home_dir, + store.clone(), + &near_config, + None, + near_config.client_config.max_gas_burnt_view, + ); + apply_chunk::apply_tx(near_config.genesis.config.genesis_height, &runtime, store, hash) + .map(|_| ()) +} + +pub(crate) fn apply_receipt( + home_dir: &Path, + near_config: NearConfig, + store: Store, + hash: CryptoHash, +) -> anyhow::Result<()> { + let runtime = NightshadeRuntime::with_config( + home_dir, + store.clone(), + &near_config, + None, + near_config.client_config.max_gas_burnt_view, + ); + apply_chunk::apply_receipt(near_config.genesis.config.genesis_height, &runtime, store, hash) + .map(|_| ()) +}