Skip to content

Commit

Permalink
fix: properly handle genesis as part of stateless validation (#10633)
Browse files Browse the repository at this point in the history
This PR removes "approve anything" shortcut when previous chunk is part
of genesis. The only difference in case of genesis is that we don't want
to execute main state transition and instead just check that post state
root matches genesis state root for that shard.

This also exposed some issue I had to fix to make tests work:
* `MockEpochManager::get_epoch_chunk_producers` returns empty Vec which
results in `Chain:: should_produce_state_witness_for_this_or_next_epoch`
returning `false`. Fixed by adding `is_chunk_producer_for_epoch` as part
of `EpochManagerAdapter` so `MockEpochManager` can override it.
* `test_chunk_state_witness_bad_shard_id` test started failing: this
actually uncovered a real issue which could result in crashing chunk
validator when state witness contains invalid shard id, fixed in
c5b2c5e

Closes #10502.
  • Loading branch information
pugachAG authored Feb 23, 2024
1 parent f051314 commit 0918295
Show file tree
Hide file tree
Showing 20 changed files with 244 additions and 161 deletions.
44 changes: 29 additions & 15 deletions chain/chain/src/chain.rs
Original file line number Diff line number Diff line change
Expand Up @@ -78,8 +78,8 @@ use near_primitives::static_clock::StaticClock;
use near_primitives::transaction::{ExecutionOutcomeWithIdAndProof, SignedTransaction};
use near_primitives::types::chunk_extra::ChunkExtra;
use near_primitives::types::{
AccountId, Balance, BlockExtra, BlockHeight, BlockHeightDelta, EpochId, MerkleHash, NumBlocks,
ShardId, StateRoot,
AccountId, Balance, BlockExtra, BlockHeight, BlockHeightDelta, EpochId, Gas, MerkleHash,
NumBlocks, ShardId, StateRoot,
};
use near_primitives::unwrap_or_return;
#[cfg(feature = "new_epoch_sync")]
Expand Down Expand Up @@ -458,14 +458,7 @@ impl Chain {
genesis.hash(),
&epoch_manager
.shard_id_to_uid(chunk_header.shard_id(), &EpochId::default())?,
ChunkExtra::new(
state_root,
CryptoHash::default(),
vec![],
0,
chain_genesis.gas_limit,
0,
),
Self::create_genesis_chunk_extra(state_root, chain_genesis.gas_limit),
);
}

Expand Down Expand Up @@ -577,6 +570,29 @@ impl Chain {
self.last_time_head_updated
}

fn create_genesis_chunk_extra(state_root: &StateRoot, gas_limit: Gas) -> ChunkExtra {
ChunkExtra::new(state_root, CryptoHash::default(), vec![], 0, gas_limit, 0)
}

pub fn genesis_chunk_extra(&self, shard_id: ShardId) -> Result<ChunkExtra, Error> {
let shard_index = shard_id as usize;
let state_root = *get_genesis_state_roots(self.chain_store.store())?
.ok_or_else(|| Error::Other("genesis state roots do not exist in the db".to_owned()))?
.get(shard_index)
.ok_or_else(|| {
Error::Other(format!("genesis state root does not exist for shard {shard_index}"))
})?;
let gas_limit = self
.genesis
.chunks()
.get(shard_index)
.ok_or_else(|| {
Error::Other(format!("genesis chunk does not exist for shard {shard_index}"))
})?
.gas_limit();
Ok(Self::create_genesis_chunk_extra(&state_root, gas_limit))
}

/// Creates a light client block for the last final block from perspective of some other block
///
/// # Arguments
Expand Down Expand Up @@ -3257,11 +3273,9 @@ impl Chain {
// Chunk validation not enabled yet.
return Ok(false);
}
let mut all_chunk_producers = self.epoch_manager.get_epoch_chunk_producers(epoch_id)?;
all_chunk_producers
.extend(self.epoch_manager.get_epoch_chunk_producers(&next_epoch_id)?.into_iter());
let mut chunk_producer_accounts = all_chunk_producers.iter().map(|v| v.account_id());
Ok(me.as_ref().map_or(false, |a| chunk_producer_accounts.contains(a)))
let Some(account_id) = me.as_ref() else { return Ok(false) };
Ok(self.epoch_manager.is_chunk_producer_for_epoch(epoch_id, account_id)?
|| self.epoch_manager.is_chunk_producer_for_epoch(&next_epoch_id, account_id)?)
}

/// Creates jobs which will update shards for the given block and incoming
Expand Down
23 changes: 19 additions & 4 deletions chain/chain/src/test_utils/kv_runtime.rs
Original file line number Diff line number Diff line change
Expand Up @@ -696,6 +696,18 @@ impl EpochManagerAdapter for MockEpochManager {
Ok(vec![])
}

/// We need to override the default implementation to make
/// `Chain::should_produce_state_witness_for_this_or_next_epoch` work
/// since `get_epoch_chunk_producers` returns empty Vec which results
/// in state transition data not being saved on disk.
fn is_chunk_producer_for_epoch(
&self,
_epoch_id: &EpochId,
_account_id: &AccountId,
) -> Result<bool, EpochError> {
Ok(true)
}

fn get_block_producer(
&self,
epoch_id: &EpochId,
Expand Down Expand Up @@ -1054,7 +1066,7 @@ impl RuntimeAdapter for KeyValueRuntime {

fn prepare_transactions(
&self,
_storage: RuntimeStorageConfig,
storage: RuntimeStorageConfig,
_chunk: PrepareTransactionsChunkContext,
_prev_block: PrepareTransactionsBlockContext,
transaction_groups: &mut dyn TransactionGroupIterator,
Expand All @@ -1065,7 +1077,11 @@ impl RuntimeAdapter for KeyValueRuntime {
while let Some(iter) = transaction_groups.next() {
res.push(iter.next().unwrap());
}
Ok(PreparedTransactions { transactions: res, limited_by: None, storage_proof: None })
Ok(PreparedTransactions {
transactions: res,
limited_by: None,
storage_proof: if storage.record_storage { Some(Default::default()) } else { None },
})
}

fn apply_chunk(
Expand All @@ -1076,7 +1092,6 @@ impl RuntimeAdapter for KeyValueRuntime {
receipts: &[Receipt],
transactions: &[SignedTransaction],
) -> Result<ApplyChunkResult, Error> {
assert!(!storage_config.record_storage);
let mut tx_results = vec![];
let shard_id = chunk.shard_id;

Expand Down Expand Up @@ -1224,7 +1239,7 @@ impl RuntimeAdapter for KeyValueRuntime {
validator_proposals: vec![],
total_gas_burnt: 0,
total_balance_burnt: 0,
proof: None,
proof: if storage_config.record_storage { Some(Default::default()) } else { None },
processed_delayed_receipts: vec![],
applied_receipts_hash: hash(&borsh::to_vec(receipts).unwrap()),
})
Expand Down
3 changes: 3 additions & 0 deletions chain/client/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -118,3 +118,6 @@ sandbox = [
new_epoch_sync = [
"near-chain/new_epoch_sync"
]
statelessnet_protocol = [
"near-chain/statelessnet_protocol",
]
23 changes: 11 additions & 12 deletions chain/client/src/client.rs
Original file line number Diff line number Diff line change
Expand Up @@ -849,15 +849,11 @@ impl Client {
let prepared_transactions =
self.prepare_transactions(shard_uid, prev_block_hash, chunk_extra.as_ref())?;
#[cfg(feature = "test_features")]
let prepared_transactions = PreparedTransactions {
transactions: Self::maybe_insert_invalid_transaction(
prepared_transactions.transactions,
prev_block_hash,
self.produce_invalid_tx_in_chunks,
),
limited_by: prepared_transactions.limited_by,
storage_proof: prepared_transactions.storage_proof,
};
let prepared_transactions = Self::maybe_insert_invalid_transaction(
prepared_transactions,
prev_block_hash,
self.produce_invalid_tx_in_chunks,
);
let num_filtered_transactions = prepared_transactions.transactions.len();
let (tx_root, _) = merklize(&prepared_transactions.transactions);
let outgoing_receipts = self.chain.get_outgoing_receipts_for_shard(
Expand Down Expand Up @@ -945,12 +941,12 @@ impl Client {

#[cfg(feature = "test_features")]
fn maybe_insert_invalid_transaction(
mut txs: Vec<SignedTransaction>,
mut txs: PreparedTransactions,
prev_block_hash: CryptoHash,
insert: bool,
) -> Vec<SignedTransaction> {
) -> PreparedTransactions {
if insert {
txs.push(SignedTransaction::new(
txs.transactions.push(SignedTransaction::new(
near_crypto::Signature::empty(near_crypto::KeyType::ED25519),
near_primitives::transaction::Transaction::new(
"test".parse().unwrap(),
Expand All @@ -960,6 +956,9 @@ impl Client {
prev_block_hash,
),
));
if txs.storage_proof.is_none() {
txs.storage_proof = Some(Default::default());
}
}
txs
}
Expand Down
134 changes: 80 additions & 54 deletions chain/client/src/stateless_validation/chunk_validator.rs
Original file line number Diff line number Diff line change
Expand Up @@ -210,6 +210,7 @@ pub(crate) fn pre_validate_chunk_state_witness(
)));
};
let is_new_chunk = chunk.is_new_chunk(block.header().height());
let is_genesis = block.header().is_genesis();
block_hash = *block.header().prev_hash();
if is_new_chunk {
prev_chunks_seen += 1;
Expand All @@ -219,7 +220,7 @@ pub(crate) fn pre_validate_chunk_state_witness(
} else if prev_chunks_seen == 1 {
blocks_after_last_last_chunk.push(block);
}
if prev_chunks_seen == 2 {
if prev_chunks_seen == 2 || is_genesis {
break;
}
}
Expand Down Expand Up @@ -288,8 +289,14 @@ pub(crate) fn pre_validate_chunk_state_witness(
};
}

Ok(PreValidationOutput {
main_transition_params: NewChunkData {
let main_transition_params = if last_chunk_block.header().is_genesis() {
MainTransition::Genesis {
chunk_extra: chain.genesis_chunk_extra(shard_id)?,
block_hash: *last_chunk_block.hash(),
shard_id,
}
} else {
MainTransition::NewChunk(NewChunkData {
chunk_header: last_chunk_block.chunks().get(shard_id as usize).unwrap().clone(),
transactions: state_witness.transactions.clone(),
receipts: receipts_to_apply,
Expand All @@ -308,7 +315,11 @@ pub(crate) fn pre_validate_chunk_state_witness(
state_patch: Default::default(),
record_storage: false,
},
},
})
};

Ok(PreValidationOutput {
main_transition_params,
implicit_transition_params: blocks_after_last_chunk
.into_iter()
.rev()
Expand All @@ -335,6 +346,21 @@ fn validate_source_receipt_proofs(
receipt_source_blocks: &[Block],
target_chunk_shard_id: ShardId,
) -> Result<Vec<Receipt>, Error> {
if receipt_source_blocks.iter().any(|block| block.header().is_genesis()) {
if receipt_source_blocks.len() != 1 {
return Err(Error::Other(
"Invalid chain state: receipt_source_blocks should not have any blocks alongside genesis".to_owned()
));
}
if !source_receipt_proofs.is_empty() {
return Err(Error::InvalidChunkStateWitness(format!(
"genesis source_receipt_proofs should be empty, actual len is {}",
source_receipt_proofs.len()
)));
}
return Ok(vec![]);
}

let mut receipts_to_apply = Vec::new();
let mut expected_proofs_len = 0;

Expand Down Expand Up @@ -415,8 +441,29 @@ fn validate_receipt_proof(
Ok(())
}

enum MainTransition {
Genesis { chunk_extra: ChunkExtra, block_hash: CryptoHash, shard_id: ShardId },
NewChunk(NewChunkData),
}

impl MainTransition {
fn block_hash(&self) -> CryptoHash {
match self {
Self::Genesis { block_hash, .. } => *block_hash,
Self::NewChunk(data) => data.block.block_hash,
}
}

fn shard_id(&self) -> ShardId {
match self {
Self::Genesis { shard_id, .. } => *shard_id,
Self::NewChunk(data) => data.chunk_header.shard_id(),
}
}
}

pub(crate) struct PreValidationOutput {
main_transition_params: NewChunkData,
main_transition_params: MainTransition,
implicit_transition_params: Vec<ApplyChunkBlockContext>,
}

Expand All @@ -426,31 +473,35 @@ pub(crate) fn validate_chunk_state_witness(
epoch_manager: &dyn EpochManagerAdapter,
runtime_adapter: &dyn RuntimeAdapter,
) -> Result<(), Error> {
let main_transition = pre_validation_output.main_transition_params;
let _timer = metrics::CHUNK_STATE_WITNESS_VALIDATION_TIME
.with_label_values(&[&main_transition.chunk_header.shard_id().to_string()])
.with_label_values(&[&state_witness.chunk_header.shard_id().to_string()])
.start_timer();
let span = tracing::debug_span!(target: "chain", "validate_chunk_state_witness").entered();
let chunk_header = main_transition.chunk_header.clone();
let epoch_id = epoch_manager.get_epoch_id(&main_transition.block.block_hash)?;
let shard_uid =
epoch_manager.shard_id_to_uid(main_transition.chunk_header.shard_id(), &epoch_id)?;
// Should we validate other fields?
let NewChunkResult { apply_result: mut main_apply_result, .. } = apply_new_chunk(
&span,
main_transition,
ShardContext {
shard_uid,
cares_about_shard_this_epoch: true,
will_shard_layout_change: false,
should_apply_chunk: true,
need_to_reshard: false,
},
runtime_adapter,
epoch_manager,
)?;
let outgoing_receipts = std::mem::take(&mut main_apply_result.outgoing_receipts);
let mut chunk_extra = apply_result_to_chunk_extra(main_apply_result, &chunk_header);
let block_hash = pre_validation_output.main_transition_params.block_hash();
let epoch_id = epoch_manager.get_epoch_id(&block_hash)?;
let shard_uid = epoch_manager
.shard_id_to_uid(pre_validation_output.main_transition_params.shard_id(), &epoch_id)?;
let (mut chunk_extra, outgoing_receipts) = match pre_validation_output.main_transition_params {
MainTransition::Genesis { chunk_extra, .. } => (chunk_extra, vec![]),
MainTransition::NewChunk(new_chunk_data) => {
let chunk_header = new_chunk_data.chunk_header.clone();
let NewChunkResult { apply_result: mut main_apply_result, .. } = apply_new_chunk(
&span,
new_chunk_data,
ShardContext {
shard_uid,
cares_about_shard_this_epoch: true,
will_shard_layout_change: false,
should_apply_chunk: true,
need_to_reshard: false,
},
runtime_adapter,
epoch_manager,
)?;
let outgoing_receipts = std::mem::take(&mut main_apply_result.outgoing_receipts);
(apply_result_to_chunk_extra(main_apply_result, &chunk_header), outgoing_receipts)
}
};
if chunk_extra.state_root() != &state_witness.main_state_transition.post_state_root {
// This is an early check, it's not for correctness, only for better
// error reporting in case of an invalid state witness due to a bug.
Expand Down Expand Up @@ -589,34 +640,9 @@ impl Client {
peer_id: PeerId,
processing_done_tracker: Option<ProcessingDoneTracker>,
) -> Result<Option<ChunkStateWitness>, Error> {
// TODO(#10502): Handle production of state witness for first chunk after genesis.
// Properly handle case for chunk right after genesis.
// Context: We are currently unable to handle production of the state witness for the
// first chunk after genesis as it's not possible to run the genesis chunk in runtime.
let prev_block_hash = witness.inner.chunk_header.prev_block_hash();
let prev_block = match self.chain.get_block(prev_block_hash) {
Ok(block) => block,
Err(_) => {
return Ok(Some(witness));
}
};
let prev_chunk_header = Chain::get_prev_chunk_header(
self.epoch_manager.as_ref(),
&prev_block,
witness.inner.chunk_header.shard_id(),
)?;
if prev_chunk_header.prev_block_hash() == &CryptoHash::default() {
let Some(signer) = self.validator_signer.as_ref() else {
return Err(Error::NotAChunkValidator);
};
send_chunk_endorsement_to_block_producers(
&witness.inner.chunk_header,
self.epoch_manager.as_ref(),
signer.as_ref(),
&self.chunk_validator.network_sender,
self.chunk_endorsement_tracker.as_ref(),
);
return Ok(None);
if self.chain.get_block(prev_block_hash).is_err() {
return Ok(Some(witness));
}

// TODO(#10265): If the previous block does not exist, we should
Expand Down
Loading

0 comments on commit 0918295

Please sign in to comment.