diff --git a/chain/chain/src/test_utils/kv_runtime.rs b/chain/chain/src/test_utils/kv_runtime.rs index f7f9950ad4b..882d89fe8ff 100644 --- a/chain/chain/src/test_utils/kv_runtime.rs +++ b/chain/chain/src/test_utils/kv_runtime.rs @@ -30,11 +30,9 @@ use near_primitives::hash::{hash, CryptoHash}; use near_primitives::receipt::{ActionReceipt, Receipt, ReceiptEnum, ReceiptV0}; use near_primitives::shard_layout; use near_primitives::shard_layout::{ShardLayout, ShardUId}; -use near_primitives::sharding::{ChunkHash, ShardChunkHeader}; +use near_primitives::sharding::ChunkHash; use near_primitives::state_part::PartId; -use near_primitives::stateless_validation::chunk_endorsement::{ - ChunkEndorsementV1, ChunkEndorsementV2, -}; +use near_primitives::stateless_validation::chunk_endorsement::ChunkEndorsement; use near_primitives::stateless_validation::contract_distribution::ChunkContractAccesses; use near_primitives::stateless_validation::partial_witness::PartialEncodedStateWitness; use near_primitives::stateless_validation::validator_assignment::ChunkValidatorAssignments; @@ -1016,17 +1014,9 @@ impl EpochManagerAdapter for MockEpochManager { } } - fn verify_chunk_endorsement( - &self, - _chunk_header: &ShardChunkHeader, - _endorsement: &ChunkEndorsementV1, - ) -> Result { - Ok(true) - } - fn verify_chunk_endorsement_signature( &self, - _endorsement: &ChunkEndorsementV2, + _endorsement: &ChunkEndorsement, ) -> Result { Ok(true) } diff --git a/chain/client/src/client.rs b/chain/client/src/client.rs index 72c5a6ad29d..70fc954e8be 100644 --- a/chain/client/src/client.rs +++ b/chain/client/src/client.rs @@ -1581,7 +1581,6 @@ impl Client { persist_chunk(partial_chunk, shard_chunk, self.chain.mut_chain_store()) .expect("Could not persist chunk"); - self.chunk_endorsement_tracker.tracker_v1.process_pending_endorsements(&chunk_header); // We're marking chunk as accepted. self.chain.blocks_with_missing_chunks.accept_chunk(&chunk_header.chunk_hash()); // If this was the last chunk that was missing for a block, it will be processed now. @@ -1968,17 +1967,6 @@ impl Client { } } - pub fn mark_chunk_header_ready_for_inclusion( - &mut self, - chunk_header: ShardChunkHeader, - chunk_producer: AccountId, - ) { - // If endorsement was received before chunk header, we can process it only now. - self.chunk_endorsement_tracker.tracker_v1.process_pending_endorsements(&chunk_header); - self.chunk_inclusion_tracker - .mark_chunk_header_ready_for_inclusion(chunk_header, chunk_producer); - } - pub fn persist_and_distribute_encoded_chunk( &mut self, encoded_chunk: EncodedShardChunk, @@ -2012,7 +2000,8 @@ impl Client { } } - self.mark_chunk_header_ready_for_inclusion(chunk_header, validator_id); + self.chunk_inclusion_tracker + .mark_chunk_header_ready_for_inclusion(chunk_header, validator_id); self.shards_manager_adapter.send(ShardsManagerRequestFromClient::DistributeEncodedChunk { partial_chunk, encoded_chunk, diff --git a/chain/client/src/client_actor.rs b/chain/client/src/client_actor.rs index e87562d3931..90416b7ad53 100644 --- a/chain/client/src/client_actor.rs +++ b/chain/client/src/client_actor.rs @@ -2135,7 +2135,9 @@ impl Handler for ClientActorInner { chunk_header, chunk_producer, } => { - self.client.mark_chunk_header_ready_for_inclusion(chunk_header, chunk_producer); + self.client + .chunk_inclusion_tracker + .mark_chunk_header_ready_for_inclusion(chunk_header, chunk_producer); } } } @@ -2165,7 +2167,7 @@ impl Handler for ClientActorInner { impl Handler for ClientActorInner { #[perf] fn handle(&mut self, msg: ChunkEndorsementMessage) { - if let Err(err) = self.client.process_chunk_endorsement(msg.0) { + if let Err(err) = self.client.chunk_endorsement_tracker.process_chunk_endorsement(msg.0) { tracing::error!(target: "client", ?err, "Error processing chunk endorsement"); } } diff --git a/chain/client/src/stateless_validation/chunk_endorsement/tracker_v2.rs b/chain/client/src/stateless_validation/chunk_endorsement.rs similarity index 84% rename from chain/client/src/stateless_validation/chunk_endorsement/tracker_v2.rs rename to chain/client/src/stateless_validation/chunk_endorsement.rs index f6a3ee1cf5d..6aa985806cd 100644 --- a/chain/client/src/stateless_validation/chunk_endorsement/tracker_v2.rs +++ b/chain/client/src/stateless_validation/chunk_endorsement.rs @@ -4,16 +4,17 @@ use std::sync::Arc; use lru::LruCache; use near_chain_primitives::Error; +use near_crypto::Signature; use near_epoch_manager::EpochManagerAdapter; -use near_primitives::sharding::ShardChunkHeader; -use near_primitives::stateless_validation::chunk_endorsement::ChunkEndorsementV2; +use near_primitives::sharding::{ChunkHash, ShardChunkHeader}; +use near_primitives::stateless_validation::chunk_endorsement::ChunkEndorsement; use near_primitives::stateless_validation::validator_assignment::ChunkEndorsementsState; use near_primitives::stateless_validation::ChunkProductionKey; use near_primitives::types::AccountId; use near_primitives::version::ProtocolFeature; use near_store::Store; -use crate::stateless_validation::validate::validate_chunk_endorsement; +use super::validate::validate_chunk_endorsement; // This is the number of unique chunks for which we would track the chunk endorsements. // Ideally, we should not be processing more than num_shards chunks at a time. @@ -25,12 +26,11 @@ pub struct ChunkEndorsementTracker { /// Used to find the chain HEAD when validating partial witnesses. store: Store, /// We store the validated chunk endorsements received from chunk validators. - /// Interior mutability is required to update the cache in process_chunk_endorsement & compute_chunk_endorsements. - chunk_endorsements: LruCache>, + chunk_endorsements: LruCache>, } impl ChunkEndorsementTracker { - pub(crate) fn new(epoch_manager: Arc, store: Store) -> Self { + pub fn new(epoch_manager: Arc, store: Store) -> Self { Self { epoch_manager, store, @@ -41,9 +41,9 @@ impl ChunkEndorsementTracker { } // Validate the chunk endorsement and store it in the cache. - pub(crate) fn process_chunk_endorsement( + pub fn process_chunk_endorsement( &mut self, - endorsement: ChunkEndorsementV2, + endorsement: ChunkEndorsement, ) -> Result<(), Error> { // Check if we have already received chunk endorsement from this validator. let key = endorsement.chunk_production_key(); @@ -57,13 +57,13 @@ impl ChunkEndorsementTracker { if validate_chunk_endorsement(self.epoch_manager.as_ref(), &endorsement, &self.store)? { self.chunk_endorsements .get_or_insert_mut(key, || HashMap::new()) - .insert(account_id.clone(), endorsement); + .insert(account_id.clone(), (endorsement.chunk_hash(), endorsement.signature())); }; Ok(()) } /// This function is called by block producer potentially multiple times if there's not enough stake. - pub(crate) fn collect_chunk_endorsements( + pub fn collect_chunk_endorsements( &mut self, chunk_header: &ShardChunkHeader, ) -> Result { @@ -97,8 +97,8 @@ impl ChunkEndorsementTracker { let entry = self.chunk_endorsements.get_or_insert(key, || HashMap::new()); let validator_signatures = entry .into_iter() - .filter(|(_, endorsement)| endorsement.chunk_hash() == &chunk_header.chunk_hash()) - .map(|(account_id, endorsement)| (account_id, endorsement.signature())) + .filter(|(_, (chunk_hash, _))| chunk_hash == &chunk_header.chunk_hash()) + .map(|(account_id, (_, signature))| (account_id, signature.clone())) .collect(); Ok(chunk_validator_assignments.compute_endorsement_state(validator_signatures)) diff --git a/chain/client/src/stateless_validation/chunk_endorsement/mod.rs b/chain/client/src/stateless_validation/chunk_endorsement/mod.rs deleted file mode 100644 index 3a1c8352450..00000000000 --- a/chain/client/src/stateless_validation/chunk_endorsement/mod.rs +++ /dev/null @@ -1,81 +0,0 @@ -use std::sync::Arc; - -use near_chain::ChainStoreAccess; -use near_chain_primitives::Error; -use near_epoch_manager::EpochManagerAdapter; -use near_primitives::sharding::ShardChunkHeader; -use near_primitives::stateless_validation::chunk_endorsement::ChunkEndorsement; -use near_primitives::stateless_validation::validator_assignment::ChunkEndorsementsState; -use near_primitives::version::ProtocolFeature; -use near_store::Store; - -use crate::Client; - -mod tracker_v1; -mod tracker_v2; - -/// Module to track chunk endorsements received from chunk validators. -pub struct ChunkEndorsementTracker { - epoch_manager: Arc, - pub tracker_v1: tracker_v1::ChunkEndorsementTracker, - pub tracker_v2: tracker_v2::ChunkEndorsementTracker, -} - -impl Client { - pub fn process_chunk_endorsement( - &mut self, - endorsement: ChunkEndorsement, - ) -> Result<(), Error> { - // TODO(ChunkEndorsementV2): Remove chunk_header once tracker_v1 is deprecated - let chunk_hash = match &endorsement { - ChunkEndorsement::V1(endorsement) => endorsement.chunk_hash(), - ChunkEndorsement::V2(endorsement) => endorsement.chunk_hash(), - }; - let chunk_header = match self.chain.chain_store().get_partial_chunk(chunk_hash) { - Ok(chunk) => Some(chunk.cloned_header()), - Err(Error::ChunkMissing(_)) => None, - Err(error) => return Err(error), - }; - self.chunk_endorsement_tracker.process_chunk_endorsement(endorsement, chunk_header) - } -} - -impl ChunkEndorsementTracker { - pub fn new(epoch_manager: Arc, store: Store) -> Self { - Self { - tracker_v1: tracker_v1::ChunkEndorsementTracker::new(epoch_manager.clone()), - tracker_v2: tracker_v2::ChunkEndorsementTracker::new(epoch_manager.clone(), store), - epoch_manager, - } - } - - // TODO(ChunkEndorsementV2): Remove chunk_header once tracker_v1 is deprecated - pub fn process_chunk_endorsement( - &mut self, - endorsement: ChunkEndorsement, - chunk_header: Option, - ) -> Result<(), Error> { - match endorsement { - ChunkEndorsement::V1(endorsement) => { - self.tracker_v1.process_chunk_endorsement(endorsement, chunk_header) - } - ChunkEndorsement::V2(endorsement) => { - self.tracker_v2.process_chunk_endorsement(endorsement) - } - } - } - - pub fn collect_chunk_endorsements( - &mut self, - chunk_header: &ShardChunkHeader, - ) -> Result { - let epoch_id = - self.epoch_manager.get_epoch_id_from_prev_block(chunk_header.prev_block_hash())?; - let protocol_version = self.epoch_manager.get_epoch_protocol_version(&epoch_id)?; - if !ProtocolFeature::ChunkEndorsementV2.enabled(protocol_version) { - self.tracker_v1.collect_chunk_endorsements(chunk_header) - } else { - self.tracker_v2.collect_chunk_endorsements(chunk_header) - } - } -} diff --git a/chain/client/src/stateless_validation/chunk_endorsement/tracker_v1.rs b/chain/client/src/stateless_validation/chunk_endorsement/tracker_v1.rs deleted file mode 100644 index 44b2954c147..00000000000 --- a/chain/client/src/stateless_validation/chunk_endorsement/tracker_v1.rs +++ /dev/null @@ -1,236 +0,0 @@ -use lru::LruCache; -use near_primitives::stateless_validation::chunk_endorsement::ChunkEndorsementV1; -use near_primitives::stateless_validation::validator_assignment::ChunkEndorsementsState; -use std::collections::HashMap; -use std::num::NonZeroUsize; -use std::sync::{Arc, Mutex}; - -use near_chain_primitives::Error; -use near_epoch_manager::EpochManagerAdapter; -use near_primitives::checked_feature; -use near_primitives::sharding::{ChunkHash, ShardChunkHeader}; -use near_primitives::types::AccountId; - -// This is the number of unique chunks for which we would track the chunk endorsements. -// Ideally, we should not be processing more than num_shards chunks at a time. -const NUM_CHUNKS_IN_CHUNK_ENDORSEMENTS_CACHE: usize = 100; - -/// Module to track chunk endorsements received from chunk validators. -pub struct ChunkEndorsementTracker { - epoch_manager: Arc, - inner: Mutex, -} - -struct ChunkEndorsementTrackerInner { - epoch_manager: Arc, - /// We store the validated chunk endorsements received from chunk validators - /// This is keyed on chunk_hash and account_id of validator to avoid duplicates. - /// Chunk endorsements would later be used as a part of block production. - chunk_endorsements: - LruCache)>, - /// We store chunk endorsements to be processed later because we did not have - /// chunks ready at the time we received that endorsements from validators. - /// This is keyed on chunk_hash and account_id of validator to avoid duplicates. - pending_chunk_endorsements: LruCache>, -} - -impl ChunkEndorsementTracker { - pub(crate) fn process_chunk_endorsement( - &self, - endorsement: ChunkEndorsementV1, - chunk_header: Option, - ) -> Result<(), Error> { - // We need the chunk header in order to process the chunk endorsement. - // If we don't have the header, then queue it up for when we do have the header. - // We must use the partial chunk (as opposed to the full chunk) in order to get - // the chunk header, because we may not be tracking that shard. - match chunk_header { - Some(chunk_header) => { - self.process_chunk_endorsement_with_chunk_header(&chunk_header, endorsement) - } - None => self.add_chunk_endorsement_to_pending_cache(endorsement), - } - } - - pub fn new(epoch_manager: Arc) -> Self { - Self { - epoch_manager: epoch_manager.clone(), - inner: Mutex::new(ChunkEndorsementTrackerInner { - epoch_manager, - chunk_endorsements: LruCache::new( - NonZeroUsize::new(NUM_CHUNKS_IN_CHUNK_ENDORSEMENTS_CACHE).unwrap(), - ), - // We can use a different cache size if needed, it does not have to be the same as for `chunk_endorsements`. - pending_chunk_endorsements: LruCache::new( - NonZeroUsize::new(NUM_CHUNKS_IN_CHUNK_ENDORSEMENTS_CACHE).unwrap(), - ), - }), - } - } - - /// Process pending endorsements for the given chunk header. - /// It removes these endorsements from the `pending_chunk_endorsements` cache. - pub fn process_pending_endorsements(&self, chunk_header: &ShardChunkHeader) { - self.inner.lock().unwrap().process_pending_endorsements(chunk_header); - } - - /// Add the chunk endorsement to a cache of pending chunk endorsements (if not yet there). - fn add_chunk_endorsement_to_pending_cache( - &self, - endorsement: ChunkEndorsementV1, - ) -> Result<(), Error> { - self.inner.lock().unwrap().process_chunk_endorsement_impl(endorsement, None, false) - } - - /// Function to process an incoming chunk endorsement from chunk validators. - /// We first verify the chunk endorsement and then store it in a cache. - /// We would later include the endorsements in the block production. - fn process_chunk_endorsement_with_chunk_header( - &self, - chunk_header: &ShardChunkHeader, - endorsement: ChunkEndorsementV1, - ) -> Result<(), Error> { - let _span = tracing::debug_span!(target: "client", "process_chunk_endorsement", chunk_hash=?chunk_header.chunk_hash(), shard_id=?chunk_header.shard_id()).entered(); - // Validate the endorsement before locking the mutex to improve performance. - if !self.epoch_manager.verify_chunk_endorsement(&chunk_header, &endorsement)? { - tracing::error!(target: "client", ?endorsement, "Invalid chunk endorsement."); - return Err(Error::InvalidChunkEndorsement); - } - self.inner.lock().unwrap().process_chunk_endorsement_impl( - endorsement, - Some(chunk_header), - true, - ) - } - - /// This function is called by block producer potentially multiple times if there's not enough stake. - /// For older protocol version, we return an empty array of chunk endorsements. - pub fn collect_chunk_endorsements( - &self, - chunk_header: &ShardChunkHeader, - ) -> Result { - self.inner.lock().unwrap().compute_chunk_endorsements_impl(chunk_header) - } -} - -impl ChunkEndorsementTrackerInner { - /// Process pending endorsements for the given chunk header. - /// It removes these endorsements from the `pending_chunk_endorsements` cache. - pub fn process_pending_endorsements(&mut self, chunk_header: &ShardChunkHeader) { - let chunk_hash = &chunk_header.chunk_hash(); - let chunk_endorsements = self.pending_chunk_endorsements.pop(chunk_hash); - let Some(chunk_endorsements) = chunk_endorsements else { - return; - }; - tracing::debug!(target: "client", ?chunk_hash, "Processing pending chunk endorsements."); - for endorsement in chunk_endorsements.values() { - if let Err(error) = - self.process_chunk_endorsement_impl(endorsement.clone(), Some(chunk_header), false) - { - tracing::debug!(target: "client", ?endorsement, ?error, "Error processing pending chunk endorsement"); - } - } - } - - /// If the chunk header is available, we will verify the chunk endorsement and then store it in a cache. - /// Otherwise, we store the endorsement in a separate cache of endorsements to be processed when the chunk is ready. - fn process_chunk_endorsement_impl( - &mut self, - endorsement: ChunkEndorsementV1, - chunk_header: Option<&ShardChunkHeader>, - already_validated: bool, - ) -> Result<(), Error> { - let chunk_hash = endorsement.chunk_hash(); - let account_id = &endorsement.account_id; - - let existing_entry = self.chunk_endorsements.peek(chunk_hash); - - // If we have already processed this chunk endorsement, return early. - if existing_entry.is_some_and(|(_, existing_endorsements)| { - existing_endorsements.contains_key(account_id) - }) { - tracing::debug!(target: "client", ?endorsement, "Already received chunk endorsement."); - return Ok(()); - } - - // If we are the current block producer, we store the chunk endorsement for each chunk which - // would later be used during block production to check whether to include the chunk or not. - // TODO(stateless_validation): It's possible for a malicious validator to send endorsements - // for 100 unique chunks thus pushing out current valid endorsements from our cache. - // Maybe add check to ensure we don't accept endorsements from chunks already included in some block? - // Maybe add check to ensure we don't accept endorsements from chunks that have too old height_created? - tracing::debug!(target: "client", ?endorsement, "Received and saved chunk endorsement."); - - // The header might be available in the endorsement cache, even if it isn't provided. - // In such case it should be treated as a non-pending endorsement. - let header = chunk_header.or_else(|| existing_entry.map(|(header, _)| header)); - - if let Some(chunk_header) = header { - if !already_validated - && !self.epoch_manager.verify_chunk_endorsement(&chunk_header, &endorsement)? - { - tracing::error!(target: "client", ?endorsement, "Invalid chunk endorsement."); - return Err(Error::InvalidChunkEndorsement); - } - - if self.chunk_endorsements.peek(chunk_hash).is_none() { - self.chunk_endorsements - .put(chunk_hash.clone(), (chunk_header.clone(), HashMap::new())); - } - self.chunk_endorsements - .get_mut(chunk_hash) - .unwrap() - .1 - .insert(account_id.clone(), endorsement); - } else { - // Chunk header is not available, store the endorsement in the pending cache. - self.pending_chunk_endorsements.get_or_insert(chunk_hash.clone(), || HashMap::new()); - self.pending_chunk_endorsements - .get_mut(chunk_hash) - .unwrap() - .insert(account_id.clone(), endorsement); - } - - Ok(()) - } - - pub fn compute_chunk_endorsements_impl( - &mut self, - chunk_header: &ShardChunkHeader, - ) -> Result { - let epoch_id = - self.epoch_manager.get_epoch_id_from_prev_block(chunk_header.prev_block_hash())?; - let protocol_version = self.epoch_manager.get_epoch_protocol_version(&epoch_id)?; - if !checked_feature!("stable", StatelessValidation, protocol_version) { - // Return an empty array of chunk endorsements for older protocol versions. - return Ok(ChunkEndorsementsState { - is_endorsed: true, - ..ChunkEndorsementsState::default() - }); - } - - let chunk_validator_assignments = self.epoch_manager.get_chunk_validator_assignments( - &epoch_id, - chunk_header.shard_id(), - chunk_header.height_created(), - )?; - // Get the chunk_endorsements for the chunk from our cache. - // Note that these chunk endorsements are already validated as part of process_chunk_endorsement. - // We can safely rely on the following details - // 1. The chunk endorsements are from valid chunk_validator for this chunk. - // 2. The chunk endorsements signatures are valid. - let Some((_header, chunk_endorsements)) = - self.chunk_endorsements.get(&chunk_header.chunk_hash()) - else { - // Early return if no chunk_endorsements found in our cache. - return Ok(ChunkEndorsementsState::default()); - }; - - let validator_signatures = chunk_endorsements - .into_iter() - .map(|(account_id, endorsement)| (account_id, endorsement.signature.clone())) - .collect(); - - Ok(chunk_validator_assignments.compute_endorsement_state(validator_signatures)) - } -} diff --git a/chain/client/src/stateless_validation/chunk_validator/mod.rs b/chain/client/src/stateless_validation/chunk_validator/mod.rs index c55db293c93..28a84577806 100644 --- a/chain/client/src/stateless_validation/chunk_validator/mod.rs +++ b/chain/client/src/stateless_validation/chunk_validator/mod.rs @@ -220,8 +220,7 @@ pub(crate) fn send_chunk_endorsement_to_block_producers( "send_chunk_endorsement", ); - let protocol_version = epoch_manager.get_epoch_protocol_version(&epoch_id).unwrap(); - let endorsement = ChunkEndorsement::new(epoch_id, chunk_header, signer, protocol_version); + let endorsement = ChunkEndorsement::new(epoch_id, chunk_header, signer); for block_producer in block_producers { network_sender.send(PeerManagerMessageRequest::NetworkRequests( NetworkRequests::ChunkEndorsement(block_producer, endorsement.clone()), diff --git a/chain/client/src/stateless_validation/validate.rs b/chain/client/src/stateless_validation/validate.rs index 33f0de1b650..e4c72aa3ef9 100644 --- a/chain/client/src/stateless_validation/validate.rs +++ b/chain/client/src/stateless_validation/validate.rs @@ -3,7 +3,7 @@ use itertools::Itertools; use near_chain::types::Tip; use near_chain_primitives::Error; use near_epoch_manager::EpochManagerAdapter; -use near_primitives::stateless_validation::chunk_endorsement::ChunkEndorsementV2; +use near_primitives::stateless_validation::chunk_endorsement::ChunkEndorsement; use near_primitives::stateless_validation::contract_distribution::{ ChunkContractAccesses, PartialEncodedContractDeploys, }; @@ -83,7 +83,7 @@ pub fn validate_partial_encoded_contract_deploys( /// - signature of endorsement and metadata is valid pub fn validate_chunk_endorsement( epoch_manager: &dyn EpochManagerAdapter, - endorsement: &ChunkEndorsementV2, + endorsement: &ChunkEndorsement, store: &Store, ) -> Result { if !validate_chunk_production_key( diff --git a/chain/client/src/test_utils/client.rs b/chain/client/src/test_utils/client.rs index 72555c486f1..09d03b0ce7b 100644 --- a/chain/client/src/test_utils/client.rs +++ b/chain/client/src/test_utils/client.rs @@ -19,9 +19,9 @@ use near_primitives::block::Block; use near_primitives::hash::CryptoHash; use near_primitives::merkle::{merklize, PartialMerkleTree}; use near_primitives::sharding::{EncodedShardChunk, ShardChunk}; -use near_primitives::stateless_validation::chunk_endorsement::ChunkEndorsementV1; +use near_primitives::stateless_validation::chunk_endorsement::ChunkEndorsement; use near_primitives::transaction::SignedTransaction; -use near_primitives::types::{BlockHeight, ShardId}; +use near_primitives::types::{BlockHeight, EpochId, ShardId}; use near_primitives::utils::MaybeValidated; use near_primitives::version::PROTOCOL_VERSION; use num_rational::Ratio; @@ -248,7 +248,8 @@ pub fn create_chunk( let mut block_merkle_tree = PartialMerkleTree::clone(&block_merkle_tree); let signer = client.validator_signer.get().unwrap(); - let endorsement = ChunkEndorsementV1::new(chunk.cloned_header().chunk_hash(), signer.as_ref()); + let endorsement = + ChunkEndorsement::new(EpochId::default(), &chunk.cloned_header(), signer.as_ref()); block_merkle_tree.insert(*last_block.hash()); let block = Block::produce( PROTOCOL_VERSION, @@ -257,7 +258,7 @@ pub fn create_chunk( next_height, last_block.header().block_ordinal() + 1, vec![chunk.cloned_header()], - vec![vec![Some(Box::new(endorsement.signature))]], + vec![vec![Some(Box::new(endorsement.signature()))]], *last_block.header().epoch_id(), *last_block.header().next_epoch_id(), None, diff --git a/chain/client/src/test_utils/test_env.rs b/chain/client/src/test_utils/test_env.rs index dfbb062b1aa..6e9ea195acb 100644 --- a/chain/client/src/test_utils/test_env.rs +++ b/chain/client/src/test_utils/test_env.rs @@ -27,7 +27,6 @@ use near_primitives::epoch_info::RngSeed; use near_primitives::errors::InvalidTxError; use near_primitives::hash::CryptoHash; use near_primitives::sharding::{ChunkHash, PartialEncodedChunk}; -use near_primitives::stateless_validation::chunk_endorsement::ChunkEndorsement; use near_primitives::stateless_validation::state_witness::ChunkStateWitness; use near_primitives::test_utils::create_test_signer; use near_primitives::transaction::{Action, FunctionCallAction, SignedTransaction}; @@ -310,6 +309,7 @@ impl TestEnv { chunk_producer, } => { self.clients[id] + .chunk_inclusion_tracker .mark_chunk_header_ready_for_inclusion(chunk_header, chunk_producer); } } @@ -412,8 +412,10 @@ impl TestEnv { account_id, endorsement, )) => { - let processing_result = - self.client(&account_id).process_chunk_endorsement(endorsement); + let processing_result = self + .client(&account_id) + .chunk_endorsement_tracker + .process_chunk_endorsement(endorsement); if !allow_errors { processing_result.unwrap(); } @@ -447,11 +449,7 @@ impl TestEnv { PeerManagerMessageRequest::NetworkRequests( NetworkRequests::ChunkEndorsement(_, endorsement), ) => { - let endorsement_chunk_hash = match endorsement { - ChunkEndorsement::V1(endorsement) => endorsement.chunk_hash(), - ChunkEndorsement::V2(endorsement) => endorsement.chunk_hash(), - }; - endorsement_found = endorsement_chunk_hash == chunk_hash; + endorsement_found = endorsement.chunk_hash() == *chunk_hash; } _ => {} }; diff --git a/chain/epoch-manager/src/adapter.rs b/chain/epoch-manager/src/adapter.rs index 9cce457eb1b..7eb20a0b83d 100644 --- a/chain/epoch-manager/src/adapter.rs +++ b/chain/epoch-manager/src/adapter.rs @@ -10,9 +10,7 @@ use near_primitives::errors::EpochError; use near_primitives::hash::CryptoHash; use near_primitives::shard_layout::{account_id_to_shard_id, ShardLayout}; use near_primitives::sharding::{ChunkHash, ShardChunkHeader}; -use near_primitives::stateless_validation::chunk_endorsement::{ - ChunkEndorsementV1, ChunkEndorsementV2, -}; +use near_primitives::stateless_validation::chunk_endorsement::ChunkEndorsement; use near_primitives::stateless_validation::contract_distribution::ChunkContractAccesses; use near_primitives::stateless_validation::partial_witness::PartialEncodedStateWitness; use near_primitives::stateless_validation::validator_assignment::ChunkValidatorAssignments; @@ -477,15 +475,9 @@ pub trait EpochManagerAdapter: Send + Sync { approvals: &[Option>], ) -> Result<(), Error>; - fn verify_chunk_endorsement( - &self, - chunk_header: &ShardChunkHeader, - endorsement: &ChunkEndorsementV1, - ) -> Result; - fn verify_chunk_endorsement_signature( &self, - endorsement: &ChunkEndorsementV2, + endorsement: &ChunkEndorsement, ) -> Result; fn verify_partial_witness_signature( @@ -1160,36 +1152,9 @@ impl EpochManagerAdapter for EpochManagerHandle { } } - // TODO(ChunkEndorsementV2): Deprecate this after shifting to ChunkEndorsementV2 - fn verify_chunk_endorsement( - &self, - chunk_header: &ShardChunkHeader, - endorsement: &ChunkEndorsementV1, - ) -> Result { - if &chunk_header.chunk_hash() != endorsement.chunk_hash() { - return Err(Error::InvalidChunkEndorsement); - } - let epoch_manager = self.read(); - let epoch_id = - epoch_manager.get_epoch_id_from_prev_block(chunk_header.prev_block_hash())?; - // Note that we are using the chunk_header.height_created param here to determine the chunk validators - // This only works when height created for a chunk is the same as the height_included during block production - let chunk_validator_assignments = epoch_manager.get_chunk_validator_assignments( - &epoch_id, - chunk_header.shard_id(), - chunk_header.height_created(), - )?; - if !chunk_validator_assignments.contains(&endorsement.account_id) { - return Err(Error::NotAValidator(format!("verify chunk endorsement"))); - } - let validator = - epoch_manager.get_validator_by_account_id(&epoch_id, &endorsement.account_id)?; - Ok(endorsement.verify(validator.public_key())) - } - fn verify_chunk_endorsement_signature( &self, - endorsement: &ChunkEndorsementV2, + endorsement: &ChunkEndorsement, ) -> Result { let epoch_manager = self.read(); let epoch_id = endorsement.chunk_production_key().epoch_id; diff --git a/chain/epoch-manager/src/tests/mod.rs b/chain/epoch-manager/src/tests/mod.rs index 1ba1a20fdf7..f9153f1c001 100644 --- a/chain/epoch-manager/src/tests/mod.rs +++ b/chain/epoch-manager/src/tests/mod.rs @@ -21,7 +21,6 @@ use near_primitives::epoch_manager::EpochConfig; use near_primitives::hash::hash; use near_primitives::shard_layout::ShardLayout; use near_primitives::sharding::{ShardChunkHeader, ShardChunkHeaderV3}; -use near_primitives::stateless_validation::chunk_endorsement::ChunkEndorsementV1; use near_primitives::stateless_validation::chunk_endorsements_bitmap::ChunkEndorsementsBitmap; use near_primitives::stateless_validation::partial_witness::PartialEncodedStateWitness; use near_primitives::types::ShardIndex; @@ -3325,69 +3324,6 @@ fn test_chunk_header(h: &[CryptoHash], signer: &ValidatorSigner) -> ShardChunkHe )) } -#[test] -fn test_verify_chunk_endorsements() { - use near_chain_primitives::Error; - use near_crypto::Signature; - use near_primitives::test_utils::create_test_signer; - use std::str::FromStr; - - let amount_staked = 1_000_000; - let account_id = AccountId::from_str("test1").unwrap(); - let validators = vec![(account_id.clone(), amount_staked)]; - let h = hash_range(6); - - let mut epoch_manager = setup_default_epoch_manager(validators, 5, 1, 2, 90, 60); - record_block(&mut epoch_manager, CryptoHash::default(), h[0], 0, vec![]); - record_block(&mut epoch_manager, h[0], h[1], 1, vec![]); - - // build a chunk endorsement and chunk header - let epoch_manager = epoch_manager.into_handle(); - let epoch_id = epoch_manager.get_epoch_id(&h[1]).unwrap(); - - // verify if we have one chunk validator - let chunk_validator_assignments = - &epoch_manager.get_chunk_validator_assignments(&epoch_id, ShardId::new(0), 1).unwrap(); - assert_eq!(chunk_validator_assignments.ordered_chunk_validators().len(), 1); - assert!(chunk_validator_assignments.contains(&account_id)); - - // verify if the test signer has same public key as the chunk validator - let (validator, _) = - epoch_manager.get_validator_by_account_id(&epoch_id, &h[0], &account_id).unwrap(); - let signer = Arc::new(create_test_signer("test1")); - assert_eq!(signer.public_key(), validator.public_key().clone()); - - // make chunk header - let chunk_header = test_chunk_header(&h, signer.as_ref()); - - // check chunk endorsement validity - let mut chunk_endorsement = ChunkEndorsementV1::new(chunk_header.chunk_hash(), signer.as_ref()); - assert!(epoch_manager.verify_chunk_endorsement(&chunk_header, &chunk_endorsement).unwrap()); - - // check invalid chunk endorsement signature - chunk_endorsement.signature = Signature::default(); - assert!(!epoch_manager.verify_chunk_endorsement(&chunk_header, &chunk_endorsement).unwrap()); - - // check chunk endorsement invalidity when chunk header and chunk endorsement don't match - let chunk_endorsement = ChunkEndorsementV1::new(h[3].into(), signer.as_ref()); - let err = - epoch_manager.verify_chunk_endorsement(&chunk_header, &chunk_endorsement).unwrap_err(); - match err { - Error::InvalidChunkEndorsement => (), - _ => assert!(false, "Expected InvalidChunkEndorsement error but got {:?}", err), - } - - // check chunk endorsement invalidity when signer is not chunk validator - let bad_signer = Arc::new(create_test_signer("test2")); - let chunk_endorsement = ChunkEndorsementV1::new(chunk_header.chunk_hash(), bad_signer.as_ref()); - let err = - epoch_manager.verify_chunk_endorsement(&chunk_header, &chunk_endorsement).unwrap_err(); - match err { - Error::NotAValidator(_) => (), - _ => assert!(false, "Expected NotAValidator error but got {:?}", err), - } -} - #[test] fn test_verify_partial_witness_signature() { use near_crypto::Signature; diff --git a/chain/network/src/network_protocol/mod.rs b/chain/network/src/network_protocol/mod.rs index a4efff4837d..17656c4a22d 100644 --- a/chain/network/src/network_protocol/mod.rs +++ b/chain/network/src/network_protocol/mod.rs @@ -8,7 +8,6 @@ mod proto_conv; mod state_sync; pub use edge::*; use near_primitives::stateless_validation::chunk_endorsement::ChunkEndorsement; -use near_primitives::stateless_validation::chunk_endorsement::ChunkEndorsementV1; use near_primitives::stateless_validation::contract_distribution::ChunkContractAccesses; use near_primitives::stateless_validation::contract_distribution::ContractCodeRequest; use near_primitives::stateless_validation::contract_distribution::ContractCodeResponse; @@ -552,8 +551,7 @@ pub enum RoutedMessageBody { _UnusedVersionedStateResponse, PartialEncodedChunkForward(PartialEncodedChunkForwardMsg), _UnusedChunkStateWitness, - /// TODO(ChunkEndorsementV2): Deprecate once we move to VersionedChunkEndorsement - ChunkEndorsement(ChunkEndorsementV1), + _UnusedChunkEndorsement, ChunkStateWitnessAck(ChunkStateWitnessAck), PartialEncodedStateWitness(PartialEncodedStateWitness), PartialEncodedStateWitnessForward(PartialEncodedStateWitness), @@ -588,8 +586,7 @@ impl RoutedMessageBody { // we may be the block_producer. pub fn allow_sending_to_self(&self) -> bool { match self { - RoutedMessageBody::ChunkEndorsement(_) - | RoutedMessageBody::PartialEncodedStateWitness(_) + RoutedMessageBody::PartialEncodedStateWitness(_) | RoutedMessageBody::PartialEncodedStateWitnessForward(_) | RoutedMessageBody::VersionedChunkEndorsement(_) => true, _ => false, @@ -642,7 +639,7 @@ impl fmt::Debug for RoutedMessageBody { RoutedMessageBody::Pong(_) => write!(f, "Pong"), RoutedMessageBody::_UnusedVersionedStateResponse => write!(f, "VersionedStateResponse"), RoutedMessageBody::_UnusedChunkStateWitness => write!(f, "ChunkStateWitness"), - RoutedMessageBody::ChunkEndorsement(_) => write!(f, "ChunkEndorsement"), + RoutedMessageBody::_UnusedChunkEndorsement => write!(f, "ChunkEndorsement"), RoutedMessageBody::ChunkStateWitnessAck(ack, ..) => { f.debug_tuple("ChunkStateWitnessAck").field(&ack.chunk_hash).finish() } diff --git a/chain/network/src/peer_manager/connection/mod.rs b/chain/network/src/peer_manager/connection/mod.rs index 0bd83fc7d4f..0f9d16376fe 100644 --- a/chain/network/src/peer_manager/connection/mod.rs +++ b/chain/network/src/peer_manager/connection/mod.rs @@ -50,7 +50,6 @@ impl tcp::Tier { pub(crate) fn is_allowed_routed(self, body: &RoutedMessageBody) -> bool { match body { RoutedMessageBody::BlockApproval(..) - | RoutedMessageBody::ChunkEndorsement(..) | RoutedMessageBody::PartialEncodedStateWitness(..) | RoutedMessageBody::PartialEncodedStateWitnessForward(..) | RoutedMessageBody::VersionedPartialEncodedChunk(..) diff --git a/chain/network/src/peer_manager/network_state/mod.rs b/chain/network/src/peer_manager/network_state/mod.rs index e8be8b7565f..c024023a92f 100644 --- a/chain/network/src/peer_manager/network_state/mod.rs +++ b/chain/network/src/peer_manager/network_state/mod.rs @@ -40,7 +40,6 @@ use near_async::time; use near_primitives::block::GenesisId; use near_primitives::hash::CryptoHash; use near_primitives::network::PeerId; -use near_primitives::stateless_validation::chunk_endorsement::ChunkEndorsement; use near_primitives::types::AccountId; use parking_lot::{Mutex, RwLock}; use std::collections::VecDeque; @@ -762,11 +761,6 @@ impl NetworkState { self.partial_witness_adapter.send(ChunkStateWitnessAckMessage(ack)); None } - RoutedMessageBody::ChunkEndorsement(endorsement) => { - let endorsement = ChunkEndorsement::V1(endorsement); - self.client.send_async(ChunkEndorsementMessage(endorsement)).await.ok(); - None - } RoutedMessageBody::PartialEncodedStateWitness(witness) => { self.partial_witness_adapter.send(PartialEncodedStateWitnessMessage(witness)); None diff --git a/chain/network/src/peer_manager/peer_manager_actor.rs b/chain/network/src/peer_manager/peer_manager_actor.rs index 27745a0b88a..b31695ff688 100644 --- a/chain/network/src/peer_manager/peer_manager_actor.rs +++ b/chain/network/src/peer_manager/peer_manager_actor.rs @@ -31,7 +31,6 @@ use near_o11y::{handler_debug_span, handler_trace_span, WithSpanContext}; use near_performance_metrics_macros::perf; use near_primitives::block::GenesisId; use near_primitives::network::{AnnounceAccount, PeerId}; -use near_primitives::stateless_validation::chunk_endorsement::ChunkEndorsement; use near_primitives::views::{ ConnectionInfoView, EdgeView, KnownPeerStateView, NetworkGraphView, PeerStoreView, RecentOutboundConnectionsView, SnapshotHostInfoView, SnapshotHostsView, @@ -1119,13 +1118,11 @@ impl PeerManagerActor { NetworkResponses::NoResponse } NetworkRequests::ChunkEndorsement(target, endorsement) => { - let msg = match endorsement { - ChunkEndorsement::V1(endorsement) => { - RoutedMessageBody::ChunkEndorsement(endorsement) - } - _ => RoutedMessageBody::VersionedChunkEndorsement(endorsement), - }; - self.state.send_message_to_account(&self.clock, &target, msg); + self.state.send_message_to_account( + &self.clock, + &target, + RoutedMessageBody::VersionedChunkEndorsement(endorsement), + ); NetworkResponses::NoResponse } NetworkRequests::PartialEncodedStateWitness(validator_witness_tuple) => { diff --git a/chain/network/src/rate_limits/messages_limits.rs b/chain/network/src/rate_limits/messages_limits.rs index feef78b85cd..172cd47e214 100644 --- a/chain/network/src/rate_limits/messages_limits.rs +++ b/chain/network/src/rate_limits/messages_limits.rs @@ -224,7 +224,6 @@ fn get_key_and_token_cost(message: &PeerMessage) -> Option<(RateLimitedPeerMessa RoutedMessageBody::PartialEncodedChunkForward(_) => { Some((PartialEncodedChunkForward, 1)) } - RoutedMessageBody::ChunkEndorsement(_) => Some((ChunkEndorsement, 1)), RoutedMessageBody::ChunkStateWitnessAck(_) => Some((ChunkStateWitnessAck, 1)), RoutedMessageBody::PartialEncodedStateWitness(_) => { Some((PartialEncodedStateWitness, 1)) @@ -244,6 +243,7 @@ fn get_key_and_token_cost(message: &PeerMessage) -> Option<(RateLimitedPeerMessa RoutedMessageBody::StatePartRequest(_) => None, // TODO RoutedMessageBody::Ping(_) | RoutedMessageBody::Pong(_) + | RoutedMessageBody::_UnusedChunkEndorsement | RoutedMessageBody::_UnusedChunkStateWitness | RoutedMessageBody::_UnusedVersionedStateResponse | RoutedMessageBody::_UnusedPartialEncodedChunk diff --git a/core/primitives/src/stateless_validation/chunk_endorsement.rs b/core/primitives/src/stateless_validation/chunk_endorsement.rs index 9c782e5cf29..6f1c38260b7 100644 --- a/core/primitives/src/stateless_validation/chunk_endorsement.rs +++ b/core/primitives/src/stateless_validation/chunk_endorsement.rs @@ -5,17 +5,17 @@ use crate::types::EpochId; use crate::validator_signer::ValidatorSigner; use borsh::{BorshDeserialize, BorshSerialize}; use near_crypto::{PublicKey, Signature}; -use near_primitives_core::types::{AccountId, BlockHeight, ProtocolVersion, ShardId}; -use near_primitives_core::version::ProtocolFeature; +use near_primitives_core::types::{AccountId, BlockHeight, ShardId}; use near_schema_checker_lib::ProtocolSchema; use super::{ChunkProductionKey, SignatureDifferentiator}; /// The endorsement of a chunk by a chunk validator. By providing this, a /// chunk validator has verified that the chunk state witness is correct. +#[allow(clippy::large_enum_variant)] #[derive(Debug, Clone, PartialEq, Eq, BorshSerialize, BorshDeserialize, ProtocolSchema)] pub enum ChunkEndorsement { - V1(ChunkEndorsementV1), + V1, // Deprecated V2(ChunkEndorsementV2), } @@ -24,63 +24,75 @@ impl ChunkEndorsement { epoch_id: EpochId, chunk_header: &ShardChunkHeader, signer: &ValidatorSigner, - protocol_version: ProtocolVersion, ) -> ChunkEndorsement { - if ProtocolFeature::ChunkEndorsementV2.enabled(protocol_version) { - ChunkEndorsement::V2(ChunkEndorsementV2::new(epoch_id, chunk_header, signer)) - } else { - ChunkEndorsement::V1(ChunkEndorsementV1::new(chunk_header.chunk_hash(), signer)) - } + let inner = ChunkEndorsementInner::new(chunk_header.chunk_hash()); + let metadata = ChunkEndorsementMetadata { + account_id: signer.validator_id().clone(), + shard_id: chunk_header.shard_id(), + epoch_id, + height_created: chunk_header.height_created(), + }; + let signature = signer.sign_chunk_endorsement(&inner); + let metadata_signature = signer.sign_chunk_endorsement_metadata(&metadata); + let endorsement = ChunkEndorsementV2 { inner, signature, metadata, metadata_signature }; + ChunkEndorsement::V2(endorsement) } - pub fn validate_signature( - chunk_hash: ChunkHash, - signature: &Signature, - public_key: &PublicKey, - ) -> bool { - let inner = ChunkEndorsementInner::new(chunk_hash); - let data = borsh::to_vec(&inner).unwrap(); - signature.verify(&data, public_key) + pub fn chunk_production_key(&self) -> ChunkProductionKey { + match self { + ChunkEndorsement::V1 => unreachable!("V1 chunk endorsement is deprecated"), + ChunkEndorsement::V2(v2) => ChunkProductionKey { + shard_id: v2.metadata.shard_id, + epoch_id: v2.metadata.epoch_id, + height_created: v2.metadata.height_created, + }, + } } - /// Returns the account ID of the chunk validator that generated this endorsement. - pub fn validator_account(&self) -> &AccountId { + pub fn account_id(&self) -> &AccountId { match self { - ChunkEndorsement::V1(v1) => &v1.account_id, + ChunkEndorsement::V1 => unreachable!("V1 chunk endorsement is deprecated"), ChunkEndorsement::V2(v2) => &v2.metadata.account_id, } } - pub fn chunk_hash(&self) -> &ChunkHash { + pub fn chunk_hash(&self) -> ChunkHash { match self { - ChunkEndorsement::V1(e) => e.chunk_hash(), - ChunkEndorsement::V2(e) => e.chunk_hash(), + ChunkEndorsement::V1 => unreachable!("V1 chunk endorsement is deprecated"), + ChunkEndorsement::V2(v2) => v2.inner.chunk_hash.clone(), } } -} -#[derive(Debug, Clone, PartialEq, Eq, BorshSerialize, BorshDeserialize, ProtocolSchema)] -pub struct ChunkEndorsementV1 { - inner: ChunkEndorsementInner, - pub account_id: AccountId, - pub signature: Signature, -} + pub fn signature(&self) -> Signature { + match self { + ChunkEndorsement::V1 => unreachable!("V1 chunk endorsement is deprecated"), + ChunkEndorsement::V2(v2) => v2.signature.clone(), + } + } -impl ChunkEndorsementV1 { - pub fn new(chunk_hash: ChunkHash, signer: &ValidatorSigner) -> Self { - let inner = ChunkEndorsementInner::new(chunk_hash); - let account_id = signer.validator_id().clone(); - let signature = signer.sign_chunk_endorsement(&inner); - Self { inner, account_id, signature } + pub fn verify(&self, public_key: &PublicKey) -> bool { + match self { + ChunkEndorsement::V1 => unreachable!("V1 chunk endorsement is deprecated"), + ChunkEndorsement::V2(v2) => v2.verify(public_key), + } } - pub fn chunk_hash(&self) -> &ChunkHash { - &self.inner.chunk_hash + pub fn validate_signature( + chunk_hash: ChunkHash, + signature: &Signature, + public_key: &PublicKey, + ) -> bool { + let inner = ChunkEndorsementInner::new(chunk_hash); + let data = borsh::to_vec(&inner).unwrap(); + signature.verify(&data, public_key) } - pub fn verify(&self, public_key: &PublicKey) -> bool { - let data = borsh::to_vec(&self.inner).unwrap(); - self.signature.verify(&data, public_key) + /// Returns the account ID of the chunk validator that generated this endorsement. + pub fn validator_account(&self) -> &AccountId { + match self { + ChunkEndorsement::V1 => unreachable!("V1 chunk endorsement is deprecated"), + ChunkEndorsement::V2(v2) => &v2.metadata.account_id, + } } } @@ -97,44 +109,7 @@ pub struct ChunkEndorsementV2 { } impl ChunkEndorsementV2 { - pub fn new( - epoch_id: EpochId, - chunk_header: &ShardChunkHeader, - signer: &ValidatorSigner, - ) -> Self { - let inner = ChunkEndorsementInner::new(chunk_header.chunk_hash()); - let metadata = ChunkEndorsementMetadata { - account_id: signer.validator_id().clone(), - shard_id: chunk_header.shard_id(), - epoch_id, - height_created: chunk_header.height_created(), - }; - let signature = signer.sign_chunk_endorsement(&inner); - let metadata_signature = signer.sign_chunk_endorsement_metadata(&metadata); - Self { inner, signature, metadata, metadata_signature } - } - - pub fn chunk_production_key(&self) -> ChunkProductionKey { - ChunkProductionKey { - shard_id: self.metadata.shard_id, - epoch_id: self.metadata.epoch_id, - height_created: self.metadata.height_created, - } - } - - pub fn account_id(&self) -> &AccountId { - &self.metadata.account_id - } - - pub fn chunk_hash(&self) -> &ChunkHash { - &self.inner.chunk_hash - } - - pub fn signature(&self) -> Signature { - self.signature.clone() - } - - pub fn verify(&self, public_key: &PublicKey) -> bool { + fn verify(&self, public_key: &PublicKey) -> bool { let inner = borsh::to_vec(&self.inner).unwrap(); let metadata = borsh::to_vec(&self.metadata).unwrap(); self.signature.verify(&inner, public_key) diff --git a/integration-tests/src/tests/client/challenges.rs b/integration-tests/src/tests/client/challenges.rs index c4c0b3b2aeb..eaf7d193b4c 100644 --- a/integration-tests/src/tests/client/challenges.rs +++ b/integration-tests/src/tests/client/challenges.rs @@ -19,11 +19,11 @@ use near_primitives::merkle::PartialMerkleTree; use near_primitives::num_rational::Ratio; use near_primitives::shard_layout::ShardUId; use near_primitives::sharding::EncodedShardChunk; -use near_primitives::stateless_validation::chunk_endorsement::ChunkEndorsementV1; +use near_primitives::stateless_validation::chunk_endorsement::ChunkEndorsement; use near_primitives::test_utils::create_test_signer; use near_primitives::transaction::SignedTransaction; use near_primitives::types::chunk_extra::ChunkExtra; -use near_primitives::types::{AccountId, ShardId}; +use near_primitives::types::{AccountId, EpochId, ShardId}; use near_primitives::version::{ProtocolFeature, PROTOCOL_VERSION}; use near_store::Trie; use nearcore::test_utils::TestEnvNightshadeSetupExt; @@ -421,7 +421,7 @@ fn test_verify_chunk_invalid_state_challenge() { let signer = client.validator_signer.get().unwrap(); let endorsement = - ChunkEndorsementV1::new(invalid_chunk.cloned_header().chunk_hash(), signer.as_ref()); + ChunkEndorsement::new(EpochId::default(), &invalid_chunk.cloned_header(), signer.as_ref()); let block = Block::produce( PROTOCOL_VERSION, PROTOCOL_VERSION, @@ -429,7 +429,7 @@ fn test_verify_chunk_invalid_state_challenge() { last_block.header().height() + 1, last_block.header().block_ordinal() + 1, vec![invalid_chunk.cloned_header()], - vec![vec![Some(Box::new(endorsement.signature))]], + vec![vec![Some(Box::new(endorsement.signature()))]], *last_block.header().epoch_id(), *last_block.header().next_epoch_id(), None, diff --git a/integration-tests/src/tests/client/process_blocks.rs b/integration-tests/src/tests/client/process_blocks.rs index c27f3066025..59bd5204f15 100644 --- a/integration-tests/src/tests/client/process_blocks.rs +++ b/integration-tests/src/tests/client/process_blocks.rs @@ -46,7 +46,7 @@ use near_primitives::shard_layout::{get_block_shard_uid, ShardUId}; use near_primitives::sharding::{ShardChunkHeader, ShardChunkHeaderInner, ShardChunkHeaderV3}; use near_primitives::state_part::PartId; use near_primitives::state_sync::StatePartKey; -use near_primitives::stateless_validation::chunk_endorsement::ChunkEndorsementV1; +use near_primitives::stateless_validation::chunk_endorsement::ChunkEndorsement; use near_primitives::stateless_validation::chunk_endorsements_bitmap::ChunkEndorsementsBitmap; use near_primitives::test_utils::create_test_signer; use near_primitives::test_utils::TestBlockBuilder; @@ -2320,8 +2320,9 @@ fn test_validate_chunk_extra() { .set_chunk_endorsements(ChunkEndorsementsBitmap::from_endorsements(vec![vec![true]])); let outcome_root = Block::compute_outcome_root(block.chunks().iter_deprecated()); block.mut_header().set_prev_outcome_root(outcome_root); - let endorsement = ChunkEndorsementV1::new(chunk_header.chunk_hash(), &validator_signer); - block.set_chunk_endorsements(vec![vec![Some(Box::new(endorsement.signature))]]); + let endorsement = + ChunkEndorsement::new(EpochId::default(), &chunk_header, &validator_signer); + block.set_chunk_endorsements(vec![vec![Some(Box::new(endorsement.signature()))]]); let body_hash = block.compute_block_body_hash().unwrap(); block.mut_header().set_block_body_hash(body_hash); block.mut_header().resign(&validator_signer); diff --git a/tools/protocol-schema-check/res/protocol_schema.toml b/tools/protocol-schema-check/res/protocol_schema.toml index 41bc1cf63dd..6421539d175 100644 --- a/tools/protocol-schema-check/res/protocol_schema.toml +++ b/tools/protocol-schema-check/res/protocol_schema.toml @@ -58,10 +58,9 @@ ChunkContractAccesses = 4097831706 ChunkContractAccessesInner = 2563086819 ChunkContractAccessesV1 = 2405344532 ChunkContractDeploys = 414270153 -ChunkEndorsement = 1294072929 +ChunkEndorsement = 1084632166 ChunkEndorsementInner = 2425301775 ChunkEndorsementMetadata = 1740861942 -ChunkEndorsementV1 = 2471185877 ChunkEndorsementV2 = 3837631596 ChunkEndorsementsBitmap = 3112808654 ChunkExtraV1 = 774877102 @@ -182,7 +181,7 @@ PeerChainInfoV2 = 1260985250 PeerId = 2447445523 PeerIdOrHash = 4080492546 PeerInfo = 3831734408 -PeerMessage = 2466712164 +PeerMessage = 3418790017 Ping = 2783493472 Pong = 3159638327 PrepareError = 4009037507 @@ -207,8 +206,8 @@ ReceiptV1 = 2994842769 ReceiptValidationError = 551721215 ReceivedData = 3601438283 RootProof = 3135729669 -RoutedMessage = 1032206672 -RoutedMessageBody = 349045054 +RoutedMessage = 1033957649 +RoutedMessageBody = 2860051583 RoutingTableUpdate = 2987752645 Secp256K1PublicKey = 4117078281 Secp256K1Signature = 3687154735