From 051328a50106cbd01c745ce3dca95a5b9ef9291a Mon Sep 17 00:00:00 2001 From: alindima Date: Wed, 6 Sep 2023 17:58:49 +0300 Subject: [PATCH 001/126] Draft of RecoveryStrategy based on linked enums --- .../network/availability-recovery/src/lib.rs | 809 ++------------- .../network/availability-recovery/src/task.rs | 938 ++++++++++++++++++ .../availability-recovery/src/tests.rs | 49 +- 3 files changed, 1045 insertions(+), 751 deletions(-) create mode 100644 polkadot/node/network/availability-recovery/src/task.rs diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index 99f42f4bf9fe..63bfc82e3d7b 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -18,16 +18,10 @@ #![warn(missing_docs)] -use std::{ - collections::{HashMap, VecDeque}, - iter::Iterator, - num::NonZeroUsize, - pin::Pin, - time::Duration, -}; +use std::{collections::HashMap, iter::Iterator, num::NonZeroUsize, pin::Pin}; use futures::{ - channel::oneshot::{self, channel}, + channel::oneshot, future::{Future, FutureExt, RemoteHandle}, pin_mut, prelude::*, @@ -35,77 +29,55 @@ use futures::{ stream::{FuturesUnordered, StreamExt}, task::{Context, Poll}, }; -use rand::seq::SliceRandom; use schnellru::{ByLength, LruMap}; +use task::{FetchChunksParams, FetchFullParams}; use fatality::Nested; use polkadot_erasure_coding::{ branch_hash, branches, obtain_chunks_v1, recovery_threshold, Error as ErasureEncodingError, }; -#[cfg(not(test))] -use polkadot_node_network_protocol::request_response::CHUNK_REQUEST_TIMEOUT; +use task::{RecoveryParams, RecoveryStrategy, RecoveryTask}; + use polkadot_node_network_protocol::{ - request_response::{ - self as req_res, outgoing::RequestError, v1 as request_v1, IncomingRequestReceiver, - OutgoingRequest, Recipient, Requests, - }, - IfDisconnected, UnifiedReputationChange as Rep, + request_response::{v1 as request_v1, IncomingRequestReceiver}, + UnifiedReputationChange as Rep, }; use polkadot_node_primitives::{AvailableData, ErasureChunk}; use polkadot_node_subsystem::{ errors::RecoveryError, jaeger, - messages::{AvailabilityRecoveryMessage, AvailabilityStoreMessage, NetworkBridgeTxMessage}, + messages::{AvailabilityRecoveryMessage, AvailabilityStoreMessage}, overseer, ActiveLeavesUpdate, FromOrchestra, OverseerSignal, SpawnedSubsystem, SubsystemError, SubsystemResult, }; use polkadot_node_subsystem_util::request_session_info; use polkadot_primitives::{ - AuthorityDiscoveryId, BlakeTwo256, BlockNumber, CandidateHash, CandidateReceipt, GroupIndex, - Hash, HashT, IndexedVec, SessionIndex, SessionInfo, ValidatorId, ValidatorIndex, + BlakeTwo256, BlockNumber, CandidateHash, CandidateReceipt, GroupIndex, Hash, HashT, + SessionIndex, SessionInfo, ValidatorIndex, }; mod error; mod futures_undead; mod metrics; +mod task; use metrics::Metrics; -use futures_undead::FuturesUndead; -use sc_network::{OutboundFailure, RequestFailure}; - #[cfg(test)] mod tests; const LOG_TARGET: &str = "parachain::availability-recovery"; -// How many parallel recovery tasks should be running at once. -const N_PARALLEL: usize = 50; - // Size of the LRU cache where we keep recovered data. const LRU_SIZE: u32 = 16; const COST_INVALID_REQUEST: Rep = Rep::CostMajor("Peer sent unparsable request"); -/// Time after which we consider a request to have failed -/// -/// and we should try more peers. Note in theory the request times out at the network level, -/// measurements have shown, that in practice requests might actually take longer to fail in -/// certain occasions. (The very least, authority discovery is not part of the timeout.) -/// -/// For the time being this value is the same as the timeout on the networking layer, but as this -/// timeout is more soft than the networking one, it might make sense to pick different values as -/// well. -#[cfg(not(test))] -const TIMEOUT_START_NEW_REQUESTS: Duration = CHUNK_REQUEST_TIMEOUT; -#[cfg(test)] -const TIMEOUT_START_NEW_REQUESTS: Duration = Duration::from_millis(100); - /// PoV size limit in bytes for which prefer fetching from backers. const SMALL_POV_LIMIT: usize = 128 * 1024; #[derive(Clone, PartialEq)] /// The strategy we use to recover the PoV. -pub enum RecoveryStrategy { +pub enum RecoveryStrategyKind { /// We always try the backing group first, then fallback to validator chunks. BackersFirstAlways, /// We try the backing group first if PoV size is lower than specified, then fallback to @@ -120,92 +92,16 @@ pub enum RecoveryStrategy { BypassAvailabilityStore, } -impl RecoveryStrategy { - /// Returns true if the strategy needs backing group index. - pub fn needs_backing_group(&self) -> bool { - match self { - RecoveryStrategy::BackersFirstAlways | RecoveryStrategy::BackersFirstIfSizeLower(_) => - true, - _ => false, - } - } - - /// Returns the PoV size limit in bytes for `BackersFirstIfSizeLower` strategy, otherwise - /// `None`. - pub fn pov_size_limit(&self) -> Option { - match *self { - RecoveryStrategy::BackersFirstIfSizeLower(limit) => Some(limit), - _ => None, - } - } -} /// The Availability Recovery Subsystem. pub struct AvailabilityRecoverySubsystem { /// PoV recovery strategy to use. - recovery_strategy: RecoveryStrategy, + recovery_strategy_kind: RecoveryStrategyKind, /// Receiver for available data requests. req_receiver: IncomingRequestReceiver, /// Metrics for this subsystem. metrics: Metrics, } -struct RequestFromBackers { - // a random shuffling of the validators from the backing group which indicates the order - // in which we connect to them and request the chunk. - shuffled_backers: Vec, - // channel to the erasure task handler. - erasure_task_tx: futures::channel::mpsc::Sender, -} - -struct RequestChunksFromValidators { - /// How many request have been unsuccessful so far. - error_count: usize, - /// Total number of responses that have been received. - /// - /// including failed ones. - total_received_responses: usize, - /// a random shuffling of the validators which indicates the order in which we connect to the - /// validators and request the chunk from them. - shuffling: VecDeque, - /// Chunks received so far. - received_chunks: HashMap, - /// Pending chunk requests with soft timeout. - requesting_chunks: FuturesUndead, (ValidatorIndex, RequestError)>>, - // channel to the erasure task handler. - erasure_task_tx: futures::channel::mpsc::Sender, -} - -struct RecoveryParams { - /// Discovery ids of `validators`. - validator_authority_keys: Vec, - - /// Validators relevant to this `RecoveryTask`. - validators: IndexedVec, - - /// The number of pieces needed. - threshold: usize, - - /// A hash of the relevant candidate. - candidate_hash: CandidateHash, - - /// The root of the erasure encoding of the para block. - erasure_root: Hash, - - /// Metrics to report - metrics: Metrics, - - /// Do not request data from availability-store - bypass_availability_store: bool, -} - -/// Source the availability data either by means -/// of direct request response protocol to -/// backers (a.k.a. fast-path), or recover from chunks. -enum Source { - RequestFromBackers(RequestFromBackers), - RequestChunks(RequestChunksFromValidators), -} - /// Expensive erasure coding computations that we want to run on a blocking thread. enum ErasureTask { /// Reconstructs `AvailableData` from chunks given `n_validators`. @@ -219,486 +115,6 @@ enum ErasureTask { Reencode(usize, Hash, AvailableData, oneshot::Sender>), } -/// A stateful reconstruction of availability data in reference to -/// a candidate hash. -struct RecoveryTask { - sender: Sender, - - /// The parameters of the recovery process. - params: RecoveryParams, - - /// The source to obtain the availability data from. - source: Source, - - // channel to the erasure task handler. - erasure_task_tx: futures::channel::mpsc::Sender, -} - -impl RequestFromBackers { - fn new( - mut backers: Vec, - erasure_task_tx: futures::channel::mpsc::Sender, - ) -> Self { - backers.shuffle(&mut rand::thread_rng()); - - RequestFromBackers { shuffled_backers: backers, erasure_task_tx } - } - - // Run this phase to completion. - async fn run( - &mut self, - params: &RecoveryParams, - sender: &mut impl overseer::AvailabilityRecoverySenderTrait, - ) -> Result { - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - erasure_root = ?params.erasure_root, - "Requesting from backers", - ); - loop { - // Pop the next backer, and proceed to next phase if we're out. - let validator_index = - self.shuffled_backers.pop().ok_or_else(|| RecoveryError::Unavailable)?; - - // Request data. - let (req, response) = OutgoingRequest::new( - Recipient::Authority( - params.validator_authority_keys[validator_index.0 as usize].clone(), - ), - req_res::v1::AvailableDataFetchingRequest { candidate_hash: params.candidate_hash }, - ); - - sender - .send_message(NetworkBridgeTxMessage::SendRequests( - vec![Requests::AvailableDataFetchingV1(req)], - IfDisconnected::ImmediateError, - )) - .await; - - match response.await { - Ok(req_res::v1::AvailableDataFetchingResponse::AvailableData(data)) => { - let (reencode_tx, reencode_rx) = channel(); - self.erasure_task_tx - .send(ErasureTask::Reencode( - params.validators.len(), - params.erasure_root, - data, - reencode_tx, - )) - .await - .map_err(|_| RecoveryError::ChannelClosed)?; - - let reencode_response = - reencode_rx.await.map_err(|_| RecoveryError::ChannelClosed)?; - - if let Some(data) = reencode_response { - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - "Received full data", - ); - - return Ok(data) - } else { - gum::debug!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - ?validator_index, - "Invalid data response", - ); - - // it doesn't help to report the peer with req/res. - } - }, - Ok(req_res::v1::AvailableDataFetchingResponse::NoSuchData) => {}, - Err(e) => gum::debug!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - ?validator_index, - err = ?e, - "Error fetching full available data." - ), - } - } - } -} - -impl RequestChunksFromValidators { - fn new( - n_validators: u32, - erasure_task_tx: futures::channel::mpsc::Sender, - ) -> Self { - let mut shuffling: Vec<_> = (0..n_validators).map(ValidatorIndex).collect(); - shuffling.shuffle(&mut rand::thread_rng()); - - RequestChunksFromValidators { - error_count: 0, - total_received_responses: 0, - shuffling: shuffling.into(), - received_chunks: HashMap::new(), - requesting_chunks: FuturesUndead::new(), - erasure_task_tx, - } - } - - fn is_unavailable(&self, params: &RecoveryParams) -> bool { - is_unavailable( - self.chunk_count(), - self.requesting_chunks.total_len(), - self.shuffling.len(), - params.threshold, - ) - } - - fn chunk_count(&self) -> usize { - self.received_chunks.len() - } - - fn insert_chunk(&mut self, validator_index: ValidatorIndex, chunk: ErasureChunk) { - self.received_chunks.insert(validator_index, chunk); - } - - fn can_conclude(&self, params: &RecoveryParams) -> bool { - self.chunk_count() >= params.threshold || self.is_unavailable(params) - } - - /// Desired number of parallel requests. - /// - /// For the given threshold (total required number of chunks) get the desired number of - /// requests we want to have running in parallel at this time. - fn get_desired_request_count(&self, threshold: usize) -> usize { - // Upper bound for parallel requests. - // We want to limit this, so requests can be processed within the timeout and we limit the - // following feedback loop: - // 1. Requests fail due to timeout - // 2. We request more chunks to make up for it - // 3. Bandwidth is spread out even more, so we get even more timeouts - // 4. We request more chunks to make up for it ... - let max_requests_boundary = std::cmp::min(N_PARALLEL, threshold); - // How many chunks are still needed? - let remaining_chunks = threshold.saturating_sub(self.chunk_count()); - // What is the current error rate, so we can make up for it? - let inv_error_rate = - self.total_received_responses.checked_div(self.error_count).unwrap_or(0); - // Actual number of requests we want to have in flight in parallel: - std::cmp::min( - max_requests_boundary, - remaining_chunks + remaining_chunks.checked_div(inv_error_rate).unwrap_or(0), - ) - } - - async fn launch_parallel_requests( - &mut self, - params: &RecoveryParams, - sender: &mut Sender, - ) where - Sender: overseer::AvailabilityRecoverySenderTrait, - { - let num_requests = self.get_desired_request_count(params.threshold); - let candidate_hash = ¶ms.candidate_hash; - let already_requesting_count = self.requesting_chunks.len(); - - gum::debug!( - target: LOG_TARGET, - ?candidate_hash, - ?num_requests, - error_count= ?self.error_count, - total_received = ?self.total_received_responses, - threshold = ?params.threshold, - ?already_requesting_count, - "Requesting availability chunks for a candidate", - ); - let mut requests = Vec::with_capacity(num_requests - already_requesting_count); - - while self.requesting_chunks.len() < num_requests { - if let Some(validator_index) = self.shuffling.pop_back() { - let validator = params.validator_authority_keys[validator_index.0 as usize].clone(); - gum::trace!( - target: LOG_TARGET, - ?validator, - ?validator_index, - ?candidate_hash, - "Requesting chunk", - ); - - // Request data. - let raw_request = req_res::v1::ChunkFetchingRequest { - candidate_hash: params.candidate_hash, - index: validator_index, - }; - - let (req, res) = OutgoingRequest::new(Recipient::Authority(validator), raw_request); - requests.push(Requests::ChunkFetchingV1(req)); - - params.metrics.on_chunk_request_issued(); - let timer = params.metrics.time_chunk_request(); - - self.requesting_chunks.push(Box::pin(async move { - let _timer = timer; - match res.await { - Ok(req_res::v1::ChunkFetchingResponse::Chunk(chunk)) => - Ok(Some(chunk.recombine_into_chunk(&raw_request))), - Ok(req_res::v1::ChunkFetchingResponse::NoSuchChunk) => Ok(None), - Err(e) => Err((validator_index, e)), - } - })); - } else { - break - } - } - - sender - .send_message(NetworkBridgeTxMessage::SendRequests( - requests, - IfDisconnected::TryConnect, - )) - .await; - } - - /// Wait for a sufficient amount of chunks to reconstruct according to the provided `params`. - async fn wait_for_chunks(&mut self, params: &RecoveryParams) { - let metrics = ¶ms.metrics; - - // Wait for all current requests to conclude or time-out, or until we reach enough chunks. - // We also declare requests undead, once `TIMEOUT_START_NEW_REQUESTS` is reached and will - // return in that case for `launch_parallel_requests` to fill up slots again. - while let Some(request_result) = - self.requesting_chunks.next_with_timeout(TIMEOUT_START_NEW_REQUESTS).await - { - self.total_received_responses += 1; - - match request_result { - Ok(Some(chunk)) => - if is_chunk_valid(params, &chunk) { - metrics.on_chunk_request_succeeded(); - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - validator_index = ?chunk.index, - "Received valid chunk", - ); - self.insert_chunk(chunk.index, chunk); - } else { - metrics.on_chunk_request_invalid(); - self.error_count += 1; - }, - Ok(None) => { - metrics.on_chunk_request_no_such_chunk(); - self.error_count += 1; - }, - Err((validator_index, e)) => { - self.error_count += 1; - - gum::trace!( - target: LOG_TARGET, - candidate_hash= ?params.candidate_hash, - err = ?e, - ?validator_index, - "Failure requesting chunk", - ); - - match e { - RequestError::InvalidResponse(_) => { - metrics.on_chunk_request_invalid(); - - gum::debug!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - err = ?e, - ?validator_index, - "Chunk fetching response was invalid", - ); - }, - RequestError::NetworkError(err) => { - // No debug logs on general network errors - that became very spammy - // occasionally. - if let RequestFailure::Network(OutboundFailure::Timeout) = err { - metrics.on_chunk_request_timeout(); - } else { - metrics.on_chunk_request_error(); - } - - self.shuffling.push_front(validator_index); - }, - RequestError::Canceled(_) => { - metrics.on_chunk_request_error(); - - self.shuffling.push_front(validator_index); - }, - } - }, - } - - // Stop waiting for requests when we either can already recover the data - // or have gotten firm 'No' responses from enough validators. - if self.can_conclude(params) { - gum::debug!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - received_chunks_count = ?self.chunk_count(), - requested_chunks_count = ?self.requesting_chunks.len(), - threshold = ?params.threshold, - "Can conclude availability for a candidate", - ); - break - } - } - } - - async fn run( - &mut self, - params: &RecoveryParams, - sender: &mut Sender, - ) -> Result - where - Sender: overseer::AvailabilityRecoverySenderTrait, - { - let metrics = ¶ms.metrics; - - // First query the store for any chunks we've got. - if !params.bypass_availability_store { - let (tx, rx) = oneshot::channel(); - sender - .send_message(AvailabilityStoreMessage::QueryAllChunks(params.candidate_hash, tx)) - .await; - - match rx.await { - Ok(chunks) => { - // This should either be length 1 or 0. If we had the whole data, - // we wouldn't have reached this stage. - let chunk_indices: Vec<_> = chunks.iter().map(|c| c.index).collect(); - self.shuffling.retain(|i| !chunk_indices.contains(i)); - - for chunk in chunks { - if is_chunk_valid(params, &chunk) { - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - validator_index = ?chunk.index, - "Found valid chunk on disk" - ); - self.insert_chunk(chunk.index, chunk); - } else { - gum::error!( - target: LOG_TARGET, - "Loaded invalid chunk from disk! Disk/Db corruption _very_ likely - please fix ASAP!" - ); - }; - } - }, - Err(oneshot::Canceled) => { - gum::warn!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - "Failed to reach the availability store" - ); - }, - } - } - - let _recovery_timer = metrics.time_full_recovery(); - - loop { - if self.is_unavailable(¶ms) { - gum::debug!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - erasure_root = ?params.erasure_root, - received = %self.chunk_count(), - requesting = %self.requesting_chunks.len(), - total_requesting = %self.requesting_chunks.total_len(), - n_validators = %params.validators.len(), - "Data recovery is not possible", - ); - - metrics.on_recovery_failed(); - - return Err(RecoveryError::Unavailable) - } - - self.launch_parallel_requests(params, sender).await; - self.wait_for_chunks(params).await; - - // If received_chunks has more than threshold entries, attempt to recover the data. - // If that fails, or a re-encoding of it doesn't match the expected erasure root, - // return Err(RecoveryError::Invalid) - if self.chunk_count() >= params.threshold { - let recovery_duration = metrics.time_erasure_recovery(); - - // Send request to reconstruct available data from chunks. - let (avilable_data_tx, available_data_rx) = channel(); - self.erasure_task_tx - .send(ErasureTask::Reconstruct( - params.validators.len(), - std::mem::take(&mut self.received_chunks), - avilable_data_tx, - )) - .await - .map_err(|_| RecoveryError::ChannelClosed)?; - - let available_data_response = - available_data_rx.await.map_err(|_| RecoveryError::ChannelClosed)?; - - return match available_data_response { - Ok(data) => { - // Send request to re-encode the chunks and check merkle root. - let (reencode_tx, reencode_rx) = channel(); - self.erasure_task_tx - .send(ErasureTask::Reencode( - params.validators.len(), - params.erasure_root, - data, - reencode_tx, - )) - .await - .map_err(|_| RecoveryError::ChannelClosed)?; - - let reencode_response = - reencode_rx.await.map_err(|_| RecoveryError::ChannelClosed)?; - - if let Some(data) = reencode_response { - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - erasure_root = ?params.erasure_root, - "Data recovery complete", - ); - metrics.on_recovery_succeeded(); - - Ok(data) - } else { - recovery_duration.map(|rd| rd.stop_and_discard()); - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - erasure_root = ?params.erasure_root, - "Data recovery - root mismatch", - ); - metrics.on_recovery_invalid(); - - Err(RecoveryError::Invalid) - } - }, - Err(err) => { - recovery_duration.map(|rd| rd.stop_and_discard()); - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - erasure_root = ?params.erasure_root, - ?err, - "Data recovery error ", - ); - metrics.on_recovery_invalid(); - - Err(RecoveryError::Invalid) - }, - } - } - } - } -} - const fn is_unavailable( received_chunks: usize, requesting_chunks: usize, @@ -777,60 +193,6 @@ fn reconstructed_data_matches_root( branches.root() == *expected_root } -impl RecoveryTask -where - Sender: overseer::AvailabilityRecoverySenderTrait, -{ - async fn run(mut self) -> Result { - // First just see if we have the data available locally. - if !self.params.bypass_availability_store { - let (tx, rx) = oneshot::channel(); - self.sender - .send_message(AvailabilityStoreMessage::QueryAvailableData( - self.params.candidate_hash, - tx, - )) - .await; - - match rx.await { - Ok(Some(data)) => return Ok(data), - Ok(None) => {}, - Err(oneshot::Canceled) => { - gum::warn!( - target: LOG_TARGET, - candidate_hash = ?self.params.candidate_hash, - "Failed to reach the availability store", - ) - }, - } - } - - self.params.metrics.on_recovery_started(); - - loop { - // These only fail if we cannot reach the underlying subsystem, which case there is - // nothing meaningful we can do. - match self.source { - Source::RequestFromBackers(ref mut from_backers) => { - match from_backers.run(&self.params, &mut self.sender).await { - Ok(data) => break Ok(data), - Err(RecoveryError::Invalid) => break Err(RecoveryError::Invalid), - Err(RecoveryError::ChannelClosed) => - break Err(RecoveryError::ChannelClosed), - Err(RecoveryError::Unavailable) => - self.source = Source::RequestChunks(RequestChunksFromValidators::new( - self.params.validators.len() as _, - self.erasure_task_tx.clone(), - )), - } - }, - Source::RequestChunks(ref mut from_all) => - break from_all.run(&self.params, &mut self.sender).await, - } - } - } -} - /// Accumulate all awaiting sides for some particular `AvailableData`. struct RecoveryHandle { candidate_hash: CandidateHash, @@ -925,6 +287,7 @@ struct State { /// An LRU cache of recently recovered data. availability_lru: LruMap, + // TODO: an LRU cache of erasure indices shuffling of all validators (per relay-parent). } impl Default for State { @@ -973,65 +336,23 @@ async fn launch_recovery_task( ctx: &mut Context, session_info: SessionInfo, receipt: CandidateReceipt, - mut backing_group: Option, response_sender: oneshot::Sender>, metrics: &Metrics, - recovery_strategy: &RecoveryStrategy, - erasure_task_tx: futures::channel::mpsc::Sender, + recovery_strategy: RecoveryStrategy, + bypass_availability_store: bool, ) -> error::Result<()> { let candidate_hash = receipt.hash(); let params = RecoveryParams { validator_authority_keys: session_info.discovery_keys.clone(), - validators: session_info.validators.clone(), + n_validators: session_info.validators.len(), threshold: recovery_threshold(session_info.validators.len())?, candidate_hash, erasure_root: receipt.descriptor.erasure_root, metrics: metrics.clone(), - bypass_availability_store: recovery_strategy == &RecoveryStrategy::BypassAvailabilityStore, + bypass_availability_store, }; - if let Some(small_pov_limit) = recovery_strategy.pov_size_limit() { - // Get our own chunk size to get an estimate of the PoV size. - let chunk_size: Result, error::Error> = - query_chunk_size(ctx, candidate_hash).await; - if let Ok(Some(chunk_size)) = chunk_size { - let pov_size_estimate = chunk_size.saturating_mul(session_info.validators.len()) / 3; - let prefer_backing_group = pov_size_estimate < small_pov_limit; - - gum::trace!( - target: LOG_TARGET, - ?candidate_hash, - pov_size_estimate, - small_pov_limit, - enabled = prefer_backing_group, - "Prefer fetch from backing group", - ); - - backing_group = backing_group.filter(|_| { - // We keep the backing group only if `1/3` of chunks sum up to less than - // `small_pov_limit`. - prefer_backing_group - }); - } - } - - let phase = backing_group - .and_then(|g| session_info.validator_groups.get(g)) - .map(|group| { - Source::RequestFromBackers(RequestFromBackers::new( - group.clone(), - erasure_task_tx.clone(), - )) - }) - .unwrap_or_else(|| { - Source::RequestChunks(RequestChunksFromValidators::new( - params.validators.len() as _, - erasure_task_tx.clone(), - )) - }); - - let recovery_task = - RecoveryTask { sender: ctx.sender().clone(), params, source: phase, erasure_task_tx }; + let recovery_task = RecoveryTask::new(ctx.sender().clone(), params, recovery_strategy); let (remote, remote_handle) = recovery_task.run().remote_handle(); @@ -1062,8 +383,8 @@ async fn handle_recover( backing_group: Option, response_sender: oneshot::Sender>, metrics: &Metrics, - recovery_strategy: &RecoveryStrategy, erasure_task_tx: futures::channel::mpsc::Sender, + recovery_strategy_kind: RecoveryStrategyKind, ) -> error::Result<()> { let candidate_hash = receipt.hash(); @@ -1098,19 +419,75 @@ async fn handle_recover( let _span = span.child("session-info-ctx-received"); match session_info { - Some(session_info) => + Some(session_info) => { + let mut skip_backing_group_if: Box bool + Send> = Box::new(|| false); + + if let RecoveryStrategyKind::BackersFirstIfSizeLower(small_pov_limit) = + recovery_strategy_kind + { + // Get our own chunk size to get an estimate of the PoV size. + let chunk_size: Result, error::Error> = + query_chunk_size(ctx, candidate_hash).await; + if let Ok(Some(chunk_size)) = chunk_size { + let pov_size_estimate = + chunk_size.saturating_mul(session_info.validators.len()) / 3; + let prefer_backing_group = pov_size_estimate < small_pov_limit; + + gum::trace!( + target: LOG_TARGET, + ?candidate_hash, + pov_size_estimate, + small_pov_limit, + enabled = prefer_backing_group, + "Prefer fetch from backing group", + ); + + skip_backing_group_if = Box::new(move || !prefer_backing_group); + } + }; + + let backing_validators = if let Some(backing_group) = backing_group { + session_info.validator_groups.get(backing_group) + } else { + None + }; + + let fetch_chunks_params = FetchChunksParams { + n_validators: session_info.validators.len(), + erasure_task_tx: erasure_task_tx.clone(), + }; + + let recovery_strategy = if let Some(backing_validators) = backing_validators { + match recovery_strategy_kind { + RecoveryStrategyKind::BackersFirstAlways | + RecoveryStrategyKind::BackersFirstIfSizeLower(_) | + RecoveryStrategyKind::BypassAvailabilityStore => RecoveryStrategy::new() + .then_fetch_full_from_backers(FetchFullParams { + group_name: "backers", + validators: backing_validators.to_vec(), + skip_if: skip_backing_group_if, + erasure_task_tx, + }) + .then_fetch_chunks_from_validators(fetch_chunks_params), + RecoveryStrategyKind::ChunksAlways => RecoveryStrategy::new() + .then_fetch_chunks_from_validators(fetch_chunks_params), + } + } else { + RecoveryStrategy::new().then_fetch_chunks_from_validators(fetch_chunks_params) + }; + launch_recovery_task( state, ctx, session_info, receipt, - backing_group, response_sender, metrics, - recovery_strategy, - erasure_task_tx, + *recovery_strategy, + recovery_strategy_kind == RecoveryStrategyKind::BypassAvailabilityStore, ) - .await, + .await + }, None => { gum::warn!(target: LOG_TARGET, "SessionInfo is `None` at {:?}", state.live_block); response_sender @@ -1155,7 +532,11 @@ impl AvailabilityRecoverySubsystem { req_receiver: IncomingRequestReceiver, metrics: Metrics, ) -> Self { - Self { recovery_strategy: RecoveryStrategy::BypassAvailabilityStore, req_receiver, metrics } + Self { + recovery_strategy_kind: RecoveryStrategyKind::BypassAvailabilityStore, + req_receiver, + metrics, + } } /// Create a new instance of `AvailabilityRecoverySubsystem` which starts with a fast path to @@ -1164,7 +545,11 @@ impl AvailabilityRecoverySubsystem { req_receiver: IncomingRequestReceiver, metrics: Metrics, ) -> Self { - Self { recovery_strategy: RecoveryStrategy::BackersFirstAlways, req_receiver, metrics } + Self { + recovery_strategy_kind: RecoveryStrategyKind::BackersFirstAlways, + req_receiver, + metrics, + } } /// Create a new instance of `AvailabilityRecoverySubsystem` which requests only chunks @@ -1172,7 +557,7 @@ impl AvailabilityRecoverySubsystem { req_receiver: IncomingRequestReceiver, metrics: Metrics, ) -> Self { - Self { recovery_strategy: RecoveryStrategy::ChunksAlways, req_receiver, metrics } + Self { recovery_strategy_kind: RecoveryStrategyKind::ChunksAlways, req_receiver, metrics } } /// Create a new instance of `AvailabilityRecoverySubsystem` which requests chunks if PoV is @@ -1182,7 +567,7 @@ impl AvailabilityRecoverySubsystem { metrics: Metrics, ) -> Self { Self { - recovery_strategy: RecoveryStrategy::BackersFirstIfSizeLower(SMALL_POV_LIMIT), + recovery_strategy_kind: RecoveryStrategyKind::BackersFirstIfSizeLower(SMALL_POV_LIMIT), req_receiver, metrics, } @@ -1190,7 +575,7 @@ impl AvailabilityRecoverySubsystem { async fn run(self, mut ctx: Context) -> SubsystemResult<()> { let mut state = State::default(); - let Self { recovery_strategy, mut req_receiver, metrics } = self; + let Self { mut req_receiver, metrics, recovery_strategy_kind } = self; let (erasure_task_tx, erasure_task_rx) = futures::channel::mpsc::channel(16); let mut erasure_task_rx = erasure_task_rx.fuse(); @@ -1275,11 +660,11 @@ impl AvailabilityRecoverySubsystem { &mut ctx, receipt, session_index, - maybe_backing_group.filter(|_| recovery_strategy.needs_backing_group()), + maybe_backing_group, response_sender, &metrics, - &recovery_strategy, erasure_task_tx.clone(), + recovery_strategy_kind.clone(), ).await { gum::warn!( target: LOG_TARGET, @@ -1295,7 +680,7 @@ impl AvailabilityRecoverySubsystem { in_req = recv_req => { match in_req.into_nested().map_err(|fatal| SubsystemError::with_origin("availability-recovery", fatal))? { Ok(req) => { - if recovery_strategy == RecoveryStrategy::BypassAvailabilityStore { + if recovery_strategy_kind == RecoveryStrategyKind::BypassAvailabilityStore { gum::debug!( target: LOG_TARGET, "Skipping request to availability-store.", diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs new file mode 100644 index 000000000000..854dc26b4dd5 --- /dev/null +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -0,0 +1,938 @@ +use crate::{ + futures_undead::FuturesUndead, is_chunk_valid, is_unavailable, metrics::Metrics, ErasureTask, + LOG_TARGET, +}; +use futures::{channel::oneshot, SinkExt}; +#[cfg(not(test))] +use polkadot_node_network_protocol::request_response::CHUNK_REQUEST_TIMEOUT; +use polkadot_node_network_protocol::request_response::{ + self as req_res, outgoing::RequestError, OutgoingRequest, Recipient, Requests, +}; +use polkadot_node_primitives::{AvailableData, ErasureChunk}; +use polkadot_node_subsystem::{ + messages::{AvailabilityStoreMessage, NetworkBridgeTxMessage}, + overseer, RecoveryError, +}; +use polkadot_primitives::{AuthorityDiscoveryId, CandidateHash, Hash, ValidatorIndex}; +use rand::seq::SliceRandom; +use sc_network::{IfDisconnected, OutboundFailure, RequestFailure}; +use std::{ + collections::{HashMap, VecDeque}, + time::Duration, +}; + +// How many parallel recovery tasks should be running at once. +const N_PARALLEL: usize = 50; + +/// Time after which we consider a request to have failed +/// +/// and we should try more peers. Note in theory the request times out at the network level, +/// measurements have shown, that in practice requests might actually take longer to fail in +/// certain occasions. (The very least, authority discovery is not part of the timeout.) +/// +/// For the time being this value is the same as the timeout on the networking layer, but as this +/// timeout is more soft than the networking one, it might make sense to pick different values as +/// well. +#[cfg(not(test))] +const TIMEOUT_START_NEW_REQUESTS: Duration = CHUNK_REQUEST_TIMEOUT; +#[cfg(test)] +const TIMEOUT_START_NEW_REQUESTS: Duration = Duration::from_millis(100); + +pub struct RecoveryParams { + /// Discovery ids of `validators`. + pub(crate) validator_authority_keys: Vec, + + /// Number of validators relevant to this `RecoveryTask`. + pub(crate) n_validators: usize, + + /// The number of pieces needed. + pub(crate) threshold: usize, + + /// A hash of the relevant candidate. + pub(crate) candidate_hash: CandidateHash, + + /// The root of the erasure encoding of the para block. + pub(crate) erasure_root: Hash, + + /// Metrics to report + pub(crate) metrics: Metrics, + + /// Do not request data from availability-store + pub(crate) bypass_availability_store: bool, +} +/// Represents intermediate data that must be passed between `RecoveryStrategy`s belonging to the +/// same `RecoveryTask` or data that is used by state methods common to multiple RecoveryStrategies. +pub struct State { + /// Chunks received so far. + received_chunks: HashMap, + requesting_chunks: FuturesUndead, (ValidatorIndex, RequestError)>>, +} + +impl State { + fn new() -> Self { + Self { received_chunks: HashMap::new(), requesting_chunks: FuturesUndead::new() } + } + + fn insert_chunk(&mut self, validator: ValidatorIndex, chunk: ErasureChunk) { + self.received_chunks.insert(validator, chunk); + } + + fn chunk_count(&self) -> usize { + self.received_chunks.len() + } + + async fn populate_from_av_store( + &mut self, + params: &RecoveryParams, + sender: &mut Sender, + ) -> Vec { + let (tx, rx) = oneshot::channel(); + sender + .send_message(AvailabilityStoreMessage::QueryAllChunks(params.candidate_hash, tx)) + .await; + + match rx.await { + Ok(chunks) => { + // This should either be length 1 or 0. If we had the whole data, + // we wouldn't have reached this stage. + let chunk_indices: Vec<_> = chunks.iter().map(|c| c.index).collect(); + + for chunk in chunks { + if is_chunk_valid(params, &chunk) { + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?params.candidate_hash, + validator_index = ?chunk.index, + "Found valid chunk on disk" + ); + self.insert_chunk(chunk.index, chunk); + } else { + gum::error!( + target: LOG_TARGET, + "Loaded invalid chunk from disk! Disk/Db corruption _very_ likely - please fix ASAP!" + ); + }; + } + + chunk_indices + }, + Err(oneshot::Canceled) => { + gum::warn!( + target: LOG_TARGET, + candidate_hash = ?params.candidate_hash, + "Failed to reach the availability store" + ); + + vec![] + }, + } + } + + async fn launch_parallel_chunk_requests( + &mut self, + params: &RecoveryParams, + sender: &mut Sender, + desired_requests_count: usize, + validators: &mut VecDeque, + ) where + Sender: overseer::AvailabilityRecoverySenderTrait, + { + let candidate_hash = ¶ms.candidate_hash; + let already_requesting_count = self.requesting_chunks.len(); + + let mut requests = Vec::with_capacity(desired_requests_count - already_requesting_count); + + while self.requesting_chunks.len() < desired_requests_count { + if let Some(validator_index) = validators.pop_back() { + let validator = params.validator_authority_keys[validator_index.0 as usize].clone(); + gum::trace!( + target: LOG_TARGET, + ?validator, + ?validator_index, + ?candidate_hash, + "Requesting chunk", + ); + + // Request data. + let raw_request = req_res::v1::ChunkFetchingRequest { + candidate_hash: params.candidate_hash, + index: validator_index, + }; + + let (req, res) = OutgoingRequest::new(Recipient::Authority(validator), raw_request); + requests.push(Requests::ChunkFetchingV1(req)); + + params.metrics.on_chunk_request_issued(); + let timer = params.metrics.time_chunk_request(); + + self.requesting_chunks.push(Box::pin(async move { + let _timer = timer; + match res.await { + Ok(req_res::v1::ChunkFetchingResponse::Chunk(chunk)) => + Ok(Some(chunk.recombine_into_chunk(&raw_request))), + Ok(req_res::v1::ChunkFetchingResponse::NoSuchChunk) => Ok(None), + Err(e) => Err((validator_index, e)), + } + })); + } else { + break + } + } + + sender + .send_message(NetworkBridgeTxMessage::SendRequests( + requests, + IfDisconnected::TryConnect, + )) + .await; + } + + /// Wait for a sufficient amount of chunks to reconstruct according to the provided `params`. + async fn wait_for_chunks( + &mut self, + params: &RecoveryParams, + validators: &mut VecDeque, + can_conclude: impl Fn(usize, usize, usize, &RecoveryParams, usize) -> bool, + ) -> (usize, usize) { + let metrics = ¶ms.metrics; + + let mut total_received_responses = 0; + let mut error_count = 0; + + // Wait for all current requests to conclude or time-out, or until we reach enough chunks. + // We also declare requests undead, once `TIMEOUT_START_NEW_REQUESTS` is reached and will + // return in that case for `launch_parallel_requests` to fill up slots again. + while let Some(request_result) = + self.requesting_chunks.next_with_timeout(TIMEOUT_START_NEW_REQUESTS).await + { + total_received_responses += 1; + + match request_result { + Ok(Some(chunk)) => + if is_chunk_valid(params, &chunk) { + metrics.on_chunk_request_succeeded(); + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?params.candidate_hash, + validator_index = ?chunk.index, + "Received valid chunk", + ); + self.insert_chunk(chunk.index, chunk); + } else { + metrics.on_chunk_request_invalid(); + error_count += 1; + }, + Ok(None) => { + metrics.on_chunk_request_no_such_chunk(); + error_count += 1; + }, + Err((validator_index, e)) => { + error_count += 1; + + gum::trace!( + target: LOG_TARGET, + candidate_hash= ?params.candidate_hash, + err = ?e, + ?validator_index, + "Failure requesting chunk", + ); + + match e { + RequestError::InvalidResponse(_) => { + metrics.on_chunk_request_invalid(); + + gum::debug!( + target: LOG_TARGET, + candidate_hash = ?params.candidate_hash, + err = ?e, + ?validator_index, + "Chunk fetching response was invalid", + ); + }, + RequestError::NetworkError(err) => { + // No debug logs on general network errors - that became very spammy + // occasionally. + if let RequestFailure::Network(OutboundFailure::Timeout) = err { + metrics.on_chunk_request_timeout(); + } else { + metrics.on_chunk_request_error(); + } + + validators.push_front(validator_index); + }, + RequestError::Canceled(_) => { + metrics.on_chunk_request_error(); + + validators.push_front(validator_index); + }, + } + }, + } + + // Stop waiting for requests when we either can already recover the data + // or have gotten firm 'No' responses from enough validators. + if can_conclude( + validators.len(), + self.requesting_chunks.total_len(), + self.chunk_count(), + params, + error_count, + ) { + gum::debug!( + target: LOG_TARGET, + candidate_hash = ?params.candidate_hash, + received_chunks_count = ?self.chunk_count(), + requested_chunks_count = ?self.requesting_chunks.len(), + threshold = ?params.threshold, + "Can conclude availability for a candidate", + ); + break + } + } + + (total_received_responses, error_count) + } +} + +/// A stateful reconstruction of availability data in reference to +/// a candidate hash. +pub struct RecoveryTask { + sender: Sender, + /// The common parameters of the recovery process, regardless of the strategy. + params: RecoveryParams, + strategy: RecoveryStrategy, + state: State, +} + +impl RecoveryTask { + pub fn new(sender: Sender, params: RecoveryParams, strategy: RecoveryStrategy) -> Self { + Self { sender, params, strategy, state: State::new() } + } +} + +impl RecoveryTask +where + Sender: overseer::AvailabilityRecoverySenderTrait, +{ + async fn in_availability_store(&mut self) -> Option { + if !self.params.bypass_availability_store { + let (tx, rx) = oneshot::channel(); + self.sender + .send_message(AvailabilityStoreMessage::QueryAvailableData( + self.params.candidate_hash, + tx, + )) + .await; + + match rx.await { + Ok(Some(data)) => return Some(data), + Ok(None) => {}, + Err(oneshot::Canceled) => { + gum::warn!( + target: LOG_TARGET, + candidate_hash = ?self.params.candidate_hash, + "Failed to reach the availability store", + ) + }, + } + } + + None + } + + pub async fn run(mut self) -> Result { + if let Some(data) = self.in_availability_store().await { + return Ok(data) + } + + self.params.metrics.on_recovery_started(); + + let _timer = self.params.metrics.time_full_recovery(); + + let res = loop { + let (current_strategy, next_strategy) = self.strategy.pop_first(); + self.strategy = next_strategy; + + // Make sure we are not referencing futures from past RecoveryStrategy runs. + if self.state.requesting_chunks.total_len() != 0 { + self.state.requesting_chunks = FuturesUndead::new(); + } + + let recovery_strategy_name = current_strategy.display_name(); + + if let Some(name) = recovery_strategy_name { + gum::info!( + target: LOG_TARGET, + candidate_hash = ?self.params.candidate_hash, + "Starting `{}` strategy", + &name, + ); + } + + let res = match current_strategy { + RecoveryStrategy::Nil => Err(RecoveryError::Unavailable), + RecoveryStrategy::FullFromBackers(inner, _) => + inner.run(&mut self.state, &mut self.sender, &self.params).await, + RecoveryStrategy::ChunksFromValidators(inner, _) => + inner.run(&mut self.state, &mut self.sender, &self.params).await, + }; + + match res { + Err(RecoveryError::Unavailable) => { + if !matches!(&self.strategy, RecoveryStrategy::Nil) { + if let Some(recovery_strategy_name) = recovery_strategy_name { + gum::warn!( + target: LOG_TARGET, + candidate_hash = ?self.params.candidate_hash, + "Recovery strategy `{}` did not conclude. Trying the next one.", + recovery_strategy_name, + ); + } + continue + } else { + // We have no other strategies to try. + gum::error!( + target: LOG_TARGET, + candidate_hash = ?self.params.candidate_hash, + "Recovery of available data failed.", + ); + break Err(RecoveryError::Unavailable) + } + }, + Err(err) => break Err(err), + Ok(data) => break Ok(data), + } + }; + + match &res { + Ok(_) => self.params.metrics.on_recovery_succeeded(), + Err(RecoveryError::Invalid) => self.params.metrics.on_recovery_invalid(), + Err(_) => self.params.metrics.on_recovery_failed(), + } + + res + } +} + +pub enum RecoveryStrategy { + Nil, + FullFromBackers(FetchFull, Box), + ChunksFromValidators(FetchChunks, Box), +} + +impl RecoveryStrategy { + pub fn new() -> Box { + Box::new(RecoveryStrategy::Nil) + } + + fn display_name(&self) -> Option<&'static str> { + match self { + Self::Nil => None, + Self::FullFromBackers(_, _) => Some("Full recovery from backers"), + Self::ChunksFromValidators(_, _) => Some("Chunks recovery"), + } + } + + pub fn then_fetch_full_from_backers(self: Box, params: FetchFullParams) -> Box { + match *self { + Self::Nil => Box::new(Self::FullFromBackers(FetchFull::new(params), self)), + Self::ChunksFromValidators(task, next) => { + let next = next.then_fetch_full_from_backers(params); + Box::new(Self::ChunksFromValidators(task, next)) + }, + Self::FullFromBackers(task, next) => { + let next = next.then_fetch_full_from_backers(params); + Box::new(Self::FullFromBackers(task, next)) + }, + } + } + + pub fn then_fetch_chunks_from_validators( + self: Box, + params: FetchChunksParams, + ) -> Box { + match *self { + Self::Nil => Box::new(Self::ChunksFromValidators(FetchChunks::new(params), self)), + Self::ChunksFromValidators(task, next) => { + let next = next.then_fetch_chunks_from_validators(params); + Box::new(Self::ChunksFromValidators(task, next)) + }, + Self::FullFromBackers(task, next) => { + let next = next.then_fetch_chunks_from_validators(params); + Box::new(Self::FullFromBackers(task, next)) + }, + } + } + + fn pop_first(self: Self) -> (Self, Self) { + match self { + Self::Nil => (Self::Nil, Self::Nil), + Self::FullFromBackers(inner, next) => + (Self::FullFromBackers(inner, Box::new(Self::Nil)), *next), + Self::ChunksFromValidators(inner, next) => + (Self::ChunksFromValidators(inner, Box::new(Self::Nil)), *next), + } + } +} + +pub struct FetchFull { + params: FetchFullParams, +} + +pub struct FetchFullParams { + pub(crate) group_name: &'static str, + pub(crate) validators: Vec, + pub(crate) skip_if: Box bool + Send>, + // channel to the erasure task handler. + pub(crate) erasure_task_tx: futures::channel::mpsc::Sender, +} + +impl FetchFull { + fn new(params: FetchFullParams) -> Self { + Self { params } + } + + async fn run( + mut self, + _: &mut State, + sender: &mut Sender, + common_params: &RecoveryParams, + ) -> Result { + if (self.params.skip_if)() { + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + erasure_root = ?common_params.erasure_root, + "Skipping requesting availability data from {}", + self.params.group_name + ); + + return Err(RecoveryError::Unavailable) + } + + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + erasure_root = ?common_params.erasure_root, + "Requesting full availability data from {}", + self.params.group_name + ); + loop { + // Pop the next validator, and proceed to next fetch_chunks_task if we're out. + let validator_index = + self.params.validators.pop().ok_or_else(|| RecoveryError::Unavailable)?; + + // Request data. + let (req, response) = OutgoingRequest::new( + Recipient::Authority( + common_params.validator_authority_keys[validator_index.0 as usize].clone(), + ), + req_res::v1::AvailableDataFetchingRequest { + candidate_hash: common_params.candidate_hash, + }, + ); + + sender + .send_message(NetworkBridgeTxMessage::SendRequests( + vec![Requests::AvailableDataFetchingV1(req)], + IfDisconnected::ImmediateError, + )) + .await; + + match response.await { + Ok(req_res::v1::AvailableDataFetchingResponse::AvailableData(data)) => { + let (reencode_tx, reencode_rx) = oneshot::channel(); + self.params + .erasure_task_tx + .send(ErasureTask::Reencode( + common_params.n_validators, + common_params.erasure_root, + data, + reencode_tx, + )) + .await + .map_err(|_| RecoveryError::ChannelClosed)?; + + let reencode_response = + reencode_rx.await.map_err(|_| RecoveryError::ChannelClosed)?; + + if let Some(data) = reencode_response { + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + "Received full data", + ); + + return Ok(data) + } else { + gum::debug!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + ?validator_index, + "Invalid data response", + ); + + // it doesn't help to report the peer with req/res. + } + }, + Ok(req_res::v1::AvailableDataFetchingResponse::NoSuchData) => {}, + Err(e) => gum::debug!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + ?validator_index, + err = ?e, + "Error fetching full available data." + ), + } + } + } +} + +pub struct FetchChunks { + /// How many request have been unsuccessful so far. + error_count: usize, + /// Total number of responses that have been received. + /// + /// including failed ones. + total_received_responses: usize, + + /// a random shuffling of the validators which indicates the order in which we connect to the + /// validators and request the chunk from them. + validators: VecDeque, + + // channel to the erasure task handler. + erasure_task_tx: futures::channel::mpsc::Sender, +} + +pub struct FetchChunksParams { + pub(crate) n_validators: usize, + // channel to the erasure task handler. + pub(crate) erasure_task_tx: futures::channel::mpsc::Sender, +} + +impl FetchChunks { + fn new(params: FetchChunksParams) -> Self { + let mut shuffling: Vec<_> = (0..params.n_validators) + .map(|i| ValidatorIndex(i.try_into().expect("number of validators must fit in a u32"))) + .collect(); + shuffling.shuffle(&mut rand::thread_rng()); + + Self { + error_count: 0, + total_received_responses: 0, + validators: shuffling.into(), + erasure_task_tx: params.erasure_task_tx, + } + } + + fn is_unavailable( + unrequested_validators: usize, + in_flight_requests: usize, + chunk_count: usize, + threshold: usize, + ) -> bool { + is_unavailable(chunk_count, in_flight_requests, unrequested_validators, threshold) + } + + /// Desired number of parallel requests. + /// + /// For the given threshold (total required number of chunks) get the desired number of + /// requests we want to have running in parallel at this time. + fn get_desired_request_count(&self, chunk_count: usize, threshold: usize) -> usize { + // Upper bound for parallel requests. + // We want to limit this, so requests can be processed within the timeout and we limit the + // following feedback loop: + // 1. Requests fail due to timeout + // 2. We request more chunks to make up for it + // 3. Bandwidth is spread out even more, so we get even more timeouts + // 4. We request more chunks to make up for it ... + let max_requests_boundary = std::cmp::min(N_PARALLEL, threshold); + // How many chunks are still needed? + let remaining_chunks = threshold.saturating_sub(chunk_count); + // What is the current error rate, so we can make up for it? + let inv_error_rate = + self.total_received_responses.checked_div(self.error_count).unwrap_or(0); + // Actual number of requests we want to have in flight in parallel: + std::cmp::min( + max_requests_boundary, + remaining_chunks + remaining_chunks.checked_div(inv_error_rate).unwrap_or(0), + ) + } + + async fn run( + mut self, + state: &mut State, + sender: &mut Sender, + common_params: &RecoveryParams, + ) -> Result { + // First query the store for any chunks we've got. + if !common_params.bypass_availability_store { + let local_chunk_indices = state.populate_from_av_store(common_params, sender).await; + self.validators.retain(|i| !local_chunk_indices.contains(i)); + } + + // No need to query the validators that have the chunks we already received. + self.validators.retain(|i| !state.received_chunks.contains_key(i)); + + loop { + // If received_chunks has more than threshold entries, attempt to recover the data. + // If that fails, or a re-encoding of it doesn't match the expected erasure root, + // return Err(RecoveryError::Invalid). + // Do this before requesting any chunks because we may have enough of them coming from + // past RecoveryStrategies. + if state.chunk_count() >= common_params.threshold { + return self.attempt_recovery(state, common_params).await + } + + if Self::is_unavailable( + self.validators.len(), + state.requesting_chunks.total_len(), + state.chunk_count(), + common_params.threshold, + ) { + gum::debug!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + erasure_root = ?common_params.erasure_root, + received = %state.chunk_count(), + requesting = %state.requesting_chunks.len(), + total_requesting = %state.requesting_chunks.total_len(), + n_validators = %common_params.n_validators, + "Data recovery from chunks is not possible", + ); + + return Err(RecoveryError::Unavailable) + } + + let desired_requests_count = + self.get_desired_request_count(state.chunk_count(), common_params.threshold); + let already_requesting_count = state.requesting_chunks.len(); + gum::debug!( + target: LOG_TARGET, + ?common_params.candidate_hash, + ?desired_requests_count, + error_count= ?self.error_count, + total_received = ?self.total_received_responses, + threshold = ?common_params.threshold, + ?already_requesting_count, + "Requesting availability chunks for a candidate", + ); + state + .launch_parallel_chunk_requests( + common_params, + sender, + desired_requests_count, + &mut self.validators, + ) + .await; + + let (total_responses, error_count) = state + .wait_for_chunks( + common_params, + &mut self.validators, + |unrequested_validators, reqs, chunk_count, params, _error_count| { + chunk_count >= params.threshold || + Self::is_unavailable( + unrequested_validators, + reqs, + chunk_count, + params.threshold, + ) + }, + ) + .await; + + self.total_received_responses += total_responses; + self.error_count += error_count; + } + } + + async fn attempt_recovery( + &mut self, + state: &mut State, + common_params: &RecoveryParams, + ) -> Result { + let recovery_duration = common_params.metrics.time_erasure_recovery(); + + // Send request to reconstruct available data from chunks. + let (avilable_data_tx, available_data_rx) = oneshot::channel(); + self.erasure_task_tx + .send(ErasureTask::Reconstruct( + common_params.n_validators, + // Safe to leave an empty vec in place, as we're stopping the recovery process if + // this reconstruct fails. + std::mem::take(&mut state.received_chunks), + avilable_data_tx, + )) + .await + .map_err(|_| RecoveryError::ChannelClosed)?; + + let available_data_response = + available_data_rx.await.map_err(|_| RecoveryError::ChannelClosed)?; + + match available_data_response { + Ok(data) => { + // Send request to re-encode the chunks and check merkle root. + let (reencode_tx, reencode_rx) = oneshot::channel(); + self.erasure_task_tx + .send(ErasureTask::Reencode( + common_params.n_validators, + common_params.erasure_root, + data, + reencode_tx, + )) + .await + .map_err(|_| RecoveryError::ChannelClosed)?; + + let reencode_response = + reencode_rx.await.map_err(|_| RecoveryError::ChannelClosed)?; + + if let Some(data) = reencode_response { + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + erasure_root = ?common_params.erasure_root, + "Data recovery from chunks complete", + ); + + Ok(data) + } else { + recovery_duration.map(|rd| rd.stop_and_discard()); + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + erasure_root = ?common_params.erasure_root, + "Data recovery error - root mismatch", + ); + + Err(RecoveryError::Invalid) + } + }, + Err(err) => { + recovery_duration.map(|rd| rd.stop_and_discard()); + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + erasure_root = ?common_params.erasure_root, + ?err, + "Data recovery error ", + ); + + Err(RecoveryError::Invalid) + }, + } + } +} + +#[cfg(test)] +mod tests { + use std::ops::Deref; + + use super::*; + use assert_matches::assert_matches; + use polkadot_erasure_coding::recovery_threshold; + use RecoveryStrategy::*; + + impl std::fmt::Debug for RecoveryStrategy { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + Nil => write!(f, "Nil"), + ChunksFromValidators(_, next) => + write!(f, "{:?} -> {}", self.display_name(), format!("{next:?}")), + FullFromBackers(_, next) => + write!(f, "{:?} -> {}", self.display_name(), format!("{next:?}")), + } + } + } + + #[test] + fn test_recovery_strategy_linked_list_ops() { + let fetch_full_params = FetchFullParams { + group_name: "backers", + validators: vec![], + skip_if: Box::new(|| true), + erasure_task_tx: futures::channel::mpsc::channel(0).0, + }; + let fetch_full_params_2 = FetchFullParams { + group_name: "approval-checkers", + validators: vec![], + skip_if: Box::new(|| true), + erasure_task_tx: futures::channel::mpsc::channel(0).0, + }; + + let fetch_chunks_params = FetchChunksParams { + n_validators: 2, + erasure_task_tx: futures::channel::mpsc::channel(0).0, + }; + let fetch_chunks_params_2 = FetchChunksParams { + n_validators: 3, + erasure_task_tx: futures::channel::mpsc::channel(0).0, + }; + let recovery_strategy = RecoveryStrategy::new() + .then_fetch_full_from_backers(fetch_full_params) + .then_fetch_full_from_backers(fetch_full_params_2) + .then_fetch_chunks_from_validators(fetch_chunks_params) + .then_fetch_chunks_from_validators(fetch_chunks_params_2); + + // Check that the builder correctly chains strategies. + assert_matches!( + recovery_strategy.deref(), + FullFromBackers(_, next) + if matches!(next.deref(), FullFromBackers(_, next) + if matches!(next.deref(), ChunksFromValidators(_, next) + if matches!(next.deref(), ChunksFromValidators(_, next) + if matches!(next.deref(), Nil) + ) + ) + ) + ); + + // Check the order for the `pop_first` operation. + let (current, next) = recovery_strategy.pop_first(); + assert_matches!(current, FullFromBackers(task, next) if task.params.group_name == "backers" && matches!(*next, Nil)); + assert_matches!(&next, FullFromBackers(task, _) if task.params.group_name == "approval-checkers"); + + let (current, next) = next.pop_first(); + assert_matches!(current, FullFromBackers(task, next) if task.params.group_name == "approval-checkers" && matches!(*next, Nil)); + assert_matches!(&next, ChunksFromValidators(task, _) if task.validators.len() == 2); + + let (current, next) = next.pop_first(); + assert_matches!(current, ChunksFromValidators(task, next) if task.validators.len() == 2 && matches!(*next, Nil)); + assert_matches!(&next, ChunksFromValidators(task, _) if task.validators.len() == 3); + + let (current, next) = next.pop_first(); + assert_matches!(current, ChunksFromValidators(task, next) if task.validators.len() == 3 && matches!(*next, Nil)); + assert_matches!(&next, Nil); + + let (current, next) = next.pop_first(); + assert_matches!(current, Nil); + assert_matches!(next, Nil); + } + + #[test] + fn parallel_request_calculation_works_as_expected() { + let num_validators = 100; + let threshold = recovery_threshold(num_validators).unwrap(); + let (erasure_task_tx, _erasure_task_rx) = futures::channel::mpsc::channel(16); + + let mut fetch_chunks_task = + FetchChunks::new(FetchChunksParams { n_validators: 100, erasure_task_tx }); + assert_eq!(fetch_chunks_task.get_desired_request_count(0, threshold), threshold); + fetch_chunks_task.error_count = 1; + fetch_chunks_task.total_received_responses = 1; + // We saturate at threshold (34): + assert_eq!(fetch_chunks_task.get_desired_request_count(0, threshold), threshold); + + fetch_chunks_task.total_received_responses = 2; + // With given error rate - still saturating: + assert_eq!(fetch_chunks_task.get_desired_request_count(1, threshold), threshold); + fetch_chunks_task.total_received_responses += 8; + // error rate: 1/10 + // remaining chunks needed: threshold (34) - 9 + // expected: 24 * (1+ 1/10) = (next greater integer) = 27 + assert_eq!(fetch_chunks_task.get_desired_request_count(9, threshold), 27); + fetch_chunks_task.error_count = 0; + // With error count zero - we should fetch exactly as needed: + assert_eq!(fetch_chunks_task.get_desired_request_count(10, threshold), threshold - 10); + } +} diff --git a/polkadot/node/network/availability-recovery/src/tests.rs b/polkadot/node/network/availability-recovery/src/tests.rs index 60c2d38ab31b..17b02080c008 100644 --- a/polkadot/node/network/availability-recovery/src/tests.rs +++ b/polkadot/node/network/availability-recovery/src/tests.rs @@ -21,15 +21,19 @@ use futures::{executor, future}; use futures_timer::Delay; use parity_scale_codec::Encode; -use polkadot_node_network_protocol::request_response::{IncomingRequest, ReqProtocolNames}; +use polkadot_node_network_protocol::request_response::{ + self as req_res, IncomingRequest, Recipient, ReqProtocolNames, Requests, +}; use super::*; -use sc_network::config::RequestResponseConfig; +use sc_network::{config::RequestResponseConfig, IfDisconnected, OutboundFailure, RequestFailure}; use polkadot_erasure_coding::{branches, obtain_chunks_v1 as obtain_chunks}; use polkadot_node_primitives::{BlockData, PoV, Proof}; -use polkadot_node_subsystem::messages::{AllMessages, RuntimeApiMessage, RuntimeApiRequest}; +use polkadot_node_subsystem::messages::{ + AllMessages, NetworkBridgeTxMessage, RuntimeApiMessage, RuntimeApiRequest, +}; use polkadot_node_subsystem_test_helpers::{ make_subsystem_context, mock::new_leaf, TestSubsystemContextHandle, }; @@ -204,7 +208,7 @@ use sp_keyring::Sr25519Keyring; enum Has { No, Yes, - NetworkError(sc_network::RequestFailure), + NetworkError(RequestFailure), /// Make request not return at all, instead the sender is returned from the function. /// /// Note, if you use `DoesNotReturn` you have to keep the returned senders alive, otherwise the @@ -214,7 +218,7 @@ enum Has { impl Has { fn timeout() -> Self { - Has::NetworkError(sc_network::RequestFailure::Network(sc_network::OutboundFailure::Timeout)) + Has::NetworkError(RequestFailure::Network(OutboundFailure::Timeout)) } } @@ -393,7 +397,7 @@ impl TestState { candidate_hash: CandidateHash, virtual_overseer: &mut VirtualOverseer, who_has: impl Fn(usize) -> Has, - ) -> Vec, sc_network::RequestFailure>>> { + ) -> Vec, RequestFailure>>> { let mut senders = Vec::new(); for _ in 0..self.validators.len() { // Receive a request for a chunk. @@ -1546,36 +1550,3 @@ fn invalid_local_chunk_is_ignored() { (virtual_overseer, req_cfg) }); } - -#[test] -fn parallel_request_calculation_works_as_expected() { - let num_validators = 100; - let threshold = recovery_threshold(num_validators).unwrap(); - let (erasure_task_tx, _erasure_task_rx) = futures::channel::mpsc::channel(16); - - let mut phase = RequestChunksFromValidators::new(100, erasure_task_tx); - assert_eq!(phase.get_desired_request_count(threshold), threshold); - phase.error_count = 1; - phase.total_received_responses = 1; - // We saturate at threshold (34): - assert_eq!(phase.get_desired_request_count(threshold), threshold); - - let dummy_chunk = - ErasureChunk { chunk: Vec::new(), index: ValidatorIndex(0), proof: Proof::dummy_proof() }; - phase.insert_chunk(ValidatorIndex(0), dummy_chunk.clone()); - phase.total_received_responses = 2; - // With given error rate - still saturating: - assert_eq!(phase.get_desired_request_count(threshold), threshold); - for i in 1..9 { - phase.insert_chunk(ValidatorIndex(i), dummy_chunk.clone()); - } - phase.total_received_responses += 8; - // error rate: 1/10 - // remaining chunks needed: threshold (34) - 9 - // expected: 24 * (1+ 1/10) = (next greater integer) = 27 - assert_eq!(phase.get_desired_request_count(threshold), 27); - phase.insert_chunk(ValidatorIndex(9), dummy_chunk.clone()); - phase.error_count = 0; - // With error count zero - we should fetch exactly as needed: - assert_eq!(phase.get_desired_request_count(threshold), threshold - phase.chunk_count()); -} From 166aae5e9c8d8b377d5227c444d7e4d5aeb50cb0 Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 8 Sep 2023 11:32:58 +0300 Subject: [PATCH 002/126] add copright header --- .../network/availability-recovery/src/task.rs | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 854dc26b4dd5..8046c1a8ff96 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -1,3 +1,21 @@ +// Copyright (C) Parity Technologies (UK) Ltd. +// This file is part of Polkadot. + +// Polkadot is free software: you can redistribute it and/or modify +// it under the terms of the GNU General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. + +// Polkadot is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU General Public License for more details. + +// You should have received a copy of the GNU General Public License +// along with Polkadot. If not, see . + +//! Recovery task and associated strategies. + use crate::{ futures_undead::FuturesUndead, is_chunk_valid, is_unavailable, metrics::Metrics, ErasureTask, LOG_TARGET, From 518d8fd7941682ea866a0e3edd55bfe392c48dc6 Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 8 Sep 2023 13:35:08 +0300 Subject: [PATCH 003/126] fix clippy --- polkadot/node/network/availability-recovery/src/task.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 8046c1a8ff96..927c96092222 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -482,7 +482,7 @@ impl RecoveryStrategy { } } - fn pop_first(self: Self) -> (Self, Self) { + fn pop_first(self) -> (Self, Self) { match self { Self::Nil => (Self::Nil, Self::Nil), Self::FullFromBackers(inner, next) => From dae34a535eebfb53637ec2d9476f677611931527 Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 8 Sep 2023 17:11:13 +0300 Subject: [PATCH 004/126] Refactor RecoveryStrategy using dynamic dispatch --- Cargo.lock | 1 + .../network/availability-recovery/Cargo.toml | 1 + .../network/availability-recovery/src/lib.rs | 46 +- .../network/availability-recovery/src/task.rs | 491 +++++++----------- 4 files changed, 221 insertions(+), 318 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index a95a7e2561d6..70c334446ba4 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -11570,6 +11570,7 @@ name = "polkadot-availability-recovery" version = "1.0.0" dependencies = [ "assert_matches", + "async-trait", "env_logger 0.9.3", "fatality", "futures", diff --git a/polkadot/node/network/availability-recovery/Cargo.toml b/polkadot/node/network/availability-recovery/Cargo.toml index 07ff09c7e70e..42c3abef547b 100644 --- a/polkadot/node/network/availability-recovery/Cargo.toml +++ b/polkadot/node/network/availability-recovery/Cargo.toml @@ -11,6 +11,7 @@ schnellru = "0.2.1" rand = "0.8.5" fatality = "0.0.6" thiserror = "1.0.48" +async-trait = "0.1.73" gum = { package = "tracing-gum", path = "../../gum" } polkadot-erasure-coding = { path = "../../../erasure-coding" } diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index 63bfc82e3d7b..aaf149cdb6a1 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -18,7 +18,12 @@ #![warn(missing_docs)] -use std::{collections::HashMap, iter::Iterator, num::NonZeroUsize, pin::Pin}; +use std::{ + collections::{HashMap, VecDeque}, + iter::Iterator, + num::NonZeroUsize, + pin::Pin, +}; use futures::{ channel::oneshot, @@ -30,7 +35,7 @@ use futures::{ task::{Context, Poll}, }; use schnellru::{ByLength, LruMap}; -use task::{FetchChunksParams, FetchFullParams}; +use task::{FetchChunks, FetchChunksParams, FetchFull, FetchFullParams}; use fatality::Nested; use polkadot_erasure_coding::{ @@ -47,8 +52,8 @@ use polkadot_node_subsystem::{ errors::RecoveryError, jaeger, messages::{AvailabilityRecoveryMessage, AvailabilityStoreMessage}, - overseer, ActiveLeavesUpdate, FromOrchestra, OverseerSignal, SpawnedSubsystem, SubsystemError, - SubsystemResult, + overseer, ActiveLeavesUpdate, FromOrchestra, OverseerSignal, SpawnedSubsystem, + SubsystemContext, SubsystemError, SubsystemResult, }; use polkadot_node_subsystem_util::request_session_info; use polkadot_primitives::{ @@ -103,7 +108,7 @@ pub struct AvailabilityRecoverySubsystem { } /// Expensive erasure coding computations that we want to run on a blocking thread. -enum ErasureTask { +pub enum ErasureTask { /// Reconstructs `AvailableData` from chunks given `n_validators`. Reconstruct( usize, @@ -287,7 +292,6 @@ struct State { /// An LRU cache of recently recovered data. availability_lru: LruMap, - // TODO: an LRU cache of erasure indices shuffling of all validators (per relay-parent). } impl Default for State { @@ -338,7 +342,7 @@ async fn launch_recovery_task( receipt: CandidateReceipt, response_sender: oneshot::Sender>, metrics: &Metrics, - recovery_strategy: RecoveryStrategy, + recovery_strategies: VecDeque::Sender>>>, bypass_availability_store: bool, ) -> error::Result<()> { let candidate_hash = receipt.hash(); @@ -352,7 +356,7 @@ async fn launch_recovery_task( bypass_availability_store, }; - let recovery_task = RecoveryTask::new(ctx.sender().clone(), params, recovery_strategy); + let recovery_task = RecoveryTask::new(ctx.sender().clone(), params, recovery_strategies); let (remote, remote_handle) = recovery_task.run().remote_handle(); @@ -457,24 +461,26 @@ async fn handle_recover( erasure_task_tx: erasure_task_tx.clone(), }; - let recovery_strategy = if let Some(backing_validators) = backing_validators { + let mut recovery_strategies: VecDeque< + Box::Sender>>, + > = VecDeque::with_capacity(2); + + if let Some(backing_validators) = backing_validators { match recovery_strategy_kind { RecoveryStrategyKind::BackersFirstAlways | RecoveryStrategyKind::BackersFirstIfSizeLower(_) | - RecoveryStrategyKind::BypassAvailabilityStore => RecoveryStrategy::new() - .then_fetch_full_from_backers(FetchFullParams { + RecoveryStrategyKind::BypassAvailabilityStore => + recovery_strategies.push_back(Box::new(FetchFull::new(FetchFullParams { group_name: "backers", validators: backing_validators.to_vec(), skip_if: skip_backing_group_if, erasure_task_tx, - }) - .then_fetch_chunks_from_validators(fetch_chunks_params), - RecoveryStrategyKind::ChunksAlways => RecoveryStrategy::new() - .then_fetch_chunks_from_validators(fetch_chunks_params), - } - } else { - RecoveryStrategy::new().then_fetch_chunks_from_validators(fetch_chunks_params) - }; + }))), + RecoveryStrategyKind::ChunksAlways => {}, + }; + } + + recovery_strategies.push_back(Box::new(FetchChunks::new(fetch_chunks_params))); launch_recovery_task( state, @@ -483,7 +489,7 @@ async fn handle_recover( receipt, response_sender, metrics, - *recovery_strategy, + recovery_strategies, recovery_strategy_kind == RecoveryStrategyKind::BypassAvailabilityStore, ) .await diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 927c96092222..339cd0c18490 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -16,6 +16,8 @@ //! Recovery task and associated strategies. +#![warn(missing_docs)] + use crate::{ futures_undead::FuturesUndead, is_chunk_valid, is_unavailable, metrics::Metrics, ErasureTask, LOG_TARGET, @@ -56,33 +58,50 @@ const TIMEOUT_START_NEW_REQUESTS: Duration = CHUNK_REQUEST_TIMEOUT; #[cfg(test)] const TIMEOUT_START_NEW_REQUESTS: Duration = Duration::from_millis(100); +#[async_trait::async_trait] +/// Common trait for runnable recovery strategies. +pub trait RecoveryStrategy: Send { + /// Main entry point of the strategy. + async fn run( + &mut self, + state: &mut State, + sender: &mut Sender, + common_params: &RecoveryParams, + ) -> Result; + + /// Return the name of the strategy for logging purposes. + fn display_name(&self) -> &'static str; +} + +/// Recovery parameters common to all strategies in a `RecoveryTask`. pub struct RecoveryParams { /// Discovery ids of `validators`. - pub(crate) validator_authority_keys: Vec, + pub validator_authority_keys: Vec, /// Number of validators relevant to this `RecoveryTask`. - pub(crate) n_validators: usize, + pub n_validators: usize, - /// The number of pieces needed. - pub(crate) threshold: usize, + /// The number of chunks needed. + pub threshold: usize, /// A hash of the relevant candidate. - pub(crate) candidate_hash: CandidateHash, + pub candidate_hash: CandidateHash, - /// The root of the erasure encoding of the para block. - pub(crate) erasure_root: Hash, + /// The root of the erasure encoding of the candidate. + pub erasure_root: Hash, - /// Metrics to report - pub(crate) metrics: Metrics, + /// Metrics to report. + pub metrics: Metrics, - /// Do not request data from availability-store - pub(crate) bypass_availability_store: bool, + /// Do not request data from availability-store. Useful for collators. + pub bypass_availability_store: bool, } -/// Represents intermediate data that must be passed between `RecoveryStrategy`s belonging to the -/// same `RecoveryTask` or data that is used by state methods common to multiple RecoveryStrategies. +/// Intermediate/common data that must be passed between `RecoveryStrategy`s belonging to the +/// same `RecoveryTask`. pub struct State { /// Chunks received so far. received_chunks: HashMap, + /// Collection of in-flight requests. requesting_chunks: FuturesUndead, (ValidatorIndex, RequestError)>>, } @@ -99,6 +118,7 @@ impl State { self.received_chunks.len() } + /// Retrieve the local chunks held in the av-store (either 0 or 1). async fn populate_from_av_store( &mut self, params: &RecoveryParams, @@ -146,6 +166,7 @@ impl State { } } + /// Launch chunk requests in parallel, according to the parameters. async fn launch_parallel_chunk_requests( &mut self, params: &RecoveryParams, @@ -314,24 +335,26 @@ impl State { /// A stateful reconstruction of availability data in reference to /// a candidate hash. -pub struct RecoveryTask { +pub struct RecoveryTask { sender: Sender, - /// The common parameters of the recovery process, regardless of the strategy. params: RecoveryParams, - strategy: RecoveryStrategy, + strategies: VecDeque>>, state: State, } -impl RecoveryTask { - pub fn new(sender: Sender, params: RecoveryParams, strategy: RecoveryStrategy) -> Self { - Self { sender, params, strategy, state: State::new() } - } -} - impl RecoveryTask where Sender: overseer::AvailabilityRecoverySenderTrait, { + /// Instantiate a new recovery task. + pub fn new( + sender: Sender, + params: RecoveryParams, + strategies: VecDeque>>, + ) -> Self { + Self { sender, params, strategies, state: State::new() } + } + async fn in_availability_store(&mut self) -> Option { if !self.params.bypass_availability_store { let (tx, rx) = oneshot::channel(); @@ -358,6 +381,8 @@ where None } + /// Run this recovery task to completion. It will loop through the configured strategies + /// in-order and return whenever the first one recovers the full `AvailableData`. pub async fn run(mut self) -> Result { if let Some(data) = self.in_availability_store().await { return Ok(data) @@ -368,57 +393,44 @@ where let _timer = self.params.metrics.time_full_recovery(); let res = loop { - let (current_strategy, next_strategy) = self.strategy.pop_first(); - self.strategy = next_strategy; - - // Make sure we are not referencing futures from past RecoveryStrategy runs. - if self.state.requesting_chunks.total_len() != 0 { - self.state.requesting_chunks = FuturesUndead::new(); - } - - let recovery_strategy_name = current_strategy.display_name(); + if let Some(mut current_strategy) = self.strategies.pop_front() { + // Make sure we are not referencing futures from past RecoveryStrategy runs. + if self.state.requesting_chunks.total_len() != 0 { + self.state.requesting_chunks = FuturesUndead::new(); + } - if let Some(name) = recovery_strategy_name { gum::info!( target: LOG_TARGET, candidate_hash = ?self.params.candidate_hash, "Starting `{}` strategy", - &name, + current_strategy.display_name(), ); - } - let res = match current_strategy { - RecoveryStrategy::Nil => Err(RecoveryError::Unavailable), - RecoveryStrategy::FullFromBackers(inner, _) => - inner.run(&mut self.state, &mut self.sender, &self.params).await, - RecoveryStrategy::ChunksFromValidators(inner, _) => - inner.run(&mut self.state, &mut self.sender, &self.params).await, - }; - - match res { - Err(RecoveryError::Unavailable) => { - if !matches!(&self.strategy, RecoveryStrategy::Nil) { - if let Some(recovery_strategy_name) = recovery_strategy_name { + let res = + current_strategy.run(&mut self.state, &mut self.sender, &self.params).await; + + match res { + Err(RecoveryError::Unavailable) => + if self.strategies.front().is_some() { gum::warn!( target: LOG_TARGET, candidate_hash = ?self.params.candidate_hash, "Recovery strategy `{}` did not conclude. Trying the next one.", - recovery_strategy_name, + current_strategy.display_name(), ); - } - continue - } else { - // We have no other strategies to try. - gum::error!( - target: LOG_TARGET, - candidate_hash = ?self.params.candidate_hash, - "Recovery of available data failed.", - ); - break Err(RecoveryError::Unavailable) - } - }, - Err(err) => break Err(err), - Ok(data) => break Ok(data), + continue + }, + Err(err) => break Err(err), + Ok(data) => break Ok(data), + } + } else { + // We have no other strategies to try. + gum::error!( + target: LOG_TARGET, + candidate_hash = ?self.params.candidate_hash, + "Recovery of available data failed.", + ); + break Err(RecoveryError::Unavailable) } }; @@ -432,86 +444,40 @@ where } } -pub enum RecoveryStrategy { - Nil, - FullFromBackers(FetchFull, Box), - ChunksFromValidators(FetchChunks, Box), -} - -impl RecoveryStrategy { - pub fn new() -> Box { - Box::new(RecoveryStrategy::Nil) - } - - fn display_name(&self) -> Option<&'static str> { - match self { - Self::Nil => None, - Self::FullFromBackers(_, _) => Some("Full recovery from backers"), - Self::ChunksFromValidators(_, _) => Some("Chunks recovery"), - } - } - - pub fn then_fetch_full_from_backers(self: Box, params: FetchFullParams) -> Box { - match *self { - Self::Nil => Box::new(Self::FullFromBackers(FetchFull::new(params), self)), - Self::ChunksFromValidators(task, next) => { - let next = next.then_fetch_full_from_backers(params); - Box::new(Self::ChunksFromValidators(task, next)) - }, - Self::FullFromBackers(task, next) => { - let next = next.then_fetch_full_from_backers(params); - Box::new(Self::FullFromBackers(task, next)) - }, - } - } - - pub fn then_fetch_chunks_from_validators( - self: Box, - params: FetchChunksParams, - ) -> Box { - match *self { - Self::Nil => Box::new(Self::ChunksFromValidators(FetchChunks::new(params), self)), - Self::ChunksFromValidators(task, next) => { - let next = next.then_fetch_chunks_from_validators(params); - Box::new(Self::ChunksFromValidators(task, next)) - }, - Self::FullFromBackers(task, next) => { - let next = next.then_fetch_chunks_from_validators(params); - Box::new(Self::FullFromBackers(task, next)) - }, - } - } - - fn pop_first(self) -> (Self, Self) { - match self { - Self::Nil => (Self::Nil, Self::Nil), - Self::FullFromBackers(inner, next) => - (Self::FullFromBackers(inner, Box::new(Self::Nil)), *next), - Self::ChunksFromValidators(inner, next) => - (Self::ChunksFromValidators(inner, Box::new(Self::Nil)), *next), - } - } -} - +/// `RecoveryStrategy` that sequentially tries to fetch the full `AvailableData` from +/// already-connected validators in the configured validator set. pub struct FetchFull { params: FetchFullParams, } pub struct FetchFullParams { - pub(crate) group_name: &'static str, - pub(crate) validators: Vec, - pub(crate) skip_if: Box bool + Send>, - // channel to the erasure task handler. - pub(crate) erasure_task_tx: futures::channel::mpsc::Sender, + /// Name of the validator group used for recovery. For logging purposes. + /// (e.g."backers"/"approval-checkers") + pub group_name: &'static str, + /// Validators that will be used for fetching the data. + pub validators: Vec, + /// Predicate that if is true, will result in skipping this strategy. + pub skip_if: Box bool + Send>, + /// Channel to the erasure task handler. + pub erasure_task_tx: futures::channel::mpsc::Sender, } impl FetchFull { - fn new(params: FetchFullParams) -> Self { + /// Create a new `FetchFull` recovery strategy. + pub fn new(mut params: FetchFullParams) -> Self { + params.validators.shuffle(&mut rand::thread_rng()); Self { params } } +} + +#[async_trait::async_trait] +impl RecoveryStrategy for FetchFull { + fn display_name(&self) -> &'static str { + "Full recovery from backers" + } - async fn run( - mut self, + async fn run( + &mut self, _: &mut State, sender: &mut Sender, common_params: &RecoveryParams, @@ -606,30 +572,32 @@ impl FetchFull { } } +/// `RecoveryStrategy` that requests chunks from validators, in parallel. pub struct FetchChunks { - /// How many request have been unsuccessful so far. + /// How many requests have been unsuccessful so far. error_count: usize, - /// Total number of responses that have been received. - /// - /// including failed ones. + /// Total number of responses that have been received, including failed ones. total_received_responses: usize, - /// a random shuffling of the validators which indicates the order in which we connect to the + /// A random shuffling of the validators which indicates the order in which we connect to the /// validators and request the chunk from them. validators: VecDeque, - // channel to the erasure task handler. + /// Channel to the erasure task handler. erasure_task_tx: futures::channel::mpsc::Sender, } +/// Parameters specific to the `FetchChunks` strategy. pub struct FetchChunksParams { - pub(crate) n_validators: usize, - // channel to the erasure task handler. - pub(crate) erasure_task_tx: futures::channel::mpsc::Sender, + /// Total number of validators. + pub n_validators: usize, + /// Channel to the erasure task handler. + pub erasure_task_tx: futures::channel::mpsc::Sender, } impl FetchChunks { - fn new(params: FetchChunksParams) -> Self { + /// Instantiate a new strategy. + pub fn new(params: FetchChunksParams) -> Self { let mut shuffling: Vec<_> = (0..params.n_validators) .map(|i| ValidatorIndex(i.try_into().expect("number of validators must fit in a u32"))) .collect(); @@ -677,8 +645,91 @@ impl FetchChunks { ) } - async fn run( - mut self, + async fn attempt_recovery( + &mut self, + state: &mut State, + common_params: &RecoveryParams, + ) -> Result { + let recovery_duration = common_params.metrics.time_erasure_recovery(); + + // Send request to reconstruct available data from chunks. + let (avilable_data_tx, available_data_rx) = oneshot::channel(); + self.erasure_task_tx + .send(ErasureTask::Reconstruct( + common_params.n_validators, + // Safe to leave an empty vec in place, as we're stopping the recovery process if + // this reconstruct fails. + std::mem::take(&mut state.received_chunks), + avilable_data_tx, + )) + .await + .map_err(|_| RecoveryError::ChannelClosed)?; + + let available_data_response = + available_data_rx.await.map_err(|_| RecoveryError::ChannelClosed)?; + + match available_data_response { + Ok(data) => { + // Send request to re-encode the chunks and check merkle root. + let (reencode_tx, reencode_rx) = oneshot::channel(); + self.erasure_task_tx + .send(ErasureTask::Reencode( + common_params.n_validators, + common_params.erasure_root, + data, + reencode_tx, + )) + .await + .map_err(|_| RecoveryError::ChannelClosed)?; + + let reencode_response = + reencode_rx.await.map_err(|_| RecoveryError::ChannelClosed)?; + + if let Some(data) = reencode_response { + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + erasure_root = ?common_params.erasure_root, + "Data recovery from chunks complete", + ); + + Ok(data) + } else { + recovery_duration.map(|rd| rd.stop_and_discard()); + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + erasure_root = ?common_params.erasure_root, + "Data recovery error - root mismatch", + ); + + Err(RecoveryError::Invalid) + } + }, + Err(err) => { + recovery_duration.map(|rd| rd.stop_and_discard()); + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + erasure_root = ?common_params.erasure_root, + ?err, + "Data recovery error ", + ); + + Err(RecoveryError::Invalid) + }, + } + } +} + +#[async_trait::async_trait] +impl RecoveryStrategy for FetchChunks { + fn display_name(&self) -> &'static str { + "Fetch chunks" + } + + async fn run( + &mut self, state: &mut State, sender: &mut Sender, common_params: &RecoveryParams, @@ -764,168 +815,12 @@ impl FetchChunks { self.error_count += error_count; } } - - async fn attempt_recovery( - &mut self, - state: &mut State, - common_params: &RecoveryParams, - ) -> Result { - let recovery_duration = common_params.metrics.time_erasure_recovery(); - - // Send request to reconstruct available data from chunks. - let (avilable_data_tx, available_data_rx) = oneshot::channel(); - self.erasure_task_tx - .send(ErasureTask::Reconstruct( - common_params.n_validators, - // Safe to leave an empty vec in place, as we're stopping the recovery process if - // this reconstruct fails. - std::mem::take(&mut state.received_chunks), - avilable_data_tx, - )) - .await - .map_err(|_| RecoveryError::ChannelClosed)?; - - let available_data_response = - available_data_rx.await.map_err(|_| RecoveryError::ChannelClosed)?; - - match available_data_response { - Ok(data) => { - // Send request to re-encode the chunks and check merkle root. - let (reencode_tx, reencode_rx) = oneshot::channel(); - self.erasure_task_tx - .send(ErasureTask::Reencode( - common_params.n_validators, - common_params.erasure_root, - data, - reencode_tx, - )) - .await - .map_err(|_| RecoveryError::ChannelClosed)?; - - let reencode_response = - reencode_rx.await.map_err(|_| RecoveryError::ChannelClosed)?; - - if let Some(data) = reencode_response { - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?common_params.candidate_hash, - erasure_root = ?common_params.erasure_root, - "Data recovery from chunks complete", - ); - - Ok(data) - } else { - recovery_duration.map(|rd| rd.stop_and_discard()); - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?common_params.candidate_hash, - erasure_root = ?common_params.erasure_root, - "Data recovery error - root mismatch", - ); - - Err(RecoveryError::Invalid) - } - }, - Err(err) => { - recovery_duration.map(|rd| rd.stop_and_discard()); - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?common_params.candidate_hash, - erasure_root = ?common_params.erasure_root, - ?err, - "Data recovery error ", - ); - - Err(RecoveryError::Invalid) - }, - } - } } #[cfg(test)] mod tests { - use std::ops::Deref; - use super::*; - use assert_matches::assert_matches; use polkadot_erasure_coding::recovery_threshold; - use RecoveryStrategy::*; - - impl std::fmt::Debug for RecoveryStrategy { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - match self { - Nil => write!(f, "Nil"), - ChunksFromValidators(_, next) => - write!(f, "{:?} -> {}", self.display_name(), format!("{next:?}")), - FullFromBackers(_, next) => - write!(f, "{:?} -> {}", self.display_name(), format!("{next:?}")), - } - } - } - - #[test] - fn test_recovery_strategy_linked_list_ops() { - let fetch_full_params = FetchFullParams { - group_name: "backers", - validators: vec![], - skip_if: Box::new(|| true), - erasure_task_tx: futures::channel::mpsc::channel(0).0, - }; - let fetch_full_params_2 = FetchFullParams { - group_name: "approval-checkers", - validators: vec![], - skip_if: Box::new(|| true), - erasure_task_tx: futures::channel::mpsc::channel(0).0, - }; - - let fetch_chunks_params = FetchChunksParams { - n_validators: 2, - erasure_task_tx: futures::channel::mpsc::channel(0).0, - }; - let fetch_chunks_params_2 = FetchChunksParams { - n_validators: 3, - erasure_task_tx: futures::channel::mpsc::channel(0).0, - }; - let recovery_strategy = RecoveryStrategy::new() - .then_fetch_full_from_backers(fetch_full_params) - .then_fetch_full_from_backers(fetch_full_params_2) - .then_fetch_chunks_from_validators(fetch_chunks_params) - .then_fetch_chunks_from_validators(fetch_chunks_params_2); - - // Check that the builder correctly chains strategies. - assert_matches!( - recovery_strategy.deref(), - FullFromBackers(_, next) - if matches!(next.deref(), FullFromBackers(_, next) - if matches!(next.deref(), ChunksFromValidators(_, next) - if matches!(next.deref(), ChunksFromValidators(_, next) - if matches!(next.deref(), Nil) - ) - ) - ) - ); - - // Check the order for the `pop_first` operation. - let (current, next) = recovery_strategy.pop_first(); - assert_matches!(current, FullFromBackers(task, next) if task.params.group_name == "backers" && matches!(*next, Nil)); - assert_matches!(&next, FullFromBackers(task, _) if task.params.group_name == "approval-checkers"); - - let (current, next) = next.pop_first(); - assert_matches!(current, FullFromBackers(task, next) if task.params.group_name == "approval-checkers" && matches!(*next, Nil)); - assert_matches!(&next, ChunksFromValidators(task, _) if task.validators.len() == 2); - - let (current, next) = next.pop_first(); - assert_matches!(current, ChunksFromValidators(task, next) if task.validators.len() == 2 && matches!(*next, Nil)); - assert_matches!(&next, ChunksFromValidators(task, _) if task.validators.len() == 3); - - let (current, next) = next.pop_first(); - assert_matches!(current, ChunksFromValidators(task, next) if task.validators.len() == 3 && matches!(*next, Nil)); - assert_matches!(&next, Nil); - - let (current, next) = next.pop_first(); - assert_matches!(current, Nil); - assert_matches!(next, Nil); - } #[test] fn parallel_request_calculation_works_as_expected() { From 640c2d59778bb3103183ae8977b9222d28acc27e Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 12 Sep 2023 14:47:39 +0300 Subject: [PATCH 005/126] address comments --- .../network/availability-recovery/src/lib.rs | 18 ++- .../network/availability-recovery/src/task.rs | 113 +++++++----------- 2 files changed, 52 insertions(+), 79 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index aaf149cdb6a1..04b71147c9f0 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -424,7 +424,7 @@ async fn handle_recover( let _span = span.child("session-info-ctx-received"); match session_info { Some(session_info) => { - let mut skip_backing_group_if: Box bool + Send> = Box::new(|| false); + let mut prefer_backing_group = true; if let RecoveryStrategyKind::BackersFirstIfSizeLower(small_pov_limit) = recovery_strategy_kind @@ -435,7 +435,7 @@ async fn handle_recover( if let Ok(Some(chunk_size)) = chunk_size { let pov_size_estimate = chunk_size.saturating_mul(session_info.validators.len()) / 3; - let prefer_backing_group = pov_size_estimate < small_pov_limit; + prefer_backing_group = pov_size_estimate < small_pov_limit; gum::trace!( target: LOG_TARGET, @@ -445,8 +445,6 @@ async fn handle_recover( enabled = prefer_backing_group, "Prefer fetch from backing group", ); - - skip_backing_group_if = Box::new(move || !prefer_backing_group); } }; @@ -466,17 +464,15 @@ async fn handle_recover( > = VecDeque::with_capacity(2); if let Some(backing_validators) = backing_validators { - match recovery_strategy_kind { - RecoveryStrategyKind::BackersFirstAlways | - RecoveryStrategyKind::BackersFirstIfSizeLower(_) | - RecoveryStrategyKind::BypassAvailabilityStore => + match (&recovery_strategy_kind, prefer_backing_group) { + (RecoveryStrategyKind::BackersFirstAlways, true) | + (RecoveryStrategyKind::BackersFirstIfSizeLower(_), true) | + (RecoveryStrategyKind::BypassAvailabilityStore, true) => recovery_strategies.push_back(Box::new(FetchFull::new(FetchFullParams { - group_name: "backers", validators: backing_validators.to_vec(), - skip_if: skip_backing_group_if, erasure_task_tx, }))), - RecoveryStrategyKind::ChunksAlways => {}, + _ => {}, }; } diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 339cd0c18490..30eaf7734a52 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -78,7 +78,7 @@ pub struct RecoveryParams { /// Discovery ids of `validators`. pub validator_authority_keys: Vec, - /// Number of validators relevant to this `RecoveryTask`. + /// Number of validators. pub n_validators: usize, /// The number of chunks needed. @@ -96,6 +96,7 @@ pub struct RecoveryParams { /// Do not request data from availability-store. Useful for collators. pub bypass_availability_store: bool, } + /// Intermediate/common data that must be passed between `RecoveryStrategy`s belonging to the /// same `RecoveryTask`. pub struct State { @@ -392,55 +393,55 @@ where let _timer = self.params.metrics.time_full_recovery(); - let res = loop { - if let Some(mut current_strategy) = self.strategies.pop_front() { - // Make sure we are not referencing futures from past RecoveryStrategy runs. - if self.state.requesting_chunks.total_len() != 0 { - self.state.requesting_chunks = FuturesUndead::new(); - } + while let Some(mut current_strategy) = self.strategies.pop_front() { + // Make sure we are not referencing futures from past RecoveryStrategy runs. + if self.state.requesting_chunks.total_len() != 0 { + self.state.requesting_chunks = FuturesUndead::new(); + } - gum::info!( - target: LOG_TARGET, - candidate_hash = ?self.params.candidate_hash, - "Starting `{}` strategy", - current_strategy.display_name(), - ); + gum::debug!( + target: LOG_TARGET, + candidate_hash = ?self.params.candidate_hash, + "Starting `{}` strategy", + current_strategy.display_name(), + ); - let res = - current_strategy.run(&mut self.state, &mut self.sender, &self.params).await; + let res = current_strategy.run(&mut self.state, &mut self.sender, &self.params).await; - match res { - Err(RecoveryError::Unavailable) => - if self.strategies.front().is_some() { - gum::warn!( - target: LOG_TARGET, - candidate_hash = ?self.params.candidate_hash, - "Recovery strategy `{}` did not conclude. Trying the next one.", - current_strategy.display_name(), - ); - continue - }, - Err(err) => break Err(err), - Ok(data) => break Ok(data), - } - } else { - // We have no other strategies to try. - gum::error!( - target: LOG_TARGET, - candidate_hash = ?self.params.candidate_hash, - "Recovery of available data failed.", - ); - break Err(RecoveryError::Unavailable) + match res { + Err(RecoveryError::Unavailable) => + if self.strategies.front().is_some() { + gum::debug!( + target: LOG_TARGET, + candidate_hash = ?self.params.candidate_hash, + "Recovery strategy `{}` did not conclude. Trying the next one.", + current_strategy.display_name(), + ); + continue + }, + Err(err) => { + match &err { + RecoveryError::Invalid => self.params.metrics.on_recovery_invalid(), + _ => self.params.metrics.on_recovery_failed(), + } + return Err(err) + }, + Ok(data) => { + self.params.metrics.on_recovery_succeeded(); + return Ok(data) + }, } - }; - - match &res { - Ok(_) => self.params.metrics.on_recovery_succeeded(), - Err(RecoveryError::Invalid) => self.params.metrics.on_recovery_invalid(), - Err(_) => self.params.metrics.on_recovery_failed(), } - res + // We have no other strategies to try. + gum::warn!( + target: LOG_TARGET, + candidate_hash = ?self.params.candidate_hash, + "Recovery of available data failed.", + ); + self.params.metrics.on_recovery_failed(); + + Err(RecoveryError::Unavailable) } } @@ -451,13 +452,8 @@ pub struct FetchFull { } pub struct FetchFullParams { - /// Name of the validator group used for recovery. For logging purposes. - /// (e.g."backers"/"approval-checkers") - pub group_name: &'static str, /// Validators that will be used for fetching the data. pub validators: Vec, - /// Predicate that if is true, will result in skipping this strategy. - pub skip_if: Box bool + Send>, /// Channel to the erasure task handler. pub erasure_task_tx: futures::channel::mpsc::Sender, } @@ -482,25 +478,6 @@ impl RecoveryStrategy sender: &mut Sender, common_params: &RecoveryParams, ) -> Result { - if (self.params.skip_if)() { - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?common_params.candidate_hash, - erasure_root = ?common_params.erasure_root, - "Skipping requesting availability data from {}", - self.params.group_name - ); - - return Err(RecoveryError::Unavailable) - } - - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?common_params.candidate_hash, - erasure_root = ?common_params.erasure_root, - "Requesting full availability data from {}", - self.params.group_name - ); loop { // Pop the next validator, and proceed to next fetch_chunks_task if we're out. let validator_index = From 2269f76f0705b84e6b36cd409d46ea8e579d80d2 Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 12 Sep 2023 17:40:25 +0300 Subject: [PATCH 006/126] erasure-coding: add algorithm for systematic recovery Signed-off-by: alindima --- Cargo.lock | 39 ++--------- polkadot/erasure-coding/Cargo.toml | 3 +- polkadot/erasure-coding/src/lib.rs | 105 ++++++++++++++++++++++++++++- 3 files changed, 113 insertions(+), 34 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 92943339c7db..a0456fc2c337 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -11637,6 +11637,7 @@ dependencies = [ "parity-scale-codec", "polkadot-node-primitives", "polkadot-primitives", + "quickcheck", "reed-solomon-novelpoly", "sp-core", "sp-trie", @@ -13783,14 +13784,13 @@ dependencies = [ [[package]] name = "reed-solomon-novelpoly" -version = "1.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3bd8f48b2066e9f69ab192797d66da804d1935bf22763204ed3675740cb0f221" +version = "1.0.1-alpha.0" +source = "git+https://github.com/paritytech/reed-solomon-novelpoly.git?branch=alindima/expose-code-params-fields#40b38eff038ee907d4e0732f28d6b8c5cf043ca5" dependencies = [ "derive_more", "fs-err", - "itertools 0.10.5", - "static_init 0.5.2", + "itertools 0.11.0", + "static_init", "thiserror", ] @@ -15605,7 +15605,7 @@ dependencies = [ "sp-transaction-storage-proof", "sp-trie", "sp-version", - "static_init 1.0.3", + "static_init", "substrate-prometheus-endpoint", "substrate-test-runtime", "substrate-test-runtime-client", @@ -17830,18 +17830,6 @@ version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" -[[package]] -name = "static_init" -version = "0.5.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "11b73400442027c4adedda20a9f9b7945234a5bd8d5f7e86da22bd5d0622369c" -dependencies = [ - "cfg_aliases", - "libc", - "parking_lot 0.11.2", - "static_init_macro 0.5.0", -] - [[package]] name = "static_init" version = "1.0.3" @@ -17853,23 +17841,10 @@ dependencies = [ "libc", "parking_lot 0.11.2", "parking_lot_core 0.8.6", - "static_init_macro 1.0.2", + "static_init_macro", "winapi", ] -[[package]] -name = "static_init_macro" -version = "0.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2261c91034a1edc3fc4d1b80e89d82714faede0515c14a75da10cb941546bbf" -dependencies = [ - "cfg_aliases", - "memchr", - "proc-macro2", - "quote", - "syn 1.0.109", -] - [[package]] name = "static_init_macro" version = "1.0.2" diff --git a/polkadot/erasure-coding/Cargo.toml b/polkadot/erasure-coding/Cargo.toml index d07b77ec4ddf..c6ad2988360a 100644 --- a/polkadot/erasure-coding/Cargo.toml +++ b/polkadot/erasure-coding/Cargo.toml @@ -8,7 +8,7 @@ license.workspace = true [dependencies] polkadot-primitives = { path = "../primitives" } polkadot-node-primitives = { package = "polkadot-node-primitives", path = "../node/primitives" } -novelpoly = { package = "reed-solomon-novelpoly", version = "1.0.0" } +novelpoly = { package = "reed-solomon-novelpoly", git = "https://github.com/paritytech/reed-solomon-novelpoly.git", branch="alindima/expose-code-params-fields" } parity-scale-codec = { version = "3.6.1", default-features = false, features = ["std", "derive"] } sp-core = { path = "../../substrate/primitives/core" } sp-trie = { path = "../../substrate/primitives/trie" } @@ -16,6 +16,7 @@ thiserror = "1.0.48" [dev-dependencies] criterion = { version = "0.4.0", default-features = false, features = ["cargo_bench_support"] } +quickcheck = { version = "1.0.3", default-features = false } [[bench]] name = "scaling_with_validators" diff --git a/polkadot/erasure-coding/src/lib.rs b/polkadot/erasure-coding/src/lib.rs index 36847b463715..1f7e4e2363e3 100644 --- a/polkadot/erasure-coding/src/lib.rs +++ b/polkadot/erasure-coding/src/lib.rs @@ -113,6 +113,67 @@ fn code_params(n_validators: usize) -> Result { }) } +/// Reconstruct the v1 available data from the set of systematic chunks. +/// +/// Provide a vector containing chunk data. If too few chunks are provided, recovery is not +/// possible. +pub fn reconstruct_from_systematic_v1( + n_validators: usize, + chunks: Vec>, +) -> Result { + reconstruct_from_systematic(n_validators, chunks) +} + +/// Reconstruct the available data from the set of systematic chunks. +/// +/// Provide a vector containing chunk data. If too few chunks are provided, recovery is not +/// possible. +pub fn reconstruct_from_systematic( + n_validators: usize, + chunks: Vec>, +) -> Result { + let code_params = code_params(n_validators)?; + let kpow2 = code_params.k(); + + let Some(first_shard) = chunks.iter().next() else { return Err(Error::NotEnoughChunks) }; + let shard_len = first_shard.len(); + + if shard_len == 0 { + return Err(Error::NonUniformChunks) + } + + if shard_len % 2 != 0 { + return Err(Error::UnevenLength) + } + + if chunks.len() <= kpow2 { + return Err(Error::NotEnoughChunks) + } + + let mut check_shard_len = true; + let mut systematic_bytes = Vec::with_capacity(shard_len * kpow2); + + for i in (0..shard_len).step_by(2) { + for chunk in chunks.iter().take(kpow2) { + if check_shard_len { + if chunk.len() != shard_len { + return Err(Error::NonUniformChunks) + } + } + + // No need to check for index out of bounds because i goes up to shard_len and + // we return an error for non uniform chunks. + systematic_bytes.push(chunk[i]); + systematic_bytes.push(chunk[i + 1]); + } + + // After the first check, stop checking the shard lengths. + check_shard_len = false; + } + + Decode::decode(&mut &systematic_bytes[..]).map_err(|_| Error::BadPayload) +} + /// Obtain erasure-coded chunks for v1 `AvailableData`, one for each validator. /// /// Works only up to 65536 validators, and `n_validators` must be non-zero. @@ -201,7 +262,7 @@ where Ok(payload_bytes) => payload_bytes, }; - Decode::decode(&mut &payload_bytes[..]).or_else(|_e| Err(Error::BadPayload)) + Decode::decode(&mut &payload_bytes[..]).map_err(|_| Error::BadPayload) } /// An iterator that yields merkle branches and chunk data for all chunks to @@ -346,13 +407,41 @@ impl<'a, I: Iterator> parity_scale_codec::Input for ShardInput< #[cfg(test)] mod tests { + use std::sync::Arc; + use super::*; use polkadot_node_primitives::{AvailableData, BlockData, PoV}; + use polkadot_primitives::{HeadData, PersistedValidationData}; + use quickcheck::{Arbitrary, Gen, QuickCheck}; // In order to adequately compute the number of entries in the Merkle // trie, we must account for the fixed 16-ary trie structure. const KEY_INDEX_NIBBLE_SIZE: usize = 4; + #[derive(Clone, Debug)] + struct ArbitraryAvailableData(AvailableData); + + impl Arbitrary for ArbitraryAvailableData { + fn arbitrary(g: &mut Gen) -> Self { + // Limit the POV len to 1 mib, otherwise the test will take forever + let pov_len = u32::arbitrary(g).saturating_add(2) % (1024 * 1024); + + let pov = (0..pov_len).map(|_| u8::arbitrary(g)).collect(); + + let pvd = PersistedValidationData { + parent_head: HeadData((0..u16::arbitrary(g)).map(|_| u8::arbitrary(g)).collect()), + relay_parent_number: u32::arbitrary(g), + relay_parent_storage_root: [u8::arbitrary(g); 32].into(), + max_pov_size: u32::arbitrary(g), + }; + + ArbitraryAvailableData(AvailableData { + pov: Arc::new(PoV { block_data: BlockData(pov) }), + validation_data: pvd, + }) + } + } + #[test] fn field_order_is_right_size() { assert_eq!(MAX_VALIDATORS, 65536); @@ -379,6 +468,20 @@ mod tests { assert_eq!(reconstructed, available_data); } + #[test] + fn round_trip_systematic_works() { + fn property(available_data: ArbitraryAvailableData, n_validators: u16) { + let n_validators = n_validators.saturating_add(2); + let chunks = obtain_chunks(n_validators as usize, &available_data.0).unwrap(); + assert_eq!( + reconstruct_from_systematic_v1(n_validators as usize, chunks).unwrap(), + available_data.0 + ); + } + + QuickCheck::new().quickcheck(property as fn(ArbitraryAvailableData, u16)) + } + #[test] fn reconstruct_does_not_panic_on_low_validator_count() { let reconstructed = reconstruct_v1(1, [].iter().cloned()); From dca9ce43741a1971806391c5f1a182cbba1a7667 Mon Sep 17 00:00:00 2001 From: alindima Date: Wed, 13 Sep 2023 17:07:12 +0300 Subject: [PATCH 007/126] WIP --- Cargo.lock | 3 +- polkadot/erasure-coding/Cargo.toml | 2 +- polkadot/erasure-coding/src/lib.rs | 10 +- .../availability-distribution/src/error.rs | 6 +- .../src/requester/fetch_task/mod.rs | 5 +- .../src/requester/mod.rs | 152 ++++++--- .../src/requester/session_cache.rs | 45 ++- .../availability-recovery/src/error.rs | 11 + .../network/availability-recovery/src/lib.rs | 77 ++++- .../network/availability-recovery/src/task.rs | 305 +++++++++++++++++- polkadot/node/overseer/src/lib.rs | 1 + polkadot/node/subsystem-util/Cargo.toml | 1 + polkadot/node/subsystem-util/src/lib.rs | 32 +- polkadot/primitives/src/lib.rs | 36 +-- polkadot/primitives/src/v5/mod.rs | 2 + 15 files changed, 557 insertions(+), 131 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index a0456fc2c337..dd62b0ba2027 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -12303,6 +12303,7 @@ dependencies = [ "polkadot-primitives-test-helpers", "prioritized-metered-channel", "rand 0.8.5", + "rand_chacha 0.3.1", "sc-client-api", "schnellru", "sp-application-crypto", @@ -13785,7 +13786,7 @@ dependencies = [ [[package]] name = "reed-solomon-novelpoly" version = "1.0.1-alpha.0" -source = "git+https://github.com/paritytech/reed-solomon-novelpoly.git?branch=alindima/expose-code-params-fields#40b38eff038ee907d4e0732f28d6b8c5cf043ca5" +source = "git+https://github.com/paritytech/reed-solomon-novelpoly.git?rev=6864dbf2018a8a92d976835447ba29bcdc8af6a0#6864dbf2018a8a92d976835447ba29bcdc8af6a0" dependencies = [ "derive_more", "fs-err", diff --git a/polkadot/erasure-coding/Cargo.toml b/polkadot/erasure-coding/Cargo.toml index c6ad2988360a..d4959fb4e823 100644 --- a/polkadot/erasure-coding/Cargo.toml +++ b/polkadot/erasure-coding/Cargo.toml @@ -8,7 +8,7 @@ license.workspace = true [dependencies] polkadot-primitives = { path = "../primitives" } polkadot-node-primitives = { package = "polkadot-node-primitives", path = "../node/primitives" } -novelpoly = { package = "reed-solomon-novelpoly", git = "https://github.com/paritytech/reed-solomon-novelpoly.git", branch="alindima/expose-code-params-fields" } +novelpoly = { package = "reed-solomon-novelpoly", git = "https://github.com/paritytech/reed-solomon-novelpoly.git", rev="6864dbf2018a8a92d976835447ba29bcdc8af6a0" } parity-scale-codec = { version = "3.6.1", default-features = false, features = ["std", "derive"] } sp-core = { path = "../../substrate/primitives/core" } sp-trie = { path = "../../substrate/primitives/trie" } diff --git a/polkadot/erasure-coding/src/lib.rs b/polkadot/erasure-coding/src/lib.rs index 1f7e4e2363e3..7c28183e6f04 100644 --- a/polkadot/erasure-coding/src/lib.rs +++ b/polkadot/erasure-coding/src/lib.rs @@ -119,7 +119,7 @@ fn code_params(n_validators: usize) -> Result { /// possible. pub fn reconstruct_from_systematic_v1( n_validators: usize, - chunks: Vec>, + chunks: Vec<&[u8]>, ) -> Result { reconstruct_from_systematic(n_validators, chunks) } @@ -130,7 +130,7 @@ pub fn reconstruct_from_systematic_v1( /// possible. pub fn reconstruct_from_systematic( n_validators: usize, - chunks: Vec>, + chunks: Vec<&[u8]>, ) -> Result { let code_params = code_params(n_validators)?; let kpow2 = code_params.k(); @@ -474,7 +474,11 @@ mod tests { let n_validators = n_validators.saturating_add(2); let chunks = obtain_chunks(n_validators as usize, &available_data.0).unwrap(); assert_eq!( - reconstruct_from_systematic_v1(n_validators as usize, chunks).unwrap(), + reconstruct_from_systematic_v1( + n_validators as usize, + chunks.iter().map(|v| &v[..]).collect() + ) + .unwrap(), available_data.0 ); } diff --git a/polkadot/node/network/availability-distribution/src/error.rs b/polkadot/node/network/availability-distribution/src/error.rs index c547a1abbc27..bcb2404e7c11 100644 --- a/polkadot/node/network/availability-distribution/src/error.rs +++ b/polkadot/node/network/availability-distribution/src/error.rs @@ -82,6 +82,9 @@ pub enum Error { #[error("Given validator index could not be found in current session")] InvalidValidatorIndex, + + #[error("Cannot find block number for given relay parent")] + BlockNumberNotFound, } /// General result abbreviation type alias. @@ -104,7 +107,8 @@ pub fn log_error( JfyiError::InvalidValidatorIndex | JfyiError::NoSuchCachedSession { .. } | JfyiError::QueryAvailableDataResponseChannel(_) | - JfyiError::QueryChunkResponseChannel(_) => gum::warn!(target: LOG_TARGET, error = %jfyi, ctx), + JfyiError::QueryChunkResponseChannel(_) | + JfyiError::BlockNumberNotFound => gum::warn!(target: LOG_TARGET, error = %jfyi, ctx), JfyiError::FetchPoV(_) | JfyiError::SendResponse | JfyiError::NoSuchPoV | diff --git a/polkadot/node/network/availability-distribution/src/requester/fetch_task/mod.rs b/polkadot/node/network/availability-distribution/src/requester/fetch_task/mod.rs index 191ee2acd973..e1fcb2df122d 100644 --- a/polkadot/node/network/availability-distribution/src/requester/fetch_task/mod.rs +++ b/polkadot/node/network/availability-distribution/src/requester/fetch_task/mod.rs @@ -35,7 +35,7 @@ use polkadot_node_subsystem::{ }; use polkadot_primitives::{ AuthorityDiscoveryId, BlakeTwo256, CandidateHash, GroupIndex, Hash, HashT, OccupiedCore, - SessionIndex, + SessionIndex, ValidatorIndex, }; use crate::{ @@ -140,6 +140,7 @@ impl FetchTaskConfig { sender: mpsc::Sender, metrics: Metrics, session_info: &SessionInfo, + chunk_index: ValidatorIndex, span: jaeger::Span, ) -> Self { let span = span @@ -167,7 +168,7 @@ impl FetchTaskConfig { .clone(), request: ChunkFetchingRequest { candidate_hash: core.candidate_hash, - index: session_info.our_index, + index: chunk_index, }, erasure_root: core.candidate_descriptor.erasure_root, relay_parent: core.candidate_descriptor.relay_parent, diff --git a/polkadot/node/network/availability-distribution/src/requester/mod.rs b/polkadot/node/network/availability-distribution/src/requester/mod.rs index 446988f7cc0d..1cfb985a14ef 100644 --- a/polkadot/node/network/availability-distribution/src/requester/mod.rs +++ b/polkadot/node/network/availability-distribution/src/requester/mod.rs @@ -18,10 +18,7 @@ //! availability. use std::{ - collections::{ - hash_map::{Entry, HashMap}, - hash_set::HashSet, - }, + collections::{hash_map::HashMap, hash_set::HashSet}, iter::IntoIterator, pin::Pin, }; @@ -37,10 +34,16 @@ use polkadot_node_subsystem::{ messages::{ChainApiMessage, RuntimeApiMessage}, overseer, ActivatedLeaf, ActiveLeavesUpdate, LeafStatus, }; -use polkadot_node_subsystem_util::runtime::{get_occupied_cores, RuntimeInfo}; -use polkadot_primitives::{CandidateHash, Hash, OccupiedCore, SessionIndex}; +use polkadot_node_subsystem_util::{ + runtime::{get_occupied_cores, RuntimeInfo}, + shuffle_validator_indices, +}; +use polkadot_primitives::{ + BlockNumber, CandidateHash, Hash, OccupiedCore, SessionIndex, ValidatorIndex, +}; +use schnellru::{ByLength, LruMap}; -use super::{FatalError, Metrics, Result, LOG_TARGET}; +use super::{error::Error, FatalError, Metrics, Result, LOG_TARGET}; #[cfg(test)] mod tests; @@ -77,6 +80,8 @@ pub struct Requester { /// Prometheus Metrics metrics: Metrics, + + chunk_index_cache: LruMap, } #[overseer::contextbounds(AvailabilityDistribution, prefix = self::overseer)] @@ -90,7 +95,14 @@ impl Requester { /// by advancing the stream. pub fn new(metrics: Metrics) -> Self { let (tx, rx) = mpsc::channel(1); - Requester { fetches: HashMap::new(), session_cache: SessionCache::new(), tx, rx, metrics } + Requester { + fetches: HashMap::new(), + session_cache: SessionCache::new(), + tx, + rx, + metrics, + chunk_index_cache: LruMap::new(ByLength::new(10)), + } } /// Update heads that need availability distribution. @@ -208,46 +220,75 @@ impl Requester { .with_string_tag("leaf", format!("{:?}", leaf)) .with_candidate(core.candidate_hash) .with_stage(jaeger::Stage::AvailabilityDistribution); - match self.fetches.entry(core.candidate_hash) { - Entry::Occupied(mut e) => + + if let Some(e) = self.fetches.get_mut(&core.candidate_hash) { // Just book keeping - we are already requesting that chunk: - { - span.add_string_tag("already-requested-chunk", "true"); - e.get_mut().add_leaf(leaf); - }, - Entry::Vacant(e) => { - span.add_string_tag("already-requested-chunk", "false"); - let tx = self.tx.clone(); - let metrics = self.metrics.clone(); - - let task_cfg = self - .session_cache - .with_session_info( - context, - runtime, - // We use leaf here, the relay_parent must be in the same session as - // the leaf. This is guaranteed by runtime which ensures that cores are - // cleared at session boundaries. At the same time, only leaves are - // guaranteed to be fetchable by the state trie. - leaf, - leaf_session_index, - |info| FetchTaskConfig::new(leaf, &core, tx, metrics, info, span), - ) - .await - .map_err(|err| { - gum::warn!( - target: LOG_TARGET, - error = ?err, - "Failed to spawn a fetch task" - ); - err + span.add_string_tag("already-requested-chunk", "true"); + e.add_leaf(leaf); + } else { + span.add_string_tag("already-requested-chunk", "false"); + let tx = self.tx.clone(); + let metrics = self.metrics.clone(); + // only interested in the map for (ourIndex) -> ValidatorIndex + // hold LruCache + // alternatively, re-compute it according to algorithm + let block_number = + get_block_number(context.sender(), core.candidate_descriptor.relay_parent) + .await?; + + let session_info = self + .session_cache + .get_session_info( + context, + runtime, + // We use leaf here, the relay_parent must be in the same session as + // the leaf. This is guaranteed by runtime which ensures that cores are + // cleared at session boundaries. At the same time, only leaves are + // guaranteed to be fetchable by the state trie. + leaf, + leaf_session_index, + ) + .await + .map_err(|err| { + gum::warn!( + target: LOG_TARGET, + error = ?err, + "Failed to spawn a fetch task" + ); + err + })?; + + if let Some(session_info) = session_info { + // TODO: optimise this n_validators calculation. + let n_validators = + session_info.validator_groups.iter().fold(0, |mut acc, group| { + acc += group.len(); + acc }); - - if let Ok(Some(task_cfg)) = task_cfg { - e.insert(FetchTask::start(task_cfg, context).await?); - } - // Not a validator, nothing to do. - }, + let chunk_index = self + .chunk_index_cache + .get_or_insert(block_number, || { + let shuffled_indices = + shuffle_validator_indices(block_number, n_validators); + shuffled_indices[session_info.our_index.0 as usize] + }) + .expect("no expected"); + + let task_cfg = FetchTaskConfig::new( + leaf, + &core, + tx, + metrics, + session_info, + *chunk_index, + span, + ); + + self.fetches + .insert(core.candidate_hash, FetchTask::start(task_cfg, context).await?); + } else { + // Error + } } } Ok(()) @@ -349,3 +390,22 @@ where .map_err(FatalError::ChainApi)?; Ok(ancestors) } + +async fn get_block_number(sender: &mut Sender, relay_parent: Hash) -> Result +where + Sender: overseer::SubsystemSender, +{ + let (tx, rx) = oneshot::channel(); + sender.send_message(ChainApiMessage::BlockNumber(relay_parent, tx)).await; + + let block_number = rx + .await + .map_err(FatalError::ChainApiSenderDropped)? + .map_err(FatalError::ChainApi)?; + + if let Some(number) = block_number { + Ok(number) + } else { + Err(Error::BlockNumberNotFound) + } +} diff --git a/polkadot/node/network/availability-distribution/src/requester/session_cache.rs b/polkadot/node/network/availability-distribution/src/requester/session_cache.rs index 8a48e19c2827..6bb8098e8317 100644 --- a/polkadot/node/network/availability-distribution/src/requester/session_cache.rs +++ b/polkadot/node/network/availability-distribution/src/requester/session_cache.rs @@ -87,39 +87,28 @@ impl SessionCache { } } - /// Tries to retrieve `SessionInfo` and calls `with_info` if successful. - /// + /// Tries to retrieve `SessionInfo`. /// If this node is not a validator, the function will return `None`. - /// - /// Use this function over any `fetch_session_info` if all you need is a reference to - /// `SessionInfo`, as it avoids an expensive clone. - pub async fn with_session_info( - &mut self, + pub async fn get_session_info<'a, Context>( + &'a mut self, ctx: &mut Context, runtime: &mut RuntimeInfo, parent: Hash, session_index: SessionIndex, - with_info: F, - ) -> Result> - where - F: FnOnce(&SessionInfo) -> R, - { - if let Some(o_info) = self.session_info_cache.get(&session_index) { - gum::trace!(target: LOG_TARGET, session_index, "Got session from lru"); - return Ok(Some(with_info(o_info))) + ) -> Result> { + if self.session_info_cache.get(&session_index).is_none() { + if let Some(info) = + Self::query_info_from_runtime(ctx, runtime, parent, session_index).await? + { + gum::trace!(target: LOG_TARGET, session_index, "Calling `with_info`"); + gum::trace!(target: LOG_TARGET, session_index, "Storing session info in lru!"); + self.session_info_cache.insert(session_index, info); + } else { + return Ok(None) + } } - if let Some(info) = - self.query_info_from_runtime(ctx, runtime, parent, session_index).await? - { - gum::trace!(target: LOG_TARGET, session_index, "Calling `with_info`"); - let r = with_info(&info); - gum::trace!(target: LOG_TARGET, session_index, "Storing session info in lru!"); - self.session_info_cache.insert(session_index, info); - Ok(Some(r)) - } else { - Ok(None) - } + Ok(self.session_info_cache.get(&session_index).map(|i| &*i)) } /// Variant of `report_bad` that never fails, but just logs errors. @@ -171,7 +160,6 @@ impl SessionCache { /// /// Returns: `None` if not a validator. async fn query_info_from_runtime( - &self, ctx: &mut Context, runtime: &mut RuntimeInfo, relay_parent: Hash, @@ -188,6 +176,9 @@ impl SessionCache { // Get our group index: let our_group = info.validator_info.our_group; + // TODO: This shuffling is fine, as it only is used for knowing which validator to ask + // in order to get the chunk. The chunk index is coming from somewhere else. + // Shuffle validators in groups: let mut rng = thread_rng(); for g in validator_groups.iter_mut() { diff --git a/polkadot/node/network/availability-recovery/src/error.rs b/polkadot/node/network/availability-recovery/src/error.rs index 47277a521b81..50acbe989edb 100644 --- a/polkadot/node/network/availability-recovery/src/error.rs +++ b/polkadot/node/network/availability-recovery/src/error.rs @@ -17,10 +17,12 @@ //! The `Error` and `Result` types used by the subsystem. use futures::channel::oneshot; +use polkadot_node_subsystem::ChainApiError; use thiserror::Error; /// Error type used by the Availability Recovery subsystem. #[derive(Debug, Error)] +// TODO: add fatality pub enum Error { #[error(transparent)] Subsystem(#[from] polkadot_node_subsystem::SubsystemError), @@ -42,6 +44,15 @@ pub enum Error { #[error(transparent)] Util(#[from] polkadot_node_subsystem_util::Error), + + #[error("Oneshot for receiving response from Chain API got cancelled")] + ChainApiSenderDropped(#[source] oneshot::Canceled), + + #[error("Retrieving response from Chain API unexpectedly failed with error: {0}")] + ChainApi(#[from] ChainApiError), + + #[error("Cannot find block number for given relay parent")] + BlockNumberNotFound, } pub type Result = std::result::Result; diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index 04b71147c9f0..4a5e3c2d5dce 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -35,7 +35,10 @@ use futures::{ task::{Context, Poll}, }; use schnellru::{ByLength, LruMap}; -use task::{FetchChunks, FetchChunksParams, FetchFull, FetchFullParams}; +use task::{ + FetchChunks, FetchChunksParams, FetchFull, FetchFullParams, FetchSystematicChunks, + FetchSystematicChunksParams, +}; use fatality::Nested; use polkadot_erasure_coding::{ @@ -51,11 +54,11 @@ use polkadot_node_primitives::{AvailableData, ErasureChunk}; use polkadot_node_subsystem::{ errors::RecoveryError, jaeger, - messages::{AvailabilityRecoveryMessage, AvailabilityStoreMessage}, + messages::{AvailabilityRecoveryMessage, AvailabilityStoreMessage, ChainApiMessage}, overseer, ActiveLeavesUpdate, FromOrchestra, OverseerSignal, SpawnedSubsystem, SubsystemContext, SubsystemError, SubsystemResult, }; -use polkadot_node_subsystem_util::request_session_info; +use polkadot_node_subsystem_util::{request_session_info, shuffle_validator_indices}; use polkadot_primitives::{ BlakeTwo256, BlockNumber, CandidateHash, CandidateReceipt, GroupIndex, Hash, HashT, SessionIndex, SessionInfo, ValidatorIndex, @@ -90,6 +93,8 @@ pub enum RecoveryStrategyKind { BackersFirstIfSizeLower(usize), /// We always recover using validator chunks. ChunksAlways, + /// First try the backing group. Then systematic chunks. + BackersThenSystematicChunks, /// Do not request data from the availability store. /// This is the useful for nodes where the /// availability-store subsystem is not expected to run, @@ -448,16 +453,20 @@ async fn handle_recover( } }; + let block_number = + get_block_number(ctx.sender(), receipt.descriptor.relay_parent).await?; + let shuffling = shuffle_validator_indices(block_number, session_info.validators.len()); + let backing_validators = if let Some(backing_group) = backing_group { session_info.validator_groups.get(backing_group) } else { None }; - let fetch_chunks_params = FetchChunksParams { - n_validators: session_info.validators.len(), - erasure_task_tx: erasure_task_tx.clone(), - }; + // let fetch_chunks_params = FetchChunksParams { + // n_validators: session_info.validators.len(), + // erasure_task_tx: erasure_task_tx.clone(), + // }; let mut recovery_strategies: VecDeque< Box::Sender>>, @@ -470,13 +479,24 @@ async fn handle_recover( (RecoveryStrategyKind::BypassAvailabilityStore, true) => recovery_strategies.push_back(Box::new(FetchFull::new(FetchFullParams { validators: backing_validators.to_vec(), - erasure_task_tx, + erasure_task_tx: erasure_task_tx.clone(), }))), _ => {}, }; } - recovery_strategies.push_back(Box::new(FetchChunks::new(fetch_chunks_params))); + if recovery_strategy_kind == RecoveryStrategyKind::BackersThenSystematicChunks { + recovery_strategies.push_back(Box::new(FetchSystematicChunks::new( + FetchSystematicChunksParams { + validators: (0..recovery_threshold(session_info.validators.len()).unwrap()) + .map(|i| (shuffling[i], ValidatorIndex(i as u32))) + .collect(), + erasure_task_tx, + }, + ))); + } + + // recovery_strategies.push_back(Box::new(FetchChunks::new(fetch_chunks_params))); launch_recovery_task( state, @@ -500,7 +520,7 @@ async fn handle_recover( } } -/// Queries a chunk from av-store. +/// Queries the full `AvailableData` from av-store. #[overseer::contextbounds(AvailabilityRecovery, prefix = self::overseer)] async fn query_full_data( ctx: &mut Context, @@ -575,6 +595,19 @@ impl AvailabilityRecoverySubsystem { } } + /// Create a new instance of `AvailabilityRecoverySubsystem` which first requests full data + /// from backers, with a fallback to recover from systematic chunks. + pub fn with_fast_path_then_systematic_chunks( + req_receiver: IncomingRequestReceiver, + metrics: Metrics, + ) -> Self { + Self { + recovery_strategy_kind: RecoveryStrategyKind::BackersThenSystematicChunks, + req_receiver, + metrics, + } + } + async fn run(self, mut ctx: Context) -> SubsystemResult<()> { let mut state = State::default(); let Self { mut req_receiver, metrics, recovery_strategy_kind } = self; @@ -650,6 +683,8 @@ impl AvailabilityRecoverySubsystem { return Ok(()); } FromOrchestra::Communication { msg } => { + gum::debug!(target: LOG_TARGET, + "Received message to recover available data"); match msg { AvailabilityRecoveryMessage::RecoverAvailableData( receipt, @@ -817,3 +852,25 @@ async fn erasure_task_thread( } } } + +async fn get_block_number( + sender: &mut Sender, + relay_parent: Hash, +) -> error::Result +where + Sender: overseer::SubsystemSender, +{ + let (tx, rx) = oneshot::channel(); + sender.send_message(ChainApiMessage::BlockNumber(relay_parent, tx)).await; + + let block_number = rx + .await + .map_err(error::Error::ChainApiSenderDropped)? + .map_err(error::Error::ChainApi)?; + + if let Some(number) = block_number { + Ok(number) + } else { + Err(error::Error::BlockNumberNotFound) + } +} diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 30eaf7734a52..f70aab6919f5 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -33,11 +33,11 @@ use polkadot_node_subsystem::{ messages::{AvailabilityStoreMessage, NetworkBridgeTxMessage}, overseer, RecoveryError, }; -use polkadot_primitives::{AuthorityDiscoveryId, CandidateHash, Hash, ValidatorIndex}; +use polkadot_primitives::{AuthorityDiscoveryId, CandidateHash, ChunkIndex, Hash, ValidatorIndex}; use rand::seq::SliceRandom; use sc_network::{IfDisconnected, OutboundFailure, RequestFailure}; use std::{ - collections::{HashMap, VecDeque}, + collections::{BTreeMap, HashMap, VecDeque}, time::Duration, }; @@ -103,7 +103,8 @@ pub struct State { /// Chunks received so far. received_chunks: HashMap, /// Collection of in-flight requests. - requesting_chunks: FuturesUndead, (ValidatorIndex, RequestError)>>, + requesting_chunks: + FuturesUndead, (ChunkIndex, ValidatorIndex, RequestError)>>, } impl State { @@ -124,7 +125,7 @@ impl State { &mut self, params: &RecoveryParams, sender: &mut Sender, - ) -> Vec { + ) -> Vec { let (tx, rx) = oneshot::channel(); sender .send_message(AvailabilityStoreMessage::QueryAllChunks(params.candidate_hash, tx)) @@ -173,7 +174,7 @@ impl State { params: &RecoveryParams, sender: &mut Sender, desired_requests_count: usize, - validators: &mut VecDeque, + validators: &mut VecDeque<(ChunkIndex, ValidatorIndex)>, ) where Sender: overseer::AvailabilityRecoverySenderTrait, { @@ -183,12 +184,13 @@ impl State { let mut requests = Vec::with_capacity(desired_requests_count - already_requesting_count); while self.requesting_chunks.len() < desired_requests_count { - if let Some(validator_index) = validators.pop_back() { + if let Some((chunk_index, validator_index)) = validators.pop_back() { let validator = params.validator_authority_keys[validator_index.0 as usize].clone(); gum::trace!( target: LOG_TARGET, ?validator, ?validator_index, + ?chunk_index, ?candidate_hash, "Requesting chunk", ); @@ -196,7 +198,7 @@ impl State { // Request data. let raw_request = req_res::v1::ChunkFetchingRequest { candidate_hash: params.candidate_hash, - index: validator_index, + index: chunk_index, }; let (req, res) = OutgoingRequest::new(Recipient::Authority(validator), raw_request); @@ -211,7 +213,7 @@ impl State { Ok(req_res::v1::ChunkFetchingResponse::Chunk(chunk)) => Ok(Some(chunk.recombine_into_chunk(&raw_request))), Ok(req_res::v1::ChunkFetchingResponse::NoSuchChunk) => Ok(None), - Err(e) => Err((validator_index, e)), + Err(e) => Err((chunk_index, validator_index, e)), } })); } else { @@ -231,7 +233,7 @@ impl State { async fn wait_for_chunks( &mut self, params: &RecoveryParams, - validators: &mut VecDeque, + validators: &mut VecDeque<(ChunkIndex, ValidatorIndex)>, can_conclude: impl Fn(usize, usize, usize, &RecoveryParams, usize) -> bool, ) -> (usize, usize) { let metrics = ¶ms.metrics; @@ -266,7 +268,7 @@ impl State { metrics.on_chunk_request_no_such_chunk(); error_count += 1; }, - Err((validator_index, e)) => { + Err((chunk_index, validator_index, e)) => { error_count += 1; gum::trace!( @@ -298,12 +300,12 @@ impl State { metrics.on_chunk_request_error(); } - validators.push_front(validator_index); + validators.push_front((chunk_index, validator_index)); }, RequestError::Canceled(_) => { metrics.on_chunk_request_error(); - validators.push_front(validator_index); + validators.push_front((chunk_index, validator_index)); }, } }, @@ -549,6 +551,276 @@ impl RecoveryStrategy } } +/// `RecoveryStrategy` that attempts to recover the systematic chunks from the validators that +/// hold them, in order to bypass the erasure code reconstruction step, which is costly. +pub struct FetchSystematicChunks { + threshold: usize, + validators: BTreeMap, + /// Channel to the erasure task handler. + erasure_task_tx: futures::channel::mpsc::Sender, +} + +/// Parameters needed for fetching systematic chunks. +pub struct FetchSystematicChunksParams { + /// Validators that hold the systematic chunks. + pub validators: BTreeMap, + /// Channel to the erasure task handler. + pub erasure_task_tx: futures::channel::mpsc::Sender, +} + +impl FetchSystematicChunks { + /// Instantiate a new systematic chunks strategy. + pub fn new(params: FetchSystematicChunksParams) -> Self { + Self { + // We maintain the copy of the systematic validators in the params field, so that we + // can use them when doing the recovery. + threshold: params.validators.len(), + validators: params.validators, + erasure_task_tx: params.erasure_task_tx, + } + } + + fn is_unavailable( + unrequested_validators: usize, + in_flight_requests: usize, + systematic_chunk_count: usize, + threshold: usize, + ) -> bool { + is_unavailable( + systematic_chunk_count, + in_flight_requests, + unrequested_validators, + threshold, + ) + } + + /// Desired number of parallel requests. + /// + /// For the given threshold (total required number of chunks) get the desired number of + /// requests we want to have running in parallel at this time. + fn get_desired_request_count(&self, chunk_count: usize, threshold: usize) -> usize { + // Upper bound for parallel requests. + let max_requests_boundary = std::cmp::min(N_PARALLEL, threshold); + // How many chunks are still needed? + let remaining_chunks = threshold.saturating_sub(chunk_count); + // Actual number of requests we want to have in flight in parallel: + // We don't have to make up for any error rate, as an error fetching a systematic chunk + // results in failure of the entire strategy. + std::cmp::min(max_requests_boundary, remaining_chunks) + } + + async fn attempt_systematic_recovery( + &mut self, + state: &mut State, + common_params: &RecoveryParams, + ) -> Result { + let recovery_duration = common_params.metrics.time_erasure_recovery(); + + let available_data = polkadot_erasure_coding::reconstruct_from_systematic_v1( + common_params.n_validators, + state.received_chunks.iter().map(|(_, chunk)| &chunk.chunk[..]).collect(), + ); + + match available_data { + Ok(data) => { + // Send request to re-encode the chunks and check merkle root. + let (reencode_tx, reencode_rx) = oneshot::channel(); + self.erasure_task_tx + .send(ErasureTask::Reencode( + common_params.n_validators, + common_params.erasure_root, + data, + reencode_tx, + )) + .await + .map_err(|_| RecoveryError::ChannelClosed)?; + + let reencode_response = + reencode_rx.await.map_err(|_| RecoveryError::ChannelClosed)?; + + if let Some(data) = reencode_response { + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + erasure_root = ?common_params.erasure_root, + "Data recovery from systematic chunks complete", + ); + + Ok(data) + } else { + recovery_duration.map(|rd| rd.stop_and_discard()); + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + erasure_root = ?common_params.erasure_root, + "Systematic data recovery error - root mismatch", + ); + + // Return a non-fatal error, because we may have gotten a non-systematic chunk + // from a validator by mistake. We may have enough chunks for the next strategy + // to pass. + Err(RecoveryError::Unavailable) + } + }, + Err(err) => { + recovery_duration.map(|rd| rd.stop_and_discard()); + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + erasure_root = ?common_params.erasure_root, + ?err, + "Systematic data recovery error ", + ); + + // Return a non-fatal error, because we may have gotten a non-systematic chunk from + // a validator by mistake. We may have enough chunks for the next strategy to pass. + Err(RecoveryError::Unavailable) + }, + } + } +} + +#[async_trait::async_trait] +impl RecoveryStrategy + for FetchSystematicChunks +{ + fn display_name(&self) -> &'static str { + "Fetch systematic chunks" + } + + async fn run( + &mut self, + state: &mut State, + sender: &mut Sender, + common_params: &RecoveryParams, + ) -> Result { + // First query the store for any chunks we've got. + if !common_params.bypass_availability_store { + let local_chunk_indices = state.populate_from_av_store(common_params, sender).await; + + self.validators.retain(|c_index, _| !local_chunk_indices.contains(c_index)); + + for c_index in &local_chunk_indices { + // If we are among the systematic validators but hold an invalid chunk, we cannot + // perform the systematic recovery. Fall through to the next strategy. + if self.validators.contains_key(c_index) && + !state.received_chunks.contains_key(c_index) + { + gum::debug!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + erasure_root = ?common_params.erasure_root, + requesting = %state.requesting_chunks.len(), + total_requesting = %state.requesting_chunks.total_len(), + n_validators = %common_params.n_validators, + "Systematic chunk recovery is not possible. We are among the systematic validators but hold an invalid chunk", + ); + return Err(RecoveryError::Unavailable) + } + } + } + + let mut systematic_chunk_count = self + .validators + .iter() + .filter(|(c_index, _)| state.received_chunks.contains_key(c_index)) + .count(); + + // No need to query the validators that have the chunks we already received. + self.validators + .retain(|c_index, _| !state.received_chunks.contains_key(c_index)); + + loop { + // If received_chunks has `systematic_chunk_threshold` entries, attempt to recover the + // data. If that fails, or a re-encoding of it doesn't match the expected erasure root, + // return Err(RecoveryError::Invalid) + if systematic_chunk_count >= self.threshold { + return self.attempt_systematic_recovery(state, common_params).await + } + + if Self::is_unavailable( + self.validators.len(), + state.requesting_chunks.total_len(), + systematic_chunk_count, + self.threshold, + ) { + gum::debug!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + erasure_root = ?common_params.erasure_root, + received = %systematic_chunk_count, + requesting = %state.requesting_chunks.len(), + total_requesting = %state.requesting_chunks.total_len(), + n_validators = %common_params.n_validators, + threshold = ?self.threshold, + "Data recovery is not possible", + ); + + return Err(RecoveryError::Unavailable) + } + + let desired_requests_count = + self.get_desired_request_count(systematic_chunk_count, self.threshold); + let already_requesting_count = state.requesting_chunks.len(); + gum::debug!( + target: LOG_TARGET, + ?common_params.candidate_hash, + ?desired_requests_count, + total_received = ?systematic_chunk_count, + threshold = ?self.threshold, + ?already_requesting_count, + "Requesting systematic availability chunks for a candidate", + ); + + let mut validators_queue = std::mem::take(&mut self.validators).into_iter().collect(); + state + .launch_parallel_chunk_requests( + common_params, + sender, + desired_requests_count, + &mut validators_queue, + ) + .await; + + let (total_responses, error_count) = state + .wait_for_chunks( + common_params, + &mut validators_queue, + |unrequested_validators, reqs, chunk_count, _params, error_count| { + error_count != 0 || + chunk_count >= self.threshold || + Self::is_unavailable( + unrequested_validators, + reqs, + chunk_count, + self.threshold, + ) + }, + ) + .await; + + // We can't afford any errors, as we need all the systematic chunks for this to work. + if error_count > 0 { + gum::debug!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + erasure_root = ?common_params.erasure_root, + received = %systematic_chunk_count, + requesting = %state.requesting_chunks.len(), + total_requesting = %state.requesting_chunks.total_len(), + n_validators = %common_params.n_validators, + threshold = ?self.threshold, + "Systematic chunk recovery is not possible. ", + ); + + return Err(RecoveryError::Unavailable) + } + + systematic_chunk_count += total_responses; + } + } +} + /// `RecoveryStrategy` that requests chunks from validators, in parallel. pub struct FetchChunks { /// How many requests have been unsuccessful so far. @@ -763,19 +1035,24 @@ impl RecoveryStrategy ?already_requesting_count, "Requesting availability chunks for a candidate", ); + + let mut validators_queue = std::mem::take(&mut self.validators) + .into_iter() + .map(|v_index| (v_index, v_index)) + .collect(); state .launch_parallel_chunk_requests( common_params, sender, desired_requests_count, - &mut self.validators, + &mut validators_queue, ) .await; let (total_responses, error_count) = state .wait_for_chunks( common_params, - &mut self.validators, + &mut validators_queue, |unrequested_validators, reqs, chunk_count, params, _error_count| { chunk_count >= params.threshold || Self::is_unavailable( diff --git a/polkadot/node/overseer/src/lib.rs b/polkadot/node/overseer/src/lib.rs index 84d5d19c3b93..7489a00824c7 100644 --- a/polkadot/node/overseer/src/lib.rs +++ b/polkadot/node/overseer/src/lib.rs @@ -507,6 +507,7 @@ pub struct Overseer { NetworkBridgeTxMessage, RuntimeApiMessage, AvailabilityStoreMessage, + ChainApiMessage, ])] availability_recovery: AvailabilityRecovery, diff --git a/polkadot/node/subsystem-util/Cargo.toml b/polkadot/node/subsystem-util/Cargo.toml index d9364e2c2c0f..ac83026d9eb5 100644 --- a/polkadot/node/subsystem-util/Cargo.toml +++ b/polkadot/node/subsystem-util/Cargo.toml @@ -15,6 +15,7 @@ parity-scale-codec = { version = "3.6.1", default-features = false, features = [ parking_lot = "0.11.2" pin-project = "1.0.9" rand = "0.8.5" +rand_chacha = { version = "0.3.1", default-features = false } thiserror = "1.0.48" fatality = "0.0.6" gum = { package = "tracing-gum", path = "../gum" } diff --git a/polkadot/node/subsystem-util/src/lib.rs b/polkadot/node/subsystem-util/src/lib.rs index daee4a8350e5..e2aa188a9c8a 100644 --- a/polkadot/node/subsystem-util/src/lib.rs +++ b/polkadot/node/subsystem-util/src/lib.rs @@ -25,17 +25,18 @@ #![warn(missing_docs)] +pub use overseer::{ + gen::{OrchestraError as OverseerError, Timeout}, + Subsystem, TimeoutExt, +}; use polkadot_node_subsystem::{ errors::{RuntimeApiError, SubsystemError}, messages::{RuntimeApiMessage, RuntimeApiRequest, RuntimeApiSender}, overseer, SubsystemSender, }; -use polkadot_primitives::{slashing, ExecutorParams}; - -pub use overseer::{ - gen::{OrchestraError as OverseerError, Timeout}, - Subsystem, TimeoutExt, -}; +use polkadot_primitives::{slashing, BlockNumber, ExecutorParams}; +use rand::seq::SliceRandom; +use rand_chacha::ChaCha8Rng; pub use polkadot_node_metrics::{metrics, Metronome}; @@ -44,12 +45,13 @@ use parity_scale_codec::Encode; use polkadot_primitives::{ vstaging as vstaging_primitives, AuthorityDiscoveryId, CandidateEvent, CandidateHash, - CommittedCandidateReceipt, CoreState, EncodeAs, GroupIndex, GroupRotationInfo, Hash, - Id as ParaId, OccupiedCoreAssumption, PersistedValidationData, ScrapedOnChainVotes, + ChunkIndex, CommittedCandidateReceipt, CoreState, EncodeAs, GroupIndex, GroupRotationInfo, + Hash, Id as ParaId, OccupiedCoreAssumption, PersistedValidationData, ScrapedOnChainVotes, SessionIndex, SessionInfo, Signed, SigningContext, ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, ValidatorSignature, }; pub use rand; +use rand::SeedableRng; use sp_application_crypto::AppCrypto; use sp_core::ByteArray; use sp_keystore::{Error as KeystoreError, KeystorePtr}; @@ -440,3 +442,17 @@ impl Validator { Signed::sign(&keystore, payload, &self.signing_context, self.index, &self.key) } } + +pub fn shuffle_validator_indices( + block_number: BlockNumber, + n_validators: usize, +) -> Vec { + let seed = block_number.to_be_bytes(); + let mut rng: ChaCha8Rng = + SeedableRng::from_seed(seed.repeat(8).try_into().expect("should never fail")); + + let mut shuffled_indices: Vec<_> = (0..n_validators).map(|i| ValidatorIndex(i as _)).collect(); + + shuffled_indices.shuffle(&mut rng); + shuffled_indices +} diff --git a/polkadot/primitives/src/lib.rs b/polkadot/primitives/src/lib.rs index 9121b3790858..d285a08499eb 100644 --- a/polkadot/primitives/src/lib.rs +++ b/polkadot/primitives/src/lib.rs @@ -40,24 +40,24 @@ pub use v5::{ AccountPublic, ApprovalVote, AssignmentId, AuthorityDiscoveryId, AvailabilityBitfield, BackedCandidate, Balance, BlakeTwo256, Block, BlockId, BlockNumber, CandidateCommitments, CandidateDescriptor, CandidateEvent, CandidateHash, CandidateIndex, CandidateReceipt, - CheckedDisputeStatementSet, CheckedMultiDisputeStatementSet, CollatorId, CollatorSignature, - CommittedCandidateReceipt, CompactStatement, ConsensusLog, CoreIndex, CoreState, DisputeState, - DisputeStatement, DisputeStatementSet, DownwardMessage, EncodeAs, ExecutorParam, - ExecutorParams, ExecutorParamsHash, ExplicitDisputeStatement, GroupIndex, GroupRotationInfo, - Hash, HashT, HeadData, Header, HrmpChannelId, Id, InboundDownwardMessage, InboundHrmpMessage, - IndexedVec, InherentData, InvalidDisputeStatementKind, Moment, MultiDisputeStatementSet, Nonce, - OccupiedCore, OccupiedCoreAssumption, OutboundHrmpMessage, ParathreadClaim, ParathreadEntry, - PersistedValidationData, PvfCheckStatement, PvfExecTimeoutKind, PvfPrepTimeoutKind, - RuntimeMetricLabel, RuntimeMetricLabelValue, RuntimeMetricLabelValues, RuntimeMetricLabels, - RuntimeMetricOp, RuntimeMetricUpdate, ScheduledCore, ScrapedOnChainVotes, SessionIndex, - SessionInfo, Signature, Signed, SignedAvailabilityBitfield, SignedAvailabilityBitfields, - SignedStatement, SigningContext, Slot, UncheckedSigned, UncheckedSignedAvailabilityBitfield, - UncheckedSignedAvailabilityBitfields, UncheckedSignedStatement, UpgradeGoAhead, - UpgradeRestriction, UpwardMessage, ValidDisputeStatementKind, ValidationCode, - ValidationCodeHash, ValidatorId, ValidatorIndex, ValidatorSignature, ValidityAttestation, - ValidityError, ASSIGNMENT_KEY_TYPE_ID, LEGACY_MIN_BACKING_VOTES, LOWEST_PUBLIC_ID, - MAX_CODE_SIZE, MAX_HEAD_DATA_SIZE, MAX_POV_SIZE, ON_DEMAND_DEFAULT_QUEUE_MAX_SIZE, - PARACHAINS_INHERENT_IDENTIFIER, PARACHAIN_KEY_TYPE_ID, + CheckedDisputeStatementSet, CheckedMultiDisputeStatementSet, ChunkIndex, CollatorId, + CollatorSignature, CommittedCandidateReceipt, CompactStatement, ConsensusLog, CoreIndex, + CoreState, DisputeState, DisputeStatement, DisputeStatementSet, DownwardMessage, EncodeAs, + ExecutorParam, ExecutorParams, ExecutorParamsHash, ExplicitDisputeStatement, GroupIndex, + GroupRotationInfo, Hash, HashT, HeadData, Header, HrmpChannelId, Id, InboundDownwardMessage, + InboundHrmpMessage, IndexedVec, InherentData, InvalidDisputeStatementKind, Moment, + MultiDisputeStatementSet, Nonce, OccupiedCore, OccupiedCoreAssumption, OutboundHrmpMessage, + ParathreadClaim, ParathreadEntry, PersistedValidationData, PvfCheckStatement, + PvfExecTimeoutKind, PvfPrepTimeoutKind, RuntimeMetricLabel, RuntimeMetricLabelValue, + RuntimeMetricLabelValues, RuntimeMetricLabels, RuntimeMetricOp, RuntimeMetricUpdate, + ScheduledCore, ScrapedOnChainVotes, SessionIndex, SessionInfo, Signature, Signed, + SignedAvailabilityBitfield, SignedAvailabilityBitfields, SignedStatement, SigningContext, Slot, + UncheckedSigned, UncheckedSignedAvailabilityBitfield, UncheckedSignedAvailabilityBitfields, + UncheckedSignedStatement, UpgradeGoAhead, UpgradeRestriction, UpwardMessage, + ValidDisputeStatementKind, ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, + ValidatorSignature, ValidityAttestation, ValidityError, ASSIGNMENT_KEY_TYPE_ID, + LEGACY_MIN_BACKING_VOTES, LOWEST_PUBLIC_ID, MAX_CODE_SIZE, MAX_HEAD_DATA_SIZE, MAX_POV_SIZE, + ON_DEMAND_DEFAULT_QUEUE_MAX_SIZE, PARACHAINS_INHERENT_IDENTIFIER, PARACHAIN_KEY_TYPE_ID, }; #[cfg(feature = "std")] diff --git a/polkadot/primitives/src/v5/mod.rs b/polkadot/primitives/src/v5/mod.rs index 30782f95611f..fb2d040d90e0 100644 --- a/polkadot/primitives/src/v5/mod.rs +++ b/polkadot/primitives/src/v5/mod.rs @@ -109,6 +109,8 @@ pub trait TypeIndex { #[cfg_attr(feature = "std", derive(Serialize, Deserialize, Hash))] pub struct ValidatorIndex(pub u32); +pub type ChunkIndex = ValidatorIndex; + // We should really get https://github.com/paritytech/polkadot/issues/2403 going .. impl From for ValidatorIndex { fn from(n: u32) -> Self { From 6f58c3aa06e6c159c8a45f25cce001096fe14710 Mon Sep 17 00:00:00 2001 From: alindima Date: Thu, 14 Sep 2023 16:46:59 +0300 Subject: [PATCH 008/126] continue implementation --- polkadot/erasure-coding/src/lib.rs | 16 ++- .../node/core/bitfield-signing/src/lib.rs | 80 +++++++++--- .../src/requester/mod.rs | 4 +- .../network/availability-recovery/src/lib.rs | 85 ++++++++++--- .../network/availability-recovery/src/task.rs | 117 ++++++++++-------- polkadot/node/overseer/src/lib.rs | 1 + polkadot/node/service/src/overseer.rs | 2 +- polkadot/node/subsystem-util/src/lib.rs | 35 +++--- polkadot/node/subsystem-util/src/tests.rs | 13 ++ polkadot/primitives/src/v5/mod.rs | 2 + 10 files changed, 245 insertions(+), 110 deletions(-) diff --git a/polkadot/erasure-coding/src/lib.rs b/polkadot/erasure-coding/src/lib.rs index 7c28183e6f04..d7557465e6d3 100644 --- a/polkadot/erasure-coding/src/lib.rs +++ b/polkadot/erasure-coding/src/lib.rs @@ -96,6 +96,14 @@ pub const fn recovery_threshold(n_validators: usize) -> Result { Ok(needed + 1) } +/// Obtain the threshold of systematic chunks that should be enough to recover the data. +/// +/// If the regular `recovery_threshold` is a power of two, then it returns the same value. +/// Otherwise, it returns the next power of two. +pub fn systematic_recovery_threshold(n_validators: usize) -> Result { + code_params(n_validators).map(|params| params.k()) +} + fn code_params(n_validators: usize) -> Result { // we need to be able to reconstruct from 1/3 - eps @@ -132,8 +140,7 @@ pub fn reconstruct_from_systematic( n_validators: usize, chunks: Vec<&[u8]>, ) -> Result { - let code_params = code_params(n_validators)?; - let kpow2 = code_params.k(); + let kpow2 = systematic_recovery_threshold(n_validators)?; let Some(first_shard) = chunks.iter().next() else { return Err(Error::NotEnoughChunks) }; let shard_len = first_shard.len(); @@ -146,7 +153,7 @@ pub fn reconstruct_from_systematic( return Err(Error::UnevenLength) } - if chunks.len() <= kpow2 { + if chunks.len() < kpow2 { return Err(Error::NotEnoughChunks) } @@ -472,11 +479,12 @@ mod tests { fn round_trip_systematic_works() { fn property(available_data: ArbitraryAvailableData, n_validators: u16) { let n_validators = n_validators.saturating_add(2); + let kpow2 = systematic_recovery_threshold(n_validators as usize).unwrap(); let chunks = obtain_chunks(n_validators as usize, &available_data.0).unwrap(); assert_eq!( reconstruct_from_systematic_v1( n_validators as usize, - chunks.iter().map(|v| &v[..]).collect() + chunks.iter().take(kpow2).map(|v| &v[..]).collect() ) .unwrap(), available_data.0 diff --git a/polkadot/node/core/bitfield-signing/src/lib.rs b/polkadot/node/core/bitfield-signing/src/lib.rs index f29e827e1090..7396ec36c3b5 100644 --- a/polkadot/node/core/bitfield-signing/src/lib.rs +++ b/polkadot/node/core/bitfield-signing/src/lib.rs @@ -30,13 +30,16 @@ use polkadot_node_subsystem::{ errors::RuntimeApiError, jaeger, messages::{ - AvailabilityStoreMessage, BitfieldDistributionMessage, RuntimeApiMessage, RuntimeApiRequest, + AvailabilityStoreMessage, BitfieldDistributionMessage, ChainApiMessage, RuntimeApiMessage, + RuntimeApiRequest, }, - overseer, ActivatedLeaf, FromOrchestra, LeafStatus, OverseerSignal, PerLeafSpan, + overseer, ActivatedLeaf, ChainApiError, FromOrchestra, LeafStatus, OverseerSignal, PerLeafSpan, SpawnedSubsystem, SubsystemError, SubsystemResult, SubsystemSender, }; -use polkadot_node_subsystem_util::{self as util, Validator}; -use polkadot_primitives::{AvailabilityBitfield, CoreState, Hash, ValidatorIndex}; +use polkadot_node_subsystem_util::{ + self as util, request_validators, shuffle_availability_chunks, Validator, +}; +use polkadot_primitives::{AvailabilityBitfield, BlockNumber, CoreState, Hash, ValidatorIndex}; use sp_keystore::{Error as KeystoreError, KeystorePtr}; use std::{collections::HashMap, iter::FromIterator, time::Duration}; use wasm_timer::{Delay, Instant}; @@ -73,31 +76,44 @@ pub enum Error { #[error("Keystore failed: {0:?}")] Keystore(KeystoreError), + + #[error("Cannot find block number for given relay parent")] + BlockNumberNotFound, + + #[error("Oneshot for receiving response from Chain API got cancelled")] + ChainApiSenderDropped(#[source] oneshot::Canceled), + + #[error("Retrieving response from Chain API unexpectedly failed with error: {0}")] + ChainApi(#[from] ChainApiError), } /// If there is a candidate pending availability, query the Availability Store /// for whether we have the availability chunk for our validator index. async fn get_core_availability( core: &CoreState, + n_validators: usize, validator_idx: ValidatorIndex, - sender: &Mutex<&mut impl SubsystemSender>, + sender: &Mutex<&mut impl overseer::BitfieldSigningSenderTrait>, span: &jaeger::Span, ) -> Result { if let CoreState::Occupied(core) = core { let _span = span.child("query-chunk-availability"); + let block_number = + get_block_number(*sender.lock().await, core.candidate_descriptor.relay_parent).await?; + let shuffled_chunk_indices = shuffle_availability_chunks(block_number, n_validators); + let chunk_index = shuffled_chunk_indices[usize::try_from(validator_idx.0) + .expect("usize is at least u32 bytes on all modern targets.")]; + let (tx, rx) = oneshot::channel(); sender .lock() .await - .send_message( - AvailabilityStoreMessage::QueryChunkAvailability( - core.candidate_hash, - validator_idx, - tx, - ) - .into(), - ) + .send_message(AvailabilityStoreMessage::QueryChunkAvailability( + core.candidate_hash, + chunk_index, + tx, + )) .await; let res = rx.await.map_err(Into::into); @@ -142,8 +158,9 @@ async fn get_availability_cores( async fn construct_availability_bitfield( relay_parent: Hash, span: &jaeger::Span, + n_validators: usize, validator_idx: ValidatorIndex, - sender: &mut impl SubsystemSender, + sender: &mut impl overseer::BitfieldSigningSenderTrait, ) -> Result { // get the set of availability cores from the runtime let availability_cores = { @@ -163,7 +180,7 @@ async fn construct_availability_bitfield( let results = future::try_join_all( availability_cores .iter() - .map(|core| get_core_availability(core, validator_idx, &sender, span)), + .map(|core| get_core_availability(core, n_validators, validator_idx, &sender, span)), ) .await?; @@ -271,13 +288,16 @@ where let span_delay = span.child("delay"); let wait_until = Instant::now() + SPAWNED_TASK_DELAY; + let validators = request_validators(leaf.hash, &mut sender).await.await??; + // now do all the work we can before we need to wait for the availability store // if we're not a validator, we can just succeed effortlessly - let validator = match Validator::new(leaf.hash, keystore.clone(), &mut sender).await { - Ok(validator) => validator, - Err(util::Error::NotAValidator) => return Ok(()), - Err(err) => return Err(Error::Util(err)), - }; + let validator = + match Validator::new(&validators, leaf.hash, keystore.clone(), &mut sender).await { + Ok(validator) => validator, + Err(util::Error::NotAValidator) => return Ok(()), + Err(err) => return Err(Error::Util(err)), + }; // wait a bit before doing anything else Delay::new_at(wait_until).await?; @@ -292,6 +312,7 @@ where let bitfield = match construct_availability_bitfield( leaf.hash, &span_availability, + validators.len(), validator.index(), &mut sender, ) @@ -332,3 +353,22 @@ where Ok(()) } + +async fn get_block_number( + sender: &mut Sender, + relay_parent: Hash, +) -> Result +where + Sender: overseer::SubsystemSender, +{ + let (tx, rx) = oneshot::channel(); + sender.send_message(ChainApiMessage::BlockNumber(relay_parent, tx)).await; + + let block_number = rx.await.map_err(Error::ChainApiSenderDropped)?.map_err(Error::ChainApi)?; + + if let Some(number) = block_number { + Ok(number) + } else { + Err(Error::BlockNumberNotFound) + } +} diff --git a/polkadot/node/network/availability-distribution/src/requester/mod.rs b/polkadot/node/network/availability-distribution/src/requester/mod.rs index 1cfb985a14ef..02b0a8569f75 100644 --- a/polkadot/node/network/availability-distribution/src/requester/mod.rs +++ b/polkadot/node/network/availability-distribution/src/requester/mod.rs @@ -36,7 +36,7 @@ use polkadot_node_subsystem::{ }; use polkadot_node_subsystem_util::{ runtime::{get_occupied_cores, RuntimeInfo}, - shuffle_validator_indices, + shuffle_availability_chunks, }; use polkadot_primitives::{ BlockNumber, CandidateHash, Hash, OccupiedCore, SessionIndex, ValidatorIndex, @@ -269,7 +269,7 @@ impl Requester { .chunk_index_cache .get_or_insert(block_number, || { let shuffled_indices = - shuffle_validator_indices(block_number, n_validators); + shuffle_availability_chunks(block_number, n_validators); shuffled_indices[session_info.our_index.0 as usize] }) .expect("no expected"); diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index 4a5e3c2d5dce..2b66c1a25ed9 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -19,7 +19,7 @@ #![warn(missing_docs)] use std::{ - collections::{HashMap, VecDeque}, + collections::{BTreeMap, HashMap, VecDeque}, iter::Iterator, num::NonZeroUsize, pin::Pin, @@ -42,7 +42,8 @@ use task::{ use fatality::Nested; use polkadot_erasure_coding::{ - branch_hash, branches, obtain_chunks_v1, recovery_threshold, Error as ErasureEncodingError, + branch_hash, branches, obtain_chunks_v1, recovery_threshold, systematic_recovery_threshold, + Error as ErasureEncodingError, }; use task::{RecoveryParams, RecoveryStrategy, RecoveryTask}; @@ -58,7 +59,7 @@ use polkadot_node_subsystem::{ overseer, ActiveLeavesUpdate, FromOrchestra, OverseerSignal, SpawnedSubsystem, SubsystemContext, SubsystemError, SubsystemResult, }; -use polkadot_node_subsystem_util::{request_session_info, shuffle_validator_indices}; +use polkadot_node_subsystem_util::{request_session_info, shuffle_availability_chunks}; use polkadot_primitives::{ BlakeTwo256, BlockNumber, CandidateHash, CandidateReceipt, GroupIndex, Hash, HashT, SessionIndex, SessionInfo, ValidatorIndex, @@ -100,6 +101,8 @@ pub enum RecoveryStrategyKind { /// availability-store subsystem is not expected to run, /// such as collators. BypassAvailabilityStore, + /// Always recover using systematic chunks, fall back to regular chunks. + SystematicChunks, } /// The Availability Recovery Subsystem. @@ -455,7 +458,8 @@ async fn handle_recover( let block_number = get_block_number(ctx.sender(), receipt.descriptor.relay_parent).await?; - let shuffling = shuffle_validator_indices(block_number, session_info.validators.len()); + let shuffling = + shuffle_availability_chunks(block_number, session_info.validators.len()); let backing_validators = if let Some(backing_group) = backing_group { session_info.validator_groups.get(backing_group) @@ -463,10 +467,22 @@ async fn handle_recover( None }; - // let fetch_chunks_params = FetchChunksParams { - // n_validators: session_info.validators.len(), - // erasure_task_tx: erasure_task_tx.clone(), - // }; + let fetch_chunks_params = FetchChunksParams { + validators: shuffling + .iter() + .enumerate() + .map(|(v_index, c_index)| { + ( + *c_index, + ValidatorIndex( + u32::try_from(v_index) + .expect("validator numbers should not exceed u32"), + ), + ) + }) + .collect(), + erasure_task_tx: erasure_task_tx.clone(), + }; let mut recovery_strategies: VecDeque< Box::Sender>>, @@ -476,7 +492,8 @@ async fn handle_recover( match (&recovery_strategy_kind, prefer_backing_group) { (RecoveryStrategyKind::BackersFirstAlways, true) | (RecoveryStrategyKind::BackersFirstIfSizeLower(_), true) | - (RecoveryStrategyKind::BypassAvailabilityStore, true) => + (RecoveryStrategyKind::BypassAvailabilityStore, true) | + (RecoveryStrategyKind::BackersThenSystematicChunks, true) => recovery_strategies.push_back(Box::new(FetchFull::new(FetchFullParams { validators: backing_validators.to_vec(), erasure_task_tx: erasure_task_tx.clone(), @@ -485,18 +502,39 @@ async fn handle_recover( }; } - if recovery_strategy_kind == RecoveryStrategyKind::BackersThenSystematicChunks { - recovery_strategies.push_back(Box::new(FetchSystematicChunks::new( - FetchSystematicChunksParams { - validators: (0..recovery_threshold(session_info.validators.len()).unwrap()) - .map(|i| (shuffling[i], ValidatorIndex(i as u32))) - .collect(), - erasure_task_tx, + if matches!( + recovery_strategy_kind, + RecoveryStrategyKind::BackersThenSystematicChunks | + RecoveryStrategyKind::SystematicChunks + ) { + let systematic_threshold = + systematic_recovery_threshold(session_info.validators.len())?; + + let validators = shuffling.into_iter().enumerate().fold( + BTreeMap::new(), + |mut acc, (v_index, c_index)| { + if usize::try_from(c_index.0) + .expect("usize is at least u32 bytes on all modern targets.") < + systematic_threshold + { + acc.insert( + c_index, + ValidatorIndex( + u32::try_from(v_index) + .expect("validator numbers should not exceed u32"), + ), + ); + } + acc }, + ); + + recovery_strategies.push_back(Box::new(FetchSystematicChunks::new( + FetchSystematicChunksParams { validators, erasure_task_tx }, ))); } - // recovery_strategies.push_back(Box::new(FetchChunks::new(fetch_chunks_params))); + recovery_strategies.push_back(Box::new(FetchChunks::new(fetch_chunks_params))); launch_recovery_task( state, @@ -608,6 +646,19 @@ impl AvailabilityRecoverySubsystem { } } + /// Create a new instance of `AvailabilityRecoverySubsystem` which first attempts to request + /// systematic chunks, with a fallback to requesting regular chunks. + pub fn with_systematic_chunks( + req_receiver: IncomingRequestReceiver, + metrics: Metrics, + ) -> Self { + Self { + recovery_strategy_kind: RecoveryStrategyKind::SystematicChunks, + req_receiver, + metrics, + } + } + async fn run(self, mut ctx: Context) -> SubsystemResult<()> { let mut state = State::default(); let Self { mut req_receiver, metrics, recovery_strategy_kind } = self; diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index f70aab6919f5..f9b440c9c620 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -63,7 +63,7 @@ const TIMEOUT_START_NEW_REQUESTS: Duration = Duration::from_millis(100); pub trait RecoveryStrategy: Send { /// Main entry point of the strategy. async fn run( - &mut self, + mut self: Box, state: &mut State, sender: &mut Sender, common_params: &RecoveryParams, @@ -234,7 +234,7 @@ impl State { &mut self, params: &RecoveryParams, validators: &mut VecDeque<(ChunkIndex, ValidatorIndex)>, - can_conclude: impl Fn(usize, usize, usize, &RecoveryParams, usize) -> bool, + can_conclude: impl Fn(usize, usize, usize, usize, usize) -> bool, ) -> (usize, usize) { let metrics = ¶ms.metrics; @@ -256,7 +256,7 @@ impl State { gum::trace!( target: LOG_TARGET, candidate_hash = ?params.candidate_hash, - validator_index = ?chunk.index, + chunk_index = ?chunk.index, "Received valid chunk", ); self.insert_chunk(chunk.index, chunk); @@ -276,6 +276,7 @@ impl State { candidate_hash= ?params.candidate_hash, err = ?e, ?validator_index, + ?chunk_index, "Failure requesting chunk", ); @@ -311,14 +312,12 @@ impl State { }, } - // Stop waiting for requests when we either can already recover the data - // or have gotten firm 'No' responses from enough validators. if can_conclude( validators.len(), self.requesting_chunks.total_len(), self.chunk_count(), - params, error_count, + total_received_responses - error_count, ) { gum::debug!( target: LOG_TARGET, @@ -395,17 +394,19 @@ where let _timer = self.params.metrics.time_full_recovery(); - while let Some(mut current_strategy) = self.strategies.pop_front() { + while let Some(current_strategy) = self.strategies.pop_front() { // Make sure we are not referencing futures from past RecoveryStrategy runs. if self.state.requesting_chunks.total_len() != 0 { self.state.requesting_chunks = FuturesUndead::new(); } + let display_name = current_strategy.display_name(); + gum::debug!( target: LOG_TARGET, candidate_hash = ?self.params.candidate_hash, "Starting `{}` strategy", - current_strategy.display_name(), + display_name ); let res = current_strategy.run(&mut self.state, &mut self.sender, &self.params).await; @@ -417,7 +418,7 @@ where target: LOG_TARGET, candidate_hash = ?self.params.candidate_hash, "Recovery strategy `{}` did not conclude. Trying the next one.", - current_strategy.display_name(), + display_name ); continue }, @@ -475,7 +476,7 @@ impl RecoveryStrategy } async fn run( - &mut self, + mut self: Box, _: &mut State, sender: &mut Sender, common_params: &RecoveryParams, @@ -572,8 +573,6 @@ impl FetchSystematicChunks { /// Instantiate a new systematic chunks strategy. pub fn new(params: FetchSystematicChunksParams) -> Self { Self { - // We maintain the copy of the systematic validators in the params field, so that we - // can use them when doing the recovery. threshold: params.validators.len(), validators: params.validators, erasure_task_tx: params.erasure_task_tx, @@ -615,10 +614,16 @@ impl FetchSystematicChunks { common_params: &RecoveryParams, ) -> Result { let recovery_duration = common_params.metrics.time_erasure_recovery(); + let mut chunks = state + .received_chunks + .iter() + .filter(|(c_index, _)| (c_index.0 as usize) < self.threshold) + .collect::>(); + chunks.sort_by(|(index_a, _), (index_b, _)| index_a.cmp(index_b)); let available_data = polkadot_erasure_coding::reconstruct_from_systematic_v1( common_params.n_validators, - state.received_chunks.iter().map(|(_, chunk)| &chunk.chunk[..]).collect(), + chunks.into_iter().map(|(_, chunk)| &chunk.chunk[..]).collect(), ); match available_data { @@ -689,7 +694,7 @@ impl RecoveryStrategy } async fn run( - &mut self, + mut self: Box, state: &mut State, sender: &mut Sender, common_params: &RecoveryParams, @@ -698,8 +703,6 @@ impl RecoveryStrategy if !common_params.bypass_availability_store { let local_chunk_indices = state.populate_from_av_store(common_params, sender).await; - self.validators.retain(|c_index, _| !local_chunk_indices.contains(c_index)); - for c_index in &local_chunk_indices { // If we are among the systematic validators but hold an invalid chunk, we cannot // perform the systematic recovery. Fall through to the next strategy. @@ -730,6 +733,11 @@ impl RecoveryStrategy self.validators .retain(|c_index, _| !state.received_chunks.contains_key(c_index)); + // Safe to `take` here, as we're consuming `self` anyway and we're not using the + // `validators` field in other methods. + let mut validators_queue: VecDeque<_> = + std::mem::take(&mut self.validators).into_iter().collect(); + loop { // If received_chunks has `systematic_chunk_threshold` entries, attempt to recover the // data. If that fails, or a re-encoding of it doesn't match the expected erasure root, @@ -739,7 +747,7 @@ impl RecoveryStrategy } if Self::is_unavailable( - self.validators.len(), + validators_queue.len(), state.requesting_chunks.total_len(), systematic_chunk_count, self.threshold, @@ -772,7 +780,6 @@ impl RecoveryStrategy "Requesting systematic availability chunks for a candidate", ); - let mut validators_queue = std::mem::take(&mut self.validators).into_iter().collect(); state .launch_parallel_chunk_requests( common_params, @@ -786,15 +793,21 @@ impl RecoveryStrategy .wait_for_chunks( common_params, &mut validators_queue, - |unrequested_validators, reqs, chunk_count, _params, error_count| { - error_count != 0 || - chunk_count >= self.threshold || - Self::is_unavailable( - unrequested_validators, - reqs, - chunk_count, - self.threshold, - ) + |unrequested_validators, + in_flight_reqs, + _chunk_count, + error_count, + success_responses| { + let is_unavailable = Self::is_unavailable( + unrequested_validators, + in_flight_reqs, + systematic_chunk_count + success_responses, + self.threshold, + ); + + error_count > 0 || + (systematic_chunk_count + success_responses) >= self.threshold || + is_unavailable }, ) .await; @@ -830,7 +843,7 @@ pub struct FetchChunks { /// A random shuffling of the validators which indicates the order in which we connect to the /// validators and request the chunk from them. - validators: VecDeque, + validators: VecDeque<(ChunkIndex, ValidatorIndex)>, /// Channel to the erasure task handler. erasure_task_tx: futures::channel::mpsc::Sender, @@ -838,24 +851,19 @@ pub struct FetchChunks { /// Parameters specific to the `FetchChunks` strategy. pub struct FetchChunksParams { - /// Total number of validators. - pub n_validators: usize, + pub validators: VecDeque<(ChunkIndex, ValidatorIndex)>, /// Channel to the erasure task handler. pub erasure_task_tx: futures::channel::mpsc::Sender, } impl FetchChunks { /// Instantiate a new strategy. - pub fn new(params: FetchChunksParams) -> Self { - let mut shuffling: Vec<_> = (0..params.n_validators) - .map(|i| ValidatorIndex(i.try_into().expect("number of validators must fit in a u32"))) - .collect(); - shuffling.shuffle(&mut rand::thread_rng()); - + pub fn new(mut params: FetchChunksParams) -> Self { + params.validators.make_contiguous().shuffle(&mut rand::thread_rng()); Self { error_count: 0, total_received_responses: 0, - validators: shuffling.into(), + validators: params.validators, erasure_task_tx: params.erasure_task_tx, } } @@ -978,7 +986,7 @@ impl RecoveryStrategy } async fn run( - &mut self, + mut self: Box, state: &mut State, sender: &mut Sender, common_params: &RecoveryParams, @@ -986,11 +994,16 @@ impl RecoveryStrategy // First query the store for any chunks we've got. if !common_params.bypass_availability_store { let local_chunk_indices = state.populate_from_av_store(common_params, sender).await; - self.validators.retain(|i| !local_chunk_indices.contains(i)); + self.validators.retain(|(c_index, _)| !local_chunk_indices.contains(c_index)); } // No need to query the validators that have the chunks we already received. - self.validators.retain(|i| !state.received_chunks.contains_key(i)); + self.validators + .retain(|(c_index, _)| !state.received_chunks.contains_key(c_index)); + + // Safe to `take` here, as we're consuming `self` anyway and we're not using the + // `validators` field in other methods. + let mut validators_queue = std::mem::take(&mut self.validators); loop { // If received_chunks has more than threshold entries, attempt to recover the data. @@ -1003,7 +1016,7 @@ impl RecoveryStrategy } if Self::is_unavailable( - self.validators.len(), + validators_queue.len(), state.requesting_chunks.total_len(), state.chunk_count(), common_params.threshold, @@ -1036,10 +1049,6 @@ impl RecoveryStrategy "Requesting availability chunks for a candidate", ); - let mut validators_queue = std::mem::take(&mut self.validators) - .into_iter() - .map(|v_index| (v_index, v_index)) - .collect(); state .launch_parallel_chunk_requests( common_params, @@ -1053,13 +1062,17 @@ impl RecoveryStrategy .wait_for_chunks( common_params, &mut validators_queue, - |unrequested_validators, reqs, chunk_count, params, _error_count| { - chunk_count >= params.threshold || + |unrequested_validators, + in_flight_reqs, + chunk_count, + _error_count, + _success_responses| { + chunk_count >= common_params.threshold || Self::is_unavailable( unrequested_validators, - reqs, + in_flight_reqs, chunk_count, - params.threshold, + common_params.threshold, ) }, ) @@ -1082,8 +1095,10 @@ mod tests { let threshold = recovery_threshold(num_validators).unwrap(); let (erasure_task_tx, _erasure_task_rx) = futures::channel::mpsc::channel(16); - let mut fetch_chunks_task = - FetchChunks::new(FetchChunksParams { n_validators: 100, erasure_task_tx }); + let mut fetch_chunks_task = FetchChunks::new(FetchChunksParams { + validators: (0..100u32).map(|i| (ValidatorIndex(i), ValidatorIndex(i))).collect(), + erasure_task_tx, + }); assert_eq!(fetch_chunks_task.get_desired_request_count(0, threshold), threshold); fetch_chunks_task.error_count = 1; fetch_chunks_task.total_received_responses = 1; diff --git a/polkadot/node/overseer/src/lib.rs b/polkadot/node/overseer/src/lib.rs index 7489a00824c7..debac7000d46 100644 --- a/polkadot/node/overseer/src/lib.rs +++ b/polkadot/node/overseer/src/lib.rs @@ -515,6 +515,7 @@ pub struct Overseer { AvailabilityStoreMessage, RuntimeApiMessage, BitfieldDistributionMessage, + ChainApiMessage ])] bitfield_signing: BitfieldSigning, diff --git a/polkadot/node/service/src/overseer.rs b/polkadot/node/service/src/overseer.rs index 33127b638e5a..8aadb2f8dff3 100644 --- a/polkadot/node/service/src/overseer.rs +++ b/polkadot/node/service/src/overseer.rs @@ -251,7 +251,7 @@ where IncomingRequestReceivers { pov_req_receiver, chunk_req_receiver }, Metrics::register(registry)?, )) - .availability_recovery(AvailabilityRecoverySubsystem::with_chunks_if_pov_large( + .availability_recovery(AvailabilityRecoverySubsystem::with_systematic_chunks( available_data_req_receiver, Metrics::register(registry)?, )) diff --git a/polkadot/node/subsystem-util/src/lib.rs b/polkadot/node/subsystem-util/src/lib.rs index e2aa188a9c8a..4c7735f18693 100644 --- a/polkadot/node/subsystem-util/src/lib.rs +++ b/polkadot/node/subsystem-util/src/lib.rs @@ -385,23 +385,20 @@ pub struct Validator { impl Validator { /// Get a struct representing this node's validator if this node is in fact a validator in the /// context of the given block. - pub async fn new(parent: Hash, keystore: KeystorePtr, sender: &mut S) -> Result + pub async fn new( + validators: &[ValidatorId], + parent: Hash, + keystore: KeystorePtr, + sender: &mut S, + ) -> Result where S: SubsystemSender, { - // Note: request_validators and request_session_index_for_child do not and cannot - // run concurrently: they both have a mutable handle to the same sender. - // However, each of them returns a oneshot::Receiver, and those are resolved concurrently. - let (validators, session_index) = futures::try_join!( - request_validators(parent, sender).await, - request_session_index_for_child(parent, sender).await, - )?; + let session_index = request_session_index_for_child(parent, sender).await.await??; - let signing_context = SigningContext { session_index: session_index?, parent_hash: parent }; + let signing_context = SigningContext { session_index, parent_hash: parent }; - let validators = validators?; - - Self::construct(&validators, signing_context, keystore) + Self::construct(validators, signing_context, keystore) } /// Construct a validator instance without performing runtime fetches. @@ -443,13 +440,21 @@ impl Validator { } } -pub fn shuffle_validator_indices( +/// Shuffle the availability chunk indices, returning a mapping of (`ValidatorIndex -> ChunkIndex`). +/// +/// The vector indices represent validator indices. +/// `BlockNumber` is used a randomness seed, so that other validators have a common view of the +/// shuffle at a given block height. +pub fn shuffle_availability_chunks( block_number: BlockNumber, n_validators: usize, ) -> Vec { let seed = block_number.to_be_bytes(); - let mut rng: ChaCha8Rng = - SeedableRng::from_seed(seed.repeat(8).try_into().expect("should never fail")); + let mut rng: ChaCha8Rng = SeedableRng::from_seed( + seed.repeat(8) + .try_into() + .expect("vector of 32 bytes is safe to cast to array of 32 bytes. qed."), + ); let mut shuffled_indices: Vec<_> = (0..n_validators).map(|i| ValidatorIndex(i as _)).collect(); diff --git a/polkadot/node/subsystem-util/src/tests.rs b/polkadot/node/subsystem-util/src/tests.rs index 9ade95d4e894..7b21b34b046d 100644 --- a/polkadot/node/subsystem-util/src/tests.rs +++ b/polkadot/node/subsystem-util/src/tests.rs @@ -21,6 +21,7 @@ use executor::block_on; use futures::{channel::mpsc, executor, FutureExt, SinkExt, StreamExt}; use polkadot_primitives_test_helpers::AlwaysZeroRng; use std::{ + collections::HashSet, sync::{ atomic::{AtomicUsize, Ordering}, Arc, @@ -94,3 +95,15 @@ fn subset_predefined_generation_check() { assert_eq!(v as usize, idx + 1); } } + +#[test] +// Just a simple test to make sure that the shuffling is indeed a permutation, to prevent future +// mistakes. +fn test_shuffle_availability_chunks() { + let block_number = 89; + let n_validators = 200; + let shuffle = shuffle_availability_chunks(block_number, n_validators); + + assert_eq!(shuffle.len(), n_validators); + assert_eq!(shuffle.iter().collect::>().len(), n_validators); +} diff --git a/polkadot/primitives/src/v5/mod.rs b/polkadot/primitives/src/v5/mod.rs index fb2d040d90e0..1f75b2c0bcfa 100644 --- a/polkadot/primitives/src/v5/mod.rs +++ b/polkadot/primitives/src/v5/mod.rs @@ -109,6 +109,8 @@ pub trait TypeIndex { #[cfg_attr(feature = "std", derive(Serialize, Deserialize, Hash))] pub struct ValidatorIndex(pub u32); +/// Type alias for chunk index. The chunk index held by a validator may not always be equal to its +/// `ValidatorIndex`, so we use a separate type to make code easier to read. pub type ChunkIndex = ValidatorIndex; // We should really get https://github.com/paritytech/polkadot/issues/2403 going .. From a805a855bbf92c8a13badb89226a05bbaeafedee Mon Sep 17 00:00:00 2001 From: alindima Date: Thu, 14 Sep 2023 17:52:17 +0300 Subject: [PATCH 009/126] Fix some tests --- .../node/core/bitfield-signing/src/tests.rs | 6 + .../availability-recovery/src/tests.rs | 239 +++++++++--------- 2 files changed, 129 insertions(+), 116 deletions(-) diff --git a/polkadot/node/core/bitfield-signing/src/tests.rs b/polkadot/node/core/bitfield-signing/src/tests.rs index 106ecc06b156..02173b65d4f9 100644 --- a/polkadot/node/core/bitfield-signing/src/tests.rs +++ b/polkadot/node/core/bitfield-signing/src/tests.rs @@ -43,6 +43,7 @@ fn construct_availability_bitfield_works() { let future = construct_availability_bitfield( relay_parent, &jaeger::Span::Disabled, + 10, validator_index, &mut sender, ) @@ -68,6 +69,11 @@ fn construct_availability_bitfield_works() { tx.send(c_hash == hash_a).unwrap(); }, + AllMessages::ChainApi( + ChainApiMessage::BlockNumber(_, tx) + ) => { + tx.send(Ok(Some(1))).unwrap(); + } o => panic!("Unknown message: {:?}", o), }, r = future => match r { diff --git a/polkadot/node/network/availability-recovery/src/tests.rs b/polkadot/node/network/availability-recovery/src/tests.rs index 17b02080c008..66808ba52ffc 100644 --- a/polkadot/node/network/availability-recovery/src/tests.rs +++ b/polkadot/node/network/availability-recovery/src/tests.rs @@ -22,12 +22,13 @@ use futures_timer::Delay; use parity_scale_codec::Encode; use polkadot_node_network_protocol::request_response::{ - self as req_res, IncomingRequest, Recipient, ReqProtocolNames, Requests, + self as req_res, v1::AvailableDataFetchingRequest, IncomingRequest, Recipient, + ReqProtocolNames, Requests, }; use super::*; -use sc_network::{config::RequestResponseConfig, IfDisconnected, OutboundFailure, RequestFailure}; +use sc_network::{IfDisconnected, OutboundFailure, RequestFailure}; use polkadot_erasure_coding::{branches, obtain_chunks_v1 as obtain_chunks}; use polkadot_node_primitives::{BlockData, PoV, Proof}; @@ -48,43 +49,17 @@ type VirtualOverseer = TestSubsystemContextHandle; // Deterministic genesis hash for protocol names const GENESIS_HASH: Hash = Hash::repeat_byte(0xff); -fn test_harness_fast_path>( - test: impl FnOnce(VirtualOverseer, RequestResponseConfig) -> T, -) { - let _ = env_logger::builder() - .is_test(true) - .filter(Some("polkadot_availability_recovery"), log::LevelFilter::Trace) - .try_init(); - - let pool = sp_core::testing::TaskExecutor::new(); - - let (context, virtual_overseer) = make_subsystem_context(pool.clone()); - - let (collation_req_receiver, req_cfg) = +fn request_receiver() -> IncomingRequestReceiver { + let receiver = IncomingRequest::get_config_receiver(&ReqProtocolNames::new(&GENESIS_HASH, None)); - let subsystem = - AvailabilityRecoverySubsystem::with_fast_path(collation_req_receiver, Metrics::new_dummy()); - let subsystem = async { - subsystem.run(context).await.unwrap(); - }; - - let test_fut = test(virtual_overseer, req_cfg); - - futures::pin_mut!(test_fut); - futures::pin_mut!(subsystem); - - executor::block_on(future::join( - async move { - let (mut overseer, _req_cfg) = test_fut.await; - overseer_signal(&mut overseer, OverseerSignal::Conclude).await; - }, - subsystem, - )) - .1 + // Don't close the sending end of the request protocol. Otherwise, the subsystem will terminate. + std::mem::forget(receiver.1.inbound_queue); + receiver.0 } -fn test_harness_chunks_only>( - test: impl FnOnce(VirtualOverseer, RequestResponseConfig) -> T, +fn test_harness>( + subsystem: AvailabilityRecoverySubsystem, + test: impl FnOnce(VirtualOverseer) -> Fut, ) { let _ = env_logger::builder() .is_test(true) @@ -95,66 +70,23 @@ fn test_harness_chunks_only, ->( - test: impl FnOnce(VirtualOverseer, RequestResponseConfig) -> T, -) { - let _ = env_logger::builder() - .is_test(true) - .filter(Some("polkadot_availability_recovery"), log::LevelFilter::Trace) - .try_init(); - - let pool = sp_core::testing::TaskExecutor::new(); - - let (context, virtual_overseer) = make_subsystem_context(pool.clone()); - - let (collation_req_receiver, req_cfg) = - IncomingRequest::get_config_receiver(&ReqProtocolNames::new(&GENESIS_HASH, None)); - let subsystem = AvailabilityRecoverySubsystem::with_chunks_if_pov_large( - collation_req_receiver, - Metrics::new_dummy(), - ); - let subsystem = subsystem.run(context); + let subsystem = async { + subsystem.run(context).await.unwrap(); + }; - let test_fut = test(virtual_overseer, req_cfg); + let test_fut = test(virtual_overseer); futures::pin_mut!(test_fut); futures::pin_mut!(subsystem); executor::block_on(future::join( async move { - let (mut overseer, _req_cfg) = test_fut.await; + let mut overseer = test_fut.await; overseer_signal(&mut overseer, OverseerSignal::Conclude).await; }, subsystem, )) .1 - .unwrap(); } const TIMEOUT: Duration = Duration::from_millis(300); @@ -280,6 +212,21 @@ impl TestState { ); } + async fn respond_to_block_number_query( + &self, + virtual_overseer: &mut VirtualOverseer, + block_number: BlockNumber, + ) { + assert_matches!( + overseer_recv(virtual_overseer).await, + AllMessages::ChainApi( + ChainApiMessage::BlockNumber(_, tx) + ) => { + let _ = tx.send(Ok(Some(block_number))); + } + ) + } + async fn respond_to_available_data_query( &self, virtual_overseer: &mut VirtualOverseer, @@ -559,8 +506,10 @@ impl Default for TestState { #[test] fn availability_is_recovered_from_chunks_if_no_group_provided() { let test_state = TestState::default(); + let subsystem = + AvailabilityRecoverySubsystem::with_fast_path(request_receiver(), Metrics::new_dummy()); - test_harness_fast_path(|mut virtual_overseer, req_cfg| async move { + test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( &mut virtual_overseer, OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( @@ -584,6 +533,7 @@ fn availability_is_recovered_from_chunks_if_no_group_provided() { .await; test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; let candidate_hash = test_state.candidate.hash(); @@ -621,6 +571,7 @@ fn availability_is_recovered_from_chunks_if_no_group_provided() { .await; test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; @@ -636,15 +587,17 @@ fn availability_is_recovered_from_chunks_if_no_group_provided() { // A request times out with `Unavailable` error. assert_eq!(rx.await.unwrap().unwrap_err(), RecoveryError::Unavailable); - (virtual_overseer, req_cfg) + virtual_overseer }); } #[test] fn availability_is_recovered_from_chunks_even_if_backing_group_supplied_if_chunks_only() { let test_state = TestState::default(); + let subsystem = + AvailabilityRecoverySubsystem::with_chunks_only(request_receiver(), Metrics::new_dummy()); - test_harness_chunks_only(|mut virtual_overseer, req_cfg| async move { + test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( &mut virtual_overseer, OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( @@ -668,6 +621,7 @@ fn availability_is_recovered_from_chunks_even_if_backing_group_supplied_if_chunk .await; test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; let candidate_hash = test_state.candidate.hash(); @@ -705,6 +659,7 @@ fn availability_is_recovered_from_chunks_even_if_backing_group_supplied_if_chunk .await; test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; @@ -720,15 +675,17 @@ fn availability_is_recovered_from_chunks_even_if_backing_group_supplied_if_chunk // A request times out with `Unavailable` error. assert_eq!(rx.await.unwrap().unwrap_err(), RecoveryError::Unavailable); - (virtual_overseer, req_cfg) + virtual_overseer }); } #[test] fn bad_merkle_path_leads_to_recovery_error() { let mut test_state = TestState::default(); + let subsystem = + AvailabilityRecoverySubsystem::with_fast_path(request_receiver(), Metrics::new_dummy()); - test_harness_fast_path(|mut virtual_overseer, req_cfg| async move { + test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( &mut virtual_overseer, OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( @@ -752,6 +709,7 @@ fn bad_merkle_path_leads_to_recovery_error() { .await; test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; let candidate_hash = test_state.candidate.hash(); @@ -776,7 +734,7 @@ fn bad_merkle_path_leads_to_recovery_error() { // A request times out with `Unavailable` error. assert_eq!(rx.await.unwrap().unwrap_err(), RecoveryError::Unavailable); - (virtual_overseer, req_cfg) + virtual_overseer }); } @@ -784,7 +742,10 @@ fn bad_merkle_path_leads_to_recovery_error() { fn wrong_chunk_index_leads_to_recovery_error() { let mut test_state = TestState::default(); - test_harness_fast_path(|mut virtual_overseer, req_cfg| async move { + let subsystem = + AvailabilityRecoverySubsystem::with_fast_path(request_receiver(), Metrics::new_dummy()); + + test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( &mut virtual_overseer, OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( @@ -808,6 +769,7 @@ fn wrong_chunk_index_leads_to_recovery_error() { .await; test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; let candidate_hash = test_state.candidate.hash(); @@ -832,7 +794,7 @@ fn wrong_chunk_index_leads_to_recovery_error() { // A request times out with `Unavailable` error as there are no good peers. assert_eq!(rx.await.unwrap().unwrap_err(), RecoveryError::Unavailable); - (virtual_overseer, req_cfg) + virtual_overseer }); } @@ -840,7 +802,10 @@ fn wrong_chunk_index_leads_to_recovery_error() { fn invalid_erasure_coding_leads_to_invalid_error() { let mut test_state = TestState::default(); - test_harness_fast_path(|mut virtual_overseer, req_cfg| async move { + let subsystem = + AvailabilityRecoverySubsystem::with_fast_path(request_receiver(), Metrics::new_dummy()); + + test_harness(subsystem, |mut virtual_overseer| async move { let pov = PoV { block_data: BlockData(vec![69; 64]) }; let (bad_chunks, bad_erasure_root) = derive_erasure_chunks_with_proofs_and_root( @@ -880,6 +845,7 @@ fn invalid_erasure_coding_leads_to_invalid_error() { .await; test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; @@ -895,15 +861,17 @@ fn invalid_erasure_coding_leads_to_invalid_error() { // f+1 'valid' chunks can't produce correct data. assert_eq!(rx.await.unwrap().unwrap_err(), RecoveryError::Invalid); - (virtual_overseer, req_cfg) + virtual_overseer }); } #[test] fn fast_path_backing_group_recovers() { let test_state = TestState::default(); + let subsystem = + AvailabilityRecoverySubsystem::with_fast_path(request_receiver(), Metrics::new_dummy()); - test_harness_fast_path(|mut virtual_overseer, req_cfg| async move { + test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( &mut virtual_overseer, OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( @@ -927,6 +895,7 @@ fn fast_path_backing_group_recovers() { .await; test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; let candidate_hash = test_state.candidate.hash(); @@ -943,15 +912,19 @@ fn fast_path_backing_group_recovers() { // Recovered data should match the original one. assert_eq!(rx.await.unwrap().unwrap(), test_state.available_data); - (virtual_overseer, req_cfg) + virtual_overseer }); } #[test] fn recovers_from_only_chunks_if_pov_large() { let test_state = TestState::default(); + let subsystem = AvailabilityRecoverySubsystem::with_chunks_if_pov_large( + request_receiver(), + Metrics::new_dummy(), + ); - test_harness_chunks_if_pov_large(|mut virtual_overseer, req_cfg| async move { + test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( &mut virtual_overseer, OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( @@ -987,6 +960,8 @@ fn recovers_from_only_chunks_if_pov_large() { } ); + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; @@ -1031,6 +1006,8 @@ fn recovers_from_only_chunks_if_pov_large() { } ); + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; @@ -1045,15 +1022,19 @@ fn recovers_from_only_chunks_if_pov_large() { // A request times out with `Unavailable` error. assert_eq!(rx.await.unwrap().unwrap_err(), RecoveryError::Unavailable); - (virtual_overseer, req_cfg) + virtual_overseer }); } #[test] fn fast_path_backing_group_recovers_if_pov_small() { let test_state = TestState::default(); + let subsystem = AvailabilityRecoverySubsystem::with_chunks_if_pov_large( + request_receiver(), + Metrics::new_dummy(), + ); - test_harness_chunks_if_pov_large(|mut virtual_overseer, req_cfg| async move { + test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( &mut virtual_overseer, OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( @@ -1094,6 +1075,8 @@ fn fast_path_backing_group_recovers_if_pov_small() { } ); + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state @@ -1102,15 +1085,17 @@ fn fast_path_backing_group_recovers_if_pov_small() { // Recovered data should match the original one. assert_eq!(rx.await.unwrap().unwrap(), test_state.available_data); - (virtual_overseer, req_cfg) + virtual_overseer }); } #[test] fn no_answers_in_fast_path_causes_chunk_requests() { let test_state = TestState::default(); + let subsystem = + AvailabilityRecoverySubsystem::with_fast_path(request_receiver(), Metrics::new_dummy()); - test_harness_fast_path(|mut virtual_overseer, req_cfg| async move { + test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( &mut virtual_overseer, OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( @@ -1134,6 +1119,7 @@ fn no_answers_in_fast_path_causes_chunk_requests() { .await; test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; let candidate_hash = test_state.candidate.hash(); @@ -1162,15 +1148,17 @@ fn no_answers_in_fast_path_causes_chunk_requests() { // Recovered data should match the original one. assert_eq!(rx.await.unwrap().unwrap(), test_state.available_data); - (virtual_overseer, req_cfg) + virtual_overseer }); } #[test] fn task_canceled_when_receivers_dropped() { let test_state = TestState::default(); + let subsystem = + AvailabilityRecoverySubsystem::with_chunks_only(request_receiver(), Metrics::new_dummy()); - test_harness_chunks_only(|mut virtual_overseer, req_cfg| async move { + test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( &mut virtual_overseer, OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( @@ -1194,10 +1182,11 @@ fn task_canceled_when_receivers_dropped() { .await; test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; for _ in 0..test_state.validators.len() { match virtual_overseer.recv().timeout(TIMEOUT).await { - None => return (virtual_overseer, req_cfg), + None => return virtual_overseer, Some(_) => continue, } } @@ -1209,8 +1198,10 @@ fn task_canceled_when_receivers_dropped() { #[test] fn chunks_retry_until_all_nodes_respond() { let test_state = TestState::default(); + let subsystem = + AvailabilityRecoverySubsystem::with_chunks_only(request_receiver(), Metrics::new_dummy()); - test_harness_chunks_only(|mut virtual_overseer, req_cfg| async move { + test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( &mut virtual_overseer, OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( @@ -1234,6 +1225,7 @@ fn chunks_retry_until_all_nodes_respond() { .await; test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; let candidate_hash = test_state.candidate.hash(); @@ -1261,15 +1253,17 @@ fn chunks_retry_until_all_nodes_respond() { // Recovered data should match the original one. assert_eq!(rx.await.unwrap().unwrap_err(), RecoveryError::Unavailable); - (virtual_overseer, req_cfg) + virtual_overseer }); } #[test] fn not_returning_requests_wont_stall_retrieval() { let test_state = TestState::default(); + let subsystem = + AvailabilityRecoverySubsystem::with_chunks_only(request_receiver(), Metrics::new_dummy()); - test_harness_chunks_only(|mut virtual_overseer, req_cfg| async move { + test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( &mut virtual_overseer, OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( @@ -1293,6 +1287,7 @@ fn not_returning_requests_wont_stall_retrieval() { .await; test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; let candidate_hash = test_state.candidate.hash(); @@ -1331,15 +1326,17 @@ fn not_returning_requests_wont_stall_retrieval() { // Recovered data should match the original one: assert_eq!(rx.await.unwrap().unwrap(), test_state.available_data); - (virtual_overseer, req_cfg) + virtual_overseer }); } #[test] fn all_not_returning_requests_still_recovers_on_return() { let test_state = TestState::default(); + let subsystem = + AvailabilityRecoverySubsystem::with_chunks_only(request_receiver(), Metrics::new_dummy()); - test_harness_chunks_only(|mut virtual_overseer, req_cfg| async move { + test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( &mut virtual_overseer, OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( @@ -1363,6 +1360,7 @@ fn all_not_returning_requests_still_recovers_on_return() { .await; test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; let candidate_hash = test_state.candidate.hash(); @@ -1406,15 +1404,17 @@ fn all_not_returning_requests_still_recovers_on_return() { // Recovered data should match the original one: assert_eq!(rx.await.unwrap().unwrap(), test_state.available_data); - (virtual_overseer, req_cfg) + virtual_overseer }); } #[test] fn returns_early_if_we_have_the_data() { let test_state = TestState::default(); + let subsystem = + AvailabilityRecoverySubsystem::with_chunks_only(request_receiver(), Metrics::new_dummy()); - test_harness_chunks_only(|mut virtual_overseer, req_cfg| async move { + test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( &mut virtual_overseer, OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( @@ -1438,18 +1438,21 @@ fn returns_early_if_we_have_the_data() { .await; test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; test_state.respond_to_available_data_query(&mut virtual_overseer, true).await; assert_eq!(rx.await.unwrap().unwrap(), test_state.available_data); - (virtual_overseer, req_cfg) + virtual_overseer }); } #[test] fn does_not_query_local_validator() { let test_state = TestState::default(); + let subsystem = + AvailabilityRecoverySubsystem::with_chunks_only(request_receiver(), Metrics::new_dummy()); - test_harness_chunks_only(|mut virtual_overseer, req_cfg| async move { + test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( &mut virtual_overseer, OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( @@ -1473,6 +1476,7 @@ fn does_not_query_local_validator() { .await; test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |i| i == 0).await; @@ -1498,15 +1502,17 @@ fn does_not_query_local_validator() { .await; assert_eq!(rx.await.unwrap().unwrap(), test_state.available_data); - (virtual_overseer, req_cfg) + virtual_overseer }); } #[test] fn invalid_local_chunk_is_ignored() { let test_state = TestState::default(); + let subsystem = + AvailabilityRecoverySubsystem::with_chunks_only(request_receiver(), Metrics::new_dummy()); - test_harness_chunks_only(|mut virtual_overseer, req_cfg| async move { + test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( &mut virtual_overseer, OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( @@ -1530,6 +1536,7 @@ fn invalid_local_chunk_is_ignored() { .await; test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state .respond_to_query_all_request_invalid(&mut virtual_overseer, |i| i == 0) @@ -1547,6 +1554,6 @@ fn invalid_local_chunk_is_ignored() { .await; assert_eq!(rx.await.unwrap().unwrap(), test_state.available_data); - (virtual_overseer, req_cfg) + virtual_overseer }); } From f3d407b7b626e71214621fa1179b9ecc802ccbac Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 15 Sep 2023 10:05:14 +0300 Subject: [PATCH 010/126] replace hashmap of chunks with btreemap this avoids sorting the chunks for systematic recovery, which is the hotpath we aim to optimise --- .../network/availability-recovery/src/lib.rs | 14 +++++++---- .../network/availability-recovery/src/task.rs | 24 ++++++++++++------- 2 files changed, 26 insertions(+), 12 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index 2b66c1a25ed9..804f36ce9693 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -19,7 +19,7 @@ #![warn(missing_docs)] use std::{ - collections::{BTreeMap, HashMap, VecDeque}, + collections::{BTreeMap, VecDeque}, iter::Iterator, num::NonZeroUsize, pin::Pin, @@ -61,7 +61,7 @@ use polkadot_node_subsystem::{ }; use polkadot_node_subsystem_util::{request_session_info, shuffle_availability_chunks}; use polkadot_primitives::{ - BlakeTwo256, BlockNumber, CandidateHash, CandidateReceipt, GroupIndex, Hash, HashT, + BlakeTwo256, BlockNumber, CandidateHash, CandidateReceipt, ChunkIndex, GroupIndex, Hash, HashT, SessionIndex, SessionInfo, ValidatorIndex, }; @@ -120,7 +120,7 @@ pub enum ErasureTask { /// Reconstructs `AvailableData` from chunks given `n_validators`. Reconstruct( usize, - HashMap, + BTreeMap, oneshot::Sender>, ), /// Re-encode `AvailableData` into erasure chunks in order to verify the provided root hash of @@ -875,7 +875,13 @@ async fn erasure_task_thread( Some(ErasureTask::Reconstruct(n_validators, chunks, sender)) => { let _ = sender.send(polkadot_erasure_coding::reconstruct_v1( n_validators, - chunks.values().map(|c| (&c.chunk[..], c.index.0 as usize)), + chunks.iter().map(|(c_index, chunk)| { + ( + &chunk.chunk[..], + usize::try_from(c_index.0) + .expect("usize is at least u32 bytes on all modern targets."), + ) + }), )); }, Some(ErasureTask::Reencode(n_validators, root, available_data, sender)) => { diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index f9b440c9c620..14e84e5a5003 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -37,7 +37,7 @@ use polkadot_primitives::{AuthorityDiscoveryId, CandidateHash, ChunkIndex, Hash, use rand::seq::SliceRandom; use sc_network::{IfDisconnected, OutboundFailure, RequestFailure}; use std::{ - collections::{BTreeMap, HashMap, VecDeque}, + collections::{BTreeMap, VecDeque}, time::Duration, }; @@ -101,7 +101,10 @@ pub struct RecoveryParams { /// same `RecoveryTask`. pub struct State { /// Chunks received so far. - received_chunks: HashMap, + /// This MUST be a `BTreeMap` in order for systematic recovery to work (the algorithm assumes + /// that chunks are ordered by their index). If we ever switch this to some non-ordered + /// collection, we need to add a sort step to the systematic recovery. + received_chunks: BTreeMap, /// Collection of in-flight requests. requesting_chunks: FuturesUndead, (ChunkIndex, ValidatorIndex, RequestError)>>, @@ -109,7 +112,7 @@ pub struct State { impl State { fn new() -> Self { - Self { received_chunks: HashMap::new(), requesting_chunks: FuturesUndead::new() } + Self { received_chunks: BTreeMap::new(), requesting_chunks: FuturesUndead::new() } } fn insert_chunk(&mut self, validator: ValidatorIndex, chunk: ErasureChunk) { @@ -614,16 +617,21 @@ impl FetchSystematicChunks { common_params: &RecoveryParams, ) -> Result { let recovery_duration = common_params.metrics.time_erasure_recovery(); - let mut chunks = state + let chunks = state .received_chunks - .iter() - .filter(|(c_index, _)| (c_index.0 as usize) < self.threshold) + .range( + ValidatorIndex(0).. + ValidatorIndex( + u32::try_from(self.threshold) + .expect("validator numbers should not exceed u32"), + ), + ) + .map(|(_, chunk)| &chunk.chunk[..]) .collect::>(); - chunks.sort_by(|(index_a, _), (index_b, _)| index_a.cmp(index_b)); let available_data = polkadot_erasure_coding::reconstruct_from_systematic_v1( common_params.n_validators, - chunks.into_iter().map(|(_, chunk)| &chunk.chunk[..]).collect(), + chunks, ); match available_data { From 67841bd3fafdf3c7bd0d707812decc467ce0339e Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 15 Sep 2023 10:34:10 +0300 Subject: [PATCH 011/126] add chunk indices cache to av-recovery --- .../src/requester/mod.rs | 17 +++--- .../network/availability-recovery/src/lib.rs | 55 ++++++++++--------- 2 files changed, 37 insertions(+), 35 deletions(-) diff --git a/polkadot/node/network/availability-distribution/src/requester/mod.rs b/polkadot/node/network/availability-distribution/src/requester/mod.rs index 02b0a8569f75..171feca2e0e4 100644 --- a/polkadot/node/network/availability-distribution/src/requester/mod.rs +++ b/polkadot/node/network/availability-distribution/src/requester/mod.rs @@ -39,7 +39,7 @@ use polkadot_node_subsystem_util::{ shuffle_availability_chunks, }; use polkadot_primitives::{ - BlockNumber, CandidateHash, Hash, OccupiedCore, SessionIndex, ValidatorIndex, + BlockNumber, CandidateHash, ChunkIndex, Hash, OccupiedCore, SessionIndex, }; use schnellru::{ByLength, LruMap}; @@ -81,7 +81,8 @@ pub struct Requester { /// Prometheus Metrics metrics: Metrics, - chunk_index_cache: LruMap, + /// Cache of our chunk indices based on the relay parent block. + chunk_index_cache: LruMap, } #[overseer::contextbounds(AvailabilityDistribution, prefix = self::overseer)] @@ -101,7 +102,9 @@ impl Requester { tx, rx, metrics, - chunk_index_cache: LruMap::new(ByLength::new(10)), + // Candidates shouldn't be pending avilability for many blocks, so keep our index for + // the last two relay parents. + chunk_index_cache: LruMap::new(ByLength::new(2)), } } @@ -229,9 +232,6 @@ impl Requester { span.add_string_tag("already-requested-chunk", "false"); let tx = self.tx.clone(); let metrics = self.metrics.clone(); - // only interested in the map for (ourIndex) -> ValidatorIndex - // hold LruCache - // alternatively, re-compute it according to algorithm let block_number = get_block_number(context.sender(), core.candidate_descriptor.relay_parent) .await?; @@ -270,9 +270,10 @@ impl Requester { .get_or_insert(block_number, || { let shuffled_indices = shuffle_availability_chunks(block_number, n_validators); - shuffled_indices[session_info.our_index.0 as usize] + shuffled_indices[usize::try_from(session_info.our_index.0) + .expect("usize is at least u32 bytes on all modern targets.")] }) - .expect("no expected"); + .expect("The index was just inserted"); let task_cfg = FetchTaskConfig::new( leaf, diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index 804f36ce9693..820143905c0f 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -300,6 +300,9 @@ struct State { /// An LRU cache of recently recovered data. availability_lru: LruMap, + + /// Cache of the chunk indices shuffle based on the relay parent block. + chunk_indices_cache: LruMap>, } impl Default for State { @@ -308,6 +311,7 @@ impl Default for State { ongoing_recoveries: FuturesUnordered::new(), live_block: (0, Hash::default()), availability_lru: LruMap::new(ByLength::new(LRU_SIZE)), + chunk_indices_cache: LruMap::new(ByLength::new(LRU_SIZE)), } } } @@ -458,8 +462,25 @@ async fn handle_recover( let block_number = get_block_number(ctx.sender(), receipt.descriptor.relay_parent).await?; - let shuffling = - shuffle_availability_chunks(block_number, session_info.validators.len()); + + let shuffling = state + .chunk_indices_cache + .get_or_insert(block_number, || { + shuffle_availability_chunks(block_number, session_info.validators.len()) + .iter() + .enumerate() + .map(|(v_index, c_index)| { + ( + *c_index, + ValidatorIndex( + u32::try_from(v_index) + .expect("validator numbers should not exceed u32"), + ), + ) + }) + .collect() + }) + .expect("The shuffling was just inserted"); let backing_validators = if let Some(backing_group) = backing_group { session_info.validator_groups.get(backing_group) @@ -468,19 +489,7 @@ async fn handle_recover( }; let fetch_chunks_params = FetchChunksParams { - validators: shuffling - .iter() - .enumerate() - .map(|(v_index, c_index)| { - ( - *c_index, - ValidatorIndex( - u32::try_from(v_index) - .expect("validator numbers should not exceed u32"), - ), - ) - }) - .collect(), + validators: shuffling.clone(), erasure_task_tx: erasure_task_tx.clone(), }; @@ -510,24 +519,16 @@ async fn handle_recover( let systematic_threshold = systematic_recovery_threshold(session_info.validators.len())?; - let validators = shuffling.into_iter().enumerate().fold( - BTreeMap::new(), - |mut acc, (v_index, c_index)| { + let validators = + shuffling.into_iter().fold(BTreeMap::new(), |mut acc, (c_index, v_index)| { if usize::try_from(c_index.0) .expect("usize is at least u32 bytes on all modern targets.") < systematic_threshold { - acc.insert( - c_index, - ValidatorIndex( - u32::try_from(v_index) - .expect("validator numbers should not exceed u32"), - ), - ); + acc.insert(*c_index, *v_index); } acc - }, - ); + }); recovery_strategies.push_back(Box::new(FetchSystematicChunks::new( FetchSystematicChunksParams { validators, erasure_task_tx }, From 3810eab0273e7ed3d8c7fd0868023d654b38bf58 Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 15 Sep 2023 10:58:59 +0300 Subject: [PATCH 012/126] some more improvements --- .../src/requester/mod.rs | 5 +- .../src/requester/session_cache.rs | 3 - .../network/availability-recovery/src/lib.rs | 16 +-- .../network/availability-recovery/src/task.rs | 111 +++++++++++------- 4 files changed, 77 insertions(+), 58 deletions(-) diff --git a/polkadot/node/network/availability-distribution/src/requester/mod.rs b/polkadot/node/network/availability-distribution/src/requester/mod.rs index 171feca2e0e4..845644abea4a 100644 --- a/polkadot/node/network/availability-distribution/src/requester/mod.rs +++ b/polkadot/node/network/availability-distribution/src/requester/mod.rs @@ -259,10 +259,9 @@ impl Requester { })?; if let Some(session_info) = session_info { - // TODO: optimise this n_validators calculation. let n_validators = - session_info.validator_groups.iter().fold(0, |mut acc, group| { - acc += group.len(); + session_info.validator_groups.iter().fold(0usize, |mut acc, group| { + acc = acc.saturating_add(group.len()); acc }); let chunk_index = self diff --git a/polkadot/node/network/availability-distribution/src/requester/session_cache.rs b/polkadot/node/network/availability-distribution/src/requester/session_cache.rs index 6bb8098e8317..821d778c22d4 100644 --- a/polkadot/node/network/availability-distribution/src/requester/session_cache.rs +++ b/polkadot/node/network/availability-distribution/src/requester/session_cache.rs @@ -176,9 +176,6 @@ impl SessionCache { // Get our group index: let our_group = info.validator_info.our_group; - // TODO: This shuffling is fine, as it only is used for knowing which validator to ask - // in order to get the chunk. The chunk index is coming from somewhere else. - // Shuffle validators in groups: let mut rng = thread_rng(); for g in validator_groups.iter_mut() { diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index 820143905c0f..4a5a8be50ec9 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -519,16 +519,16 @@ async fn handle_recover( let systematic_threshold = systematic_recovery_threshold(session_info.validators.len())?; - let validators = - shuffling.into_iter().fold(BTreeMap::new(), |mut acc, (c_index, v_index)| { - if usize::try_from(c_index.0) + // Only get the validators according to the threshold. + let validators = shuffling + .clone() + .into_iter() + .filter(|(c_index, _)| { + usize::try_from(c_index.0) .expect("usize is at least u32 bytes on all modern targets.") < systematic_threshold - { - acc.insert(*c_index, *v_index); - } - acc - }); + }) + .collect(); recovery_strategies.push_back(Box::new(FetchSystematicChunks::new( FetchSystematicChunksParams { validators, erasure_task_tx }, diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 14e84e5a5003..b037a1234026 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -107,7 +107,7 @@ pub struct State { received_chunks: BTreeMap, /// Collection of in-flight requests. requesting_chunks: - FuturesUndead, (ChunkIndex, ValidatorIndex, RequestError)>>, + FuturesUndead<(ChunkIndex, ValidatorIndex, Result, RequestError>)>, } impl State { @@ -184,7 +184,15 @@ impl State { let candidate_hash = ¶ms.candidate_hash; let already_requesting_count = self.requesting_chunks.len(); - let mut requests = Vec::with_capacity(desired_requests_count - already_requesting_count); + let to_launch = desired_requests_count - already_requesting_count; + let mut requests = Vec::with_capacity(to_launch); + + gum::trace!( + target: LOG_TARGET, + ?candidate_hash, + "Attempting to launch {} requests", + to_launch + ); while self.requesting_chunks.len() < desired_requests_count { if let Some((chunk_index, validator_index)) = validators.pop_back() { @@ -212,12 +220,14 @@ impl State { self.requesting_chunks.push(Box::pin(async move { let _timer = timer; - match res.await { + let res = match res.await { Ok(req_res::v1::ChunkFetchingResponse::Chunk(chunk)) => Ok(Some(chunk.recombine_into_chunk(&raw_request))), Ok(req_res::v1::ChunkFetchingResponse::NoSuchChunk) => Ok(None), - Err(e) => Err((chunk_index, validator_index, e)), - } + Err(e) => Err(e), + }; + + (chunk_index, validator_index, res) })); } else { break @@ -247,11 +257,13 @@ impl State { // Wait for all current requests to conclude or time-out, or until we reach enough chunks. // We also declare requests undead, once `TIMEOUT_START_NEW_REQUESTS` is reached and will // return in that case for `launch_parallel_requests` to fill up slots again. - while let Some(request_result) = + while let Some(res) = self.requesting_chunks.next_with_timeout(TIMEOUT_START_NEW_REQUESTS).await { total_received_responses += 1; + let (chunk_index, validator_index, request_result) = res; + match request_result { Ok(Some(chunk)) => if is_chunk_valid(params, &chunk) { @@ -259,7 +271,8 @@ impl State { gum::trace!( target: LOG_TARGET, candidate_hash = ?params.candidate_hash, - chunk_index = ?chunk.index, + chunk_index = ?chunk_index, + ?validator_index, "Received valid chunk", ); self.insert_chunk(chunk.index, chunk); @@ -269,28 +282,36 @@ impl State { }, Ok(None) => { metrics.on_chunk_request_no_such_chunk(); + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?params.candidate_hash, + chunk_index = ?chunk_index, + ?validator_index, + "Validator did not have the requested chunk", + ); error_count += 1; }, - Err((chunk_index, validator_index, e)) => { + Err(err) => { error_count += 1; gum::trace!( target: LOG_TARGET, candidate_hash= ?params.candidate_hash, - err = ?e, + ?err, + chunk_index = ?chunk_index, ?validator_index, - ?chunk_index, "Failure requesting chunk", ); - match e { + match err { RequestError::InvalidResponse(_) => { metrics.on_chunk_request_invalid(); gum::debug!( target: LOG_TARGET, candidate_hash = ?params.candidate_hash, - err = ?e, + ?err, + chunk_index = ?chunk_index, ?validator_index, "Chunk fetching response was invalid", ); @@ -328,7 +349,7 @@ impl State { received_chunks_count = ?self.chunk_count(), requested_chunks_count = ?self.requesting_chunks.len(), threshold = ?params.threshold, - "Can conclude availability for a candidate", + "Can conclude availability recovery for a candidate", ); break } @@ -558,8 +579,10 @@ impl RecoveryStrategy /// `RecoveryStrategy` that attempts to recover the systematic chunks from the validators that /// hold them, in order to bypass the erasure code reconstruction step, which is costly. pub struct FetchSystematicChunks { + /// Systematic recovery threshold. threshold: usize, - validators: BTreeMap, + /// Validators that hold the systematic chunks. + validators: VecDeque<(ChunkIndex, ValidatorIndex)>, /// Channel to the erasure task handler. erasure_task_tx: futures::channel::mpsc::Sender, } @@ -567,7 +590,7 @@ pub struct FetchSystematicChunks { /// Parameters needed for fetching systematic chunks. pub struct FetchSystematicChunksParams { /// Validators that hold the systematic chunks. - pub validators: BTreeMap, + pub validators: VecDeque<(ChunkIndex, ValidatorIndex)>, /// Channel to the erasure task handler. pub erasure_task_tx: futures::channel::mpsc::Sender, } @@ -656,7 +679,7 @@ impl FetchSystematicChunks { target: LOG_TARGET, candidate_hash = ?common_params.candidate_hash, erasure_root = ?common_params.erasure_root, - "Data recovery from systematic chunks complete", + "Recovery from systematic chunks complete", ); Ok(data) @@ -711,11 +734,11 @@ impl RecoveryStrategy if !common_params.bypass_availability_store { let local_chunk_indices = state.populate_from_av_store(common_params, sender).await; - for c_index in &local_chunk_indices { + for our_c_index in &local_chunk_indices { // If we are among the systematic validators but hold an invalid chunk, we cannot // perform the systematic recovery. Fall through to the next strategy. - if self.validators.contains_key(c_index) && - !state.received_chunks.contains_key(c_index) + if self.validators.iter().find(|(c_index, _)| c_index == our_c_index).is_some() && + !state.received_chunks.contains_key(our_c_index) { gum::debug!( target: LOG_TARGET, @@ -724,6 +747,7 @@ impl RecoveryStrategy requesting = %state.requesting_chunks.len(), total_requesting = %state.requesting_chunks.total_len(), n_validators = %common_params.n_validators, + chunk_index = ?our_c_index, "Systematic chunk recovery is not possible. We are among the systematic validators but hold an invalid chunk", ); return Err(RecoveryError::Unavailable) @@ -731,25 +755,23 @@ impl RecoveryStrategy } } - let mut systematic_chunk_count = self - .validators - .iter() - .filter(|(c_index, _)| state.received_chunks.contains_key(c_index)) - .count(); + // Instead of counting the chunks we already have, perform the difference after we remove + // them from the queue. + let mut systematic_chunk_count = self.validators.len(); // No need to query the validators that have the chunks we already received. self.validators - .retain(|c_index, _| !state.received_chunks.contains_key(c_index)); + .retain(|(c_index, _)| !state.received_chunks.contains_key(c_index)); + + systematic_chunk_count -= self.validators.len(); // Safe to `take` here, as we're consuming `self` anyway and we're not using the // `validators` field in other methods. - let mut validators_queue: VecDeque<_> = - std::mem::take(&mut self.validators).into_iter().collect(); + let mut validators_queue: VecDeque<_> = std::mem::take(&mut self.validators); loop { // If received_chunks has `systematic_chunk_threshold` entries, attempt to recover the - // data. If that fails, or a re-encoding of it doesn't match the expected erasure root, - // return Err(RecoveryError::Invalid) + // data. if systematic_chunk_count >= self.threshold { return self.attempt_systematic_recovery(state, common_params).await } @@ -764,11 +786,11 @@ impl RecoveryStrategy target: LOG_TARGET, candidate_hash = ?common_params.candidate_hash, erasure_root = ?common_params.erasure_root, - received = %systematic_chunk_count, + %systematic_chunk_count, requesting = %state.requesting_chunks.len(), total_requesting = %state.requesting_chunks.total_len(), n_validators = %common_params.n_validators, - threshold = ?self.threshold, + systematic_threshold = ?self.threshold, "Data recovery is not possible", ); @@ -783,7 +805,7 @@ impl RecoveryStrategy ?common_params.candidate_hash, ?desired_requests_count, total_received = ?systematic_chunk_count, - threshold = ?self.threshold, + systematic_threshold = ?self.threshold, ?already_requesting_count, "Requesting systematic availability chunks for a candidate", ); @@ -803,23 +825,25 @@ impl RecoveryStrategy &mut validators_queue, |unrequested_validators, in_flight_reqs, + // Don't use this chunk count, as it may contain non-systematic chunks. _chunk_count, error_count, success_responses| { + let chunk_count = systematic_chunk_count + success_responses; let is_unavailable = Self::is_unavailable( unrequested_validators, in_flight_reqs, - systematic_chunk_count + success_responses, + chunk_count, self.threshold, ); - error_count > 0 || - (systematic_chunk_count + success_responses) >= self.threshold || - is_unavailable + error_count > 0 || chunk_count >= self.threshold || is_unavailable }, ) .await; + systematic_chunk_count += total_responses - error_count; + // We can't afford any errors, as we need all the systematic chunks for this to work. if error_count > 0 { gum::debug!( @@ -830,14 +854,12 @@ impl RecoveryStrategy requesting = %state.requesting_chunks.len(), total_requesting = %state.requesting_chunks.total_len(), n_validators = %common_params.n_validators, - threshold = ?self.threshold, - "Systematic chunk recovery is not possible. ", + systematic_threshold = ?self.threshold, + "Systematic chunk recovery is not possible. Got an error while requesting a chunk", ); return Err(RecoveryError::Unavailable) } - - systematic_chunk_count += total_responses; } } } @@ -848,17 +870,15 @@ pub struct FetchChunks { error_count: usize, /// Total number of responses that have been received, including failed ones. total_received_responses: usize, - - /// A random shuffling of the validators which indicates the order in which we connect to the - /// validators and request the chunk from them. + /// The collection of chunk indices and the respective validators holding the chunks. validators: VecDeque<(ChunkIndex, ValidatorIndex)>, - /// Channel to the erasure task handler. erasure_task_tx: futures::channel::mpsc::Sender, } /// Parameters specific to the `FetchChunks` strategy. pub struct FetchChunksParams { + /// The collection of chunk indices and the respective validators holding the chunks. pub validators: VecDeque<(ChunkIndex, ValidatorIndex)>, /// Channel to the erasure task handler. pub erasure_task_tx: futures::channel::mpsc::Sender, @@ -867,7 +887,10 @@ pub struct FetchChunksParams { impl FetchChunks { /// Instantiate a new strategy. pub fn new(mut params: FetchChunksParams) -> Self { + // Shuffle the validators to make sure that we don't request chunks from the same + // validators over and over. params.validators.make_contiguous().shuffle(&mut rand::thread_rng()); + Self { error_count: 0, total_received_responses: 0, From bf39ba0cddec7c1438bf83106cd0f2833852ca99 Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 15 Sep 2023 12:24:15 +0300 Subject: [PATCH 013/126] don't use the backing group if chunk size query failed --- polkadot/node/network/availability-recovery/src/lib.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index 04b71147c9f0..4a6a805ae451 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -445,6 +445,10 @@ async fn handle_recover( enabled = prefer_backing_group, "Prefer fetch from backing group", ); + } else { + // we have a POV limit but were not able to query the chunk size, so don't use + // the backing group. + prefer_backing_group = false; } }; From 34408b54abbdee49e187baee399bc85393c5522a Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 15 Sep 2023 12:29:47 +0300 Subject: [PATCH 014/126] add ImmediateError to chunks recovery strategy --- polkadot/node/network/availability-recovery/src/task.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 30eaf7734a52..8acd4f2040cb 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -222,7 +222,7 @@ impl State { sender .send_message(NetworkBridgeTxMessage::SendRequests( requests, - IfDisconnected::TryConnect, + IfDisconnected::ImmediateError, )) .await; } From b68e6715af6acf25a683a6406d3b9e71076e2cbf Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 15 Sep 2023 13:34:41 +0300 Subject: [PATCH 015/126] modify and add metrics --- .../availability-recovery/src/metrics.rs | 139 +++++++++++++----- .../network/availability-recovery/src/task.rs | 61 ++++++-- 2 files changed, 148 insertions(+), 52 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/metrics.rs b/polkadot/node/network/availability-recovery/src/metrics.rs index aa7216739507..10683c49ef38 100644 --- a/polkadot/node/network/availability-recovery/src/metrics.rs +++ b/polkadot/node/network/availability-recovery/src/metrics.rs @@ -14,11 +14,21 @@ // You should have received a copy of the GNU General Public License // along with Polkadot. If not, see . +use polkadot_node_subsystem::prometheus::HistogramVec; use polkadot_node_subsystem_util::metrics::{ self, - prometheus::{self, Counter, CounterVec, Histogram, Opts, PrometheusError, Registry, U64}, + prometheus::{ + self, prometheus::HistogramTimer, Counter, CounterVec, Histogram, Opts, PrometheusError, + Registry, U64, + }, }; +/// Label value used for regular chunk fetching. +pub const REGULAR_CHUNK_LABEL: &str = "regular"; + +/// Label value used for systematic chunk fetching. +pub const SYSTEMATIC_CHUNK_LABEL: &str = "systematic"; + /// Availability Distribution metrics. #[derive(Clone, Default)] pub struct Metrics(Option); @@ -28,11 +38,17 @@ struct MetricsInner { /// Number of sent chunk requests. /// /// Gets incremented on each sent chunk requests. - chunk_requests_issued: Counter, + /// + /// Split by chunk type: + /// - `regular` + /// - `systematic` + chunk_requests_issued: CounterVec, /// A counter for finished chunk requests. /// - /// Split by result: + /// Split by the chunk type (`regular` or `systematic`) + /// + /// Also split by result: /// - `no_such_chunk` ... peer did not have the requested chunk /// - `timeout` ... request timed out. /// - `network_error` ... Some networking issue except timeout @@ -41,10 +57,19 @@ struct MetricsInner { chunk_requests_finished: CounterVec, /// The duration of request to response. - time_chunk_request: Histogram, + /// + /// Split by chunk type (`regular` or `systematic`). + time_chunk_request: HistogramVec, /// The duration between the pure recovery and verification. - time_erasure_recovery: Histogram, + /// + /// Split by chunk type (`regular` or `systematic`). + time_erasure_recovery: HistogramVec, + + /// How much time it takes to reconstruct the available data from chunks. + /// + /// Split by chunk type (`regular` or `systematic`), ass the algorithms are different. + time_erasure_reconstruct: HistogramVec, /// How much time it takes to re-encode the data into erasure chunks in order to verify /// the root hash of the provided Merkle tree. See `reconstructed_data_matches_root`. @@ -55,6 +80,9 @@ struct MetricsInner { time_full_recovery: Histogram, /// Number of full recoveries that have been finished one way or the other. + /// + /// Split by recovery `strategy_type` (`full_from_backers, systematic_chunks, regular_chunks`) + /// Also split by `result` type. full_recoveries_finished: CounterVec, /// Number of full recoveries that have been started on this subsystem. @@ -70,72 +98,98 @@ impl Metrics { Metrics(None) } - /// Increment counter on fetched labels. - pub fn on_chunk_request_issued(&self) { + /// Increment counter for chunk requests. + pub fn on_chunk_request_issued(&self, chunk_type: &str) { if let Some(metrics) = &self.0 { - metrics.chunk_requests_issued.inc() + metrics.chunk_requests_issued.with_label_values(&[chunk_type]).inc() } } /// A chunk request timed out. - pub fn on_chunk_request_timeout(&self) { + pub fn on_chunk_request_timeout(&self, chunk_type: &str) { if let Some(metrics) = &self.0 { - metrics.chunk_requests_finished.with_label_values(&["timeout"]).inc() + metrics + .chunk_requests_finished + .with_label_values(&[chunk_type, "timeout"]) + .inc() } } /// A chunk request failed because validator did not have its chunk. - pub fn on_chunk_request_no_such_chunk(&self) { + pub fn on_chunk_request_no_such_chunk(&self, chunk_type: &str) { if let Some(metrics) = &self.0 { - metrics.chunk_requests_finished.with_label_values(&["no_such_chunk"]).inc() + metrics + .chunk_requests_finished + .with_label_values(&[chunk_type, "no_such_chunk"]) + .inc() } } /// A chunk request failed for some non timeout related network error. - pub fn on_chunk_request_error(&self) { + pub fn on_chunk_request_error(&self, chunk_type: &str) { if let Some(metrics) = &self.0 { - metrics.chunk_requests_finished.with_label_values(&["error"]).inc() + metrics.chunk_requests_finished.with_label_values(&[chunk_type, "error"]).inc() } } /// A chunk request succeeded, but was not valid. - pub fn on_chunk_request_invalid(&self) { + pub fn on_chunk_request_invalid(&self, chunk_type: &str) { if let Some(metrics) = &self.0 { - metrics.chunk_requests_finished.with_label_values(&["invalid"]).inc() + metrics + .chunk_requests_finished + .with_label_values(&[chunk_type, "invalid"]) + .inc() } } /// A chunk request succeeded. - pub fn on_chunk_request_succeeded(&self) { + pub fn on_chunk_request_succeeded(&self, chunk_type: &str) { if let Some(metrics) = &self.0 { - metrics.chunk_requests_finished.with_label_values(&["success"]).inc() + metrics + .chunk_requests_finished + .with_label_values(&[chunk_type, "success"]) + .inc() } } /// Get a timer to time request/response duration. - pub fn time_chunk_request(&self) -> Option { - self.0.as_ref().map(|metrics| metrics.time_chunk_request.start_timer()) + pub fn time_chunk_request(&self, chunk_type: &str) -> Option { + self.0.as_ref().map(|metrics| { + metrics.time_chunk_request.with_label_values(&[chunk_type]).start_timer() + }) } /// Get a timer to time erasure code recover. - pub fn time_erasure_recovery(&self) -> Option { - self.0.as_ref().map(|metrics| metrics.time_erasure_recovery.start_timer()) + pub fn time_erasure_recovery(&self, chunk_type: &str) -> Option { + self.0.as_ref().map(|metrics| { + metrics.time_erasure_recovery.with_label_values(&[chunk_type]).start_timer() + }) + } + + /// Get a timer for available data reconstruction. + pub fn time_erasure_reconstruct(&self, chunk_type: &str) -> Option { + self.0.as_ref().map(|metrics| { + metrics.time_erasure_reconstruct.with_label_values(&[chunk_type]).start_timer() + }) } /// Get a timer to time chunk encoding. - pub fn time_reencode_chunks(&self) -> Option { + pub fn time_reencode_chunks(&self) -> Option { self.0.as_ref().map(|metrics| metrics.time_reencode_chunks.start_timer()) } /// Get a timer to measure the time of the complete recovery process. - pub fn time_full_recovery(&self) -> Option { + pub fn time_full_recovery(&self) -> Option { self.0.as_ref().map(|metrics| metrics.time_full_recovery.start_timer()) } /// A full recovery succeeded. - pub fn on_recovery_succeeded(&self) { + pub fn on_recovery_succeeded(&self, strategy_type: &str) { if let Some(metrics) = &self.0 { - metrics.full_recoveries_finished.with_label_values(&["success"]).inc() + metrics + .full_recoveries_finished + .with_label_values(&["success", strategy_type]) + .inc() } } @@ -147,9 +201,12 @@ impl Metrics { } /// A full recovery failed (data was recovered, but invalid). - pub fn on_recovery_invalid(&self) { + pub fn on_recovery_invalid(&self, strategy_type: &str) { if let Some(metrics) = &self.0 { - metrics.full_recoveries_finished.with_label_values(&["invalid"]).inc() + metrics + .full_recoveries_finished + .with_label_values(&["invalid", strategy_type]) + .inc() } } @@ -165,9 +222,10 @@ impl metrics::Metrics for Metrics { fn try_register(registry: &Registry) -> Result { let metrics = MetricsInner { chunk_requests_issued: prometheus::register( - Counter::new( - "polkadot_parachain_availability_recovery_chunk_requests_issued", - "Total number of issued chunk requests.", + CounterVec::new( + Opts::new("polkadot_parachain_availability_recovery_chunk_requests_issued", + "Total number of issued chunk requests."), + &["type"] )?, registry, )?, @@ -177,22 +235,29 @@ impl metrics::Metrics for Metrics { "polkadot_parachain_availability_recovery_chunk_requests_finished", "Total number of chunk requests finished.", ), - &["result"], + &["result", "type"], )?, registry, )?, time_chunk_request: prometheus::register( - prometheus::Histogram::with_opts(prometheus::HistogramOpts::new( + prometheus::HistogramVec::new(prometheus::HistogramOpts::new( "polkadot_parachain_availability_recovery_time_chunk_request", "Time spent waiting for a response to a chunk request", - ))?, + ), &["type"])?, registry, )?, time_erasure_recovery: prometheus::register( - prometheus::Histogram::with_opts(prometheus::HistogramOpts::new( + prometheus::HistogramVec::new(prometheus::HistogramOpts::new( "polkadot_parachain_availability_recovery_time_erasure_recovery", "Time spent to recover the erasure code and verify the merkle root by re-encoding as erasure chunks", - ))?, + ), &["type"])?, + registry, + )?, + time_erasure_reconstruct: prometheus::register( + prometheus::HistogramVec::new(prometheus::HistogramOpts::new( + "polkadot_parachain_availability_recovery_time_erasure_reconstruct", + "Time spent to reconstruct the data from chunks", + ), &["type"])?, registry, )?, time_reencode_chunks: prometheus::register( @@ -215,7 +280,7 @@ impl metrics::Metrics for Metrics { "polkadot_parachain_availability_recovery_recoveries_finished", "Total number of recoveries that finished.", ), - &["result"], + &["result", "strategy_type"], )?, registry, )?, diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index b037a1234026..764bc09c7095 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -19,8 +19,10 @@ #![warn(missing_docs)] use crate::{ - futures_undead::FuturesUndead, is_chunk_valid, is_unavailable, metrics::Metrics, ErasureTask, - LOG_TARGET, + futures_undead::FuturesUndead, + is_chunk_valid, is_unavailable, + metrics::{Metrics, REGULAR_CHUNK_LABEL, SYSTEMATIC_CHUNK_LABEL}, + ErasureTask, LOG_TARGET, }; use futures::{channel::oneshot, SinkExt}; #[cfg(not(test))] @@ -71,6 +73,9 @@ pub trait RecoveryStrategy: S /// Return the name of the strategy for logging purposes. fn display_name(&self) -> &'static str; + + /// Return the strategy type for use as a metric label. + fn strategy_type(&self) -> &'static str; } /// Recovery parameters common to all strategies in a `RecoveryTask`. @@ -174,6 +179,7 @@ impl State { /// Launch chunk requests in parallel, according to the parameters. async fn launch_parallel_chunk_requests( &mut self, + chunk_type: &str, params: &RecoveryParams, sender: &mut Sender, desired_requests_count: usize, @@ -215,8 +221,8 @@ impl State { let (req, res) = OutgoingRequest::new(Recipient::Authority(validator), raw_request); requests.push(Requests::ChunkFetchingV1(req)); - params.metrics.on_chunk_request_issued(); - let timer = params.metrics.time_chunk_request(); + params.metrics.on_chunk_request_issued(chunk_type); + let timer = params.metrics.time_chunk_request(chunk_type); self.requesting_chunks.push(Box::pin(async move { let _timer = timer; @@ -245,6 +251,7 @@ impl State { /// Wait for a sufficient amount of chunks to reconstruct according to the provided `params`. async fn wait_for_chunks( &mut self, + chunk_type: &str, params: &RecoveryParams, validators: &mut VecDeque<(ChunkIndex, ValidatorIndex)>, can_conclude: impl Fn(usize, usize, usize, usize, usize) -> bool, @@ -267,7 +274,7 @@ impl State { match request_result { Ok(Some(chunk)) => if is_chunk_valid(params, &chunk) { - metrics.on_chunk_request_succeeded(); + metrics.on_chunk_request_succeeded(chunk_type); gum::trace!( target: LOG_TARGET, candidate_hash = ?params.candidate_hash, @@ -277,11 +284,11 @@ impl State { ); self.insert_chunk(chunk.index, chunk); } else { - metrics.on_chunk_request_invalid(); + metrics.on_chunk_request_invalid(chunk_type); error_count += 1; }, Ok(None) => { - metrics.on_chunk_request_no_such_chunk(); + metrics.on_chunk_request_no_such_chunk(chunk_type); gum::trace!( target: LOG_TARGET, candidate_hash = ?params.candidate_hash, @@ -305,7 +312,7 @@ impl State { match err { RequestError::InvalidResponse(_) => { - metrics.on_chunk_request_invalid(); + metrics.on_chunk_request_invalid(chunk_type); gum::debug!( target: LOG_TARGET, @@ -320,15 +327,15 @@ impl State { // No debug logs on general network errors - that became very spammy // occasionally. if let RequestFailure::Network(OutboundFailure::Timeout) = err { - metrics.on_chunk_request_timeout(); + metrics.on_chunk_request_timeout(chunk_type); } else { - metrics.on_chunk_request_error(); + metrics.on_chunk_request_error(chunk_type); } validators.push_front((chunk_index, validator_index)); }, RequestError::Canceled(_) => { - metrics.on_chunk_request_error(); + metrics.on_chunk_request_error(chunk_type); validators.push_front((chunk_index, validator_index)); }, @@ -425,6 +432,7 @@ where } let display_name = current_strategy.display_name(); + let strategy_type = current_strategy.strategy_type(); gum::debug!( target: LOG_TARGET, @@ -448,13 +456,14 @@ where }, Err(err) => { match &err { - RecoveryError::Invalid => self.params.metrics.on_recovery_invalid(), + RecoveryError::Invalid => + self.params.metrics.on_recovery_invalid(strategy_type), _ => self.params.metrics.on_recovery_failed(), } return Err(err) }, Ok(data) => { - self.params.metrics.on_recovery_succeeded(); + self.params.metrics.on_recovery_succeeded(strategy_type); return Ok(data) }, } @@ -499,6 +508,10 @@ impl RecoveryStrategy "Full recovery from backers" } + fn strategy_type(&self) -> &'static str { + "full_from_backers" + } + async fn run( mut self: Box, _: &mut State, @@ -639,7 +652,9 @@ impl FetchSystematicChunks { state: &mut State, common_params: &RecoveryParams, ) -> Result { - let recovery_duration = common_params.metrics.time_erasure_recovery(); + let recovery_duration = common_params.metrics.time_erasure_recovery(SYSTEMATIC_CHUNK_LABEL); + let reconstruct_duration = + common_params.metrics.time_erasure_reconstruct(SYSTEMATIC_CHUNK_LABEL); let chunks = state .received_chunks .range( @@ -659,6 +674,8 @@ impl FetchSystematicChunks { match available_data { Ok(data) => { + drop(reconstruct_duration); + // Send request to re-encode the chunks and check merkle root. let (reencode_tx, reencode_rx) = oneshot::channel(); self.erasure_task_tx @@ -699,7 +716,9 @@ impl FetchSystematicChunks { } }, Err(err) => { + reconstruct_duration.map(|rd| rd.stop_and_discard()); recovery_duration.map(|rd| rd.stop_and_discard()); + gum::trace!( target: LOG_TARGET, candidate_hash = ?common_params.candidate_hash, @@ -724,6 +743,10 @@ impl RecoveryStrategy "Fetch systematic chunks" } + fn strategy_type(&self) -> &'static str { + "systematic_chunks" + } + async fn run( mut self: Box, state: &mut State, @@ -812,6 +835,7 @@ impl RecoveryStrategy state .launch_parallel_chunk_requests( + SYSTEMATIC_CHUNK_LABEL, common_params, sender, desired_requests_count, @@ -821,6 +845,7 @@ impl RecoveryStrategy let (total_responses, error_count) = state .wait_for_chunks( + SYSTEMATIC_CHUNK_LABEL, common_params, &mut validators_queue, |unrequested_validators, @@ -938,7 +963,7 @@ impl FetchChunks { state: &mut State, common_params: &RecoveryParams, ) -> Result { - let recovery_duration = common_params.metrics.time_erasure_recovery(); + let recovery_duration = common_params.metrics.time_erasure_recovery(REGULAR_CHUNK_LABEL); // Send request to reconstruct available data from chunks. let (avilable_data_tx, available_data_rx) = oneshot::channel(); @@ -1016,6 +1041,10 @@ impl RecoveryStrategy "Fetch chunks" } + fn strategy_type(&self) -> &'static str { + "regular_chunks" + } + async fn run( mut self: Box, state: &mut State, @@ -1082,6 +1111,7 @@ impl RecoveryStrategy state .launch_parallel_chunk_requests( + REGULAR_CHUNK_LABEL, common_params, sender, desired_requests_count, @@ -1091,6 +1121,7 @@ impl RecoveryStrategy let (total_responses, error_count) = state .wait_for_chunks( + REGULAR_CHUNK_LABEL, common_params, &mut validators_queue, |unrequested_validators, From 1569e49e3dfb547df3cf195c1e209715522b2011 Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 18 Sep 2023 10:41:45 +0300 Subject: [PATCH 016/126] more review comments --- .../network/availability-recovery/src/lib.rs | 97 +++++++++---------- 1 file changed, 46 insertions(+), 51 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index 4a6a805ae451..98087bca6f0b 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -424,63 +424,58 @@ async fn handle_recover( let _span = span.child("session-info-ctx-received"); match session_info { Some(session_info) => { - let mut prefer_backing_group = true; - - if let RecoveryStrategyKind::BackersFirstIfSizeLower(small_pov_limit) = - recovery_strategy_kind - { - // Get our own chunk size to get an estimate of the PoV size. - let chunk_size: Result, error::Error> = - query_chunk_size(ctx, candidate_hash).await; - if let Ok(Some(chunk_size)) = chunk_size { - let pov_size_estimate = - chunk_size.saturating_mul(session_info.validators.len()) / 3; - prefer_backing_group = pov_size_estimate < small_pov_limit; - - gum::trace!( - target: LOG_TARGET, - ?candidate_hash, - pov_size_estimate, - small_pov_limit, - enabled = prefer_backing_group, - "Prefer fetch from backing group", - ); - } else { - // we have a POV limit but were not able to query the chunk size, so don't use - // the backing group. - prefer_backing_group = false; - } - }; - - let backing_validators = if let Some(backing_group) = backing_group { - session_info.validator_groups.get(backing_group) - } else { - None - }; - - let fetch_chunks_params = FetchChunksParams { - n_validators: session_info.validators.len(), - erasure_task_tx: erasure_task_tx.clone(), - }; - let mut recovery_strategies: VecDeque< Box::Sender>>, > = VecDeque::with_capacity(2); - if let Some(backing_validators) = backing_validators { - match (&recovery_strategy_kind, prefer_backing_group) { - (RecoveryStrategyKind::BackersFirstAlways, true) | - (RecoveryStrategyKind::BackersFirstIfSizeLower(_), true) | - (RecoveryStrategyKind::BypassAvailabilityStore, true) => - recovery_strategies.push_back(Box::new(FetchFull::new(FetchFullParams { - validators: backing_validators.to_vec(), - erasure_task_tx, - }))), - _ => {}, - }; + if let Some(backing_group) = backing_group { + if let Some(backing_validators) = session_info.validator_groups.get(backing_group) { + let mut small_pov_size = true; + + if let RecoveryStrategyKind::BackersFirstIfSizeLower(small_pov_limit) = + recovery_strategy_kind + { + // Get our own chunk size to get an estimate of the PoV size. + let chunk_size: Result, error::Error> = + query_chunk_size(ctx, candidate_hash).await; + if let Ok(Some(chunk_size)) = chunk_size { + let pov_size_estimate = + chunk_size.saturating_mul(session_info.validators.len()) / 3; + small_pov_size = pov_size_estimate < small_pov_limit; + + gum::trace!( + target: LOG_TARGET, + ?candidate_hash, + pov_size_estimate, + small_pov_limit, + enabled = small_pov_size, + "Prefer fetch from backing group", + ); + } else { + // we have a POV limit but were not able to query the chunk size, so + // don't use the backing group. + small_pov_size = false; + } + }; + + match (&recovery_strategy_kind, small_pov_size) { + (RecoveryStrategyKind::BackersFirstAlways, _) | + (RecoveryStrategyKind::BypassAvailabilityStore, _) | + (RecoveryStrategyKind::BackersFirstIfSizeLower(_), true) => recovery_strategies.push_back( + Box::new(FetchFull::new(FetchFullParams { + validators: backing_validators.to_vec(), + erasure_task_tx: erasure_task_tx.clone(), + })), + ), + _ => {}, + }; + } } - recovery_strategies.push_back(Box::new(FetchChunks::new(fetch_chunks_params))); + recovery_strategies.push_back(Box::new(FetchChunks::new(FetchChunksParams { + n_validators: session_info.validators.len(), + erasure_task_tx, + }))); launch_recovery_task( state, From d5c32d13c58e52cfbd7c0879670281b79e84c379 Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 18 Sep 2023 11:55:46 +0300 Subject: [PATCH 017/126] fix test --- polkadot/node/network/availability-recovery/src/tests.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/polkadot/node/network/availability-recovery/src/tests.rs b/polkadot/node/network/availability-recovery/src/tests.rs index 17b02080c008..63ccf0e94f91 100644 --- a/polkadot/node/network/availability-recovery/src/tests.rs +++ b/polkadot/node/network/availability-recovery/src/tests.rs @@ -1014,7 +1014,7 @@ fn recovers_from_only_chunks_if_pov_large() { AvailabilityRecoveryMessage::RecoverAvailableData( new_candidate.clone(), test_state.session_index, - None, + Some(GroupIndex(0)), tx, ), ) From cb42bef4ae047bbe3bf1681874daebf72fc7acb9 Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 18 Sep 2023 12:12:07 +0300 Subject: [PATCH 018/126] rollback to using TryConnect for fetch chunks --- polkadot/node/network/availability-recovery/src/task.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 8acd4f2040cb..30eaf7734a52 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -222,7 +222,7 @@ impl State { sender .send_message(NetworkBridgeTxMessage::SendRequests( requests, - IfDisconnected::ImmediateError, + IfDisconnected::TryConnect, )) .await; } From 06f00a2118578eec508f715729feeea69dfa921a Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 18 Sep 2023 13:30:52 +0300 Subject: [PATCH 019/126] add runtime API knob for shuffling enablement --- .../node/core/bitfield-signing/src/lib.rs | 32 ++++++----- polkadot/node/core/runtime-api/src/cache.rs | 18 +++++++ polkadot/node/core/runtime-api/src/lib.rs | 12 +++++ .../src/requester/mod.rs | 32 +++++++---- .../src/requester/session_cache.rs | 3 +- .../availability-recovery/src/error.rs | 3 ++ .../network/availability-recovery/src/lib.rs | 53 ++++++++++++------- polkadot/node/subsystem-types/src/messages.rs | 22 +++++--- .../subsystem-types/src/runtime_client.rs | 26 ++++++--- polkadot/node/subsystem-util/src/lib.rs | 41 +++++++++----- .../node/subsystem-util/src/runtime/error.rs | 2 +- .../node/subsystem-util/src/runtime/mod.rs | 30 ++++++++++- polkadot/node/subsystem-util/src/tests.rs | 4 +- polkadot/primitives/src/runtime_api.rs | 5 ++ polkadot/primitives/src/vstaging/mod.rs | 15 ++++++ .../src/runtime_api_impl/vstaging.rs | 13 ++++- 16 files changed, 235 insertions(+), 76 deletions(-) diff --git a/polkadot/node/core/bitfield-signing/src/lib.rs b/polkadot/node/core/bitfield-signing/src/lib.rs index 7396ec36c3b5..9ff7184c30ed 100644 --- a/polkadot/node/core/bitfield-signing/src/lib.rs +++ b/polkadot/node/core/bitfield-signing/src/lib.rs @@ -27,7 +27,6 @@ use futures::{ FutureExt, }; use polkadot_node_subsystem::{ - errors::RuntimeApiError, jaeger, messages::{ AvailabilityStoreMessage, BitfieldDistributionMessage, ChainApiMessage, RuntimeApiMessage, @@ -37,7 +36,8 @@ use polkadot_node_subsystem::{ SpawnedSubsystem, SubsystemError, SubsystemResult, SubsystemSender, }; use polkadot_node_subsystem_util::{ - self as util, request_validators, shuffle_availability_chunks, Validator, + self as util, availability_chunk_indices, request_validators, + runtime::request_availability_chunk_shuffling_params, Validator, }; use polkadot_primitives::{AvailabilityBitfield, BlockNumber, CoreState, Hash, ValidatorIndex}; use sp_keystore::{Error as KeystoreError, KeystorePtr}; @@ -72,7 +72,7 @@ pub enum Error { MpscSend(#[from] mpsc::SendError), #[error(transparent)] - Runtime(#[from] RuntimeApiError), + Runtime(#[from] util::runtime::Error), #[error("Keystore failed: {0:?}")] Keystore(KeystoreError), @@ -90,6 +90,7 @@ pub enum Error { /// If there is a candidate pending availability, query the Availability Store /// for whether we have the availability chunk for our validator index. async fn get_core_availability( + relay_parent: Hash, core: &CoreState, n_validators: usize, validator_idx: ValidatorIndex, @@ -101,8 +102,13 @@ async fn get_core_availability( let block_number = get_block_number(*sender.lock().await, core.candidate_descriptor.relay_parent).await?; - let shuffled_chunk_indices = shuffle_availability_chunks(block_number, n_validators); - let chunk_index = shuffled_chunk_indices[usize::try_from(validator_idx.0) + let maybe_av_chunk_shuffling_params = + request_availability_chunk_shuffling_params(relay_parent, *sender.lock().await) + .await + .map_err(Error::from)?; + let chunk_indices = + availability_chunk_indices(maybe_av_chunk_shuffling_params, block_number, n_validators); + let chunk_index = chunk_indices[usize::try_from(validator_idx.0) .expect("usize is at least u32 bytes on all modern targets.")]; let (tx, rx) = oneshot::channel(); @@ -146,7 +152,7 @@ async fn get_availability_cores( .await; match rx.await { Ok(Ok(out)) => Ok(out), - Ok(Err(runtime_err)) => Err(runtime_err.into()), + Ok(Err(runtime_err)) => Err(Error::Runtime(runtime_err.into())), Err(err) => Err(err.into()), } } @@ -177,11 +183,9 @@ async fn construct_availability_bitfield( // Handle all cores concurrently // `try_join_all` returns all results in the same order as the input futures. - let results = future::try_join_all( - availability_cores - .iter() - .map(|core| get_core_availability(core, n_validators, validator_idx, &sender, span)), - ) + let results = future::try_join_all(availability_cores.iter().map(|core| { + get_core_availability(relay_parent, core, n_validators, validator_idx, &sender, span) + })) .await?; let core_bits = FromIterator::from_iter(results.into_iter()); @@ -288,7 +292,11 @@ where let span_delay = span.child("delay"); let wait_until = Instant::now() + SPAWNED_TASK_DELAY; - let validators = request_validators(leaf.hash, &mut sender).await.await??; + let validators = request_validators(leaf.hash, &mut sender) + .await + .await + .map_err(|e| Error::Runtime(e.into()))? + .map_err(|e| Error::Runtime(e.into()))?; // now do all the work we can before we need to wait for the availability store // if we're not a validator, we can just succeed effortlessly diff --git a/polkadot/node/core/runtime-api/src/cache.rs b/polkadot/node/core/runtime-api/src/cache.rs index 7f41d74e616c..1a87d8f52b52 100644 --- a/polkadot/node/core/runtime-api/src/cache.rs +++ b/polkadot/node/core/runtime-api/src/cache.rs @@ -66,6 +66,7 @@ pub(crate) struct RequestResultCache { key_ownership_proof: LruMap<(Hash, ValidatorId), Option>, minimum_backing_votes: LruMap, + availability_chunk_shuffling_params: LruMap, staging_para_backing_state: LruMap<(Hash, ParaId), Option>, staging_async_backing_params: LruMap, @@ -99,6 +100,7 @@ impl Default for RequestResultCache { unapplied_slashes: LruMap::new(ByLength::new(DEFAULT_CACHE_CAP)), key_ownership_proof: LruMap::new(ByLength::new(DEFAULT_CACHE_CAP)), minimum_backing_votes: LruMap::new(ByLength::new(DEFAULT_CACHE_CAP)), + availability_chunk_shuffling_params: LruMap::new(ByLength::new(DEFAULT_CACHE_CAP)), staging_para_backing_state: LruMap::new(ByLength::new(DEFAULT_CACHE_CAP)), staging_async_backing_params: LruMap::new(ByLength::new(DEFAULT_CACHE_CAP)), @@ -448,6 +450,21 @@ impl RequestResultCache { self.minimum_backing_votes.insert(session_index, minimum_backing_votes); } + pub(crate) fn availability_chunk_shuffling_params( + &mut self, + key: &Hash, + ) -> Option<&vstaging::AvailabilityChunkShufflingParams> { + self.availability_chunk_shuffling_params.get(key).map(|v| &*v) + } + + pub(crate) fn cache_availability_chunk_shuffling_params( + &mut self, + key: Hash, + params: vstaging::AvailabilityChunkShufflingParams, + ) { + self.availability_chunk_shuffling_params.insert(key, params); + } + pub(crate) fn staging_para_backing_state( &mut self, key: (Hash, ParaId), @@ -524,6 +541,7 @@ pub(crate) enum RequestResult { vstaging::slashing::OpaqueKeyOwnershipProof, Option<()>, ), + AvailabilityChunkShufflingParams(Hash, vstaging::AvailabilityChunkShufflingParams), StagingParaBackingState(Hash, ParaId, Option), StagingAsyncBackingParams(Hash, vstaging::AsyncBackingParams), diff --git a/polkadot/node/core/runtime-api/src/lib.rs b/polkadot/node/core/runtime-api/src/lib.rs index ec9bf10fa6e3..07e2c8b3ad9c 100644 --- a/polkadot/node/core/runtime-api/src/lib.rs +++ b/polkadot/node/core/runtime-api/src/lib.rs @@ -166,6 +166,9 @@ where .requests_cache .cache_key_ownership_proof((relay_parent, validator_id), key_ownership_proof), SubmitReportDisputeLost(_, _, _, _) => {}, + AvailabilityChunkShufflingParams(relay_parent, params) => self + .requests_cache + .cache_availability_chunk_shuffling_params(relay_parent, params), StagingParaBackingState(relay_parent, para_id, constraints) => self .requests_cache @@ -313,6 +316,9 @@ where Some(Request::MinimumBackingVotes(index, sender)) } }, + Request::AvailabilityChunkShufflingParams(sender) => + query!(availability_chunk_shuffling_params(), sender) + .map(|sender| Request::AvailabilityChunkShufflingParams(sender)), } } @@ -569,6 +575,12 @@ where ver = Request::MINIMUM_BACKING_VOTES_RUNTIME_REQUIREMENT, sender ), + Request::AvailabilityChunkShufflingParams(sender) => query!( + AvailabilityChunkShufflingParams, + availability_chunk_shuffling_params(), + ver = Request::AVAILABILITY_CHUNK_SHUFFLING_PARAMS_RUNTIME_REQUIREMENT, + sender + ), Request::StagingParaBackingState(para, sender) => { query!( diff --git a/polkadot/node/network/availability-distribution/src/requester/mod.rs b/polkadot/node/network/availability-distribution/src/requester/mod.rs index 845644abea4a..72de0e920160 100644 --- a/polkadot/node/network/availability-distribution/src/requester/mod.rs +++ b/polkadot/node/network/availability-distribution/src/requester/mod.rs @@ -35,8 +35,8 @@ use polkadot_node_subsystem::{ overseer, ActivatedLeaf, ActiveLeavesUpdate, LeafStatus, }; use polkadot_node_subsystem_util::{ - runtime::{get_occupied_cores, RuntimeInfo}, - shuffle_availability_chunks, + availability_chunk_indices, + runtime::{get_occupied_cores, request_availability_chunk_shuffling_params, RuntimeInfo}, }; use polkadot_primitives::{ BlockNumber, CandidateHash, ChunkIndex, Hash, OccupiedCore, SessionIndex, @@ -264,14 +264,28 @@ impl Requester { acc = acc.saturating_add(group.len()); acc }); + + if self.chunk_index_cache.peek(&block_number).is_none() { + let maybe_av_chunk_shuffling_params = + // TODO: think some more if this relay parent is ok to use + request_availability_chunk_shuffling_params(leaf, context.sender()) + .await?; + + let chunk_indices = availability_chunk_indices( + maybe_av_chunk_shuffling_params, + block_number, + n_validators, + ); + self.chunk_index_cache.insert( + block_number, + chunk_indices[usize::try_from(session_info.our_index.0) + .expect("usize is at least u32 bytes on all modern targets.")], + ); + } + let chunk_index = self .chunk_index_cache - .get_or_insert(block_number, || { - let shuffled_indices = - shuffle_availability_chunks(block_number, n_validators); - shuffled_indices[usize::try_from(session_info.our_index.0) - .expect("usize is at least u32 bytes on all modern targets.")] - }) + .get(&block_number) .expect("The index was just inserted"); let task_cfg = FetchTaskConfig::new( @@ -286,8 +300,6 @@ impl Requester { self.fetches .insert(core.candidate_hash, FetchTask::start(task_cfg, context).await?); - } else { - // Error } } } diff --git a/polkadot/node/network/availability-distribution/src/requester/session_cache.rs b/polkadot/node/network/availability-distribution/src/requester/session_cache.rs index 821d778c22d4..f6fec74796d7 100644 --- a/polkadot/node/network/availability-distribution/src/requester/session_cache.rs +++ b/polkadot/node/network/availability-distribution/src/requester/session_cache.rs @@ -96,11 +96,12 @@ impl SessionCache { parent: Hash, session_index: SessionIndex, ) -> Result> { + gum::trace!(target: LOG_TARGET, session_index, "Calling `get_session_info`"); + if self.session_info_cache.get(&session_index).is_none() { if let Some(info) = Self::query_info_from_runtime(ctx, runtime, parent, session_index).await? { - gum::trace!(target: LOG_TARGET, session_index, "Calling `with_info`"); gum::trace!(target: LOG_TARGET, session_index, "Storing session info in lru!"); self.session_info_cache.insert(session_index, info); } else { diff --git a/polkadot/node/network/availability-recovery/src/error.rs b/polkadot/node/network/availability-recovery/src/error.rs index 50acbe989edb..82757cb21c4b 100644 --- a/polkadot/node/network/availability-recovery/src/error.rs +++ b/polkadot/node/network/availability-recovery/src/error.rs @@ -33,6 +33,9 @@ pub enum Error { #[error("failed to query session info")] CanceledSessionInfo(#[source] oneshot::Canceled), + #[error("failed to query availability chunks shuffling params")] + RequestAvailabilityChunkShufflingParams(#[source] polkadot_node_subsystem_util::runtime::Error), + #[error("failed to send response")] CanceledResponseSender, diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index 4a5a8be50ec9..42023db1eeab 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -59,7 +59,10 @@ use polkadot_node_subsystem::{ overseer, ActiveLeavesUpdate, FromOrchestra, OverseerSignal, SpawnedSubsystem, SubsystemContext, SubsystemError, SubsystemResult, }; -use polkadot_node_subsystem_util::{request_session_info, shuffle_availability_chunks}; +use polkadot_node_subsystem_util::{ + availability_chunk_indices, request_session_info, + runtime::request_availability_chunk_shuffling_params, +}; use polkadot_primitives::{ BlakeTwo256, BlockNumber, CandidateHash, CandidateReceipt, ChunkIndex, GroupIndex, Hash, HashT, SessionIndex, SessionInfo, ValidatorIndex, @@ -463,23 +466,35 @@ async fn handle_recover( let block_number = get_block_number(ctx.sender(), receipt.descriptor.relay_parent).await?; - let shuffling = state - .chunk_indices_cache - .get_or_insert(block_number, || { - shuffle_availability_chunks(block_number, session_info.validators.len()) - .iter() - .enumerate() - .map(|(v_index, c_index)| { - ( - *c_index, - ValidatorIndex( - u32::try_from(v_index) - .expect("validator numbers should not exceed u32"), - ), - ) - }) - .collect() + if state.chunk_indices_cache.peek(&block_number).is_none() { + let maybe_av_chunk_shuffling_params = + // TODO: think some more if this relay parent is ok to use + request_availability_chunk_shuffling_params(state.live_block.1, ctx.sender()) + .await.map_err(error::Error::RequestAvailabilityChunkShufflingParams)?; + + let chunk_indices = availability_chunk_indices( + maybe_av_chunk_shuffling_params, + block_number, + session_info.validators.len(), + ) + .iter() + .enumerate() + .map(|(v_index, c_index)| { + ( + *c_index, + ValidatorIndex( + u32::try_from(v_index) + .expect("validator numbers should not exceed u32"), + ), + ) }) + .collect(); + state.chunk_indices_cache.insert(block_number, chunk_indices); + } + + let chunk_indices = state + .chunk_indices_cache + .get(&block_number) .expect("The shuffling was just inserted"); let backing_validators = if let Some(backing_group) = backing_group { @@ -489,7 +504,7 @@ async fn handle_recover( }; let fetch_chunks_params = FetchChunksParams { - validators: shuffling.clone(), + validators: chunk_indices.clone(), erasure_task_tx: erasure_task_tx.clone(), }; @@ -520,7 +535,7 @@ async fn handle_recover( systematic_recovery_threshold(session_info.validators.len())?; // Only get the validators according to the threshold. - let validators = shuffling + let validators = chunk_indices .clone() .into_iter() .filter(|(c_index, _)| { diff --git a/polkadot/node/subsystem-types/src/messages.rs b/polkadot/node/subsystem-types/src/messages.rs index a53908d3c2cb..9c1b73cac20f 100644 --- a/polkadot/node/subsystem-types/src/messages.rs +++ b/polkadot/node/subsystem-types/src/messages.rs @@ -39,14 +39,15 @@ use polkadot_node_primitives::{ ValidationResult, }; use polkadot_primitives::{ - slashing, vstaging as vstaging_primitives, AuthorityDiscoveryId, BackedCandidate, BlockNumber, - CandidateEvent, CandidateHash, CandidateIndex, CandidateReceipt, CollatorId, - CommittedCandidateReceipt, CoreState, DisputeState, ExecutorParams, GroupIndex, - GroupRotationInfo, Hash, Header as BlockHeader, Id as ParaId, InboundDownwardMessage, - InboundHrmpMessage, MultiDisputeStatementSet, OccupiedCoreAssumption, PersistedValidationData, - PvfCheckStatement, PvfExecTimeoutKind, SessionIndex, SessionInfo, SignedAvailabilityBitfield, - SignedAvailabilityBitfields, ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, - ValidatorSignature, + slashing, + vstaging::{self as vstaging_primitives, AvailabilityChunkShufflingParams}, + AuthorityDiscoveryId, BackedCandidate, BlockNumber, CandidateEvent, CandidateHash, + CandidateIndex, CandidateReceipt, CollatorId, CommittedCandidateReceipt, CoreState, + DisputeState, ExecutorParams, GroupIndex, GroupRotationInfo, Hash, Header as BlockHeader, + Id as ParaId, InboundDownwardMessage, InboundHrmpMessage, MultiDisputeStatementSet, + OccupiedCoreAssumption, PersistedValidationData, PvfCheckStatement, PvfExecTimeoutKind, + SessionIndex, SessionInfo, SignedAvailabilityBitfield, SignedAvailabilityBitfields, + ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, ValidatorSignature, }; use polkadot_statement_table::v2::Misbehavior; use std::{ @@ -695,6 +696,8 @@ pub enum RuntimeApiRequest { ), /// Get the minimum required backing votes. MinimumBackingVotes(SessionIndex, RuntimeApiSender), + /// Get the params for availability chunk shuffling. + AvailabilityChunkShufflingParams(RuntimeApiSender), /// Get the backing state of the given para. /// This is a staging API that will not be available on production runtimes. @@ -726,6 +729,9 @@ impl RuntimeApiRequest { /// `MinimumBackingVotes` pub const MINIMUM_BACKING_VOTES_RUNTIME_REQUIREMENT: u32 = 6; + /// `AvailabilityChunkShufflingParams` + pub const AVAILABILITY_CHUNK_SHUFFLING_PARAMS_RUNTIME_REQUIREMENT: u32 = 7; + /// Minimum version for backing state, required for async backing. /// /// 99 for now, should be adjusted to VSTAGING/actual runtime version once released. diff --git a/polkadot/node/subsystem-types/src/runtime_client.rs b/polkadot/node/subsystem-types/src/runtime_client.rs index 06aa351efb4b..3ff4d9488c86 100644 --- a/polkadot/node/subsystem-types/src/runtime_client.rs +++ b/polkadot/node/subsystem-types/src/runtime_client.rs @@ -16,12 +16,13 @@ use async_trait::async_trait; use polkadot_primitives::{ - runtime_api::ParachainHost, vstaging, Block, BlockNumber, CandidateCommitments, CandidateEvent, - CandidateHash, CommittedCandidateReceipt, CoreState, DisputeState, ExecutorParams, - GroupRotationInfo, Hash, Id, InboundDownwardMessage, InboundHrmpMessage, - OccupiedCoreAssumption, PersistedValidationData, PvfCheckStatement, ScrapedOnChainVotes, - SessionIndex, SessionInfo, ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, - ValidatorSignature, + runtime_api::ParachainHost, + vstaging::{self, AvailabilityChunkShufflingParams}, + Block, BlockNumber, CandidateCommitments, CandidateEvent, CandidateHash, + CommittedCandidateReceipt, CoreState, DisputeState, ExecutorParams, GroupRotationInfo, Hash, + Id, InboundDownwardMessage, InboundHrmpMessage, OccupiedCoreAssumption, + PersistedValidationData, PvfCheckStatement, ScrapedOnChainVotes, SessionIndex, SessionInfo, + ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, ValidatorSignature, }; use sc_transaction_pool_api::OffchainTransactionPoolFactory; use sp_api::{ApiError, ApiExt, ProvideRuntimeApi}; @@ -240,6 +241,12 @@ pub trait RuntimeApiSubsystemClient { session_index: SessionIndex, ) -> Result; + /// Get the parameters for availability-chunk shuffling. + async fn availability_chunk_shuffling_params( + &self, + at: Hash, + ) -> Result; + // === Asynchronous backing API === /// Returns candidate's acceptance limitations for asynchronous backing for a relay parent. @@ -489,6 +496,13 @@ where self.client.runtime_api().minimum_backing_votes(at) } + async fn availability_chunk_shuffling_params( + &self, + at: Hash, + ) -> Result { + self.client.runtime_api().availability_chunk_shuffling_params(at) + } + async fn staging_para_backing_state( &self, at: Hash, diff --git a/polkadot/node/subsystem-util/src/lib.rs b/polkadot/node/subsystem-util/src/lib.rs index 4c7735f18693..745455a77955 100644 --- a/polkadot/node/subsystem-util/src/lib.rs +++ b/polkadot/node/subsystem-util/src/lib.rs @@ -34,7 +34,9 @@ use polkadot_node_subsystem::{ messages::{RuntimeApiMessage, RuntimeApiRequest, RuntimeApiSender}, overseer, SubsystemSender, }; -use polkadot_primitives::{slashing, BlockNumber, ExecutorParams}; +use polkadot_primitives::{ + slashing, vstaging::AvailabilityChunkShufflingParams, BlockNumber, ExecutorParams, +}; use rand::seq::SliceRandom; use rand_chacha::ChaCha8Rng; @@ -440,24 +442,35 @@ impl Validator { } } -/// Shuffle the availability chunk indices, returning a mapping of (`ValidatorIndex -> ChunkIndex`). +/// Return a mapping of the availability chunk indices(`ValidatorIndex -> ChunkIndex`). /// /// The vector indices represent validator indices. -/// `BlockNumber` is used a randomness seed, so that other validators have a common view of the -/// shuffle at a given block height. -pub fn shuffle_availability_chunks( +/// If it's enabled, shuffle the indices using the `BlockNumber` as a randomness seed, so that other +/// validators have a common view of the shuffle at a given block height. +/// Otherwise, return the identity vector. +pub fn availability_chunk_indices( + maybe_params: Option, block_number: BlockNumber, n_validators: usize, ) -> Vec { - let seed = block_number.to_be_bytes(); - let mut rng: ChaCha8Rng = SeedableRng::from_seed( - seed.repeat(8) - .try_into() - .expect("vector of 32 bytes is safe to cast to array of 32 bytes. qed."), - ); + let mut indices: Vec<_> = (0..n_validators) + .map(|i| ValidatorIndex(u32::try_from(i).expect("validator numbers should not exceed u32"))) + .collect(); - let mut shuffled_indices: Vec<_> = (0..n_validators).map(|i| ValidatorIndex(i as _)).collect(); + if let Some(AvailabilityChunkShufflingParams { activate_at: Some(activation_block_number) }) = + maybe_params + { + if block_number >= activation_block_number { + let seed = block_number.to_be_bytes(); + let mut rng: ChaCha8Rng = SeedableRng::from_seed( + seed.repeat(8) + .try_into() + .expect("vector of 32 bytes is safe to cast to array of 32 bytes. qed."), + ); + + indices.shuffle(&mut rng); + } + } - shuffled_indices.shuffle(&mut rng); - shuffled_indices + indices } diff --git a/polkadot/node/subsystem-util/src/runtime/error.rs b/polkadot/node/subsystem-util/src/runtime/error.rs index 8751693b078a..1111b119e95f 100644 --- a/polkadot/node/subsystem-util/src/runtime/error.rs +++ b/polkadot/node/subsystem-util/src/runtime/error.rs @@ -28,7 +28,7 @@ pub enum Error { /// Runtime API subsystem is down, which means we're shutting down. #[fatal] #[error("Runtime request got canceled")] - RuntimeRequestCanceled(oneshot::Canceled), + RuntimeRequestCanceled(#[from] oneshot::Canceled), /// Some request to the runtime failed. /// For example if we prune a block we're requesting info about. diff --git a/polkadot/node/subsystem-util/src/runtime/mod.rs b/polkadot/node/subsystem-util/src/runtime/mod.rs index c078b17d2175..ca269ae7f087 100644 --- a/polkadot/node/subsystem-util/src/runtime/mod.rs +++ b/polkadot/node/subsystem-util/src/runtime/mod.rs @@ -30,7 +30,8 @@ use polkadot_node_subsystem::{ }; use polkadot_node_subsystem_types::UnpinHandle; use polkadot_primitives::{ - vstaging, CandidateEvent, CandidateHash, CoreState, EncodeAs, ExecutorParams, GroupIndex, + vstaging::{self, AvailabilityChunkShufflingParams}, + CandidateEvent, CandidateHash, CoreState, EncodeAs, ExecutorParams, GroupIndex, GroupRotationInfo, Hash, IndexedVec, OccupiedCore, ScrapedOnChainVotes, SessionIndex, SessionInfo, Signed, SigningContext, UncheckedSigned, ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, LEGACY_MIN_BACKING_VOTES, @@ -508,3 +509,30 @@ pub async fn request_min_backing_votes( min_backing_votes_res } } + +/// Request the availability chunk shuffling params. +/// Prior to runtime API version 7, just return `None`. +pub async fn request_availability_chunk_shuffling_params( + parent: Hash, + sender: &mut impl overseer::SubsystemSender, +) -> Result> { + let res = recv_runtime( + request_from_runtime(parent, sender, |tx| { + RuntimeApiRequest::AvailabilityChunkShufflingParams(tx) + }) + .await, + ) + .await; + + if let Err(Error::RuntimeRequest(RuntimeApiError::NotSupported { .. })) = res { + gum::trace!( + target: LOG_TARGET, + ?parent, + "Querying the availability chunk shuffling params from the runtime is not supported by the current Runtime API", + ); + + Ok(None) + } else { + res.map(Some) + } +} diff --git a/polkadot/node/subsystem-util/src/tests.rs b/polkadot/node/subsystem-util/src/tests.rs index 7b21b34b046d..2c0fe937dbe2 100644 --- a/polkadot/node/subsystem-util/src/tests.rs +++ b/polkadot/node/subsystem-util/src/tests.rs @@ -99,10 +99,10 @@ fn subset_predefined_generation_check() { #[test] // Just a simple test to make sure that the shuffling is indeed a permutation, to prevent future // mistakes. -fn test_shuffle_availability_chunks() { +fn test_availability_chunk_indices() { let block_number = 89; let n_validators = 200; - let shuffle = shuffle_availability_chunks(block_number, n_validators); + let shuffle = availability_chunk_indices(block_number, n_validators); assert_eq!(shuffle.len(), n_validators); assert_eq!(shuffle.iter().collect::>().len(), n_validators); diff --git a/polkadot/primitives/src/runtime_api.rs b/polkadot/primitives/src/runtime_api.rs index e5f1aa4276ef..7f6c023c5335 100644 --- a/polkadot/primitives/src/runtime_api.rs +++ b/polkadot/primitives/src/runtime_api.rs @@ -247,6 +247,11 @@ sp_api::decl_runtime_apis! { #[api_version(6)] fn minimum_backing_votes() -> u32; + /// Get the availability-chunk shuffling params. + /// This is a staging method! Do not use on production runtimes! + #[api_version(7)] + fn availability_chunk_shuffling_params() -> vstaging::AvailabilityChunkShufflingParams; + /***** Asynchronous backing *****/ /// Returns the state of parachain backing for a given para. diff --git a/polkadot/primitives/src/vstaging/mod.rs b/polkadot/primitives/src/vstaging/mod.rs index ea341ee5b4fc..07af4a4fd034 100644 --- a/polkadot/primitives/src/vstaging/mod.rs +++ b/polkadot/primitives/src/vstaging/mod.rs @@ -135,3 +135,18 @@ pub struct BackingState { /// constraints and each subsequent builds on top of the previous head-data. pub pending_availability: Vec>, } + +/// Parameters for availability chunk shuffling. +/// Previously, the ChunkIndex assigned to a validator was equal to its ValidatorIndex. +/// +/// This enables us to shuffle the chunk indices, per-block. +#[derive(RuntimeDebug, Clone, PartialEq, Encode, Decode, TypeInfo)] +pub struct AvailabilityChunkShufflingParams { + /// If it's `None`, it's not enabled. If it's some block number, consider it's enabled at this + /// specific block height. + /// + /// It can't simply return a bool as this may break availability recovery. If a + /// runtime upgrade were to set this value to bool, parachain candidates that were backed in + /// previous blocks couldn't be recovered, because we'd use the new, breaking, shuffle. + pub activate_at: Option, +} diff --git a/polkadot/runtime/parachains/src/runtime_api_impl/vstaging.rs b/polkadot/runtime/parachains/src/runtime_api_impl/vstaging.rs index deef19d90710..6d450605081d 100644 --- a/polkadot/runtime/parachains/src/runtime_api_impl/vstaging.rs +++ b/polkadot/runtime/parachains/src/runtime_api_impl/vstaging.rs @@ -20,8 +20,9 @@ use crate::{configuration, dmp, hrmp, inclusion, initializer, paras, shared}; use frame_system::pallet_prelude::BlockNumberFor; use primitives::{ vstaging::{ - AsyncBackingParams, BackingState, CandidatePendingAvailability, Constraints, - InboundHrmpLimitations, OutboundHrmpChannelLimitations, + AsyncBackingParams, AvailabilityChunkShufflingParams, BackingState, + CandidatePendingAvailability, Constraints, InboundHrmpLimitations, + OutboundHrmpChannelLimitations, }, Id as ParaId, }; @@ -123,3 +124,11 @@ pub fn async_backing_params() -> AsyncBackingParams { pub fn minimum_backing_votes() -> u32 { >::config().minimum_backing_votes } + +/// Return whether the shuffling of availability chunk indices is enabled. +pub fn availability_chunk_shuffling_params( +) -> AvailabilityChunkShufflingParams { + // TODO: add this value to the state and add a migration for it. + + AvailabilityChunkShufflingParams { activate_at: None } +} From 2b58895f0374f15838845f7142ec14781ca35ed9 Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 18 Sep 2023 15:17:43 +0300 Subject: [PATCH 020/126] av-recovery and av-distr: use relay parent from candidate_receipt when querying the runtime api --- .../availability-distribution/src/requester/mod.rs | 8 +++++--- polkadot/node/network/availability-recovery/src/lib.rs | 10 ++++++---- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/polkadot/node/network/availability-distribution/src/requester/mod.rs b/polkadot/node/network/availability-distribution/src/requester/mod.rs index 72de0e920160..4c94dd59020d 100644 --- a/polkadot/node/network/availability-distribution/src/requester/mod.rs +++ b/polkadot/node/network/availability-distribution/src/requester/mod.rs @@ -267,9 +267,11 @@ impl Requester { if self.chunk_index_cache.peek(&block_number).is_none() { let maybe_av_chunk_shuffling_params = - // TODO: think some more if this relay parent is ok to use - request_availability_chunk_shuffling_params(leaf, context.sender()) - .await?; + request_availability_chunk_shuffling_params( + core.candidate_descriptor.relay_parent, + context.sender(), + ) + .await?; let chunk_indices = availability_chunk_indices( maybe_av_chunk_shuffling_params, diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index 42023db1eeab..c56cc3a82aec 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -467,10 +467,12 @@ async fn handle_recover( get_block_number(ctx.sender(), receipt.descriptor.relay_parent).await?; if state.chunk_indices_cache.peek(&block_number).is_none() { - let maybe_av_chunk_shuffling_params = - // TODO: think some more if this relay parent is ok to use - request_availability_chunk_shuffling_params(state.live_block.1, ctx.sender()) - .await.map_err(error::Error::RequestAvailabilityChunkShufflingParams)?; + let maybe_av_chunk_shuffling_params = request_availability_chunk_shuffling_params( + receipt.descriptor.relay_parent, + ctx.sender(), + ) + .await + .map_err(error::Error::RequestAvailabilityChunkShufflingParams)?; let chunk_indices = availability_chunk_indices( maybe_av_chunk_shuffling_params, From f860b7d5f4d9dd8fc07fa2cb142df0a0171f9ccd Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 18 Sep 2023 15:19:39 +0300 Subject: [PATCH 021/126] add avail_chunk_shuffling params to HostConfiguration --- polkadot/erasure-coding/src/lib.rs | 2 +- polkadot/primitives/src/vstaging/mod.rs | 15 +- .../runtime/parachains/src/configuration.rs | 26 +- .../parachains/src/configuration/migration.rs | 1 + .../src/configuration/migration/v10.rs | 329 ++++++++++++++++++ .../src/configuration/migration/v9.rs | 106 +++++- .../parachains/src/configuration/tests.rs | 8 + .../src/runtime_api_impl/vstaging.rs | 4 +- polkadot/runtime/rococo/src/lib.rs | 25 +- 9 files changed, 491 insertions(+), 25 deletions(-) create mode 100644 polkadot/runtime/parachains/src/configuration/migration/v10.rs diff --git a/polkadot/erasure-coding/src/lib.rs b/polkadot/erasure-coding/src/lib.rs index d7557465e6d3..848858a6a215 100644 --- a/polkadot/erasure-coding/src/lib.rs +++ b/polkadot/erasure-coding/src/lib.rs @@ -99,7 +99,7 @@ pub const fn recovery_threshold(n_validators: usize) -> Result { /// Obtain the threshold of systematic chunks that should be enough to recover the data. /// /// If the regular `recovery_threshold` is a power of two, then it returns the same value. -/// Otherwise, it returns the next power of two. +/// Otherwise, it returns the next lower power of two. pub fn systematic_recovery_threshold(n_validators: usize) -> Result { code_params(n_validators).map(|params| params.k()) } diff --git a/polkadot/primitives/src/vstaging/mod.rs b/polkadot/primitives/src/vstaging/mod.rs index 07af4a4fd034..c4a29042a6fa 100644 --- a/polkadot/primitives/src/vstaging/mod.rs +++ b/polkadot/primitives/src/vstaging/mod.rs @@ -23,21 +23,14 @@ use sp_std::prelude::*; use parity_scale_codec::{Decode, Encode}; use primitives::RuntimeDebug; use scale_info::TypeInfo; +use serde::{Deserialize, Serialize}; /// Useful type alias for Para IDs. pub type ParaId = Id; /// Candidate's acceptance limitations for asynchronous backing per relay parent. #[derive( - RuntimeDebug, - Copy, - Clone, - PartialEq, - Encode, - Decode, - TypeInfo, - serde::Serialize, - serde::Deserialize, + RuntimeDebug, Copy, Clone, PartialEq, Encode, Decode, TypeInfo, Serialize, Deserialize, )] pub struct AsyncBackingParams { @@ -140,7 +133,9 @@ pub struct BackingState { /// Previously, the ChunkIndex assigned to a validator was equal to its ValidatorIndex. /// /// This enables us to shuffle the chunk indices, per-block. -#[derive(RuntimeDebug, Clone, PartialEq, Encode, Decode, TypeInfo)] +#[derive( + RuntimeDebug, Clone, PartialEq, Encode, Decode, TypeInfo, Default, Serialize, Deserialize, +)] pub struct AvailabilityChunkShufflingParams { /// If it's `None`, it's not enabled. If it's some block number, consider it's enabled at this /// specific block height. diff --git a/polkadot/runtime/parachains/src/configuration.rs b/polkadot/runtime/parachains/src/configuration.rs index 33039cd08ca4..2482560b87e7 100644 --- a/polkadot/runtime/parachains/src/configuration.rs +++ b/polkadot/runtime/parachains/src/configuration.rs @@ -26,8 +26,9 @@ use polkadot_parachain_primitives::primitives::{ MAX_HORIZONTAL_MESSAGE_NUM, MAX_UPWARD_MESSAGE_NUM, }; use primitives::{ - vstaging::AsyncBackingParams, Balance, ExecutorParams, SessionIndex, LEGACY_MIN_BACKING_VOTES, - MAX_CODE_SIZE, MAX_HEAD_DATA_SIZE, MAX_POV_SIZE, ON_DEMAND_DEFAULT_QUEUE_MAX_SIZE, + vstaging::{AsyncBackingParams, AvailabilityChunkShufflingParams}, + Balance, ExecutorParams, SessionIndex, LEGACY_MIN_BACKING_VOTES, MAX_CODE_SIZE, + MAX_HEAD_DATA_SIZE, MAX_POV_SIZE, ON_DEMAND_DEFAULT_QUEUE_MAX_SIZE, }; use sp_runtime::{traits::Zero, Perbill}; use sp_std::prelude::*; @@ -259,6 +260,8 @@ pub struct HostConfiguration { /// The minimum number of valid backing statements required to consider a parachain candidate /// backable. pub minimum_backing_votes: u32, + /// Parameters for availability chunk shuffling. + pub availability_chunk_shuffling_params: AvailabilityChunkShufflingParams, } impl> Default for HostConfiguration { @@ -310,6 +313,7 @@ impl> Default for HostConfiguration /// v7-v8: /// v8-v9: - const STORAGE_VERSION: StorageVersion = StorageVersion::new(9); + /// TODO: add PR link here + const STORAGE_VERSION: StorageVersion = StorageVersion::new(10); #[pallet::pallet] #[pallet::storage_version(STORAGE_VERSION)] @@ -1187,6 +1192,21 @@ pub mod pallet { config.minimum_backing_votes = new; }) } + /// Set availability chunk shuffling params. + #[pallet::call_index(53)] + #[pallet::weight(( + T::WeightInfo::set_config_with_option_u32(), // TODO: The same size in bytes? Benchmark. + DispatchClass::Operational + ))] + pub fn set_availability_chunk_shuffling_params( + origin: OriginFor, + new: AvailabilityChunkShufflingParams, + ) -> DispatchResult { + ensure_root(origin)?; + Self::schedule_config_update(|config| { + config.availability_chunk_shuffling_params = new; + }) + } } #[pallet::hooks] diff --git a/polkadot/runtime/parachains/src/configuration/migration.rs b/polkadot/runtime/parachains/src/configuration/migration.rs index 26f8a85b496d..db323d3aad93 100644 --- a/polkadot/runtime/parachains/src/configuration/migration.rs +++ b/polkadot/runtime/parachains/src/configuration/migration.rs @@ -16,6 +16,7 @@ //! A module that is responsible for migration of storage. +pub mod v10; pub mod v6; pub mod v7; pub mod v8; diff --git a/polkadot/runtime/parachains/src/configuration/migration/v10.rs b/polkadot/runtime/parachains/src/configuration/migration/v10.rs new file mode 100644 index 000000000000..1ec4b16da2a1 --- /dev/null +++ b/polkadot/runtime/parachains/src/configuration/migration/v10.rs @@ -0,0 +1,329 @@ +// Copyright (C) Parity Technologies (UK) Ltd. +// This file is part of Polkadot. + +// Polkadot is free software: you can redistribute it and/or modify +// it under the terms of the GNU General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. + +// Polkadot is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU General Public License for more details. + +// You should have received a copy of the GNU General Public License +// along with Polkadot. If not, see . + +//! A module that is responsible for migration of storage. + +use crate::configuration::{self, Config, Pallet}; +use frame_support::{ + pallet_prelude::*, + traits::{Defensive, StorageVersion}, + weights::Weight, +}; +use frame_system::pallet_prelude::BlockNumberFor; +use primitives::{SessionIndex, LEGACY_MIN_BACKING_VOTES}; +use sp_runtime::Perbill; +use sp_std::vec::Vec; + +use frame_support::traits::OnRuntimeUpgrade; + +use super::v9::V9HostConfiguration; + +type V10HostConfiguration = configuration::HostConfiguration; + +mod v9 { + use super::*; + + #[frame_support::storage_alias] + pub(crate) type ActiveConfig = + StorageValue, V9HostConfiguration>, OptionQuery>; + + #[frame_support::storage_alias] + pub(crate) type PendingConfigs = StorageValue< + Pallet, + Vec<(SessionIndex, V9HostConfiguration>)>, + OptionQuery, + >; +} + +mod v10 { + use super::*; + + #[frame_support::storage_alias] + pub(crate) type ActiveConfig = + StorageValue, V10HostConfiguration>, OptionQuery>; + + #[frame_support::storage_alias] + pub(crate) type PendingConfigs = StorageValue< + Pallet, + Vec<(SessionIndex, V10HostConfiguration>)>, + OptionQuery, + >; +} + +pub struct MigrateToV10(sp_std::marker::PhantomData); +impl OnRuntimeUpgrade for MigrateToV10 { + #[cfg(feature = "try-runtime")] + fn pre_upgrade() -> Result, sp_runtime::TryRuntimeError> { + log::trace!(target: crate::configuration::LOG_TARGET, "Running pre_upgrade() for HostConfiguration MigrateToV10"); + Ok(Vec::new()) + } + + fn on_runtime_upgrade() -> Weight { + log::info!(target: configuration::LOG_TARGET, "HostConfiguration MigrateToV10 started"); + if StorageVersion::get::>() == 9 { + let weight_consumed = migrate_to_v10::(); + + log::info!(target: configuration::LOG_TARGET, "HostConfiguration MigrateToV10 executed successfully"); + StorageVersion::new(10).put::>(); + + weight_consumed + } else { + log::warn!(target: configuration::LOG_TARGET, "HostConfiguration MigrateToV10 should be removed."); + T::DbWeight::get().reads(1) + } + } + + #[cfg(feature = "try-runtime")] + fn post_upgrade(_state: Vec) -> Result<(), sp_runtime::TryRuntimeError> { + log::trace!(target: crate::configuration::LOG_TARGET, "Running post_upgrade() for HostConfiguration MigrateToV10"); + ensure!( + StorageVersion::get::>() >= 10, + "Storage version should be >= 10 after the migration" + ); + + Ok(()) + } +} + +fn migrate_to_v10() -> Weight { + // Unusual formatting is justified: + // - make it easier to verify that fields assign what they supposed to assign. + // - this code is transient and will be removed after all migrations are done. + // - this code is important enough to optimize for legibility sacrificing consistency. + #[rustfmt::skip] + let translate = + |pre: V9HostConfiguration>| -> + V10HostConfiguration> + { + V10HostConfiguration { +max_code_size : pre.max_code_size, +max_head_data_size : pre.max_head_data_size, +max_upward_queue_count : pre.max_upward_queue_count, +max_upward_queue_size : pre.max_upward_queue_size, +max_upward_message_size : pre.max_upward_message_size, +max_upward_message_num_per_candidate : pre.max_upward_message_num_per_candidate, +hrmp_max_message_num_per_candidate : pre.hrmp_max_message_num_per_candidate, +validation_upgrade_cooldown : pre.validation_upgrade_cooldown, +validation_upgrade_delay : pre.validation_upgrade_delay, +max_pov_size : pre.max_pov_size, +max_downward_message_size : pre.max_downward_message_size, +hrmp_sender_deposit : pre.hrmp_sender_deposit, +hrmp_recipient_deposit : pre.hrmp_recipient_deposit, +hrmp_channel_max_capacity : pre.hrmp_channel_max_capacity, +hrmp_channel_max_total_size : pre.hrmp_channel_max_total_size, +hrmp_max_parachain_inbound_channels : pre.hrmp_max_parachain_inbound_channels, +hrmp_max_parachain_outbound_channels : pre.hrmp_max_parachain_outbound_channels, +hrmp_channel_max_message_size : pre.hrmp_channel_max_message_size, +code_retention_period : pre.code_retention_period, +on_demand_cores : pre.on_demand_cores, +on_demand_retries : pre.on_demand_retries, +group_rotation_frequency : pre.group_rotation_frequency, +paras_availability_period : pre.paras_availability_period, +scheduling_lookahead : pre.scheduling_lookahead, +max_validators_per_core : pre.max_validators_per_core, +max_validators : pre.max_validators, +dispute_period : pre.dispute_period, +dispute_post_conclusion_acceptance_period: pre.dispute_post_conclusion_acceptance_period, +no_show_slots : pre.no_show_slots, +n_delay_tranches : pre.n_delay_tranches, +zeroth_delay_tranche_width : pre.zeroth_delay_tranche_width, +needed_approvals : pre.needed_approvals, +relay_vrf_modulo_samples : pre.relay_vrf_modulo_samples, +pvf_voting_ttl : pre.pvf_voting_ttl, +minimum_validation_upgrade_delay : pre.minimum_validation_upgrade_delay, +async_backing_params : pre.async_backing_params, +executor_params : pre.executor_params, +on_demand_queue_max_size : 10_000u32, +on_demand_base_fee : 10_000_000u128, +on_demand_fee_variability : Perbill::from_percent(3), +on_demand_target_queue_utilization : Perbill::from_percent(25), +on_demand_ttl : 5u32.into(), +minimum_backing_votes : LEGACY_MIN_BACKING_VOTES, +availability_chunk_shuffling_params : Default::default() + } + }; + + let v9 = v9::ActiveConfig::::get() + .defensive_proof("Could not decode old config") + .unwrap_or_default(); + let v10 = translate(v9); + v10::ActiveConfig::::set(Some(v10)); + + // Allowed to be empty. + let pending_v9 = v9::PendingConfigs::::get().unwrap_or_default(); + let mut pending_v10 = Vec::new(); + + for (session, v9) in pending_v9.into_iter() { + let v10 = translate(v9); + pending_v10.push((session, v10)); + } + v10::PendingConfigs::::set(Some(pending_v10.clone())); + + let num_configs = (pending_v10.len() + 1) as u64; + T::DbWeight::get().reads_writes(num_configs, num_configs) +} + +#[cfg(test)] +mod tests { + use primitives::vstaging::AvailabilityChunkShufflingParams; + + use super::*; + use crate::mock::{new_test_ext, Test}; + + #[test] + fn v10_deserialized_from_actual_data() { + // Example how to get new `raw_config`: + // We'll obtain the raw_config at a specified a block + // Steps: + // 1. Go to Polkadot.js -> Developer -> Chain state -> Storage: https://polkadot.js.org/apps/#/chainstate + // 2. Set these parameters: + // 2.1. selected state query: configuration; activeConfig(): + // PolkadotRuntimeParachainsConfigurationHostConfiguration + // 2.2. blockhash to query at: + // 0xf89d3ab5312c5f70d396dc59612f0aa65806c798346f9db4b35278baed2e0e53 (the hash of + // the block) + // 2.3. Note the value of encoded storage key -> + // 0x06de3d8a54d27e44a9d5ce189618f22db4b49d95320d9021994c850f25b8e385 for the + // referenced block. + // 2.4. You'll also need the decoded values to update the test. + // 3. Go to Polkadot.js -> Developer -> Chain state -> Raw storage + // 3.1 Enter the encoded storage key and you get the raw config. + + // This exceeds the maximal line width length, but that's fine, since this is not code and + // doesn't need to be read and also leaving it as one line allows to easily copy it. + let raw_config = + hex_literal::hex![" + 0000300000800000080000000000100000c8000005000000050000000200000002000000000000000000000000005000000010000400000000000000000000000000000000000000000000000000000000000000000000000800000000200000040000000000100000b004000000000000000000001027000080b2e60e80c3c901809698000000000000000000000000000500000014000000040000000100000001050000000006000000640000000200000019000000000000000a000000020000000200000005000000020000000164000000" + ]; + + let v10 = + V10HostConfiguration::::decode(&mut &raw_config[..]).unwrap(); + + // We check only a sample of the values here. If we missed any fields or messed up data + // types that would skew all the fields coming after. + assert_eq!(v10.max_code_size, 3_145_728); + assert_eq!(v10.validation_upgrade_cooldown, 2); + assert_eq!(v10.max_pov_size, 5_242_880); + assert_eq!(v10.hrmp_channel_max_message_size, 1_048_576); + assert_eq!(v10.n_delay_tranches, 25); + assert_eq!(v10.minimum_validation_upgrade_delay, 5); + assert_eq!(v10.group_rotation_frequency, 20); + assert_eq!(v10.on_demand_cores, 0); + assert_eq!(v10.on_demand_base_fee, 10_000_000); + assert_eq!(v10.minimum_backing_votes, LEGACY_MIN_BACKING_VOTES); + assert_eq!( + v10.availability_chunk_shuffling_params, + AvailabilityChunkShufflingParams { activate_at: Some(100) } + ); + } + + #[test] + fn test_migrate_to_v10() { + // Host configuration has lots of fields. However, in this migration we only add one + // field. The most important part to check are a couple of the last fields. We also pick + // extra fields to check arbitrarily, e.g. depending on their position (i.e. the middle) and + // also their type. + // + // We specify only the picked fields and the rest should be provided by the `Default` + // implementation. That implementation is copied over between the two types and should work + // fine. + let v9 = V9HostConfiguration:: { + needed_approvals: 69, + paras_availability_period: 55, + hrmp_recipient_deposit: 1337, + max_pov_size: 1111, + minimum_validation_upgrade_delay: 20, + ..Default::default() + }; + + let mut pending_configs = Vec::new(); + pending_configs.push((100, v9.clone())); + pending_configs.push((300, v9.clone())); + + new_test_ext(Default::default()).execute_with(|| { + // Implant the v8 version in the state. + v9::ActiveConfig::::set(Some(v9)); + v9::PendingConfigs::::set(Some(pending_configs)); + + migrate_to_v10::(); + + let v10 = v10::ActiveConfig::::get().unwrap(); + let mut configs_to_check = v10::PendingConfigs::::get().unwrap(); + configs_to_check.push((0, v10.clone())); + + for (_, v9) in configs_to_check { + #[rustfmt::skip] + { + assert_eq!(v9.max_code_size , v10.max_code_size); + assert_eq!(v9.max_head_data_size , v10.max_head_data_size); + assert_eq!(v9.max_upward_queue_count , v10.max_upward_queue_count); + assert_eq!(v9.max_upward_queue_size , v10.max_upward_queue_size); + assert_eq!(v9.max_upward_message_size , v10.max_upward_message_size); + assert_eq!(v9.max_upward_message_num_per_candidate , v10.max_upward_message_num_per_candidate); + assert_eq!(v9.hrmp_max_message_num_per_candidate , v10.hrmp_max_message_num_per_candidate); + assert_eq!(v9.validation_upgrade_cooldown , v10.validation_upgrade_cooldown); + assert_eq!(v9.validation_upgrade_delay , v10.validation_upgrade_delay); + assert_eq!(v9.max_pov_size , v10.max_pov_size); + assert_eq!(v9.max_downward_message_size , v10.max_downward_message_size); + assert_eq!(v9.hrmp_max_parachain_outbound_channels , v10.hrmp_max_parachain_outbound_channels); + assert_eq!(v9.hrmp_sender_deposit , v10.hrmp_sender_deposit); + assert_eq!(v9.hrmp_recipient_deposit , v10.hrmp_recipient_deposit); + assert_eq!(v9.hrmp_channel_max_capacity , v10.hrmp_channel_max_capacity); + assert_eq!(v9.hrmp_channel_max_total_size , v10.hrmp_channel_max_total_size); + assert_eq!(v9.hrmp_max_parachain_inbound_channels , v10.hrmp_max_parachain_inbound_channels); + assert_eq!(v9.hrmp_channel_max_message_size , v10.hrmp_channel_max_message_size); + assert_eq!(v9.code_retention_period , v10.code_retention_period); + assert_eq!(v9.on_demand_cores , v10.on_demand_cores); + assert_eq!(v9.on_demand_retries , v10.on_demand_retries); + assert_eq!(v9.group_rotation_frequency , v10.group_rotation_frequency); + assert_eq!(v9.paras_availability_period , v10.paras_availability_period); + assert_eq!(v9.scheduling_lookahead , v10.scheduling_lookahead); + assert_eq!(v9.max_validators_per_core , v10.max_validators_per_core); + assert_eq!(v9.max_validators , v10.max_validators); + assert_eq!(v9.dispute_period , v10.dispute_period); + assert_eq!(v9.no_show_slots , v10.no_show_slots); + assert_eq!(v9.n_delay_tranches , v10.n_delay_tranches); + assert_eq!(v9.zeroth_delay_tranche_width , v10.zeroth_delay_tranche_width); + assert_eq!(v9.needed_approvals , v10.needed_approvals); + assert_eq!(v9.relay_vrf_modulo_samples , v10.relay_vrf_modulo_samples); + assert_eq!(v9.pvf_voting_ttl , v10.pvf_voting_ttl); + assert_eq!(v9.minimum_validation_upgrade_delay , v10.minimum_validation_upgrade_delay); + assert_eq!(v9.async_backing_params.allowed_ancestry_len, v10.async_backing_params.allowed_ancestry_len); + assert_eq!(v9.async_backing_params.max_candidate_depth , v10.async_backing_params.max_candidate_depth); + assert_eq!(v9.executor_params , v10.executor_params); + assert_eq!(v9.minimum_backing_votes , v10.minimum_backing_votes); + }; // ; makes this a statement. `rustfmt::skip` cannot be put on an expression. + } + }); + } + + // Test that migration doesn't panic in case there're no pending configurations upgrades in + // pallet's storage. + #[test] + fn test_migrate_to_v10_no_pending() { + let v9 = V9HostConfiguration::::default(); + + new_test_ext(Default::default()).execute_with(|| { + // Implant the v9 version in the state. + v9::ActiveConfig::::set(Some(v9)); + // Ensure there're no pending configs. + v9::PendingConfigs::::set(None); + + // Shouldn't fail. + migrate_to_v10::(); + }); + } +} diff --git a/polkadot/runtime/parachains/src/configuration/migration/v9.rs b/polkadot/runtime/parachains/src/configuration/migration/v9.rs index 64d71e628f4e..ec4672d905bc 100644 --- a/polkadot/runtime/parachains/src/configuration/migration/v9.rs +++ b/polkadot/runtime/parachains/src/configuration/migration/v9.rs @@ -23,14 +23,116 @@ use frame_support::{ weights::Weight, }; use frame_system::pallet_prelude::BlockNumberFor; -use primitives::{SessionIndex, LEGACY_MIN_BACKING_VOTES}; +use primitives::{ + vstaging::AsyncBackingParams, Balance, ExecutorParams, SessionIndex, LEGACY_MIN_BACKING_VOTES, + ON_DEMAND_DEFAULT_QUEUE_MAX_SIZE, +}; use sp_runtime::Perbill; use sp_std::vec::Vec; use frame_support::traits::OnRuntimeUpgrade; use super::v8::V8HostConfiguration; -type V9HostConfiguration = configuration::HostConfiguration; +/// All configuration of the runtime with respect to paras. +#[derive(Clone, Encode, Decode, Debug)] +pub struct V9HostConfiguration { + pub max_code_size: u32, + pub max_head_data_size: u32, + pub max_upward_queue_count: u32, + pub max_upward_queue_size: u32, + pub max_upward_message_size: u32, + pub max_upward_message_num_per_candidate: u32, + pub hrmp_max_message_num_per_candidate: u32, + pub validation_upgrade_cooldown: BlockNumber, + pub validation_upgrade_delay: BlockNumber, + pub async_backing_params: AsyncBackingParams, + pub max_pov_size: u32, + pub max_downward_message_size: u32, + pub hrmp_max_parachain_outbound_channels: u32, + pub hrmp_sender_deposit: Balance, + pub hrmp_recipient_deposit: Balance, + pub hrmp_channel_max_capacity: u32, + pub hrmp_channel_max_total_size: u32, + pub hrmp_max_parachain_inbound_channels: u32, + pub hrmp_channel_max_message_size: u32, + pub executor_params: ExecutorParams, + pub code_retention_period: BlockNumber, + pub on_demand_cores: u32, + pub on_demand_retries: u32, + pub on_demand_queue_max_size: u32, + pub on_demand_target_queue_utilization: Perbill, + pub on_demand_fee_variability: Perbill, + pub on_demand_base_fee: Balance, + pub on_demand_ttl: BlockNumber, + pub group_rotation_frequency: BlockNumber, + pub paras_availability_period: BlockNumber, + pub scheduling_lookahead: u32, + pub max_validators_per_core: Option, + pub max_validators: Option, + pub dispute_period: SessionIndex, + pub dispute_post_conclusion_acceptance_period: BlockNumber, + pub no_show_slots: u32, + pub n_delay_tranches: u32, + pub zeroth_delay_tranche_width: u32, + pub needed_approvals: u32, + pub relay_vrf_modulo_samples: u32, + pub pvf_voting_ttl: SessionIndex, + pub minimum_validation_upgrade_delay: BlockNumber, + pub minimum_backing_votes: u32, +} + +impl> Default for V9HostConfiguration { + fn default() -> Self { + Self { + async_backing_params: AsyncBackingParams { + max_candidate_depth: 0, + allowed_ancestry_len: 0, + }, + group_rotation_frequency: 1u32.into(), + paras_availability_period: 1u32.into(), + no_show_slots: 1u32.into(), + validation_upgrade_cooldown: Default::default(), + validation_upgrade_delay: 2u32.into(), + code_retention_period: Default::default(), + max_code_size: Default::default(), + max_pov_size: Default::default(), + max_head_data_size: Default::default(), + on_demand_cores: Default::default(), + on_demand_retries: Default::default(), + scheduling_lookahead: 1, + max_validators_per_core: Default::default(), + max_validators: None, + dispute_period: 6, + dispute_post_conclusion_acceptance_period: 100.into(), + n_delay_tranches: Default::default(), + zeroth_delay_tranche_width: Default::default(), + needed_approvals: Default::default(), + relay_vrf_modulo_samples: Default::default(), + max_upward_queue_count: Default::default(), + max_upward_queue_size: Default::default(), + max_downward_message_size: Default::default(), + max_upward_message_size: Default::default(), + max_upward_message_num_per_candidate: Default::default(), + hrmp_sender_deposit: Default::default(), + hrmp_recipient_deposit: Default::default(), + hrmp_channel_max_capacity: Default::default(), + hrmp_channel_max_total_size: Default::default(), + hrmp_max_parachain_inbound_channels: Default::default(), + hrmp_channel_max_message_size: Default::default(), + hrmp_max_parachain_outbound_channels: Default::default(), + hrmp_max_message_num_per_candidate: Default::default(), + pvf_voting_ttl: 2u32.into(), + minimum_validation_upgrade_delay: 2.into(), + executor_params: Default::default(), + on_demand_queue_max_size: ON_DEMAND_DEFAULT_QUEUE_MAX_SIZE, + on_demand_base_fee: 10_000_000u128, + on_demand_fee_variability: Perbill::from_percent(3), + on_demand_target_queue_utilization: Perbill::from_percent(25), + on_demand_ttl: 5u32.into(), + minimum_backing_votes: LEGACY_MIN_BACKING_VOTES, + } + } +} mod v8 { use super::*; diff --git a/polkadot/runtime/parachains/src/configuration/tests.rs b/polkadot/runtime/parachains/src/configuration/tests.rs index ea39628c9587..0e9aa999d898 100644 --- a/polkadot/runtime/parachains/src/configuration/tests.rs +++ b/polkadot/runtime/parachains/src/configuration/tests.rs @@ -318,6 +318,9 @@ fn setting_pending_config_members() { on_demand_target_queue_utilization: Perbill::from_percent(25), on_demand_ttl: 5u32, minimum_backing_votes: 5, + availability_chunk_shuffling_params: AvailabilityChunkShufflingParams { + activate_at: Some(100), + }, }; Configuration::set_validation_upgrade_cooldown( @@ -473,6 +476,11 @@ fn setting_pending_config_members() { new_config.minimum_backing_votes, ) .unwrap(); + Configuration::set_availability_chunk_shuffling_params( + RuntimeOrigin::root(), + new_config.availability_chunk_shuffling_params.clone(), + ) + .unwrap(); assert_eq!(PendingConfigs::::get(), vec![(shared::SESSION_DELAY, new_config)],); }) diff --git a/polkadot/runtime/parachains/src/runtime_api_impl/vstaging.rs b/polkadot/runtime/parachains/src/runtime_api_impl/vstaging.rs index 6d450605081d..4a03885ee458 100644 --- a/polkadot/runtime/parachains/src/runtime_api_impl/vstaging.rs +++ b/polkadot/runtime/parachains/src/runtime_api_impl/vstaging.rs @@ -128,7 +128,5 @@ pub fn minimum_backing_votes() -> u32 { /// Return whether the shuffling of availability chunk indices is enabled. pub fn availability_chunk_shuffling_params( ) -> AvailabilityChunkShufflingParams { - // TODO: add this value to the state and add a migration for it. - - AvailabilityChunkShufflingParams { activate_at: None } + >::config().availability_chunk_shuffling_params } diff --git a/polkadot/runtime/rococo/src/lib.rs b/polkadot/runtime/rococo/src/lib.rs index e043852901f1..a6a3f0908d84 100644 --- a/polkadot/runtime/rococo/src/lib.rs +++ b/polkadot/runtime/rococo/src/lib.rs @@ -23,11 +23,12 @@ use pallet_nis::WithMaximumOf; use parity_scale_codec::{Decode, Encode, MaxEncodedLen}; use primitives::{ - slashing, AccountId, AccountIndex, Balance, BlockNumber, CandidateEvent, CandidateHash, - CommittedCandidateReceipt, CoreState, DisputeState, ExecutorParams, GroupRotationInfo, Hash, - Id as ParaId, InboundDownwardMessage, InboundHrmpMessage, Moment, Nonce, - OccupiedCoreAssumption, PersistedValidationData, ScrapedOnChainVotes, SessionInfo, Signature, - ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, PARACHAIN_KEY_TYPE_ID, + slashing, vstaging::AvailabilityChunkShufflingParams, AccountId, AccountIndex, Balance, + BlockNumber, CandidateEvent, CandidateHash, CommittedCandidateReceipt, CoreState, DisputeState, + ExecutorParams, GroupRotationInfo, Hash, Id as ParaId, InboundDownwardMessage, + InboundHrmpMessage, Moment, Nonce, OccupiedCoreAssumption, PersistedValidationData, + ScrapedOnChainVotes, SessionInfo, Signature, ValidationCode, ValidationCodeHash, ValidatorId, + ValidatorIndex, PARACHAIN_KEY_TYPE_ID, }; use runtime_common::{ assigned_slots, auctions, claims, crowdloan, impl_runtime_weights, impls::ToAuthor, @@ -46,7 +47,9 @@ use runtime_parachains::{ inclusion::{AggregateMessageOrigin, UmpQueueId}, initializer as parachains_initializer, origin as parachains_origin, paras as parachains_paras, paras_inherent as parachains_paras_inherent, - runtime_api_impl::v5 as parachains_runtime_api_impl, + runtime_api_impl::{ + v5 as parachains_runtime_api_impl, vstaging as parachains_staging_runtime_api_impl, + }, scheduler as parachains_scheduler, session_info as parachains_session_info, shared as parachains_shared, }; @@ -1552,6 +1555,7 @@ pub mod migrations { parachains_configuration::migration::v8::MigrateToV8, parachains_configuration::migration::v9::MigrateToV9, paras_registrar::migration::VersionCheckedMigrateToV1, + parachains_configuration::migration::v10::MigrateToV10, ); } @@ -1713,6 +1717,7 @@ sp_api::impl_runtime_apis! { } } + #[api_version(7)] impl primitives::runtime_api::ParachainHost for Runtime { fn validators() -> Vec { parachains_runtime_api_impl::validators::() @@ -1843,6 +1848,14 @@ sp_api::impl_runtime_apis! { key_ownership_proof, ) } + + fn minimum_backing_votes() -> u32 { + parachains_staging_runtime_api_impl::minimum_backing_votes::() + } + + fn availability_chunk_shuffling_params() -> AvailabilityChunkShufflingParams { + parachains_staging_runtime_api_impl::availability_chunk_shuffling_params::() + } } #[api_version(3)] From b82b2a00b56483a71186cc0835c51fed042e69e3 Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 19 Sep 2023 09:52:21 +0300 Subject: [PATCH 022/126] replace BypassAvStore variant with a separate flag --- .../network/availability-recovery/src/lib.rs | 32 ++++++++++++------- 1 file changed, 21 insertions(+), 11 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index 98087bca6f0b..e2146981da92 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -90,17 +90,17 @@ pub enum RecoveryStrategyKind { BackersFirstIfSizeLower(usize), /// We always recover using validator chunks. ChunksAlways, - /// Do not request data from the availability store. - /// This is the useful for nodes where the - /// availability-store subsystem is not expected to run, - /// such as collators. - BypassAvailabilityStore, } /// The Availability Recovery Subsystem. pub struct AvailabilityRecoverySubsystem { /// PoV recovery strategy to use. recovery_strategy_kind: RecoveryStrategyKind, + // If this is true, do not request data from the availability store. + /// This is the useful for nodes where the + /// availability-store subsystem is not expected to run, + /// such as collators. + bypass_availability_store: bool, /// Receiver for available data requests. req_receiver: IncomingRequestReceiver, /// Metrics for this subsystem. @@ -389,6 +389,7 @@ async fn handle_recover( metrics: &Metrics, erasure_task_tx: futures::channel::mpsc::Sender, recovery_strategy_kind: RecoveryStrategyKind, + bypass_availability_store: bool, ) -> error::Result<()> { let candidate_hash = receipt.hash(); @@ -460,7 +461,6 @@ async fn handle_recover( match (&recovery_strategy_kind, small_pov_size) { (RecoveryStrategyKind::BackersFirstAlways, _) | - (RecoveryStrategyKind::BypassAvailabilityStore, _) | (RecoveryStrategyKind::BackersFirstIfSizeLower(_), true) => recovery_strategies.push_back( Box::new(FetchFull::new(FetchFullParams { validators: backing_validators.to_vec(), @@ -485,7 +485,7 @@ async fn handle_recover( response_sender, metrics, recovery_strategies, - recovery_strategy_kind == RecoveryStrategyKind::BypassAvailabilityStore, + bypass_availability_store, ) .await }, @@ -534,7 +534,8 @@ impl AvailabilityRecoverySubsystem { metrics: Metrics, ) -> Self { Self { - recovery_strategy_kind: RecoveryStrategyKind::BypassAvailabilityStore, + recovery_strategy_kind: RecoveryStrategyKind::BackersFirstIfSizeLower(SMALL_POV_LIMIT), + bypass_availability_store: true, req_receiver, metrics, } @@ -548,6 +549,7 @@ impl AvailabilityRecoverySubsystem { ) -> Self { Self { recovery_strategy_kind: RecoveryStrategyKind::BackersFirstAlways, + bypass_availability_store: false, req_receiver, metrics, } @@ -558,7 +560,12 @@ impl AvailabilityRecoverySubsystem { req_receiver: IncomingRequestReceiver, metrics: Metrics, ) -> Self { - Self { recovery_strategy_kind: RecoveryStrategyKind::ChunksAlways, req_receiver, metrics } + Self { + recovery_strategy_kind: RecoveryStrategyKind::ChunksAlways, + bypass_availability_store: false, + req_receiver, + metrics, + } } /// Create a new instance of `AvailabilityRecoverySubsystem` which requests chunks if PoV is @@ -569,6 +576,7 @@ impl AvailabilityRecoverySubsystem { ) -> Self { Self { recovery_strategy_kind: RecoveryStrategyKind::BackersFirstIfSizeLower(SMALL_POV_LIMIT), + bypass_availability_store: false, req_receiver, metrics, } @@ -576,7 +584,8 @@ impl AvailabilityRecoverySubsystem { async fn run(self, mut ctx: Context) -> SubsystemResult<()> { let mut state = State::default(); - let Self { mut req_receiver, metrics, recovery_strategy_kind } = self; + let Self { mut req_receiver, metrics, recovery_strategy_kind, bypass_availability_store } = + self; let (erasure_task_tx, erasure_task_rx) = futures::channel::mpsc::channel(16); let mut erasure_task_rx = erasure_task_rx.fuse(); @@ -666,6 +675,7 @@ impl AvailabilityRecoverySubsystem { &metrics, erasure_task_tx.clone(), recovery_strategy_kind.clone(), + bypass_availability_store ).await { gum::warn!( target: LOG_TARGET, @@ -681,7 +691,7 @@ impl AvailabilityRecoverySubsystem { in_req = recv_req => { match in_req.into_nested().map_err(|fatal| SubsystemError::with_origin("availability-recovery", fatal))? { Ok(req) => { - if recovery_strategy_kind == RecoveryStrategyKind::BypassAvailabilityStore { + if bypass_availability_store { gum::debug!( target: LOG_TARGET, "Skipping request to availability-store.", From 8585628341aa1b40b74b6a432014f74d72229aef Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 19 Sep 2023 10:04:23 +0300 Subject: [PATCH 023/126] move requesting_chunks to the strategy --- .../network/availability-recovery/src/task.rs | 42 ++++++++++--------- 1 file changed, 22 insertions(+), 20 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 30eaf7734a52..d5bc2da84944 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -102,13 +102,11 @@ pub struct RecoveryParams { pub struct State { /// Chunks received so far. received_chunks: HashMap, - /// Collection of in-flight requests. - requesting_chunks: FuturesUndead, (ValidatorIndex, RequestError)>>, } impl State { fn new() -> Self { - Self { received_chunks: HashMap::new(), requesting_chunks: FuturesUndead::new() } + Self { received_chunks: HashMap::new() } } fn insert_chunk(&mut self, validator: ValidatorIndex, chunk: ErasureChunk) { @@ -174,15 +172,18 @@ impl State { sender: &mut Sender, desired_requests_count: usize, validators: &mut VecDeque, + requesting_chunks: &mut FuturesUndead< + Result, (ValidatorIndex, RequestError)>, + >, ) where Sender: overseer::AvailabilityRecoverySenderTrait, { let candidate_hash = ¶ms.candidate_hash; - let already_requesting_count = self.requesting_chunks.len(); + let already_requesting_count = requesting_chunks.len(); let mut requests = Vec::with_capacity(desired_requests_count - already_requesting_count); - while self.requesting_chunks.len() < desired_requests_count { + while requesting_chunks.len() < desired_requests_count { if let Some(validator_index) = validators.pop_back() { let validator = params.validator_authority_keys[validator_index.0 as usize].clone(); gum::trace!( @@ -205,7 +206,7 @@ impl State { params.metrics.on_chunk_request_issued(); let timer = params.metrics.time_chunk_request(); - self.requesting_chunks.push(Box::pin(async move { + requesting_chunks.push(Box::pin(async move { let _timer = timer; match res.await { Ok(req_res::v1::ChunkFetchingResponse::Chunk(chunk)) => @@ -232,6 +233,9 @@ impl State { &mut self, params: &RecoveryParams, validators: &mut VecDeque, + requesting_chunks: &mut FuturesUndead< + Result, (ValidatorIndex, RequestError)>, + >, can_conclude: impl Fn(usize, usize, usize, &RecoveryParams, usize) -> bool, ) -> (usize, usize) { let metrics = ¶ms.metrics; @@ -243,7 +247,7 @@ impl State { // We also declare requests undead, once `TIMEOUT_START_NEW_REQUESTS` is reached and will // return in that case for `launch_parallel_requests` to fill up slots again. while let Some(request_result) = - self.requesting_chunks.next_with_timeout(TIMEOUT_START_NEW_REQUESTS).await + requesting_chunks.next_with_timeout(TIMEOUT_START_NEW_REQUESTS).await { total_received_responses += 1; @@ -313,7 +317,7 @@ impl State { // or have gotten firm 'No' responses from enough validators. if can_conclude( validators.len(), - self.requesting_chunks.total_len(), + requesting_chunks.total_len(), self.chunk_count(), params, error_count, @@ -322,7 +326,7 @@ impl State { target: LOG_TARGET, candidate_hash = ?params.candidate_hash, received_chunks_count = ?self.chunk_count(), - requested_chunks_count = ?self.requesting_chunks.len(), + requested_chunks_count = ?requesting_chunks.len(), threshold = ?params.threshold, "Can conclude availability for a candidate", ); @@ -394,11 +398,6 @@ where let _timer = self.params.metrics.time_full_recovery(); while let Some(mut current_strategy) = self.strategies.pop_front() { - // Make sure we are not referencing futures from past RecoveryStrategy runs. - if self.state.requesting_chunks.total_len() != 0 { - self.state.requesting_chunks = FuturesUndead::new(); - } - gum::debug!( target: LOG_TARGET, candidate_hash = ?self.params.candidate_hash, @@ -555,11 +554,11 @@ pub struct FetchChunks { error_count: usize, /// Total number of responses that have been received, including failed ones. total_received_responses: usize, - + /// Collection of in-flight requests. + requesting_chunks: FuturesUndead, (ValidatorIndex, RequestError)>>, /// A random shuffling of the validators which indicates the order in which we connect to the /// validators and request the chunk from them. validators: VecDeque, - /// Channel to the erasure task handler. erasure_task_tx: futures::channel::mpsc::Sender, } @@ -583,6 +582,7 @@ impl FetchChunks { Self { error_count: 0, total_received_responses: 0, + requesting_chunks: FuturesUndead::new(), validators: shuffling.into(), erasure_task_tx: params.erasure_task_tx, } @@ -732,7 +732,7 @@ impl RecoveryStrategy if Self::is_unavailable( self.validators.len(), - state.requesting_chunks.total_len(), + self.requesting_chunks.total_len(), state.chunk_count(), common_params.threshold, ) { @@ -741,8 +741,8 @@ impl RecoveryStrategy candidate_hash = ?common_params.candidate_hash, erasure_root = ?common_params.erasure_root, received = %state.chunk_count(), - requesting = %state.requesting_chunks.len(), - total_requesting = %state.requesting_chunks.total_len(), + requesting = %self.requesting_chunks.len(), + total_requesting = %self.requesting_chunks.total_len(), n_validators = %common_params.n_validators, "Data recovery from chunks is not possible", ); @@ -752,7 +752,7 @@ impl RecoveryStrategy let desired_requests_count = self.get_desired_request_count(state.chunk_count(), common_params.threshold); - let already_requesting_count = state.requesting_chunks.len(); + let already_requesting_count = self.requesting_chunks.len(); gum::debug!( target: LOG_TARGET, ?common_params.candidate_hash, @@ -769,6 +769,7 @@ impl RecoveryStrategy sender, desired_requests_count, &mut self.validators, + &mut self.requesting_chunks, ) .await; @@ -776,6 +777,7 @@ impl RecoveryStrategy .wait_for_chunks( common_params, &mut self.validators, + &mut self.requesting_chunks, |unrequested_validators, reqs, chunk_count, params, _error_count| { chunk_count >= params.threshold || Self::is_unavailable( From 818d9c0181311ed60724a5a326bfdcb6fd6ab5bb Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 19 Sep 2023 11:34:39 +0300 Subject: [PATCH 024/126] fix tests --- .../availability-recovery/src/tests.rs | 88 +++++++++++++------ 1 file changed, 61 insertions(+), 27 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/tests.rs b/polkadot/node/network/availability-recovery/src/tests.rs index 5e6daa88a684..5e2090890e30 100644 --- a/polkadot/node/network/availability-recovery/src/tests.rs +++ b/polkadot/node/network/availability-recovery/src/tests.rs @@ -40,7 +40,8 @@ use polkadot_node_subsystem_test_helpers::{ }; use polkadot_node_subsystem_util::TimeoutExt; use polkadot_primitives::{ - AuthorityDiscoveryId, Hash, HeadData, IndexedVec, PersistedValidationData, ValidatorId, + vstaging::AvailabilityChunkShufflingParams, AuthorityDiscoveryId, Hash, HeadData, IndexedVec, + PersistedValidationData, ValidatorId, }; use polkadot_primitives_test_helpers::{dummy_candidate_receipt, dummy_hash}; @@ -179,7 +180,7 @@ impl TestState { self.validators.len() - self.threshold() + 1 } - async fn test_runtime_api(&self, virtual_overseer: &mut VirtualOverseer) { + async fn test_runtime_api_session_info(&self, virtual_overseer: &mut VirtualOverseer) { assert_matches!( overseer_recv(virtual_overseer).await, AllMessages::RuntimeApi(RuntimeApiMessage::Request( @@ -212,6 +213,26 @@ impl TestState { ); } + async fn test_runtime_api_chunk_shuffling(&self, virtual_overseer: &mut VirtualOverseer) { + assert_matches!( + overseer_recv(virtual_overseer).await, + AllMessages::RuntimeApi(RuntimeApiMessage::Request( + relay_parent, + RuntimeApiRequest::AvailabilityChunkShufflingParams( + tx, + ) + )) => { + // assert_eq!(relay_parent, self.current); + + tx.send(Ok( + AvailabilityChunkShufflingParams { + activate_at: None + } + )).unwrap(); + } + ); + } + async fn respond_to_block_number_query( &self, virtual_overseer: &mut VirtualOverseer, @@ -532,8 +553,9 @@ fn availability_is_recovered_from_chunks_if_no_group_provided() { ) .await; - test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -570,7 +592,7 @@ fn availability_is_recovered_from_chunks_if_no_group_provided() { ) .await; - test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; @@ -620,8 +642,9 @@ fn availability_is_recovered_from_chunks_even_if_backing_group_supplied_if_chunk ) .await; - test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -658,7 +681,7 @@ fn availability_is_recovered_from_chunks_even_if_backing_group_supplied_if_chunk ) .await; - test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; @@ -708,8 +731,9 @@ fn bad_merkle_path_leads_to_recovery_error() { ) .await; - test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -768,8 +792,9 @@ fn wrong_chunk_index_leads_to_recovery_error() { ) .await; - test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -844,8 +869,9 @@ fn invalid_erasure_coding_leads_to_invalid_error() { ) .await; - test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; @@ -894,8 +920,9 @@ fn fast_path_backing_group_recovers() { ) .await; - test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -947,7 +974,9 @@ fn recovers_from_only_chunks_if_pov_large() { ) .await; - test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -960,8 +989,6 @@ fn recovers_from_only_chunks_if_pov_large() { } ); - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; @@ -995,7 +1022,8 @@ fn recovers_from_only_chunks_if_pov_large() { ) .await; - test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; assert_matches!( overseer_recv(&mut virtual_overseer).await, @@ -1006,8 +1034,6 @@ fn recovers_from_only_chunks_if_pov_large() { } ); - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; @@ -1057,7 +1083,9 @@ fn fast_path_backing_group_recovers_if_pov_small() { ) .await; - test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -1075,8 +1103,6 @@ fn fast_path_backing_group_recovers_if_pov_small() { } ); - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state @@ -1118,8 +1144,9 @@ fn no_answers_in_fast_path_causes_chunk_requests() { ) .await; - test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -1181,8 +1208,9 @@ fn task_canceled_when_receivers_dropped() { ) .await; - test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; for _ in 0..test_state.validators.len() { match virtual_overseer.recv().timeout(TIMEOUT).await { @@ -1224,8 +1252,9 @@ fn chunks_retry_until_all_nodes_respond() { ) .await; - test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -1286,8 +1315,9 @@ fn not_returning_requests_wont_stall_retrieval() { ) .await; - test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -1359,8 +1389,9 @@ fn all_not_returning_requests_still_recovers_on_return() { ) .await; - test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -1437,8 +1468,9 @@ fn returns_early_if_we_have_the_data() { ) .await; - test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; test_state.respond_to_available_data_query(&mut virtual_overseer, true).await; assert_eq!(rx.await.unwrap().unwrap(), test_state.available_data); @@ -1475,8 +1507,9 @@ fn does_not_query_local_validator() { ) .await; - test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |i| i == 0).await; @@ -1535,8 +1568,9 @@ fn invalid_local_chunk_is_ignored() { ) .await; - test_state.test_runtime_api(&mut virtual_overseer).await; + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state .respond_to_query_all_request_invalid(&mut virtual_overseer, |i| i == 0) From 6b8afcb69b2afe5912843aaceb0060cd3f207ca7 Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 19 Sep 2023 16:50:14 +0300 Subject: [PATCH 025/126] add erasure-coding benchmark for systematic recovery --- polkadot/erasure-coding/benches/README.md | 6 ++- .../benches/scaling_with_validators.rs | 37 +++++++++++++++++-- 2 files changed, 38 insertions(+), 5 deletions(-) diff --git a/polkadot/erasure-coding/benches/README.md b/polkadot/erasure-coding/benches/README.md index 94fca5400c61..20f79827d280 100644 --- a/polkadot/erasure-coding/benches/README.md +++ b/polkadot/erasure-coding/benches/README.md @@ -7,7 +7,8 @@ cargo bench ## `scaling_with_validators` This benchmark evaluates the performance of constructing the chunks and the erasure root from PoV and -reconstructing the PoV from chunks. You can see the results of running this bench on 5950x below. +reconstructing the PoV from chunks (either from systematic chunks or regular chunks). +You can see the results of running this bench on 5950x below (only including recovery from regular chunks). Interestingly, with `10_000` chunks (validators) its slower than with `50_000` for both construction and reconstruction. ``` @@ -37,3 +38,6 @@ reconstruct/10000 time: [496.35 ms 505.17 ms 515.42 ms] reconstruct/50000 time: [276.56 ms 277.53 ms 278.58 ms] thrpt: [17.948 MiB/s 18.016 MiB/s 18.079 MiB/s] ``` + +Results from running on an Apple M2 Pro, systematic recovery is generally 40 times faster than +regular recovery, achieving 1 Gib/s. diff --git a/polkadot/erasure-coding/benches/scaling_with_validators.rs b/polkadot/erasure-coding/benches/scaling_with_validators.rs index 759385bbdef4..0c30f3e0c02b 100644 --- a/polkadot/erasure-coding/benches/scaling_with_validators.rs +++ b/polkadot/erasure-coding/benches/scaling_with_validators.rs @@ -53,12 +53,16 @@ fn construct_and_reconstruct_5mb_pov(c: &mut Criterion) { } group.finish(); - let mut group = c.benchmark_group("reconstruct"); + let mut group = c.benchmark_group("reconstruct_regular"); for n_validators in N_VALIDATORS { let all_chunks = chunks(n_validators, &pov); - let mut c: Vec<_> = all_chunks.iter().enumerate().map(|(i, c)| (&c[..], i)).collect(); - let last_chunks = c.split_off((c.len() - 1) * 2 / 3); + let chunks: Vec<_> = all_chunks + .iter() + .enumerate() + .take(polkadot_erasure_coding::recovery_threshold(n_validators).unwrap()) + .map(|(i, c)| (&c[..], i)) + .collect(); group.throughput(Throughput::Bytes(pov.len() as u64)); group.bench_with_input( @@ -67,7 +71,32 @@ fn construct_and_reconstruct_5mb_pov(c: &mut Criterion) { |b, &n| { b.iter(|| { let _pov: Vec = - polkadot_erasure_coding::reconstruct(n, last_chunks.clone()).unwrap(); + polkadot_erasure_coding::reconstruct(n, chunks.clone()).unwrap(); + }); + }, + ); + } + group.finish(); + + let mut group = c.benchmark_group("reconstruct_systematic"); + for n_validators in N_VALIDATORS { + let all_chunks = chunks(n_validators, &pov); + + let chunks = all_chunks + .iter() + .take(polkadot_erasure_coding::systematic_recovery_threshold(n_validators).unwrap()) + .map(|c| &c[..]) + .collect::>(); + + group.throughput(Throughput::Bytes(pov.len() as u64)); + group.bench_with_input( + BenchmarkId::from_parameter(n_validators), + &n_validators, + |b, &n| { + b.iter(|| { + let _pov: Vec = + polkadot_erasure_coding::reconstruct_from_systematic(n, chunks.clone()) + .unwrap(); }); }, ); From ce633b446470a69f3cb5fec3c81aa8967db8f16f Mon Sep 17 00:00:00 2001 From: alindima Date: Wed, 20 Sep 2023 15:02:48 +0300 Subject: [PATCH 026/126] fix clippy --- polkadot/node/core/runtime-api/src/tests.rs | 19 +++++++++++++------ .../network/availability-recovery/src/task.rs | 2 +- .../availability-recovery/src/tests.rs | 4 +--- polkadot/primitives/src/vstaging/mod.rs | 2 +- 4 files changed, 16 insertions(+), 11 deletions(-) diff --git a/polkadot/node/core/runtime-api/src/tests.rs b/polkadot/node/core/runtime-api/src/tests.rs index bb7c29689611..3785e7b1e027 100644 --- a/polkadot/node/core/runtime-api/src/tests.rs +++ b/polkadot/node/core/runtime-api/src/tests.rs @@ -20,12 +20,12 @@ use polkadot_node_primitives::{BabeAllowedSlots, BabeEpoch, BabeEpochConfigurati use polkadot_node_subsystem::SpawnGlue; use polkadot_node_subsystem_test_helpers::make_subsystem_context; use polkadot_primitives::{ - vstaging, AuthorityDiscoveryId, BlockNumber, CandidateCommitments, CandidateEvent, - CandidateHash, CommittedCandidateReceipt, CoreState, DisputeState, ExecutorParams, - GroupRotationInfo, Id as ParaId, InboundDownwardMessage, InboundHrmpMessage, - OccupiedCoreAssumption, PersistedValidationData, PvfCheckStatement, ScrapedOnChainVotes, - SessionIndex, SessionInfo, Slot, ValidationCode, ValidationCodeHash, ValidatorId, - ValidatorIndex, ValidatorSignature, + vstaging::{self, AvailabilityChunkShufflingParams}, + AuthorityDiscoveryId, BlockNumber, CandidateCommitments, CandidateEvent, CandidateHash, + CommittedCandidateReceipt, CoreState, DisputeState, ExecutorParams, GroupRotationInfo, + Id as ParaId, InboundDownwardMessage, InboundHrmpMessage, OccupiedCoreAssumption, + PersistedValidationData, PvfCheckStatement, ScrapedOnChainVotes, SessionIndex, SessionInfo, + Slot, ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, ValidatorSignature, }; use sp_api::ApiError; use sp_core::testing::TaskExecutor; @@ -268,6 +268,13 @@ impl RuntimeApiSubsystemClient for MockSubsystemClient { async fn minimum_backing_votes(&self, _: Hash, _: SessionIndex) -> Result { todo!("Not required for tests") } + + async fn availability_chunk_shuffling_params( + &self, + _: Hash, + ) -> Result { + todo!("Not required for tests") + } } #[test] diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 956708dcc90a..90bfb76e2ed8 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -765,7 +765,7 @@ impl RecoveryStrategy for our_c_index in &local_chunk_indices { // If we are among the systematic validators but hold an invalid chunk, we cannot // perform the systematic recovery. Fall through to the next strategy. - if self.validators.iter().find(|(c_index, _)| c_index == our_c_index).is_some() && + if self.validators.iter().any(|(c_index, _)| c_index == our_c_index) && !state.received_chunks.contains_key(our_c_index) { gum::debug!( diff --git a/polkadot/node/network/availability-recovery/src/tests.rs b/polkadot/node/network/availability-recovery/src/tests.rs index 5e2090890e30..2df2d132a090 100644 --- a/polkadot/node/network/availability-recovery/src/tests.rs +++ b/polkadot/node/network/availability-recovery/src/tests.rs @@ -217,13 +217,11 @@ impl TestState { assert_matches!( overseer_recv(virtual_overseer).await, AllMessages::RuntimeApi(RuntimeApiMessage::Request( - relay_parent, + _relay_parent, RuntimeApiRequest::AvailabilityChunkShufflingParams( tx, ) )) => { - // assert_eq!(relay_parent, self.current); - tx.send(Ok( AvailabilityChunkShufflingParams { activate_at: None diff --git a/polkadot/primitives/src/vstaging/mod.rs b/polkadot/primitives/src/vstaging/mod.rs index c4a29042a6fa..3852f331b99b 100644 --- a/polkadot/primitives/src/vstaging/mod.rs +++ b/polkadot/primitives/src/vstaging/mod.rs @@ -141,7 +141,7 @@ pub struct AvailabilityChunkShufflingParams { /// specific block height. /// /// It can't simply return a bool as this may break availability recovery. If a - /// runtime upgrade were to set this value to bool, parachain candidates that were backed in + /// runtime upgrade were to set this value to true, parachain candidates that were backed in /// previous blocks couldn't be recovered, because we'd use the new, breaking, shuffle. pub activate_at: Option, } From 0fb256a47791ebb45ddb685d8ffca1826c22ac5c Mon Sep 17 00:00:00 2001 From: alindima Date: Wed, 20 Sep 2023 18:34:24 +0300 Subject: [PATCH 027/126] av-distribution is not sending av-recovery messages --- polkadot/node/overseer/src/lib.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/polkadot/node/overseer/src/lib.rs b/polkadot/node/overseer/src/lib.rs index debac7000d46..6f984a0f4055 100644 --- a/polkadot/node/overseer/src/lib.rs +++ b/polkadot/node/overseer/src/lib.rs @@ -496,7 +496,6 @@ pub struct Overseer { #[subsystem(AvailabilityDistributionMessage, sends: [ AvailabilityStoreMessage, - AvailabilityRecoveryMessage, ChainApiMessage, RuntimeApiMessage, NetworkBridgeTxMessage, From b2c92b3df96e238336e5401f60cbfdc9b1556cb7 Mon Sep 17 00:00:00 2001 From: alindima Date: Wed, 20 Sep 2023 18:34:47 +0300 Subject: [PATCH 028/126] fix clippy --- .../src/blockchain_rpc_client.rs | 9 ++++++++- .../relay-chain-rpc-interface/src/rpc_client.rs | 14 +++++++++++++- 2 files changed, 21 insertions(+), 2 deletions(-) diff --git a/cumulus/client/relay-chain-minimal-node/src/blockchain_rpc_client.rs b/cumulus/client/relay-chain-minimal-node/src/blockchain_rpc_client.rs index 57e16bc4283c..72ae522bef81 100644 --- a/cumulus/client/relay-chain-minimal-node/src/blockchain_rpc_client.rs +++ b/cumulus/client/relay-chain-minimal-node/src/blockchain_rpc_client.rs @@ -23,7 +23,7 @@ use polkadot_core_primitives::{Block, BlockNumber, Hash, Header}; use polkadot_overseer::RuntimeApiSubsystemClient; use polkadot_primitives::{ slashing, - vstaging::{AsyncBackingParams, BackingState}, + vstaging::{AsyncBackingParams, AvailabilityChunkShufflingParams, BackingState}, }; use sc_authority_discovery::{AuthorityDiscovery, Error as AuthorityDiscoveryError}; use sp_api::{ApiError, RuntimeApiInfo}; @@ -346,6 +346,13 @@ impl RuntimeApiSubsystemClient for BlockChainRpcClient { Ok(self.rpc_client.parachain_host_minimum_backing_votes(at, session_index).await?) } + async fn availability_chunk_shuffling_params( + &self, + at: Hash, + ) -> Result { + Ok(self.rpc_client.parachain_host_availability_chunk_shuffling_params(at).await?) + } + async fn staging_async_backing_params(&self, at: Hash) -> Result { Ok(self.rpc_client.parachain_host_staging_async_backing_params(at).await?) } diff --git a/cumulus/client/relay-chain-rpc-interface/src/rpc_client.rs b/cumulus/client/relay-chain-rpc-interface/src/rpc_client.rs index c1e92b249d77..b636a0b62b63 100644 --- a/cumulus/client/relay-chain-rpc-interface/src/rpc_client.rs +++ b/cumulus/client/relay-chain-rpc-interface/src/rpc_client.rs @@ -31,7 +31,7 @@ use parity_scale_codec::{Decode, Encode}; use cumulus_primitives_core::{ relay_chain::{ slashing, - vstaging::{AsyncBackingParams, BackingState}, + vstaging::{AsyncBackingParams, AvailabilityChunkShufflingParams, BackingState}, BlockNumber, CandidateCommitments, CandidateEvent, CandidateHash, CommittedCandidateReceipt, CoreState, DisputeState, ExecutorParams, GroupRotationInfo, Hash as RelayHash, Header as RelayHeader, InboundHrmpMessage, OccupiedCoreAssumption, @@ -598,6 +598,18 @@ impl RelayChainRpcClient { .await } + pub async fn parachain_host_availability_chunk_shuffling_params( + &self, + at: RelayHash, + ) -> Result { + self.call_remote_runtime_function( + "ParachainHost_availability_chunk_shuffling_params", + at, + None::<()>, + ) + .await + } + #[allow(missing_docs)] pub async fn parachain_host_staging_async_backing_params( &self, From 3c3371ac38533af717ff214bfa9e9bba16ef5688 Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 22 Sep 2023 15:06:53 +0300 Subject: [PATCH 029/126] add client_features runtime API replace the AvChunksShufflingParams with a bitfield that enables/disables client-side features. This will be useful to coordinate enablement of different client features/optimisation in the future. --- Cargo.lock | 1 + .../src/blockchain_rpc_client.rs | 9 ++-- .../src/rpc_client.rs | 14 +++---- .../node/core/bitfield-signing/src/lib.rs | 13 +++--- polkadot/node/core/runtime-api/src/cache.rs | 21 ++++------ polkadot/node/core/runtime-api/src/lib.rs | 18 ++++---- polkadot/node/core/runtime-api/src/tests.rs | 7 +--- .../src/requester/mod.rs | 15 ++++--- .../availability-recovery/src/error.rs | 4 +- .../network/availability-recovery/src/lib.rs | 15 +++---- .../availability-recovery/src/tests.rs | 42 +++++++++---------- polkadot/node/subsystem-types/src/messages.rs | 10 ++--- .../subsystem-types/src/runtime_client.rs | 27 +++++------- polkadot/node/subsystem-util/src/lib.rs | 12 ++---- .../node/subsystem-util/src/runtime/mod.rs | 13 +++--- polkadot/primitives/Cargo.toml | 1 + polkadot/primitives/src/runtime_api.rs | 2 +- polkadot/primitives/src/vstaging/mod.rs | 22 ++++------ .../runtime/parachains/src/configuration.rs | 16 +++---- .../src/configuration/migration/v10.rs | 13 +++--- .../parachains/src/configuration/tests.rs | 10 ++--- .../src/runtime_api_impl/vstaging.rs | 12 +++--- polkadot/runtime/rococo/src/lib.rs | 8 ++-- 23 files changed, 128 insertions(+), 177 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 80d1a32a12a5..68885d09bf85 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -12467,6 +12467,7 @@ dependencies = [ name = "polkadot-primitives" version = "1.0.0" dependencies = [ + "bitflags 1.3.2", "bitvec", "hex-literal", "parity-scale-codec", diff --git a/cumulus/client/relay-chain-minimal-node/src/blockchain_rpc_client.rs b/cumulus/client/relay-chain-minimal-node/src/blockchain_rpc_client.rs index 72ae522bef81..6f5ea7174a61 100644 --- a/cumulus/client/relay-chain-minimal-node/src/blockchain_rpc_client.rs +++ b/cumulus/client/relay-chain-minimal-node/src/blockchain_rpc_client.rs @@ -23,7 +23,7 @@ use polkadot_core_primitives::{Block, BlockNumber, Hash, Header}; use polkadot_overseer::RuntimeApiSubsystemClient; use polkadot_primitives::{ slashing, - vstaging::{AsyncBackingParams, AvailabilityChunkShufflingParams, BackingState}, + vstaging::{AsyncBackingParams, BackingState, ClientFeatures}, }; use sc_authority_discovery::{AuthorityDiscovery, Error as AuthorityDiscoveryError}; use sp_api::{ApiError, RuntimeApiInfo}; @@ -346,11 +346,8 @@ impl RuntimeApiSubsystemClient for BlockChainRpcClient { Ok(self.rpc_client.parachain_host_minimum_backing_votes(at, session_index).await?) } - async fn availability_chunk_shuffling_params( - &self, - at: Hash, - ) -> Result { - Ok(self.rpc_client.parachain_host_availability_chunk_shuffling_params(at).await?) + async fn client_features(&self, at: Hash) -> Result { + Ok(self.rpc_client.parachain_host_client_features(at).await?) } async fn staging_async_backing_params(&self, at: Hash) -> Result { diff --git a/cumulus/client/relay-chain-rpc-interface/src/rpc_client.rs b/cumulus/client/relay-chain-rpc-interface/src/rpc_client.rs index b636a0b62b63..e4f048f0e834 100644 --- a/cumulus/client/relay-chain-rpc-interface/src/rpc_client.rs +++ b/cumulus/client/relay-chain-rpc-interface/src/rpc_client.rs @@ -31,7 +31,7 @@ use parity_scale_codec::{Decode, Encode}; use cumulus_primitives_core::{ relay_chain::{ slashing, - vstaging::{AsyncBackingParams, AvailabilityChunkShufflingParams, BackingState}, + vstaging::{AsyncBackingParams, BackingState, ClientFeatures}, BlockNumber, CandidateCommitments, CandidateEvent, CandidateHash, CommittedCandidateReceipt, CoreState, DisputeState, ExecutorParams, GroupRotationInfo, Hash as RelayHash, Header as RelayHeader, InboundHrmpMessage, OccupiedCoreAssumption, @@ -598,16 +598,12 @@ impl RelayChainRpcClient { .await } - pub async fn parachain_host_availability_chunk_shuffling_params( + pub async fn parachain_host_client_features( &self, at: RelayHash, - ) -> Result { - self.call_remote_runtime_function( - "ParachainHost_availability_chunk_shuffling_params", - at, - None::<()>, - ) - .await + ) -> Result { + self.call_remote_runtime_function("ParachainHost_client_features", at, None::<()>) + .await } #[allow(missing_docs)] diff --git a/polkadot/node/core/bitfield-signing/src/lib.rs b/polkadot/node/core/bitfield-signing/src/lib.rs index 9ff7184c30ed..e762eff06763 100644 --- a/polkadot/node/core/bitfield-signing/src/lib.rs +++ b/polkadot/node/core/bitfield-signing/src/lib.rs @@ -36,8 +36,8 @@ use polkadot_node_subsystem::{ SpawnedSubsystem, SubsystemError, SubsystemResult, SubsystemSender, }; use polkadot_node_subsystem_util::{ - self as util, availability_chunk_indices, request_validators, - runtime::request_availability_chunk_shuffling_params, Validator, + self as util, availability_chunk_indices, request_validators, runtime::request_client_features, + Validator, }; use polkadot_primitives::{AvailabilityBitfield, BlockNumber, CoreState, Hash, ValidatorIndex}; use sp_keystore::{Error as KeystoreError, KeystorePtr}; @@ -102,12 +102,11 @@ async fn get_core_availability( let block_number = get_block_number(*sender.lock().await, core.candidate_descriptor.relay_parent).await?; - let maybe_av_chunk_shuffling_params = - request_availability_chunk_shuffling_params(relay_parent, *sender.lock().await) - .await - .map_err(Error::from)?; + let maybe_client_features = request_client_features(relay_parent, *sender.lock().await) + .await + .map_err(Error::from)?; let chunk_indices = - availability_chunk_indices(maybe_av_chunk_shuffling_params, block_number, n_validators); + availability_chunk_indices(maybe_client_features, block_number, n_validators); let chunk_index = chunk_indices[usize::try_from(validator_idx.0) .expect("usize is at least u32 bytes on all modern targets.")]; diff --git a/polkadot/node/core/runtime-api/src/cache.rs b/polkadot/node/core/runtime-api/src/cache.rs index 1a87d8f52b52..9689d35e4345 100644 --- a/polkadot/node/core/runtime-api/src/cache.rs +++ b/polkadot/node/core/runtime-api/src/cache.rs @@ -66,7 +66,7 @@ pub(crate) struct RequestResultCache { key_ownership_proof: LruMap<(Hash, ValidatorId), Option>, minimum_backing_votes: LruMap, - availability_chunk_shuffling_params: LruMap, + client_features: LruMap, staging_para_backing_state: LruMap<(Hash, ParaId), Option>, staging_async_backing_params: LruMap, @@ -100,7 +100,7 @@ impl Default for RequestResultCache { unapplied_slashes: LruMap::new(ByLength::new(DEFAULT_CACHE_CAP)), key_ownership_proof: LruMap::new(ByLength::new(DEFAULT_CACHE_CAP)), minimum_backing_votes: LruMap::new(ByLength::new(DEFAULT_CACHE_CAP)), - availability_chunk_shuffling_params: LruMap::new(ByLength::new(DEFAULT_CACHE_CAP)), + client_features: LruMap::new(ByLength::new(DEFAULT_CACHE_CAP)), staging_para_backing_state: LruMap::new(ByLength::new(DEFAULT_CACHE_CAP)), staging_async_backing_params: LruMap::new(ByLength::new(DEFAULT_CACHE_CAP)), @@ -450,19 +450,12 @@ impl RequestResultCache { self.minimum_backing_votes.insert(session_index, minimum_backing_votes); } - pub(crate) fn availability_chunk_shuffling_params( - &mut self, - key: &Hash, - ) -> Option<&vstaging::AvailabilityChunkShufflingParams> { - self.availability_chunk_shuffling_params.get(key).map(|v| &*v) + pub(crate) fn client_features(&mut self, key: &Hash) -> Option { + self.client_features.get(key).copied() } - pub(crate) fn cache_availability_chunk_shuffling_params( - &mut self, - key: Hash, - params: vstaging::AvailabilityChunkShufflingParams, - ) { - self.availability_chunk_shuffling_params.insert(key, params); + pub(crate) fn cache_client_features(&mut self, key: Hash, features: vstaging::ClientFeatures) { + self.client_features.insert(key, features); } pub(crate) fn staging_para_backing_state( @@ -541,7 +534,7 @@ pub(crate) enum RequestResult { vstaging::slashing::OpaqueKeyOwnershipProof, Option<()>, ), - AvailabilityChunkShufflingParams(Hash, vstaging::AvailabilityChunkShufflingParams), + ClientFeatures(Hash, vstaging::ClientFeatures), StagingParaBackingState(Hash, ParaId, Option), StagingAsyncBackingParams(Hash, vstaging::AsyncBackingParams), diff --git a/polkadot/node/core/runtime-api/src/lib.rs b/polkadot/node/core/runtime-api/src/lib.rs index 07e2c8b3ad9c..ba4525abfb06 100644 --- a/polkadot/node/core/runtime-api/src/lib.rs +++ b/polkadot/node/core/runtime-api/src/lib.rs @@ -166,9 +166,8 @@ where .requests_cache .cache_key_ownership_proof((relay_parent, validator_id), key_ownership_proof), SubmitReportDisputeLost(_, _, _, _) => {}, - AvailabilityChunkShufflingParams(relay_parent, params) => self - .requests_cache - .cache_availability_chunk_shuffling_params(relay_parent, params), + ClientFeatures(relay_parent, params) => + self.requests_cache.cache_client_features(relay_parent, params), StagingParaBackingState(relay_parent, para_id, constraints) => self .requests_cache @@ -316,9 +315,8 @@ where Some(Request::MinimumBackingVotes(index, sender)) } }, - Request::AvailabilityChunkShufflingParams(sender) => - query!(availability_chunk_shuffling_params(), sender) - .map(|sender| Request::AvailabilityChunkShufflingParams(sender)), + Request::ClientFeatures(sender) => + query!(client_features(), sender).map(|sender| Request::ClientFeatures(sender)), } } @@ -575,10 +573,10 @@ where ver = Request::MINIMUM_BACKING_VOTES_RUNTIME_REQUIREMENT, sender ), - Request::AvailabilityChunkShufflingParams(sender) => query!( - AvailabilityChunkShufflingParams, - availability_chunk_shuffling_params(), - ver = Request::AVAILABILITY_CHUNK_SHUFFLING_PARAMS_RUNTIME_REQUIREMENT, + Request::ClientFeatures(sender) => query!( + ClientFeatures, + client_features(), + ver = Request::CLIENT_FEATURES_RUNTIME_REQUIREMENT, sender ), diff --git a/polkadot/node/core/runtime-api/src/tests.rs b/polkadot/node/core/runtime-api/src/tests.rs index 3785e7b1e027..af975235dde7 100644 --- a/polkadot/node/core/runtime-api/src/tests.rs +++ b/polkadot/node/core/runtime-api/src/tests.rs @@ -20,7 +20,7 @@ use polkadot_node_primitives::{BabeAllowedSlots, BabeEpoch, BabeEpochConfigurati use polkadot_node_subsystem::SpawnGlue; use polkadot_node_subsystem_test_helpers::make_subsystem_context; use polkadot_primitives::{ - vstaging::{self, AvailabilityChunkShufflingParams}, + vstaging::{self, ClientFeatures}, AuthorityDiscoveryId, BlockNumber, CandidateCommitments, CandidateEvent, CandidateHash, CommittedCandidateReceipt, CoreState, DisputeState, ExecutorParams, GroupRotationInfo, Id as ParaId, InboundDownwardMessage, InboundHrmpMessage, OccupiedCoreAssumption, @@ -269,10 +269,7 @@ impl RuntimeApiSubsystemClient for MockSubsystemClient { todo!("Not required for tests") } - async fn availability_chunk_shuffling_params( - &self, - _: Hash, - ) -> Result { + async fn client_features(&self, _: Hash) -> Result { todo!("Not required for tests") } } diff --git a/polkadot/node/network/availability-distribution/src/requester/mod.rs b/polkadot/node/network/availability-distribution/src/requester/mod.rs index 4c94dd59020d..aab39c74d262 100644 --- a/polkadot/node/network/availability-distribution/src/requester/mod.rs +++ b/polkadot/node/network/availability-distribution/src/requester/mod.rs @@ -36,7 +36,7 @@ use polkadot_node_subsystem::{ }; use polkadot_node_subsystem_util::{ availability_chunk_indices, - runtime::{get_occupied_cores, request_availability_chunk_shuffling_params, RuntimeInfo}, + runtime::{get_occupied_cores, request_client_features, RuntimeInfo}, }; use polkadot_primitives::{ BlockNumber, CandidateHash, ChunkIndex, Hash, OccupiedCore, SessionIndex, @@ -266,15 +266,14 @@ impl Requester { }); if self.chunk_index_cache.peek(&block_number).is_none() { - let maybe_av_chunk_shuffling_params = - request_availability_chunk_shuffling_params( - core.candidate_descriptor.relay_parent, - context.sender(), - ) - .await?; + let maybe_client_features = request_client_features( + core.candidate_descriptor.relay_parent, + context.sender(), + ) + .await?; let chunk_indices = availability_chunk_indices( - maybe_av_chunk_shuffling_params, + maybe_client_features, block_number, n_validators, ); diff --git a/polkadot/node/network/availability-recovery/src/error.rs b/polkadot/node/network/availability-recovery/src/error.rs index 82757cb21c4b..b51b0c13b538 100644 --- a/polkadot/node/network/availability-recovery/src/error.rs +++ b/polkadot/node/network/availability-recovery/src/error.rs @@ -33,8 +33,8 @@ pub enum Error { #[error("failed to query session info")] CanceledSessionInfo(#[source] oneshot::Canceled), - #[error("failed to query availability chunks shuffling params")] - RequestAvailabilityChunkShufflingParams(#[source] polkadot_node_subsystem_util::runtime::Error), + #[error("failed to query client features from runtime")] + RequestClientFeatures(#[source] polkadot_node_subsystem_util::runtime::Error), #[error("failed to send response")] CanceledResponseSender, diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index 600ade55802d..457b22f238d9 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -60,8 +60,7 @@ use polkadot_node_subsystem::{ SubsystemContext, SubsystemError, SubsystemResult, }; use polkadot_node_subsystem_util::{ - availability_chunk_indices, request_session_info, - runtime::request_availability_chunk_shuffling_params, + availability_chunk_indices, request_session_info, runtime::request_client_features, }; use polkadot_primitives::{ BlakeTwo256, BlockNumber, CandidateHash, CandidateReceipt, ChunkIndex, GroupIndex, Hash, HashT, @@ -444,15 +443,13 @@ async fn handle_recover( get_block_number(ctx.sender(), receipt.descriptor.relay_parent).await?; if state.chunk_indices_cache.peek(&block_number).is_none() { - let maybe_av_chunk_shuffling_params = request_availability_chunk_shuffling_params( - receipt.descriptor.relay_parent, - ctx.sender(), - ) - .await - .map_err(error::Error::RequestAvailabilityChunkShufflingParams)?; + let maybe_client_features = + request_client_features(receipt.descriptor.relay_parent, ctx.sender()) + .await + .map_err(error::Error::RequestClientFeatures)?; let chunk_indices = availability_chunk_indices( - maybe_av_chunk_shuffling_params, + maybe_client_features, block_number, session_info.validators.len(), ) diff --git a/polkadot/node/network/availability-recovery/src/tests.rs b/polkadot/node/network/availability-recovery/src/tests.rs index 2df2d132a090..0876f250ecf1 100644 --- a/polkadot/node/network/availability-recovery/src/tests.rs +++ b/polkadot/node/network/availability-recovery/src/tests.rs @@ -40,7 +40,7 @@ use polkadot_node_subsystem_test_helpers::{ }; use polkadot_node_subsystem_util::TimeoutExt; use polkadot_primitives::{ - vstaging::AvailabilityChunkShufflingParams, AuthorityDiscoveryId, Hash, HeadData, IndexedVec, + vstaging::ClientFeatures, AuthorityDiscoveryId, Hash, HeadData, IndexedVec, PersistedValidationData, ValidatorId, }; use polkadot_primitives_test_helpers::{dummy_candidate_receipt, dummy_hash}; @@ -213,19 +213,17 @@ impl TestState { ); } - async fn test_runtime_api_chunk_shuffling(&self, virtual_overseer: &mut VirtualOverseer) { + async fn test_runtime_api_client_features(&self, virtual_overseer: &mut VirtualOverseer) { assert_matches!( overseer_recv(virtual_overseer).await, AllMessages::RuntimeApi(RuntimeApiMessage::Request( _relay_parent, - RuntimeApiRequest::AvailabilityChunkShufflingParams( + RuntimeApiRequest::ClientFeatures( tx, ) )) => { tx.send(Ok( - AvailabilityChunkShufflingParams { - activate_at: None - } + ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING )).unwrap(); } ); @@ -553,7 +551,7 @@ fn availability_is_recovered_from_chunks_if_no_group_provided() { test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; + test_state.test_runtime_api_client_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -642,7 +640,7 @@ fn availability_is_recovered_from_chunks_even_if_backing_group_supplied_if_chunk test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; + test_state.test_runtime_api_client_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -731,7 +729,7 @@ fn bad_merkle_path_leads_to_recovery_error() { test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; + test_state.test_runtime_api_client_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -792,7 +790,7 @@ fn wrong_chunk_index_leads_to_recovery_error() { test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; + test_state.test_runtime_api_client_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -869,7 +867,7 @@ fn invalid_erasure_coding_leads_to_invalid_error() { test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; + test_state.test_runtime_api_client_features(&mut virtual_overseer).await; test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; @@ -920,7 +918,7 @@ fn fast_path_backing_group_recovers() { test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; + test_state.test_runtime_api_client_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -974,7 +972,7 @@ fn recovers_from_only_chunks_if_pov_large() { test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; + test_state.test_runtime_api_client_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -1083,7 +1081,7 @@ fn fast_path_backing_group_recovers_if_pov_small() { test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; + test_state.test_runtime_api_client_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -1144,7 +1142,7 @@ fn no_answers_in_fast_path_causes_chunk_requests() { test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; + test_state.test_runtime_api_client_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -1208,7 +1206,7 @@ fn task_canceled_when_receivers_dropped() { test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; + test_state.test_runtime_api_client_features(&mut virtual_overseer).await; for _ in 0..test_state.validators.len() { match virtual_overseer.recv().timeout(TIMEOUT).await { @@ -1252,7 +1250,7 @@ fn chunks_retry_until_all_nodes_respond() { test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; + test_state.test_runtime_api_client_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -1315,7 +1313,7 @@ fn not_returning_requests_wont_stall_retrieval() { test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; + test_state.test_runtime_api_client_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -1389,7 +1387,7 @@ fn all_not_returning_requests_still_recovers_on_return() { test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; + test_state.test_runtime_api_client_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -1468,7 +1466,7 @@ fn returns_early_if_we_have_the_data() { test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; + test_state.test_runtime_api_client_features(&mut virtual_overseer).await; test_state.respond_to_available_data_query(&mut virtual_overseer, true).await; assert_eq!(rx.await.unwrap().unwrap(), test_state.available_data); @@ -1507,7 +1505,7 @@ fn does_not_query_local_validator() { test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; + test_state.test_runtime_api_client_features(&mut virtual_overseer).await; test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |i| i == 0).await; @@ -1568,7 +1566,7 @@ fn invalid_local_chunk_is_ignored() { test_state.test_runtime_api_session_info(&mut virtual_overseer).await; test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - test_state.test_runtime_api_chunk_shuffling(&mut virtual_overseer).await; + test_state.test_runtime_api_client_features(&mut virtual_overseer).await; test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state .respond_to_query_all_request_invalid(&mut virtual_overseer, |i| i == 0) diff --git a/polkadot/node/subsystem-types/src/messages.rs b/polkadot/node/subsystem-types/src/messages.rs index 9c1b73cac20f..61c87772b086 100644 --- a/polkadot/node/subsystem-types/src/messages.rs +++ b/polkadot/node/subsystem-types/src/messages.rs @@ -40,7 +40,7 @@ use polkadot_node_primitives::{ }; use polkadot_primitives::{ slashing, - vstaging::{self as vstaging_primitives, AvailabilityChunkShufflingParams}, + vstaging::{self as vstaging_primitives, ClientFeatures}, AuthorityDiscoveryId, BackedCandidate, BlockNumber, CandidateEvent, CandidateHash, CandidateIndex, CandidateReceipt, CollatorId, CommittedCandidateReceipt, CoreState, DisputeState, ExecutorParams, GroupIndex, GroupRotationInfo, Hash, Header as BlockHeader, @@ -696,8 +696,8 @@ pub enum RuntimeApiRequest { ), /// Get the minimum required backing votes. MinimumBackingVotes(SessionIndex, RuntimeApiSender), - /// Get the params for availability chunk shuffling. - AvailabilityChunkShufflingParams(RuntimeApiSender), + /// Get the client features. + ClientFeatures(RuntimeApiSender), /// Get the backing state of the given para. /// This is a staging API that will not be available on production runtimes. @@ -729,8 +729,8 @@ impl RuntimeApiRequest { /// `MinimumBackingVotes` pub const MINIMUM_BACKING_VOTES_RUNTIME_REQUIREMENT: u32 = 6; - /// `AvailabilityChunkShufflingParams` - pub const AVAILABILITY_CHUNK_SHUFFLING_PARAMS_RUNTIME_REQUIREMENT: u32 = 7; + /// `Client features` + pub const CLIENT_FEATURES_RUNTIME_REQUIREMENT: u32 = 7; /// Minimum version for backing state, required for async backing. /// diff --git a/polkadot/node/subsystem-types/src/runtime_client.rs b/polkadot/node/subsystem-types/src/runtime_client.rs index 3ff4d9488c86..ca90d0d7b62a 100644 --- a/polkadot/node/subsystem-types/src/runtime_client.rs +++ b/polkadot/node/subsystem-types/src/runtime_client.rs @@ -16,13 +16,12 @@ use async_trait::async_trait; use polkadot_primitives::{ - runtime_api::ParachainHost, - vstaging::{self, AvailabilityChunkShufflingParams}, - Block, BlockNumber, CandidateCommitments, CandidateEvent, CandidateHash, - CommittedCandidateReceipt, CoreState, DisputeState, ExecutorParams, GroupRotationInfo, Hash, - Id, InboundDownwardMessage, InboundHrmpMessage, OccupiedCoreAssumption, - PersistedValidationData, PvfCheckStatement, ScrapedOnChainVotes, SessionIndex, SessionInfo, - ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, ValidatorSignature, + runtime_api::ParachainHost, vstaging, Block, BlockNumber, CandidateCommitments, CandidateEvent, + CandidateHash, CommittedCandidateReceipt, CoreState, DisputeState, ExecutorParams, + GroupRotationInfo, Hash, Id, InboundDownwardMessage, InboundHrmpMessage, + OccupiedCoreAssumption, PersistedValidationData, PvfCheckStatement, ScrapedOnChainVotes, + SessionIndex, SessionInfo, ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, + ValidatorSignature, }; use sc_transaction_pool_api::OffchainTransactionPoolFactory; use sp_api::{ApiError, ApiExt, ProvideRuntimeApi}; @@ -241,11 +240,8 @@ pub trait RuntimeApiSubsystemClient { session_index: SessionIndex, ) -> Result; - /// Get the parameters for availability-chunk shuffling. - async fn availability_chunk_shuffling_params( - &self, - at: Hash, - ) -> Result; + /// Get the client features. + async fn client_features(&self, at: Hash) -> Result; // === Asynchronous backing API === @@ -496,11 +492,8 @@ where self.client.runtime_api().minimum_backing_votes(at) } - async fn availability_chunk_shuffling_params( - &self, - at: Hash, - ) -> Result { - self.client.runtime_api().availability_chunk_shuffling_params(at) + async fn client_features(&self, at: Hash) -> Result { + self.client.runtime_api().client_features(at) } async fn staging_para_backing_state( diff --git a/polkadot/node/subsystem-util/src/lib.rs b/polkadot/node/subsystem-util/src/lib.rs index 745455a77955..6acc1b39ce9d 100644 --- a/polkadot/node/subsystem-util/src/lib.rs +++ b/polkadot/node/subsystem-util/src/lib.rs @@ -34,9 +34,7 @@ use polkadot_node_subsystem::{ messages::{RuntimeApiMessage, RuntimeApiRequest, RuntimeApiSender}, overseer, SubsystemSender, }; -use polkadot_primitives::{ - slashing, vstaging::AvailabilityChunkShufflingParams, BlockNumber, ExecutorParams, -}; +use polkadot_primitives::{slashing, vstaging::ClientFeatures, BlockNumber, ExecutorParams}; use rand::seq::SliceRandom; use rand_chacha::ChaCha8Rng; @@ -449,7 +447,7 @@ impl Validator { /// validators have a common view of the shuffle at a given block height. /// Otherwise, return the identity vector. pub fn availability_chunk_indices( - maybe_params: Option, + maybe_client_features: Option, block_number: BlockNumber, n_validators: usize, ) -> Vec { @@ -457,10 +455,8 @@ pub fn availability_chunk_indices( .map(|i| ValidatorIndex(u32::try_from(i).expect("validator numbers should not exceed u32"))) .collect(); - if let Some(AvailabilityChunkShufflingParams { activate_at: Some(activation_block_number) }) = - maybe_params - { - if block_number >= activation_block_number { + if let Some(features) = maybe_client_features { + if features.contains(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING) { let seed = block_number.to_be_bytes(); let mut rng: ChaCha8Rng = SeedableRng::from_seed( seed.repeat(8) diff --git a/polkadot/node/subsystem-util/src/runtime/mod.rs b/polkadot/node/subsystem-util/src/runtime/mod.rs index ca269ae7f087..bcf3ce895f88 100644 --- a/polkadot/node/subsystem-util/src/runtime/mod.rs +++ b/polkadot/node/subsystem-util/src/runtime/mod.rs @@ -30,7 +30,7 @@ use polkadot_node_subsystem::{ }; use polkadot_node_subsystem_types::UnpinHandle; use polkadot_primitives::{ - vstaging::{self, AvailabilityChunkShufflingParams}, + vstaging::{self, ClientFeatures}, CandidateEvent, CandidateHash, CoreState, EncodeAs, ExecutorParams, GroupIndex, GroupRotationInfo, Hash, IndexedVec, OccupiedCore, ScrapedOnChainVotes, SessionIndex, SessionInfo, Signed, SigningContext, UncheckedSigned, ValidationCode, ValidationCodeHash, @@ -512,15 +512,12 @@ pub async fn request_min_backing_votes( /// Request the availability chunk shuffling params. /// Prior to runtime API version 7, just return `None`. -pub async fn request_availability_chunk_shuffling_params( +pub async fn request_client_features( parent: Hash, sender: &mut impl overseer::SubsystemSender, -) -> Result> { +) -> Result> { let res = recv_runtime( - request_from_runtime(parent, sender, |tx| { - RuntimeApiRequest::AvailabilityChunkShufflingParams(tx) - }) - .await, + request_from_runtime(parent, sender, |tx| RuntimeApiRequest::ClientFeatures(tx)).await, ) .await; @@ -528,7 +525,7 @@ pub async fn request_availability_chunk_shuffling_params( gum::trace!( target: LOG_TARGET, ?parent, - "Querying the availability chunk shuffling params from the runtime is not supported by the current Runtime API", + "Querying the client features from the runtime is not supported by the current Runtime API", ); Ok(None) diff --git a/polkadot/primitives/Cargo.toml b/polkadot/primitives/Cargo.toml index 9d17b70b8174..20b7011acbb9 100644 --- a/polkadot/primitives/Cargo.toml +++ b/polkadot/primitives/Cargo.toml @@ -6,6 +6,7 @@ edition.workspace = true license.workspace = true [dependencies] +bitflags = "1.3.2" bitvec = { version = "1.0.0", default-features = false, features = ["alloc"] } hex-literal = "0.4.1" parity-scale-codec = { version = "3.6.1", default-features = false, features = ["bit-vec", "derive"] } diff --git a/polkadot/primitives/src/runtime_api.rs b/polkadot/primitives/src/runtime_api.rs index 7f6c023c5335..c167b9b35404 100644 --- a/polkadot/primitives/src/runtime_api.rs +++ b/polkadot/primitives/src/runtime_api.rs @@ -250,7 +250,7 @@ sp_api::decl_runtime_apis! { /// Get the availability-chunk shuffling params. /// This is a staging method! Do not use on production runtimes! #[api_version(7)] - fn availability_chunk_shuffling_params() -> vstaging::AvailabilityChunkShufflingParams; + fn client_features() -> vstaging::ClientFeatures; /***** Asynchronous backing *****/ diff --git a/polkadot/primitives/src/vstaging/mod.rs b/polkadot/primitives/src/vstaging/mod.rs index 3852f331b99b..c427649dad12 100644 --- a/polkadot/primitives/src/vstaging/mod.rs +++ b/polkadot/primitives/src/vstaging/mod.rs @@ -129,19 +129,11 @@ pub struct BackingState { pub pending_availability: Vec>, } -/// Parameters for availability chunk shuffling. -/// Previously, the ChunkIndex assigned to a validator was equal to its ValidatorIndex. -/// -/// This enables us to shuffle the chunk indices, per-block. -#[derive( - RuntimeDebug, Clone, PartialEq, Encode, Decode, TypeInfo, Default, Serialize, Deserialize, -)] -pub struct AvailabilityChunkShufflingParams { - /// If it's `None`, it's not enabled. If it's some block number, consider it's enabled at this - /// specific block height. - /// - /// It can't simply return a bool as this may break availability recovery. If a - /// runtime upgrade were to set this value to true, parachain candidates that were backed in - /// previous blocks couldn't be recovered, because we'd use the new, breaking, shuffle. - pub activate_at: Option, +bitflags::bitflags! { + #[derive(Default, TypeInfo, Encode, Decode, Serialize, Deserialize)] + /// Bit indices in the `HostCoonfiguration.client_features` that correspond to different client features. + pub struct ClientFeatures: u8 { + /// Is availability chunk shuffling enabled. + const AVAILABILITY_CHUNK_SHUFFLING = 0b0001; + } } diff --git a/polkadot/runtime/parachains/src/configuration.rs b/polkadot/runtime/parachains/src/configuration.rs index 2482560b87e7..92ceb2b7ce33 100644 --- a/polkadot/runtime/parachains/src/configuration.rs +++ b/polkadot/runtime/parachains/src/configuration.rs @@ -26,7 +26,7 @@ use polkadot_parachain_primitives::primitives::{ MAX_HORIZONTAL_MESSAGE_NUM, MAX_UPWARD_MESSAGE_NUM, }; use primitives::{ - vstaging::{AsyncBackingParams, AvailabilityChunkShufflingParams}, + vstaging::{AsyncBackingParams, ClientFeatures}, Balance, ExecutorParams, SessionIndex, LEGACY_MIN_BACKING_VOTES, MAX_CODE_SIZE, MAX_HEAD_DATA_SIZE, MAX_POV_SIZE, ON_DEMAND_DEFAULT_QUEUE_MAX_SIZE, }; @@ -260,8 +260,8 @@ pub struct HostConfiguration { /// The minimum number of valid backing statements required to consider a parachain candidate /// backable. pub minimum_backing_votes: u32, - /// Parameters for availability chunk shuffling. - pub availability_chunk_shuffling_params: AvailabilityChunkShufflingParams, + /// Client features enablement. + pub client_features: ClientFeatures, } impl> Default for HostConfiguration { @@ -313,7 +313,7 @@ impl> Default for HostConfiguration, - new: AvailabilityChunkShufflingParams, + enable: bool, ) -> DispatchResult { ensure_root(origin)?; Self::schedule_config_update(|config| { - config.availability_chunk_shuffling_params = new; + config.client_features.set(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING, enable) }) } } diff --git a/polkadot/runtime/parachains/src/configuration/migration/v10.rs b/polkadot/runtime/parachains/src/configuration/migration/v10.rs index 1ec4b16da2a1..671807695ae6 100644 --- a/polkadot/runtime/parachains/src/configuration/migration/v10.rs +++ b/polkadot/runtime/parachains/src/configuration/migration/v10.rs @@ -23,7 +23,7 @@ use frame_support::{ weights::Weight, }; use frame_system::pallet_prelude::BlockNumberFor; -use primitives::{SessionIndex, LEGACY_MIN_BACKING_VOTES}; +use primitives::{vstaging::ClientFeatures, SessionIndex, LEGACY_MIN_BACKING_VOTES}; use sp_runtime::Perbill; use sp_std::vec::Vec; @@ -152,7 +152,7 @@ on_demand_fee_variability : Perbill::from_percent(3), on_demand_target_queue_utilization : Perbill::from_percent(25), on_demand_ttl : 5u32.into(), minimum_backing_votes : LEGACY_MIN_BACKING_VOTES, -availability_chunk_shuffling_params : Default::default() +client_features : ClientFeatures::empty() } }; @@ -178,8 +178,6 @@ availability_chunk_shuffling_params : Default::default() #[cfg(test)] mod tests { - use primitives::vstaging::AvailabilityChunkShufflingParams; - use super::*; use crate::mock::{new_test_ext, Test}; @@ -206,7 +204,7 @@ mod tests { // doesn't need to be read and also leaving it as one line allows to easily copy it. let raw_config = hex_literal::hex![" - 0000300000800000080000000000100000c8000005000000050000000200000002000000000000000000000000005000000010000400000000000000000000000000000000000000000000000000000000000000000000000800000000200000040000000000100000b004000000000000000000001027000080b2e60e80c3c901809698000000000000000000000000000500000014000000040000000100000001050000000006000000640000000200000019000000000000000a000000020000000200000005000000020000000164000000" + 0000300000800000080000000000100000c8000005000000050000000200000002000000000000000000000000005000000010000400000000000000000000000000000000000000000000000000000000000000000000000800000000200000040000000000100000b004000000000000000000001027000080b2e60e80c3c901809698000000000000000000000000000500000014000000040000000100000001050000000006000000640000000200000019000000000000000a0000000200000002000000050000000200000001" ]; let v10 = @@ -225,8 +223,8 @@ mod tests { assert_eq!(v10.on_demand_base_fee, 10_000_000); assert_eq!(v10.minimum_backing_votes, LEGACY_MIN_BACKING_VOTES); assert_eq!( - v10.availability_chunk_shuffling_params, - AvailabilityChunkShufflingParams { activate_at: Some(100) } + v10.client_features, + ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING ); } @@ -305,6 +303,7 @@ mod tests { assert_eq!(v9.async_backing_params.max_candidate_depth , v10.async_backing_params.max_candidate_depth); assert_eq!(v9.executor_params , v10.executor_params); assert_eq!(v9.minimum_backing_votes , v10.minimum_backing_votes); + assert_eq!(v9.client_features , v10.client_features); }; // ; makes this a statement. `rustfmt::skip` cannot be put on an expression. } }); diff --git a/polkadot/runtime/parachains/src/configuration/tests.rs b/polkadot/runtime/parachains/src/configuration/tests.rs index 0e9aa999d898..becffd890b25 100644 --- a/polkadot/runtime/parachains/src/configuration/tests.rs +++ b/polkadot/runtime/parachains/src/configuration/tests.rs @@ -318,9 +318,7 @@ fn setting_pending_config_members() { on_demand_target_queue_utilization: Perbill::from_percent(25), on_demand_ttl: 5u32, minimum_backing_votes: 5, - availability_chunk_shuffling_params: AvailabilityChunkShufflingParams { - activate_at: Some(100), - }, + client_features: ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING, }; Configuration::set_validation_upgrade_cooldown( @@ -476,9 +474,11 @@ fn setting_pending_config_members() { new_config.minimum_backing_votes, ) .unwrap(); - Configuration::set_availability_chunk_shuffling_params( + Configuration::set_availability_chunk_shuffling_client_feature( RuntimeOrigin::root(), - new_config.availability_chunk_shuffling_params.clone(), + new_config + .client_features + .contains(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING), ) .unwrap(); diff --git a/polkadot/runtime/parachains/src/runtime_api_impl/vstaging.rs b/polkadot/runtime/parachains/src/runtime_api_impl/vstaging.rs index 4a03885ee458..02be0b126bed 100644 --- a/polkadot/runtime/parachains/src/runtime_api_impl/vstaging.rs +++ b/polkadot/runtime/parachains/src/runtime_api_impl/vstaging.rs @@ -20,9 +20,8 @@ use crate::{configuration, dmp, hrmp, inclusion, initializer, paras, shared}; use frame_system::pallet_prelude::BlockNumberFor; use primitives::{ vstaging::{ - AsyncBackingParams, AvailabilityChunkShufflingParams, BackingState, - CandidatePendingAvailability, Constraints, InboundHrmpLimitations, - OutboundHrmpChannelLimitations, + AsyncBackingParams, BackingState, CandidatePendingAvailability, ClientFeatures, + Constraints, InboundHrmpLimitations, OutboundHrmpChannelLimitations, }, Id as ParaId, }; @@ -125,8 +124,7 @@ pub fn minimum_backing_votes() -> u32 { >::config().minimum_backing_votes } -/// Return whether the shuffling of availability chunk indices is enabled. -pub fn availability_chunk_shuffling_params( -) -> AvailabilityChunkShufflingParams { - >::config().availability_chunk_shuffling_params +/// Returns the current state of the client features. +pub fn client_features() -> ClientFeatures { + >::config().client_features } diff --git a/polkadot/runtime/rococo/src/lib.rs b/polkadot/runtime/rococo/src/lib.rs index 236b44f9b126..67772480789d 100644 --- a/polkadot/runtime/rococo/src/lib.rs +++ b/polkadot/runtime/rococo/src/lib.rs @@ -23,8 +23,8 @@ use pallet_nis::WithMaximumOf; use parity_scale_codec::{Decode, Encode, MaxEncodedLen}; use primitives::{ - slashing, vstaging::AvailabilityChunkShufflingParams, AccountId, AccountIndex, Balance, - BlockNumber, CandidateEvent, CandidateHash, CommittedCandidateReceipt, CoreState, DisputeState, + slashing, vstaging::ClientFeatures, AccountId, AccountIndex, Balance, BlockNumber, + CandidateEvent, CandidateHash, CommittedCandidateReceipt, CoreState, DisputeState, ExecutorParams, GroupRotationInfo, Hash, Id as ParaId, InboundDownwardMessage, InboundHrmpMessage, Moment, Nonce, OccupiedCoreAssumption, PersistedValidationData, ScrapedOnChainVotes, SessionInfo, Signature, ValidationCode, ValidationCodeHash, ValidatorId, @@ -1862,8 +1862,8 @@ sp_api::impl_runtime_apis! { parachains_staging_runtime_api_impl::minimum_backing_votes::() } - fn availability_chunk_shuffling_params() -> AvailabilityChunkShufflingParams { - parachains_staging_runtime_api_impl::availability_chunk_shuffling_params::() + fn client_features() -> ClientFeatures { + parachains_staging_runtime_api_impl::client_features::() } } From d57396b924432fd0aed565fc855958fdfb12096f Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 22 Sep 2023 15:23:07 +0300 Subject: [PATCH 030/126] rustfmt --- .../runtime/parachains/src/configuration/migration/v10.rs | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/polkadot/runtime/parachains/src/configuration/migration/v10.rs b/polkadot/runtime/parachains/src/configuration/migration/v10.rs index 671807695ae6..66c65925b32c 100644 --- a/polkadot/runtime/parachains/src/configuration/migration/v10.rs +++ b/polkadot/runtime/parachains/src/configuration/migration/v10.rs @@ -222,10 +222,7 @@ mod tests { assert_eq!(v10.on_demand_cores, 0); assert_eq!(v10.on_demand_base_fee, 10_000_000); assert_eq!(v10.minimum_backing_votes, LEGACY_MIN_BACKING_VOTES); - assert_eq!( - v10.client_features, - ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING - ); + assert_eq!(v10.client_features, ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING); } #[test] From 0a46a21194ab511833108a2c0e6a20a0bff42c18 Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 22 Sep 2023 15:57:41 +0300 Subject: [PATCH 031/126] fix test --- polkadot/node/subsystem-util/src/tests.rs | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/polkadot/node/subsystem-util/src/tests.rs b/polkadot/node/subsystem-util/src/tests.rs index 2c0fe937dbe2..d49762211bc0 100644 --- a/polkadot/node/subsystem-util/src/tests.rs +++ b/polkadot/node/subsystem-util/src/tests.rs @@ -97,13 +97,21 @@ fn subset_predefined_generation_check() { } #[test] -// Just a simple test to make sure that the shuffling is indeed a permutation, to prevent future -// mistakes. fn test_availability_chunk_indices() { let block_number = 89; let n_validators = 200; - let shuffle = availability_chunk_indices(block_number, n_validators); + let shuffle = availability_chunk_indices(None, block_number, n_validators); + assert_eq!(shuffle, (0..n_validators).map(|i| ValidatorIndex(i as u32)).collect::>()); + + let shuffle = availability_chunk_indices(Some(ClientFeatures::empty()), block_number, n_validators); + assert_eq!(shuffle, (0..n_validators).map(|i| ValidatorIndex(i as u32)).collect::>()); + + let shuffle = availability_chunk_indices(ClientFeatures::from_bits(0b10), block_number, n_validators); + assert_eq!(shuffle, (0..n_validators).map(|i| ValidatorIndex(i as u32)).collect::>()); + + let shuffle = availability_chunk_indices(Some(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING), block_number, n_validators); + assert_ne!(shuffle, (0..n_validators).map(|i| ValidatorIndex(i as u32)).collect::>()); assert_eq!(shuffle.len(), n_validators); assert_eq!(shuffle.iter().collect::>().len(), n_validators); } From e2ce0cdb776a6ed1f29fbebaaa0b141a3f93f70f Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 22 Sep 2023 16:38:28 +0300 Subject: [PATCH 032/126] add client_features to westend, fix fmt and clippy --- .../emulated/common/src/lib.rs | 4 ++-- polkadot/node/subsystem-util/src/tests.rs | 12 +++++++++--- polkadot/runtime/westend/src/lib.rs | 18 +++++++++++------- 3 files changed, 22 insertions(+), 12 deletions(-) diff --git a/cumulus/parachains/integration-tests/emulated/common/src/lib.rs b/cumulus/parachains/integration-tests/emulated/common/src/lib.rs index 7461165f2a19..330e1a213711 100644 --- a/cumulus/parachains/integration-tests/emulated/common/src/lib.rs +++ b/cumulus/parachains/integration-tests/emulated/common/src/lib.rs @@ -64,7 +64,7 @@ decl_test_relay_chains! { Hrmp: kusama_runtime::Hrmp, } }, - #[api_version(6)] + #[api_version(7)] pub struct Westend { genesis = westend::genesis(), on_init = (), @@ -79,7 +79,7 @@ decl_test_relay_chains! { Balances: westend_runtime::Balances, } }, - #[api_version(5)] + #[api_version(7)] pub struct Rococo { genesis = rococo::genesis(), on_init = (), diff --git a/polkadot/node/subsystem-util/src/tests.rs b/polkadot/node/subsystem-util/src/tests.rs index d49762211bc0..a22cfed82996 100644 --- a/polkadot/node/subsystem-util/src/tests.rs +++ b/polkadot/node/subsystem-util/src/tests.rs @@ -104,13 +104,19 @@ fn test_availability_chunk_indices() { let shuffle = availability_chunk_indices(None, block_number, n_validators); assert_eq!(shuffle, (0..n_validators).map(|i| ValidatorIndex(i as u32)).collect::>()); - let shuffle = availability_chunk_indices(Some(ClientFeatures::empty()), block_number, n_validators); + let shuffle = + availability_chunk_indices(Some(ClientFeatures::empty()), block_number, n_validators); assert_eq!(shuffle, (0..n_validators).map(|i| ValidatorIndex(i as u32)).collect::>()); - let shuffle = availability_chunk_indices(ClientFeatures::from_bits(0b10), block_number, n_validators); + let shuffle = + availability_chunk_indices(ClientFeatures::from_bits(0b10), block_number, n_validators); assert_eq!(shuffle, (0..n_validators).map(|i| ValidatorIndex(i as u32)).collect::>()); - let shuffle = availability_chunk_indices(Some(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING), block_number, n_validators); + let shuffle = availability_chunk_indices( + Some(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING), + block_number, + n_validators, + ); assert_ne!(shuffle, (0..n_validators).map(|i| ValidatorIndex(i as u32)).collect::>()); assert_eq!(shuffle.len(), n_validators); assert_eq!(shuffle.iter().collect::>().len(), n_validators); diff --git a/polkadot/runtime/westend/src/lib.rs b/polkadot/runtime/westend/src/lib.rs index 9af18b5be2bb..f17e4e08929d 100644 --- a/polkadot/runtime/westend/src/lib.rs +++ b/polkadot/runtime/westend/src/lib.rs @@ -43,12 +43,12 @@ use pallet_session::historical as session_historical; use pallet_transaction_payment::{CurrencyAdapter, FeeDetails, RuntimeDispatchInfo}; use parity_scale_codec::{Decode, Encode, MaxEncodedLen}; use primitives::{ - slashing, AccountId, AccountIndex, Balance, BlockNumber, CandidateEvent, CandidateHash, - CommittedCandidateReceipt, CoreState, DisputeState, ExecutorParams, GroupRotationInfo, Hash, - Id as ParaId, InboundDownwardMessage, InboundHrmpMessage, Moment, Nonce, - OccupiedCoreAssumption, PersistedValidationData, PvfCheckStatement, ScrapedOnChainVotes, - SessionInfo, Signature, ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, - ValidatorSignature, PARACHAIN_KEY_TYPE_ID, + slashing, vstaging::ClientFeatures, AccountId, AccountIndex, Balance, BlockNumber, + CandidateEvent, CandidateHash, CommittedCandidateReceipt, CoreState, DisputeState, + ExecutorParams, GroupRotationInfo, Hash, Id as ParaId, InboundDownwardMessage, + InboundHrmpMessage, Moment, Nonce, OccupiedCoreAssumption, PersistedValidationData, + PvfCheckStatement, ScrapedOnChainVotes, SessionInfo, Signature, ValidationCode, + ValidationCodeHash, ValidatorId, ValidatorIndex, ValidatorSignature, PARACHAIN_KEY_TYPE_ID, }; use runtime_common::{ assigned_slots, auctions, crowdloan, elections::OnChainAccuracy, impl_runtime_weights, @@ -1580,7 +1580,7 @@ sp_api::impl_runtime_apis! { } } - #[api_version(6)] + #[api_version(7)] impl primitives::runtime_api::ParachainHost for Runtime { fn validators() -> Vec { parachains_runtime_api_impl::validators::() @@ -1715,6 +1715,10 @@ sp_api::impl_runtime_apis! { fn minimum_backing_votes() -> u32 { parachains_staging_runtime_api_impl::minimum_backing_votes::() } + + fn client_features() -> ClientFeatures { + parachains_staging_runtime_api_impl::client_features::() + } } impl beefy_primitives::BeefyApi for Runtime { From cc06a71a28d79f145a2188a9dfada37af6ca4375 Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 22 Sep 2023 17:15:57 +0300 Subject: [PATCH 033/126] try fixing clippy again --- .../parachains/integration-tests/emulated/common/src/lib.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/cumulus/parachains/integration-tests/emulated/common/src/lib.rs b/cumulus/parachains/integration-tests/emulated/common/src/lib.rs index 330e1a213711..fb772f5f2352 100644 --- a/cumulus/parachains/integration-tests/emulated/common/src/lib.rs +++ b/cumulus/parachains/integration-tests/emulated/common/src/lib.rs @@ -23,6 +23,7 @@ use constants::{ bridge_hub_polkadot, bridge_hub_rococo, collectives, kusama, penpal, polkadot, rococo, westend, }; use impls::{RococoWococoMessageHandler, WococoRococoMessageHandler}; +use polkadot_primitives::runtime_api::runtime_decl_for_parachain_host::ParachainHostV7; // Substrate use frame_support::traits::OnInitialize; @@ -64,7 +65,7 @@ decl_test_relay_chains! { Hrmp: kusama_runtime::Hrmp, } }, - #[api_version(7)] + #[api_version(6)] pub struct Westend { genesis = westend::genesis(), on_init = (), @@ -79,7 +80,7 @@ decl_test_relay_chains! { Balances: westend_runtime::Balances, } }, - #[api_version(7)] + #[api_version(5)] pub struct Rococo { genesis = rococo::genesis(), on_init = (), From a83b178310a2400f3a6619ce0db83ad90815c271 Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 26 Sep 2023 17:05:44 +0300 Subject: [PATCH 034/126] fix copy-paste mistake --- .../parachains/src/configuration/migration/v10.rs | 12 ++++++------ .../parachains/src/configuration/migration/v9.rs | 10 +++++----- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/polkadot/runtime/parachains/src/configuration/migration/v10.rs b/polkadot/runtime/parachains/src/configuration/migration/v10.rs index 66c65925b32c..001075e8d5f5 100644 --- a/polkadot/runtime/parachains/src/configuration/migration/v10.rs +++ b/polkadot/runtime/parachains/src/configuration/migration/v10.rs @@ -146,12 +146,12 @@ pvf_voting_ttl : pre.pvf_voting_ttl, minimum_validation_upgrade_delay : pre.minimum_validation_upgrade_delay, async_backing_params : pre.async_backing_params, executor_params : pre.executor_params, -on_demand_queue_max_size : 10_000u32, -on_demand_base_fee : 10_000_000u128, -on_demand_fee_variability : Perbill::from_percent(3), -on_demand_target_queue_utilization : Perbill::from_percent(25), -on_demand_ttl : 5u32.into(), -minimum_backing_votes : LEGACY_MIN_BACKING_VOTES, +on_demand_queue_max_size : pre.on_demand_queue_max_size, +on_demand_base_fee : pre.on_demand_base_fee, +on_demand_fee_variability : pre.on_demand_fee_variability, +on_demand_target_queue_utilization : pre.on_demand_target_queue_utilization, +on_demand_ttl : pre.on_demand_ttl, +minimum_backing_votes : pre.minimum_backing_votes, client_features : ClientFeatures::empty() } }; diff --git a/polkadot/runtime/parachains/src/configuration/migration/v9.rs b/polkadot/runtime/parachains/src/configuration/migration/v9.rs index ec4672d905bc..05be791db3fb 100644 --- a/polkadot/runtime/parachains/src/configuration/migration/v9.rs +++ b/polkadot/runtime/parachains/src/configuration/migration/v9.rs @@ -247,11 +247,11 @@ pvf_voting_ttl : pre.pvf_voting_ttl, minimum_validation_upgrade_delay : pre.minimum_validation_upgrade_delay, async_backing_params : pre.async_backing_params, executor_params : pre.executor_params, -on_demand_queue_max_size : 10_000u32, -on_demand_base_fee : 10_000_000u128, -on_demand_fee_variability : Perbill::from_percent(3), -on_demand_target_queue_utilization : Perbill::from_percent(25), -on_demand_ttl : 5u32.into(), +on_demand_queue_max_size : pre.on_demand_queue_max_size, +on_demand_base_fee : pre.on_demand_base_fee, +on_demand_fee_variability : pre.on_demand_fee_variability, +on_demand_target_queue_utilization : pre.on_demand_target_queue_utilization, +on_demand_ttl : pre.on_demand_ttl, minimum_backing_votes : LEGACY_MIN_BACKING_VOTES } }; From 6e06d279f352292929acc7b6f710aec651223ba4 Mon Sep 17 00:00:00 2001 From: alindima Date: Wed, 27 Sep 2023 15:34:32 +0300 Subject: [PATCH 035/126] fix metrics and logs --- .../node/network/availability-recovery/src/metrics.rs | 10 +++++++--- .../node/network/availability-recovery/src/task.rs | 9 +++++---- 2 files changed, 12 insertions(+), 7 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/metrics.rs b/polkadot/node/network/availability-recovery/src/metrics.rs index 10683c49ef38..f5c3523b9870 100644 --- a/polkadot/node/network/availability-recovery/src/metrics.rs +++ b/polkadot/node/network/availability-recovery/src/metrics.rs @@ -81,7 +81,8 @@ struct MetricsInner { /// Number of full recoveries that have been finished one way or the other. /// - /// Split by recovery `strategy_type` (`full_from_backers, systematic_chunks, regular_chunks`) + /// Split by recovery `strategy_type` (`full_from_backers, systematic_chunks, regular_chunks, + /// all`). `all` is used for failed recoveries that tried all available strategies. /// Also split by `result` type. full_recoveries_finished: CounterVec, @@ -194,9 +195,12 @@ impl Metrics { } /// A full recovery failed (data not available). - pub fn on_recovery_failed(&self) { + pub fn on_recovery_failed(&self, strategy_type: &str) { if let Some(metrics) = &self.0 { - metrics.full_recoveries_finished.with_label_values(&["failure"]).inc() + metrics + .full_recoveries_finished + .with_label_values(&["failure", strategy_type]) + .inc() } } diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 90bfb76e2ed8..7bb4f61d4af6 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -147,7 +147,7 @@ impl State { gum::trace!( target: LOG_TARGET, candidate_hash = ?params.candidate_hash, - validator_index = ?chunk.index, + chunk_index = ?chunk.index, "Found valid chunk on disk" ); self.insert_chunk(chunk.index, chunk); @@ -459,7 +459,7 @@ where match &err { RecoveryError::Invalid => self.params.metrics.on_recovery_invalid(strategy_type), - _ => self.params.metrics.on_recovery_failed(), + _ => self.params.metrics.on_recovery_failed(strategy_type), } return Err(err) }, @@ -476,7 +476,8 @@ where candidate_hash = ?self.params.candidate_hash, "Recovery of available data failed.", ); - self.params.metrics.on_recovery_failed(); + + self.params.metrics.on_recovery_failed("all"); Err(RecoveryError::Unavailable) } @@ -666,7 +667,7 @@ impl FetchSystematicChunks { ValidatorIndex(0).. ValidatorIndex( u32::try_from(self.threshold) - .expect("validator numbers should not exceed u32"), + .expect("validator count should not exceed u32"), ), ) .map(|(_, chunk)| &chunk.chunk[..]) From ce10f68fac717567158faa029d30a694b6ed39c7 Mon Sep 17 00:00:00 2001 From: alindima Date: Wed, 27 Sep 2023 15:56:07 +0300 Subject: [PATCH 036/126] add new shuffling algorithm this enables better load distribution within a relay block --- .../node/core/bitfield-signing/src/lib.rs | 94 +++++--- .../src/requester/mod.rs | 47 ++-- .../src/requester/session_cache.rs | 11 +- .../network/availability-recovery/src/lib.rs | 92 ++++---- polkadot/node/primitives/src/lib.rs | 1 + polkadot/node/subsystem-util/src/lib.rs | 218 ++++++++++++++++-- .../src/configuration/migration/v10.rs | 4 +- 7 files changed, 331 insertions(+), 136 deletions(-) diff --git a/polkadot/node/core/bitfield-signing/src/lib.rs b/polkadot/node/core/bitfield-signing/src/lib.rs index e762eff06763..bd659cb67b8a 100644 --- a/polkadot/node/core/bitfield-signing/src/lib.rs +++ b/polkadot/node/core/bitfield-signing/src/lib.rs @@ -28,20 +28,21 @@ use futures::{ }; use polkadot_node_subsystem::{ jaeger, - messages::{ - AvailabilityStoreMessage, BitfieldDistributionMessage, ChainApiMessage, RuntimeApiMessage, - RuntimeApiRequest, - }, + messages::{AvailabilityStoreMessage, BitfieldDistributionMessage, ChainApiMessage}, overseer, ActivatedLeaf, ChainApiError, FromOrchestra, LeafStatus, OverseerSignal, PerLeafSpan, - SpawnedSubsystem, SubsystemError, SubsystemResult, SubsystemSender, + SpawnedSubsystem, SubsystemError, SubsystemResult, }; use polkadot_node_subsystem_util::{ - self as util, availability_chunk_indices, request_validators, runtime::request_client_features, + self as util, availability_chunk_index, request_validators, runtime::request_client_features, Validator, }; use polkadot_primitives::{AvailabilityBitfield, BlockNumber, CoreState, Hash, ValidatorIndex}; use sp_keystore::{Error as KeystoreError, KeystorePtr}; use std::{collections::HashMap, iter::FromIterator, time::Duration}; +use util::{ + request_availability_cores, request_session_index_for_child, request_session_info, + runtime::recv_runtime, +}; use wasm_timer::{Delay, Instant}; mod metrics; @@ -90,25 +91,61 @@ pub enum Error { /// If there is a candidate pending availability, query the Availability Store /// for whether we have the availability chunk for our validator index. async fn get_core_availability( - relay_parent: Hash, core: &CoreState, n_validators: usize, - validator_idx: ValidatorIndex, + validator_index: ValidatorIndex, sender: &Mutex<&mut impl overseer::BitfieldSigningSenderTrait>, span: &jaeger::Span, ) -> Result { if let CoreState::Occupied(core) = core { let _span = span.child("query-chunk-availability"); + let relay_parent = core.candidate_descriptor.relay_parent; + + let Ok(block_number) = get_block_number(*sender.lock().await, relay_parent).await else { + gum::warn!( + target: LOG_TARGET, + ?relay_parent, + ?core.candidate_hash, + para_id = %core.para_id(), + "Failed to get block number." + ); + + return Ok(false) + }; + + let session_index = + recv_runtime(request_session_index_for_child(relay_parent, *sender.lock().await).await) + .await?; + + let Some(session_info) = recv_runtime( + request_session_info(relay_parent, session_index, *sender.lock().await).await, + ) + .await? + else { + gum::warn!( + target: LOG_TARGET, + ?relay_parent, + session_index, + ?core.candidate_hash, + para_id = %core.para_id(), + "Failed to get session info." + ); + + return Ok(false) + }; - let block_number = - get_block_number(*sender.lock().await, core.candidate_descriptor.relay_parent).await?; let maybe_client_features = request_client_features(relay_parent, *sender.lock().await) .await .map_err(Error::from)?; - let chunk_indices = - availability_chunk_indices(maybe_client_features, block_number, n_validators); - let chunk_index = chunk_indices[usize::try_from(validator_idx.0) - .expect("usize is at least u32 bytes on all modern targets.")]; + + let chunk_index = availability_chunk_index( + maybe_client_features.as_ref(), + session_info.random_seed, + n_validators, + block_number, + core.para_id(), + validator_index, + ); let (tx, rx) = oneshot::channel(); sender @@ -137,25 +174,6 @@ async fn get_core_availability( } } -/// delegates to the v1 runtime API -async fn get_availability_cores( - relay_parent: Hash, - sender: &mut impl SubsystemSender, -) -> Result, Error> { - let (tx, rx) = oneshot::channel(); - sender - .send_message( - RuntimeApiMessage::Request(relay_parent, RuntimeApiRequest::AvailabilityCores(tx)) - .into(), - ) - .await; - match rx.await { - Ok(Ok(out)) => Ok(out), - Ok(Err(runtime_err)) => Err(Error::Runtime(runtime_err.into())), - Err(err) => Err(err.into()), - } -} - /// - get the list of core states from the runtime /// - for each core, concurrently determine chunk availability (see `get_core_availability`) /// - return the bitfield if there were no errors at any point in this process (otherwise, it's @@ -170,7 +188,7 @@ async fn construct_availability_bitfield( // get the set of availability cores from the runtime let availability_cores = { let _span = span.child("get-availability-cores"); - get_availability_cores(relay_parent, sender).await? + recv_runtime(request_availability_cores(relay_parent, sender).await).await? }; // Wrap the sender in a Mutex to share it between the futures. @@ -182,9 +200,11 @@ async fn construct_availability_bitfield( // Handle all cores concurrently // `try_join_all` returns all results in the same order as the input futures. - let results = future::try_join_all(availability_cores.iter().map(|core| { - get_core_availability(relay_parent, core, n_validators, validator_idx, &sender, span) - })) + let results = future::try_join_all( + availability_cores + .iter() + .map(|core| get_core_availability(core, n_validators, validator_idx, &sender, span)), + ) .await?; let core_bits = FromIterator::from_iter(results.into_iter()); diff --git a/polkadot/node/network/availability-distribution/src/requester/mod.rs b/polkadot/node/network/availability-distribution/src/requester/mod.rs index aab39c74d262..481e62001838 100644 --- a/polkadot/node/network/availability-distribution/src/requester/mod.rs +++ b/polkadot/node/network/availability-distribution/src/requester/mod.rs @@ -35,13 +35,10 @@ use polkadot_node_subsystem::{ overseer, ActivatedLeaf, ActiveLeavesUpdate, LeafStatus, }; use polkadot_node_subsystem_util::{ - availability_chunk_indices, runtime::{get_occupied_cores, request_client_features, RuntimeInfo}, + ChunkIndexCacheRegistry, }; -use polkadot_primitives::{ - BlockNumber, CandidateHash, ChunkIndex, Hash, OccupiedCore, SessionIndex, -}; -use schnellru::{ByLength, LruMap}; +use polkadot_primitives::{BlockNumber, CandidateHash, Hash, OccupiedCore, SessionIndex}; use super::{error::Error, FatalError, Metrics, Result, LOG_TARGET}; @@ -81,8 +78,8 @@ pub struct Requester { /// Prometheus Metrics metrics: Metrics, - /// Cache of our chunk indices based on the relay parent block. - chunk_index_cache: LruMap, + /// Cache of our chunk indices based on the relay parent block and core index. + chunk_indices: ChunkIndexCacheRegistry, } #[overseer::contextbounds(AvailabilityDistribution, prefix = self::overseer)] @@ -102,9 +99,9 @@ impl Requester { tx, rx, metrics, - // Candidates shouldn't be pending avilability for many blocks, so keep our index for + // Candidates shouldn't be pending availability for many blocks, so keep our index for // the last two relay parents. - chunk_index_cache: LruMap::new(ByLength::new(2)), + chunk_indices: ChunkIndexCacheRegistry::new(2), } } @@ -265,29 +262,31 @@ impl Requester { acc }); - if self.chunk_index_cache.peek(&block_number).is_none() { + let chunk_index = if let Some(chunk_index) = + self.chunk_indices.query_cache_for_validator( + block_number, + session_info.session_index, + core.para_id(), + session_info.our_index, + ) { + chunk_index + } else { let maybe_client_features = request_client_features( core.candidate_descriptor.relay_parent, context.sender(), ) .await?; - let chunk_indices = availability_chunk_indices( + self.chunk_indices.populate_for_validator( maybe_client_features, - block_number, + session_info.random_seed, n_validators, - ); - self.chunk_index_cache.insert( block_number, - chunk_indices[usize::try_from(session_info.our_index.0) - .expect("usize is at least u32 bytes on all modern targets.")], - ); - } - - let chunk_index = self - .chunk_index_cache - .get(&block_number) - .expect("The index was just inserted"); + session_info.session_index, + core.para_id(), + session_info.our_index, + ) + }; let task_cfg = FetchTaskConfig::new( leaf, @@ -295,7 +294,7 @@ impl Requester { tx, metrics, session_info, - *chunk_index, + chunk_index, span, ); diff --git a/polkadot/node/network/availability-distribution/src/requester/session_cache.rs b/polkadot/node/network/availability-distribution/src/requester/session_cache.rs index f6fec74796d7..0bc5aaf63468 100644 --- a/polkadot/node/network/availability-distribution/src/requester/session_cache.rs +++ b/polkadot/node/network/availability-distribution/src/requester/session_cache.rs @@ -62,6 +62,9 @@ pub struct SessionInfo { /// /// `None`, if we are not in fact part of any group. pub our_group: Option, + + /// Per-session randomness gathered from BABE. + pub random_seed: [u8; 32], } /// Report of bad validators. @@ -197,7 +200,13 @@ impl SessionCache { }) .collect(); - let info = SessionInfo { validator_groups, our_index, session_index, our_group }; + let info = SessionInfo { + validator_groups, + our_index, + session_index, + our_group, + random_seed: info.session_info.random_seed, + }; return Ok(Some(info)) } return Ok(None) diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index 457b22f238d9..6a1ec27e655d 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -47,6 +47,7 @@ use polkadot_erasure_coding::{ }; use task::{RecoveryParams, RecoveryStrategy, RecoveryTask}; +use error::{Error, Result}; use polkadot_node_network_protocol::{ request_response::{v1 as request_v1, IncomingRequestReceiver}, UnifiedReputationChange as Rep, @@ -60,7 +61,7 @@ use polkadot_node_subsystem::{ SubsystemContext, SubsystemError, SubsystemResult, }; use polkadot_node_subsystem_util::{ - availability_chunk_indices, request_session_info, runtime::request_client_features, + request_session_info, runtime::request_client_features, ChunkIndexCacheRegistry, }; use polkadot_primitives::{ BlakeTwo256, BlockNumber, CandidateHash, CandidateReceipt, ChunkIndex, GroupIndex, Hash, HashT, @@ -76,6 +77,8 @@ use metrics::Metrics; #[cfg(test)] mod tests; +type RecoveryResult = std::result::Result; + const LOG_TARGET: &str = "parachain::availability-recovery"; // Size of the LRU cache where we keep recovered data. @@ -123,7 +126,7 @@ pub enum ErasureTask { Reconstruct( usize, BTreeMap, - oneshot::Sender>, + oneshot::Sender>, ), /// Re-encode `AvailableData` into erasure chunks in order to verify the provided root hash of /// the Merkle tree. @@ -148,7 +151,7 @@ fn is_chunk_valid(params: &RecoveryParams, chunk: &ErasureChunk) -> bool { gum::debug!( target: LOG_TARGET, candidate_hash = ?params.candidate_hash, - validator_index = ?chunk.index, + chunk_index = ?chunk.index, error = ?e, "Invalid Merkle proof", ); @@ -160,7 +163,7 @@ fn is_chunk_valid(params: &RecoveryParams, chunk: &ErasureChunk) -> bool { gum::debug!( target: LOG_TARGET, candidate_hash = ?params.candidate_hash, - validator_index = ?chunk.index, + chunk_index = ?chunk.index, "Merkle proof mismatch" ); return false @@ -211,12 +214,12 @@ fn reconstructed_data_matches_root( /// Accumulate all awaiting sides for some particular `AvailableData`. struct RecoveryHandle { candidate_hash: CandidateHash, - remote: RemoteHandle>, - awaiting: Vec>>, + remote: RemoteHandle, + awaiting: Vec>, } impl Future for RecoveryHandle { - type Output = Option<(CandidateHash, Result)>; + type Output = Option<(CandidateHash, RecoveryResult)>; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let mut indices_to_remove = Vec::new(); @@ -270,7 +273,7 @@ enum CachedRecovery { impl CachedRecovery { /// Convert back to `Result` to deliver responses. - fn into_result(self) -> Result { + fn into_result(self) -> RecoveryResult { match self { Self::Valid(d) => Ok(d), Self::Invalid => Err(RecoveryError::Invalid), @@ -278,9 +281,9 @@ impl CachedRecovery { } } -impl TryFrom> for CachedRecovery { +impl TryFrom for CachedRecovery { type Error = (); - fn try_from(o: Result) -> Result { + fn try_from(o: RecoveryResult) -> std::result::Result { match o { Ok(d) => Ok(Self::Valid(d)), Err(RecoveryError::Invalid) => Ok(Self::Invalid), @@ -304,7 +307,7 @@ struct State { availability_lru: LruMap, /// Cache of the chunk indices shuffle based on the relay parent block. - chunk_indices_cache: LruMap>, + chunk_indices: ChunkIndexCacheRegistry, } impl Default for State { @@ -313,7 +316,7 @@ impl Default for State { ongoing_recoveries: FuturesUnordered::new(), live_block: (0, Hash::default()), availability_lru: LruMap::new(ByLength::new(LRU_SIZE)), - chunk_indices_cache: LruMap::new(ByLength::new(LRU_SIZE)), + chunk_indices: ChunkIndexCacheRegistry::new(LRU_SIZE), } } } @@ -354,11 +357,11 @@ async fn launch_recovery_task( ctx: &mut Context, session_info: SessionInfo, receipt: CandidateReceipt, - response_sender: oneshot::Sender>, + response_sender: oneshot::Sender, metrics: &Metrics, recovery_strategies: VecDeque::Sender>>>, bypass_availability_store: bool, -) -> error::Result<()> { +) -> Result<()> { let candidate_hash = receipt.hash(); let params = RecoveryParams { validator_authority_keys: session_info.discovery_keys.clone(), @@ -399,12 +402,12 @@ async fn handle_recover( receipt: CandidateReceipt, session_index: SessionIndex, backing_group: Option, - response_sender: oneshot::Sender>, + response_sender: oneshot::Sender, metrics: &Metrics, erasure_task_tx: futures::channel::mpsc::Sender, recovery_strategy_kind: RecoveryStrategyKind, bypass_availability_store: bool, -) -> error::Result<()> { +) -> Result<()> { let candidate_hash = receipt.hash(); let span = jaeger::Span::new(candidate_hash, "availbility-recovery") @@ -434,7 +437,7 @@ async fn handle_recover( let session_info = request_session_info(state.live_block.1, session_index, ctx.sender()) .await .await - .map_err(error::Error::CanceledSessionInfo)??; + .map_err(Error::CanceledSessionInfo)??; let _span = span.child("session-info-ctx-received"); match session_info { @@ -442,36 +445,39 @@ async fn handle_recover( let block_number = get_block_number(ctx.sender(), receipt.descriptor.relay_parent).await?; - if state.chunk_indices_cache.peek(&block_number).is_none() { + let chunk_indices = if let Some(chunk_indices) = state + .chunk_indices + .query_cache_for_para(block_number, session_index, receipt.descriptor.para_id) + { + chunk_indices + } else { let maybe_client_features = request_client_features(receipt.descriptor.relay_parent, ctx.sender()) .await - .map_err(error::Error::RequestClientFeatures)?; + .map_err(Error::RequestClientFeatures)?; - let chunk_indices = availability_chunk_indices( + state.chunk_indices.populate_for_para( maybe_client_features, - block_number, + session_info.random_seed, session_info.validators.len(), + block_number, + session_index, + receipt.descriptor.para_id, ) + }; + + let chunk_indices: VecDeque<_> = chunk_indices .iter() .enumerate() .map(|(v_index, c_index)| { ( *c_index, ValidatorIndex( - u32::try_from(v_index) - .expect("validator numbers should not exceed u32"), + u32::try_from(v_index).expect("validator count should not exceed u32"), ), ) }) .collect(); - state.chunk_indices_cache.insert(block_number, chunk_indices); - } - - let chunk_indices = state - .chunk_indices_cache - .get(&block_number) - .expect("The shuffling was just inserted"); let mut recovery_strategies: VecDeque< Box::Sender>>, @@ -485,7 +491,7 @@ async fn handle_recover( recovery_strategy_kind { // Get our own chunk size to get an estimate of the PoV size. - let chunk_size: Result, error::Error> = + let chunk_size: Result> = query_chunk_size(ctx, candidate_hash).await; if let Ok(Some(chunk_size)) = chunk_size { let pov_size_estimate = @@ -569,7 +575,7 @@ async fn handle_recover( gum::warn!(target: LOG_TARGET, "SessionInfo is `None` at {:?}", state.live_block); response_sender .send(Err(RecoveryError::Unavailable)) - .map_err(|_| error::Error::CanceledResponseSender)?; + .map_err(|_| Error::CanceledResponseSender)?; Ok(()) }, } @@ -580,12 +586,12 @@ async fn handle_recover( async fn query_full_data( ctx: &mut Context, candidate_hash: CandidateHash, -) -> error::Result> { +) -> Result> { let (tx, rx) = oneshot::channel(); ctx.send_message(AvailabilityStoreMessage::QueryAvailableData(candidate_hash, tx)) .await; - rx.await.map_err(error::Error::CanceledQueryFullData) + rx.await.map_err(Error::CanceledQueryFullData) } /// Queries a chunk from av-store. @@ -593,12 +599,12 @@ async fn query_full_data( async fn query_chunk_size( ctx: &mut Context, candidate_hash: CandidateHash, -) -> error::Result> { +) -> Result> { let (tx, rx) = oneshot::channel(); ctx.send_message(AvailabilityStoreMessage::QueryChunkSize(candidate_hash, tx)) .await; - rx.await.map_err(error::Error::CanceledQueryFullData) + rx.await.map_err(Error::CanceledQueryFullData) } #[overseer::contextbounds(AvailabilityRecovery, prefix = self::overseer)] @@ -762,8 +768,6 @@ impl AvailabilityRecoverySubsystem { return Ok(()); } FromOrchestra::Communication { msg } => { - gum::debug!(target: LOG_TARGET, - "Received message to recover available data"); match msg { AvailabilityRecoveryMessage::RecoverAvailableData( receipt, @@ -939,24 +943,18 @@ async fn erasure_task_thread( } } -async fn get_block_number( - sender: &mut Sender, - relay_parent: Hash, -) -> error::Result +async fn get_block_number(sender: &mut Sender, relay_parent: Hash) -> Result where Sender: overseer::SubsystemSender, { let (tx, rx) = oneshot::channel(); sender.send_message(ChainApiMessage::BlockNumber(relay_parent, tx)).await; - let block_number = rx - .await - .map_err(error::Error::ChainApiSenderDropped)? - .map_err(error::Error::ChainApi)?; + let block_number = rx.await.map_err(Error::ChainApiSenderDropped)?.map_err(Error::ChainApi)?; if let Some(number) = block_number { Ok(number) } else { - Err(error::Error::BlockNumberNotFound) + Err(Error::BlockNumberNotFound) } } diff --git a/polkadot/node/primitives/src/lib.rs b/polkadot/node/primitives/src/lib.rs index 463c7f960ba0..baab116f73ae 100644 --- a/polkadot/node/primitives/src/lib.rs +++ b/polkadot/node/primitives/src/lib.rs @@ -37,6 +37,7 @@ use polkadot_primitives::{ }; pub use sp_consensus_babe::{ AllowedSlots as BabeAllowedSlots, BabeEpochConfiguration, Epoch as BabeEpoch, + Randomness as BabeRandomness, }; pub use polkadot_parachain_primitives::primitives::{ diff --git a/polkadot/node/subsystem-util/src/lib.rs b/polkadot/node/subsystem-util/src/lib.rs index 6acc1b39ce9d..dcd259396ff3 100644 --- a/polkadot/node/subsystem-util/src/lib.rs +++ b/polkadot/node/subsystem-util/src/lib.rs @@ -29,13 +29,14 @@ pub use overseer::{ gen::{OrchestraError as OverseerError, Timeout}, Subsystem, TimeoutExt, }; +use polkadot_node_primitives::BabeRandomness; use polkadot_node_subsystem::{ errors::{RuntimeApiError, SubsystemError}, messages::{RuntimeApiMessage, RuntimeApiRequest, RuntimeApiSender}, overseer, SubsystemSender, }; use polkadot_primitives::{slashing, vstaging::ClientFeatures, BlockNumber, ExecutorParams}; -use rand::seq::SliceRandom; +use rand::{seq::SliceRandom, Rng}; use rand_chacha::ChaCha8Rng; pub use polkadot_node_metrics::{metrics, Metronome}; @@ -52,6 +53,7 @@ use polkadot_primitives::{ }; pub use rand; use rand::SeedableRng; +use schnellru::{ByLength, LruMap}; use sp_application_crypto::AppCrypto; use sp_core::ByteArray; use sp_keystore::{Error as KeystoreError, KeystorePtr}; @@ -440,33 +442,199 @@ impl Validator { } } -/// Return a mapping of the availability chunk indices(`ValidatorIndex -> ChunkIndex`). -/// -/// The vector indices represent validator indices. -/// If it's enabled, shuffle the indices using the `BlockNumber` as a randomness seed, so that other -/// validators have a common view of the shuffle at a given block height. -/// Otherwise, return the identity vector. -pub fn availability_chunk_indices( - maybe_client_features: Option, - block_number: BlockNumber, - n_validators: usize, -) -> Vec { - let mut indices: Vec<_> = (0..n_validators) - .map(|i| ValidatorIndex(u32::try_from(i).expect("validator numbers should not exceed u32"))) - .collect(); - - if let Some(features) = maybe_client_features { - if features.contains(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING) { - let seed = block_number.to_be_bytes(); - let mut rng: ChaCha8Rng = SeedableRng::from_seed( - seed.repeat(8) - .try_into() - .expect("vector of 32 bytes is safe to cast to array of 32 bytes. qed."), +/// Object used for holding and computing assigned chunk indices for validators. +pub struct ChunkIndexCacheRegistry( + LruMap<(BlockNumber, SessionIndex), (Vec, Option)>, +); + +impl ChunkIndexCacheRegistry { + /// Initialize with the cache capacity. + pub fn new(capacity: u32) -> Self { + Self(LruMap::new(ByLength::new(capacity))) + } + + /// Return the per-validator chunk index if present in the cache. + pub fn query_cache_for_validator( + &mut self, + block_number: BlockNumber, + session_index: SessionIndex, + para_id: ParaId, + validator_index: ValidatorIndex, + ) -> Option { + if let Some((shuffle, maybe_client_features)) = self.0.get(&(block_number, session_index)) { + Some(Self::compute_chunk_index_for_validator( + maybe_client_features.as_ref(), + shuffle, + para_id, + validator_index, + )) + } else { + None + } + } + + /// Return the per-para chunk index vector if present in the cache. + pub fn query_cache_for_para( + &mut self, + block_number: BlockNumber, + session_index: SessionIndex, + para_id: ParaId, + ) -> Option> { + if let Some((shuffle, maybe_client_features)) = self.0.get(&(block_number, session_index)) { + let core_start_index = Self::compute_para_start_index( + maybe_client_features.as_ref(), + shuffle.len(), + para_id, ); - indices.shuffle(&mut rng); + let chunk_indices = shuffle + .clone() + .into_iter() + .cycle() + .skip(core_start_index) + .take(shuffle.len()) + .collect(); + + Some(chunk_indices) + } else { + None } } - indices + /// Return and populate the cache with the per-validator chunk index. + /// Should only be called if `query_cache_for_validator` returns `None`. + pub fn populate_for_validator( + &mut self, + maybe_client_features: Option, + babe_randomness: BabeRandomness, + n_validators: usize, + block_number: BlockNumber, + session_index: SessionIndex, + para_id: ParaId, + validator_index: ValidatorIndex, + ) -> ChunkIndex { + let shuffle = Self::get_shuffle( + maybe_client_features.as_ref(), + block_number, + babe_randomness, + n_validators, + ); + self.0.insert((block_number, session_index), (shuffle, maybe_client_features)); + + self.query_cache_for_validator(block_number, session_index, para_id, validator_index) + .expect("We just inserted the entry.") + } + + /// Return and populate the cache with the per-para chunk index vector. + /// Should only be called if `query_cache_for_para` returns `None`. + pub fn populate_for_para( + &mut self, + maybe_client_features: Option, + babe_randomness: BabeRandomness, + n_validators: usize, + block_number: BlockNumber, + session_index: SessionIndex, + para_id: ParaId, + ) -> Vec { + let shuffle = Self::get_shuffle( + maybe_client_features.as_ref(), + block_number, + babe_randomness, + n_validators, + ); + self.0.insert((block_number, session_index), (shuffle, maybe_client_features)); + + self.query_cache_for_para(block_number, session_index, para_id) + .expect("We just inserted the entry.") + } + + fn get_shuffle( + maybe_client_features: Option<&ClientFeatures>, + block_number: BlockNumber, + mut babe_randomness: BabeRandomness, + n_validators: usize, + ) -> Vec { + let mut indices: Vec<_> = (0..n_validators) + .map(|i| { + ValidatorIndex(u32::try_from(i).expect("validator count should not exceed u32")) + }) + .collect(); + + if let Some(features) = maybe_client_features { + if features.contains(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING) { + let block_number_bytes = block_number.to_be_bytes(); + for i in 0..32 { + babe_randomness[i] ^= block_number_bytes[i % block_number_bytes.len()]; + } + + let mut rng: ChaCha8Rng = SeedableRng::from_seed(babe_randomness); + + indices.shuffle(&mut rng); + } + } + + indices + } + + /// Return the availability chunk start index for this para. + fn compute_para_start_index( + maybe_client_features: Option<&ClientFeatures>, + n_validators: usize, + para_id: ParaId, + ) -> usize { + if let Some(features) = maybe_client_features { + if features.contains(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING) { + let mut rng: ChaCha8Rng = + SeedableRng::from_seed( + u32::from(para_id).to_be_bytes().repeat(8).try_into().expect( + "vector of 32 bytes is safe to cast to array of 32 bytes. qed.", + ), + ); + return rng.gen_range(0..n_validators) + } + } + + 0 + } + + fn compute_chunk_index_for_validator( + maybe_client_features: Option<&ClientFeatures>, + shuffle: &Vec, + para_id: ParaId, + validator_index: ValidatorIndex, + ) -> ChunkIndex { + let core_start_index = + Self::compute_para_start_index(maybe_client_features, shuffle.len(), para_id); + + let chunk_index = shuffle[(core_start_index + + usize::try_from(validator_index.0) + .expect("usize is at least u32 bytes on all modern targets.")) % + shuffle.len()]; + chunk_index + } +} + +/// Compute the per-validator availability chunk index. +/// It's preferred to use the `ChunkIndexCacheRegistry` if you also need a cache. +pub fn availability_chunk_index( + maybe_client_features: Option<&ClientFeatures>, + babe_randomness: BabeRandomness, + n_validators: usize, + block_number: BlockNumber, + para_id: ParaId, + validator_index: ValidatorIndex, +) -> ChunkIndex { + let shuffle = ChunkIndexCacheRegistry::get_shuffle( + maybe_client_features, + block_number, + babe_randomness, + n_validators, + ); + + ChunkIndexCacheRegistry::compute_chunk_index_for_validator( + maybe_client_features, + &shuffle, + para_id, + validator_index, + ) } diff --git a/polkadot/runtime/parachains/src/configuration/migration/v10.rs b/polkadot/runtime/parachains/src/configuration/migration/v10.rs index 001075e8d5f5..14fbbfe31ef2 100644 --- a/polkadot/runtime/parachains/src/configuration/migration/v10.rs +++ b/polkadot/runtime/parachains/src/configuration/migration/v10.rs @@ -23,8 +23,7 @@ use frame_support::{ weights::Weight, }; use frame_system::pallet_prelude::BlockNumberFor; -use primitives::{vstaging::ClientFeatures, SessionIndex, LEGACY_MIN_BACKING_VOTES}; -use sp_runtime::Perbill; +use primitives::{vstaging::ClientFeatures, SessionIndex}; use sp_std::vec::Vec; use frame_support::traits::OnRuntimeUpgrade; @@ -180,6 +179,7 @@ client_features : ClientFeatures::empty() mod tests { use super::*; use crate::mock::{new_test_ext, Test}; + use primitives::LEGACY_MIN_BACKING_VOTES; #[test] fn v10_deserialized_from_actual_data() { From b3ea7c90194ccb3c9621558dd65858f96ca264ab Mon Sep 17 00:00:00 2001 From: alindima Date: Wed, 27 Sep 2023 17:02:05 +0300 Subject: [PATCH 037/126] add newtype for ChunkIndex also switched the type in ChunkFetchingRequest, because the Encode/Decode implementations should be identical --- polkadot/node/core/av-store/src/lib.rs | 29 ++++++----- polkadot/node/core/av-store/src/tests.rs | 50 +++++++++---------- .../src/requester/fetch_task/mod.rs | 6 +-- .../src/requester/fetch_task/tests.rs | 4 +- .../src/responder.rs | 10 ++-- .../src/tests/mock.rs | 8 +-- .../src/tests/state.rs | 12 ++--- .../network/availability-recovery/src/task.rs | 16 +++--- .../availability-recovery/src/tests.rs | 2 +- .../protocol/src/request_response/v1.rs | 4 +- polkadot/node/primitives/src/lib.rs | 6 +-- polkadot/node/subsystem-types/src/messages.rs | 6 +-- polkadot/node/subsystem-util/src/lib.rs | 4 +- polkadot/primitives/src/v5/mod.rs | 24 +++++++-- 14 files changed, 95 insertions(+), 86 deletions(-) diff --git a/polkadot/node/core/av-store/src/lib.rs b/polkadot/node/core/av-store/src/lib.rs index ef7dcecac075..47298f9f2dde 100644 --- a/polkadot/node/core/av-store/src/lib.rs +++ b/polkadot/node/core/av-store/src/lib.rs @@ -48,7 +48,7 @@ use polkadot_node_subsystem::{ }; use polkadot_node_subsystem_util as util; use polkadot_primitives::{ - BlockNumber, CandidateEvent, CandidateHash, CandidateReceipt, Hash, Header, ValidatorIndex, + BlockNumber, CandidateEvent, CandidateHash, CandidateReceipt, ChunkIndex, Hash, Header, }; mod metrics; @@ -208,7 +208,7 @@ fn load_chunk( db: &Arc, config: &Config, candidate_hash: &CandidateHash, - chunk_index: ValidatorIndex, + chunk_index: ChunkIndex, ) -> Result, Error> { let key = (CHUNK_PREFIX, candidate_hash, chunk_index).encode(); @@ -219,7 +219,7 @@ fn write_chunk( tx: &mut DBTransaction, config: &Config, candidate_hash: &CandidateHash, - chunk_index: ValidatorIndex, + chunk_index: ChunkIndex, erasure_chunk: &ErasureChunk, ) { let key = (CHUNK_PREFIX, candidate_hash, chunk_index).encode(); @@ -231,7 +231,7 @@ fn delete_chunk( tx: &mut DBTransaction, config: &Config, candidate_hash: &CandidateHash, - chunk_index: ValidatorIndex, + chunk_index: ChunkIndex, ) { let key = (CHUNK_PREFIX, candidate_hash, chunk_index).encode(); @@ -1107,22 +1107,21 @@ fn process_message( .map_or(false, |m| m.data_available); let _ = tx.send(a); }, - AvailabilityStoreMessage::QueryChunk(candidate, validator_index, tx) => { + AvailabilityStoreMessage::QueryChunk(candidate, chunk_index, tx) => { let _timer = subsystem.metrics.time_get_chunk(); - let _ = - tx.send(load_chunk(&subsystem.db, &subsystem.config, &candidate, validator_index)?); + let _ = tx.send(load_chunk(&subsystem.db, &subsystem.config, &candidate, chunk_index)?); }, AvailabilityStoreMessage::QueryChunkSize(candidate, tx) => { let meta = load_meta(&subsystem.db, &subsystem.config, &candidate)?; - let validator_index = meta.map_or(None, |meta| meta.chunks_stored.first_one()); + let chunk_index = meta.map_or(None, |meta| meta.chunks_stored.first_one()); - let maybe_chunk_size = if let Some(validator_index) = validator_index { + let maybe_chunk_size = if let Some(chunk_index) = chunk_index { load_chunk( &subsystem.db, &subsystem.config, &candidate, - ValidatorIndex(validator_index as u32), + ChunkIndex(chunk_index as u32), )? .map(|erasure_chunk| erasure_chunk.chunk.len()) } else { @@ -1145,7 +1144,7 @@ fn process_message( &subsystem.db, &subsystem.config, &candidate, - ValidatorIndex(index as _), + ChunkIndex(index as _), )? { Some(c) => chunks.push(c), None => { @@ -1163,9 +1162,9 @@ fn process_message( }, } }, - AvailabilityStoreMessage::QueryChunkAvailability(candidate, validator_index, tx) => { + AvailabilityStoreMessage::QueryChunkAvailability(candidate, chunk_index, tx) => { let a = load_meta(&subsystem.db, &subsystem.config, &candidate)?.map_or(false, |m| { - *m.chunks_stored.get(validator_index.0 as usize).as_deref().unwrap_or(&false) + *m.chunks_stored.get(chunk_index.0 as usize).as_deref().unwrap_or(&false) }); let _ = tx.send(a); }, @@ -1316,7 +1315,7 @@ fn store_available_data( |(index, (chunk, proof))| ErasureChunk { chunk: chunk.clone(), proof, - index: ValidatorIndex(index as u32), + index: ChunkIndex(index as u32), }, ); @@ -1366,7 +1365,7 @@ fn prune_all(db: &Arc, config: &Config, now: Duration) -> Result<( // delete chunks. for (i, b) in meta.chunks_stored.iter().enumerate() { if *b { - delete_chunk(&mut tx, config, &candidate_hash, ValidatorIndex(i as _)); + delete_chunk(&mut tx, config, &candidate_hash, ChunkIndex(i as _)); } } diff --git a/polkadot/node/core/av-store/src/tests.rs b/polkadot/node/core/av-store/src/tests.rs index 652bf2a3fda4..13c7c7d3bc31 100644 --- a/polkadot/node/core/av-store/src/tests.rs +++ b/polkadot/node/core/av-store/src/tests.rs @@ -272,8 +272,8 @@ fn runtime_api_error_does_not_stop_the_subsystem() { // but that's fine, we're still alive let (tx, rx) = oneshot::channel(); let candidate_hash = CandidateHash(Hash::repeat_byte(33)); - let validator_index = ValidatorIndex(5); - let query_chunk = AvailabilityStoreMessage::QueryChunk(candidate_hash, validator_index, tx); + let chunk_index = ChunkIndex(5); + let query_chunk = AvailabilityStoreMessage::QueryChunk(candidate_hash, chunk_index, tx); overseer_send(&mut virtual_overseer, query_chunk.into()).await; @@ -288,12 +288,12 @@ fn store_chunk_works() { test_harness(TestState::default(), store.clone(), |mut virtual_overseer| async move { let candidate_hash = CandidateHash(Hash::repeat_byte(33)); - let validator_index = ValidatorIndex(5); + let chunk_index = ChunkIndex(5); let n_validators = 10; let chunk = ErasureChunk { chunk: vec![1, 2, 3], - index: validator_index, + index: chunk_index, proof: Proof::try_from(vec![vec![3, 4, 5]]).unwrap(), }; @@ -321,7 +321,7 @@ fn store_chunk_works() { assert_eq!(rx.await.unwrap(), Ok(())); let (tx, rx) = oneshot::channel(); - let query_chunk = AvailabilityStoreMessage::QueryChunk(candidate_hash, validator_index, tx); + let query_chunk = AvailabilityStoreMessage::QueryChunk(candidate_hash, chunk_index, tx); overseer_send(&mut virtual_overseer, query_chunk).await; @@ -336,11 +336,11 @@ fn store_chunk_does_nothing_if_no_entry_already() { test_harness(TestState::default(), store.clone(), |mut virtual_overseer| async move { let candidate_hash = CandidateHash(Hash::repeat_byte(33)); - let validator_index = ValidatorIndex(5); + let chunk_index = ChunkIndex(5); let chunk = ErasureChunk { chunk: vec![1, 2, 3], - index: validator_index, + index: chunk_index, proof: Proof::try_from(vec![vec![3, 4, 5]]).unwrap(), }; @@ -353,7 +353,7 @@ fn store_chunk_does_nothing_if_no_entry_already() { assert_eq!(rx.await.unwrap(), Err(())); let (tx, rx) = oneshot::channel(); - let query_chunk = AvailabilityStoreMessage::QueryChunk(candidate_hash, validator_index, tx); + let query_chunk = AvailabilityStoreMessage::QueryChunk(candidate_hash, chunk_index, tx); overseer_send(&mut virtual_overseer, query_chunk).await; @@ -368,7 +368,7 @@ fn query_chunk_checks_meta() { test_harness(TestState::default(), store.clone(), |mut virtual_overseer| async move { let candidate_hash = CandidateHash(Hash::repeat_byte(33)); - let validator_index = ValidatorIndex(5); + let chunk_index = ChunkIndex(5); let n_validators = 10; // Ensure an entry already exists. In reality this would come from watching @@ -382,7 +382,7 @@ fn query_chunk_checks_meta() { data_available: false, chunks_stored: { let mut v = bitvec::bitvec![u8, BitOrderLsb0; 0; n_validators]; - v.set(validator_index.0 as usize, true); + v.set(chunk_index.0 as usize, true); v }, state: State::Unavailable(BETimestamp(0)), @@ -392,7 +392,7 @@ fn query_chunk_checks_meta() { let (tx, rx) = oneshot::channel(); let query_chunk = - AvailabilityStoreMessage::QueryChunkAvailability(candidate_hash, validator_index, tx); + AvailabilityStoreMessage::QueryChunkAvailability(candidate_hash, chunk_index, tx); overseer_send(&mut virtual_overseer, query_chunk.into()).await; assert!(rx.await.unwrap()); @@ -400,7 +400,7 @@ fn query_chunk_checks_meta() { let (tx, rx) = oneshot::channel(); let query_chunk = AvailabilityStoreMessage::QueryChunkAvailability( candidate_hash, - ValidatorIndex(validator_index.0 + 1), + ChunkIndex(chunk_index.0 + 1), tx, ); @@ -416,7 +416,7 @@ fn store_available_data_erasure_mismatch() { let test_state = TestState::default(); test_harness(test_state.clone(), store.clone(), |mut virtual_overseer| async move { let candidate_hash = CandidateHash(Hash::repeat_byte(1)); - let validator_index = ValidatorIndex(5); + let chunk_index = ChunkIndex(5); let n_validators = 10; let pov = PoV { block_data: BlockData(vec![4, 5, 6]) }; @@ -441,9 +441,7 @@ fn store_available_data_erasure_mismatch() { assert!(query_available_data(&mut virtual_overseer, candidate_hash).await.is_none()); - assert!(query_chunk(&mut virtual_overseer, candidate_hash, validator_index) - .await - .is_none()); + assert!(query_chunk(&mut virtual_overseer, candidate_hash, chunk_index).await.is_none()); virtual_overseer }); @@ -455,7 +453,7 @@ fn store_block_works() { let test_state = TestState::default(); test_harness(test_state.clone(), store.clone(), |mut virtual_overseer| async move { let candidate_hash = CandidateHash(Hash::repeat_byte(1)); - let validator_index = ValidatorIndex(5); + let chunk_index = ChunkIndex(5); let n_validators = 10; let pov = PoV { block_data: BlockData(vec![4, 5, 6]) }; @@ -483,14 +481,12 @@ fn store_block_works() { let pov = query_available_data(&mut virtual_overseer, candidate_hash).await.unwrap(); assert_eq!(pov, available_data); - let chunk = query_chunk(&mut virtual_overseer, candidate_hash, validator_index) - .await - .unwrap(); + let chunk = query_chunk(&mut virtual_overseer, candidate_hash, chunk_index).await.unwrap(); let branch = branches.nth(5).unwrap(); let expected_chunk = ErasureChunk { chunk: branch.1.to_vec(), - index: ValidatorIndex(5), + index: ChunkIndex(5), proof: Proof::try_from(branch.0).unwrap(), }; @@ -533,7 +529,7 @@ fn store_pov_and_query_chunk_works() { assert_eq!(rx.await.unwrap(), Ok(())); for i in 0..n_validators { - let chunk = query_chunk(&mut virtual_overseer, candidate_hash, ValidatorIndex(i as _)) + let chunk = query_chunk(&mut virtual_overseer, candidate_hash, ChunkIndex(i as _)) .await .unwrap(); @@ -598,7 +594,7 @@ fn query_all_chunks_works() { let chunk = ErasureChunk { chunk: vec![1, 2, 3], - index: ValidatorIndex(1), + index: ChunkIndex(1), proof: Proof::try_from(vec![vec![3, 4, 5]]).unwrap(), }; @@ -1116,7 +1112,7 @@ async fn query_available_data( async fn query_chunk( virtual_overseer: &mut VirtualOverseer, candidate_hash: CandidateHash, - index: ValidatorIndex, + index: ChunkIndex, ) -> Option { let (tx, rx) = oneshot::channel(); @@ -1133,7 +1129,7 @@ async fn has_all_chunks( expect_present: bool, ) -> bool { for i in 0..n_validators { - if query_chunk(virtual_overseer, candidate_hash, ValidatorIndex(i)).await.is_some() != + if query_chunk(virtual_overseer, candidate_hash, ChunkIndex(i)).await.is_some() != expect_present { return false @@ -1206,12 +1202,12 @@ fn query_chunk_size_works() { test_harness(TestState::default(), store.clone(), |mut virtual_overseer| async move { let candidate_hash = CandidateHash(Hash::repeat_byte(33)); - let validator_index = ValidatorIndex(5); + let chunk_index = ChunkIndex(5); let n_validators = 10; let chunk = ErasureChunk { chunk: vec![1, 2, 3], - index: validator_index, + index: chunk_index, proof: Proof::try_from(vec![vec![3, 4, 5]]).unwrap(), }; diff --git a/polkadot/node/network/availability-distribution/src/requester/fetch_task/mod.rs b/polkadot/node/network/availability-distribution/src/requester/fetch_task/mod.rs index e1fcb2df122d..d8e13ccc67a0 100644 --- a/polkadot/node/network/availability-distribution/src/requester/fetch_task/mod.rs +++ b/polkadot/node/network/availability-distribution/src/requester/fetch_task/mod.rs @@ -34,8 +34,8 @@ use polkadot_node_subsystem::{ overseer, }; use polkadot_primitives::{ - AuthorityDiscoveryId, BlakeTwo256, CandidateHash, GroupIndex, Hash, HashT, OccupiedCore, - SessionIndex, ValidatorIndex, + AuthorityDiscoveryId, BlakeTwo256, CandidateHash, ChunkIndex, GroupIndex, Hash, HashT, + OccupiedCore, SessionIndex, }; use crate::{ @@ -140,7 +140,7 @@ impl FetchTaskConfig { sender: mpsc::Sender, metrics: Metrics, session_info: &SessionInfo, - chunk_index: ValidatorIndex, + chunk_index: ChunkIndex, span: jaeger::Span, ) -> Self { let span = span diff --git a/polkadot/node/network/availability-distribution/src/requester/fetch_task/tests.rs b/polkadot/node/network/availability-distribution/src/requester/fetch_task/tests.rs index 460f20499ed5..c7ef74e439d7 100644 --- a/polkadot/node/network/availability-distribution/src/requester/fetch_task/tests.rs +++ b/polkadot/node/network/availability-distribution/src/requester/fetch_task/tests.rs @@ -77,7 +77,7 @@ fn task_stores_valid_chunk() { let pov = PoV { block_data: BlockData(vec![45, 46, 47]) }; let (root_hash, chunk) = get_valid_chunk_data(pov); task.erasure_root = root_hash; - task.request.index = chunk.index; + task.request.index = chunk.index.into(); let validators = vec![Sr25519Keyring::Alice.public().into()]; task.group = validators; @@ -138,7 +138,7 @@ fn task_stores_valid_chunk_if_there_is_one() { let pov = PoV { block_data: BlockData(vec![45, 46, 47]) }; let (root_hash, chunk) = get_valid_chunk_data(pov); task.erasure_root = root_hash; - task.request.index = chunk.index; + task.request.index = chunk.index.into(); let validators = [ // Only Alice has valid chunk - should succeed, even though she is tried last. diff --git a/polkadot/node/network/availability-distribution/src/responder.rs b/polkadot/node/network/availability-distribution/src/responder.rs index 54b188f7f01f..5d768f84a567 100644 --- a/polkadot/node/network/availability-distribution/src/responder.rs +++ b/polkadot/node/network/availability-distribution/src/responder.rs @@ -27,7 +27,7 @@ use polkadot_node_network_protocol::{ }; use polkadot_node_primitives::{AvailableData, ErasureChunk}; use polkadot_node_subsystem::{jaeger, messages::AvailabilityStoreMessage, SubsystemSender}; -use polkadot_primitives::{CandidateHash, ValidatorIndex}; +use polkadot_primitives::{CandidateHash, ChunkIndex}; use crate::{ error::{JfyiError, Result}, @@ -217,22 +217,20 @@ where async fn query_chunk( sender: &mut Sender, candidate_hash: CandidateHash, - validator_index: ValidatorIndex, + chunk_index: ChunkIndex, ) -> std::result::Result, JfyiError> where Sender: SubsystemSender, { let (tx, rx) = oneshot::channel(); sender - .send_message( - AvailabilityStoreMessage::QueryChunk(candidate_hash, validator_index, tx).into(), - ) + .send_message(AvailabilityStoreMessage::QueryChunk(candidate_hash, chunk_index, tx).into()) .await; let result = rx.await.map_err(|e| { gum::trace!( target: LOG_TARGET, - ?validator_index, + ?chunk_index, ?candidate_hash, error = ?e, "Error retrieving chunk", diff --git a/polkadot/node/network/availability-distribution/src/tests/mock.rs b/polkadot/node/network/availability-distribution/src/tests/mock.rs index 3df662fe546c..9077658c28b0 100644 --- a/polkadot/node/network/availability-distribution/src/tests/mock.rs +++ b/polkadot/node/network/availability-distribution/src/tests/mock.rs @@ -23,9 +23,9 @@ use sp_keyring::Sr25519Keyring; use polkadot_erasure_coding::{branches, obtain_chunks_v1 as obtain_chunks}; use polkadot_node_primitives::{AvailableData, BlockData, ErasureChunk, PoV, Proof}; use polkadot_primitives::{ - CandidateCommitments, CandidateDescriptor, CandidateHash, CommittedCandidateReceipt, - GroupIndex, Hash, HeadData, Id as ParaId, IndexedVec, OccupiedCore, PersistedValidationData, - SessionInfo, ValidatorIndex, + CandidateCommitments, CandidateDescriptor, CandidateHash, ChunkIndex, + CommittedCandidateReceipt, GroupIndex, Hash, HeadData, Id as ParaId, IndexedVec, OccupiedCore, + PersistedValidationData, SessionInfo, ValidatorIndex, }; use polkadot_primitives_test_helpers::{ dummy_collator, dummy_collator_signature, dummy_hash, dummy_validation_code, @@ -149,7 +149,7 @@ pub fn get_valid_chunk_data(pov: PoV) -> (Hash, ErasureChunk) { .enumerate() .map(|(index, (proof, chunk))| ErasureChunk { chunk: chunk.to_vec(), - index: ValidatorIndex(index as _), + index: ChunkIndex(index as _), proof: Proof::try_from(proof).unwrap(), }) .next() diff --git a/polkadot/node/network/availability-distribution/src/tests/state.rs b/polkadot/node/network/availability-distribution/src/tests/state.rs index 101d917c0db5..0a90a785e08f 100644 --- a/polkadot/node/network/availability-distribution/src/tests/state.rs +++ b/polkadot/node/network/availability-distribution/src/tests/state.rs @@ -46,8 +46,8 @@ use polkadot_node_subsystem::{ }; use polkadot_node_subsystem_test_helpers as test_helpers; use polkadot_primitives::{ - CandidateHash, CoreState, ExecutorParams, GroupIndex, Hash, Id as ParaId, ScheduledCore, - SessionInfo, ValidatorIndex, + CandidateHash, ChunkIndex, CoreState, ExecutorParams, GroupIndex, Hash, Id as ParaId, + ScheduledCore, SessionInfo, }; use test_helpers::mock::{make_ferdie_keystore, new_leaf}; @@ -75,9 +75,9 @@ pub struct TestState { /// Whenever the subsystem tries to fetch an erasure chunk one item of the given vec will be /// popped. So you can experiment with serving invalid chunks or no chunks on request and see /// whether the subsystem still succeeds with its goal. - pub chunks: HashMap<(CandidateHash, ValidatorIndex), Vec>>, + pub chunks: HashMap<(CandidateHash, ChunkIndex), Vec>>, /// All chunks that are valid and should be accepted. - pub valid_chunks: HashSet<(CandidateHash, ValidatorIndex)>, + pub valid_chunks: HashSet<(CandidateHash, ChunkIndex)>, pub session_info: SessionInfo, /// Cores per relay chain block. pub cores: HashMap>, @@ -131,8 +131,8 @@ impl Default for TestState { // Skip chunks for our own group (won't get fetched): let mut chunks_other_groups = p_chunks.into_iter(); chunks_other_groups.next(); - for (validator_index, chunk) in chunks_other_groups { - chunks.insert((validator_index, chunk.index), vec![Some(chunk)]); + for (candidate, chunk) in chunks_other_groups { + chunks.insert((candidate, chunk.index), vec![Some(chunk)]); } } (cores, chunks) diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 7bb4f61d4af6..158d034854a7 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -284,7 +284,7 @@ impl State { gum::trace!( target: LOG_TARGET, candidate_hash = ?params.candidate_hash, - chunk_index = ?chunk_index, + ?chunk_index, ?validator_index, "Received valid chunk", ); @@ -298,7 +298,7 @@ impl State { gum::trace!( target: LOG_TARGET, candidate_hash = ?params.candidate_hash, - chunk_index = ?chunk_index, + ?chunk_index, ?validator_index, "Validator did not have the requested chunk", ); @@ -311,7 +311,7 @@ impl State { target: LOG_TARGET, candidate_hash= ?params.candidate_hash, ?err, - chunk_index = ?chunk_index, + ?chunk_index, ?validator_index, "Failure requesting chunk", ); @@ -324,7 +324,7 @@ impl State { target: LOG_TARGET, candidate_hash = ?params.candidate_hash, ?err, - chunk_index = ?chunk_index, + ?chunk_index, ?validator_index, "Chunk fetching response was invalid", ); @@ -362,7 +362,7 @@ impl State { received_chunks_count = ?self.chunk_count(), requested_chunks_count = ?requesting_chunks.len(), threshold = ?params.threshold, - "Can conclude availability recovery for a candidate", + "Can conclude availability recovery strategy", ); break } @@ -664,8 +664,8 @@ impl FetchSystematicChunks { let chunks = state .received_chunks .range( - ValidatorIndex(0).. - ValidatorIndex( + ChunkIndex(0).. + ChunkIndex( u32::try_from(self.threshold) .expect("validator count should not exceed u32"), ), @@ -1172,7 +1172,7 @@ mod tests { let (erasure_task_tx, _erasure_task_rx) = futures::channel::mpsc::channel(16); let mut fetch_chunks_task = FetchChunks::new(FetchChunksParams { - validators: (0..100u32).map(|i| (ValidatorIndex(i), ValidatorIndex(i))).collect(), + validators: (0..100u32).map(|i| (ChunkIndex(i), ValidatorIndex(i))).collect(), erasure_task_tx, }); assert_eq!(fetch_chunks_task.get_desired_request_count(0, threshold), threshold); diff --git a/polkadot/node/network/availability-recovery/src/tests.rs b/polkadot/node/network/availability-recovery/src/tests.rs index 0876f250ecf1..6702f431328f 100644 --- a/polkadot/node/network/availability-recovery/src/tests.rs +++ b/polkadot/node/network/availability-recovery/src/tests.rs @@ -439,7 +439,7 @@ fn derive_erasure_chunks_with_proofs_and_root( .enumerate() .map(|(index, (proof, chunk))| ErasureChunk { chunk: chunk.to_vec(), - index: ValidatorIndex(index as _), + index: ChunkIndex(index as _), proof: Proof::try_from(proof).unwrap(), }) .collect::>(); diff --git a/polkadot/node/network/protocol/src/request_response/v1.rs b/polkadot/node/network/protocol/src/request_response/v1.rs index 0832593a6a3d..ec8e9ef4b277 100644 --- a/polkadot/node/network/protocol/src/request_response/v1.rs +++ b/polkadot/node/network/protocol/src/request_response/v1.rs @@ -22,7 +22,7 @@ use polkadot_node_primitives::{ AvailableData, DisputeMessage, ErasureChunk, PoV, Proof, UncheckedDisputeMessage, }; use polkadot_primitives::{ - CandidateHash, CandidateReceipt, CommittedCandidateReceipt, Hash, Id as ParaId, ValidatorIndex, + CandidateHash, CandidateReceipt, ChunkIndex, CommittedCandidateReceipt, Hash, Id as ParaId, }; use super::{IsRequest, Protocol}; @@ -33,7 +33,7 @@ pub struct ChunkFetchingRequest { /// Hash of candidate we want a chunk for. pub candidate_hash: CandidateHash, /// The index of the chunk to fetch. - pub index: ValidatorIndex, + pub index: ChunkIndex, } /// Receive a requested erasure chunk. diff --git a/polkadot/node/primitives/src/lib.rs b/polkadot/node/primitives/src/lib.rs index baab116f73ae..17fe32a49ba4 100644 --- a/polkadot/node/primitives/src/lib.rs +++ b/polkadot/node/primitives/src/lib.rs @@ -30,10 +30,10 @@ use parity_scale_codec::{Decode, Encode, Error as CodecError, Input}; use serde::{de, Deserialize, Deserializer, Serialize, Serializer}; use polkadot_primitives::{ - BlakeTwo256, BlockNumber, CandidateCommitments, CandidateHash, CollatorPair, + BlakeTwo256, BlockNumber, CandidateCommitments, CandidateHash, ChunkIndex, CollatorPair, CommittedCandidateReceipt, CompactStatement, EncodeAs, Hash, HashT, HeadData, Id as ParaId, PersistedValidationData, SessionIndex, Signed, UncheckedSigned, ValidationCode, - ValidationCodeHash, ValidatorIndex, MAX_CODE_SIZE, MAX_POV_SIZE, + ValidationCodeHash, MAX_CODE_SIZE, MAX_POV_SIZE, }; pub use sp_consensus_babe::{ AllowedSlots as BabeAllowedSlots, BabeEpochConfiguration, Epoch as BabeEpoch, @@ -631,7 +631,7 @@ pub struct ErasureChunk { /// The erasure-encoded chunk of data belonging to the candidate block. pub chunk: Vec, /// The index of this erasure-encoded chunk of data. - pub index: ValidatorIndex, + pub index: ChunkIndex, /// Proof for this chunk's branch in the Merkle tree. pub proof: Proof, } diff --git a/polkadot/node/subsystem-types/src/messages.rs b/polkadot/node/subsystem-types/src/messages.rs index 61c87772b086..968c41ca2870 100644 --- a/polkadot/node/subsystem-types/src/messages.rs +++ b/polkadot/node/subsystem-types/src/messages.rs @@ -42,7 +42,7 @@ use polkadot_primitives::{ slashing, vstaging::{self as vstaging_primitives, ClientFeatures}, AuthorityDiscoveryId, BackedCandidate, BlockNumber, CandidateEvent, CandidateHash, - CandidateIndex, CandidateReceipt, CollatorId, CommittedCandidateReceipt, CoreState, + CandidateIndex, CandidateReceipt, ChunkIndex, CollatorId, CommittedCandidateReceipt, CoreState, DisputeState, ExecutorParams, GroupIndex, GroupRotationInfo, Hash, Header as BlockHeader, Id as ParaId, InboundDownwardMessage, InboundHrmpMessage, MultiDisputeStatementSet, OccupiedCoreAssumption, PersistedValidationData, PvfCheckStatement, PvfExecTimeoutKind, @@ -487,7 +487,7 @@ pub enum AvailabilityStoreMessage { QueryDataAvailability(CandidateHash, oneshot::Sender), /// Query an `ErasureChunk` from the AV store by the candidate hash and validator index. - QueryChunk(CandidateHash, ValidatorIndex, oneshot::Sender>), + QueryChunk(CandidateHash, ChunkIndex, oneshot::Sender>), /// Get the size of an `ErasureChunk` from the AV store by the candidate hash. QueryChunkSize(CandidateHash, oneshot::Sender>), @@ -500,7 +500,7 @@ pub enum AvailabilityStoreMessage { /// This is useful in cases like bitfield signing, when existence /// matters, but we don't want to necessarily pass around large /// quantities of data to get a single bit of information. - QueryChunkAvailability(CandidateHash, ValidatorIndex, oneshot::Sender), + QueryChunkAvailability(CandidateHash, ChunkIndex, oneshot::Sender), /// Store an `ErasureChunk` in the AV store. /// diff --git a/polkadot/node/subsystem-util/src/lib.rs b/polkadot/node/subsystem-util/src/lib.rs index dcd259396ff3..95cd840e9525 100644 --- a/polkadot/node/subsystem-util/src/lib.rs +++ b/polkadot/node/subsystem-util/src/lib.rs @@ -555,9 +555,7 @@ impl ChunkIndexCacheRegistry { n_validators: usize, ) -> Vec { let mut indices: Vec<_> = (0..n_validators) - .map(|i| { - ValidatorIndex(u32::try_from(i).expect("validator count should not exceed u32")) - }) + .map(|i| ChunkIndex(u32::try_from(i).expect("validator count should not exceed u32"))) .collect(); if let Some(features) = maybe_client_features { diff --git a/polkadot/primitives/src/v5/mod.rs b/polkadot/primitives/src/v5/mod.rs index 7c3ee4f6517f..a08c83b86f63 100644 --- a/polkadot/primitives/src/v5/mod.rs +++ b/polkadot/primitives/src/v5/mod.rs @@ -109,9 +109,27 @@ pub trait TypeIndex { #[cfg_attr(feature = "std", derive(Serialize, Deserialize, Hash))] pub struct ValidatorIndex(pub u32); -/// Type alias for chunk index. The chunk index held by a validator may not always be equal to its -/// `ValidatorIndex`, so we use a separate type to make code easier to read. -pub type ChunkIndex = ValidatorIndex; +/// Index of an availability chunk. +/// +/// The underlying type is identical to `ValidatorIndex`, because +/// the number of chunks will always be equal to the number of validators. +/// However, the chunk index held by a validator may not always be equal to its `ValidatorIndex`, so +/// we use a separate type to make code easier to read. +#[derive(Eq, Ord, PartialEq, PartialOrd, Copy, Clone, Encode, Decode, TypeInfo, RuntimeDebug)] +#[cfg_attr(feature = "std", derive(Serialize, Deserialize, Hash))] +pub struct ChunkIndex(pub u32); + +impl From for ValidatorIndex { + fn from(c_index: ChunkIndex) -> Self { + ValidatorIndex(c_index.0) + } +} + +impl From for ChunkIndex { + fn from(v_index: ValidatorIndex) -> Self { + ChunkIndex(v_index.0) + } +} // We should really get https://github.com/paritytech/polkadot/issues/2403 going .. impl From for ValidatorIndex { From f28e01ed587193c19c400cdce9e62a2a9d4d9f5f Mon Sep 17 00:00:00 2001 From: alindima Date: Wed, 27 Sep 2023 18:00:39 +0300 Subject: [PATCH 038/126] add v10 config migration to westend --- polkadot/runtime/westend/src/lib.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/polkadot/runtime/westend/src/lib.rs b/polkadot/runtime/westend/src/lib.rs index acbb79d97504..a4190ad72a82 100644 --- a/polkadot/runtime/westend/src/lib.rs +++ b/polkadot/runtime/westend/src/lib.rs @@ -1508,6 +1508,7 @@ pub mod migrations { parachains_configuration::migration::v9::MigrateToV9, paras_registrar::migration::VersionCheckedMigrateToV1, pallet_referenda::migration::v1::MigrateV0ToV1, + parachains_configuration::migration::v10::MigrateToV10, ); } From 21811ef122a5cba22020e5e3846a60502b16c1ba Mon Sep 17 00:00:00 2001 From: alindima Date: Thu, 28 Sep 2023 09:38:35 +0300 Subject: [PATCH 039/126] replace u8 ClientFeatures with u64 --- polkadot/primitives/src/vstaging/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/polkadot/primitives/src/vstaging/mod.rs b/polkadot/primitives/src/vstaging/mod.rs index 5a211fe2d319..649d62d6bc40 100644 --- a/polkadot/primitives/src/vstaging/mod.rs +++ b/polkadot/primitives/src/vstaging/mod.rs @@ -25,7 +25,7 @@ use serde::{Deserialize, Serialize}; bitflags::bitflags! { #[derive(Default, TypeInfo, Encode, Decode, Serialize, Deserialize)] /// Bit indices in the `HostCoonfiguration.client_features` that correspond to different client features. - pub struct ClientFeatures: u8 { + pub struct ClientFeatures: u64 { /// Is availability chunk shuffling enabled. const AVAILABILITY_CHUNK_SHUFFLING = 0b1; } From 67954e6176561fb5ca3f8589b44cf31070be0c90 Mon Sep 17 00:00:00 2001 From: alindima Date: Thu, 28 Sep 2023 10:01:51 +0300 Subject: [PATCH 040/126] some fixes to get clippy to pass --- .../emulated/common/src/lib.rs | 2 +- .../node/core/bitfield-signing/src/lib.rs | 60 +++++---- .../node/core/bitfield-signing/src/tests.rs | 13 +- .../src/requester/fetch_task/tests.rs | 6 +- polkadot/node/subsystem-util/src/tests.rs | 119 +++++++++--------- 5 files changed, 109 insertions(+), 91 deletions(-) diff --git a/cumulus/parachains/integration-tests/emulated/common/src/lib.rs b/cumulus/parachains/integration-tests/emulated/common/src/lib.rs index 34547f90d221..af1730ca9452 100644 --- a/cumulus/parachains/integration-tests/emulated/common/src/lib.rs +++ b/cumulus/parachains/integration-tests/emulated/common/src/lib.rs @@ -23,7 +23,7 @@ use constants::{ bridge_hub_polkadot, bridge_hub_rococo, collectives, kusama, penpal, polkadot, rococo, westend, }; use impls::{RococoWococoMessageHandler, WococoRococoMessageHandler}; -use polkadot_primitives::runtime_api::runtime_decl_for_parachain_host::ParachainHostV7; +use polkadot_primitives::runtime_api::runtime_decl_for_parachain_host::ParachainHostV8; // Substrate use frame_support::traits::OnInitialize; diff --git a/polkadot/node/core/bitfield-signing/src/lib.rs b/polkadot/node/core/bitfield-signing/src/lib.rs index bd659cb67b8a..aeda42a814ce 100644 --- a/polkadot/node/core/bitfield-signing/src/lib.rs +++ b/polkadot/node/core/bitfield-signing/src/lib.rs @@ -36,7 +36,9 @@ use polkadot_node_subsystem_util::{ self as util, availability_chunk_index, request_validators, runtime::request_client_features, Validator, }; -use polkadot_primitives::{AvailabilityBitfield, BlockNumber, CoreState, Hash, ValidatorIndex}; +use polkadot_primitives::{ + vstaging::ClientFeatures, AvailabilityBitfield, BlockNumber, CoreState, Hash, ValidatorIndex, +}; use sp_keystore::{Error as KeystoreError, KeystorePtr}; use std::{collections::HashMap, iter::FromIterator, time::Duration}; use util::{ @@ -113,34 +115,46 @@ async fn get_core_availability( return Ok(false) }; - let session_index = - recv_runtime(request_session_index_for_child(relay_parent, *sender.lock().await).await) - .await?; - - let Some(session_info) = recv_runtime( - request_session_info(relay_parent, session_index, *sender.lock().await).await, - ) - .await? - else { - gum::warn!( - target: LOG_TARGET, - ?relay_parent, - session_index, - ?core.candidate_hash, - para_id = %core.para_id(), - "Failed to get session info." - ); - - return Ok(false) - }; - let maybe_client_features = request_client_features(relay_parent, *sender.lock().await) .await .map_err(Error::from)?; + // Init this to all zeros. It won't be used unless + // `ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING` is enabled. We do this to avoid querying + // the runtime API for session index and session info unless the feature is enabled. + let mut babe_randomness = [0; 32]; + + if let Some(client_features) = maybe_client_features { + if client_features.contains(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING) { + let session_index = recv_runtime( + request_session_index_for_child(relay_parent, *sender.lock().await).await, + ) + .await?; + + let Some(session_info) = recv_runtime( + request_session_info(relay_parent, session_index, *sender.lock().await).await, + ) + .await? + else { + gum::warn!( + target: LOG_TARGET, + ?relay_parent, + session_index, + ?core.candidate_hash, + para_id = %core.para_id(), + "Failed to get session info." + ); + + return Ok(false) + }; + + babe_randomness = session_info.random_seed; + } + } + let chunk_index = availability_chunk_index( maybe_client_features.as_ref(), - session_info.random_seed, + babe_randomness, n_validators, block_number, core.para_id(), diff --git a/polkadot/node/core/bitfield-signing/src/tests.rs b/polkadot/node/core/bitfield-signing/src/tests.rs index 02173b65d4f9..ef97c00bfe4a 100644 --- a/polkadot/node/core/bitfield-signing/src/tests.rs +++ b/polkadot/node/core/bitfield-signing/src/tests.rs @@ -16,8 +16,8 @@ use super::*; use futures::{executor::block_on, pin_mut, StreamExt}; -use polkadot_node_subsystem::messages::AllMessages; -use polkadot_primitives::{CandidateHash, OccupiedCore}; +use polkadot_node_subsystem::messages::{AllMessages, RuntimeApiMessage, RuntimeApiRequest}; +use polkadot_primitives::{vstaging::ClientFeatures, CandidateHash, OccupiedCore}; use test_helpers::dummy_candidate_descriptor; fn occupied_core(para_id: u32, candidate_hash: CandidateHash) -> CoreState { @@ -62,10 +62,15 @@ fn construct_availability_bitfield_works() { assert_eq!(relay_parent, rp); tx.send(Ok(vec![CoreState::Free, occupied_core(1, hash_a), occupied_core(2, hash_b)])).unwrap(); } + AllMessages::RuntimeApi( + RuntimeApiMessage::Request(_, RuntimeApiRequest::ClientFeatures(tx)), + ) => { + tx.send(Ok(ClientFeatures::empty())).unwrap(); + } AllMessages::AvailabilityStore( - AvailabilityStoreMessage::QueryChunkAvailability(c_hash, vidx, tx), + AvailabilityStoreMessage::QueryChunkAvailability(c_hash, cidx, tx), ) => { - assert_eq!(validator_index, vidx); + assert_eq!(validator_index, cidx.into()); tx.send(c_hash == hash_a).unwrap(); }, diff --git a/polkadot/node/network/availability-distribution/src/requester/fetch_task/tests.rs b/polkadot/node/network/availability-distribution/src/requester/fetch_task/tests.rs index c7ef74e439d7..88014b80f016 100644 --- a/polkadot/node/network/availability-distribution/src/requester/fetch_task/tests.rs +++ b/polkadot/node/network/availability-distribution/src/requester/fetch_task/tests.rs @@ -31,7 +31,7 @@ use sp_keyring::Sr25519Keyring; use polkadot_node_network_protocol::request_response::{v1, Recipient}; use polkadot_node_primitives::{BlockData, PoV, Proof}; use polkadot_node_subsystem::messages::AllMessages; -use polkadot_primitives::{CandidateHash, ValidatorIndex}; +use polkadot_primitives::{CandidateHash, ChunkIndex}; use super::*; use crate::{metrics::Metrics, tests::mock::get_valid_chunk_data}; @@ -109,7 +109,7 @@ fn task_does_not_accept_wrongly_indexed_chunk() { let pov = PoV { block_data: BlockData(vec![45, 46, 47]) }; let (root_hash, chunk) = get_valid_chunk_data(pov); task.erasure_root = root_hash; - task.request.index = ValidatorIndex(chunk.index.0 + 1); + task.request.index = ChunkIndex(chunk.index.0 + 1); let validators = vec![Sr25519Keyring::Alice.public().into()]; task.group = validators; @@ -285,7 +285,7 @@ fn get_test_running_task() -> (RunningTask, mpsc::Receiver) { group: Vec::new(), request: ChunkFetchingRequest { candidate_hash: CandidateHash([43u8; 32].into()), - index: ValidatorIndex(0), + index: ChunkIndex(0), }, erasure_root: Hash::repeat_byte(99), relay_parent: Hash::repeat_byte(71), diff --git a/polkadot/node/subsystem-util/src/tests.rs b/polkadot/node/subsystem-util/src/tests.rs index ff9809541285..859cc32e0a17 100644 --- a/polkadot/node/subsystem-util/src/tests.rs +++ b/polkadot/node/subsystem-util/src/tests.rs @@ -21,7 +21,6 @@ use executor::block_on; use futures::{channel::mpsc, executor, FutureExt, SinkExt, StreamExt}; use polkadot_primitives_test_helpers::AlwaysZeroRng; use std::{ - collections::HashSet, sync::{ atomic::{AtomicUsize, Ordering}, Arc, @@ -96,62 +95,62 @@ fn subset_predefined_generation_check() { } } -#[test] -fn test_availability_chunk_indices() { - let block_number = 89; - let n_validators = 11; - let babe_randomness = [12u8; 32]; - let n_cores = 4; - - let client_features = Some(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING); - let (shuffle, core_start_pos) = availability_chunk_indices( - client_features, - block_number, - babe_randomness, - n_validators, - n_cores, - ); - // assert_eq!(shuffle, (0..n_validators).map(|i| ValidatorIndex(i as u32)).collect::>()); - // assert_eq!(core_start_pos, repeat(ValidatorIndex(0)).take(n_cores).collect::>()); - - for index in 0..n_cores { - for validator in 0..n_validators { - let chunk_index = availability_chunk_index( - client_features, - block_number, - babe_randomness, - n_validators, - CoreIndex(index as u32), - n_cores, - ValidatorIndex(validator as u32), - ); - - assert_eq!( - &chunk_index, - (shuffle - .iter() - .cycle() - .skip(core_start_pos[index].0 as usize) - .take(n_validators) - .collect::>())[validator] - ); - } - } - - // let shuffle = - // availability_chunk_indices(Some(ClientFeatures::empty()), block_number, n_validators); - // assert_eq!(shuffle, (0..n_validators).map(|i| ValidatorIndex(i as u32)).collect::>()); - - // let shuffle = - // availability_chunk_indices(ClientFeatures::from_bits(0b10), block_number, n_validators); - // assert_eq!(shuffle, (0..n_validators).map(|i| ValidatorIndex(i as u32)).collect::>()); - - // let shuffle = availability_chunk_indices( - // Some(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING), - // block_number, - // n_validators, - // ); - // assert_ne!(shuffle, (0..n_validators).map(|i| ValidatorIndex(i as u32)).collect::>()); - // assert_eq!(shuffle.len(), n_validators); - // assert_eq!(shuffle.iter().collect::>().len(), n_validators); -} +// #[test] +// fn test_availability_chunk_indices() { +// let block_number = 89; +// let n_validators = 11; +// let babe_randomness = [12u8; 32]; +// let n_cores = 4; + +// let client_features = Some(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING); +// let (shuffle, core_start_pos) = availability_chunk_indices( +// client_features, +// block_number, +// babe_randomness, +// n_validators, +// n_cores, +// ); +// // assert_eq!(shuffle, (0..n_validators).map(|i| ValidatorIndex(i as u32)).collect::>()); +// // assert_eq!(core_start_pos, repeat(ValidatorIndex(0)).take(n_cores).collect::>()); + +// for index in 0..n_cores { +// for validator in 0..n_validators { +// let chunk_index = availability_chunk_index( +// client_features, +// block_number, +// babe_randomness, +// n_validators, +// CoreIndex(index as u32), +// n_cores, +// ValidatorIndex(validator as u32), +// ); + +// assert_eq!( +// &chunk_index, +// (shuffle +// .iter() +// .cycle() +// .skip(core_start_pos[index].0 as usize) +// .take(n_validators) +// .collect::>())[validator] +// ); +// } +// } + +// // let shuffle = +// // availability_chunk_indices(Some(ClientFeatures::empty()), block_number, n_validators); +// // assert_eq!(shuffle, (0..n_validators).map(|i| ValidatorIndex(i as u32)).collect::>()); + +// // let shuffle = +// // availability_chunk_indices(ClientFeatures::from_bits(0b10), block_number, n_validators); +// // assert_eq!(shuffle, (0..n_validators).map(|i| ValidatorIndex(i as u32)).collect::>()); + +// // let shuffle = availability_chunk_indices( +// // Some(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING), +// // block_number, +// // n_validators, +// // ); +// // assert_ne!(shuffle, (0..n_validators).map(|i| ValidatorIndex(i as u32)).collect::>()); +// // assert_eq!(shuffle.len(), n_validators); +// // assert_eq!(shuffle.iter().collect::>().len(), n_validators); +// } From c93985bb6ab23f6b141f9bd81431a9be68f5ae17 Mon Sep 17 00:00:00 2001 From: alindima Date: Thu, 28 Sep 2023 10:29:59 +0300 Subject: [PATCH 041/126] use Pallet::current_storage_version()::put() instead of explicitly specifying the version on upgrade --- .../runtime/parachains/src/configuration/migration/v10.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/polkadot/runtime/parachains/src/configuration/migration/v10.rs b/polkadot/runtime/parachains/src/configuration/migration/v10.rs index 14fbbfe31ef2..d883928b59ad 100644 --- a/polkadot/runtime/parachains/src/configuration/migration/v10.rs +++ b/polkadot/runtime/parachains/src/configuration/migration/v10.rs @@ -72,11 +72,11 @@ impl OnRuntimeUpgrade for MigrateToV10 { fn on_runtime_upgrade() -> Weight { log::info!(target: configuration::LOG_TARGET, "HostConfiguration MigrateToV10 started"); - if StorageVersion::get::>() == 9 { + if Pallet::::on_chain_storage_version() == StorageVersion::new(9) { let weight_consumed = migrate_to_v10::(); log::info!(target: configuration::LOG_TARGET, "HostConfiguration MigrateToV10 executed successfully"); - StorageVersion::new(10).put::>(); + Pallet::::current_storage_version().put::>(); weight_consumed } else { @@ -89,7 +89,7 @@ impl OnRuntimeUpgrade for MigrateToV10 { fn post_upgrade(_state: Vec) -> Result<(), sp_runtime::TryRuntimeError> { log::trace!(target: crate::configuration::LOG_TARGET, "Running post_upgrade() for HostConfiguration MigrateToV10"); ensure!( - StorageVersion::get::>() >= 10, + Pallet::::on_chain_storage_version() >= StorageVersion::new(10) "Storage version should be >= 10 after the migration" ); From 201b09e68bc873ecb2ce28d070710f390c34b659 Mon Sep 17 00:00:00 2001 From: alindima Date: Thu, 28 Sep 2023 16:07:41 +0300 Subject: [PATCH 042/126] replace RecoveryError::Unavailable with Invalid --- polkadot/node/network/availability-recovery/src/task.rs | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 158d034854a7..06043b201c51 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -715,10 +715,7 @@ impl FetchSystematicChunks { "Systematic data recovery error - root mismatch", ); - // Return a non-fatal error, because we may have gotten a non-systematic chunk - // from a validator by mistake. We may have enough chunks for the next strategy - // to pass. - Err(RecoveryError::Unavailable) + Err(RecoveryError::Invalid) } }, Err(err) => { @@ -733,9 +730,7 @@ impl FetchSystematicChunks { "Systematic data recovery error ", ); - // Return a non-fatal error, because we may have gotten a non-systematic chunk from - // a validator by mistake. We may have enough chunks for the next strategy to pass. - Err(RecoveryError::Unavailable) + Err(RecoveryError::Invalid) }, } } From 9c51bf29fc1aa0e05e816af7a3934be441a1b726 Mon Sep 17 00:00:00 2001 From: alindima Date: Thu, 28 Sep 2023 16:08:23 +0300 Subject: [PATCH 043/126] add tests to availability-distribution --- Cargo.lock | 36 +++++++++++++ .../availability-distribution/Cargo.toml | 1 + .../src/requester/fetch_task/tests.rs | 16 +++--- .../src/requester/tests.rs | 21 ++++++-- .../src/tests/mock.rs | 18 ++++--- .../src/tests/mod.rs | 27 ++++++---- .../src/tests/state.rs | 51 +++++++++++++++---- 7 files changed, 133 insertions(+), 37 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index bc1990a6c563..03ee2ec55b4d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -11530,6 +11530,7 @@ dependencies = [ "polkadot-primitives", "polkadot-primitives-test-helpers", "rand 0.8.5", + "rstest", "sc-network", "schnellru", "sp-core", @@ -13912,6 +13913,12 @@ version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e5ea92a5b6195c6ef2a0295ea818b312502c6fc94dde986c5553242e18fd4ce2" +[[package]] +name = "relative-path" +version = "1.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c707298afce11da2efef2f600116fa93ffa7a032b5d7b628aa17711ec81383ca" + [[package]] name = "remote-ext-tests-bags-list" version = "1.0.0" @@ -14231,6 +14238,35 @@ dependencies = [ "winapi", ] +[[package]] +name = "rstest" +version = "0.18.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "97eeab2f3c0a199bc4be135c36c924b6590b88c377d416494288c14f2db30199" +dependencies = [ + "futures", + "futures-timer", + "rstest_macros", + "rustc_version 0.4.0", +] + +[[package]] +name = "rstest_macros" +version = "0.18.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d428f8247852f894ee1be110b375111b586d4fa431f6c46e64ba5a0dcccbe605" +dependencies = [ + "cfg-if", + "glob", + "proc-macro2", + "quote", + "regex", + "relative-path", + "rustc_version 0.4.0", + "syn 2.0.37", + "unicode-ident", +] + [[package]] name = "rtcp" version = "0.7.2" diff --git a/polkadot/node/network/availability-distribution/Cargo.toml b/polkadot/node/network/availability-distribution/Cargo.toml index c3c7aa4e0ea5..339e20c5ead6 100644 --- a/polkadot/node/network/availability-distribution/Cargo.toml +++ b/polkadot/node/network/availability-distribution/Cargo.toml @@ -32,3 +32,4 @@ sc-network = { path = "../../../../substrate/client/network" } futures-timer = "3.0.2" assert_matches = "1.4.0" polkadot-primitives-test-helpers = { path = "../../../primitives/test-helpers" } +rstest = "0.18.2" diff --git a/polkadot/node/network/availability-distribution/src/requester/fetch_task/tests.rs b/polkadot/node/network/availability-distribution/src/requester/fetch_task/tests.rs index 88014b80f016..db6835848a0b 100644 --- a/polkadot/node/network/availability-distribution/src/requester/fetch_task/tests.rs +++ b/polkadot/node/network/availability-distribution/src/requester/fetch_task/tests.rs @@ -74,12 +74,12 @@ fn task_does_not_accept_invalid_chunk() { #[test] fn task_stores_valid_chunk() { let (mut task, rx) = get_test_running_task(); + let validators = vec![Sr25519Keyring::Alice.public().into()]; let pov = PoV { block_data: BlockData(vec![45, 46, 47]) }; - let (root_hash, chunk) = get_valid_chunk_data(pov); + let (root_hash, chunk) = get_valid_chunk_data(pov, 10, ChunkIndex(0)); task.erasure_root = root_hash; task.request.index = chunk.index.into(); - let validators = vec![Sr25519Keyring::Alice.public().into()]; task.group = validators; let test = TestRun { @@ -106,12 +106,12 @@ fn task_stores_valid_chunk() { #[test] fn task_does_not_accept_wrongly_indexed_chunk() { let (mut task, rx) = get_test_running_task(); + let validators = vec![Sr25519Keyring::Alice.public().into()]; let pov = PoV { block_data: BlockData(vec![45, 46, 47]) }; - let (root_hash, chunk) = get_valid_chunk_data(pov); + let (root_hash, chunk) = get_valid_chunk_data(pov, 10, ChunkIndex(0)); task.erasure_root = root_hash; task.request.index = ChunkIndex(chunk.index.0 + 1); - let validators = vec![Sr25519Keyring::Alice.public().into()]; task.group = validators; let test = TestRun { @@ -136,9 +136,6 @@ fn task_does_not_accept_wrongly_indexed_chunk() { fn task_stores_valid_chunk_if_there_is_one() { let (mut task, rx) = get_test_running_task(); let pov = PoV { block_data: BlockData(vec![45, 46, 47]) }; - let (root_hash, chunk) = get_valid_chunk_data(pov); - task.erasure_root = root_hash; - task.request.index = chunk.index.into(); let validators = [ // Only Alice has valid chunk - should succeed, even though she is tried last. @@ -151,6 +148,11 @@ fn task_stores_valid_chunk_if_there_is_one() { .iter() .map(|v| v.public().into()) .collect::>(); + + let (root_hash, chunk) = get_valid_chunk_data(pov, 10, ChunkIndex(0)); + task.erasure_root = root_hash; + task.request.index = chunk.index.into(); + task.group = validators; let test = TestRun { diff --git a/polkadot/node/network/availability-distribution/src/requester/tests.rs b/polkadot/node/network/availability-distribution/src/requester/tests.rs index c4252b4e439e..21d24c6a340a 100644 --- a/polkadot/node/network/availability-distribution/src/requester/tests.rs +++ b/polkadot/node/network/availability-distribution/src/requester/tests.rs @@ -25,8 +25,8 @@ use polkadot_node_primitives::{BlockData, ErasureChunk, PoV}; use polkadot_node_subsystem_test_helpers::mock::new_leaf; use polkadot_node_subsystem_util::runtime::RuntimeInfo; use polkadot_primitives::{ - BlockNumber, CoreState, ExecutorParams, GroupIndex, Hash, Id as ParaId, ScheduledCore, - SessionIndex, SessionInfo, + vstaging::ClientFeatures, BlockNumber, ChunkIndex, CoreState, ExecutorParams, GroupIndex, Hash, + Id as ParaId, ScheduledCore, SessionIndex, SessionInfo, }; use sp_core::traits::SpawnNamed; @@ -50,7 +50,7 @@ use super::Requester; fn get_erasure_chunk() -> ErasureChunk { let pov = PoV { block_data: BlockData(vec![45, 46, 47]) }; - get_valid_chunk_data(pov).1 + get_valid_chunk_data(pov, 10, ChunkIndex(0)).1 } #[derive(Clone)] @@ -125,6 +125,10 @@ fn spawn_virtual_overseer( tx.send(Ok(Some(ExecutorParams::default()))) .expect("Receiver should be alive."); }, + RuntimeApiRequest::ClientFeatures(tx) => { + tx.send(Ok(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING)) + .expect("Receiver should be alive."); + }, RuntimeApiRequest::AvailabilityCores(tx) => { let para_id = ParaId::from(1_u32); let maybe_block_position = @@ -142,6 +146,8 @@ fn spawn_virtual_overseer( group_responsible: GroupIndex(1), para_id, relay_parent: hash, + n_validators: 10, + chunk_index: ChunkIndex(0), } .build() .0, @@ -172,6 +178,15 @@ fn spawn_virtual_overseer( .send(Ok(ancestors)) .expect("Receiver is expected to be alive"); }, + AllMessages::ChainApi(ChainApiMessage::BlockNumber(hash, response_channel)) => { + response_channel + .send(Ok(test_state + .relay_chain + .iter() + .position(|h| *h == hash) + .map(|pos| pos as u32))) + .expect("Receiver is expected to be alive"); + }, msg => panic!("Unexpected overseer message: {:?}", msg), } } diff --git a/polkadot/node/network/availability-distribution/src/tests/mock.rs b/polkadot/node/network/availability-distribution/src/tests/mock.rs index 9077658c28b0..b41c493a1072 100644 --- a/polkadot/node/network/availability-distribution/src/tests/mock.rs +++ b/polkadot/node/network/availability-distribution/src/tests/mock.rs @@ -75,13 +75,16 @@ pub struct OccupiedCoreBuilder { pub group_responsible: GroupIndex, pub para_id: ParaId, pub relay_parent: Hash, + pub n_validators: usize, + pub chunk_index: ChunkIndex, } impl OccupiedCoreBuilder { pub fn build(self) -> (OccupiedCore, (CandidateHash, ErasureChunk)) { let pov = PoV { block_data: BlockData(vec![45, 46, 47]) }; let pov_hash = pov.hash(); - let (erasure_root, chunk) = get_valid_chunk_data(pov.clone()); + let (erasure_root, chunk) = + get_valid_chunk_data(pov.clone(), self.n_validators, self.chunk_index); let candidate_receipt = TestCandidateBuilder { para_id: self.para_id, pov_hash, @@ -133,8 +136,11 @@ impl TestCandidateBuilder { } // Get chunk for index 0 -pub fn get_valid_chunk_data(pov: PoV) -> (Hash, ErasureChunk) { - let fake_validator_count = 10; +pub fn get_valid_chunk_data( + pov: PoV, + n_validators: usize, + chunk_index: ChunkIndex, +) -> (Hash, ErasureChunk) { let persisted = PersistedValidationData { parent_head: HeadData(vec![7, 8, 9]), relay_parent_number: Default::default(), @@ -142,7 +148,7 @@ pub fn get_valid_chunk_data(pov: PoV) -> (Hash, ErasureChunk) { relay_parent_storage_root: Default::default(), }; let available_data = AvailableData { validation_data: persisted, pov: Arc::new(pov) }; - let chunks = obtain_chunks(fake_validator_count, &available_data).unwrap(); + let chunks = obtain_chunks(n_validators, &available_data).unwrap(); let branches = branches(chunks.as_ref()); let root = branches.root(); let chunk = branches @@ -152,7 +158,7 @@ pub fn get_valid_chunk_data(pov: PoV) -> (Hash, ErasureChunk) { index: ChunkIndex(index as _), proof: Proof::try_from(proof).unwrap(), }) - .next() - .expect("There really should be 10 chunks."); + .nth(chunk_index.0 as usize) + .expect("There really should be enough chunks."); (root, chunk) } diff --git a/polkadot/node/network/availability-distribution/src/tests/mod.rs b/polkadot/node/network/availability-distribution/src/tests/mod.rs index d0aee7e5e010..f830d6c0bb8c 100644 --- a/polkadot/node/network/availability-distribution/src/tests/mod.rs +++ b/polkadot/node/network/availability-distribution/src/tests/mod.rs @@ -17,9 +17,10 @@ use std::collections::HashSet; use futures::{executor, future, Future}; +use rstest::rstest; use polkadot_node_network_protocol::request_response::{IncomingRequest, ReqProtocolNames}; -use polkadot_primitives::{CoreState, Hash}; +use polkadot_primitives::{vstaging::ClientFeatures, CoreState, Hash}; use sp_keystore::KeystorePtr; use polkadot_node_subsystem_test_helpers as test_helpers; @@ -65,16 +66,20 @@ fn test_harness>( /// Simple basic check, whether the subsystem works as expected. /// /// Exceptional cases are tested as unit tests in `fetch_task`. -#[test] -fn check_basic() { - let state = TestState::default(); +#[rstest] +#[case(ClientFeatures::empty())] +#[case(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING)] +fn check_basic(#[case] client_features: ClientFeatures) { + let state = TestState::new(client_features); test_harness(state.keystore.clone(), move |harness| state.run(harness)); } /// Check whether requester tries all validators in group. -#[test] -fn check_fetch_tries_all() { - let mut state = TestState::default(); +#[rstest] +#[case(ClientFeatures::empty())] +#[case(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING)] +fn check_fetch_tries_all(#[case] client_features: ClientFeatures) { + let mut state = TestState::new(client_features); for (_, v) in state.chunks.iter_mut() { // 4 validators in group, so this should still succeed: v.push(None); @@ -88,9 +93,11 @@ fn check_fetch_tries_all() { /// /// Check that requester will retry the fetch on error on the next block still pending /// availability. -#[test] -fn check_fetch_retry() { - let mut state = TestState::default(); +#[rstest] +#[case(ClientFeatures::empty())] +#[case(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING)] +fn check_fetch_retry(#[case] client_features: ClientFeatures) { + let mut state = TestState::new(client_features); state .cores .insert(state.relay_chain[2], state.cores.get(&state.relay_chain[1]).unwrap().clone()); diff --git a/polkadot/node/network/availability-distribution/src/tests/state.rs b/polkadot/node/network/availability-distribution/src/tests/state.rs index 0a90a785e08f..d27c0bdd0e1b 100644 --- a/polkadot/node/network/availability-distribution/src/tests/state.rs +++ b/polkadot/node/network/availability-distribution/src/tests/state.rs @@ -20,7 +20,7 @@ use std::{ }; use polkadot_node_subsystem_test_helpers::TestSubsystemContextHandle; -use polkadot_node_subsystem_util::TimeoutExt; +use polkadot_node_subsystem_util::{availability_chunk_index, TimeoutExt}; use futures::{ channel::{mpsc, oneshot}, @@ -46,8 +46,8 @@ use polkadot_node_subsystem::{ }; use polkadot_node_subsystem_test_helpers as test_helpers; use polkadot_primitives::{ - CandidateHash, ChunkIndex, CoreState, ExecutorParams, GroupIndex, Hash, Id as ParaId, - ScheduledCore, SessionInfo, + vstaging::ClientFeatures, CandidateHash, ChunkIndex, CoreState, ExecutorParams, GroupIndex, + Hash, Id as ParaId, ScheduledCore, SessionInfo, ValidatorIndex, }; use test_helpers::mock::{make_ferdie_keystore, new_leaf}; @@ -82,10 +82,12 @@ pub struct TestState { /// Cores per relay chain block. pub cores: HashMap>, pub keystore: KeystorePtr, + pub client_features: ClientFeatures, } -impl Default for TestState { - fn default() -> Self { +impl TestState { + /// Initialize a default test state. + pub fn new(client_features: ClientFeatures) -> Self { let relay_chain: Vec<_> = (1u8..10).map(Hash::repeat_byte).collect(); let chain_a = ParaId::from(1); let chain_b = ParaId::from(2); @@ -113,7 +115,17 @@ impl Default for TestState { advanced.next(); relay_chain.iter().zip(advanced) }; - for (relay_parent, relay_child) in heads { + for (block_number, (relay_parent, relay_child)) in heads.enumerate() { + let our_chunk_index = availability_chunk_index( + Some(&client_features), + session_info.random_seed, + session_info.validators.len(), + block_number as u32, + // Use the second para id, as we're in the first group and hold the entire POV. + 2.into(), + ValidatorIndex(0), + ); + let (p_cores, p_chunks): (Vec<_>, Vec<_>) = chain_ids .iter() .enumerate() @@ -122,6 +134,8 @@ impl Default for TestState { group_responsible: GroupIndex(i as _), para_id: *para_id, relay_parent: *relay_parent, + n_validators: session_info.validators.len(), + chunk_index: our_chunk_index, } .build(); (CoreState::Occupied(core), chunk) @@ -144,11 +158,10 @@ impl Default for TestState { session_info, cores, keystore, + client_features, } } -} -impl TestState { /// Run, but fail after some timeout. pub async fn run(self, harness: TestHarness) { // Make sure test won't run forever. @@ -226,12 +239,12 @@ impl TestState { }, AllMessages::AvailabilityStore(AvailabilityStoreMessage::QueryChunk( candidate_hash, - validator_index, + chunk_index, tx, )) => { let chunk = self .chunks - .get_mut(&(candidate_hash, validator_index)) + .get_mut(&(candidate_hash, chunk_index)) .and_then(Vec::pop) .flatten(); tx.send(chunk).expect("Receiver is expected to be alive"); @@ -269,6 +282,10 @@ impl TestState { tx.send(Ok(self.cores[&hash].clone())) .expect("Receiver should still be alive"); }, + RuntimeApiRequest::ClientFeatures(tx) => { + tx.send(Ok(self.client_features)) + .expect("Receiver should still be alive"); + }, _ => { panic!("Unexpected runtime request: {:?}", req); }, @@ -282,7 +299,19 @@ impl TestState { .unwrap_or_default(); response_channel.send(Ok(ancestors)).expect("Receiver is expected to be alive"); }, - _ => {}, + AllMessages::ChainApi(ChainApiMessage::BlockNumber(hash, response_channel)) => { + response_channel + .send(Ok(self + .relay_chain + .iter() + .position(|h| *h == hash) + .map(|pos| pos as u32))) + .expect("Receiver is expected to be alive"); + }, + + _ => { + panic!("Received unexpected message") + }, } } From fabc0446e57be67560501383f47f9d8d67f67e69 Mon Sep 17 00:00:00 2001 From: alindima Date: Thu, 28 Sep 2023 16:40:38 +0300 Subject: [PATCH 044/126] add comma --- polkadot/runtime/parachains/src/configuration/migration/v10.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/polkadot/runtime/parachains/src/configuration/migration/v10.rs b/polkadot/runtime/parachains/src/configuration/migration/v10.rs index d883928b59ad..0cc77a1195d4 100644 --- a/polkadot/runtime/parachains/src/configuration/migration/v10.rs +++ b/polkadot/runtime/parachains/src/configuration/migration/v10.rs @@ -89,7 +89,7 @@ impl OnRuntimeUpgrade for MigrateToV10 { fn post_upgrade(_state: Vec) -> Result<(), sp_runtime::TryRuntimeError> { log::trace!(target: crate::configuration::LOG_TARGET, "Running post_upgrade() for HostConfiguration MigrateToV10"); ensure!( - Pallet::::on_chain_storage_version() >= StorageVersion::new(10) + Pallet::::on_chain_storage_version() >= StorageVersion::new(10), "Storage version should be >= 10 after the migration" ); From 42832146303e47b06afa157442ea43696db91835 Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 3 Oct 2023 12:49:06 +0200 Subject: [PATCH 045/126] fix migration test --- polkadot/runtime/parachains/src/configuration/migration/v10.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/polkadot/runtime/parachains/src/configuration/migration/v10.rs b/polkadot/runtime/parachains/src/configuration/migration/v10.rs index 0cc77a1195d4..50ed66b9fd83 100644 --- a/polkadot/runtime/parachains/src/configuration/migration/v10.rs +++ b/polkadot/runtime/parachains/src/configuration/migration/v10.rs @@ -204,7 +204,7 @@ mod tests { // doesn't need to be read and also leaving it as one line allows to easily copy it. let raw_config = hex_literal::hex![" - 0000300000800000080000000000100000c8000005000000050000000200000002000000000000000000000000005000000010000400000000000000000000000000000000000000000000000000000000000000000000000800000000200000040000000000100000b004000000000000000000001027000080b2e60e80c3c901809698000000000000000000000000000500000014000000040000000100000001050000000006000000640000000200000019000000000000000a0000000200000002000000050000000200000001" + 0000300000800000080000000000100000c8000005000000050000000200000002000000000000000000000000005000000010000400000000000000000000000000000000000000000000000000000000000000000000000800000000200000040000000000100000b004000000000000000000001027000080b2e60e80c3c9018096980000000000000000000000000005000000140000000400000001000000010100000000060000006400000002000000190000000000000002000000020000000200000005000000020000000100000000000000" ]; let v10 = From a9ef2a4c85263ea223714e7108a81d640cd128b9 Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 9 Oct 2023 16:45:47 +0200 Subject: [PATCH 046/126] add some more tests --- Cargo.lock | 1 + .../network/availability-recovery/Cargo.toml | 1 + .../network/availability-recovery/src/lib.rs | 77 +++-- .../availability-recovery/src/tests.rs | 303 ++++++++++++++---- 4 files changed, 300 insertions(+), 82 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index fab07abb0cb3..4ad2758c55ad 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -11489,6 +11489,7 @@ dependencies = [ "polkadot-primitives", "polkadot-primitives-test-helpers", "rand 0.8.5", + "rstest", "sc-network", "schnellru", "sp-application-crypto", diff --git a/polkadot/node/network/availability-recovery/Cargo.toml b/polkadot/node/network/availability-recovery/Cargo.toml index 42c3abef547b..3884a49cc17f 100644 --- a/polkadot/node/network/availability-recovery/Cargo.toml +++ b/polkadot/node/network/availability-recovery/Cargo.toml @@ -28,6 +28,7 @@ assert_matches = "1.4.0" env_logger = "0.9.0" futures-timer = "3.0.2" log = "0.4.17" +rstest = "0.18.2" sp-core = { path = "../../../../substrate/primitives/core" } sp-keyring = { path = "../../../../substrate/primitives/keyring" } diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index 6a1ec27e655d..e72cec275681 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -97,6 +97,9 @@ pub enum RecoveryStrategyKind { /// We try the backing group first if PoV size is lower than specified, then fallback to /// validator chunks. BackersFirstIfSizeLower(usize), + /// We try the backing group first if PoV size is lower than specified, then fallback to + /// systematic chunks. + BackersFirstIfSizeLowerThenSystematicChunks(usize), /// We always recover using validator chunks. ChunksAlways, /// First try the backing group. Then systematic chunks. @@ -487,35 +490,43 @@ async fn handle_recover( if let Some(backing_validators) = session_info.validator_groups.get(backing_group) { let mut small_pov_size = true; - if let RecoveryStrategyKind::BackersFirstIfSizeLower(small_pov_limit) = - recovery_strategy_kind - { - // Get our own chunk size to get an estimate of the PoV size. - let chunk_size: Result> = - query_chunk_size(ctx, candidate_hash).await; - if let Ok(Some(chunk_size)) = chunk_size { - let pov_size_estimate = - chunk_size.saturating_mul(session_info.validators.len()) / 3; - small_pov_size = pov_size_estimate < small_pov_limit; - - gum::trace!( - target: LOG_TARGET, - ?candidate_hash, - pov_size_estimate, - small_pov_limit, - enabled = small_pov_size, - "Prefer fetch from backing group", - ); - } else { - // we have a POV limit but were not able to query the chunk size, so - // don't use the backing group. - small_pov_size = false; - } + match recovery_strategy_kind { + RecoveryStrategyKind::BackersFirstIfSizeLower(small_pov_limit) | + RecoveryStrategyKind::BackersFirstIfSizeLowerThenSystematicChunks( + small_pov_limit, + ) => { + // Get our own chunk size to get an estimate of the PoV size. + let chunk_size: Result> = + query_chunk_size(ctx, candidate_hash).await; + if let Ok(Some(chunk_size)) = chunk_size { + let pov_size_estimate = + chunk_size.saturating_mul(session_info.validators.len()) / 3; + small_pov_size = pov_size_estimate < small_pov_limit; + + gum::trace!( + target: LOG_TARGET, + ?candidate_hash, + pov_size_estimate, + small_pov_limit, + enabled = small_pov_size, + "Prefer fetch from backing group", + ); + } else { + // we have a POV limit but were not able to query the chunk size, so + // don't use the backing group. + small_pov_size = false; + } + }, + _ => {}, }; match (&recovery_strategy_kind, small_pov_size) { (RecoveryStrategyKind::BackersFirstAlways, _) | (RecoveryStrategyKind::BackersFirstIfSizeLower(_), true) | + ( + RecoveryStrategyKind::BackersFirstIfSizeLowerThenSystematicChunks(_), + true, + ) | (RecoveryStrategyKind::BackersThenSystematicChunks, _) => recovery_strategies.push_back( Box::new(FetchFull::new(FetchFullParams { validators: backing_validators.to_vec(), @@ -530,7 +541,8 @@ async fn handle_recover( if matches!( recovery_strategy_kind, RecoveryStrategyKind::BackersThenSystematicChunks | - RecoveryStrategyKind::SystematicChunks + RecoveryStrategyKind::SystematicChunks | + RecoveryStrategyKind::BackersFirstIfSizeLowerThenSystematicChunks(_) ) { let systematic_threshold = systematic_recovery_threshold(session_info.validators.len())?; @@ -664,6 +676,21 @@ impl AvailabilityRecoverySubsystem { } } + /// Create a new instance of `AvailabilityRecoverySubsystem` which requests systematic chunks if + /// PoV is above a threshold. + pub fn with_systematic_chunks_if_pov_large( + req_receiver: IncomingRequestReceiver, + metrics: Metrics, + ) -> Self { + Self { + recovery_strategy_kind: + RecoveryStrategyKind::BackersFirstIfSizeLowerThenSystematicChunks(SMALL_POV_LIMIT), + bypass_availability_store: false, + req_receiver, + metrics, + } + } + /// Create a new instance of `AvailabilityRecoverySubsystem` which first requests full data /// from backers, with a fallback to recover from systematic chunks. pub fn with_fast_path_then_systematic_chunks( diff --git a/polkadot/node/network/availability-recovery/src/tests.rs b/polkadot/node/network/availability-recovery/src/tests.rs index 6702f431328f..826cabe1e519 100644 --- a/polkadot/node/network/availability-recovery/src/tests.rs +++ b/polkadot/node/network/availability-recovery/src/tests.rs @@ -14,23 +14,20 @@ // You should have received a copy of the GNU General Public License // along with Polkadot. If not, see . +use super::*; use std::{sync::Arc, time::Duration}; use assert_matches::assert_matches; use futures::{executor, future}; use futures_timer::Delay; +use rstest::rstest; use parity_scale_codec::Encode; +use polkadot_erasure_coding::{branches, obtain_chunks_v1 as obtain_chunks}; use polkadot_node_network_protocol::request_response::{ self as req_res, v1::AvailableDataFetchingRequest, IncomingRequest, Recipient, ReqProtocolNames, Requests, }; - -use super::*; - -use sc_network::{IfDisconnected, OutboundFailure, RequestFailure}; - -use polkadot_erasure_coding::{branches, obtain_chunks_v1 as obtain_chunks}; use polkadot_node_primitives::{BlockData, PoV, Proof}; use polkadot_node_subsystem::messages::{ AllMessages, NetworkBridgeTxMessage, RuntimeApiMessage, RuntimeApiRequest, @@ -44,6 +41,8 @@ use polkadot_primitives::{ PersistedValidationData, ValidatorId, }; use polkadot_primitives_test_helpers::{dummy_candidate_receipt, dummy_hash}; +use sc_network::{IfDisconnected, OutboundFailure, RequestFailure}; +use sp_keyring::Sr25519Keyring; type VirtualOverseer = TestSubsystemContextHandle; @@ -135,8 +134,6 @@ async fn overseer_recv( msg } -use sp_keyring::Sr25519Keyring; - #[derive(Debug)] enum Has { No, @@ -176,6 +173,10 @@ impl TestState { recovery_threshold(self.validators.len()).unwrap() } + fn systematic_threshold(&self) -> usize { + systematic_recovery_threshold(self.validators.len()).unwrap() + } + fn impossibility_threshold(&self) -> usize { self.validators.len() - self.threshold() + 1 } @@ -310,6 +311,7 @@ impl TestState { virtual_overseer: &mut VirtualOverseer, n: usize, who_has: impl Fn(usize) -> Has, + systematic_recovery: bool, ) -> Vec, RequestFailure>>> { // arbitrary order. let mut i = 0; @@ -331,10 +333,15 @@ impl TestState { Requests::ChunkFetchingV1(req) => { assert_eq!(req.payload.candidate_hash, candidate_hash); - let validator_index = req.payload.index.0 as usize; - let available_data = match who_has(validator_index) { + let chunk_index = req.payload.index.0 as usize; + + if systematic_recovery { + assert!(chunk_index <= self.systematic_threshold(), "requsted non-systematic chunk"); + } + + let available_data = match who_has(chunk_index) { Has::No => Ok(None), - Has::Yes => Ok(Some(self.chunks[validator_index].clone().into())), + Has::Yes => Ok(Some(self.chunks[chunk_index].clone().into())), Has::NetworkError(e) => Err(e), Has::DoesNotReturn => { senders.push(req.pending_response); @@ -364,7 +371,7 @@ impl TestState { ) -> Vec, RequestFailure>>> { let mut senders = Vec::new(); for _ in 0..self.validators.len() { - // Receive a request for a chunk. + // Receive a request for the full `AvailableData`. assert_matches!( overseer_recv(virtual_overseer).await, AllMessages::NetworkBridgeTx( @@ -520,11 +527,24 @@ impl Default for TestState { } } -#[test] -fn availability_is_recovered_from_chunks_if_no_group_provided() { +#[rstest] +#[case(true)] +#[case(false)] +fn availability_is_recovered_from_chunks_if_no_group_provided(#[case] systematic_recovery: bool) { let test_state = TestState::default(); - let subsystem = - AvailabilityRecoverySubsystem::with_fast_path(request_receiver(), Metrics::new_dummy()); + let (subsystem, threshold) = match systematic_recovery { + true => ( + AvailabilityRecoverySubsystem::with_fast_path_then_systematic_chunks( + request_receiver(), + Metrics::new_dummy(), + ), + test_state.systematic_threshold(), + ), + false => ( + AvailabilityRecoverySubsystem::with_fast_path(request_receiver(), Metrics::new_dummy()), + test_state.threshold(), + ), + }; test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( @@ -562,8 +582,9 @@ fn availability_is_recovered_from_chunks_if_no_group_provided() { .test_chunk_requests( candidate_hash, &mut virtual_overseer, - test_state.threshold(), + threshold, |_| Has::Yes, + systematic_recovery, ) .await; @@ -594,12 +615,28 @@ fn availability_is_recovered_from_chunks_if_no_group_provided() { test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + if systematic_recovery { + test_state + .test_chunk_requests( + new_candidate.hash(), + &mut virtual_overseer, + threshold, + |_| Has::No, + systematic_recovery, + ) + .await; + test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + } + + // Even if the recovery is systematic, we'll always fall back to regular recovery, so keep + // this around. test_state .test_chunk_requests( new_candidate.hash(), &mut virtual_overseer, test_state.impossibility_threshold(), |_| Has::No, + false, ) .await; @@ -609,11 +646,29 @@ fn availability_is_recovered_from_chunks_if_no_group_provided() { }); } -#[test] -fn availability_is_recovered_from_chunks_even_if_backing_group_supplied_if_chunks_only() { +#[rstest] +#[case(true)] +#[case(false)] +fn availability_is_recovered_from_chunks_even_if_backing_group_supplied_if_chunks_only( + #[case] systematic_recovery: bool, +) { let test_state = TestState::default(); - let subsystem = - AvailabilityRecoverySubsystem::with_chunks_only(request_receiver(), Metrics::new_dummy()); + let (subsystem, threshold) = match systematic_recovery { + true => ( + AvailabilityRecoverySubsystem::with_systematic_chunks( + request_receiver(), + Metrics::new_dummy(), + ), + test_state.systematic_threshold(), + ), + false => ( + AvailabilityRecoverySubsystem::with_chunks_only( + request_receiver(), + Metrics::new_dummy(), + ), + test_state.threshold(), + ), + }; test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( @@ -651,8 +706,9 @@ fn availability_is_recovered_from_chunks_even_if_backing_group_supplied_if_chunk .test_chunk_requests( candidate_hash, &mut virtual_overseer, - test_state.threshold(), + threshold, |_| Has::Yes, + systematic_recovery, ) .await; @@ -671,7 +727,7 @@ fn availability_is_recovered_from_chunks_even_if_backing_group_supplied_if_chunk AvailabilityRecoveryMessage::RecoverAvailableData( new_candidate.clone(), test_state.session_index, - None, + Some(GroupIndex(0)), tx, ), ) @@ -683,12 +739,28 @@ fn availability_is_recovered_from_chunks_even_if_backing_group_supplied_if_chunk test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + if systematic_recovery { + test_state + .test_chunk_requests( + new_candidate.hash(), + &mut virtual_overseer, + threshold, + |_| Has::No, + systematic_recovery, + ) + .await; + test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + } + + // Even if the recovery is systematic, we'll always fall back to regular recovery, so keep + // this around. test_state .test_chunk_requests( new_candidate.hash(), &mut virtual_overseer, test_state.impossibility_threshold(), |_| Has::No, + false, ) .await; @@ -698,11 +770,21 @@ fn availability_is_recovered_from_chunks_even_if_backing_group_supplied_if_chunk }); } -#[test] -fn bad_merkle_path_leads_to_recovery_error() { +#[rstest] +#[case(true)] +#[case(false)] +fn bad_merkle_path_leads_to_recovery_error(#[case] systematic_recovery: bool) { let mut test_state = TestState::default(); - let subsystem = - AvailabilityRecoverySubsystem::with_fast_path(request_receiver(), Metrics::new_dummy()); + let subsystem = match systematic_recovery { + true => AvailabilityRecoverySubsystem::with_systematic_chunks( + request_receiver(), + Metrics::new_dummy(), + ), + false => AvailabilityRecoverySubsystem::with_chunks_only( + request_receiver(), + Metrics::new_dummy(), + ), + }; test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( @@ -743,12 +825,26 @@ fn bad_merkle_path_leads_to_recovery_error() { test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + if systematic_recovery { + test_state + .test_chunk_requests( + candidate_hash, + &mut virtual_overseer, + test_state.systematic_threshold(), + |_| Has::No, + systematic_recovery, + ) + .await; + test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + } + test_state .test_chunk_requests( candidate_hash, &mut virtual_overseer, test_state.impossibility_threshold(), |_| Has::Yes, + false, ) .await; @@ -758,12 +854,22 @@ fn bad_merkle_path_leads_to_recovery_error() { }); } -#[test] -fn wrong_chunk_index_leads_to_recovery_error() { +#[rstest] +#[case(true)] +#[case(false)] +fn wrong_chunk_index_leads_to_recovery_error(#[case] systematic_recovery: bool) { let mut test_state = TestState::default(); - let subsystem = - AvailabilityRecoverySubsystem::with_fast_path(request_receiver(), Metrics::new_dummy()); + let subsystem = match systematic_recovery { + true => AvailabilityRecoverySubsystem::with_systematic_chunks( + request_receiver(), + Metrics::new_dummy(), + ), + false => AvailabilityRecoverySubsystem::with_chunks_only( + request_receiver(), + Metrics::new_dummy(), + ), + }; test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( @@ -794,22 +900,38 @@ fn wrong_chunk_index_leads_to_recovery_error() { let candidate_hash = test_state.candidate.hash(); - // These chunks should fail the index check as they don't have the correct index for - // validator. + // Simulate that we are holding a chunk with an invalid index in the av-store. test_state.chunks[1] = test_state.chunks[0].clone(); + + test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; + test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + + // These chunks should fail the index check as they don't have the correct index. test_state.chunks[2] = test_state.chunks[0].clone(); test_state.chunks[3] = test_state.chunks[0].clone(); test_state.chunks[4] = test_state.chunks[0].clone(); - test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; - test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + if systematic_recovery { + test_state + .test_chunk_requests( + candidate_hash, + &mut virtual_overseer, + test_state.systematic_threshold(), + |_| Has::Yes, + systematic_recovery, + ) + .await; + + test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + } test_state .test_chunk_requests( candidate_hash, &mut virtual_overseer, test_state.impossibility_threshold(), - |_| Has::No, + |_| Has::Yes, + false, ) .await; @@ -819,12 +941,25 @@ fn wrong_chunk_index_leads_to_recovery_error() { }); } -#[test] -fn invalid_erasure_coding_leads_to_invalid_error() { +#[rstest] +#[case(true)] +#[case(false)] +fn invalid_erasure_coding_leads_to_invalid_error(#[case] systematic_recovery: bool) { let mut test_state = TestState::default(); - let subsystem = - AvailabilityRecoverySubsystem::with_fast_path(request_receiver(), Metrics::new_dummy()); + let (subsystem, threshold) = match systematic_recovery { + true => ( + AvailabilityRecoverySubsystem::with_fast_path_then_systematic_chunks( + request_receiver(), + Metrics::new_dummy(), + ), + test_state.systematic_threshold(), + ), + false => ( + AvailabilityRecoverySubsystem::with_fast_path(request_receiver(), Metrics::new_dummy()), + test_state.threshold(), + ), + }; test_harness(subsystem, |mut virtual_overseer| async move { let pov = PoV { block_data: BlockData(vec![69; 64]) }; @@ -876,8 +1011,9 @@ fn invalid_erasure_coding_leads_to_invalid_error() { .test_chunk_requests( candidate_hash, &mut virtual_overseer, - test_state.threshold(), + threshold, |_| Has::Yes, + systematic_recovery, ) .await; @@ -939,13 +1075,28 @@ fn fast_path_backing_group_recovers() { }); } -#[test] -fn recovers_from_only_chunks_if_pov_large() { +#[rstest] +#[case(true)] +#[case(false)] +fn recovers_from_only_chunks_if_pov_large(#[case] systematic_recovery: bool) { let test_state = TestState::default(); - let subsystem = AvailabilityRecoverySubsystem::with_chunks_if_pov_large( - request_receiver(), - Metrics::new_dummy(), - ); + + let (subsystem, threshold) = match systematic_recovery { + true => ( + AvailabilityRecoverySubsystem::with_systematic_chunks_if_pov_large( + request_receiver(), + Metrics::new_dummy(), + ), + test_state.systematic_threshold(), + ), + false => ( + AvailabilityRecoverySubsystem::with_chunks_if_pov_large( + request_receiver(), + Metrics::new_dummy(), + ), + test_state.threshold(), + ), + }; test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( @@ -992,8 +1143,9 @@ fn recovers_from_only_chunks_if_pov_large() { .test_chunk_requests( candidate_hash, &mut virtual_overseer, - test_state.threshold(), + threshold, |_| Has::Yes, + systematic_recovery, ) .await; @@ -1033,12 +1185,25 @@ fn recovers_from_only_chunks_if_pov_large() { test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + if systematic_recovery { + test_state + .test_chunk_requests( + new_candidate.hash(), + &mut virtual_overseer, + test_state.systematic_threshold(), + |_| Has::No, + systematic_recovery, + ) + .await; + test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + } test_state .test_chunk_requests( new_candidate.hash(), &mut virtual_overseer, test_state.impossibility_threshold(), |_| Has::No, + false, ) .await; @@ -1048,13 +1213,22 @@ fn recovers_from_only_chunks_if_pov_large() { }); } -#[test] -fn fast_path_backing_group_recovers_if_pov_small() { +#[rstest] +#[case(true)] +#[case(false)] +fn fast_path_backing_group_recovers_if_pov_small(#[case] systematic_recovery: bool) { let test_state = TestState::default(); - let subsystem = AvailabilityRecoverySubsystem::with_chunks_if_pov_large( - request_receiver(), - Metrics::new_dummy(), - ); + + let subsystem = match systematic_recovery { + true => AvailabilityRecoverySubsystem::with_systematic_chunks_if_pov_large( + request_receiver(), + Metrics::new_dummy(), + ), + false => AvailabilityRecoverySubsystem::with_chunks_if_pov_large( + request_receiver(), + Metrics::new_dummy(), + ), + }; test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( @@ -1166,6 +1340,7 @@ fn no_answers_in_fast_path_causes_chunk_requests() { &mut virtual_overseer, test_state.threshold(), |_| Has::Yes, + false, ) .await; @@ -1263,6 +1438,7 @@ fn chunks_retry_until_all_nodes_respond() { &mut virtual_overseer, test_state.validators.len() - test_state.threshold(), |_| Has::timeout(), + false, ) .await; @@ -1273,6 +1449,7 @@ fn chunks_retry_until_all_nodes_respond() { &mut virtual_overseer, test_state.impossibility_threshold(), |_| Has::No, + false, ) .await; @@ -1325,9 +1502,13 @@ fn not_returning_requests_wont_stall_retrieval() { // Not returning senders won't cause the retrieval to stall: let _senders = test_state - .test_chunk_requests(candidate_hash, &mut virtual_overseer, not_returning_count, |_| { - Has::DoesNotReturn - }) + .test_chunk_requests( + candidate_hash, + &mut virtual_overseer, + not_returning_count, + |_| Has::DoesNotReturn, + false, + ) .await; test_state @@ -1337,6 +1518,7 @@ fn not_returning_requests_wont_stall_retrieval() { // Should start over: test_state.validators.len() + 3, |_| Has::timeout(), + false, ) .await; @@ -1347,6 +1529,7 @@ fn not_returning_requests_wont_stall_retrieval() { &mut virtual_overseer, test_state.threshold(), |_| Has::Yes, + false, ) .await; @@ -1400,6 +1583,7 @@ fn all_not_returning_requests_still_recovers_on_return() { &mut virtual_overseer, test_state.validators.len(), |_| Has::DoesNotReturn, + false, ) .await; @@ -1415,6 +1599,7 @@ fn all_not_returning_requests_still_recovers_on_return() { // Should start over: test_state.validators.len() + 3, |_| Has::timeout(), + false, ), ) .await; @@ -1426,6 +1611,7 @@ fn all_not_returning_requests_still_recovers_on_return() { &mut virtual_overseer, test_state.threshold(), |_| Has::Yes, + false, ) .await; @@ -1517,6 +1703,7 @@ fn does_not_query_local_validator() { &mut virtual_overseer, test_state.validators.len(), |i| if i == 0 { panic!("requested from local validator") } else { Has::timeout() }, + false, ) .await; @@ -1527,6 +1714,7 @@ fn does_not_query_local_validator() { &mut virtual_overseer, test_state.threshold() - 1, |i| if i == 0 { panic!("requested from local validator") } else { Has::Yes }, + false, ) .await; @@ -1580,6 +1768,7 @@ fn invalid_local_chunk_is_ignored() { &mut virtual_overseer, test_state.threshold() - 1, |i| if i == 0 { panic!("requested from local validator") } else { Has::Yes }, + false, ) .await; From 2035f30ef6d8eb0f29418b7e6f45c955410742a0 Mon Sep 17 00:00:00 2001 From: alindima Date: Wed, 11 Oct 2023 16:51:47 +0200 Subject: [PATCH 047/126] add more tests and small fix --- .../network/availability-recovery/src/task.rs | 14 +- .../availability-recovery/src/tests.rs | 217 +++++++++++++----- 2 files changed, 169 insertions(+), 62 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 06043b201c51..672c76313f6c 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -242,12 +242,14 @@ impl State { } } - sender - .send_message(NetworkBridgeTxMessage::SendRequests( - requests, - IfDisconnected::TryConnect, - )) - .await; + if requests.len() != 0 { + sender + .send_message(NetworkBridgeTxMessage::SendRequests( + requests, + IfDisconnected::TryConnect, + )) + .await; + } } /// Wait for a sufficient amount of chunks to reconstruct according to the provided `params`. diff --git a/polkadot/node/network/availability-recovery/src/tests.rs b/polkadot/node/network/availability-recovery/src/tests.rs index 826cabe1e519..133f1b45c3e7 100644 --- a/polkadot/node/network/availability-recovery/src/tests.rs +++ b/polkadot/node/network/availability-recovery/src/tests.rs @@ -1285,11 +1285,25 @@ fn fast_path_backing_group_recovers_if_pov_small(#[case] systematic_recovery: bo }); } -#[test] -fn no_answers_in_fast_path_causes_chunk_requests() { +#[rstest] +#[case(true)] +#[case(false)] +fn no_answers_in_fast_path_causes_chunk_requests(#[case] systematic_recovery: bool) { let test_state = TestState::default(); - let subsystem = - AvailabilityRecoverySubsystem::with_fast_path(request_receiver(), Metrics::new_dummy()); + + let (subsystem, threshold) = match systematic_recovery { + true => ( + AvailabilityRecoverySubsystem::with_fast_path_then_systematic_chunks( + request_receiver(), + Metrics::new_dummy(), + ), + test_state.systematic_threshold(), + ), + false => ( + AvailabilityRecoverySubsystem::with_fast_path(request_receiver(), Metrics::new_dummy()), + test_state.threshold(), + ), + }; test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( @@ -1338,9 +1352,9 @@ fn no_answers_in_fast_path_causes_chunk_requests() { .test_chunk_requests( candidate_hash, &mut virtual_overseer, - test_state.threshold(), + threshold, |_| Has::Yes, - false, + systematic_recovery, ) .await; @@ -1350,11 +1364,22 @@ fn no_answers_in_fast_path_causes_chunk_requests() { }); } -#[test] -fn task_canceled_when_receivers_dropped() { +#[rstest] +#[case(true)] +#[case(false)] +fn task_canceled_when_receivers_dropped(#[case] systematic_recovery: bool) { let test_state = TestState::default(); - let subsystem = - AvailabilityRecoverySubsystem::with_chunks_only(request_receiver(), Metrics::new_dummy()); + + let subsystem = match systematic_recovery { + true => AvailabilityRecoverySubsystem::with_systematic_chunks( + request_receiver(), + Metrics::new_dummy(), + ), + false => AvailabilityRecoverySubsystem::with_chunks_only( + request_receiver(), + Metrics::new_dummy(), + ), + }; test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( @@ -1394,11 +1419,21 @@ fn task_canceled_when_receivers_dropped() { }); } -#[test] -fn chunks_retry_until_all_nodes_respond() { +#[rstest] +#[case(true)] +#[case(false)] +fn chunks_retry_until_all_nodes_respond(#[case] systematic_recovery: bool) { let test_state = TestState::default(); - let subsystem = - AvailabilityRecoverySubsystem::with_chunks_only(request_receiver(), Metrics::new_dummy()); + let subsystem = match systematic_recovery { + true => AvailabilityRecoverySubsystem::with_systematic_chunks( + request_receiver(), + Metrics::new_dummy(), + ), + false => AvailabilityRecoverySubsystem::with_chunks_only( + request_receiver(), + Metrics::new_dummy(), + ), + }; test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( @@ -1432,17 +1467,30 @@ fn chunks_retry_until_all_nodes_respond() { test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + if systematic_recovery { + test_state + .test_chunk_requests( + candidate_hash, + &mut virtual_overseer, + test_state.systematic_threshold(), + |_| Has::timeout(), + true, + ) + .await; + test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + } + test_state .test_chunk_requests( candidate_hash, &mut virtual_overseer, - test_state.validators.len() - test_state.threshold(), + test_state.impossibility_threshold(), |_| Has::timeout(), false, ) .await; - // we get to go another round! + // We get to go another round! Actually, we get to go an infinite number of times. test_state .test_chunk_requests( candidate_hash, @@ -1453,15 +1501,16 @@ fn chunks_retry_until_all_nodes_respond() { ) .await; - // Recovered data should match the original one. + // Recovery is impossible. assert_eq!(rx.await.unwrap().unwrap_err(), RecoveryError::Unavailable); virtual_overseer }); } #[test] -fn not_returning_requests_wont_stall_retrieval() { +fn network_bridge_not_returning_responses_wont_stall_retrieval() { let test_state = TestState::default(); + let subsystem = AvailabilityRecoverySubsystem::with_chunks_only(request_receiver(), Metrics::new_dummy()); @@ -1539,11 +1588,21 @@ fn not_returning_requests_wont_stall_retrieval() { }); } -#[test] -fn all_not_returning_requests_still_recovers_on_return() { +#[rstest] +#[case(true)] +#[case(false)] +fn all_not_returning_requests_still_recovers_on_return(#[case] systematic_recovery: bool) { let test_state = TestState::default(); - let subsystem = - AvailabilityRecoverySubsystem::with_chunks_only(request_receiver(), Metrics::new_dummy()); + let subsystem = match systematic_recovery { + true => AvailabilityRecoverySubsystem::with_systematic_chunks( + request_receiver(), + Metrics::new_dummy(), + ), + false => AvailabilityRecoverySubsystem::with_chunks_only( + request_receiver(), + Metrics::new_dummy(), + ), + }; test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( @@ -1576,12 +1635,17 @@ fn all_not_returning_requests_still_recovers_on_return() { test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + let n = if systematic_recovery { + test_state.systematic_threshold() + } else { + test_state.validators.len() + }; let senders = test_state .test_chunk_requests( candidate_hash, &mut virtual_overseer, - test_state.validators.len(), + n, |_| Has::DoesNotReturn, false, ) @@ -1593,14 +1657,21 @@ fn all_not_returning_requests_still_recovers_on_return() { // Now retrieval should be able to recover. std::mem::drop(senders); }, - test_state.test_chunk_requests( - candidate_hash, - &mut virtual_overseer, - // Should start over: - test_state.validators.len() + 3, - |_| Has::timeout(), - false, - ), + async { + if systematic_recovery { + test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + } + test_state + .test_chunk_requests( + candidate_hash, + &mut virtual_overseer, + // Should start over: + test_state.validators.len(), + |_| Has::timeout(), + false, + ) + .await + }, ) .await; @@ -1621,11 +1692,21 @@ fn all_not_returning_requests_still_recovers_on_return() { }); } -#[test] -fn returns_early_if_we_have_the_data() { +#[rstest] +#[case(true)] +#[case(false)] +fn returns_early_if_we_have_the_data(#[case] systematic_recovery: bool) { let test_state = TestState::default(); - let subsystem = - AvailabilityRecoverySubsystem::with_chunks_only(request_receiver(), Metrics::new_dummy()); + let subsystem = match systematic_recovery { + true => AvailabilityRecoverySubsystem::with_systematic_chunks( + request_receiver(), + Metrics::new_dummy(), + ), + false => AvailabilityRecoverySubsystem::with_chunks_only( + request_receiver(), + Metrics::new_dummy(), + ), + }; test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( @@ -1660,11 +1741,27 @@ fn returns_early_if_we_have_the_data() { }); } -#[test] -fn does_not_query_local_validator() { +#[rstest] +#[case(true)] +#[case(false)] +fn does_not_query_local_validator(#[case] systematic_recovery: bool) { let test_state = TestState::default(); - let subsystem = - AvailabilityRecoverySubsystem::with_chunks_only(request_receiver(), Metrics::new_dummy()); + let (subsystem, threshold) = match systematic_recovery { + true => ( + AvailabilityRecoverySubsystem::with_systematic_chunks( + request_receiver(), + Metrics::new_dummy(), + ), + test_state.systematic_threshold(), + ), + false => ( + AvailabilityRecoverySubsystem::with_chunks_only( + request_receiver(), + Metrics::new_dummy(), + ), + test_state.threshold(), + ), + }; test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( @@ -1697,24 +1794,14 @@ fn does_not_query_local_validator() { let candidate_hash = test_state.candidate.hash(); - test_state - .test_chunk_requests( - candidate_hash, - &mut virtual_overseer, - test_state.validators.len(), - |i| if i == 0 { panic!("requested from local validator") } else { Has::timeout() }, - false, - ) - .await; - // second round, make sure it uses the local chunk. test_state .test_chunk_requests( candidate_hash, &mut virtual_overseer, - test_state.threshold() - 1, + threshold - 1, |i| if i == 0 { panic!("requested from local validator") } else { Has::Yes }, - false, + systematic_recovery, ) .await; @@ -1723,11 +1810,21 @@ fn does_not_query_local_validator() { }); } -#[test] -fn invalid_local_chunk_is_ignored() { +#[rstest] +#[case(true)] +#[case(false)] +fn invalid_local_chunk(#[case] systematic_recovery: bool) { let test_state = TestState::default(); - let subsystem = - AvailabilityRecoverySubsystem::with_chunks_only(request_receiver(), Metrics::new_dummy()); + let subsystem = match systematic_recovery { + true => AvailabilityRecoverySubsystem::with_systematic_chunks( + request_receiver(), + Metrics::new_dummy(), + ), + false => AvailabilityRecoverySubsystem::with_chunks_only( + request_receiver(), + Metrics::new_dummy(), + ), + }; test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( @@ -1762,11 +1859,19 @@ fn invalid_local_chunk_is_ignored() { let candidate_hash = test_state.candidate.hash(); + // If systematic recovery detects invalid local chunk, it'll directly go to regular + // recovery. + if systematic_recovery { + test_state + .respond_to_query_all_request_invalid(&mut virtual_overseer, |i| i == 0) + .await; + } + test_state .test_chunk_requests( candidate_hash, &mut virtual_overseer, - test_state.threshold() - 1, + test_state.threshold(), |i| if i == 0 { panic!("requested from local validator") } else { Has::Yes }, false, ) From fac014c03034aaf2dc04f1687f32d4af5140d73a Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 16 Oct 2023 09:35:15 +0200 Subject: [PATCH 048/126] bump novelpoly --- Cargo.lock | 5 +++-- polkadot/erasure-coding/Cargo.toml | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 092a59fc6c40..eddfdf5ed42f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -13753,8 +13753,9 @@ dependencies = [ [[package]] name = "reed-solomon-novelpoly" -version = "1.0.1-alpha.0" -source = "git+https://github.com/paritytech/reed-solomon-novelpoly.git?rev=6864dbf2018a8a92d976835447ba29bcdc8af6a0#6864dbf2018a8a92d976835447ba29bcdc8af6a0" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "58130877ca403ab42c864fbac74bb319a0746c07a634a92a5cfc7f54af272582" dependencies = [ "derive_more", "fs-err", diff --git a/polkadot/erasure-coding/Cargo.toml b/polkadot/erasure-coding/Cargo.toml index d4959fb4e823..1fe2214a946a 100644 --- a/polkadot/erasure-coding/Cargo.toml +++ b/polkadot/erasure-coding/Cargo.toml @@ -8,7 +8,7 @@ license.workspace = true [dependencies] polkadot-primitives = { path = "../primitives" } polkadot-node-primitives = { package = "polkadot-node-primitives", path = "../node/primitives" } -novelpoly = { package = "reed-solomon-novelpoly", git = "https://github.com/paritytech/reed-solomon-novelpoly.git", rev="6864dbf2018a8a92d976835447ba29bcdc8af6a0" } +novelpoly = { package = "reed-solomon-novelpoly", version = "1.0.2" } parity-scale-codec = { version = "3.6.1", default-features = false, features = ["std", "derive"] } sp-core = { path = "../../substrate/primitives/core" } sp-trie = { path = "../../substrate/primitives/trie" } From 7a71f0f818569f0c4dead9e65f0663d7a38df710 Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 16 Oct 2023 09:36:42 +0200 Subject: [PATCH 049/126] av-recovery: revert to with_chunks_if_pov_large for testing purposes --- polkadot/node/service/src/overseer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/polkadot/node/service/src/overseer.rs b/polkadot/node/service/src/overseer.rs index 2f3589096feb..7d1add118241 100644 --- a/polkadot/node/service/src/overseer.rs +++ b/polkadot/node/service/src/overseer.rs @@ -249,7 +249,7 @@ where IncomingRequestReceivers { pov_req_receiver, chunk_req_receiver }, Metrics::register(registry)?, )) - .availability_recovery(AvailabilityRecoverySubsystem::with_systematic_chunks( + .availability_recovery(AvailabilityRecoverySubsystem::with_chunks_if_pov_large( available_data_req_receiver, Metrics::register(registry)?, )) From 69732955cbd14060afa9c867d2c2a142b48bef61 Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 16 Oct 2023 10:32:46 +0200 Subject: [PATCH 050/126] improve migrate_to_v10 test --- .../src/configuration/migration/v10.rs | 173 +++++++----------- 1 file changed, 66 insertions(+), 107 deletions(-) diff --git a/polkadot/runtime/parachains/src/configuration/migration/v10.rs b/polkadot/runtime/parachains/src/configuration/migration/v10.rs index 50ed66b9fd83..c04e24b2d70d 100644 --- a/polkadot/runtime/parachains/src/configuration/migration/v10.rs +++ b/polkadot/runtime/parachains/src/configuration/migration/v10.rs @@ -97,68 +97,65 @@ impl OnRuntimeUpgrade for MigrateToV10 { } } -fn migrate_to_v10() -> Weight { - // Unusual formatting is justified: - // - make it easier to verify that fields assign what they supposed to assign. - // - this code is transient and will be removed after all migrations are done. - // - this code is important enough to optimize for legibility sacrificing consistency. - #[rustfmt::skip] - let translate = - |pre: V9HostConfiguration>| -> - V10HostConfiguration> - { - V10HostConfiguration { -max_code_size : pre.max_code_size, -max_head_data_size : pre.max_head_data_size, -max_upward_queue_count : pre.max_upward_queue_count, -max_upward_queue_size : pre.max_upward_queue_size, -max_upward_message_size : pre.max_upward_message_size, -max_upward_message_num_per_candidate : pre.max_upward_message_num_per_candidate, -hrmp_max_message_num_per_candidate : pre.hrmp_max_message_num_per_candidate, -validation_upgrade_cooldown : pre.validation_upgrade_cooldown, -validation_upgrade_delay : pre.validation_upgrade_delay, -max_pov_size : pre.max_pov_size, -max_downward_message_size : pre.max_downward_message_size, -hrmp_sender_deposit : pre.hrmp_sender_deposit, -hrmp_recipient_deposit : pre.hrmp_recipient_deposit, -hrmp_channel_max_capacity : pre.hrmp_channel_max_capacity, -hrmp_channel_max_total_size : pre.hrmp_channel_max_total_size, -hrmp_max_parachain_inbound_channels : pre.hrmp_max_parachain_inbound_channels, -hrmp_max_parachain_outbound_channels : pre.hrmp_max_parachain_outbound_channels, -hrmp_channel_max_message_size : pre.hrmp_channel_max_message_size, -code_retention_period : pre.code_retention_period, -on_demand_cores : pre.on_demand_cores, -on_demand_retries : pre.on_demand_retries, -group_rotation_frequency : pre.group_rotation_frequency, -paras_availability_period : pre.paras_availability_period, -scheduling_lookahead : pre.scheduling_lookahead, -max_validators_per_core : pre.max_validators_per_core, -max_validators : pre.max_validators, -dispute_period : pre.dispute_period, -dispute_post_conclusion_acceptance_period: pre.dispute_post_conclusion_acceptance_period, -no_show_slots : pre.no_show_slots, -n_delay_tranches : pre.n_delay_tranches, -zeroth_delay_tranche_width : pre.zeroth_delay_tranche_width, -needed_approvals : pre.needed_approvals, -relay_vrf_modulo_samples : pre.relay_vrf_modulo_samples, -pvf_voting_ttl : pre.pvf_voting_ttl, -minimum_validation_upgrade_delay : pre.minimum_validation_upgrade_delay, -async_backing_params : pre.async_backing_params, -executor_params : pre.executor_params, -on_demand_queue_max_size : pre.on_demand_queue_max_size, -on_demand_base_fee : pre.on_demand_base_fee, -on_demand_fee_variability : pre.on_demand_fee_variability, -on_demand_target_queue_utilization : pre.on_demand_target_queue_utilization, -on_demand_ttl : pre.on_demand_ttl, -minimum_backing_votes : pre.minimum_backing_votes, -client_features : ClientFeatures::empty() - } - }; +// Unusual formatting is justified: +// - make it easier to verify that fields assign what they supposed to assign. +// - this code is transient and will be removed after all migrations are done. +// - this code is important enough to optimize for legibility sacrificing consistency. +#[rustfmt::skip] +fn translate(pre: V9HostConfiguration>) -> V10HostConfiguration> { + V10HostConfiguration { + max_code_size : pre.max_code_size, + max_head_data_size : pre.max_head_data_size, + max_upward_queue_count : pre.max_upward_queue_count, + max_upward_queue_size : pre.max_upward_queue_size, + max_upward_message_size : pre.max_upward_message_size, + max_upward_message_num_per_candidate : pre.max_upward_message_num_per_candidate, + hrmp_max_message_num_per_candidate : pre.hrmp_max_message_num_per_candidate, + validation_upgrade_cooldown : pre.validation_upgrade_cooldown, + validation_upgrade_delay : pre.validation_upgrade_delay, + max_pov_size : pre.max_pov_size, + max_downward_message_size : pre.max_downward_message_size, + hrmp_sender_deposit : pre.hrmp_sender_deposit, + hrmp_recipient_deposit : pre.hrmp_recipient_deposit, + hrmp_channel_max_capacity : pre.hrmp_channel_max_capacity, + hrmp_channel_max_total_size : pre.hrmp_channel_max_total_size, + hrmp_max_parachain_inbound_channels : pre.hrmp_max_parachain_inbound_channels, + hrmp_max_parachain_outbound_channels : pre.hrmp_max_parachain_outbound_channels, + hrmp_channel_max_message_size : pre.hrmp_channel_max_message_size, + code_retention_period : pre.code_retention_period, + on_demand_cores : pre.on_demand_cores, + on_demand_retries : pre.on_demand_retries, + group_rotation_frequency : pre.group_rotation_frequency, + paras_availability_period : pre.paras_availability_period, + scheduling_lookahead : pre.scheduling_lookahead, + max_validators_per_core : pre.max_validators_per_core, + max_validators : pre.max_validators, + dispute_period : pre.dispute_period, + dispute_post_conclusion_acceptance_period: pre.dispute_post_conclusion_acceptance_period, + no_show_slots : pre.no_show_slots, + n_delay_tranches : pre.n_delay_tranches, + zeroth_delay_tranche_width : pre.zeroth_delay_tranche_width, + needed_approvals : pre.needed_approvals, + relay_vrf_modulo_samples : pre.relay_vrf_modulo_samples, + pvf_voting_ttl : pre.pvf_voting_ttl, + minimum_validation_upgrade_delay : pre.minimum_validation_upgrade_delay, + async_backing_params : pre.async_backing_params, + executor_params : pre.executor_params, + on_demand_queue_max_size : pre.on_demand_queue_max_size, + on_demand_base_fee : pre.on_demand_base_fee, + on_demand_fee_variability : pre.on_demand_fee_variability, + on_demand_target_queue_utilization : pre.on_demand_target_queue_utilization, + on_demand_ttl : pre.on_demand_ttl, + minimum_backing_votes : pre.minimum_backing_votes, + client_features : ClientFeatures::empty() + } +} +fn migrate_to_v10() -> Weight { let v9 = v9::ActiveConfig::::get() .defensive_proof("Could not decode old config") .unwrap_or_default(); - let v10 = translate(v9); + let v10 = translate::(v9); v10::ActiveConfig::::set(Some(v10)); // Allowed to be empty. @@ -166,7 +163,7 @@ client_features : ClientFeatures::empty() let mut pending_v10 = Vec::new(); for (session, v9) in pending_v9.into_iter() { - let v10 = translate(v9); + let v10 = translate::(v9); pending_v10.push((session, v10)); } v10::PendingConfigs::::set(Some(pending_v10.clone())); @@ -225,6 +222,8 @@ mod tests { assert_eq!(v10.client_features, ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING); } + // Test that `migrate_to_v10`` correctly applies the `translate` function to current and pending + // configs. #[test] fn test_migrate_to_v10() { // Host configuration has lots of fields. However, in this migration we only add one @@ -249,59 +248,19 @@ mod tests { pending_configs.push((300, v9.clone())); new_test_ext(Default::default()).execute_with(|| { - // Implant the v8 version in the state. - v9::ActiveConfig::::set(Some(v9)); + // Implant the v9 version in the state. + v9::ActiveConfig::::set(Some(v9.clone())); v9::PendingConfigs::::set(Some(pending_configs)); migrate_to_v10::(); - let v10 = v10::ActiveConfig::::get().unwrap(); + let v10 = translate::(v9); let mut configs_to_check = v10::PendingConfigs::::get().unwrap(); - configs_to_check.push((0, v10.clone())); - - for (_, v9) in configs_to_check { - #[rustfmt::skip] - { - assert_eq!(v9.max_code_size , v10.max_code_size); - assert_eq!(v9.max_head_data_size , v10.max_head_data_size); - assert_eq!(v9.max_upward_queue_count , v10.max_upward_queue_count); - assert_eq!(v9.max_upward_queue_size , v10.max_upward_queue_size); - assert_eq!(v9.max_upward_message_size , v10.max_upward_message_size); - assert_eq!(v9.max_upward_message_num_per_candidate , v10.max_upward_message_num_per_candidate); - assert_eq!(v9.hrmp_max_message_num_per_candidate , v10.hrmp_max_message_num_per_candidate); - assert_eq!(v9.validation_upgrade_cooldown , v10.validation_upgrade_cooldown); - assert_eq!(v9.validation_upgrade_delay , v10.validation_upgrade_delay); - assert_eq!(v9.max_pov_size , v10.max_pov_size); - assert_eq!(v9.max_downward_message_size , v10.max_downward_message_size); - assert_eq!(v9.hrmp_max_parachain_outbound_channels , v10.hrmp_max_parachain_outbound_channels); - assert_eq!(v9.hrmp_sender_deposit , v10.hrmp_sender_deposit); - assert_eq!(v9.hrmp_recipient_deposit , v10.hrmp_recipient_deposit); - assert_eq!(v9.hrmp_channel_max_capacity , v10.hrmp_channel_max_capacity); - assert_eq!(v9.hrmp_channel_max_total_size , v10.hrmp_channel_max_total_size); - assert_eq!(v9.hrmp_max_parachain_inbound_channels , v10.hrmp_max_parachain_inbound_channels); - assert_eq!(v9.hrmp_channel_max_message_size , v10.hrmp_channel_max_message_size); - assert_eq!(v9.code_retention_period , v10.code_retention_period); - assert_eq!(v9.on_demand_cores , v10.on_demand_cores); - assert_eq!(v9.on_demand_retries , v10.on_demand_retries); - assert_eq!(v9.group_rotation_frequency , v10.group_rotation_frequency); - assert_eq!(v9.paras_availability_period , v10.paras_availability_period); - assert_eq!(v9.scheduling_lookahead , v10.scheduling_lookahead); - assert_eq!(v9.max_validators_per_core , v10.max_validators_per_core); - assert_eq!(v9.max_validators , v10.max_validators); - assert_eq!(v9.dispute_period , v10.dispute_period); - assert_eq!(v9.no_show_slots , v10.no_show_slots); - assert_eq!(v9.n_delay_tranches , v10.n_delay_tranches); - assert_eq!(v9.zeroth_delay_tranche_width , v10.zeroth_delay_tranche_width); - assert_eq!(v9.needed_approvals , v10.needed_approvals); - assert_eq!(v9.relay_vrf_modulo_samples , v10.relay_vrf_modulo_samples); - assert_eq!(v9.pvf_voting_ttl , v10.pvf_voting_ttl); - assert_eq!(v9.minimum_validation_upgrade_delay , v10.minimum_validation_upgrade_delay); - assert_eq!(v9.async_backing_params.allowed_ancestry_len, v10.async_backing_params.allowed_ancestry_len); - assert_eq!(v9.async_backing_params.max_candidate_depth , v10.async_backing_params.max_candidate_depth); - assert_eq!(v9.executor_params , v10.executor_params); - assert_eq!(v9.minimum_backing_votes , v10.minimum_backing_votes); - assert_eq!(v9.client_features , v10.client_features); - }; // ; makes this a statement. `rustfmt::skip` cannot be put on an expression. + configs_to_check.push((0, v10::ActiveConfig::::get().unwrap())); + + for (_, config) in configs_to_check { + assert_eq!(config, v10); + assert_eq!(config.client_features, ClientFeatures::empty()); } }); } From 1af06576c6155640a4cc11dd5877c08ace041d81 Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 16 Oct 2023 11:42:14 +0200 Subject: [PATCH 051/126] address some review comments --- polkadot/erasure-coding/src/lib.rs | 14 ++++---------- polkadot/node/core/bitfield-signing/src/lib.rs | 5 +---- 2 files changed, 5 insertions(+), 14 deletions(-) diff --git a/polkadot/erasure-coding/src/lib.rs b/polkadot/erasure-coding/src/lib.rs index 848858a6a215..e5d59452401f 100644 --- a/polkadot/erasure-coding/src/lib.rs +++ b/polkadot/erasure-coding/src/lib.rs @@ -149,6 +149,10 @@ pub fn reconstruct_from_systematic( return Err(Error::NonUniformChunks) } + if chunks.iter().any(|c| c.len() != shard_len) { + return Err(Error::NonUniformChunks) + } + if shard_len % 2 != 0 { return Err(Error::UnevenLength) } @@ -157,25 +161,15 @@ pub fn reconstruct_from_systematic( return Err(Error::NotEnoughChunks) } - let mut check_shard_len = true; let mut systematic_bytes = Vec::with_capacity(shard_len * kpow2); for i in (0..shard_len).step_by(2) { for chunk in chunks.iter().take(kpow2) { - if check_shard_len { - if chunk.len() != shard_len { - return Err(Error::NonUniformChunks) - } - } - // No need to check for index out of bounds because i goes up to shard_len and // we return an error for non uniform chunks. systematic_bytes.push(chunk[i]); systematic_bytes.push(chunk[i + 1]); } - - // After the first check, stop checking the shard lengths. - check_shard_len = false; } Decode::decode(&mut &systematic_bytes[..]).map_err(|_| Error::BadPayload) diff --git a/polkadot/node/core/bitfield-signing/src/lib.rs b/polkadot/node/core/bitfield-signing/src/lib.rs index aeda42a814ce..ecef5ca0a272 100644 --- a/polkadot/node/core/bitfield-signing/src/lib.rs +++ b/polkadot/node/core/bitfield-signing/src/lib.rs @@ -83,9 +83,6 @@ pub enum Error { #[error("Cannot find block number for given relay parent")] BlockNumberNotFound, - #[error("Oneshot for receiving response from Chain API got cancelled")] - ChainApiSenderDropped(#[source] oneshot::Canceled), - #[error("Retrieving response from Chain API unexpectedly failed with error: {0}")] ChainApi(#[from] ChainApiError), } @@ -405,7 +402,7 @@ where let (tx, rx) = oneshot::channel(); sender.send_message(ChainApiMessage::BlockNumber(relay_parent, tx)).await; - let block_number = rx.await.map_err(Error::ChainApiSenderDropped)?.map_err(Error::ChainApi)?; + let block_number = rx.await.map_err(Error::Oneshot)?.map_err(Error::ChainApi)?; if let Some(number) = block_number { Ok(number) From 38e13619100d2b83fd640b15f2ae022f52510ba0 Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 16 Oct 2023 12:35:43 +0200 Subject: [PATCH 052/126] some more tests --- polkadot/node/subsystem-util/src/tests.rs | 227 ++++++++++++++++------ 1 file changed, 168 insertions(+), 59 deletions(-) diff --git a/polkadot/node/subsystem-util/src/tests.rs b/polkadot/node/subsystem-util/src/tests.rs index 859cc32e0a17..56a50f62ca21 100644 --- a/polkadot/node/subsystem-util/src/tests.rs +++ b/polkadot/node/subsystem-util/src/tests.rs @@ -21,6 +21,7 @@ use executor::block_on; use futures::{channel::mpsc, executor, FutureExt, SinkExt, StreamExt}; use polkadot_primitives_test_helpers::AlwaysZeroRng; use std::{ + collections::HashSet, sync::{ atomic::{AtomicUsize, Ordering}, Arc, @@ -95,62 +96,170 @@ fn subset_predefined_generation_check() { } } -// #[test] -// fn test_availability_chunk_indices() { -// let block_number = 89; -// let n_validators = 11; -// let babe_randomness = [12u8; 32]; -// let n_cores = 4; - -// let client_features = Some(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING); -// let (shuffle, core_start_pos) = availability_chunk_indices( -// client_features, -// block_number, -// babe_randomness, -// n_validators, -// n_cores, -// ); -// // assert_eq!(shuffle, (0..n_validators).map(|i| ValidatorIndex(i as u32)).collect::>()); -// // assert_eq!(core_start_pos, repeat(ValidatorIndex(0)).take(n_cores).collect::>()); - -// for index in 0..n_cores { -// for validator in 0..n_validators { -// let chunk_index = availability_chunk_index( -// client_features, -// block_number, -// babe_randomness, -// n_validators, -// CoreIndex(index as u32), -// n_cores, -// ValidatorIndex(validator as u32), -// ); - -// assert_eq!( -// &chunk_index, -// (shuffle -// .iter() -// .cycle() -// .skip(core_start_pos[index].0 as usize) -// .take(n_validators) -// .collect::>())[validator] -// ); -// } -// } - -// // let shuffle = -// // availability_chunk_indices(Some(ClientFeatures::empty()), block_number, n_validators); -// // assert_eq!(shuffle, (0..n_validators).map(|i| ValidatorIndex(i as u32)).collect::>()); - -// // let shuffle = -// // availability_chunk_indices(ClientFeatures::from_bits(0b10), block_number, n_validators); -// // assert_eq!(shuffle, (0..n_validators).map(|i| ValidatorIndex(i as u32)).collect::>()); - -// // let shuffle = availability_chunk_indices( -// // Some(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING), -// // block_number, -// // n_validators, -// // ); -// // assert_ne!(shuffle, (0..n_validators).map(|i| ValidatorIndex(i as u32)).collect::>()); -// // assert_eq!(shuffle.len(), n_validators); -// // assert_eq!(shuffle.iter().collect::>().len(), n_validators); -// } +#[test] +fn test_availability_chunk_indices() { + let block_number = 89; + let n_validators = 11u32; + let babe_randomness = [12u8; 32]; + let session_index = 0; + let n_paras = 5u32; + + // Test the `_for_validator` methods + { + let para_id = 2.into(); + let mut index_registry = ChunkIndexCacheRegistry::new(2); + + for validator in 0..n_validators { + assert!(index_registry + .query_cache_for_validator(block_number, session_index, para_id, validator.into()) + .is_none()); + } + + for validator in 0..n_validators { + // Check that if the client feature is not set, we'll always return the validator index. + let chunk_index = index_registry.populate_for_validator( + None, + babe_randomness, + n_validators as usize, + block_number, + session_index, + para_id, + validator.into(), + ); + assert_eq!( + index_registry + .query_cache_for_validator( + block_number, + session_index, + para_id, + validator.into() + ) + .unwrap(), + chunk_index + ); + assert_eq!(chunk_index.0, validator); + assert_eq!( + chunk_index, + availability_chunk_index( + None, + babe_randomness, + n_validators as usize, + block_number, + para_id, + validator.into(), + ) + ); + + // Check for when the client feature is set. + let chunk_index = index_registry.populate_for_validator( + Some(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING), + babe_randomness, + n_validators as usize, + block_number, + session_index, + para_id, + validator.into(), + ); + assert_eq!( + index_registry + .query_cache_for_validator( + block_number, + session_index, + para_id, + validator.into() + ) + .unwrap(), + chunk_index + ); + assert_ne!(chunk_index.0, validator); + assert_eq!( + chunk_index, + availability_chunk_index( + Some(&ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING), + babe_randomness, + n_validators as usize, + block_number, + para_id, + validator.into(), + ) + ); + } + } + + // Test the `_for_para` methods + { + let mut index_registry = ChunkIndexCacheRegistry::new(2); + + for para in 0..n_paras { + assert!(index_registry + .query_cache_for_para(block_number, session_index, para.into()) + .is_none()); + } + + for para in 0..n_paras { + // Check that if the client feature is not set, we'll always return the identity vector. + let chunk_indices = index_registry.populate_for_para( + None, + babe_randomness, + n_validators as usize, + block_number, + session_index, + para.into(), + ); + assert_eq!( + index_registry + .query_cache_for_para(block_number, session_index, para.into()) + .unwrap(), + chunk_indices + ); + assert_eq!(chunk_indices, (0..n_validators).map(|i| ChunkIndex(i)).collect::>()); + + for validator in 0..n_validators { + assert_eq!( + availability_chunk_index( + None, + babe_randomness, + n_validators as usize, + block_number, + para.into(), + validator.into(), + ), + chunk_indices[validator as usize] + ); + } + + // Check for when the client feature is set. + let chunk_indices = index_registry.populate_for_para( + Some(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING), + babe_randomness, + n_validators as usize, + block_number, + session_index, + para.into(), + ); + assert_eq!( + index_registry + .query_cache_for_para(block_number, session_index, para.into()) + .unwrap(), + chunk_indices + ); + assert_eq!(chunk_indices.len(), n_validators as usize); + assert_ne!(chunk_indices, (0..n_validators).map(|i| ChunkIndex(i)).collect::>()); + assert_eq!(chunk_indices.iter().collect::>().len(), n_validators as usize); + + for validator in 0..n_validators { + assert_eq!( + availability_chunk_index( + Some(&ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING), + babe_randomness, + n_validators as usize, + block_number, + para.into(), + validator.into(), + ), + chunk_indices[validator as usize] + ); + } + } + } +} From 1318a21b538f8d5bb10376679e93d794b95d866e Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 16 Oct 2023 13:44:37 +0200 Subject: [PATCH 053/126] enable systematic recovery for testing purposes --- polkadot/node/service/src/overseer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/polkadot/node/service/src/overseer.rs b/polkadot/node/service/src/overseer.rs index 7d1add118241..bcc4a2040f29 100644 --- a/polkadot/node/service/src/overseer.rs +++ b/polkadot/node/service/src/overseer.rs @@ -249,7 +249,7 @@ where IncomingRequestReceivers { pov_req_receiver, chunk_req_receiver }, Metrics::register(registry)?, )) - .availability_recovery(AvailabilityRecoverySubsystem::with_chunks_if_pov_large( + .availability_recovery(AvailabilityRecoverySubsystem::with_systematic_chunks_if_pov_large( available_data_req_receiver, Metrics::register(registry)?, )) From a98cdd68f8549c0646a6fa7f03ab3eb01d526f59 Mon Sep 17 00:00:00 2001 From: alindima Date: Wed, 18 Oct 2023 18:06:45 +0300 Subject: [PATCH 054/126] don't request chunks that past strategies deemed not available also add more tests --- .../network/availability-recovery/src/task.rs | 31 +- .../availability-recovery/src/tests.rs | 287 +++++++++++++++++- 2 files changed, 297 insertions(+), 21 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 672c76313f6c..e948c53cec36 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -39,7 +39,7 @@ use polkadot_primitives::{AuthorityDiscoveryId, CandidateHash, ChunkIndex, Hash, use rand::seq::SliceRandom; use sc_network::{IfDisconnected, OutboundFailure, RequestFailure}; use std::{ - collections::{BTreeMap, VecDeque}, + collections::{BTreeMap, BTreeSet, VecDeque}, time::Duration, }; @@ -110,11 +110,16 @@ pub struct State { /// that chunks are ordered by their index). If we ever switch this to some non-ordered /// collection, we need to add a sort step to the systematic recovery. received_chunks: BTreeMap, + + /// A record of the chunks that are not available. + /// Useful so that subsequent strategies don't waste time requesting chunks that are known to + /// be invalid or from validators that did not have the chunk. + chunks_not_available: BTreeSet, } impl State { fn new() -> Self { - Self { received_chunks: BTreeMap::new() } + Self { received_chunks: BTreeMap::new(), chunks_not_available: BTreeSet::new() } } fn insert_chunk(&mut self, chunk_index: ChunkIndex, chunk: ErasureChunk) { @@ -294,6 +299,7 @@ impl State { } else { metrics.on_chunk_request_invalid(chunk_type); error_count += 1; + self.chunks_not_available.insert(chunk_index); }, Ok(None) => { metrics.on_chunk_request_no_such_chunk(chunk_type); @@ -305,6 +311,7 @@ impl State { "Validator did not have the requested chunk", ); error_count += 1; + self.chunks_not_available.insert(chunk_index); }, Err(err) => { error_count += 1; @@ -330,6 +337,8 @@ impl State { ?validator_index, "Chunk fetching response was invalid", ); + + self.chunks_not_available.insert(chunk_index); }, RequestError::NetworkError(err) => { // No debug logs on general network errors - that became very spammy @@ -785,9 +794,12 @@ impl RecoveryStrategy // them from the queue. let mut systematic_chunk_count = self.validators.len(); - // No need to query the validators that have the chunks we already received. - self.validators - .retain(|(c_index, _)| !state.received_chunks.contains_key(c_index)); + // No need to query the validators that have the chunks we already received or that we know + // don't have the data from previous strategies. + self.validators.retain(|(c_index, _)| { + !state.received_chunks.contains_key(c_index) && + !state.chunks_not_available.contains(c_index) + }); systematic_chunk_count -= self.validators.len(); @@ -1066,9 +1078,12 @@ impl RecoveryStrategy self.validators.retain(|(c_index, _)| !local_chunk_indices.contains(c_index)); } - // No need to query the validators that have the chunks we already received. - self.validators - .retain(|(c_index, _)| !state.received_chunks.contains_key(c_index)); + // No need to query the validators that have the chunks we already received or that we know + // don't have the data from previous strategies. + self.validators.retain(|(c_index, _)| { + !state.received_chunks.contains_key(c_index) && + !state.chunks_not_available.contains(c_index) + }); // Safe to `take` here, as we're consuming `self` anyway and we're not using the // `validators` field in other methods. diff --git a/polkadot/node/network/availability-recovery/src/tests.rs b/polkadot/node/network/availability-recovery/src/tests.rs index 133f1b45c3e7..ccc21e92a9c6 100644 --- a/polkadot/node/network/availability-recovery/src/tests.rs +++ b/polkadot/node/network/availability-recovery/src/tests.rs @@ -230,6 +230,22 @@ impl TestState { ); } + async fn test_runtime_api_empty_client_features(&self, virtual_overseer: &mut VirtualOverseer) { + assert_matches!( + overseer_recv(virtual_overseer).await, + AllMessages::RuntimeApi(RuntimeApiMessage::Request( + _relay_parent, + RuntimeApiRequest::ClientFeatures( + tx, + ) + )) => { + tx.send(Ok( + ClientFeatures::empty() + )).unwrap(); + } + ); + } + async fn respond_to_block_number_query( &self, virtual_overseer: &mut VirtualOverseer, @@ -462,6 +478,8 @@ impl Default for TestState { Sr25519Keyring::Bob, Sr25519Keyring::Charlie, Sr25519Keyring::Dave, + Sr25519Keyring::One, + Sr25519Keyring::Two, ]; let validator_public = validator_pubkeys(&validators); @@ -816,11 +834,9 @@ fn bad_merkle_path_leads_to_recovery_error(#[case] systematic_recovery: bool) { let candidate_hash = test_state.candidate.hash(); // Create some faulty chunks. - test_state.chunks[0].chunk = vec![0; 32]; - test_state.chunks[1].chunk = vec![1; 32]; - test_state.chunks[2].chunk = vec![2; 32]; - test_state.chunks[3].chunk = vec![3; 32]; - test_state.chunks[4].chunk = vec![4; 32]; + for chunk in test_state.chunks.iter_mut() { + chunk.chunk = vec![0; 32]; + } test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; @@ -900,16 +916,15 @@ fn wrong_chunk_index_leads_to_recovery_error(#[case] systematic_recovery: bool) let candidate_hash = test_state.candidate.hash(); - // Simulate that we are holding a chunk with an invalid index in the av-store. - test_state.chunks[1] = test_state.chunks[0].clone(); - test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; - // These chunks should fail the index check as they don't have the correct index. - test_state.chunks[2] = test_state.chunks[0].clone(); - test_state.chunks[3] = test_state.chunks[0].clone(); - test_state.chunks[4] = test_state.chunks[0].clone(); + // Chunks should fail the index check as they don't have the correct index. + let first_chunk = test_state.chunks[0].clone(); + test_state.chunks[0] = test_state.chunks[1].clone(); + for c_index in 1..test_state.chunks.len() { + test_state.chunks[c_index] = first_chunk.clone(); + } if systematic_recovery { test_state @@ -929,7 +944,7 @@ fn wrong_chunk_index_leads_to_recovery_error(#[case] systematic_recovery: bool) .test_chunk_requests( candidate_hash, &mut virtual_overseer, - test_state.impossibility_threshold(), + test_state.chunks.len() - 1, |_| Has::Yes, false, ) @@ -1692,6 +1707,9 @@ fn all_not_returning_requests_still_recovers_on_return(#[case] systematic_recove }); } +#[test] +fn with_availability_store_skip() {} + #[rstest] #[case(true)] #[case(false)] @@ -1741,6 +1759,74 @@ fn returns_early_if_we_have_the_data(#[case] systematic_recovery: bool) { }); } +#[test] +fn returns_early_if_present_in_the_subsystem_cache() { + let test_state = TestState::default(); + let subsystem = + AvailabilityRecoverySubsystem::with_fast_path(request_receiver(), Metrics::new_dummy()); + + test_harness(subsystem, |mut virtual_overseer| async move { + overseer_signal( + &mut virtual_overseer, + OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( + test_state.current, + 1, + ))), + ) + .await; + + let (tx, rx) = oneshot::channel(); + + overseer_send( + &mut virtual_overseer, + AvailabilityRecoveryMessage::RecoverAvailableData( + test_state.candidate.clone(), + test_state.session_index, + Some(GroupIndex(0)), + tx, + ), + ) + .await; + + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_client_features(&mut virtual_overseer).await; + + let candidate_hash = test_state.candidate.hash(); + + let who_has = |i| match i { + 3 => Has::Yes, + _ => Has::No, + }; + + test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; + + test_state + .test_full_data_requests(candidate_hash, &mut virtual_overseer, who_has) + .await; + + // Recovered data should match the original one. + assert_eq!(rx.await.unwrap().unwrap(), test_state.available_data); + + // A second recovery for the same candidate will return early as it'll be present in the + // cache. + let (tx, rx) = oneshot::channel(); + overseer_send( + &mut virtual_overseer, + AvailabilityRecoveryMessage::RecoverAvailableData( + test_state.candidate.clone(), + test_state.session_index, + Some(GroupIndex(0)), + tx, + ), + ) + .await; + assert_eq!(rx.await.unwrap().unwrap(), test_state.available_data); + + virtual_overseer + }); +} + #[rstest] #[case(true)] #[case(false)] @@ -1881,3 +1967,178 @@ fn invalid_local_chunk(#[case] systematic_recovery: bool) { virtual_overseer }); } + +#[test] +fn systematic_chunks_are_not_requested_again_in_regular_recovery() { + // Run this test multiple times, as the order in which requests are made is random and we want + // to make sure that we catch regressions. + for _ in 0..TestState::default().chunks.len() { + let test_state = TestState::default(); + let subsystem = AvailabilityRecoverySubsystem::with_systematic_chunks( + request_receiver(), + Metrics::new_dummy(), + ); + + test_harness(subsystem, |mut virtual_overseer| async move { + overseer_signal( + &mut virtual_overseer, + OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( + test_state.current, + 1, + ))), + ) + .await; + + let (tx, rx) = oneshot::channel(); + + overseer_send( + &mut virtual_overseer, + AvailabilityRecoveryMessage::RecoverAvailableData( + test_state.candidate.clone(), + test_state.session_index, + None, + tx, + ), + ) + .await; + + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_client_features(&mut virtual_overseer).await; + test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; + test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + + test_state + .test_chunk_requests( + test_state.candidate.hash(), + &mut virtual_overseer, + test_state.systematic_threshold(), + |i| if i == 0 { Has::No } else { Has::Yes }, + true, + ) + .await; + + // Falls back to regular recovery. + test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + + test_state + .test_chunk_requests( + test_state.candidate.hash(), + &mut virtual_overseer, + 1, + |i: usize| { + if i < test_state.systematic_threshold() { + panic!("Already requested") + } else { + Has::Yes + } + }, + false, + ) + .await; + + assert_eq!(rx.await.unwrap().unwrap(), test_state.available_data); + virtual_overseer + }); + } +} + +#[rstest] +#[case(true, true)] +#[case(true, false)] +#[case(false, true)] +#[case(false, false)] +fn chunk_indices_are_shuffled(#[case] systematic_recovery: bool, #[case] shuffling_enabled: bool) { + let test_state = TestState::default(); + let subsystem = match systematic_recovery { + true => AvailabilityRecoverySubsystem::with_systematic_chunks( + request_receiver(), + Metrics::new_dummy(), + ), + false => AvailabilityRecoverySubsystem::with_chunks_only( + request_receiver(), + Metrics::new_dummy(), + ), + }; + + test_harness(subsystem, |mut virtual_overseer| async move { + overseer_signal( + &mut virtual_overseer, + OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( + test_state.current, + 1, + ))), + ) + .await; + + let (tx, _rx) = oneshot::channel(); + + overseer_send( + &mut virtual_overseer, + AvailabilityRecoveryMessage::RecoverAvailableData( + test_state.candidate.clone(), + test_state.session_index, + None, + tx, + ), + ) + .await; + + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + + if shuffling_enabled { + test_state.test_runtime_api_client_features(&mut virtual_overseer).await; + } else { + test_state.test_runtime_api_empty_client_features(&mut virtual_overseer).await; + } + + test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; + test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + + let mut chunk_indices: Vec<(usize, usize)> = vec![]; + + assert_matches!( + overseer_recv(&mut virtual_overseer).await, + AllMessages::NetworkBridgeTx( + NetworkBridgeTxMessage::SendRequests( + requests, + _if_disconnected, + ) + ) => { + for req in requests { + assert_matches!( + req, + Requests::ChunkFetchingV1(req) => { + assert_eq!(req.payload.candidate_hash, test_state.candidate.hash()); + + let chunk_index = req.payload.index.0 as usize; + let validator_index = test_state.validator_authority_id.iter().enumerate().find(|(_, id)| { + if let Recipient::Authority(auth_id) = &req.peer { + if *id == auth_id { + return true + } + } + false + }).expect("validator not found").0; + + if systematic_recovery { + assert!(chunk_index <= test_state.systematic_threshold(), "requsted non-systematic chunk"); + } + + chunk_indices.push((chunk_index, validator_index)); + } + ) + } + } + ); + + if shuffling_enabled { + assert!(!chunk_indices.iter().any(|(c_index, v_index)| c_index == v_index)); + } else { + assert!(chunk_indices.iter().all(|(c_index, v_index)| c_index == v_index)); + } + + virtual_overseer + }); +} From 9dc87ae61aaf304fecca15d9055ffbe0fe57426d Mon Sep 17 00:00:00 2001 From: alindima Date: Thu, 19 Oct 2023 10:21:00 +0300 Subject: [PATCH 055/126] more tests --- .../availability-recovery/src/tests.rs | 69 +++++++++++++------ 1 file changed, 48 insertions(+), 21 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/tests.rs b/polkadot/node/network/availability-recovery/src/tests.rs index ccc21e92a9c6..590143eec028 100644 --- a/polkadot/node/network/availability-recovery/src/tests.rs +++ b/polkadot/node/network/availability-recovery/src/tests.rs @@ -1091,26 +1091,38 @@ fn fast_path_backing_group_recovers() { } #[rstest] -#[case(true)] -#[case(false)] -fn recovers_from_only_chunks_if_pov_large(#[case] systematic_recovery: bool) { +#[case(true, false)] +#[case(false, true)] +#[case(false, false)] +fn recovers_from_only_chunks_if_pov_large( + #[case] systematic_recovery: bool, + #[case] skip_availability_store: bool, +) { let test_state = TestState::default(); - let (subsystem, threshold) = match systematic_recovery { - true => ( + let (subsystem, threshold) = match (systematic_recovery, skip_availability_store) { + (true, false) => ( AvailabilityRecoverySubsystem::with_systematic_chunks_if_pov_large( request_receiver(), Metrics::new_dummy(), ), test_state.systematic_threshold(), ), - false => ( + (false, false) => ( AvailabilityRecoverySubsystem::with_chunks_if_pov_large( request_receiver(), Metrics::new_dummy(), ), test_state.threshold(), ), + (false, true) => ( + AvailabilityRecoverySubsystem::with_availability_store_skip( + request_receiver(), + Metrics::new_dummy(), + ), + test_state.threshold(), + ), + (_, _) => unreachable!(), }; test_harness(subsystem, |mut virtual_overseer| async move { @@ -1151,8 +1163,10 @@ fn recovers_from_only_chunks_if_pov_large(#[case] systematic_recovery: bool) { } ); - test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; - test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + if !skip_availability_store { + test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; + test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + } test_state .test_chunk_requests( @@ -1197,8 +1211,10 @@ fn recovers_from_only_chunks_if_pov_large(#[case] systematic_recovery: bool) { } ); - test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; - test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + if !skip_availability_store { + test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; + test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + } if systematic_recovery { test_state @@ -1210,7 +1226,9 @@ fn recovers_from_only_chunks_if_pov_large(#[case] systematic_recovery: bool) { systematic_recovery, ) .await; - test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + if !skip_availability_store { + test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + } } test_state .test_chunk_requests( @@ -1229,20 +1247,30 @@ fn recovers_from_only_chunks_if_pov_large(#[case] systematic_recovery: bool) { } #[rstest] -#[case(true)] -#[case(false)] -fn fast_path_backing_group_recovers_if_pov_small(#[case] systematic_recovery: bool) { +#[case(true, false)] +#[case(false, true)] +#[case(false, false)] +fn fast_path_backing_group_recovers_if_pov_small( + #[case] systematic_recovery: bool, + #[case] skip_availability_store: bool, +) { let test_state = TestState::default(); - let subsystem = match systematic_recovery { - true => AvailabilityRecoverySubsystem::with_systematic_chunks_if_pov_large( + let subsystem = match (systematic_recovery, skip_availability_store) { + (true, false) => AvailabilityRecoverySubsystem::with_systematic_chunks_if_pov_large( + request_receiver(), + Metrics::new_dummy(), + ), + + (false, false) => AvailabilityRecoverySubsystem::with_chunks_if_pov_large( request_receiver(), Metrics::new_dummy(), ), - false => AvailabilityRecoverySubsystem::with_chunks_if_pov_large( + (false, true) => AvailabilityRecoverySubsystem::with_availability_store_skip( request_receiver(), Metrics::new_dummy(), ), + (_, _) => unreachable!(), }; test_harness(subsystem, |mut virtual_overseer| async move { @@ -1288,7 +1316,9 @@ fn fast_path_backing_group_recovers_if_pov_small(#[case] systematic_recovery: bo } ); - test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; + if !skip_availability_store { + test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; + } test_state .test_full_data_requests(candidate_hash, &mut virtual_overseer, who_has) @@ -1707,9 +1737,6 @@ fn all_not_returning_requests_still_recovers_on_return(#[case] systematic_recove }); } -#[test] -fn with_availability_store_skip() {} - #[rstest] #[case(true)] #[case(false)] From c462e25b067a29210b6b082742796105439538e8 Mon Sep 17 00:00:00 2001 From: alindima Date: Thu, 19 Oct 2023 12:12:43 +0300 Subject: [PATCH 056/126] fix infinite request loop for network errors. Prior to this commit, the regular chunk recovery code (also on master) would be stuck in an infinite loop if all validators are not responsive or if they all return invalid chunks Introduces a fix and test. --- .../network/availability-recovery/src/task.rs | 103 +++++++++++++++--- .../availability-recovery/src/tests.rs | 100 +++++++++++++++++ 2 files changed, 187 insertions(+), 16 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index e948c53cec36..4826cb6de146 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -39,7 +39,7 @@ use polkadot_primitives::{AuthorityDiscoveryId, CandidateHash, ChunkIndex, Hash, use rand::seq::SliceRandom; use sc_network::{IfDisconnected, OutboundFailure, RequestFailure}; use std::{ - collections::{BTreeMap, BTreeSet, VecDeque}, + collections::{BTreeMap, HashMap, VecDeque}, time::Duration, }; @@ -60,6 +60,15 @@ const TIMEOUT_START_NEW_REQUESTS: Duration = CHUNK_REQUEST_TIMEOUT; #[cfg(test)] const TIMEOUT_START_NEW_REQUESTS: Duration = Duration::from_millis(100); +/// The maximum number of times systematic chunk recovery will retry making a request for a given +/// (validator,chunk) pair, if the error was not fatal. Added so that we don't get stuck in an +/// infinite retry loop. +pub const SYSTEMATIC_CHUNKS_REQ_RETRY_THRESHOLD: u32 = 0; +/// The maximum number of times regular chunk recovery will retry making a request for a given +/// (validator,chunk) pair, if the error was not fatal. Added so that we don't get stuck in an +/// infinite retry loop. +pub const REGULAR_CHUNKS_REQ_RETRY_THRESHOLD: u32 = 5; + #[async_trait::async_trait] /// Common trait for runnable recovery strategies. pub trait RecoveryStrategy: Send { @@ -102,6 +111,12 @@ pub struct RecoveryParams { pub bypass_availability_store: bool, } +/// Utility type used for recording the result of requesting a chunk from a validator. +pub enum ErrorRecord { + NonFatal(u32), + Fatal, +} + /// Intermediate/common data that must be passed between `RecoveryStrategy`s belonging to the /// same `RecoveryTask`. pub struct State { @@ -111,15 +126,13 @@ pub struct State { /// collection, we need to add a sort step to the systematic recovery. received_chunks: BTreeMap, - /// A record of the chunks that are not available. - /// Useful so that subsequent strategies don't waste time requesting chunks that are known to - /// be invalid or from validators that did not have the chunk. - chunks_not_available: BTreeSet, + /// A record of errors returned when requesting a chunk from a validator. + historical_errors: HashMap<(ChunkIndex, ValidatorIndex), ErrorRecord>, } impl State { fn new() -> Self { - Self { received_chunks: BTreeMap::new(), chunks_not_available: BTreeSet::new() } + Self { received_chunks: BTreeMap::new(), historical_errors: HashMap::new() } } fn insert_chunk(&mut self, chunk_index: ChunkIndex, chunk: ErasureChunk) { @@ -130,6 +143,38 @@ impl State { self.received_chunks.len() } + fn record_error_fatal(&mut self, chunk_index: ChunkIndex, validator_index: ValidatorIndex) { + self.historical_errors + .insert((chunk_index, validator_index), ErrorRecord::Fatal); + } + + fn record_error_non_fatal(&mut self, chunk_index: ChunkIndex, validator_index: ValidatorIndex) { + self.historical_errors + .entry((chunk_index, validator_index)) + .and_modify(|record| { + if let ErrorRecord::NonFatal(ref mut count) = record { + *count = count.saturating_add(1); + } + }) + .or_insert(ErrorRecord::NonFatal(1)); + } + + fn can_retry_request( + &self, + chunk_index: ChunkIndex, + validator_index: ValidatorIndex, + retry_threshold: u32, + ) -> bool { + match self.historical_errors.get(&(chunk_index, validator_index)) { + None => true, + Some(entry) => match entry { + ErrorRecord::Fatal => false, + ErrorRecord::NonFatal(count) if *count < retry_threshold => true, + ErrorRecord::NonFatal(_) => false, + }, + } + } + /// Retrieve the local chunks held in the av-store (either 0 or 1). async fn populate_from_av_store( &mut self, @@ -262,6 +307,7 @@ impl State { &mut self, chunk_type: &str, params: &RecoveryParams, + retry_threshold: u32, validators: &mut VecDeque<(ChunkIndex, ValidatorIndex)>, requesting_chunks: &mut FuturesUndead<( ChunkIndex, @@ -280,6 +326,7 @@ impl State { // return in that case for `launch_parallel_requests` to fill up slots again. while let Some(res) = requesting_chunks.next_with_timeout(TIMEOUT_START_NEW_REQUESTS).await { + println!("Validator count: {}", validators.len()); total_received_responses += 1; let (chunk_index, validator_index, request_result) = res; @@ -299,7 +346,9 @@ impl State { } else { metrics.on_chunk_request_invalid(chunk_type); error_count += 1; - self.chunks_not_available.insert(chunk_index); + // Record that we got an invalid chunk so that subsequent strategies don't + // try requesting this again. + self.record_error_fatal(chunk_index, validator_index); }, Ok(None) => { metrics.on_chunk_request_no_such_chunk(chunk_type); @@ -311,7 +360,9 @@ impl State { "Validator did not have the requested chunk", ); error_count += 1; - self.chunks_not_available.insert(chunk_index); + // Record that the validator did not have this chunk so that subsequent + // strategies don't try requesting this again. + self.record_error_fatal(chunk_index, validator_index); }, Err(err) => { error_count += 1; @@ -338,7 +389,9 @@ impl State { "Chunk fetching response was invalid", ); - self.chunks_not_available.insert(chunk_index); + // Record that we got an invalid chunk so that subsequent strategies + // don't try requesting this again. + self.record_error_fatal(chunk_index, validator_index); }, RequestError::NetworkError(err) => { // No debug logs on general network errors - that became very spammy @@ -349,12 +402,22 @@ impl State { metrics.on_chunk_request_error(chunk_type); } - validators.push_front((chunk_index, validator_index)); + self.record_error_non_fatal(chunk_index, validator_index); + // Record that we got a non-fatal error so that this or subsequent + // strategies will retry requesting this only a limited number of times. + if self.can_retry_request(chunk_index, validator_index, retry_threshold) + { + validators.push_front((chunk_index, validator_index)); + } }, RequestError::Canceled(_) => { metrics.on_chunk_request_error(chunk_type); - validators.push_front((chunk_index, validator_index)); + self.record_error_non_fatal(chunk_index, validator_index); + if self.can_retry_request(chunk_index, validator_index, retry_threshold) + { + validators.push_front((chunk_index, validator_index)); + } }, } }, @@ -796,9 +859,13 @@ impl RecoveryStrategy // No need to query the validators that have the chunks we already received or that we know // don't have the data from previous strategies. - self.validators.retain(|(c_index, _)| { + self.validators.retain(|(c_index, v_index)| { !state.received_chunks.contains_key(c_index) && - !state.chunks_not_available.contains(c_index) + state.can_retry_request( + *c_index, + *v_index, + SYSTEMATIC_CHUNKS_REQ_RETRY_THRESHOLD, + ) }); systematic_chunk_count -= self.validators.len(); @@ -863,6 +930,7 @@ impl RecoveryStrategy .wait_for_chunks( SYSTEMATIC_CHUNK_LABEL, common_params, + SYSTEMATIC_CHUNKS_REQ_RETRY_THRESHOLD, &mut validators_queue, &mut self.requesting_chunks, |unrequested_validators, @@ -879,7 +947,9 @@ impl RecoveryStrategy self.threshold, ); - error_count > 0 || chunk_count >= self.threshold || is_unavailable + error_count > (SYSTEMATIC_CHUNKS_REQ_RETRY_THRESHOLD as usize) || + chunk_count >= self.threshold || + is_unavailable }, ) .await; @@ -1080,9 +1150,9 @@ impl RecoveryStrategy // No need to query the validators that have the chunks we already received or that we know // don't have the data from previous strategies. - self.validators.retain(|(c_index, _)| { + self.validators.retain(|(c_index, v_index)| { !state.received_chunks.contains_key(c_index) && - !state.chunks_not_available.contains(c_index) + state.can_retry_request(*c_index, *v_index, REGULAR_CHUNKS_REQ_RETRY_THRESHOLD) }); // Safe to `take` here, as we're consuming `self` anyway and we're not using the @@ -1148,6 +1218,7 @@ impl RecoveryStrategy .wait_for_chunks( REGULAR_CHUNK_LABEL, common_params, + REGULAR_CHUNKS_REQ_RETRY_THRESHOLD, &mut validators_queue, &mut self.requesting_chunks, |unrequested_validators, diff --git a/polkadot/node/network/availability-recovery/src/tests.rs b/polkadot/node/network/availability-recovery/src/tests.rs index 590143eec028..3be0b4e96575 100644 --- a/polkadot/node/network/availability-recovery/src/tests.rs +++ b/polkadot/node/network/availability-recovery/src/tests.rs @@ -14,6 +14,8 @@ // You should have received a copy of the GNU General Public License // along with Polkadot. If not, see . +use crate::task::REGULAR_CHUNKS_REQ_RETRY_THRESHOLD; + use super::*; use std::{sync::Arc, time::Duration}; @@ -2169,3 +2171,101 @@ fn chunk_indices_are_shuffled(#[case] systematic_recovery: bool, #[case] shuffli virtual_overseer }); } + +#[rstest] +#[case(true)] +#[case(false)] +fn number_of_request_retries_is_bounded(#[case] should_fail: bool) { + let mut test_state = TestState::default(); + // We need the number of validators to be evenly divisible by the threshold for this test to be + // easier to write. + let n_validators = 6; + test_state.validators.truncate(n_validators); + test_state.validator_authority_id.truncate(n_validators); + let mut temp = test_state.validator_public.to_vec(); + temp.truncate(n_validators); + test_state.validator_public = temp.into(); + + let (chunks, erasure_root) = derive_erasure_chunks_with_proofs_and_root( + n_validators, + &test_state.available_data, + |_, _| {}, + ); + test_state.chunks = chunks; + test_state.candidate.descriptor.erasure_root = erasure_root; + + let subsystem = + AvailabilityRecoverySubsystem::with_chunks_only(request_receiver(), Metrics::new_dummy()); + + test_harness(subsystem, |mut virtual_overseer| async move { + overseer_signal( + &mut virtual_overseer, + OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( + test_state.current, + 1, + ))), + ) + .await; + + let (tx, rx) = oneshot::channel(); + + overseer_send( + &mut virtual_overseer, + AvailabilityRecoveryMessage::RecoverAvailableData( + test_state.candidate.clone(), + test_state.session_index, + None, + tx, + ), + ) + .await; + + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_client_features(&mut virtual_overseer).await; + test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; + test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + + // Network errors are considered non-fatal for regular chunk recovery but should be retried + // `REGULAR_CHUNKS_REQ_RETRY_THRESHOLD` times. + for _ in 1..REGULAR_CHUNKS_REQ_RETRY_THRESHOLD { + test_state + .test_chunk_requests( + test_state.candidate.hash(), + &mut virtual_overseer, + test_state.chunks.len(), + |_| Has::timeout(), + false, + ) + .await; + } + + if should_fail { + test_state + .test_chunk_requests( + test_state.candidate.hash(), + &mut virtual_overseer, + test_state.chunks.len(), + |_| Has::timeout(), + false, + ) + .await; + + assert_eq!(rx.await.unwrap().unwrap_err(), RecoveryError::Unavailable); + } else { + test_state + .test_chunk_requests( + test_state.candidate.hash(), + &mut virtual_overseer, + test_state.threshold(), + |_| Has::Yes, + false, + ) + .await; + + assert_eq!(rx.await.unwrap().unwrap(), test_state.available_data); + } + + virtual_overseer + }); +} From 716606ec939bf5a93f7d6a074436169f1733a74e Mon Sep 17 00:00:00 2001 From: alindima Date: Thu, 19 Oct 2023 17:09:48 +0300 Subject: [PATCH 057/126] add extensive unit tests for av-recovery --- .../network/availability-recovery/src/task.rs | 478 +++++++++++++++++- .../availability-recovery/src/tests.rs | 4 +- polkadot/primitives/src/v6/mod.rs | 6 + 3 files changed, 484 insertions(+), 4 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 4826cb6de146..bd8dc44120cd 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -88,6 +88,7 @@ pub trait RecoveryStrategy: S } /// Recovery parameters common to all strategies in a `RecoveryTask`. +#[derive(Clone)] pub struct RecoveryParams { /// Discovery ids of `validators`. pub validator_authority_keys: Vec, @@ -326,7 +327,6 @@ impl State { // return in that case for `launch_parallel_requests` to fill up slots again. while let Some(res) = requesting_chunks.next_with_timeout(TIMEOUT_START_NEW_REQUESTS).await { - println!("Validator count: {}", validators.len()); total_received_responses += 1; let (chunk_index, validator_index, request_result) = res; @@ -1245,8 +1245,482 @@ impl RecoveryStrategy #[cfg(test)] mod tests { - use super::*; + use super::{super::tests::*, *}; + use assert_matches::assert_matches; + use futures::{ + channel::mpsc::UnboundedReceiver, executor, future, Future, FutureExt, StreamExt, + }; + use parity_scale_codec::Error as DecodingError; use polkadot_erasure_coding::recovery_threshold; + use polkadot_node_primitives::{BlockData, PoV}; + use polkadot_node_subsystem::{AllMessages, TimeoutExt}; + use polkadot_node_subsystem_test_helpers::{sender_receiver, TestSubsystemSender}; + use polkadot_primitives::{HeadData, PersistedValidationData}; + use polkadot_primitives_test_helpers::dummy_hash; + use sp_keyring::Sr25519Keyring; + use std::sync::Arc; + + const TIMEOUT: Duration = Duration::from_secs(1); + + impl Default for RecoveryParams { + fn default() -> Self { + let validators = vec![ + Sr25519Keyring::Ferdie, + Sr25519Keyring::Alice, + Sr25519Keyring::Bob, + Sr25519Keyring::Charlie, + Sr25519Keyring::Dave, + Sr25519Keyring::One, + Sr25519Keyring::Two, + ]; + + Self { + validator_authority_keys: validator_authority_id(&validators), + n_validators: validators.len(), + threshold: recovery_threshold(validators.len()).unwrap(), + candidate_hash: CandidateHash(dummy_hash()), + erasure_root: dummy_hash(), + metrics: Metrics::new_dummy(), + bypass_availability_store: false, + } + } + } + + impl RecoveryParams { + fn create_chunks(&mut self) -> Vec { + let validation_data = PersistedValidationData { + parent_head: HeadData(vec![7, 8, 9]), + relay_parent_number: Default::default(), + max_pov_size: 1024, + relay_parent_storage_root: Default::default(), + }; + + let (chunks, erasure_root) = derive_erasure_chunks_with_proofs_and_root( + self.n_validators, + &AvailableData { + validation_data, + pov: Arc::new(PoV { block_data: BlockData(vec![42; 64]) }), + }, + |_, _| {}, + ); + + self.erasure_root = erasure_root; + + chunks + } + } + + fn test_harness, TestFut: Future>( + receiver_future: impl FnOnce(UnboundedReceiver) -> RecvFut, + test: impl FnOnce(TestSubsystemSender) -> TestFut, + ) { + let _ = env_logger::builder() + .is_test(true) + .filter(Some("polkadot_availability_recovery"), log::LevelFilter::Trace) + .try_init(); + + let (sender, receiver) = sender_receiver(); + + let test_fut = test(sender); + let receiver_future = receiver_future(receiver); + + futures::pin_mut!(test_fut); + futures::pin_mut!(receiver_future); + + executor::block_on(future::join(test_fut, receiver_future)).1 + } + + #[test] + fn test_historical_errors() { + let retry_threshold = 2; + let mut state = State::new(); + + assert!(state.can_retry_request(0.into(), 0.into(), retry_threshold)); + assert!(state.can_retry_request(0.into(), 0.into(), 0)); + state.record_error_non_fatal(0.into(), 0.into()); + assert!(state.can_retry_request(0.into(), 0.into(), retry_threshold)); + state.record_error_non_fatal(0.into(), 0.into()); + assert!(!state.can_retry_request(0.into(), 0.into(), retry_threshold)); + state.record_error_non_fatal(0.into(), 0.into()); + assert!(!state.can_retry_request(0.into(), 0.into(), retry_threshold)); + + assert!(state.can_retry_request(0.into(), 0.into(), 5)); + + state.record_error_fatal(1.into(), 1.into()); + assert!(!state.can_retry_request(1.into(), 1.into(), retry_threshold)); + state.record_error_non_fatal(1.into(), 1.into()); + assert!(!state.can_retry_request(1.into(), 1.into(), retry_threshold)); + + assert!(state.can_retry_request(4.into(), 4.into(), 0)); + assert!(state.can_retry_request(4.into(), 4.into(), retry_threshold)); + } + + #[test] + fn test_populate_from_av_store() { + let params = RecoveryParams::default(); + + // Failed to reach the av store + { + let params = params.clone(); + let candidate_hash = params.candidate_hash.clone(); + let mut state = State::new(); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + assert_matches!( + receiver.next().timeout(TIMEOUT).await.unwrap().unwrap(), + AllMessages::AvailabilityStore(AvailabilityStoreMessage::QueryAllChunks(hash, tx)) => { + assert_eq!(hash, candidate_hash); + drop(tx); + }); + }, + |mut sender| async move { + let local_chunk_indices = + state.populate_from_av_store(¶ms, &mut sender).await; + + assert_eq!(state.chunk_count(), 0); + assert_eq!(local_chunk_indices.len(), 0); + }, + ); + } + + // Found invalid chunk + { + let mut params = params.clone(); + let candidate_hash = params.candidate_hash.clone(); + let mut state = State::new(); + let chunks = params.create_chunks(); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + assert_matches!( + receiver.next().timeout(TIMEOUT).await.unwrap().unwrap(), + AllMessages::AvailabilityStore(AvailabilityStoreMessage::QueryAllChunks(hash, tx)) => { + assert_eq!(hash, candidate_hash); + let mut chunk = chunks[0].clone(); + chunk.index = 3.into(); + tx.send(vec![chunk]).unwrap(); + }); + }, + |mut sender| async move { + let local_chunk_indices = + state.populate_from_av_store(¶ms, &mut sender).await; + + assert_eq!(state.chunk_count(), 0); + assert_eq!(local_chunk_indices.len(), 1); + }, + ); + } + + // Found valid chunk + { + let mut params = params.clone(); + let candidate_hash = params.candidate_hash.clone(); + let mut state = State::new(); + let chunks = params.create_chunks(); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + assert_matches!( + receiver.next().timeout(TIMEOUT).await.unwrap().unwrap(), + AllMessages::AvailabilityStore(AvailabilityStoreMessage::QueryAllChunks(hash, tx)) => { + assert_eq!(hash, candidate_hash); + tx.send(vec![chunks[1].clone()]).unwrap(); + }); + }, + |mut sender| async move { + let local_chunk_indices = + state.populate_from_av_store(¶ms, &mut sender).await; + + assert_eq!(state.chunk_count(), 1); + assert_eq!(local_chunk_indices.len(), 1); + }, + ); + } + } + + #[test] + fn test_launch_parallel_chunk_requests() { + let params = RecoveryParams::default(); + + // No validators to request from. + { + let params = params.clone(); + let mut state = State::new(); + let mut ongoing_reqs = FuturesUndead::new(); + let mut validators = VecDeque::new(); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + // Shouldn't send any requests. + assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); + }, + |mut sender| async move { + state + .launch_parallel_chunk_requests( + "regular", + ¶ms, + &mut sender, + 3, + &mut validators, + &mut ongoing_reqs, + ) + .await; + + assert_eq!(ongoing_reqs.total_len(), 0); + }, + ); + } + + // Has validators but no need to request more. + { + let params = params.clone(); + let mut state = State::new(); + let mut ongoing_reqs = FuturesUndead::new(); + let mut validators = VecDeque::new(); + validators.push_back((ChunkIndex(1), ValidatorIndex(1))); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + // Shouldn't send any requests. + assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); + }, + |mut sender| async move { + state + .launch_parallel_chunk_requests( + "regular", + ¶ms, + &mut sender, + 0, + &mut validators, + &mut ongoing_reqs, + ) + .await; + + assert_eq!(ongoing_reqs.total_len(), 0); + }, + ); + } + + // Has validators but no need to request more. + { + let params = params.clone(); + let mut state = State::new(); + let mut ongoing_reqs = FuturesUndead::new(); + ongoing_reqs.push(async { todo!() }.boxed()); + ongoing_reqs.soft_cancel(); + let mut validators = VecDeque::new(); + validators.push_back((ChunkIndex(1), ValidatorIndex(1))); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + // Shouldn't send any requests. + assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); + }, + |mut sender| async move { + state + .launch_parallel_chunk_requests( + "regular", + ¶ms, + &mut sender, + 0, + &mut validators, + &mut ongoing_reqs, + ) + .await; + + assert_eq!(ongoing_reqs.total_len(), 1); + assert_eq!(ongoing_reqs.len(), 0); + }, + ); + } + + // Needs to request more. + { + let params = params.clone(); + let mut state = State::new(); + let mut ongoing_reqs = FuturesUndead::new(); + ongoing_reqs.push(async { todo!() }.boxed()); + ongoing_reqs.soft_cancel(); + ongoing_reqs.push(async { todo!() }.boxed()); + let mut validators = (0..3).map(|i| (ChunkIndex(i), ValidatorIndex(i))).collect(); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + assert_matches!( + receiver.next().timeout(TIMEOUT).await.unwrap().unwrap(), + AllMessages::NetworkBridgeTx(NetworkBridgeTxMessage::SendRequests(requests, _)) if requests.len() == 3 + ); + }, + |mut sender| async move { + state + .launch_parallel_chunk_requests( + "regular", + ¶ms, + &mut sender, + 10, + &mut validators, + &mut ongoing_reqs, + ) + .await; + + assert_eq!(ongoing_reqs.total_len(), 5); + assert_eq!(ongoing_reqs.len(), 4); + }, + ); + } + } + + #[test] + fn test_wait_for_chunks() { + let params = RecoveryParams::default(); + let retry_threshold = 2; + + // No ongoing requests. + { + let params = params.clone(); + let mut state = State::new(); + let mut ongoing_reqs: FuturesUndead<( + ChunkIndex, + ValidatorIndex, + Result, RequestError>, + )> = FuturesUndead::new(); + let mut validators = VecDeque::new(); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + // Shouldn't send any requests. + assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); + }, + |_| async move { + let (total_responses, error_count) = state + .wait_for_chunks( + "regular", + ¶ms, + retry_threshold, + &mut validators, + &mut ongoing_reqs, + |_, _, _, _, _| false, + ) + .await; + assert_eq!(total_responses, 0); + assert_eq!(error_count, 0); + assert_eq!(state.chunk_count(), 0); + }, + ); + } + + // Complex scenario. + { + let mut params = params.clone(); + let chunks = params.create_chunks(); + let mut state = State::new(); + let mut ongoing_reqs = FuturesUndead::new(); + ongoing_reqs + .push(future::ready((0.into(), 0.into(), Ok(Some(chunks[0].clone())))).boxed()); + ongoing_reqs.soft_cancel(); + ongoing_reqs + .push(future::ready((1.into(), 1.into(), Ok(Some(chunks[1].clone())))).boxed()); + ongoing_reqs.push(future::ready((2.into(), 2.into(), Ok(None))).boxed()); + ongoing_reqs.push( + future::ready(( + 3.into(), + 3.into(), + Err(RequestError::from(DecodingError::from("err"))), + )) + .boxed(), + ); + ongoing_reqs.push( + future::ready(( + 4.into(), + 4.into(), + Err(RequestError::NetworkError(RequestFailure::NotConnected)), + )) + .boxed(), + ); + + let mut validators = + (5..=params.n_validators as u32).map(|i| (i.into(), i.into())).collect(); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + // Shouldn't send any requests. + assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); + }, + |_| async move { + let (total_responses, error_count) = state + .wait_for_chunks( + "regular", + ¶ms, + retry_threshold, + &mut validators, + &mut ongoing_reqs, + |_, _, _, _, _| false, + ) + .await; + assert_eq!(total_responses, 5); + assert_eq!(error_count, 3); + assert_eq!(state.chunk_count(), 2); + + let expected_validators: VecDeque<_> = + (4..=params.n_validators as u32).map(|i| (i.into(), i.into())).collect(); + + assert_eq!(validators, expected_validators); + + // This time we'll go over the recoverable error threshold. + ongoing_reqs.push( + future::ready(( + 4.into(), + 4.into(), + Err(RequestError::NetworkError(RequestFailure::NotConnected)), + )) + .boxed(), + ); + + let (total_responses, error_count) = state + .wait_for_chunks( + "regular", + ¶ms, + retry_threshold, + &mut validators, + &mut ongoing_reqs, + |_, _, _, _, _| false, + ) + .await; + assert_eq!(total_responses, 1); + assert_eq!(error_count, 1); + assert_eq!(state.chunk_count(), 2); + + validators.pop_front(); + let expected_validators: VecDeque<_> = + (5..=params.n_validators as u32).map(|i| (i.into(), i.into())).collect(); + + assert_eq!(validators, expected_validators); + + // Check that can_conclude returning true terminates the loop. + let (total_responses, error_count) = state + .wait_for_chunks( + "regular", + ¶ms, + retry_threshold, + &mut validators, + &mut ongoing_reqs, + |_, _, _, _, _| true, + ) + .await; + assert_eq!(total_responses, 0); + assert_eq!(error_count, 0); + assert_eq!(state.chunk_count(), 2); + + assert_eq!(validators, expected_validators); + }, + ); + } + } + + // TODO: Test RecoveryTask::run(); + // TODO: test is_unavailable + // TODO: Test get_desired_request_count #[test] fn parallel_request_calculation_works_as_expected() { diff --git a/polkadot/node/network/availability-recovery/src/tests.rs b/polkadot/node/network/availability-recovery/src/tests.rs index 3be0b4e96575..f6683f874027 100644 --- a/polkadot/node/network/availability-recovery/src/tests.rs +++ b/polkadot/node/network/availability-recovery/src/tests.rs @@ -441,11 +441,11 @@ fn validator_pubkeys(val_ids: &[Sr25519Keyring]) -> IndexedVec Vec { +pub fn validator_authority_id(val_ids: &[Sr25519Keyring]) -> Vec { val_ids.iter().map(|v| v.public().into()).collect() } -fn derive_erasure_chunks_with_proofs_and_root( +pub fn derive_erasure_chunks_with_proofs_and_root( n_validators: usize, available_data: &AvailableData, alter_chunk: impl Fn(usize, &mut Vec), diff --git a/polkadot/primitives/src/v6/mod.rs b/polkadot/primitives/src/v6/mod.rs index 5218962ec730..76b1612006a6 100644 --- a/polkadot/primitives/src/v6/mod.rs +++ b/polkadot/primitives/src/v6/mod.rs @@ -136,6 +136,12 @@ impl From for ChunkIndex { } } +impl From for ChunkIndex { + fn from(n: u32) -> Self { + ChunkIndex(n) + } +} + // We should really get https://github.com/paritytech/polkadot/issues/2403 going .. impl From for ValidatorIndex { fn from(n: u32) -> Self { From c0b1c6c5f8c6cbb699164e59396ae15db775d787 Mon Sep 17 00:00:00 2001 From: alindima Date: Thu, 19 Oct 2023 17:24:33 +0300 Subject: [PATCH 058/126] fix clippy --- polkadot/node/network/availability-recovery/src/task.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index bd8dc44120cd..680fcf0a528b 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -1362,7 +1362,7 @@ mod tests { // Failed to reach the av store { let params = params.clone(); - let candidate_hash = params.candidate_hash.clone(); + let candidate_hash = params.candidate_hash; let mut state = State::new(); test_harness( @@ -1387,7 +1387,7 @@ mod tests { // Found invalid chunk { let mut params = params.clone(); - let candidate_hash = params.candidate_hash.clone(); + let candidate_hash = params.candidate_hash; let mut state = State::new(); let chunks = params.create_chunks(); @@ -1415,7 +1415,7 @@ mod tests { // Found valid chunk { let mut params = params.clone(); - let candidate_hash = params.candidate_hash.clone(); + let candidate_hash = params.candidate_hash; let mut state = State::new(); let chunks = params.create_chunks(); From 862501b3fce0c1094cddfa24e4f764074bd39e5d Mon Sep 17 00:00:00 2001 From: alindima Date: Thu, 19 Oct 2023 17:29:18 +0300 Subject: [PATCH 059/126] runtime config: enable AVAILABILITY_CHUNK_SHUFFLING for new chains --- polkadot/runtime/parachains/src/configuration.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/polkadot/runtime/parachains/src/configuration.rs b/polkadot/runtime/parachains/src/configuration.rs index 81812a1e3222..3082f32badea 100644 --- a/polkadot/runtime/parachains/src/configuration.rs +++ b/polkadot/runtime/parachains/src/configuration.rs @@ -313,7 +313,7 @@ impl> Default for HostConfiguration Date: Tue, 24 Oct 2023 12:28:08 +0300 Subject: [PATCH 060/126] use fatality in av-recovery --- Cargo.lock | 1 + .../availability-recovery/src/error.rs | 60 +++++- .../network/availability-recovery/src/lib.rs | 204 ++++++------------ .../network/availability-recovery/src/task.rs | 44 +++- polkadot/node/subsystem-types/Cargo.toml | 1 + polkadot/node/subsystem-types/src/errors.rs | 26 +-- 6 files changed, 172 insertions(+), 164 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 6568c4b9a11d..1d020e8b6d92 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -12448,6 +12448,7 @@ version = "1.0.0" dependencies = [ "async-trait", "derive_more", + "fatality", "futures", "orchestra", "polkadot-node-jaeger", diff --git a/polkadot/node/network/availability-recovery/src/error.rs b/polkadot/node/network/availability-recovery/src/error.rs index b51b0c13b538..11cba585348d 100644 --- a/polkadot/node/network/availability-recovery/src/error.rs +++ b/polkadot/node/network/availability-recovery/src/error.rs @@ -16,23 +16,36 @@ //! The `Error` and `Result` types used by the subsystem. +use crate::LOG_TARGET; +use fatality::{fatality, Nested}; use futures::channel::oneshot; -use polkadot_node_subsystem::ChainApiError; -use thiserror::Error; +use polkadot_node_network_protocol::request_response::incoming; +use polkadot_node_subsystem::{ChainApiError, RecoveryError, SubsystemError}; +use polkadot_primitives::Hash; /// Error type used by the Availability Recovery subsystem. -#[derive(Debug, Error)] -// TODO: add fatality +#[fatality(splitable)] pub enum Error { - #[error(transparent)] - Subsystem(#[from] polkadot_node_subsystem::SubsystemError), + #[fatal] + #[error("Spawning subsystem task failed: {0}")] + SpawnTask(#[source] SubsystemError), + + /// Receiving subsystem message from overseer failed. + #[fatal] + #[error("Receiving message from overseer failed: {0}")] + SubsystemReceive(#[source] SubsystemError), + #[fatal] #[error("failed to query full data from store")] CanceledQueryFullData(#[source] oneshot::Canceled), + #[fatal] #[error("failed to query session info")] CanceledSessionInfo(#[source] oneshot::Canceled), + #[error("`SessionInfo` is `None` at {0}")] + SessionInfoUnavailable(Hash), + #[error("failed to query client features from runtime")] RequestClientFeatures(#[source] polkadot_node_subsystem_util::runtime::Error), @@ -45,9 +58,9 @@ pub enum Error { #[error(transparent)] Erasure(#[from] polkadot_erasure_coding::Error), - #[error(transparent)] - Util(#[from] polkadot_node_subsystem_util::Error), - + // #[error(transparent)] + // Util(#[from] polkadot_node_subsystem_util::Error), + #[fatal] #[error("Oneshot for receiving response from Chain API got cancelled")] ChainApiSenderDropped(#[source] oneshot::Canceled), @@ -56,6 +69,35 @@ pub enum Error { #[error("Cannot find block number for given relay parent")] BlockNumberNotFound, + + #[fatal(forward)] + #[error("Error during recovery: {0}")] + Recovery(#[from] RecoveryError), + + #[fatal(forward)] + #[error("Retrieving next incoming request failed: {0}")] + IncomingRequest(#[from] incoming::Error), } pub type Result = std::result::Result; + +/// Utility for eating top level errors and log them. +/// +/// We basically always want to try and continue on error, unless the error is fatal for the entire +/// subsystem. +pub fn log_error(result: Result<()>) -> std::result::Result<(), FatalError> { + match result.into_nested()? { + Ok(()) => Ok(()), + Err(jfyi) => { + jfyi.log(); + Ok(()) + }, + } +} + +impl JfyiError { + /// Log a `JfyiError`. + pub fn log(self) { + gum::warn!(target: LOG_TARGET, error = ?self); + } +} diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index e72cec275681..06aa7532313c 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -40,14 +40,13 @@ use task::{ FetchSystematicChunksParams, }; -use fatality::Nested; use polkadot_erasure_coding::{ - branch_hash, branches, obtain_chunks_v1, recovery_threshold, systematic_recovery_threshold, + branches, obtain_chunks_v1, recovery_threshold, systematic_recovery_threshold, Error as ErasureEncodingError, }; use task::{RecoveryParams, RecoveryStrategy, RecoveryTask}; -use error::{Error, Result}; +use error::{log_error, Error, FatalError, Result}; use polkadot_node_network_protocol::{ request_response::{v1 as request_v1, IncomingRequestReceiver}, UnifiedReputationChange as Rep, @@ -58,14 +57,14 @@ use polkadot_node_subsystem::{ jaeger, messages::{AvailabilityRecoveryMessage, AvailabilityStoreMessage, ChainApiMessage}, overseer, ActiveLeavesUpdate, FromOrchestra, OverseerSignal, SpawnedSubsystem, - SubsystemContext, SubsystemError, SubsystemResult, + SubsystemContext, SubsystemError, }; use polkadot_node_subsystem_util::{ request_session_info, runtime::request_client_features, ChunkIndexCacheRegistry, }; use polkadot_primitives::{ - BlakeTwo256, BlockNumber, CandidateHash, CandidateReceipt, ChunkIndex, GroupIndex, Hash, HashT, - SessionIndex, SessionInfo, ValidatorIndex, + BlockNumber, CandidateHash, CandidateReceipt, ChunkIndex, GroupIndex, Hash, SessionIndex, + SessionInfo, ValidatorIndex, }; mod error; @@ -136,44 +135,6 @@ pub enum ErasureTask { Reencode(usize, Hash, AvailableData, oneshot::Sender>), } -const fn is_unavailable( - received_chunks: usize, - requesting_chunks: usize, - unrequested_validators: usize, - threshold: usize, -) -> bool { - received_chunks + requesting_chunks + unrequested_validators < threshold -} - -/// Check validity of a chunk. -fn is_chunk_valid(params: &RecoveryParams, chunk: &ErasureChunk) -> bool { - let anticipated_hash = - match branch_hash(¶ms.erasure_root, chunk.proof(), chunk.index.0 as usize) { - Ok(hash) => hash, - Err(e) => { - gum::debug!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - chunk_index = ?chunk.index, - error = ?e, - "Invalid Merkle proof", - ); - return false - }, - }; - let erasure_chunk_hash = BlakeTwo256::hash(&chunk.chunk); - if anticipated_hash != erasure_chunk_hash { - gum::debug!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - chunk_index = ?chunk.index, - "Merkle proof mismatch" - ); - return false - } - true -} - /// Re-encode the data into erasure chunks in order to verify /// the root hash of the provided Merkle tree, which is built /// on-top of the encoded chunks. @@ -336,9 +297,10 @@ impl AvailabilityRecoverySubsystem { } /// Handles a signal from the overseer. -async fn handle_signal(state: &mut State, signal: OverseerSignal) -> SubsystemResult { +/// Returns true if subsystem receives a deadly signal. +async fn handle_signal(state: &mut State, signal: OverseerSignal) -> bool { match signal { - OverseerSignal::Conclude => Ok(true), + OverseerSignal::Conclude => true, OverseerSignal::ActiveLeaves(ActiveLeavesUpdate { activated, .. }) => { // if activated is non-empty, set state.live_block to the highest block in `activated` if let Some(activated) = activated { @@ -347,9 +309,9 @@ async fn handle_signal(state: &mut State, signal: OverseerSignal) -> SubsystemRe } } - Ok(false) + false }, - OverseerSignal::BlockFinalized(_, _) => Ok(false), + OverseerSignal::BlockFinalized(_, _) => false, } } @@ -386,15 +348,8 @@ async fn launch_recovery_task( awaiting: vec![response_sender], }); - if let Err(e) = ctx.spawn("recovery-task", Box::pin(remote)) { - gum::warn!( - target: LOG_TARGET, - err = ?e, - "Failed to spawn a recovery task", - ); - } - - Ok(()) + ctx.spawn("recovery-task", Box::pin(remote)) + .map_err(|err| Error::SpawnTask(err)) } /// Handles an availability recovery request. @@ -419,14 +374,7 @@ async fn handle_recover( if let Some(result) = state.availability_lru.get(&candidate_hash).cloned().map(|v| v.into_result()) { - if let Err(e) = response_sender.send(result) { - gum::warn!( - target: LOG_TARGET, - err = ?e, - "Error responding with an availability recovery result", - ); - } - return Ok(()) + return response_sender.send(result).map_err(|_| Error::CanceledResponseSender) } if let Some(i) = @@ -584,11 +532,11 @@ async fn handle_recover( .await }, None => { - gum::warn!(target: LOG_TARGET, "SessionInfo is `None` at {:?}", state.live_block); response_sender .send(Err(RecoveryError::Unavailable)) .map_err(|_| Error::CanceledResponseSender)?; - Ok(()) + + Err(Error::SessionInfoUnavailable(state.live_block.1)) }, } } @@ -719,7 +667,7 @@ impl AvailabilityRecoverySubsystem { } } - async fn run(self, mut ctx: Context) -> SubsystemResult<()> { + async fn run(self, mut ctx: Context) -> std::result::Result<(), FatalError> { let mut state = State::default(); let Self { mut req_receiver, metrics, recovery_strategy_kind, bypass_availability_store } = self; @@ -757,52 +705,42 @@ impl AvailabilityRecoverySubsystem { loop { let recv_req = req_receiver.recv(|| vec![COST_INVALID_REQUEST]).fuse(); pin_mut!(recv_req); - futures::select! { + let res = futures::select! { erasure_task = erasure_task_rx.next() => { match erasure_task { Some(task) => { - let send_result = to_pool + to_pool .next() .expect("Pool size is `NonZeroUsize`; qed") .send(task) .await - .map_err(|_| RecoveryError::ChannelClosed); - - if let Err(err) = send_result { - gum::warn!( - target: LOG_TARGET, - ?err, - "Failed to send erasure coding task", - ); - } + .map_err(|_| RecoveryError::ChannelClosed) }, None => { - gum::debug!( - target: LOG_TARGET, - "Erasure task channel closed", - ); - - return Err(SubsystemError::with_origin("availability-recovery", RecoveryError::ChannelClosed)) + Err(RecoveryError::ChannelClosed) } - } + }.map_err(Into::into) } - v = ctx.recv().fuse() => { - match v? { - FromOrchestra::Signal(signal) => if handle_signal( - &mut state, - signal, - ).await? { - return Ok(()); - } - FromOrchestra::Communication { msg } => { - match msg { - AvailabilityRecoveryMessage::RecoverAvailableData( - receipt, - session_index, - maybe_backing_group, - response_sender, - ) => { - if let Err(e) = handle_recover( + signal = ctx.recv().fuse() => { + match signal { + Ok(signal) => { + match signal { + FromOrchestra::Signal(signal) => if handle_signal( + &mut state, + signal, + ).await { + return Ok(()); + } else { + Ok(()) + }, + FromOrchestra::Communication { + msg: AvailabilityRecoveryMessage::RecoverAvailableData( + receipt, + session_index, + maybe_backing_group, + response_sender, + ) + } => handle_recover( &mut state, &mut ctx, receipt, @@ -813,20 +751,14 @@ impl AvailabilityRecoverySubsystem { erasure_task_tx.clone(), recovery_strategy_kind.clone(), bypass_availability_store - ).await { - gum::warn!( - target: LOG_TARGET, - err = ?e, - "Error handling a recovery request", - ); - } - } + ).await } - } + }, + Err(e) => Err(Error::SubsystemReceive(e)) } } in_req = recv_req => { - match in_req.into_nested().map_err(|fatal| SubsystemError::with_origin("availability-recovery", fatal))? { + match in_req { Ok(req) => { if bypass_availability_store { gum::debug!( @@ -834,40 +766,42 @@ impl AvailabilityRecoverySubsystem { "Skipping request to availability-store.", ); let _ = req.send_response(None.into()); - continue - } - match query_full_data(&mut ctx, req.payload.candidate_hash).await { - Ok(res) => { - let _ = req.send_response(res.into()); - } - Err(e) => { - gum::debug!( - target: LOG_TARGET, - err = ?e, - "Failed to query available data.", - ); - - let _ = req.send_response(None.into()); + Ok(()) + } else { + match query_full_data(&mut ctx, req.payload.candidate_hash).await { + Ok(res) => { + let _ = req.send_response(res.into()); + Ok(()) + } + Err(e) => { + let _ = req.send_response(None.into()); + Err(e) + } } } } - Err(jfyi) => { - gum::debug!( - target: LOG_TARGET, - error = ?jfyi, - "Decoding incoming request failed" - ); - continue - } + Err(e) => Err(Error::IncomingRequest(e)) } } output = state.ongoing_recoveries.select_next_some() => { + let mut res = Ok(()); if let Some((candidate_hash, result)) = output { + if let Err(ref e) = result { + res = Err(Error::Recovery(e.clone())); + } + if let Ok(recovery) = CachedRecovery::try_from(result) { state.availability_lru.insert(candidate_hash, recovery); } } + + res } + }; + + // Only bubble up fatal errors, but log all of them. + if let Err(e) = res { + log_error(Err(e))?; } } } diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 680fcf0a528b..49197c5b1ad9 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -20,11 +20,11 @@ use crate::{ futures_undead::FuturesUndead, - is_chunk_valid, is_unavailable, metrics::{Metrics, REGULAR_CHUNK_LABEL, SYSTEMATIC_CHUNK_LABEL}, ErasureTask, LOG_TARGET, }; use futures::{channel::oneshot, SinkExt}; +use polkadot_erasure_coding::branch_hash; #[cfg(not(test))] use polkadot_node_network_protocol::request_response::CHUNK_REQUEST_TIMEOUT; use polkadot_node_network_protocol::request_response::{ @@ -35,7 +35,9 @@ use polkadot_node_subsystem::{ messages::{AvailabilityStoreMessage, NetworkBridgeTxMessage}, overseer, RecoveryError, }; -use polkadot_primitives::{AuthorityDiscoveryId, CandidateHash, ChunkIndex, Hash, ValidatorIndex}; +use polkadot_primitives::{ + AuthorityDiscoveryId, BlakeTwo256, CandidateHash, ChunkIndex, Hash, HashT, ValidatorIndex, +}; use rand::seq::SliceRandom; use sc_network::{IfDisconnected, OutboundFailure, RequestFailure}; use std::{ @@ -69,6 +71,44 @@ pub const SYSTEMATIC_CHUNKS_REQ_RETRY_THRESHOLD: u32 = 0; /// infinite retry loop. pub const REGULAR_CHUNKS_REQ_RETRY_THRESHOLD: u32 = 5; +const fn is_unavailable( + received_chunks: usize, + requesting_chunks: usize, + unrequested_validators: usize, + threshold: usize, +) -> bool { + received_chunks + requesting_chunks + unrequested_validators < threshold +} + +/// Check validity of a chunk. +fn is_chunk_valid(params: &RecoveryParams, chunk: &ErasureChunk) -> bool { + let anticipated_hash = + match branch_hash(¶ms.erasure_root, chunk.proof(), chunk.index.0 as usize) { + Ok(hash) => hash, + Err(e) => { + gum::debug!( + target: LOG_TARGET, + candidate_hash = ?params.candidate_hash, + chunk_index = ?chunk.index, + error = ?e, + "Invalid Merkle proof", + ); + return false + }, + }; + let erasure_chunk_hash = BlakeTwo256::hash(&chunk.chunk); + if anticipated_hash != erasure_chunk_hash { + gum::debug!( + target: LOG_TARGET, + candidate_hash = ?params.candidate_hash, + chunk_index = ?chunk.index, + "Merkle proof mismatch" + ); + return false + } + true +} + #[async_trait::async_trait] /// Common trait for runnable recovery strategies. pub trait RecoveryStrategy: Send { diff --git a/polkadot/node/subsystem-types/Cargo.toml b/polkadot/node/subsystem-types/Cargo.toml index a1c00cb0652e..cab5994e5d9c 100644 --- a/polkadot/node/subsystem-types/Cargo.toml +++ b/polkadot/node/subsystem-types/Cargo.toml @@ -9,6 +9,7 @@ license.workspace = true [dependencies] derive_more = "0.99.17" futures = "0.3.21" +fatality = "0.0.6" polkadot-primitives = { path = "../../primitives" } polkadot-node-primitives = { path = "../primitives" } polkadot-node-network-protocol = { path = "../network/protocol" } diff --git a/polkadot/node/subsystem-types/src/errors.rs b/polkadot/node/subsystem-types/src/errors.rs index 44136362a69e..b8e70641243e 100644 --- a/polkadot/node/subsystem-types/src/errors.rs +++ b/polkadot/node/subsystem-types/src/errors.rs @@ -18,6 +18,7 @@ use crate::JaegerError; use ::orchestra::OrchestraError as OverseerError; +use fatality::fatality; /// A description of an error causing the runtime API request to be unservable. #[derive(thiserror::Error, Debug, Clone)] @@ -68,32 +69,21 @@ impl core::fmt::Display for ChainApiError { impl std::error::Error for ChainApiError {} /// An error that may happen during Availability Recovery process. -#[derive(PartialEq, Debug, Clone)] +#[derive(PartialEq, Clone)] +#[fatality(splitable)] +#[allow(missing_docs)] pub enum RecoveryError { - /// A chunk is recovered but is invalid. + #[error("Invalid data")] Invalid, - /// A requested chunk is unavailable. + #[error("Data is unavailable")] Unavailable, - /// Erasure task channel closed, usually means node is shutting down. + #[fatal] + #[error("Erasure task channel closed")] ChannelClosed, } -impl std::fmt::Display for RecoveryError { - fn fmt(&self, f: &mut core::fmt::Formatter) -> Result<(), core::fmt::Error> { - let msg = match self { - RecoveryError::Invalid => "Invalid", - RecoveryError::Unavailable => "Unavailable", - RecoveryError::ChannelClosed => "ChannelClosed", - }; - - write!(f, "{}", msg) - } -} - -impl std::error::Error for RecoveryError {} - /// An error type that describes faults that may happen /// /// These are: From 25ba4fd8364ccbd9136cf6c1f0ccef8f23d133db Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 24 Oct 2023 13:39:19 +0300 Subject: [PATCH 061/126] bump SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT to 2 --- .../network/availability-recovery/src/task.rs | 65 ++++--------- .../availability-recovery/src/tests.rs | 91 ++++++++++++------- 2 files changed, 77 insertions(+), 79 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 49197c5b1ad9..258a44f09647 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -62,14 +62,14 @@ const TIMEOUT_START_NEW_REQUESTS: Duration = CHUNK_REQUEST_TIMEOUT; #[cfg(test)] const TIMEOUT_START_NEW_REQUESTS: Duration = Duration::from_millis(100); -/// The maximum number of times systematic chunk recovery will retry making a request for a given +/// The maximum number of times systematic chunk recovery will try making a request for a given /// (validator,chunk) pair, if the error was not fatal. Added so that we don't get stuck in an /// infinite retry loop. -pub const SYSTEMATIC_CHUNKS_REQ_RETRY_THRESHOLD: u32 = 0; -/// The maximum number of times regular chunk recovery will retry making a request for a given +pub const SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT: u32 = 2; +/// The maximum number of times regular chunk recovery will try making a request for a given /// (validator,chunk) pair, if the error was not fatal. Added so that we don't get stuck in an /// infinite retry loop. -pub const REGULAR_CHUNKS_REQ_RETRY_THRESHOLD: u32 = 5; +pub const REGULAR_CHUNKS_REQ_RETRY_LIMIT: u32 = 5; const fn is_unavailable( received_chunks: usize, @@ -355,7 +355,7 @@ impl State { ValidatorIndex, Result, RequestError>, )>, - can_conclude: impl Fn(usize, usize, usize, usize, usize) -> bool, + can_conclude: impl Fn(usize, usize, usize, usize) -> bool, ) -> (usize, usize) { let metrics = ¶ms.metrics; @@ -429,8 +429,8 @@ impl State { "Chunk fetching response was invalid", ); - // Record that we got an invalid chunk so that subsequent strategies - // don't try requesting this again. + // Record that we got an invalid chunk so that this or subsequent + // strategies don't try requesting this again. self.record_error_fatal(chunk_index, validator_index); }, RequestError::NetworkError(err) => { @@ -467,7 +467,6 @@ impl State { validators.len(), requesting_chunks.total_len(), self.chunk_count(), - error_count, total_received_responses - error_count, ) { gum::debug!( @@ -901,11 +900,7 @@ impl RecoveryStrategy // don't have the data from previous strategies. self.validators.retain(|(c_index, v_index)| { !state.received_chunks.contains_key(c_index) && - state.can_retry_request( - *c_index, - *v_index, - SYSTEMATIC_CHUNKS_REQ_RETRY_THRESHOLD, - ) + state.can_retry_request(*c_index, *v_index, SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT) }); systematic_chunk_count -= self.validators.len(); @@ -936,7 +931,7 @@ impl RecoveryStrategy total_requesting = %self.requesting_chunks.total_len(), n_validators = %common_params.n_validators, systematic_threshold = ?self.threshold, - "Data recovery is not possible", + "Data recovery from systematic chunks is not possible", ); return Err(RecoveryError::Unavailable) @@ -970,14 +965,13 @@ impl RecoveryStrategy .wait_for_chunks( SYSTEMATIC_CHUNK_LABEL, common_params, - SYSTEMATIC_CHUNKS_REQ_RETRY_THRESHOLD, + SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT, &mut validators_queue, &mut self.requesting_chunks, |unrequested_validators, in_flight_reqs, // Don't use this chunk count, as it may contain non-systematic chunks. _chunk_count, - error_count, success_responses| { let chunk_count = systematic_chunk_count + success_responses; let is_unavailable = Self::is_unavailable( @@ -987,31 +981,12 @@ impl RecoveryStrategy self.threshold, ); - error_count > (SYSTEMATIC_CHUNKS_REQ_RETRY_THRESHOLD as usize) || - chunk_count >= self.threshold || - is_unavailable + chunk_count >= self.threshold || is_unavailable }, ) .await; systematic_chunk_count += total_responses - error_count; - - // We can't afford any errors, as we need all the systematic chunks for this to work. - if error_count > 0 { - gum::debug!( - target: LOG_TARGET, - candidate_hash = ?common_params.candidate_hash, - erasure_root = ?common_params.erasure_root, - received = %systematic_chunk_count, - requesting = %self.requesting_chunks.len(), - total_requesting = %self.requesting_chunks.total_len(), - n_validators = %common_params.n_validators, - systematic_threshold = ?self.threshold, - "Systematic chunk recovery is not possible. Got an error while requesting a chunk", - ); - - return Err(RecoveryError::Unavailable) - } } } } @@ -1192,7 +1167,7 @@ impl RecoveryStrategy // don't have the data from previous strategies. self.validators.retain(|(c_index, v_index)| { !state.received_chunks.contains_key(c_index) && - state.can_retry_request(*c_index, *v_index, REGULAR_CHUNKS_REQ_RETRY_THRESHOLD) + state.can_retry_request(*c_index, *v_index, REGULAR_CHUNKS_REQ_RETRY_LIMIT) }); // Safe to `take` here, as we're consuming `self` anyway and we're not using the @@ -1258,14 +1233,10 @@ impl RecoveryStrategy .wait_for_chunks( REGULAR_CHUNK_LABEL, common_params, - REGULAR_CHUNKS_REQ_RETRY_THRESHOLD, + REGULAR_CHUNKS_REQ_RETRY_LIMIT, &mut validators_queue, &mut self.requesting_chunks, - |unrequested_validators, - in_flight_reqs, - chunk_count, - _error_count, - _success_responses| { + |unrequested_validators, in_flight_reqs, chunk_count, _success_responses| { chunk_count >= common_params.threshold || Self::is_unavailable( unrequested_validators, @@ -1640,7 +1611,7 @@ mod tests { retry_threshold, &mut validators, &mut ongoing_reqs, - |_, _, _, _, _| false, + |_, _, _, _| false, ) .await; assert_eq!(total_responses, 0); @@ -1695,7 +1666,7 @@ mod tests { retry_threshold, &mut validators, &mut ongoing_reqs, - |_, _, _, _, _| false, + |_, _, _, _| false, ) .await; assert_eq!(total_responses, 5); @@ -1724,7 +1695,7 @@ mod tests { retry_threshold, &mut validators, &mut ongoing_reqs, - |_, _, _, _, _| false, + |_, _, _, _| false, ) .await; assert_eq!(total_responses, 1); @@ -1745,7 +1716,7 @@ mod tests { retry_threshold, &mut validators, &mut ongoing_reqs, - |_, _, _, _, _| true, + |_, _, _, _| true, ) .await; assert_eq!(total_responses, 0); diff --git a/polkadot/node/network/availability-recovery/src/tests.rs b/polkadot/node/network/availability-recovery/src/tests.rs index f6683f874027..104933dfdf40 100644 --- a/polkadot/node/network/availability-recovery/src/tests.rs +++ b/polkadot/node/network/availability-recovery/src/tests.rs @@ -14,7 +14,7 @@ // You should have received a copy of the GNU General Public License // along with Polkadot. If not, see . -use crate::task::REGULAR_CHUNKS_REQ_RETRY_THRESHOLD; +use crate::task::{REGULAR_CHUNKS_REQ_RETRY_LIMIT, SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT}; use super::*; use std::{sync::Arc, time::Duration}; @@ -1515,15 +1515,17 @@ fn chunks_retry_until_all_nodes_respond(#[case] systematic_recovery: bool) { test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; if systematic_recovery { - test_state - .test_chunk_requests( - candidate_hash, - &mut virtual_overseer, - test_state.systematic_threshold(), - |_| Has::timeout(), - true, - ) - .await; + for _ in 0..SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT { + test_state + .test_chunk_requests( + candidate_hash, + &mut virtual_overseer, + test_state.systematic_threshold(), + |_| Has::timeout(), + true, + ) + .await; + } test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; } @@ -1537,7 +1539,8 @@ fn chunks_retry_until_all_nodes_respond(#[case] systematic_recovery: bool) { ) .await; - // We get to go another round! Actually, we get to go an infinite number of times. + // We get to go another round! Actually, we get to go `REGULAR_CHUNKS_REQ_RETRY_LIMIT` + // number of times. test_state .test_chunk_requests( candidate_hash, @@ -1694,34 +1697,35 @@ fn all_not_returning_requests_still_recovers_on_return(#[case] systematic_recove &mut virtual_overseer, n, |_| Has::DoesNotReturn, - false, + systematic_recovery, ) .await; future::join( async { Delay::new(Duration::from_millis(10)).await; - // Now retrieval should be able to recover. + // Now retrieval should be able progress. std::mem::drop(senders); }, async { - if systematic_recovery { - test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; - } test_state .test_chunk_requests( candidate_hash, &mut virtual_overseer, // Should start over: - test_state.validators.len(), + n, |_| Has::timeout(), - false, + systematic_recovery, ) .await }, ) .await; + if systematic_recovery { + test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + } + // we get to go another round! test_state .test_chunk_requests( @@ -2047,7 +2051,7 @@ fn systematic_chunks_are_not_requested_again_in_regular_recovery() { ) .await; - // Falls back to regular recovery. + // Falls back to regular recovery, since one validator returned a fatal error. test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; test_state @@ -2173,9 +2177,13 @@ fn chunk_indices_are_shuffled(#[case] systematic_recovery: bool, #[case] shuffli } #[rstest] -#[case(true)] -#[case(false)] -fn number_of_request_retries_is_bounded(#[case] should_fail: bool) { +#[case(true, false)] +#[case(false, true)] +#[case(false, false)] +fn number_of_request_retries_is_bounded( + #[case] systematic_recovery: bool, + #[case] should_fail: bool, +) { let mut test_state = TestState::default(); // We need the number of validators to be evenly divisible by the threshold for this test to be // easier to write. @@ -2194,8 +2202,22 @@ fn number_of_request_retries_is_bounded(#[case] should_fail: bool) { test_state.chunks = chunks; test_state.candidate.descriptor.erasure_root = erasure_root; - let subsystem = - AvailabilityRecoverySubsystem::with_chunks_only(request_receiver(), Metrics::new_dummy()); + let (subsystem, retry_limit) = match systematic_recovery { + false => ( + AvailabilityRecoverySubsystem::with_chunks_only( + request_receiver(), + Metrics::new_dummy(), + ), + REGULAR_CHUNKS_REQ_RETRY_LIMIT, + ), + true => ( + AvailabilityRecoverySubsystem::with_systematic_chunks( + request_receiver(), + Metrics::new_dummy(), + ), + SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT, + ), + }; test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( @@ -2226,16 +2248,21 @@ fn number_of_request_retries_is_bounded(#[case] should_fail: bool) { test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; - // Network errors are considered non-fatal for regular chunk recovery but should be retried - // `REGULAR_CHUNKS_REQ_RETRY_THRESHOLD` times. - for _ in 1..REGULAR_CHUNKS_REQ_RETRY_THRESHOLD { + let validator_count_per_iteration = if systematic_recovery { + test_state.systematic_threshold() + } else { + test_state.chunks.len() + }; + + // Network errors are considered non-fatal but should be retried a limited number of times. + for _ in 1..retry_limit { test_state .test_chunk_requests( test_state.candidate.hash(), &mut virtual_overseer, - test_state.chunks.len(), + validator_count_per_iteration, |_| Has::timeout(), - false, + systematic_recovery, ) .await; } @@ -2245,9 +2272,9 @@ fn number_of_request_retries_is_bounded(#[case] should_fail: bool) { .test_chunk_requests( test_state.candidate.hash(), &mut virtual_overseer, - test_state.chunks.len(), + validator_count_per_iteration, |_| Has::timeout(), - false, + systematic_recovery, ) .await; @@ -2259,7 +2286,7 @@ fn number_of_request_retries_is_bounded(#[case] should_fail: bool) { &mut virtual_overseer, test_state.threshold(), |_| Has::Yes, - false, + systematic_recovery, ) .await; From 850c0a3d9e341498417f837eda19a91f95636c9b Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 24 Oct 2023 13:43:09 +0300 Subject: [PATCH 062/126] rename historical_errors to recorded_errors --- .../node/network/availability-recovery/src/task.rs | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 258a44f09647..dcff80fc4d10 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -168,12 +168,12 @@ pub struct State { received_chunks: BTreeMap, /// A record of errors returned when requesting a chunk from a validator. - historical_errors: HashMap<(ChunkIndex, ValidatorIndex), ErrorRecord>, + recorded_errors: HashMap<(ChunkIndex, ValidatorIndex), ErrorRecord>, } impl State { fn new() -> Self { - Self { received_chunks: BTreeMap::new(), historical_errors: HashMap::new() } + Self { received_chunks: BTreeMap::new(), recorded_errors: HashMap::new() } } fn insert_chunk(&mut self, chunk_index: ChunkIndex, chunk: ErasureChunk) { @@ -185,12 +185,11 @@ impl State { } fn record_error_fatal(&mut self, chunk_index: ChunkIndex, validator_index: ValidatorIndex) { - self.historical_errors - .insert((chunk_index, validator_index), ErrorRecord::Fatal); + self.recorded_errors.insert((chunk_index, validator_index), ErrorRecord::Fatal); } fn record_error_non_fatal(&mut self, chunk_index: ChunkIndex, validator_index: ValidatorIndex) { - self.historical_errors + self.recorded_errors .entry((chunk_index, validator_index)) .and_modify(|record| { if let ErrorRecord::NonFatal(ref mut count) = record { @@ -206,7 +205,7 @@ impl State { validator_index: ValidatorIndex, retry_threshold: u32, ) -> bool { - match self.historical_errors.get(&(chunk_index, validator_index)) { + match self.recorded_errors.get(&(chunk_index, validator_index)) { None => true, Some(entry) => match entry { ErrorRecord::Fatal => false, @@ -1342,7 +1341,7 @@ mod tests { } #[test] - fn test_historical_errors() { + fn test_recorded_errors() { let retry_threshold = 2; let mut state = State::new(); From ab4d8d1acda2e1761eecf71e3defa7ff0e4d3182 Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 24 Oct 2023 14:53:30 +0300 Subject: [PATCH 063/126] deduplicate get_block_number --- .../node/core/bitfield-signing/src/lib.rs | 62 +++++++++---------- .../node/core/bitfield-signing/src/tests.rs | 4 +- .../availability-distribution/src/error.rs | 2 +- .../src/requester/mod.rs | 31 +++------- .../availability-recovery/src/error.rs | 10 +-- .../network/availability-recovery/src/lib.rs | 28 +++------ .../availability-recovery/src/tests.rs | 2 +- polkadot/node/subsystem-util/src/lib.rs | 18 +++++- 8 files changed, 69 insertions(+), 88 deletions(-) diff --git a/polkadot/node/core/bitfield-signing/src/lib.rs b/polkadot/node/core/bitfield-signing/src/lib.rs index 10e38be58777..b27ec4e6f798 100644 --- a/polkadot/node/core/bitfield-signing/src/lib.rs +++ b/polkadot/node/core/bitfield-signing/src/lib.rs @@ -28,16 +28,16 @@ use futures::{ }; use polkadot_node_subsystem::{ jaeger, - messages::{AvailabilityStoreMessage, BitfieldDistributionMessage, ChainApiMessage}, + messages::{AvailabilityStoreMessage, BitfieldDistributionMessage}, overseer, ActivatedLeaf, ChainApiError, FromOrchestra, OverseerSignal, PerLeafSpan, SpawnedSubsystem, SubsystemError, SubsystemResult, }; use polkadot_node_subsystem_util::{ - self as util, availability_chunk_index, request_validators, runtime::request_client_features, - Validator, + self as util, availability_chunk_index, get_block_number, request_validators, + runtime::request_client_features, Validator, }; use polkadot_primitives::{ - vstaging::ClientFeatures, AvailabilityBitfield, BlockNumber, CoreState, Hash, ValidatorIndex, + vstaging::ClientFeatures, AvailabilityBitfield, CoreState, Hash, ValidatorIndex, }; use sp_keystore::{Error as KeystoreError, KeystorePtr}; use std::{collections::HashMap, iter::FromIterator, time::Duration}; @@ -100,16 +100,33 @@ async fn get_core_availability( let _span = span.child("query-chunk-availability"); let relay_parent = core.candidate_descriptor.relay_parent; - let Ok(block_number) = get_block_number(*sender.lock().await, relay_parent).await else { - gum::warn!( - target: LOG_TARGET, - ?relay_parent, - ?core.candidate_hash, - para_id = %core.para_id(), - "Failed to get block number." - ); + let block_number = get_block_number::<_, Error>(*sender.lock().await, relay_parent).await; - return Ok(false) + let block_number = match block_number { + Ok(Some(block_number)) => block_number, + Ok(None) => { + gum::warn!( + target: LOG_TARGET, + ?relay_parent, + ?core.candidate_hash, + para_id = %core.para_id(), + "Failed to get block number." + ); + + return Ok(false) + }, + Err(err) => { + gum::warn!( + target: LOG_TARGET, + ?relay_parent, + ?core.candidate_hash, + para_id = %core.para_id(), + error = ?err, + "Failed to get block number." + ); + + return Ok(false) + }, }; let maybe_client_features = request_client_features(relay_parent, *sender.lock().await) @@ -381,22 +398,3 @@ where Ok(()) } - -async fn get_block_number( - sender: &mut Sender, - relay_parent: Hash, -) -> Result -where - Sender: overseer::SubsystemSender, -{ - let (tx, rx) = oneshot::channel(); - sender.send_message(ChainApiMessage::BlockNumber(relay_parent, tx)).await; - - let block_number = rx.await.map_err(Error::Oneshot)?.map_err(Error::ChainApi)?; - - if let Some(number) = block_number { - Ok(number) - } else { - Err(Error::BlockNumberNotFound) - } -} diff --git a/polkadot/node/core/bitfield-signing/src/tests.rs b/polkadot/node/core/bitfield-signing/src/tests.rs index ef97c00bfe4a..40ee0763d6b6 100644 --- a/polkadot/node/core/bitfield-signing/src/tests.rs +++ b/polkadot/node/core/bitfield-signing/src/tests.rs @@ -16,7 +16,9 @@ use super::*; use futures::{executor::block_on, pin_mut, StreamExt}; -use polkadot_node_subsystem::messages::{AllMessages, RuntimeApiMessage, RuntimeApiRequest}; +use polkadot_node_subsystem::messages::{ + AllMessages, ChainApiMessage, RuntimeApiMessage, RuntimeApiRequest, +}; use polkadot_primitives::{vstaging::ClientFeatures, CandidateHash, OccupiedCore}; use test_helpers::dummy_candidate_descriptor; diff --git a/polkadot/node/network/availability-distribution/src/error.rs b/polkadot/node/network/availability-distribution/src/error.rs index bcb2404e7c11..c0f0b5119cfb 100644 --- a/polkadot/node/network/availability-distribution/src/error.rs +++ b/polkadot/node/network/availability-distribution/src/error.rs @@ -49,7 +49,7 @@ pub enum Error { #[fatal] #[error("Oneshot for receiving response from Chain API got cancelled")] - ChainApiSenderDropped(#[source] oneshot::Canceled), + ChainApiSenderDropped(#[from] oneshot::Canceled), #[fatal] #[error("Retrieving response from Chain API unexpectedly failed with error: {0}")] diff --git a/polkadot/node/network/availability-distribution/src/requester/mod.rs b/polkadot/node/network/availability-distribution/src/requester/mod.rs index 0460c4afee31..84750ef4dd91 100644 --- a/polkadot/node/network/availability-distribution/src/requester/mod.rs +++ b/polkadot/node/network/availability-distribution/src/requester/mod.rs @@ -35,10 +35,11 @@ use polkadot_node_subsystem::{ overseer, ActivatedLeaf, ActiveLeavesUpdate, }; use polkadot_node_subsystem_util::{ + get_block_number, runtime::{get_occupied_cores, request_client_features, RuntimeInfo}, ChunkIndexCacheRegistry, }; -use polkadot_primitives::{BlockNumber, CandidateHash, Hash, OccupiedCore, SessionIndex}; +use polkadot_primitives::{CandidateHash, Hash, OccupiedCore, SessionIndex}; use super::{error::Error, FatalError, Metrics, Result, LOG_TARGET}; @@ -228,9 +229,12 @@ impl Requester { span.add_string_tag("already-requested-chunk", "false"); let tx = self.tx.clone(); let metrics = self.metrics.clone(); - let block_number = - get_block_number(context.sender(), core.candidate_descriptor.relay_parent) - .await?; + let block_number = get_block_number::<_, Error>( + context.sender(), + core.candidate_descriptor.relay_parent, + ) + .await? + .ok_or(Error::BlockNumberNotFound)?; let session_info = self .session_cache @@ -401,22 +405,3 @@ where .map_err(FatalError::ChainApi)?; Ok(ancestors) } - -async fn get_block_number(sender: &mut Sender, relay_parent: Hash) -> Result -where - Sender: overseer::SubsystemSender, -{ - let (tx, rx) = oneshot::channel(); - sender.send_message(ChainApiMessage::BlockNumber(relay_parent, tx)).await; - - let block_number = rx - .await - .map_err(FatalError::ChainApiSenderDropped)? - .map_err(FatalError::ChainApi)?; - - if let Some(number) = block_number { - Ok(number) - } else { - Err(Error::BlockNumberNotFound) - } -} diff --git a/polkadot/node/network/availability-recovery/src/error.rs b/polkadot/node/network/availability-recovery/src/error.rs index 11cba585348d..b46ffb6ea61c 100644 --- a/polkadot/node/network/availability-recovery/src/error.rs +++ b/polkadot/node/network/availability-recovery/src/error.rs @@ -39,10 +39,6 @@ pub enum Error { #[error("failed to query full data from store")] CanceledQueryFullData(#[source] oneshot::Canceled), - #[fatal] - #[error("failed to query session info")] - CanceledSessionInfo(#[source] oneshot::Canceled), - #[error("`SessionInfo` is `None` at {0}")] SessionInfoUnavailable(Hash), @@ -58,11 +54,9 @@ pub enum Error { #[error(transparent)] Erasure(#[from] polkadot_erasure_coding::Error), - // #[error(transparent)] - // Util(#[from] polkadot_node_subsystem_util::Error), #[fatal] - #[error("Oneshot for receiving response from Chain API got cancelled")] - ChainApiSenderDropped(#[source] oneshot::Canceled), + #[error(transparent)] + Oneshot(#[from] oneshot::Canceled), #[error("Retrieving response from Chain API unexpectedly failed with error: {0}")] ChainApi(#[from] ChainApiError), diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index 06aa7532313c..059b1b43204b 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -55,12 +55,13 @@ use polkadot_node_primitives::{AvailableData, ErasureChunk}; use polkadot_node_subsystem::{ errors::RecoveryError, jaeger, - messages::{AvailabilityRecoveryMessage, AvailabilityStoreMessage, ChainApiMessage}, + messages::{AvailabilityRecoveryMessage, AvailabilityStoreMessage}, overseer, ActiveLeavesUpdate, FromOrchestra, OverseerSignal, SpawnedSubsystem, SubsystemContext, SubsystemError, }; use polkadot_node_subsystem_util::{ - request_session_info, runtime::request_client_features, ChunkIndexCacheRegistry, + get_block_number, request_session_info, runtime::request_client_features, + ChunkIndexCacheRegistry, }; use polkadot_primitives::{ BlockNumber, CandidateHash, CandidateReceipt, ChunkIndex, GroupIndex, Hash, SessionIndex, @@ -387,14 +388,15 @@ async fn handle_recover( let _span = span.child("not-cached"); let session_info = request_session_info(state.live_block.1, session_index, ctx.sender()) .await - .await - .map_err(Error::CanceledSessionInfo)??; + .await??; let _span = span.child("session-info-ctx-received"); match session_info { Some(session_info) => { let block_number = - get_block_number(ctx.sender(), receipt.descriptor.relay_parent).await?; + get_block_number::<_, Error>(ctx.sender(), receipt.descriptor.relay_parent) + .await? + .ok_or(Error::BlockNumberNotFound)?; let chunk_indices = if let Some(chunk_indices) = state .chunk_indices @@ -903,19 +905,3 @@ async fn erasure_task_thread( } } } - -async fn get_block_number(sender: &mut Sender, relay_parent: Hash) -> Result -where - Sender: overseer::SubsystemSender, -{ - let (tx, rx) = oneshot::channel(); - sender.send_message(ChainApiMessage::BlockNumber(relay_parent, tx)).await; - - let block_number = rx.await.map_err(Error::ChainApiSenderDropped)?.map_err(Error::ChainApi)?; - - if let Some(number) = block_number { - Ok(number) - } else { - Err(Error::BlockNumberNotFound) - } -} diff --git a/polkadot/node/network/availability-recovery/src/tests.rs b/polkadot/node/network/availability-recovery/src/tests.rs index 104933dfdf40..badde4ec9654 100644 --- a/polkadot/node/network/availability-recovery/src/tests.rs +++ b/polkadot/node/network/availability-recovery/src/tests.rs @@ -32,7 +32,7 @@ use polkadot_node_network_protocol::request_response::{ }; use polkadot_node_primitives::{BlockData, PoV, Proof}; use polkadot_node_subsystem::messages::{ - AllMessages, NetworkBridgeTxMessage, RuntimeApiMessage, RuntimeApiRequest, + AllMessages, ChainApiMessage, NetworkBridgeTxMessage, RuntimeApiMessage, RuntimeApiRequest, }; use polkadot_node_subsystem_test_helpers::{ make_subsystem_context, mock::new_leaf, TestSubsystemContextHandle, diff --git a/polkadot/node/subsystem-util/src/lib.rs b/polkadot/node/subsystem-util/src/lib.rs index 95d368c5a52d..0aa5a8b79e51 100644 --- a/polkadot/node/subsystem-util/src/lib.rs +++ b/polkadot/node/subsystem-util/src/lib.rs @@ -33,8 +33,9 @@ use polkadot_node_primitives::BabeRandomness; use polkadot_node_subsystem::{ errors::{RuntimeApiError, SubsystemError}, messages::{RuntimeApiMessage, RuntimeApiRequest, RuntimeApiSender}, - overseer, SubsystemSender, + overseer, ChainApiError, SubsystemSender, }; +use polkadot_node_subsystem_types::messages::ChainApiMessage; use rand::{seq::SliceRandom, Rng}; use rand_chacha::ChaCha8Rng; @@ -636,3 +637,18 @@ pub fn availability_chunk_index( validator_index, ) } + +/// Get the block number by hash. +pub async fn get_block_number( + sender: &mut Sender, + relay_parent: Hash, +) -> Result, E> +where + Sender: overseer::SubsystemSender, + E: From + From, +{ + let (tx, rx) = oneshot::channel(); + sender.send_message(ChainApiMessage::BlockNumber(relay_parent, tx)).await; + + rx.await?.map_err(Into::into) +} From d9282ad4f65a115714b91562e973d2647f97d5dd Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 24 Oct 2023 15:41:48 +0300 Subject: [PATCH 064/126] address some review comments --- .../network/availability-recovery/src/task.rs | 13 ++++++++++++- polkadot/node/subsystem-util/src/lib.rs | 17 +++++++---------- 2 files changed, 19 insertions(+), 11 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index dcff80fc4d10..9de8e78e5620 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -354,7 +354,18 @@ impl State { ValidatorIndex, Result, RequestError>, )>, - can_conclude: impl Fn(usize, usize, usize, usize) -> bool, + // Function that returns `true` when this strategy can conclude. Either if we got enough + // chunks or if it's impossible. + can_conclude: impl Fn( + // Number of validators left in the queue + usize, + // Number of in flight requests + usize, + // Number of valid chunks received so far + usize, + // Number of valid chunks received in this iteration + usize, + ) -> bool, ) -> (usize, usize) { let metrics = ¶ms.metrics; diff --git a/polkadot/node/subsystem-util/src/lib.rs b/polkadot/node/subsystem-util/src/lib.rs index 0aa5a8b79e51..0c04c1628dfd 100644 --- a/polkadot/node/subsystem-util/src/lib.rs +++ b/polkadot/node/subsystem-util/src/lib.rs @@ -463,7 +463,7 @@ impl ChunkIndexCacheRegistry { validator_index: ValidatorIndex, ) -> Option { if let Some((shuffle, maybe_client_features)) = self.0.get(&(block_number, session_index)) { - Some(Self::compute_chunk_index_for_validator( + Some(Self::chunk_index_for_validator( maybe_client_features.as_ref(), shuffle, para_id, @@ -482,11 +482,8 @@ impl ChunkIndexCacheRegistry { para_id: ParaId, ) -> Option> { if let Some((shuffle, maybe_client_features)) = self.0.get(&(block_number, session_index)) { - let core_start_index = Self::compute_para_start_index( - maybe_client_features.as_ref(), - shuffle.len(), - para_id, - ); + let core_start_index = + Self::para_start_index(maybe_client_features.as_ref(), shuffle.len(), para_id); let chunk_indices = shuffle .clone() @@ -576,7 +573,7 @@ impl ChunkIndexCacheRegistry { } /// Return the availability chunk start index for this para. - fn compute_para_start_index( + fn para_start_index( maybe_client_features: Option<&ClientFeatures>, n_validators: usize, para_id: ParaId, @@ -596,14 +593,14 @@ impl ChunkIndexCacheRegistry { 0 } - fn compute_chunk_index_for_validator( + fn chunk_index_for_validator( maybe_client_features: Option<&ClientFeatures>, shuffle: &Vec, para_id: ParaId, validator_index: ValidatorIndex, ) -> ChunkIndex { let core_start_index = - Self::compute_para_start_index(maybe_client_features, shuffle.len(), para_id); + Self::para_start_index(maybe_client_features, shuffle.len(), para_id); let chunk_index = shuffle[(core_start_index + usize::try_from(validator_index.0) @@ -630,7 +627,7 @@ pub fn availability_chunk_index( n_validators, ); - ChunkIndexCacheRegistry::compute_chunk_index_for_validator( + ChunkIndexCacheRegistry::chunk_index_for_validator( maybe_client_features, &shuffle, para_id, From 9deec5f6f5a82e9fd0d2a4111bdab5b211df92a9 Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 24 Oct 2023 17:01:25 +0300 Subject: [PATCH 065/126] more tests --- .../network/availability-recovery/src/task.rs | 327 +++++++++++++++--- 1 file changed, 286 insertions(+), 41 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 9de8e78e5620..da94c30390ae 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -1272,7 +1272,7 @@ mod tests { channel::mpsc::UnboundedReceiver, executor, future, Future, FutureExt, StreamExt, }; use parity_scale_codec::Error as DecodingError; - use polkadot_erasure_coding::recovery_threshold; + use polkadot_erasure_coding::{recovery_threshold, systematic_recovery_threshold}; use polkadot_node_primitives::{BlockData, PoV}; use polkadot_node_subsystem::{AllMessages, TimeoutExt}; use polkadot_node_subsystem_test_helpers::{sender_receiver, TestSubsystemSender}; @@ -1309,19 +1309,9 @@ mod tests { impl RecoveryParams { fn create_chunks(&mut self) -> Vec { - let validation_data = PersistedValidationData { - parent_head: HeadData(vec![7, 8, 9]), - relay_parent_number: Default::default(), - max_pov_size: 1024, - relay_parent_storage_root: Default::default(), - }; - let (chunks, erasure_root) = derive_erasure_chunks_with_proofs_and_root( self.n_validators, - &AvailableData { - validation_data, - pov: Arc::new(PoV { block_data: BlockData(vec![42; 64]) }), - }, + &dummy_available_data(), |_, _| {}, ); @@ -1331,6 +1321,20 @@ mod tests { } } + fn dummy_available_data() -> AvailableData { + let validation_data = PersistedValidationData { + parent_head: HeadData(vec![7, 8, 9]), + relay_parent_number: Default::default(), + max_pov_size: 1024, + relay_parent_storage_root: Default::default(), + }; + + AvailableData { + validation_data, + pov: Arc::new(PoV { block_data: BlockData(vec![42; 64]) }), + } + } + fn test_harness, TestFut: Future>( receiver_future: impl FnOnce(UnboundedReceiver) -> RecvFut, test: impl FnOnce(TestSubsystemSender) -> TestFut, @@ -1739,36 +1743,277 @@ mod tests { } } - // TODO: Test RecoveryTask::run(); - // TODO: test is_unavailable - // TODO: Test get_desired_request_count + #[test] + fn test_recovery_strategy_run() { + let params = RecoveryParams::default(); + + struct GoodStrategy; + #[async_trait::async_trait] + impl RecoveryStrategy for GoodStrategy { + fn display_name(&self) -> &'static str { + "GoodStrategy" + } + + fn strategy_type(&self) -> &'static str { + "good_strategy" + } + + async fn run( + mut self: Box, + _state: &mut State, + _sender: &mut Sender, + _common_params: &RecoveryParams, + ) -> Result { + Ok(dummy_available_data()) + } + } + + struct UnavailableStrategy; + #[async_trait::async_trait] + impl RecoveryStrategy + for UnavailableStrategy + { + fn display_name(&self) -> &'static str { + "UnavailableStrategy" + } + + fn strategy_type(&self) -> &'static str { + "unavailable_strategy" + } + + async fn run( + mut self: Box, + _state: &mut State, + _sender: &mut Sender, + _common_params: &RecoveryParams, + ) -> Result { + Err(RecoveryError::Unavailable) + } + } + + struct InvalidStrategy; + #[async_trait::async_trait] + impl RecoveryStrategy + for InvalidStrategy + { + fn display_name(&self) -> &'static str { + "InvalidStrategy" + } + + fn strategy_type(&self) -> &'static str { + "invalid_strategy" + } + + async fn run( + mut self: Box, + _state: &mut State, + _sender: &mut Sender, + _common_params: &RecoveryParams, + ) -> Result { + Err(RecoveryError::Invalid) + } + } + + // No recovery strategies. + { + let mut params = params.clone(); + let strategies = VecDeque::new(); + params.bypass_availability_store = true; + + test_harness( + |mut receiver: UnboundedReceiver| async move { + // Shouldn't send any requests. + assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); + }, + |sender| async move { + let task = RecoveryTask::new(sender, params, strategies); + + assert_eq!(task.run().await.unwrap_err(), RecoveryError::Unavailable); + }, + ); + } + + // If we have the data in av-store, returns early. + { + let params = params.clone(); + let strategies = VecDeque::new(); + let candidate_hash = params.candidate_hash; + + test_harness( + |mut receiver: UnboundedReceiver| async move { + assert_matches!( + receiver.next().timeout(TIMEOUT).await.unwrap().unwrap(), + AllMessages::AvailabilityStore(AvailabilityStoreMessage::QueryAvailableData(hash, tx)) => { + assert_eq!(hash, candidate_hash); + tx.send(Some(dummy_available_data())).unwrap(); + }); + }, + |sender| async move { + let task = RecoveryTask::new(sender, params, strategies); + + assert_eq!(task.run().await.unwrap(), dummy_available_data()); + }, + ); + } + + // Strategy returning `RecoveryError::Invalid`` will short-circuit the entire task. + { + let mut params = params.clone(); + params.bypass_availability_store = true; + let mut strategies: VecDeque>> = + VecDeque::new(); + strategies.push_back(Box::new(InvalidStrategy)); + strategies.push_back(Box::new(GoodStrategy)); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + // Shouldn't send any requests. + assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); + }, + |sender| async move { + let task = RecoveryTask::new(sender, params, strategies); + + assert_eq!(task.run().await.unwrap_err(), RecoveryError::Invalid); + }, + ); + } + + // Strategy returning `Unavailable` will fall back to the next one. + { + let params = params.clone(); + let candidate_hash = params.candidate_hash; + let mut strategies: VecDeque>> = + VecDeque::new(); + strategies.push_back(Box::new(UnavailableStrategy)); + strategies.push_back(Box::new(GoodStrategy)); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + assert_matches!( + receiver.next().timeout(TIMEOUT).await.unwrap().unwrap(), + AllMessages::AvailabilityStore(AvailabilityStoreMessage::QueryAvailableData(hash, tx)) => { + assert_eq!(hash, candidate_hash); + tx.send(Some(dummy_available_data())).unwrap(); + }); + }, + |sender| async move { + let task = RecoveryTask::new(sender, params, strategies); + + assert_eq!(task.run().await.unwrap(), dummy_available_data()); + }, + ); + } + + // More complex scenario. + { + let params = params.clone(); + let candidate_hash = params.candidate_hash; + let mut strategies: VecDeque>> = + VecDeque::new(); + strategies.push_back(Box::new(UnavailableStrategy)); + strategies.push_back(Box::new(UnavailableStrategy)); + strategies.push_back(Box::new(GoodStrategy)); + strategies.push_back(Box::new(InvalidStrategy)); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + assert_matches!( + receiver.next().timeout(TIMEOUT).await.unwrap().unwrap(), + AllMessages::AvailabilityStore(AvailabilityStoreMessage::QueryAvailableData(hash, tx)) => { + assert_eq!(hash, candidate_hash); + tx.send(Some(dummy_available_data())).unwrap(); + }); + }, + |sender| async move { + let task = RecoveryTask::new(sender, params, strategies); + + assert_eq!(task.run().await.unwrap(), dummy_available_data()); + }, + ); + } + } #[test] - fn parallel_request_calculation_works_as_expected() { - let num_validators = 100; - let threshold = recovery_threshold(num_validators).unwrap(); - let (erasure_task_tx, _erasure_task_rx) = futures::channel::mpsc::channel(16); - - let mut fetch_chunks_task = FetchChunks::new(FetchChunksParams { - validators: (0..100u32).map(|i| (ChunkIndex(i), ValidatorIndex(i))).collect(), - erasure_task_tx, - }); - assert_eq!(fetch_chunks_task.get_desired_request_count(0, threshold), threshold); - fetch_chunks_task.error_count = 1; - fetch_chunks_task.total_received_responses = 1; - // We saturate at threshold (34): - assert_eq!(fetch_chunks_task.get_desired_request_count(0, threshold), threshold); - - fetch_chunks_task.total_received_responses = 2; - // With given error rate - still saturating: - assert_eq!(fetch_chunks_task.get_desired_request_count(1, threshold), threshold); - fetch_chunks_task.total_received_responses += 8; - // error rate: 1/10 - // remaining chunks needed: threshold (34) - 9 - // expected: 24 * (1+ 1/10) = (next greater integer) = 27 - assert_eq!(fetch_chunks_task.get_desired_request_count(9, threshold), 27); - fetch_chunks_task.error_count = 0; - // With error count zero - we should fetch exactly as needed: - assert_eq!(fetch_chunks_task.get_desired_request_count(10, threshold), threshold - 10); + fn test_is_unavailable() { + assert_eq!(is_unavailable(0, 0, 0, 0), false); + assert_eq!(is_unavailable(2, 2, 2, 0), false); + // Already reached the threshold. + assert_eq!(is_unavailable(3, 0, 10, 3), false); + assert_eq!(is_unavailable(3, 2, 0, 3), false); + assert_eq!(is_unavailable(3, 2, 10, 3), false); + // It's still possible to reach the threshold + assert_eq!(is_unavailable(0, 0, 10, 3), false); + assert_eq!(is_unavailable(0, 0, 3, 3), false); + assert_eq!(is_unavailable(1, 1, 1, 3), false); + // Not possible to reach the threshold + assert_eq!(is_unavailable(0, 0, 0, 3), true); + assert_eq!(is_unavailable(2, 3, 2, 10), true); + } + + #[test] + fn test_get_desired_request_count() { + // Systematic chunk recovery + { + let num_validators = 100; + let threshold = systematic_recovery_threshold(num_validators).unwrap(); + let (erasure_task_tx, _erasure_task_rx) = futures::channel::mpsc::channel(16); + + let systematic_chunks_task = FetchChunks::new(FetchChunksParams { + validators: (0..100u32).map(|i| (i.into(), i.into())).collect(), + erasure_task_tx, + }); + assert_eq!(systematic_chunks_task.get_desired_request_count(0, threshold), threshold); + assert_eq!( + systematic_chunks_task.get_desired_request_count(5, threshold), + threshold - 5 + ); + assert_eq!( + systematic_chunks_task.get_desired_request_count(num_validators * 2, threshold), + 0 + ); + assert_eq!( + systematic_chunks_task.get_desired_request_count(0, N_PARALLEL * 2), + N_PARALLEL + ); + assert_eq!( + systematic_chunks_task.get_desired_request_count(N_PARALLEL, N_PARALLEL + 2), + 2 + ); + } + + // Regular chunk recovery + { + let num_validators = 100; + let threshold = recovery_threshold(num_validators).unwrap(); + let (erasure_task_tx, _erasure_task_rx) = futures::channel::mpsc::channel(16); + + let mut fetch_chunks_task = FetchChunks::new(FetchChunksParams { + validators: (0..100u32).map(|i| (i.into(), i.into())).collect(), + erasure_task_tx, + }); + assert_eq!(fetch_chunks_task.get_desired_request_count(0, threshold), threshold); + fetch_chunks_task.error_count = 1; + fetch_chunks_task.total_received_responses = 1; + // We saturate at threshold (34): + assert_eq!(fetch_chunks_task.get_desired_request_count(0, threshold), threshold); + + // We saturate at the parallel limit. + assert_eq!(fetch_chunks_task.get_desired_request_count(0, N_PARALLEL + 2), N_PARALLEL); + + fetch_chunks_task.total_received_responses = 2; + // With given error rate - still saturating: + assert_eq!(fetch_chunks_task.get_desired_request_count(1, threshold), threshold); + fetch_chunks_task.total_received_responses = 10; + // error rate: 1/10 + // remaining chunks needed: threshold (34) - 9 + // expected: 24 * (1+ 1/10) = (next greater integer) = 27 + assert_eq!(fetch_chunks_task.get_desired_request_count(9, threshold), 27); + // We saturate at the parallel limit. + assert_eq!(fetch_chunks_task.get_desired_request_count(9, N_PARALLEL + 9), N_PARALLEL); + + fetch_chunks_task.error_count = 0; + // With error count zero - we should fetch exactly as needed: + assert_eq!(fetch_chunks_task.get_desired_request_count(10, threshold), threshold - 10); + } } } From 55243854a301d3343e0261551dd0d8f27ba216ad Mon Sep 17 00:00:00 2001 From: alindima Date: Wed, 25 Oct 2023 15:38:24 +0300 Subject: [PATCH 066/126] remove unused env_logger from test --- polkadot/node/network/availability-recovery/src/task.rs | 5 ----- 1 file changed, 5 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index da94c30390ae..43a14cde2818 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -1339,11 +1339,6 @@ mod tests { receiver_future: impl FnOnce(UnboundedReceiver) -> RecvFut, test: impl FnOnce(TestSubsystemSender) -> TestFut, ) { - let _ = env_logger::builder() - .is_test(true) - .filter(Some("polkadot_availability_recovery"), log::LevelFilter::Trace) - .try_init(); - let (sender, receiver) = sender_receiver(); let test_fut = test(sender); From 13c4253f555dcfcc820cf45463b8571d12f18d22 Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 3 Nov 2023 10:57:50 +0200 Subject: [PATCH 067/126] small metrics and logging improvements --- .../network/availability-recovery/src/lib.rs | 17 ++--- .../availability-recovery/src/metrics.rs | 19 ++---- .../network/availability-recovery/src/task.rs | 66 +++++++++++-------- 3 files changed, 53 insertions(+), 49 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index 703db5694efb..3fdaeee0971b 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -453,14 +453,15 @@ async fn handle_recover( chunk_size.saturating_mul(session_info.validators.len()) / 3; small_pov_size = pov_size_estimate < small_pov_limit; - gum::trace!( - target: LOG_TARGET, - ?candidate_hash, - pov_size_estimate, - small_pov_limit, - enabled = small_pov_size, - "Prefer fetch from backing group", - ); + if small_pov_size { + gum::trace!( + target: LOG_TARGET, + ?candidate_hash, + pov_size_estimate, + small_pov_limit, + "Prefer fetch from backing group", + ); + } } else { // we have a POV limit but were not able to query the chunk size, so // don't use the backing group. diff --git a/polkadot/node/network/availability-recovery/src/metrics.rs b/polkadot/node/network/availability-recovery/src/metrics.rs index f5c3523b9870..114d897f33d9 100644 --- a/polkadot/node/network/availability-recovery/src/metrics.rs +++ b/polkadot/node/network/availability-recovery/src/metrics.rs @@ -23,12 +23,6 @@ use polkadot_node_subsystem_util::metrics::{ }, }; -/// Label value used for regular chunk fetching. -pub const REGULAR_CHUNK_LABEL: &str = "regular"; - -/// Label value used for systematic chunk fetching. -pub const SYSTEMATIC_CHUNK_LABEL: &str = "systematic"; - /// Availability Distribution metrics. #[derive(Clone, Default)] pub struct Metrics(Option); @@ -40,13 +34,13 @@ struct MetricsInner { /// Gets incremented on each sent chunk requests. /// /// Split by chunk type: - /// - `regular` - /// - `systematic` + /// - `regular_chunks` + /// - `systematic_chunks` chunk_requests_issued: CounterVec, /// A counter for finished chunk requests. /// - /// Split by the chunk type (`regular` or `systematic`) + /// Split by the chunk type (`regular_chunks` or `systematic_chunks`) /// /// Also split by result: /// - `no_such_chunk` ... peer did not have the requested chunk @@ -58,17 +52,18 @@ struct MetricsInner { /// The duration of request to response. /// - /// Split by chunk type (`regular` or `systematic`). + /// Split by chunk type (`regular_chunks` or `systematic_chunks`). time_chunk_request: HistogramVec, /// The duration between the pure recovery and verification. /// - /// Split by chunk type (`regular` or `systematic`). + /// Split by recovery type (`regular_chunks`, `systematic_chunks` or `full_from_backers`). time_erasure_recovery: HistogramVec, /// How much time it takes to reconstruct the available data from chunks. /// - /// Split by chunk type (`regular` or `systematic`), ass the algorithms are different. + /// Split by chunk type (`regular_chunks` or `systematic_chunks`), as the algorithms are + /// different. time_erasure_reconstruct: HistogramVec, /// How much time it takes to re-encode the data into erasure chunks in order to verify diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 43a14cde2818..8bcc50209081 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -18,11 +18,7 @@ #![warn(missing_docs)] -use crate::{ - futures_undead::FuturesUndead, - metrics::{Metrics, REGULAR_CHUNK_LABEL, SYSTEMATIC_CHUNK_LABEL}, - ErasureTask, LOG_TARGET, -}; +use crate::{futures_undead::FuturesUndead, metrics::Metrics, ErasureTask, LOG_TARGET}; use futures::{channel::oneshot, SinkExt}; use polkadot_erasure_coding::branch_hash; #[cfg(not(test))] @@ -266,7 +262,7 @@ impl State { /// Launch chunk requests in parallel, according to the parameters. async fn launch_parallel_chunk_requests( &mut self, - chunk_type: &str, + strategy_type: &str, params: &RecoveryParams, sender: &mut Sender, desired_requests_count: usize, @@ -313,8 +309,8 @@ impl State { let (req, res) = OutgoingRequest::new(Recipient::Authority(validator), raw_request); requests.push(Requests::ChunkFetchingV1(req)); - params.metrics.on_chunk_request_issued(chunk_type); - let timer = params.metrics.time_chunk_request(chunk_type); + params.metrics.on_chunk_request_issued(strategy_type); + let timer = params.metrics.time_chunk_request(strategy_type); requesting_chunks.push(Box::pin(async move { let _timer = timer; @@ -345,7 +341,7 @@ impl State { /// Wait for a sufficient amount of chunks to reconstruct according to the provided `params`. async fn wait_for_chunks( &mut self, - chunk_type: &str, + strategy_type: &str, params: &RecoveryParams, retry_threshold: u32, validators: &mut VecDeque<(ChunkIndex, ValidatorIndex)>, @@ -384,7 +380,7 @@ impl State { match request_result { Ok(Some(chunk)) => if is_chunk_valid(params, &chunk) { - metrics.on_chunk_request_succeeded(chunk_type); + metrics.on_chunk_request_succeeded(strategy_type); gum::trace!( target: LOG_TARGET, candidate_hash = ?params.candidate_hash, @@ -394,14 +390,14 @@ impl State { ); self.insert_chunk(chunk.index, chunk); } else { - metrics.on_chunk_request_invalid(chunk_type); + metrics.on_chunk_request_invalid(strategy_type); error_count += 1; // Record that we got an invalid chunk so that subsequent strategies don't // try requesting this again. self.record_error_fatal(chunk_index, validator_index); }, Ok(None) => { - metrics.on_chunk_request_no_such_chunk(chunk_type); + metrics.on_chunk_request_no_such_chunk(strategy_type); gum::trace!( target: LOG_TARGET, candidate_hash = ?params.candidate_hash, @@ -428,7 +424,7 @@ impl State { match err { RequestError::InvalidResponse(_) => { - metrics.on_chunk_request_invalid(chunk_type); + metrics.on_chunk_request_invalid(strategy_type); gum::debug!( target: LOG_TARGET, @@ -447,9 +443,9 @@ impl State { // No debug logs on general network errors - that became very spammy // occasionally. if let RequestFailure::Network(OutboundFailure::Timeout) = err { - metrics.on_chunk_request_timeout(chunk_type); + metrics.on_chunk_request_timeout(strategy_type); } else { - metrics.on_chunk_request_error(chunk_type); + metrics.on_chunk_request_error(strategy_type); } self.record_error_non_fatal(chunk_index, validator_index); @@ -461,7 +457,7 @@ impl State { } }, RequestError::Canceled(_) => { - metrics.on_chunk_request_error(chunk_type); + metrics.on_chunk_request_error(strategy_type); self.record_error_non_fatal(chunk_index, validator_index); if self.can_retry_request(chunk_index, validator_index, retry_threshold) @@ -643,8 +639,10 @@ impl RecoveryStrategy sender: &mut Sender, common_params: &RecoveryParams, ) -> Result { + let strategy_type = RecoveryStrategy::::strategy_type(&*self); + loop { - // Pop the next validator, and proceed to next fetch_chunks_task if we're out. + // Pop the next validator. let validator_index = self.params.validators.pop().ok_or_else(|| RecoveryError::Unavailable)?; @@ -667,6 +665,8 @@ impl RecoveryStrategy match response.await { Ok(req_res::v1::AvailableDataFetchingResponse::AvailableData(data)) => { + let recovery_duration = + common_params.metrics.time_erasure_recovery(strategy_type); let (reencode_tx, reencode_rx) = oneshot::channel(); self.params .erasure_task_tx @@ -691,6 +691,8 @@ impl RecoveryStrategy return Ok(data) } else { + recovery_duration.map(|rd| rd.stop_and_discard()); + gum::debug!( target: LOG_TARGET, candidate_hash = ?common_params.candidate_hash, @@ -776,14 +778,14 @@ impl FetchSystematicChunks { std::cmp::min(max_requests_boundary, remaining_chunks) } - async fn attempt_systematic_recovery( + async fn attempt_systematic_recovery( &mut self, state: &mut State, common_params: &RecoveryParams, ) -> Result { - let recovery_duration = common_params.metrics.time_erasure_recovery(SYSTEMATIC_CHUNK_LABEL); - let reconstruct_duration = - common_params.metrics.time_erasure_reconstruct(SYSTEMATIC_CHUNK_LABEL); + let strategy_type = RecoveryStrategy::::strategy_type(self); + let recovery_duration = common_params.metrics.time_erasure_recovery(strategy_type); + let reconstruct_duration = common_params.metrics.time_erasure_reconstruct(strategy_type); let chunks = state .received_chunks .range( @@ -923,7 +925,7 @@ impl RecoveryStrategy // If received_chunks has `systematic_chunk_threshold` entries, attempt to recover the // data. if systematic_chunk_count >= self.threshold { - return self.attempt_systematic_recovery(state, common_params).await + return self.attempt_systematic_recovery::(state, common_params).await } if Self::is_unavailable( @@ -960,9 +962,11 @@ impl RecoveryStrategy "Requesting systematic availability chunks for a candidate", ); + let strategy_type = RecoveryStrategy::::strategy_type(&*self); + state .launch_parallel_chunk_requests( - SYSTEMATIC_CHUNK_LABEL, + strategy_type, common_params, sender, desired_requests_count, @@ -973,7 +977,7 @@ impl RecoveryStrategy let (total_responses, error_count) = state .wait_for_chunks( - SYSTEMATIC_CHUNK_LABEL, + strategy_type, common_params, SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT, &mut validators_queue, @@ -1074,12 +1078,14 @@ impl FetchChunks { ) } - async fn attempt_recovery( + async fn attempt_recovery( &mut self, state: &mut State, common_params: &RecoveryParams, ) -> Result { - let recovery_duration = common_params.metrics.time_erasure_recovery(REGULAR_CHUNK_LABEL); + let recovery_duration = common_params + .metrics + .time_erasure_recovery(RecoveryStrategy::::strategy_type(self)); // Send request to reconstruct available data from chunks. let (avilable_data_tx, available_data_rx) = oneshot::channel(); @@ -1191,7 +1197,7 @@ impl RecoveryStrategy // Do this before requesting any chunks because we may have enough of them coming from // past RecoveryStrategies. if state.chunk_count() >= common_params.threshold { - return self.attempt_recovery(state, common_params).await + return self.attempt_recovery::(state, common_params).await } if Self::is_unavailable( @@ -1228,9 +1234,11 @@ impl RecoveryStrategy "Requesting availability chunks for a candidate", ); + let strategy_type = RecoveryStrategy::::strategy_type(&*self); + state .launch_parallel_chunk_requests( - REGULAR_CHUNK_LABEL, + strategy_type, common_params, sender, desired_requests_count, @@ -1241,7 +1249,7 @@ impl RecoveryStrategy let (total_responses, error_count) = state .wait_for_chunks( - REGULAR_CHUNK_LABEL, + strategy_type, common_params, REGULAR_CHUNKS_REQ_RETRY_LIMIT, &mut validators_queue, From 81ebaa8a131fc8771b8e8b96859f71d130575967 Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 7 Nov 2023 10:48:32 +0200 Subject: [PATCH 068/126] add more explicit error message on decode --- polkadot/erasure-coding/src/lib.rs | 7 +++++-- polkadot/node/network/availability-recovery/src/task.rs | 4 ++-- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/polkadot/erasure-coding/src/lib.rs b/polkadot/erasure-coding/src/lib.rs index e5d59452401f..c80331e20c3e 100644 --- a/polkadot/erasure-coding/src/lib.rs +++ b/polkadot/erasure-coding/src/lib.rs @@ -69,6 +69,9 @@ pub enum Error { /// Bad payload in reconstructed bytes. #[error("Reconstructed payload invalid")] BadPayload, + /// Unable to decode reconstructed bytes. + #[error("Unable to decode reconstructed payload: {0}")] + Decode(#[source] parity_scale_codec::Error), /// Invalid branch proof. #[error("Invalid branch proof")] InvalidBranchProof, @@ -172,7 +175,7 @@ pub fn reconstruct_from_systematic( } } - Decode::decode(&mut &systematic_bytes[..]).map_err(|_| Error::BadPayload) + Decode::decode(&mut &systematic_bytes[..]).map_err(|err| Error::Decode(err)) } /// Obtain erasure-coded chunks for v1 `AvailableData`, one for each validator. @@ -263,7 +266,7 @@ where Ok(payload_bytes) => payload_bytes, }; - Decode::decode(&mut &payload_bytes[..]).map_err(|_| Error::BadPayload) + Decode::decode(&mut &payload_bytes[..]).map_err(|err| Error::Decode(err)) } /// An iterator that yields merkle branches and chunk data for all chunks to diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 8bcc50209081..3b1187307a8b 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -852,7 +852,7 @@ impl FetchSystematicChunks { candidate_hash = ?common_params.candidate_hash, erasure_root = ?common_params.erasure_root, ?err, - "Systematic data recovery error ", + "Systematic data recovery error", ); Err(RecoveryError::Invalid) @@ -1148,7 +1148,7 @@ impl FetchChunks { candidate_hash = ?common_params.candidate_hash, erasure_root = ?common_params.erasure_root, ?err, - "Data recovery error ", + "Data recovery error", ); Err(RecoveryError::Invalid) From f681e1e97dd9dc0422122ac542bd347f7b2a98e2 Mon Sep 17 00:00:00 2001 From: alindima Date: Wed, 8 Nov 2023 12:36:24 +0200 Subject: [PATCH 069/126] add more metrics for full data requests --- .../availability-recovery/src/metrics.rs | 77 ++++++++++++++++++- .../network/availability-recovery/src/task.rs | 36 +++++++-- 2 files changed, 103 insertions(+), 10 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/metrics.rs b/polkadot/node/network/availability-recovery/src/metrics.rs index 114d897f33d9..fd6892615865 100644 --- a/polkadot/node/network/availability-recovery/src/metrics.rs +++ b/polkadot/node/network/availability-recovery/src/metrics.rs @@ -45,11 +45,25 @@ struct MetricsInner { /// Also split by result: /// - `no_such_chunk` ... peer did not have the requested chunk /// - `timeout` ... request timed out. - /// - `network_error` ... Some networking issue except timeout + /// - `error` ... Some networking issue except timeout /// - `invalid` ... Chunk was received, but not valid. /// - `success` chunk_requests_finished: CounterVec, + /// Number of sent available data requests. + full_data_requests_issued: Counter, + + /// Counter for finished available data requests. + /// + /// Split by the result type: + /// + /// - `no_such_data` ... peer did not have the requested data + /// - `timeout` ... request timed out. + /// - `error` ... Some networking issue except timeout + /// - `invalid` ... data was received, but not valid. + /// - `success` + full_data_requests_finished: CounterVec, + /// The duration of request to response. /// /// Split by chunk type (`regular_chunks` or `systematic_chunks`). @@ -101,6 +115,13 @@ impl Metrics { } } + /// Increment counter for full data requests. + pub fn on_full_request_issued(&self) { + if let Some(metrics) = &self.0 { + metrics.full_data_requests_issued.inc() + } + } + /// A chunk request timed out. pub fn on_chunk_request_timeout(&self, chunk_type: &str) { if let Some(metrics) = &self.0 { @@ -111,6 +132,13 @@ impl Metrics { } } + /// A full data request timed out. + pub fn on_full_request_timeout(&self) { + if let Some(metrics) = &self.0 { + metrics.full_data_requests_finished.with_label_values(&["timeout"]).inc() + } + } + /// A chunk request failed because validator did not have its chunk. pub fn on_chunk_request_no_such_chunk(&self, chunk_type: &str) { if let Some(metrics) = &self.0 { @@ -121,6 +149,13 @@ impl Metrics { } } + /// A full data request failed because the validator did not have it. + pub fn on_full_request_no_such_data(&self) { + if let Some(metrics) = &self.0 { + metrics.full_data_requests_finished.with_label_values(&["no_such_data"]).inc() + } + } + /// A chunk request failed for some non timeout related network error. pub fn on_chunk_request_error(&self, chunk_type: &str) { if let Some(metrics) = &self.0 { @@ -128,6 +163,13 @@ impl Metrics { } } + /// A full data request failed for some non timeout related network error. + pub fn on_full_request_error(&self) { + if let Some(metrics) = &self.0 { + metrics.full_data_requests_finished.with_label_values(&["error"]).inc() + } + } + /// A chunk request succeeded, but was not valid. pub fn on_chunk_request_invalid(&self, chunk_type: &str) { if let Some(metrics) = &self.0 { @@ -138,6 +180,13 @@ impl Metrics { } } + /// A full data request succeeded, but was not valid. + pub fn on_full_request_invalid(&self) { + if let Some(metrics) = &self.0 { + metrics.full_data_requests_finished.with_label_values(&["invalid"]).inc() + } + } + /// A chunk request succeeded. pub fn on_chunk_request_succeeded(&self, chunk_type: &str) { if let Some(metrics) = &self.0 { @@ -148,6 +197,13 @@ impl Metrics { } } + /// A full data request succeeded. + pub fn on_full_request_succeeded(&self) { + if let Some(metrics) = &self.0 { + metrics.full_data_requests_finished.with_label_values(&["success"]).inc() + } + } + /// Get a timer to time request/response duration. pub fn time_chunk_request(&self, chunk_type: &str) -> Option { self.0.as_ref().map(|metrics| { @@ -228,6 +284,13 @@ impl metrics::Metrics for Metrics { )?, registry, )?, + full_data_requests_issued: prometheus::register( + Counter::new( + "polkadot_parachain_availability_recovery_full_data_requests_issued", + "Total number of issued full data requests.", + )?, + registry, + )?, chunk_requests_finished: prometheus::register( CounterVec::new( Opts::new( @@ -238,6 +301,16 @@ impl metrics::Metrics for Metrics { )?, registry, )?, + full_data_requests_finished: prometheus::register( + CounterVec::new( + Opts::new( + "polkadot_parachain_availability_recovery_full_data_requests_finished", + "Total number of full data requests finished.", + ), + &["result"], + )?, + registry, + )?, time_chunk_request: prometheus::register( prometheus::HistogramVec::new(prometheus::HistogramOpts::new( "polkadot_parachain_availability_recovery_time_chunk_request", @@ -285,7 +358,7 @@ impl metrics::Metrics for Metrics { )?, full_recoveries_started: prometheus::register( Counter::new( - "polkadot_parachain_availability_recovery_recovieries_started", + "polkadot_parachain_availability_recovery_recoveries_started", "Total number of started recoveries.", )?, registry, diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 3b1187307a8b..45cd9a192242 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -663,6 +663,8 @@ impl RecoveryStrategy )) .await; + common_params.metrics.on_full_request_issued(); + match response.await { Ok(req_res::v1::AvailableDataFetchingResponse::AvailableData(data)) => { let recovery_duration = @@ -689,8 +691,10 @@ impl RecoveryStrategy "Received full data", ); + common_params.metrics.on_full_request_succeeded(); return Ok(data) } else { + common_params.metrics.on_full_request_invalid(); recovery_duration.map(|rd| rd.stop_and_discard()); gum::debug!( @@ -703,14 +707,30 @@ impl RecoveryStrategy // it doesn't help to report the peer with req/res. } }, - Ok(req_res::v1::AvailableDataFetchingResponse::NoSuchData) => {}, - Err(e) => gum::debug!( - target: LOG_TARGET, - candidate_hash = ?common_params.candidate_hash, - ?validator_index, - err = ?e, - "Error fetching full available data." - ), + Ok(req_res::v1::AvailableDataFetchingResponse::NoSuchData) => { + common_params.metrics.on_full_request_no_such_data(); + }, + Err(e) => { + match &e { + RequestError::Canceled(_) => common_params.metrics.on_full_request_error(), + RequestError::InvalidResponse(_) => + common_params.metrics.on_full_request_invalid(), + RequestError::NetworkError(req_failure) => { + if let RequestFailure::Network(OutboundFailure::Timeout) = req_failure { + common_params.metrics.on_full_request_timeout(); + } else { + common_params.metrics.on_full_request_error(); + } + }, + }; + gum::debug!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + ?validator_index, + err = ?e, + "Error fetching full available data." + ); + }, } } } From 60f0e797a24411e8aac2ba8a7c90e1153cc6d88f Mon Sep 17 00:00:00 2001 From: alindima Date: Wed, 8 Nov 2023 13:33:48 +0200 Subject: [PATCH 070/126] move ChunkIndexCacheRegistry to its own module --- .../node/core/bitfield-signing/src/lib.rs | 4 +- .../src/requester/mod.rs | 2 +- .../network/availability-recovery/src/lib.rs | 4 +- .../subsystem-util/src/availability_chunks.rs | 391 ++++++++++++++++++ polkadot/node/subsystem-util/src/lib.rs | 213 +--------- polkadot/node/subsystem-util/src/tests.rs | 169 -------- 6 files changed, 404 insertions(+), 379 deletions(-) create mode 100644 polkadot/node/subsystem-util/src/availability_chunks.rs diff --git a/polkadot/node/core/bitfield-signing/src/lib.rs b/polkadot/node/core/bitfield-signing/src/lib.rs index b27ec4e6f798..0e5c5e726a4c 100644 --- a/polkadot/node/core/bitfield-signing/src/lib.rs +++ b/polkadot/node/core/bitfield-signing/src/lib.rs @@ -33,8 +33,8 @@ use polkadot_node_subsystem::{ SpawnedSubsystem, SubsystemError, SubsystemResult, }; use polkadot_node_subsystem_util::{ - self as util, availability_chunk_index, get_block_number, request_validators, - runtime::request_client_features, Validator, + self as util, availability_chunks::availability_chunk_index, get_block_number, + request_validators, runtime::request_client_features, Validator, }; use polkadot_primitives::{ vstaging::ClientFeatures, AvailabilityBitfield, CoreState, Hash, ValidatorIndex, diff --git a/polkadot/node/network/availability-distribution/src/requester/mod.rs b/polkadot/node/network/availability-distribution/src/requester/mod.rs index 84750ef4dd91..c6cfdcdead14 100644 --- a/polkadot/node/network/availability-distribution/src/requester/mod.rs +++ b/polkadot/node/network/availability-distribution/src/requester/mod.rs @@ -35,9 +35,9 @@ use polkadot_node_subsystem::{ overseer, ActivatedLeaf, ActiveLeavesUpdate, }; use polkadot_node_subsystem_util::{ + availability_chunks::ChunkIndexCacheRegistry, get_block_number, runtime::{get_occupied_cores, request_client_features, RuntimeInfo}, - ChunkIndexCacheRegistry, }; use polkadot_primitives::{CandidateHash, Hash, OccupiedCore, SessionIndex}; diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index 3fdaeee0971b..821b8756baba 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -60,8 +60,8 @@ use polkadot_node_subsystem::{ SubsystemContext, SubsystemError, }; use polkadot_node_subsystem_util::{ - get_block_number, request_session_info, runtime::request_client_features, - ChunkIndexCacheRegistry, + availability_chunks::ChunkIndexCacheRegistry, get_block_number, request_session_info, + runtime::request_client_features, }; use polkadot_primitives::{ BlockNumber, CandidateHash, CandidateReceipt, ChunkIndex, GroupIndex, Hash, SessionIndex, diff --git a/polkadot/node/subsystem-util/src/availability_chunks.rs b/polkadot/node/subsystem-util/src/availability_chunks.rs new file mode 100644 index 000000000000..ca8d782516e0 --- /dev/null +++ b/polkadot/node/subsystem-util/src/availability_chunks.rs @@ -0,0 +1,391 @@ +use polkadot_node_primitives::BabeRandomness; +use polkadot_primitives::{ + vstaging::ClientFeatures, BlockNumber, ChunkIndex, Id as ParaId, SessionIndex, ValidatorIndex, +}; +use rand::{seq::SliceRandom, Rng, SeedableRng}; +use rand_chacha::ChaCha8Rng; + +use schnellru::{ByLength, LruMap}; + +/// Object used for holding and computing assigned chunk indices for validators. +pub struct ChunkIndexCacheRegistry( + LruMap<(BlockNumber, SessionIndex), (Vec, Option)>, +); + +impl ChunkIndexCacheRegistry { + /// Initialize with the cache capacity. + pub fn new(capacity: u32) -> Self { + Self(LruMap::new(ByLength::new(capacity))) + } + + /// Return the per-validator chunk index if present in the cache. + pub fn query_cache_for_validator( + &mut self, + block_number: BlockNumber, + session_index: SessionIndex, + para_id: ParaId, + validator_index: ValidatorIndex, + ) -> Option { + if let Some((shuffle, maybe_client_features)) = self.0.get(&(block_number, session_index)) { + Some(Self::chunk_index_for_validator( + maybe_client_features.as_ref(), + shuffle, + para_id, + validator_index, + )) + } else { + None + } + } + + /// Return the per-para chunk index vector if present in the cache. + pub fn query_cache_for_para( + &mut self, + block_number: BlockNumber, + session_index: SessionIndex, + para_id: ParaId, + ) -> Option> { + if let Some((shuffle, maybe_client_features)) = self.0.get(&(block_number, session_index)) { + let core_start_index = + Self::para_start_index(maybe_client_features.as_ref(), shuffle.len(), para_id); + + let chunk_indices = shuffle + .clone() + .into_iter() + .cycle() + .skip(core_start_index) + .take(shuffle.len()) + .collect(); + + Some(chunk_indices) + } else { + None + } + } + + /// Return and populate the cache with the per-validator chunk index. + /// Should only be called if `query_cache_for_validator` returns `None`. + pub fn populate_for_validator( + &mut self, + maybe_client_features: Option, + babe_randomness: BabeRandomness, + n_validators: usize, + block_number: BlockNumber, + session_index: SessionIndex, + para_id: ParaId, + validator_index: ValidatorIndex, + ) -> ChunkIndex { + let shuffle = Self::get_shuffle( + maybe_client_features.as_ref(), + block_number, + babe_randomness, + n_validators, + ); + self.0.insert((block_number, session_index), (shuffle, maybe_client_features)); + + self.query_cache_for_validator(block_number, session_index, para_id, validator_index) + .expect("We just inserted the entry.") + } + + /// Return and populate the cache with the per-para chunk index vector. + /// Should only be called if `query_cache_for_para` returns `None`. + pub fn populate_for_para( + &mut self, + maybe_client_features: Option, + babe_randomness: BabeRandomness, + n_validators: usize, + block_number: BlockNumber, + session_index: SessionIndex, + para_id: ParaId, + ) -> Vec { + let shuffle = Self::get_shuffle( + maybe_client_features.as_ref(), + block_number, + babe_randomness, + n_validators, + ); + self.0.insert((block_number, session_index), (shuffle, maybe_client_features)); + + self.query_cache_for_para(block_number, session_index, para_id) + .expect("We just inserted the entry.") + } + + fn get_shuffle( + maybe_client_features: Option<&ClientFeatures>, + block_number: BlockNumber, + mut babe_randomness: BabeRandomness, + n_validators: usize, + ) -> Vec { + let mut indices: Vec<_> = (0..n_validators) + .map(|i| ChunkIndex(u32::try_from(i).expect("validator count should not exceed u32"))) + .collect(); + + if let Some(features) = maybe_client_features { + if features.contains(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING) { + let block_number_bytes = block_number.to_be_bytes(); + for i in 0..32 { + babe_randomness[i] ^= block_number_bytes[i % block_number_bytes.len()]; + } + + let mut rng: ChaCha8Rng = SeedableRng::from_seed(babe_randomness); + + indices.shuffle(&mut rng); + } + } + + indices + } + + /// Return the availability chunk start index for this para. + fn para_start_index( + maybe_client_features: Option<&ClientFeatures>, + n_validators: usize, + para_id: ParaId, + ) -> usize { + if let Some(features) = maybe_client_features { + if features.contains(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING) { + let mut rng: ChaCha8Rng = + SeedableRng::from_seed( + u32::from(para_id).to_be_bytes().repeat(8).try_into().expect( + "vector of 32 bytes is safe to cast to array of 32 bytes. qed.", + ), + ); + return rng.gen_range(0..n_validators) + } + } + + 0 + } + + fn chunk_index_for_validator( + maybe_client_features: Option<&ClientFeatures>, + shuffle: &Vec, + para_id: ParaId, + validator_index: ValidatorIndex, + ) -> ChunkIndex { + let core_start_index = + Self::para_start_index(maybe_client_features, shuffle.len(), para_id); + + let chunk_index = shuffle[(core_start_index + + usize::try_from(validator_index.0) + .expect("usize is at least u32 bytes on all modern targets.")) % + shuffle.len()]; + chunk_index + } +} + +/// Compute the per-validator availability chunk index. +/// It's preferred to use the `ChunkIndexCacheRegistry` if you also need a cache. +pub fn availability_chunk_index( + maybe_client_features: Option<&ClientFeatures>, + babe_randomness: BabeRandomness, + n_validators: usize, + block_number: BlockNumber, + para_id: ParaId, + validator_index: ValidatorIndex, +) -> ChunkIndex { + let shuffle = ChunkIndexCacheRegistry::get_shuffle( + maybe_client_features, + block_number, + babe_randomness, + n_validators, + ); + + ChunkIndexCacheRegistry::chunk_index_for_validator( + maybe_client_features, + &shuffle, + para_id, + validator_index, + ) +} + +#[cfg(test)] +mod tests { + use std::collections::HashSet; + + use super::*; + + #[test] + fn test_availability_chunk_indices() { + let block_number = 89; + let n_validators = 11u32; + let babe_randomness = [12u8; 32]; + let session_index = 0; + let n_paras = 5u32; + + // Test the `_for_validator` methods + { + let para_id = 2.into(); + let mut index_registry = ChunkIndexCacheRegistry::new(2); + + for validator in 0..n_validators { + assert!(index_registry + .query_cache_for_validator( + block_number, + session_index, + para_id, + validator.into() + ) + .is_none()); + } + + for validator in 0..n_validators { + // Check that if the client feature is not set, we'll always return the validator + // index. + let chunk_index = index_registry.populate_for_validator( + None, + babe_randomness, + n_validators as usize, + block_number, + session_index, + para_id, + validator.into(), + ); + assert_eq!( + index_registry + .query_cache_for_validator( + block_number, + session_index, + para_id, + validator.into() + ) + .unwrap(), + chunk_index + ); + assert_eq!(chunk_index.0, validator); + assert_eq!( + chunk_index, + availability_chunk_index( + None, + babe_randomness, + n_validators as usize, + block_number, + para_id, + validator.into(), + ) + ); + + // Check for when the client feature is set. + let chunk_index = index_registry.populate_for_validator( + Some(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING), + babe_randomness, + n_validators as usize, + block_number, + session_index, + para_id, + validator.into(), + ); + assert_eq!( + index_registry + .query_cache_for_validator( + block_number, + session_index, + para_id, + validator.into() + ) + .unwrap(), + chunk_index + ); + assert_ne!(chunk_index.0, validator); + assert_eq!( + chunk_index, + availability_chunk_index( + Some(&ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING), + babe_randomness, + n_validators as usize, + block_number, + para_id, + validator.into(), + ) + ); + } + } + + // Test the `_for_para` methods + { + let mut index_registry = ChunkIndexCacheRegistry::new(2); + + for para in 0..n_paras { + assert!(index_registry + .query_cache_for_para(block_number, session_index, para.into()) + .is_none()); + } + + for para in 0..n_paras { + // Check that if the client feature is not set, we'll always return the identity + // vector. + let chunk_indices = index_registry.populate_for_para( + None, + babe_randomness, + n_validators as usize, + block_number, + session_index, + para.into(), + ); + assert_eq!( + index_registry + .query_cache_for_para(block_number, session_index, para.into()) + .unwrap(), + chunk_indices + ); + assert_eq!( + chunk_indices, + (0..n_validators).map(|i| ChunkIndex(i)).collect::>() + ); + + for validator in 0..n_validators { + assert_eq!( + availability_chunk_index( + None, + babe_randomness, + n_validators as usize, + block_number, + para.into(), + validator.into(), + ), + chunk_indices[validator as usize] + ); + } + + // Check for when the client feature is set. + let chunk_indices = index_registry.populate_for_para( + Some(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING), + babe_randomness, + n_validators as usize, + block_number, + session_index, + para.into(), + ); + assert_eq!( + index_registry + .query_cache_for_para(block_number, session_index, para.into()) + .unwrap(), + chunk_indices + ); + assert_eq!(chunk_indices.len(), n_validators as usize); + assert_ne!( + chunk_indices, + (0..n_validators).map(|i| ChunkIndex(i)).collect::>() + ); + assert_eq!( + chunk_indices.iter().collect::>().len(), + n_validators as usize + ); + + for validator in 0..n_validators { + assert_eq!( + availability_chunk_index( + Some(&ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING), + babe_randomness, + n_validators as usize, + block_number, + para.into(), + validator.into(), + ), + chunk_indices[validator as usize] + ); + } + } + } + } +} diff --git a/polkadot/node/subsystem-util/src/lib.rs b/polkadot/node/subsystem-util/src/lib.rs index 0c04c1628dfd..be64b0a209f5 100644 --- a/polkadot/node/subsystem-util/src/lib.rs +++ b/polkadot/node/subsystem-util/src/lib.rs @@ -29,15 +29,12 @@ pub use overseer::{ gen::{OrchestraError as OverseerError, Timeout}, Subsystem, TimeoutExt, }; -use polkadot_node_primitives::BabeRandomness; use polkadot_node_subsystem::{ errors::{RuntimeApiError, SubsystemError}, messages::{RuntimeApiMessage, RuntimeApiRequest, RuntimeApiSender}, overseer, ChainApiError, SubsystemSender, }; use polkadot_node_subsystem_types::messages::ChainApiMessage; -use rand::{seq::SliceRandom, Rng}; -use rand_chacha::ChaCha8Rng; pub use polkadot_node_metrics::{metrics, Metronome}; @@ -45,33 +42,31 @@ use futures::channel::{mpsc, oneshot}; use parity_scale_codec::Encode; use polkadot_primitives::{ - slashing, vstaging::ClientFeatures, AsyncBackingParams, AuthorityDiscoveryId, BlockNumber, - CandidateEvent, CandidateHash, ChunkIndex, CommittedCandidateReceipt, CoreState, EncodeAs, - ExecutorParams, GroupIndex, GroupRotationInfo, Hash, Id as ParaId, OccupiedCoreAssumption, - PersistedValidationData, ScrapedOnChainVotes, SessionIndex, SessionInfo, Signed, - SigningContext, ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, - ValidatorSignature, + slashing, AsyncBackingParams, AuthorityDiscoveryId, BlockNumber, CandidateEvent, CandidateHash, + CommittedCandidateReceipt, CoreState, EncodeAs, ExecutorParams, GroupIndex, GroupRotationInfo, + Hash, Id as ParaId, OccupiedCoreAssumption, PersistedValidationData, ScrapedOnChainVotes, + SessionIndex, SessionInfo, Signed, SigningContext, ValidationCode, ValidationCodeHash, + ValidatorId, ValidatorIndex, ValidatorSignature, }; pub use rand; -use rand::SeedableRng; -use schnellru::{ByLength, LruMap}; use sp_application_crypto::AppCrypto; use sp_core::ByteArray; use sp_keystore::{Error as KeystoreError, KeystorePtr}; use std::time::Duration; use thiserror::Error; +pub use determine_new_blocks::determine_new_blocks; pub use metered; pub use polkadot_node_network_protocol::MIN_GOSSIP_PEERS; -pub use determine_new_blocks::determine_new_blocks; - /// These reexports are required so that external crates can use the `delegated_subsystem` macro /// properly. pub mod reexports { pub use polkadot_overseer::gen::{SpawnedSubsystem, Spawner, Subsystem, SubsystemContext}; } +/// Helpers for the validator->chunk index mapping. +pub mod availability_chunks; /// A utility for managing the implicit view of the relay-chain derived from active /// leaves and the minimum allowed relay-parents that parachain candidates can have /// and be backed in those leaves' children. @@ -443,198 +438,6 @@ impl Validator { } } -/// Object used for holding and computing assigned chunk indices for validators. -pub struct ChunkIndexCacheRegistry( - LruMap<(BlockNumber, SessionIndex), (Vec, Option)>, -); - -impl ChunkIndexCacheRegistry { - /// Initialize with the cache capacity. - pub fn new(capacity: u32) -> Self { - Self(LruMap::new(ByLength::new(capacity))) - } - - /// Return the per-validator chunk index if present in the cache. - pub fn query_cache_for_validator( - &mut self, - block_number: BlockNumber, - session_index: SessionIndex, - para_id: ParaId, - validator_index: ValidatorIndex, - ) -> Option { - if let Some((shuffle, maybe_client_features)) = self.0.get(&(block_number, session_index)) { - Some(Self::chunk_index_for_validator( - maybe_client_features.as_ref(), - shuffle, - para_id, - validator_index, - )) - } else { - None - } - } - - /// Return the per-para chunk index vector if present in the cache. - pub fn query_cache_for_para( - &mut self, - block_number: BlockNumber, - session_index: SessionIndex, - para_id: ParaId, - ) -> Option> { - if let Some((shuffle, maybe_client_features)) = self.0.get(&(block_number, session_index)) { - let core_start_index = - Self::para_start_index(maybe_client_features.as_ref(), shuffle.len(), para_id); - - let chunk_indices = shuffle - .clone() - .into_iter() - .cycle() - .skip(core_start_index) - .take(shuffle.len()) - .collect(); - - Some(chunk_indices) - } else { - None - } - } - - /// Return and populate the cache with the per-validator chunk index. - /// Should only be called if `query_cache_for_validator` returns `None`. - pub fn populate_for_validator( - &mut self, - maybe_client_features: Option, - babe_randomness: BabeRandomness, - n_validators: usize, - block_number: BlockNumber, - session_index: SessionIndex, - para_id: ParaId, - validator_index: ValidatorIndex, - ) -> ChunkIndex { - let shuffle = Self::get_shuffle( - maybe_client_features.as_ref(), - block_number, - babe_randomness, - n_validators, - ); - self.0.insert((block_number, session_index), (shuffle, maybe_client_features)); - - self.query_cache_for_validator(block_number, session_index, para_id, validator_index) - .expect("We just inserted the entry.") - } - - /// Return and populate the cache with the per-para chunk index vector. - /// Should only be called if `query_cache_for_para` returns `None`. - pub fn populate_for_para( - &mut self, - maybe_client_features: Option, - babe_randomness: BabeRandomness, - n_validators: usize, - block_number: BlockNumber, - session_index: SessionIndex, - para_id: ParaId, - ) -> Vec { - let shuffle = Self::get_shuffle( - maybe_client_features.as_ref(), - block_number, - babe_randomness, - n_validators, - ); - self.0.insert((block_number, session_index), (shuffle, maybe_client_features)); - - self.query_cache_for_para(block_number, session_index, para_id) - .expect("We just inserted the entry.") - } - - fn get_shuffle( - maybe_client_features: Option<&ClientFeatures>, - block_number: BlockNumber, - mut babe_randomness: BabeRandomness, - n_validators: usize, - ) -> Vec { - let mut indices: Vec<_> = (0..n_validators) - .map(|i| ChunkIndex(u32::try_from(i).expect("validator count should not exceed u32"))) - .collect(); - - if let Some(features) = maybe_client_features { - if features.contains(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING) { - let block_number_bytes = block_number.to_be_bytes(); - for i in 0..32 { - babe_randomness[i] ^= block_number_bytes[i % block_number_bytes.len()]; - } - - let mut rng: ChaCha8Rng = SeedableRng::from_seed(babe_randomness); - - indices.shuffle(&mut rng); - } - } - - indices - } - - /// Return the availability chunk start index for this para. - fn para_start_index( - maybe_client_features: Option<&ClientFeatures>, - n_validators: usize, - para_id: ParaId, - ) -> usize { - if let Some(features) = maybe_client_features { - if features.contains(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING) { - let mut rng: ChaCha8Rng = - SeedableRng::from_seed( - u32::from(para_id).to_be_bytes().repeat(8).try_into().expect( - "vector of 32 bytes is safe to cast to array of 32 bytes. qed.", - ), - ); - return rng.gen_range(0..n_validators) - } - } - - 0 - } - - fn chunk_index_for_validator( - maybe_client_features: Option<&ClientFeatures>, - shuffle: &Vec, - para_id: ParaId, - validator_index: ValidatorIndex, - ) -> ChunkIndex { - let core_start_index = - Self::para_start_index(maybe_client_features, shuffle.len(), para_id); - - let chunk_index = shuffle[(core_start_index + - usize::try_from(validator_index.0) - .expect("usize is at least u32 bytes on all modern targets.")) % - shuffle.len()]; - chunk_index - } -} - -/// Compute the per-validator availability chunk index. -/// It's preferred to use the `ChunkIndexCacheRegistry` if you also need a cache. -pub fn availability_chunk_index( - maybe_client_features: Option<&ClientFeatures>, - babe_randomness: BabeRandomness, - n_validators: usize, - block_number: BlockNumber, - para_id: ParaId, - validator_index: ValidatorIndex, -) -> ChunkIndex { - let shuffle = ChunkIndexCacheRegistry::get_shuffle( - maybe_client_features, - block_number, - babe_randomness, - n_validators, - ); - - ChunkIndexCacheRegistry::chunk_index_for_validator( - maybe_client_features, - &shuffle, - para_id, - validator_index, - ) -} - /// Get the block number by hash. pub async fn get_block_number( sender: &mut Sender, diff --git a/polkadot/node/subsystem-util/src/tests.rs b/polkadot/node/subsystem-util/src/tests.rs index 56a50f62ca21..9ade95d4e894 100644 --- a/polkadot/node/subsystem-util/src/tests.rs +++ b/polkadot/node/subsystem-util/src/tests.rs @@ -21,7 +21,6 @@ use executor::block_on; use futures::{channel::mpsc, executor, FutureExt, SinkExt, StreamExt}; use polkadot_primitives_test_helpers::AlwaysZeroRng; use std::{ - collections::HashSet, sync::{ atomic::{AtomicUsize, Ordering}, Arc, @@ -95,171 +94,3 @@ fn subset_predefined_generation_check() { assert_eq!(v as usize, idx + 1); } } - -#[test] -fn test_availability_chunk_indices() { - let block_number = 89; - let n_validators = 11u32; - let babe_randomness = [12u8; 32]; - let session_index = 0; - let n_paras = 5u32; - - // Test the `_for_validator` methods - { - let para_id = 2.into(); - let mut index_registry = ChunkIndexCacheRegistry::new(2); - - for validator in 0..n_validators { - assert!(index_registry - .query_cache_for_validator(block_number, session_index, para_id, validator.into()) - .is_none()); - } - - for validator in 0..n_validators { - // Check that if the client feature is not set, we'll always return the validator index. - let chunk_index = index_registry.populate_for_validator( - None, - babe_randomness, - n_validators as usize, - block_number, - session_index, - para_id, - validator.into(), - ); - assert_eq!( - index_registry - .query_cache_for_validator( - block_number, - session_index, - para_id, - validator.into() - ) - .unwrap(), - chunk_index - ); - assert_eq!(chunk_index.0, validator); - assert_eq!( - chunk_index, - availability_chunk_index( - None, - babe_randomness, - n_validators as usize, - block_number, - para_id, - validator.into(), - ) - ); - - // Check for when the client feature is set. - let chunk_index = index_registry.populate_for_validator( - Some(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING), - babe_randomness, - n_validators as usize, - block_number, - session_index, - para_id, - validator.into(), - ); - assert_eq!( - index_registry - .query_cache_for_validator( - block_number, - session_index, - para_id, - validator.into() - ) - .unwrap(), - chunk_index - ); - assert_ne!(chunk_index.0, validator); - assert_eq!( - chunk_index, - availability_chunk_index( - Some(&ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING), - babe_randomness, - n_validators as usize, - block_number, - para_id, - validator.into(), - ) - ); - } - } - - // Test the `_for_para` methods - { - let mut index_registry = ChunkIndexCacheRegistry::new(2); - - for para in 0..n_paras { - assert!(index_registry - .query_cache_for_para(block_number, session_index, para.into()) - .is_none()); - } - - for para in 0..n_paras { - // Check that if the client feature is not set, we'll always return the identity vector. - let chunk_indices = index_registry.populate_for_para( - None, - babe_randomness, - n_validators as usize, - block_number, - session_index, - para.into(), - ); - assert_eq!( - index_registry - .query_cache_for_para(block_number, session_index, para.into()) - .unwrap(), - chunk_indices - ); - assert_eq!(chunk_indices, (0..n_validators).map(|i| ChunkIndex(i)).collect::>()); - - for validator in 0..n_validators { - assert_eq!( - availability_chunk_index( - None, - babe_randomness, - n_validators as usize, - block_number, - para.into(), - validator.into(), - ), - chunk_indices[validator as usize] - ); - } - - // Check for when the client feature is set. - let chunk_indices = index_registry.populate_for_para( - Some(ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING), - babe_randomness, - n_validators as usize, - block_number, - session_index, - para.into(), - ); - assert_eq!( - index_registry - .query_cache_for_para(block_number, session_index, para.into()) - .unwrap(), - chunk_indices - ); - assert_eq!(chunk_indices.len(), n_validators as usize); - assert_ne!(chunk_indices, (0..n_validators).map(|i| ChunkIndex(i)).collect::>()); - assert_eq!(chunk_indices.iter().collect::>().len(), n_validators as usize); - - for validator in 0..n_validators { - assert_eq!( - availability_chunk_index( - Some(&ClientFeatures::AVAILABILITY_CHUNK_SHUFFLING), - babe_randomness, - n_validators as usize, - block_number, - para.into(), - validator.into(), - ), - chunk_indices[validator as usize] - ); - } - } - } -} From 90fdec81167ecdf52c3db157fcdcd05757d604e2 Mon Sep 17 00:00:00 2001 From: alindima Date: Wed, 8 Nov 2023 15:13:00 +0200 Subject: [PATCH 071/126] fix test compilation --- .../node/network/availability-distribution/src/tests/state.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/polkadot/node/network/availability-distribution/src/tests/state.rs b/polkadot/node/network/availability-distribution/src/tests/state.rs index d27c0bdd0e1b..c3727ccc8bb0 100644 --- a/polkadot/node/network/availability-distribution/src/tests/state.rs +++ b/polkadot/node/network/availability-distribution/src/tests/state.rs @@ -20,7 +20,7 @@ use std::{ }; use polkadot_node_subsystem_test_helpers::TestSubsystemContextHandle; -use polkadot_node_subsystem_util::{availability_chunk_index, TimeoutExt}; +use polkadot_node_subsystem_util::{availability_chunks::availability_chunk_index, TimeoutExt}; use futures::{ channel::{mpsc, oneshot}, From 68ad6a6a68cd1fc446fcb4d4452d1f2e79bd3ad3 Mon Sep 17 00:00:00 2001 From: alindima Date: Wed, 8 Nov 2023 15:16:22 +0200 Subject: [PATCH 072/126] add license header --- .../subsystem-util/src/availability_chunks.rs | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/polkadot/node/subsystem-util/src/availability_chunks.rs b/polkadot/node/subsystem-util/src/availability_chunks.rs index ca8d782516e0..3a658ea761c2 100644 --- a/polkadot/node/subsystem-util/src/availability_chunks.rs +++ b/polkadot/node/subsystem-util/src/availability_chunks.rs @@ -1,3 +1,19 @@ +// Copyright (C) Parity Technologies (UK) Ltd. +// This file is part of Polkadot. + +// Polkadot is free software: you can redistribute it and/or modify +// it under the terms of the GNU General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. + +// Polkadot is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU General Public License for more details. + +// You should have received a copy of the GNU General Public License +// along with Polkadot. If not, see . + use polkadot_node_primitives::BabeRandomness; use polkadot_primitives::{ vstaging::ClientFeatures, BlockNumber, ChunkIndex, Id as ParaId, SessionIndex, ValidatorIndex, From 6e34e90e76babd34d94fb51b8c056b9d753bd710 Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 10 Nov 2023 14:57:10 +0200 Subject: [PATCH 073/126] fix cumulus pov_recovery with RPC node --- .../src/active_candidate_recovery.rs | 1 + cumulus/client/pov-recovery/src/lib.rs | 37 +++++-- cumulus/test/service/src/lib.rs | 3 +- polkadot/node/core/approval-voting/src/lib.rs | 1 + .../node/core/approval-voting/src/tests.rs | 2 +- .../src/participation/mod.rs | 1 + .../src/participation/tests.rs | 12 +-- .../network/availability-recovery/src/lib.rs | 12 ++- .../availability-recovery/src/tests.rs | 101 +++++++++++++++++- polkadot/node/subsystem-types/src/messages.rs | 3 +- .../src/node/approval/approval-voting.md | 2 +- .../availability/availability-recovery.md | 4 +- 12 files changed, 155 insertions(+), 24 deletions(-) diff --git a/cumulus/client/pov-recovery/src/active_candidate_recovery.rs b/cumulus/client/pov-recovery/src/active_candidate_recovery.rs index 322b19c796a8..05842c705398 100644 --- a/cumulus/client/pov-recovery/src/active_candidate_recovery.rs +++ b/cumulus/client/pov-recovery/src/active_candidate_recovery.rs @@ -57,6 +57,7 @@ impl ActiveCandidateRecovery { candidate.receipt.clone(), candidate.session_index, None, + Some(candidate.relay_parent_block_number), tx, ), "ActiveCandidateRecovery", diff --git a/cumulus/client/pov-recovery/src/lib.rs b/cumulus/client/pov-recovery/src/lib.rs index b050bc66799c..8e021b6227e4 100644 --- a/cumulus/client/pov-recovery/src/lib.rs +++ b/cumulus/client/pov-recovery/src/lib.rs @@ -55,7 +55,8 @@ use polkadot_node_primitives::{AvailableData, POV_BOMB_LIMIT}; use polkadot_node_subsystem::messages::AvailabilityRecoveryMessage; use polkadot_overseer::Handle as OverseerHandle; use polkadot_primitives::{ - CandidateReceipt, CommittedCandidateReceipt, Id as ParaId, SessionIndex, + BlockId, BlockNumber as RCBlockNumber, CandidateReceipt, CommittedCandidateReceipt, + Id as ParaId, SessionIndex, }; use cumulus_primitives_core::ParachainBlockData; @@ -145,6 +146,7 @@ struct Candidate { session_index: SessionIndex, block_number: NumberFor, parent_hash: Block::Hash, + relay_parent_block_number: RCBlockNumber, // Lazy recovery has been submitted. // Should be true iff a block is either queued to be recovered or // recovery is currently in progress. @@ -264,10 +266,11 @@ where } /// Handle a new pending candidate. - fn handle_pending_candidate( + async fn handle_pending_candidate( &mut self, receipt: CommittedCandidateReceipt, session_index: SessionIndex, + relay_parent_block_number: RCBlockNumber, ) { let header = match Block::Header::decode(&mut &receipt.commitments.head_data.0[..]) { Ok(header) => header, @@ -298,6 +301,7 @@ where block_number: *header.number(), receipt: receipt.to_plain(), session_index, + relay_parent_block_number, parent_hash: *header.parent_hash(), waiting_recovery: false, }, @@ -561,8 +565,8 @@ where loop { select! { pending_candidate = pending_candidates.next() => { - if let Some((receipt, session_index)) = pending_candidate { - self.handle_pending_candidate(receipt, session_index); + if let Some((receipt, session_index, relay_parent_block_nunber)) = pending_candidate { + self.handle_pending_candidate(receipt, session_index, relay_parent_block_nunber).await; } else { tracing::debug!(target: LOG_TARGET, "Pending candidates stream ended"); return; @@ -610,7 +614,8 @@ async fn pending_candidates( relay_chain_client: impl RelayChainInterface + Clone, para_id: ParaId, sync_service: Arc, -) -> RelayChainResult> { +) -> RelayChainResult> +{ let import_notification_stream = relay_chain_client.import_notification_stream().await?; let filtered_stream = import_notification_stream.filter_map(move |n| { @@ -645,9 +650,29 @@ async fn pending_candidates( "Failed to fetch session index.", ) }); + let relay_parent_header = match client_for_closure.header(BlockId::Hash(hash)).await { + Ok(Some(header)) => header, + Ok(None) => { + tracing::error!( + target: LOG_TARGET, + "Failed to retrieve relay parent block number", + ); + return None + }, + Err(err) => { + tracing::error!( + target: LOG_TARGET, + error = ?err, + "Failed to retrieve relay parent block number", + ); + return None + }, + }; if let Ok(Some(candidate)) = pending_availability_result { - session_index_result.map(|session_index| (candidate, session_index)).ok() + session_index_result + .map(|session_index| (candidate, session_index, relay_parent_header.number)) + .ok() } else { None } diff --git a/cumulus/test/service/src/lib.rs b/cumulus/test/service/src/lib.rs index 6fd3e4d43d75..c3b46ef18bc9 100644 --- a/cumulus/test/service/src/lib.rs +++ b/cumulus/test/service/src/lib.rs @@ -163,7 +163,8 @@ impl RecoveryHandle for FailingRecoveryHandle { // For every 5th block we immediately signal unavailability to trigger // a retry. if self.counter % 5 == 0 { - let AvailabilityRecoveryMessage::RecoverAvailableData(_, _, _, back_sender) = message; + let AvailabilityRecoveryMessage::RecoverAvailableData(_, _, _, _, back_sender) = + message; tracing::info!(target: LOG_TARGET, "Failing pov recovery."); back_sender .send(Err(RecoveryError::Unavailable)) diff --git a/polkadot/node/core/approval-voting/src/lib.rs b/polkadot/node/core/approval-voting/src/lib.rs index 94f7fcaf9411..fffcd58702df 100644 --- a/polkadot/node/core/approval-voting/src/lib.rs +++ b/polkadot/node/core/approval-voting/src/lib.rs @@ -2767,6 +2767,7 @@ async fn launch_approval( candidate.clone(), session_index, Some(backing_group), + None, a_tx, )) .await; diff --git a/polkadot/node/core/approval-voting/src/tests.rs b/polkadot/node/core/approval-voting/src/tests.rs index 0c0dcfde9b66..101f0bca2369 100644 --- a/polkadot/node/core/approval-voting/src/tests.rs +++ b/polkadot/node/core/approval-voting/src/tests.rs @@ -2756,7 +2756,7 @@ async fn recover_available_data(virtual_overseer: &mut VirtualOverseer) { assert_matches!( virtual_overseer.recv().await, AllMessages::AvailabilityRecovery( - AvailabilityRecoveryMessage::RecoverAvailableData(_, _, _, tx) + AvailabilityRecoveryMessage::RecoverAvailableData(_, _, _, _, tx) ) => { tx.send(Ok(available_data)).unwrap(); }, diff --git a/polkadot/node/core/dispute-coordinator/src/participation/mod.rs b/polkadot/node/core/dispute-coordinator/src/participation/mod.rs index 90268516e9df..39af39ed4ab6 100644 --- a/polkadot/node/core/dispute-coordinator/src/participation/mod.rs +++ b/polkadot/node/core/dispute-coordinator/src/participation/mod.rs @@ -305,6 +305,7 @@ async fn participate( req.candidate_receipt().clone(), req.session(), None, + None, recover_available_data_tx, )) .await; diff --git a/polkadot/node/core/dispute-coordinator/src/participation/tests.rs b/polkadot/node/core/dispute-coordinator/src/participation/tests.rs index 0aa0d7720051..b760f54fc844 100644 --- a/polkadot/node/core/dispute-coordinator/src/participation/tests.rs +++ b/polkadot/node/core/dispute-coordinator/src/participation/tests.rs @@ -132,7 +132,7 @@ pub async fn participation_missing_availability(ctx_handle: &mut VirtualOverseer assert_matches!( ctx_handle.recv().await, AllMessages::AvailabilityRecovery( - AvailabilityRecoveryMessage::RecoverAvailableData(_, _, _, tx) + AvailabilityRecoveryMessage::RecoverAvailableData(_, _, _, _, tx) ) => { tx.send(Err(RecoveryError::Unavailable)).unwrap(); }, @@ -151,7 +151,7 @@ async fn recover_available_data(virtual_overseer: &mut VirtualOverseer) { assert_matches!( virtual_overseer.recv().await, AllMessages::AvailabilityRecovery( - AvailabilityRecoveryMessage::RecoverAvailableData(_, _, _, tx) + AvailabilityRecoveryMessage::RecoverAvailableData(_, _, _, _, tx) ) => { tx.send(Ok(available_data)).unwrap(); }, @@ -195,7 +195,7 @@ fn same_req_wont_get_queued_if_participation_is_already_running() { assert_matches!( ctx_handle.recv().await, AllMessages::AvailabilityRecovery( - AvailabilityRecoveryMessage::RecoverAvailableData(_, _, _, tx) + AvailabilityRecoveryMessage::RecoverAvailableData(_, _, _, _, tx) ) => { tx.send(Err(RecoveryError::Unavailable)).unwrap(); }, @@ -260,7 +260,7 @@ fn reqs_get_queued_when_out_of_capacity() { { match ctx_handle.recv().await { AllMessages::AvailabilityRecovery( - AvailabilityRecoveryMessage::RecoverAvailableData(_, _, _, tx), + AvailabilityRecoveryMessage::RecoverAvailableData(_, _, _, _, tx), ) => { tx.send(Err(RecoveryError::Unavailable)).unwrap(); recover_available_data_msg_count += 1; @@ -346,7 +346,7 @@ fn cannot_participate_if_cannot_recover_available_data() { assert_matches!( ctx_handle.recv().await, AllMessages::AvailabilityRecovery( - AvailabilityRecoveryMessage::RecoverAvailableData(_, _, _, tx) + AvailabilityRecoveryMessage::RecoverAvailableData(_, _, _, _, tx) ) => { tx.send(Err(RecoveryError::Unavailable)).unwrap(); }, @@ -413,7 +413,7 @@ fn cast_invalid_vote_if_available_data_is_invalid() { assert_matches!( ctx_handle.recv().await, AllMessages::AvailabilityRecovery( - AvailabilityRecoveryMessage::RecoverAvailableData(_, _, _, tx) + AvailabilityRecoveryMessage::RecoverAvailableData(_, _, _, _, tx) ) => { tx.send(Err(RecoveryError::Invalid)).unwrap(); }, diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index 821b8756baba..c62f1df6f3f9 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -366,6 +366,7 @@ async fn handle_recover( erasure_task_tx: futures::channel::mpsc::Sender, recovery_strategy_kind: RecoveryStrategyKind, bypass_availability_store: bool, + maybe_block_number: Option, ) -> Result<()> { let candidate_hash = receipt.hash(); @@ -393,10 +394,13 @@ async fn handle_recover( let _span = span.child("session-info-ctx-received"); match session_info { Some(session_info) => { - let block_number = + let block_number = if let Some(block_number) = maybe_block_number { + block_number + } else { get_block_number::<_, Error>(ctx.sender(), receipt.descriptor.relay_parent) .await? - .ok_or(Error::BlockNumberNotFound)?; + .ok_or(Error::BlockNumberNotFound)? + }; let chunk_indices = if let Some(chunk_indices) = state .chunk_indices @@ -741,6 +745,7 @@ impl AvailabilityRecoverySubsystem { receipt, session_index, maybe_backing_group, + maybe_block_number, response_sender, ) } => handle_recover( @@ -753,7 +758,8 @@ impl AvailabilityRecoverySubsystem { &metrics, erasure_task_tx.clone(), recovery_strategy_kind.clone(), - bypass_availability_store + bypass_availability_store, + maybe_block_number ).await } }, diff --git a/polkadot/node/network/availability-recovery/src/tests.rs b/polkadot/node/network/availability-recovery/src/tests.rs index badde4ec9654..ec9c888db9ac 100644 --- a/polkadot/node/network/availability-recovery/src/tests.rs +++ b/polkadot/node/network/availability-recovery/src/tests.rs @@ -584,12 +584,14 @@ fn availability_is_recovered_from_chunks_if_no_group_provided(#[case] systematic test_state.candidate.clone(), test_state.session_index, None, + None, tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; + // It's not requesting the block number here. test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; test_state.test_runtime_api_client_features(&mut virtual_overseer).await; @@ -624,6 +626,7 @@ fn availability_is_recovered_from_chunks_if_no_group_provided(#[case] systematic new_candidate.clone(), test_state.session_index, None, + None, tx, ), ) @@ -708,6 +711,7 @@ fn availability_is_recovered_from_chunks_even_if_backing_group_supplied_if_chunk test_state.candidate.clone(), test_state.session_index, Some(GroupIndex(0)), + None, tx, ), ) @@ -748,6 +752,7 @@ fn availability_is_recovered_from_chunks_even_if_backing_group_supplied_if_chunk new_candidate.clone(), test_state.session_index, Some(GroupIndex(0)), + None, tx, ), ) @@ -824,6 +829,7 @@ fn bad_merkle_path_leads_to_recovery_error(#[case] systematic_recovery: bool) { test_state.candidate.clone(), test_state.session_index, None, + None, tx, ), ) @@ -907,6 +913,7 @@ fn wrong_chunk_index_leads_to_recovery_error(#[case] systematic_recovery: bool) test_state.candidate.clone(), test_state.session_index, None, + None, tx, ), ) @@ -1012,6 +1019,7 @@ fn invalid_erasure_coding_leads_to_invalid_error(#[case] systematic_recovery: bo test_state.candidate.clone(), test_state.session_index, None, + None, tx, ), ) @@ -1040,8 +1048,10 @@ fn invalid_erasure_coding_leads_to_invalid_error(#[case] systematic_recovery: bo }); } -#[test] -fn fast_path_backing_group_recovers() { +#[rstest] +#[case(Some(1))] +#[case(None)] +fn fast_path_backing_group_recovers(#[case] relay_parent_block_number: Option) { let test_state = TestState::default(); let subsystem = AvailabilityRecoverySubsystem::with_fast_path(request_receiver(), Metrics::new_dummy()); @@ -1064,13 +1074,16 @@ fn fast_path_backing_group_recovers() { test_state.candidate.clone(), test_state.session_index, Some(GroupIndex(0)), + relay_parent_block_number, tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + if relay_parent_block_number.is_none() { + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + } test_state.test_runtime_api_client_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -1145,6 +1158,7 @@ fn recovers_from_only_chunks_if_pov_large( test_state.candidate.clone(), test_state.session_index, Some(GroupIndex(0)), + None, tx, ), ) @@ -1196,6 +1210,7 @@ fn recovers_from_only_chunks_if_pov_large( new_candidate.clone(), test_state.session_index, Some(GroupIndex(0)), + None, tx, ), ) @@ -1293,6 +1308,7 @@ fn fast_path_backing_group_recovers_if_pov_small( test_state.candidate.clone(), test_state.session_index, Some(GroupIndex(0)), + None, tx, ), ) @@ -1370,6 +1386,7 @@ fn no_answers_in_fast_path_causes_chunk_requests(#[case] systematic_recovery: bo test_state.candidate.clone(), test_state.session_index, Some(GroupIndex(0)), + None, tx, ), ) @@ -1446,6 +1463,7 @@ fn task_canceled_when_receivers_dropped(#[case] systematic_recovery: bool) { test_state.candidate.clone(), test_state.session_index, None, + None, tx, ), ) @@ -1500,6 +1518,7 @@ fn chunks_retry_until_all_nodes_respond(#[case] systematic_recovery: bool) { test_state.candidate.clone(), test_state.session_index, Some(GroupIndex(0)), + None, tx, ), ) @@ -1582,6 +1601,7 @@ fn network_bridge_not_returning_responses_wont_stall_retrieval() { test_state.candidate.clone(), test_state.session_index, Some(GroupIndex(0)), + None, tx, ), ) @@ -1672,6 +1692,7 @@ fn all_not_returning_requests_still_recovers_on_return(#[case] systematic_recove test_state.candidate.clone(), test_state.session_index, Some(GroupIndex(0)), + None, tx, ), ) @@ -1777,6 +1798,7 @@ fn returns_early_if_we_have_the_data(#[case] systematic_recovery: bool) { test_state.candidate.clone(), test_state.session_index, None, + None, tx, ), ) @@ -1816,6 +1838,7 @@ fn returns_early_if_present_in_the_subsystem_cache() { test_state.candidate.clone(), test_state.session_index, Some(GroupIndex(0)), + None, tx, ), ) @@ -1850,6 +1873,7 @@ fn returns_early_if_present_in_the_subsystem_cache() { test_state.candidate.clone(), test_state.session_index, Some(GroupIndex(0)), + None, tx, ), ) @@ -1900,6 +1924,7 @@ fn does_not_query_local_validator(#[case] systematic_recovery: bool) { test_state.candidate.clone(), test_state.session_index, None, + None, tx, ), ) @@ -1963,6 +1988,7 @@ fn invalid_local_chunk(#[case] systematic_recovery: bool) { test_state.candidate.clone(), test_state.session_index, None, + None, tx, ), ) @@ -2030,6 +2056,7 @@ fn systematic_chunks_are_not_requested_again_in_regular_recovery() { test_state.candidate.clone(), test_state.session_index, None, + None, tx, ), ) @@ -2112,6 +2139,7 @@ fn chunk_indices_are_shuffled(#[case] systematic_recovery: bool, #[case] shuffli test_state.candidate.clone(), test_state.session_index, None, + None, tx, ), ) @@ -2237,6 +2265,7 @@ fn number_of_request_retries_is_bounded( test_state.candidate.clone(), test_state.session_index, None, + None, tx, ), ) @@ -2296,3 +2325,69 @@ fn number_of_request_retries_is_bounded( virtual_overseer }); } + +#[rstest] +#[case(false)] +#[case(true)] +fn block_number_not_requested_if_provided(#[case] systematic_recovery: bool) { + let test_state = TestState::default(); + let (subsystem, threshold) = match systematic_recovery { + true => ( + AvailabilityRecoverySubsystem::with_systematic_chunks( + request_receiver(), + Metrics::new_dummy(), + ), + test_state.systematic_threshold(), + ), + false => ( + AvailabilityRecoverySubsystem::with_chunks_only( + request_receiver(), + Metrics::new_dummy(), + ), + test_state.threshold(), + ), + }; + + test_harness(subsystem, |mut virtual_overseer| async move { + overseer_signal( + &mut virtual_overseer, + OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( + test_state.current, + 1, + ))), + ) + .await; + + let (tx, rx) = oneshot::channel(); + + overseer_send( + &mut virtual_overseer, + AvailabilityRecoveryMessage::RecoverAvailableData( + test_state.candidate.clone(), + test_state.session_index, + None, + Some(1), + tx, + ), + ) + .await; + + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; + test_state.test_runtime_api_client_features(&mut virtual_overseer).await; + test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; + test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + + test_state + .test_chunk_requests( + test_state.candidate.hash(), + &mut virtual_overseer, + threshold, + |_| Has::Yes, + systematic_recovery, + ) + .await; + + assert_eq!(rx.await.unwrap().unwrap(), test_state.available_data); + virtual_overseer + }); +} diff --git a/polkadot/node/subsystem-types/src/messages.rs b/polkadot/node/subsystem-types/src/messages.rs index e7b66c224a7b..08ed5751d455 100644 --- a/polkadot/node/subsystem-types/src/messages.rs +++ b/polkadot/node/subsystem-types/src/messages.rs @@ -471,7 +471,8 @@ pub enum AvailabilityRecoveryMessage { RecoverAvailableData( CandidateReceipt, SessionIndex, - Option, // Optional backing group to request from first. + Option, // Optional backing group to request from first. + Option, // Optional block number of the relay parent of the candidate. oneshot::Sender>, ), } diff --git a/polkadot/roadmap/implementers-guide/src/node/approval/approval-voting.md b/polkadot/roadmap/implementers-guide/src/node/approval/approval-voting.md index 1a17f90d9ba3..c11fdb1d4307 100644 --- a/polkadot/roadmap/implementers-guide/src/node/approval/approval-voting.md +++ b/polkadot/roadmap/implementers-guide/src/node/approval/approval-voting.md @@ -385,7 +385,7 @@ On receiving an `ApprovedAncestor(Hash, BlockNumber, response_channel)`: * Requires `(SessionIndex, SessionInfo, CandidateReceipt, ValidatorIndex, backing_group, block_hash, candidate_index)` * Extract the public key of the `ValidatorIndex` from the `SessionInfo` for the session. -* Issue an `AvailabilityRecoveryMessage::RecoverAvailableData(candidate, session_index, Some(backing_group), +* Issue an `AvailabilityRecoveryMessage::RecoverAvailableData(candidate, session_index, Some(backing_group), None, response_sender)` * Load the historical validation code of the parachain by dispatching a `RuntimeApiRequest::ValidationCodeByHash(descriptor.validation_code_hash)` against the state of `block_hash`. diff --git a/polkadot/roadmap/implementers-guide/src/node/availability/availability-recovery.md b/polkadot/roadmap/implementers-guide/src/node/availability/availability-recovery.md index e3bb14db3a55..ae9393693668 100644 --- a/polkadot/roadmap/implementers-guide/src/node/availability/availability-recovery.md +++ b/polkadot/roadmap/implementers-guide/src/node/availability/availability-recovery.md @@ -16,7 +16,7 @@ connect to and query randomly chosen validators until we have received `f + 1` p Input: * `NetworkBridgeUpdate(update)` -* `AvailabilityRecoveryMessage::RecoverAvailableData(candidate, session, backing_group, response)` +* `AvailabilityRecoveryMessage::RecoverAvailableData(candidate, session, backing_group, maybe_relay_parent_block_number, response)` Output: @@ -90,7 +90,7 @@ Ignore `BlockFinalized` signals. On `Conclude`, shut down the subsystem. -#### `AvailabilityRecoveryMessage::RecoverAvailableData(receipt, session, Option, response)` +#### `AvailabilityRecoveryMessage::RecoverAvailableData(receipt, session, Option, Option response)` 1. Check the `availability_lru` for the candidate and return the data if so. 1. Check if there is already an recovery handle for the request. If so, add the response handle to it. From 5feed4afbce36ac4b748fe15e1a80ca988fe3e4d Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 10 Nov 2023 15:06:52 +0200 Subject: [PATCH 074/126] fix test --- polkadot/node/overseer/src/tests.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/polkadot/node/overseer/src/tests.rs b/polkadot/node/overseer/src/tests.rs index 254f5fe45120..2d9802742cc9 100644 --- a/polkadot/node/overseer/src/tests.rs +++ b/polkadot/node/overseer/src/tests.rs @@ -856,6 +856,7 @@ fn test_availability_recovery_msg() -> AvailabilityRecoveryMessage { dummy_candidate_receipt(dummy_hash()), Default::default(), None, + None, sender, ) } From db1225b4b083b37c88a7ace4e3c9b614099564d5 Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 14 Nov 2023 12:00:20 +0200 Subject: [PATCH 075/126] add more tests for #2287 --- .../availability-recovery/src/tests.rs | 102 ++++++++++++++---- 1 file changed, 81 insertions(+), 21 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/tests.rs b/polkadot/node/network/availability-recovery/src/tests.rs index ec9c888db9ac..df9129b60dc5 100644 --- a/polkadot/node/network/availability-recovery/src/tests.rs +++ b/polkadot/node/network/availability-recovery/src/tests.rs @@ -531,6 +531,7 @@ impl Default for TestState { candidate.descriptor.erasure_root = erasure_root; candidate.descriptor.relay_parent = Hash::repeat_byte(10); + candidate.descriptor.pov_hash = Hash::repeat_byte(3); Self { validators, @@ -1048,6 +1049,62 @@ fn invalid_erasure_coding_leads_to_invalid_error(#[case] systematic_recovery: bo }); } +#[test] +fn invalid_pov_hash_leads_to_invalid_error() { + let mut test_state = TestState::default(); + + let subsystem = + AvailabilityRecoverySubsystem::for_collator(request_receiver(), Metrics::new_dummy()); + + test_harness(subsystem, |mut virtual_overseer| async move { + let pov = PoV { block_data: BlockData(vec![69; 64]) }; + + test_state.candidate.descriptor.pov_hash = pov.hash(); + + let candidate_hash = test_state.candidate.hash(); + + overseer_signal( + &mut virtual_overseer, + OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( + test_state.current, + 1, + ))), + ) + .await; + + let (tx, rx) = oneshot::channel(); + + overseer_send( + &mut virtual_overseer, + AvailabilityRecoveryMessage::RecoverAvailableData( + test_state.candidate.clone(), + test_state.session_index, + None, + None, + tx, + ), + ) + .await; + + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; + test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_client_features(&mut virtual_overseer).await; + + test_state + .test_chunk_requests( + candidate_hash, + &mut virtual_overseer, + test_state.threshold(), + |_| Has::Yes, + false, + ) + .await; + + assert_eq!(rx.await.unwrap().unwrap_err(), RecoveryError::Invalid); + virtual_overseer + }); +} + #[rstest] #[case(Some(1))] #[case(None)] @@ -1111,11 +1168,11 @@ fn fast_path_backing_group_recovers(#[case] relay_parent_block_number: Option ( AvailabilityRecoverySubsystem::with_systematic_chunks_if_pov_large( request_receiver(), @@ -1130,13 +1187,16 @@ fn recovers_from_only_chunks_if_pov_large( ), test_state.threshold(), ), - (false, true) => ( - AvailabilityRecoverySubsystem::with_availability_store_skip( - request_receiver(), - Metrics::new_dummy(), - ), - test_state.threshold(), - ), + (false, true) => { + test_state.candidate.descriptor.pov_hash = test_state.available_data.pov.hash(); + ( + AvailabilityRecoverySubsystem::for_collator( + request_receiver(), + Metrics::new_dummy(), + ), + test_state.threshold(), + ) + }, (_, _) => unreachable!(), }; @@ -1179,7 +1239,7 @@ fn recovers_from_only_chunks_if_pov_large( } ); - if !skip_availability_store { + if !for_collator { test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; } @@ -1228,7 +1288,7 @@ fn recovers_from_only_chunks_if_pov_large( } ); - if !skip_availability_store { + if !for_collator { test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; } @@ -1243,7 +1303,7 @@ fn recovers_from_only_chunks_if_pov_large( systematic_recovery, ) .await; - if !skip_availability_store { + if !for_collator { test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; } } @@ -1269,11 +1329,11 @@ fn recovers_from_only_chunks_if_pov_large( #[case(false, false)] fn fast_path_backing_group_recovers_if_pov_small( #[case] systematic_recovery: bool, - #[case] skip_availability_store: bool, + #[case] for_collator: bool, ) { - let test_state = TestState::default(); + let mut test_state = TestState::default(); - let subsystem = match (systematic_recovery, skip_availability_store) { + let subsystem = match (systematic_recovery, for_collator) { (true, false) => AvailabilityRecoverySubsystem::with_systematic_chunks_if_pov_large( request_receiver(), Metrics::new_dummy(), @@ -1283,10 +1343,10 @@ fn fast_path_backing_group_recovers_if_pov_small( request_receiver(), Metrics::new_dummy(), ), - (false, true) => AvailabilityRecoverySubsystem::with_availability_store_skip( - request_receiver(), - Metrics::new_dummy(), - ), + (false, true) => { + test_state.candidate.descriptor.pov_hash = test_state.available_data.pov.hash(); + AvailabilityRecoverySubsystem::for_collator(request_receiver(), Metrics::new_dummy()) + }, (_, _) => unreachable!(), }; @@ -1334,7 +1394,7 @@ fn fast_path_backing_group_recovers_if_pov_small( } ); - if !skip_availability_store { + if !for_collator { test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; } From b4a125bb07b10f3dde592c0c134fb5608740896b Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 14 Nov 2023 18:02:26 +0200 Subject: [PATCH 076/126] add backers as backup for requesting systematic chunks each backer will only be used for requesting one chunk, to not overwhelm them --- Cargo.lock | 1 + .../network/availability-recovery/Cargo.toml | 1 + .../network/availability-recovery/src/lib.rs | 4 + .../network/availability-recovery/src/task.rs | 160 +++++++++++++- .../availability-recovery/src/tests.rs | 200 ++++++++++++++---- 5 files changed, 315 insertions(+), 51 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index f4e0201ce828..31816f369087 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -11992,6 +11992,7 @@ dependencies = [ "sp-application-crypto", "sp-core", "sp-keyring", + "sp-tracing 10.0.0", "thiserror", "tracing-gum", ] diff --git a/polkadot/node/network/availability-recovery/Cargo.toml b/polkadot/node/network/availability-recovery/Cargo.toml index d997cc828b54..d9f6054f4973 100644 --- a/polkadot/node/network/availability-recovery/Cargo.toml +++ b/polkadot/node/network/availability-recovery/Cargo.toml @@ -31,6 +31,7 @@ futures-timer = "3.0.2" log = "0.4.17" rstest = "0.18.2" +sp-tracing = { path = "../../../../substrate/primitives/tracing" } sp-core = { path = "../../../../substrate/primitives/core" } sp-keyring = { path = "../../../../substrate/primitives/keyring" } sp-application-crypto = { path = "../../../../substrate/primitives/application-crypto" } diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index 96bd328ad9f7..372fb77551dd 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -451,6 +451,7 @@ async fn handle_recover( }) .collect(); + let mut backer_group = None; let mut recovery_strategies: VecDeque< Box::Sender>>, > = VecDeque::with_capacity(2); @@ -505,6 +506,8 @@ async fn handle_recover( ), _ => {}, }; + + backer_group = Some(backing_validators); } } @@ -531,6 +534,7 @@ async fn handle_recover( recovery_strategies.push_back(Box::new(FetchSystematicChunks::new( FetchSystematicChunksParams { validators, + backers: backer_group.map(|v| v.to_vec()).unwrap_or_else(|| vec![]), erasure_task_tx: erasure_task_tx.clone(), }, ))); diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs index 6af4b5d4ba96..7215d0ec12a1 100644 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ b/polkadot/node/network/availability-recovery/src/task.rs @@ -358,6 +358,9 @@ impl State { ValidatorIndex, Result, RequestError>, )>, + // If supplied, these validators will be used as a backup for requesting chunks. They + // should hold all chunks. Each of them will only be used to query one chunk. + backup_validators: &mut Vec, // Function that returns `true` when this strategy can conclude. Either if we got enough // chunks or if it's impossible. can_conclude: impl Fn( @@ -385,6 +388,8 @@ impl State { let (chunk_index, validator_index, request_result) = res; + let mut is_error = false; + match request_result { Ok(Some(chunk)) => if is_chunk_valid(params, &chunk) { @@ -403,6 +408,7 @@ impl State { // Record that we got an invalid chunk so that subsequent strategies don't // try requesting this again. self.record_error_fatal(chunk_index, validator_index); + is_error = true; }, Ok(None) => { metrics.on_chunk_request_no_such_chunk(strategy_type); @@ -417,6 +423,7 @@ impl State { // Record that the validator did not have this chunk so that subsequent // strategies don't try requesting this again. self.record_error_fatal(chunk_index, validator_index); + is_error = true; }, Err(err) => { error_count += 1; @@ -430,6 +437,8 @@ impl State { "Failure requesting chunk", ); + is_error = true; + match err { RequestError::InvalidResponse(_) => { metrics.on_chunk_request_invalid(strategy_type); @@ -456,27 +465,42 @@ impl State { metrics.on_chunk_request_error(strategy_type); } - self.record_error_non_fatal(chunk_index, validator_index); // Record that we got a non-fatal error so that this or subsequent // strategies will retry requesting this only a limited number of times. - if self.can_retry_request(chunk_index, validator_index, retry_threshold) - { - validators.push_front((chunk_index, validator_index)); - } + self.record_error_non_fatal(chunk_index, validator_index); }, RequestError::Canceled(_) => { metrics.on_chunk_request_error(strategy_type); + // Record that we got a non-fatal error so that this or subsequent + // strategies will retry requesting this only a limited number of times. self.record_error_non_fatal(chunk_index, validator_index); - if self.can_retry_request(chunk_index, validator_index, retry_threshold) - { - validators.push_front((chunk_index, validator_index)); - } }, } }, } + if is_error && !self.received_chunks.contains_key(&chunk_index) { + // First, see if we can retry the request. + if self.can_retry_request(chunk_index, validator_index, retry_threshold) { + validators.push_front((chunk_index, validator_index)); + } else { + // Otherwise, try requesting from a backer as a backup, if we've not already + // requested the same chunk from it. + + let position = backup_validators + .iter() + .position(|v| !self.recorded_errors.contains_key(&(chunk_index, *v))); + if let Some(position) = position { + let backer = backup_validators.swap_remove(position); + validators.push_front((chunk_index, backer)); + println!("There"); + } else { + println!("here"); + } + } + } + if can_conclude( validators.len(), requesting_chunks.total_len(), @@ -485,6 +509,7 @@ impl State { ) { gum::debug!( target: LOG_TARGET, + validators_len = validators.len(), candidate_hash = ?params.candidate_hash, received_chunks_count = ?self.chunk_count(), requested_chunks_count = ?requesting_chunks.len(), @@ -760,6 +785,8 @@ pub struct FetchSystematicChunks { threshold: usize, /// Validators that hold the systematic chunks. validators: VecDeque<(ChunkIndex, ValidatorIndex)>, + /// Backers. to be used as a backup. + backers: Vec, /// Collection of in-flight requests. requesting_chunks: FuturesUndead<(ChunkIndex, ValidatorIndex, Result, RequestError>)>, @@ -771,6 +798,8 @@ pub struct FetchSystematicChunks { pub struct FetchSystematicChunksParams { /// Validators that hold the systematic chunks. pub validators: VecDeque<(ChunkIndex, ValidatorIndex)>, + /// Validators in the backing group, to be used as a backup for requesting systematic chunks. + pub backers: Vec, /// Channel to the erasure task handler. pub erasure_task_tx: futures::channel::mpsc::Sender, } @@ -781,6 +810,7 @@ impl FetchSystematicChunks { Self { threshold: params.validators.len(), validators: params.validators, + backers: params.backers, requesting_chunks: FuturesUndead::new(), erasure_task_tx: params.erasure_task_tx, } @@ -1019,6 +1049,7 @@ impl RecoveryStrategy SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT, &mut validators_queue, &mut self.requesting_chunks, + &mut self.backers, |unrequested_validators, in_flight_reqs, // Don't use this chunk count, as it may contain non-systematic chunks. @@ -1306,6 +1337,7 @@ impl RecoveryStrategy REGULAR_CHUNKS_REQ_RETRY_LIMIT, &mut validators_queue, &mut self.requesting_chunks, + &mut vec![], |unrequested_validators, in_flight_reqs, chunk_count, _success_responses| { chunk_count >= common_params.threshold || Self::is_unavailable( @@ -1684,6 +1716,7 @@ mod tests { retry_threshold, &mut validators, &mut ongoing_reqs, + &mut vec![], |_, _, _, _| false, ) .await; @@ -1739,6 +1772,7 @@ mod tests { retry_threshold, &mut validators, &mut ongoing_reqs, + &mut vec![], |_, _, _, _| false, ) .await; @@ -1768,6 +1802,7 @@ mod tests { retry_threshold, &mut validators, &mut ongoing_reqs, + &mut vec![], |_, _, _, _| false, ) .await; @@ -1789,6 +1824,7 @@ mod tests { retry_threshold, &mut validators, &mut ongoing_reqs, + &mut vec![], |_, _, _, _| true, ) .await; @@ -1800,6 +1836,112 @@ mod tests { }, ); } + + // Complex scenario with backups in the backing group. + { + let mut params = params.clone(); + let chunks = params.create_chunks(); + let mut state = State::new(); + let mut ongoing_reqs = FuturesUndead::new(); + ongoing_reqs + .push(future::ready((0.into(), 0.into(), Ok(Some(chunks[0].clone())))).boxed()); + ongoing_reqs.soft_cancel(); + ongoing_reqs + .push(future::ready((1.into(), 1.into(), Ok(Some(chunks[1].clone())))).boxed()); + ongoing_reqs.push(future::ready((2.into(), 2.into(), Ok(None))).boxed()); + ongoing_reqs.push( + future::ready(( + 3.into(), + 3.into(), + Err(RequestError::from(DecodingError::from("err"))), + )) + .boxed(), + ); + ongoing_reqs.push( + future::ready(( + 4.into(), + 4.into(), + Err(RequestError::NetworkError(RequestFailure::NotConnected)), + )) + .boxed(), + ); + + let mut validators = + (5..=params.n_validators as u32).map(|i| (i.into(), i.into())).collect(); + let mut backup_backers = vec![ + 2.into(), + 0.into(), + 4.into(), + 3.into(), + (params.n_validators as u32 + 1).into(), + (params.n_validators as u32 + 2).into(), + ]; + + test_harness( + |mut receiver: UnboundedReceiver| async move { + // Shouldn't send any requests. + assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); + }, + |_| async move { + let (total_responses, error_count) = state + .wait_for_chunks( + "regular", + ¶ms, + retry_threshold, + &mut validators, + &mut ongoing_reqs, + &mut backup_backers, + |_, _, _, _| false, + ) + .await; + assert_eq!(total_responses, 5); + assert_eq!(error_count, 3); + assert_eq!(state.chunk_count(), 2); + + let mut expected_validators: VecDeque<_> = + (5..=params.n_validators as u32).map(|i| (i.into(), i.into())).collect(); + // We picked a backer as a backup for chunks 2 and 3. + expected_validators.push_front((2.into(), 0.into())); + expected_validators.push_front((3.into(), 2.into())); + expected_validators.push_front((4.into(), 4.into())); + + assert_eq!(validators, expected_validators); + + // This time we'll go over the recoverable error threshold for chunk 4. + ongoing_reqs.push( + future::ready(( + 4.into(), + 4.into(), + Err(RequestError::NetworkError(RequestFailure::NotConnected)), + )) + .boxed(), + ); + + validators.pop_front(); + + let (total_responses, error_count) = state + .wait_for_chunks( + "regular", + ¶ms, + retry_threshold, + &mut validators, + &mut ongoing_reqs, + &mut backup_backers, + |_, _, _, _| false, + ) + .await; + assert_eq!(total_responses, 1); + assert_eq!(error_count, 1); + assert_eq!(state.chunk_count(), 2); + + expected_validators.pop_front(); + expected_validators + .push_front((4.into(), (params.n_validators as u32 + 1).into())); + + assert_eq!(validators, expected_validators); + }, + ); + } } #[test] diff --git a/polkadot/node/network/availability-recovery/src/tests.rs b/polkadot/node/network/availability-recovery/src/tests.rs index df9129b60dc5..e7702d99dc8e 100644 --- a/polkadot/node/network/availability-recovery/src/tests.rs +++ b/polkadot/node/network/availability-recovery/src/tests.rs @@ -63,10 +63,7 @@ fn test_harness>( subsystem: AvailabilityRecoverySubsystem, test: impl FnOnce(VirtualOverseer) -> Fut, ) { - let _ = env_logger::builder() - .is_test(true) - .filter(Some("polkadot_availability_recovery"), log::LevelFilter::Trace) - .try_init(); + sp_tracing::init_for_tests(); let pool = sp_core::testing::TaskExecutor::new(); @@ -159,6 +156,7 @@ struct TestState { validators: Vec, validator_public: IndexedVec, validator_authority_id: Vec, + validator_groups: IndexedVec>, current: Hash, candidate: CandidateReceipt, session_index: SessionIndex, @@ -199,8 +197,7 @@ impl TestState { tx.send(Ok(Some(SessionInfo { validators: self.validator_public.clone(), discovery_keys: self.validator_authority_id.clone(), - // all validators in the same group. - validator_groups: IndexedVec::>::from(vec![(0..self.validators.len()).map(|i| ValidatorIndex(i as _)).collect()]), + validator_groups: self.validator_groups.clone(), assignment_keys: vec![], n_cores: 0, zeroth_delay_tranche_width: 0, @@ -328,7 +325,7 @@ impl TestState { candidate_hash: CandidateHash, virtual_overseer: &mut VirtualOverseer, n: usize, - who_has: impl Fn(usize) -> Has, + mut who_has: impl FnMut(usize) -> Has, systematic_recovery: bool, ) -> Vec, RequestFailure>>> { // arbitrary order. @@ -386,9 +383,11 @@ impl TestState { candidate_hash: CandidateHash, virtual_overseer: &mut VirtualOverseer, who_has: impl Fn(usize) -> Has, + group_index: GroupIndex, ) -> Vec, RequestFailure>>> { let mut senders = Vec::new(); - for _ in 0..self.validators.len() { + let expected_validators = self.validator_groups.get(group_index).unwrap(); + for _ in 0..expected_validators.len() { // Receive a request for the full `AvailableData`. assert_matches!( overseer_recv(virtual_overseer).await, @@ -408,6 +407,7 @@ impl TestState { .iter() .position(|a| Recipient::Authority(a.clone()) == req.peer) .unwrap(); + assert!(expected_validators.contains(&ValidatorIndex(validator_index as u32))); let available_data = match who_has(validator_index) { Has::No => Ok(None), @@ -486,6 +486,11 @@ impl Default for TestState { let validator_public = validator_pubkeys(&validators); let validator_authority_id = validator_authority_id(&validators); + let validator_groups = vec![ + vec![1.into(), 0.into(), 3.into(), 4.into()], + vec![5.into(), 6.into()], + vec![2.into()], + ]; let current = Hash::repeat_byte(1); @@ -537,6 +542,10 @@ impl Default for TestState { validators, validator_public, validator_authority_id, + validator_groups: IndexedVec::>::try_from( + validator_groups, + ) + .unwrap(), current, candidate, session_index, @@ -592,7 +601,6 @@ fn availability_is_recovered_from_chunks_if_no_group_provided(#[case] systematic .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - // It's not requesting the block number here. test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; test_state.test_runtime_api_client_features(&mut virtual_overseer).await; @@ -752,7 +760,7 @@ fn availability_is_recovered_from_chunks_even_if_backing_group_supplied_if_chunk AvailabilityRecoveryMessage::RecoverAvailableData( new_candidate.clone(), test_state.session_index, - Some(GroupIndex(0)), + Some(GroupIndex(1)), None, tx, ), @@ -770,28 +778,40 @@ fn availability_is_recovered_from_chunks_even_if_backing_group_supplied_if_chunk .test_chunk_requests( new_candidate.hash(), &mut virtual_overseer, - threshold, + threshold * SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT as usize, |_| Has::No, systematic_recovery, ) .await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; - } + // Even if the recovery is systematic, we'll always fall back to regular recovery, so + // keep this around. + test_state + .test_chunk_requests( + new_candidate.hash(), + &mut virtual_overseer, + test_state.impossibility_threshold() - threshold, + |_| Has::No, + false, + ) + .await; - // Even if the recovery is systematic, we'll always fall back to regular recovery, so keep - // this around. - test_state - .test_chunk_requests( - new_candidate.hash(), - &mut virtual_overseer, - test_state.impossibility_threshold(), - |_| Has::No, - false, - ) - .await; + // A request times out with `Unavailable` error. + assert_eq!(rx.await.unwrap().unwrap_err(), RecoveryError::Unavailable); + } else { + test_state + .test_chunk_requests( + new_candidate.hash(), + &mut virtual_overseer, + test_state.impossibility_threshold(), + |_| Has::No, + false, + ) + .await; - // A request times out with `Unavailable` error. - assert_eq!(rx.await.unwrap().unwrap_err(), RecoveryError::Unavailable); + // A request times out with `Unavailable` error. + assert_eq!(rx.await.unwrap().unwrap_err(), RecoveryError::Unavailable); + } virtual_overseer }); } @@ -1153,7 +1173,7 @@ fn fast_path_backing_group_recovers(#[case] relay_parent_block_number: Option Date: Mon, 20 Nov 2023 13:50:10 +0200 Subject: [PATCH 077/126] fix clippy and remove unneeded dep --- Cargo.lock | 1 - polkadot/node/network/availability-recovery/Cargo.toml | 1 - polkadot/node/subsystem-util/src/availability_chunks.rs | 2 +- 3 files changed, 1 insertion(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 56e61df6125a..bae83f07dc72 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -12074,7 +12074,6 @@ version = "1.0.0" dependencies = [ "assert_matches", "async-trait", - "env_logger 0.9.3", "fatality", "futures", "futures-timer", diff --git a/polkadot/node/network/availability-recovery/Cargo.toml b/polkadot/node/network/availability-recovery/Cargo.toml index d9f6054f4973..152eb8d9bced 100644 --- a/polkadot/node/network/availability-recovery/Cargo.toml +++ b/polkadot/node/network/availability-recovery/Cargo.toml @@ -26,7 +26,6 @@ sc-network = { path = "../../../../substrate/client/network" } [dev-dependencies] assert_matches = "1.4.0" -env_logger = "0.9.0" futures-timer = "3.0.2" log = "0.4.17" rstest = "0.18.2" diff --git a/polkadot/node/subsystem-util/src/availability_chunks.rs b/polkadot/node/subsystem-util/src/availability_chunks.rs index 4e18ee14c2cf..dd9c9e57a45c 100644 --- a/polkadot/node/subsystem-util/src/availability_chunks.rs +++ b/polkadot/node/subsystem-util/src/availability_chunks.rs @@ -228,7 +228,7 @@ mod tests { pub fn node_features_with_shuffling() -> NodeFeatures { let mut node_features = NodeFeatures::new(); - node_features.resize(node_features::AVAILABILITY_CHUNK_SHUFFLING.into() + 1, false); + node_features.resize(node_features::AVAILABILITY_CHUNK_SHUFFLING as usize + 1, false); node_features.set(node_features::AVAILABILITY_CHUNK_SHUFFLING.into(), true); node_features } From 894880b5dad085f346f1e1f7942cadfee4582b32 Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 19 Dec 2023 12:10:08 +0200 Subject: [PATCH 078/126] update lockfile --- Cargo.lock | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.lock b/Cargo.lock index a9dfa0916801..b745482a55e1 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -14738,7 +14738,7 @@ dependencies = [ "regex", "relative-path", "rustc_version 0.4.0", - "syn 2.0.40", + "syn 2.0.41", "unicode-ident", ] From fada5d91aab706914be2ab82f65384936e7beaf7 Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 19 Dec 2023 14:44:38 +0200 Subject: [PATCH 079/126] integrate reed-solomon from master git branch will be switched to a standalone release before merging --- Cargo.lock | 4 +- polkadot/erasure-coding/Cargo.toml | 2 +- polkadot/erasure-coding/src/lib.rs | 142 ++++++++--------------------- 3 files changed, 38 insertions(+), 110 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index b745482a55e1..f6d1199f9243 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -14241,12 +14241,10 @@ dependencies = [ [[package]] name = "reed-solomon-novelpoly" version = "1.0.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "58130877ca403ab42c864fbac74bb319a0746c07a634a92a5cfc7f54af272582" +source = "git+https://github.com/paritytech/reed-solomon-novelpoly.git#be3751093e60adc20c19967f5443158552829011" dependencies = [ "derive_more", "fs-err", - "itertools 0.11.0", "static_init", "thiserror", ] diff --git a/polkadot/erasure-coding/Cargo.toml b/polkadot/erasure-coding/Cargo.toml index 0b5174ef9a6c..7c51e806f779 100644 --- a/polkadot/erasure-coding/Cargo.toml +++ b/polkadot/erasure-coding/Cargo.toml @@ -12,7 +12,7 @@ workspace = true [dependencies] polkadot-primitives = { path = "../primitives" } polkadot-node-primitives = { package = "polkadot-node-primitives", path = "../node/primitives" } -novelpoly = { package = "reed-solomon-novelpoly", version = "1.0.2" } +novelpoly = { package = "reed-solomon-novelpoly", git = "https://github.com/paritytech/reed-solomon-novelpoly.git" } parity-scale-codec = { version = "3.6.1", default-features = false, features = ["std", "derive"] } sp-core = { path = "../../substrate/primitives/core" } sp-trie = { path = "../../substrate/primitives/trie" } diff --git a/polkadot/erasure-coding/src/lib.rs b/polkadot/erasure-coding/src/lib.rs index c80331e20c3e..f8e22ef630b5 100644 --- a/polkadot/erasure-coding/src/lib.rs +++ b/polkadot/erasure-coding/src/lib.rs @@ -86,6 +86,20 @@ pub enum Error { UnknownCodeParam, } +impl From for Error { + fn from(error: novelpoly::Error) -> Self { + match error { + novelpoly::Error::NeedMoreShards { .. } => Self::NotEnoughChunks, + novelpoly::Error::ParamterMustBePowerOf2 { .. } => Self::UnevenLength, + novelpoly::Error::WantedShardCountTooHigh(_) => Self::TooManyValidators, + novelpoly::Error::WantedShardCountTooLow(_) => Self::NotEnoughValidators, + novelpoly::Error::PayloadSizeIsZero { .. } => Self::BadPayload, + novelpoly::Error::InconsistentShardLengths { .. } => Self::NonUniformChunks, + _ => Self::UnknownReconstruction, + } + } +} + /// Obtain a threshold of chunks that should be enough to recover the data. pub const fn recovery_threshold(n_validators: usize) -> Result { if n_validators > MAX_VALIDATORS { @@ -137,45 +151,36 @@ pub fn reconstruct_from_systematic_v1( /// Reconstruct the available data from the set of systematic chunks. /// -/// Provide a vector containing chunk data. If too few chunks are provided, recovery is not -/// possible. +/// Provide a vector containing the first k chunks in order. If too few chunks are provided, +/// recovery is not possible. pub fn reconstruct_from_systematic( n_validators: usize, chunks: Vec<&[u8]>, ) -> Result { - let kpow2 = systematic_recovery_threshold(n_validators)?; - - let Some(first_shard) = chunks.iter().next() else { return Err(Error::NotEnoughChunks) }; - let shard_len = first_shard.len(); - - if shard_len == 0 { - return Err(Error::NonUniformChunks) - } + let code_params = code_params(n_validators)?; + let k = code_params.k(); - if chunks.iter().any(|c| c.len() != shard_len) { - return Err(Error::NonUniformChunks) + for chunk_data in chunks.iter().take(k) { + if chunk_data.len() % 2 != 0 { + return Err(Error::UnevenLength) + } } + let Some(first_shard) = chunks.iter().next() else { return Err(Error::NotEnoughChunks) }; + let shard_len = first_shard.len(); if shard_len % 2 != 0 { return Err(Error::UnevenLength) } - if chunks.len() < kpow2 { - return Err(Error::NotEnoughChunks) - } - - let mut systematic_bytes = Vec::with_capacity(shard_len * kpow2); + let bytes = code_params.make_encoder().reconstruct_from_systematic( + chunks + .into_iter() + .take(k) + .map(|data| WrappedShard::new(data.to_vec())) + .collect(), + )?; - for i in (0..shard_len).step_by(2) { - for chunk in chunks.iter().take(kpow2) { - // No need to check for index out of bounds because i goes up to shard_len and - // we return an error for non uniform chunks. - systematic_bytes.push(chunk[i]); - systematic_bytes.push(chunk[i + 1]); - } - } - - Decode::decode(&mut &systematic_bytes[..]).map_err(|err| Error::Decode(err)) + Decode::decode(&mut &bytes[..]).map_err(|err| Error::Decode(err)) } /// Obtain erasure-coded chunks for v1 `AvailableData`, one for each validator. @@ -231,40 +236,15 @@ where { let params = code_params(n_validators)?; let mut received_shards: Vec> = vec![None; n_validators]; - let mut shard_len = None; for (chunk_data, chunk_idx) in chunks.into_iter().take(n_validators) { - if chunk_idx >= n_validators { - return Err(Error::ChunkIndexOutOfBounds { chunk_index: chunk_idx, n_validators }) - } - - let shard_len = shard_len.get_or_insert_with(|| chunk_data.len()); - - if *shard_len % 2 != 0 { + if chunk_data.len() % 2 != 0 { return Err(Error::UnevenLength) } - if *shard_len != chunk_data.len() || *shard_len == 0 { - return Err(Error::NonUniformChunks) - } - received_shards[chunk_idx] = Some(WrappedShard::new(chunk_data.to_vec())); } - let res = params.make_encoder().reconstruct(received_shards); - - let payload_bytes = match res { - Err(e) => match e { - novelpoly::Error::NeedMoreShards { .. } => return Err(Error::NotEnoughChunks), - novelpoly::Error::ParamterMustBePowerOf2 { .. } => return Err(Error::UnevenLength), - novelpoly::Error::WantedShardCountTooHigh(_) => return Err(Error::TooManyValidators), - novelpoly::Error::WantedShardCountTooLow(_) => return Err(Error::NotEnoughValidators), - novelpoly::Error::PayloadSizeIsZero { .. } => return Err(Error::BadPayload), - novelpoly::Error::InconsistentShardLengths { .. } => - return Err(Error::NonUniformChunks), - _ => return Err(Error::UnknownReconstruction), - }, - Ok(payload_bytes) => payload_bytes, - }; + let payload_bytes = params.make_encoder().reconstruct(received_shards)?; Decode::decode(&mut &payload_bytes[..]).map_err(|err| Error::Decode(err)) } @@ -359,56 +339,6 @@ pub fn branch_hash(root: &H256, branch_nodes: &Proof, index: usize) -> Result

{ - remaining_len: usize, - shards: I, - cur_shard: Option<(&'a [u8], usize)>, -} - -impl<'a, I: Iterator> parity_scale_codec::Input for ShardInput<'a, I> { - fn remaining_len(&mut self) -> Result, parity_scale_codec::Error> { - Ok(Some(self.remaining_len)) - } - - fn read(&mut self, into: &mut [u8]) -> Result<(), parity_scale_codec::Error> { - let mut read_bytes = 0; - - loop { - if read_bytes == into.len() { - break - } - - let cur_shard = self.cur_shard.take().or_else(|| self.shards.next().map(|s| (s, 0))); - let (active_shard, mut in_shard) = match cur_shard { - Some((s, i)) => (s, i), - None => break, - }; - - if in_shard >= active_shard.len() { - continue - } - - let remaining_len_out = into.len() - read_bytes; - let remaining_len_shard = active_shard.len() - in_shard; - - let write_len = std::cmp::min(remaining_len_out, remaining_len_shard); - into[read_bytes..][..write_len].copy_from_slice(&active_shard[in_shard..][..write_len]); - - in_shard += write_len; - read_bytes += write_len; - self.cur_shard = Some((active_shard, in_shard)) - } - - self.remaining_len -= read_bytes; - if read_bytes == into.len() { - Ok(()) - } else { - Err("slice provided too big for input".into()) - } - } -} - #[cfg(test)] mod tests { use std::sync::Arc; @@ -428,7 +358,7 @@ mod tests { impl Arbitrary for ArbitraryAvailableData { fn arbitrary(g: &mut Gen) -> Self { // Limit the POV len to 1 mib, otherwise the test will take forever - let pov_len = u32::arbitrary(g).saturating_add(2) % (1024 * 1024); + let pov_len = (u32::arbitrary(g) % (1024 * 1024)).max(2); let pov = (0..pov_len).map(|_| u8::arbitrary(g)).collect(); @@ -475,7 +405,7 @@ mod tests { #[test] fn round_trip_systematic_works() { fn property(available_data: ArbitraryAvailableData, n_validators: u16) { - let n_validators = n_validators.saturating_add(2); + let n_validators = n_validators.max(2); let kpow2 = systematic_recovery_threshold(n_validators as usize).unwrap(); let chunks = obtain_chunks(n_validators as usize, &available_data.0).unwrap(); assert_eq!( From 965c139ebb28253f48dc2f159c37ec5eabc9cfe7 Mon Sep 17 00:00:00 2001 From: alindima Date: Thu, 11 Jan 2024 17:18:11 +0200 Subject: [PATCH 080/126] Adapt code to new design from RFC - add CollationFetching new networking req protocol - use ValidatorIndex in av-store instead of chunk_index - switch to simpler mapping algorithm from RFC - code restructuring and refactoring - more tests --- Cargo.lock | 1 + .../src/active_candidate_recovery.rs | 2 +- cumulus/client/pov-recovery/src/lib.rs | 35 +- .../src/collator_overseer.rs | 1 + polkadot/erasure-coding/src/lib.rs | 6 - polkadot/node/core/av-store/src/lib.rs | 91 +- polkadot/node/core/av-store/src/tests.rs | 348 ++- polkadot/node/core/backing/src/lib.rs | 73 +- .../node/core/bitfield-signing/src/lib.rs | 124 +- .../node/core/bitfield-signing/src/tests.rs | 26 +- polkadot/node/jaeger/src/spans.rs | 8 +- .../availability-distribution/Cargo.toml | 1 + .../availability-distribution/src/error.rs | 6 +- .../availability-distribution/src/lib.rs | 41 +- .../src/requester/fetch_task/mod.rs | 128 +- .../src/requester/fetch_task/tests.rs | 283 ++- .../src/requester/mod.rs | 65 +- .../src/requester/session_cache.rs | 15 +- .../src/requester/tests.rs | 21 +- .../src/responder.rs | 134 +- .../src/tests/mod.rs | 116 +- .../src/tests/state.rs | 200 +- .../availability-recovery/src/error.rs | 8 +- .../network/availability-recovery/src/lib.rs | 341 ++- .../network/availability-recovery/src/task.rs | 2223 ----------------- .../availability-recovery/src/task/mod.rs | 192 ++ .../src/task/strategy/chunks.rs | 340 +++ .../src/task/strategy/full.rs | 175 ++ .../src/task/strategy/mod.rs | 1534 ++++++++++++ .../src/task/strategy/systematic.rs | 350 +++ .../availability-recovery/src/tests.rs | 1040 +++++--- polkadot/node/network/bridge/src/tx/mod.rs | 10 +- .../protocol/src/request_response/mod.rs | 12 +- .../protocol/src/request_response/outgoing.rs | 36 +- .../protocol/src/request_response/v1.rs | 18 +- .../protocol/src/request_response/v2.rs | 62 +- polkadot/node/service/src/lib.rs | 7 +- polkadot/node/service/src/overseer.rs | 21 +- polkadot/node/subsystem-types/src/messages.rs | 20 +- polkadot/node/subsystem-util/Cargo.toml | 1 + .../subsystem-util/src/availability_chunks.rs | 515 ++-- polkadot/node/subsystem-util/src/lib.rs | 39 +- .../node/subsystem-util/src/runtime/mod.rs | 15 +- polkadot/primitives/src/v6/mod.rs | 8 + polkadot/primitives/src/vstaging/mod.rs | 5 +- prdoc/pr_1644.prdoc | 8 + substrate/client/network/src/service.rs | 2 +- 47 files changed, 4864 insertions(+), 3843 deletions(-) delete mode 100644 polkadot/node/network/availability-recovery/src/task.rs create mode 100644 polkadot/node/network/availability-recovery/src/task/mod.rs create mode 100644 polkadot/node/network/availability-recovery/src/task/strategy/chunks.rs create mode 100644 polkadot/node/network/availability-recovery/src/task/strategy/full.rs create mode 100644 polkadot/node/network/availability-recovery/src/task/strategy/mod.rs create mode 100644 polkadot/node/network/availability-recovery/src/task/strategy/systematic.rs create mode 100644 prdoc/pr_1644.prdoc diff --git a/Cargo.lock b/Cargo.lock index 18ea630783be..dfb4192f3716 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -12988,6 +12988,7 @@ dependencies = [ "parity-scale-codec", "parking_lot 0.12.1", "pin-project", + "polkadot-erasure-coding", "polkadot-node-jaeger", "polkadot-node-metrics", "polkadot-node-network-protocol", diff --git a/cumulus/client/pov-recovery/src/active_candidate_recovery.rs b/cumulus/client/pov-recovery/src/active_candidate_recovery.rs index 2661339879af..c41c543f04d1 100644 --- a/cumulus/client/pov-recovery/src/active_candidate_recovery.rs +++ b/cumulus/client/pov-recovery/src/active_candidate_recovery.rs @@ -56,7 +56,7 @@ impl ActiveCandidateRecovery { candidate.receipt.clone(), candidate.session_index, None, - Some(candidate.relay_parent_block_number), + None, tx, ), "ActiveCandidateRecovery", diff --git a/cumulus/client/pov-recovery/src/lib.rs b/cumulus/client/pov-recovery/src/lib.rs index c126ef0ed0b1..69544c516fb1 100644 --- a/cumulus/client/pov-recovery/src/lib.rs +++ b/cumulus/client/pov-recovery/src/lib.rs @@ -55,8 +55,7 @@ use polkadot_node_primitives::{PoV, POV_BOMB_LIMIT}; use polkadot_node_subsystem::messages::AvailabilityRecoveryMessage; use polkadot_overseer::Handle as OverseerHandle; use polkadot_primitives::{ - BlockId, BlockNumber as RCBlockNumber, CandidateReceipt, CommittedCandidateReceipt, - Id as ParaId, SessionIndex, + CandidateReceipt, CommittedCandidateReceipt, Id as ParaId, SessionIndex, }; use cumulus_primitives_core::ParachainBlockData; @@ -146,7 +145,6 @@ struct Candidate { session_index: SessionIndex, block_number: NumberFor, parent_hash: Block::Hash, - relay_parent_block_number: RCBlockNumber, // Lazy recovery has been submitted. // Should be true iff a block is either queued to be recovered or // recovery is currently in progress. @@ -270,7 +268,6 @@ where &mut self, receipt: CommittedCandidateReceipt, session_index: SessionIndex, - relay_parent_block_number: RCBlockNumber, ) { let header = match Block::Header::decode(&mut &receipt.commitments.head_data.0[..]) { Ok(header) => header, @@ -301,7 +298,6 @@ where block_number: *header.number(), receipt: receipt.to_plain(), session_index, - relay_parent_block_number, parent_hash: *header.parent_hash(), waiting_recovery: false, }, @@ -559,8 +555,8 @@ where loop { select! { pending_candidate = pending_candidates.next() => { - if let Some((receipt, session_index, relay_parent_block_nunber)) = pending_candidate { - self.handle_pending_candidate(receipt, session_index, relay_parent_block_nunber).await; + if let Some((receipt, session_index)) = pending_candidate { + self.handle_pending_candidate(receipt, session_index).await; } else { tracing::debug!(target: LOG_TARGET, "Pending candidates stream ended"); return; @@ -619,8 +615,7 @@ async fn pending_candidates( relay_chain_client: impl RelayChainInterface + Clone, para_id: ParaId, sync_service: Arc, -) -> RelayChainResult> -{ +) -> RelayChainResult> { let import_notification_stream = relay_chain_client.import_notification_stream().await?; let filtered_stream = import_notification_stream.filter_map(move |n| { @@ -655,29 +650,9 @@ async fn pending_candidates( "Failed to fetch session index.", ) }); - let relay_parent_header = match client_for_closure.header(BlockId::Hash(hash)).await { - Ok(Some(header)) => header, - Ok(None) => { - tracing::error!( - target: LOG_TARGET, - "Failed to retrieve relay parent block number", - ); - return None - }, - Err(err) => { - tracing::error!( - target: LOG_TARGET, - error = ?err, - "Failed to retrieve relay parent block number", - ); - return None - }, - }; if let Ok(Some(candidate)) = pending_availability_result { - session_index_result - .map(|session_index| (candidate, session_index, relay_parent_header.number)) - .ok() + session_index_result.map(|session_index| (candidate, session_index)).ok() } else { None } diff --git a/cumulus/client/relay-chain-minimal-node/src/collator_overseer.rs b/cumulus/client/relay-chain-minimal-node/src/collator_overseer.rs index 5f5bf338ef99..3020639de985 100644 --- a/cumulus/client/relay-chain-minimal-node/src/collator_overseer.rs +++ b/cumulus/client/relay-chain-minimal-node/src/collator_overseer.rs @@ -111,6 +111,7 @@ fn build_overseer( .availability_distribution(DummySubsystem) .availability_recovery(AvailabilityRecoverySubsystem::for_collator( available_data_req_receiver, + &req_protocol_names, Metrics::register(registry)?, )) .availability_store(DummySubsystem) diff --git a/polkadot/erasure-coding/src/lib.rs b/polkadot/erasure-coding/src/lib.rs index f8e22ef630b5..f763b656c5c3 100644 --- a/polkadot/erasure-coding/src/lib.rs +++ b/polkadot/erasure-coding/src/lib.rs @@ -166,12 +166,6 @@ pub fn reconstruct_from_systematic( } } - let Some(first_shard) = chunks.iter().next() else { return Err(Error::NotEnoughChunks) }; - let shard_len = first_shard.len(); - if shard_len % 2 != 0 { - return Err(Error::UnevenLength) - } - let bytes = code_params.make_encoder().reconstruct_from_systematic( chunks .into_iter() diff --git a/polkadot/node/core/av-store/src/lib.rs b/polkadot/node/core/av-store/src/lib.rs index 47298f9f2dde..e9c6b83b098c 100644 --- a/polkadot/node/core/av-store/src/lib.rs +++ b/polkadot/node/core/av-store/src/lib.rs @@ -48,8 +48,10 @@ use polkadot_node_subsystem::{ }; use polkadot_node_subsystem_util as util; use polkadot_primitives::{ - BlockNumber, CandidateEvent, CandidateHash, CandidateReceipt, ChunkIndex, Hash, Header, + vstaging::NodeFeatures, BlockNumber, CandidateEvent, CandidateHash, CandidateReceipt, + ChunkIndex, CoreIndex, Hash, Header, ValidatorIndex, }; +use util::availability_chunks::availability_chunk_indices; mod metrics; pub use self::metrics::*; @@ -208,9 +210,9 @@ fn load_chunk( db: &Arc, config: &Config, candidate_hash: &CandidateHash, - chunk_index: ChunkIndex, + validator_index: ValidatorIndex, ) -> Result, Error> { - let key = (CHUNK_PREFIX, candidate_hash, chunk_index).encode(); + let key = (CHUNK_PREFIX, candidate_hash, validator_index).encode(); query_inner(db, config.col_data, &key) } @@ -219,10 +221,10 @@ fn write_chunk( tx: &mut DBTransaction, config: &Config, candidate_hash: &CandidateHash, - chunk_index: ChunkIndex, + validator_index: ValidatorIndex, erasure_chunk: &ErasureChunk, ) { - let key = (CHUNK_PREFIX, candidate_hash, chunk_index).encode(); + let key = (CHUNK_PREFIX, candidate_hash, validator_index).encode(); tx.put_vec(config.col_data, &key, erasure_chunk.encode()); } @@ -231,9 +233,9 @@ fn delete_chunk( tx: &mut DBTransaction, config: &Config, candidate_hash: &CandidateHash, - chunk_index: ChunkIndex, + validator_index: ValidatorIndex, ) { - let key = (CHUNK_PREFIX, candidate_hash, chunk_index).encode(); + let key = (CHUNK_PREFIX, candidate_hash, validator_index).encode(); tx.delete(config.col_data, &key[..]); } @@ -1107,21 +1109,22 @@ fn process_message( .map_or(false, |m| m.data_available); let _ = tx.send(a); }, - AvailabilityStoreMessage::QueryChunk(candidate, chunk_index, tx) => { + AvailabilityStoreMessage::QueryChunk(candidate, validator_index, tx) => { let _timer = subsystem.metrics.time_get_chunk(); - let _ = tx.send(load_chunk(&subsystem.db, &subsystem.config, &candidate, chunk_index)?); + let _ = + tx.send(load_chunk(&subsystem.db, &subsystem.config, &candidate, validator_index)?); }, AvailabilityStoreMessage::QueryChunkSize(candidate, tx) => { let meta = load_meta(&subsystem.db, &subsystem.config, &candidate)?; - let chunk_index = meta.map_or(None, |meta| meta.chunks_stored.first_one()); + let validator_index = meta.map_or(None, |meta| meta.chunks_stored.first_one()); - let maybe_chunk_size = if let Some(chunk_index) = chunk_index { + let maybe_chunk_size = if let Some(validator_index) = validator_index { load_chunk( &subsystem.db, &subsystem.config, &candidate, - ChunkIndex(chunk_index as u32), + ValidatorIndex(validator_index as u32), )? .map(|erasure_chunk| erasure_chunk.chunk.len()) } else { @@ -1138,20 +1141,23 @@ fn process_message( Some(meta) => { let mut chunks = Vec::new(); - for (index, _) in meta.chunks_stored.iter().enumerate().filter(|(_, b)| **b) { + for (validator_index, _) in + meta.chunks_stored.iter().enumerate().filter(|(_, b)| **b) + { + let validator_index = ValidatorIndex(validator_index as _); let _timer = subsystem.metrics.time_get_chunk(); match load_chunk( &subsystem.db, &subsystem.config, &candidate, - ChunkIndex(index as _), + validator_index, )? { - Some(c) => chunks.push(c), + Some(c) => chunks.push((validator_index, c)), None => { gum::warn!( target: LOG_TARGET, ?candidate, - index, + ?validator_index, "No chunk found for set bit in meta" ); }, @@ -1162,17 +1168,23 @@ fn process_message( }, } }, - AvailabilityStoreMessage::QueryChunkAvailability(candidate, chunk_index, tx) => { + AvailabilityStoreMessage::QueryChunkAvailability(candidate, validator_index, tx) => { let a = load_meta(&subsystem.db, &subsystem.config, &candidate)?.map_or(false, |m| { - *m.chunks_stored.get(chunk_index.0 as usize).as_deref().unwrap_or(&false) + *m.chunks_stored.get(validator_index.0 as usize).as_deref().unwrap_or(&false) }); let _ = tx.send(a); }, - AvailabilityStoreMessage::StoreChunk { candidate_hash, chunk, tx } => { + AvailabilityStoreMessage::StoreChunk { candidate_hash, validator_index, chunk, tx } => { subsystem.metrics.on_chunks_received(1); let _timer = subsystem.metrics.time_store_chunk(); - match store_chunk(&subsystem.db, &subsystem.config, candidate_hash, chunk) { + match store_chunk( + &subsystem.db, + &subsystem.config, + candidate_hash, + validator_index, + chunk, + ) { Ok(true) => { let _ = tx.send(Ok(())); }, @@ -1190,6 +1202,8 @@ fn process_message( n_validators, available_data, expected_erasure_root, + core_index, + node_features, tx, } => { subsystem.metrics.on_chunks_received(n_validators as _); @@ -1202,6 +1216,8 @@ fn process_message( n_validators as _, available_data, expected_erasure_root, + core_index, + node_features, ); match res { @@ -1232,6 +1248,7 @@ fn store_chunk( db: &Arc, config: &Config, candidate_hash: CandidateHash, + validator_index: ValidatorIndex, chunk: ErasureChunk, ) -> Result { let mut tx = DBTransaction::new(); @@ -1241,12 +1258,12 @@ fn store_chunk( None => return Ok(false), // we weren't informed of this candidate by import events. }; - match meta.chunks_stored.get(chunk.index.0 as usize).map(|b| *b) { + match meta.chunks_stored.get(validator_index.0 as usize).map(|b| *b) { Some(true) => return Ok(true), // already stored. Some(false) => { - meta.chunks_stored.set(chunk.index.0 as usize, true); + meta.chunks_stored.set(validator_index.0 as usize, true); - write_chunk(&mut tx, config, &candidate_hash, chunk.index, &chunk); + write_chunk(&mut tx, config, &candidate_hash, validator_index, &chunk); write_meta(&mut tx, config, &candidate_hash, &meta); }, None => return Ok(false), // out of bounds. @@ -1256,6 +1273,7 @@ fn store_chunk( target: LOG_TARGET, ?candidate_hash, chunk_index = %chunk.index.0, + validator_index = %validator_index.0, "Stored chunk index for candidate.", ); @@ -1263,13 +1281,14 @@ fn store_chunk( Ok(true) } -// Ok(true) on success, Ok(false) on failure, and Err on internal error. fn store_available_data( subsystem: &AvailabilityStoreSubsystem, candidate_hash: CandidateHash, n_validators: usize, available_data: AvailableData, expected_erasure_root: Hash, + core_index: CoreIndex, + node_features: NodeFeatures, ) -> Result<(), Error> { let mut tx = DBTransaction::new(); @@ -1311,16 +1330,26 @@ fn store_available_data( drop(erasure_span); - let erasure_chunks = chunks.iter().zip(branches.map(|(proof, _)| proof)).enumerate().map( - |(index, (chunk, proof))| ErasureChunk { + let erasure_chunks: Vec<_> = chunks + .iter() + .zip(branches.map(|(proof, _)| proof)) + .enumerate() + .map(|(index, (chunk, proof))| ErasureChunk { chunk: chunk.clone(), proof, index: ChunkIndex(index as u32), - }, - ); + }) + .collect(); - for chunk in erasure_chunks { - write_chunk(&mut tx, &subsystem.config, &candidate_hash, chunk.index, &chunk); + let chunk_indices = availability_chunk_indices(Some(&node_features), n_validators, core_index)?; + for (validator_index, chunk_index) in chunk_indices.into_iter().enumerate() { + write_chunk( + &mut tx, + &subsystem.config, + &candidate_hash, + ValidatorIndex(validator_index as u32), + &erasure_chunks[chunk_index.0 as usize], + ); } meta.data_available = true; @@ -1365,7 +1394,7 @@ fn prune_all(db: &Arc, config: &Config, now: Duration) -> Result<( // delete chunks. for (i, b) in meta.chunks_stored.iter().enumerate() { if *b { - delete_chunk(&mut tx, config, &candidate_hash, ChunkIndex(i as _)); + delete_chunk(&mut tx, config, &candidate_hash, ValidatorIndex(i as _)); } } diff --git a/polkadot/node/core/av-store/src/tests.rs b/polkadot/node/core/av-store/src/tests.rs index 13c7c7d3bc31..14c4297c2657 100644 --- a/polkadot/node/core/av-store/src/tests.rs +++ b/polkadot/node/core/av-store/src/tests.rs @@ -18,6 +18,7 @@ use super::*; use assert_matches::assert_matches; use futures::{channel::oneshot, executor, future, Future}; +use util::availability_chunks::availability_chunk_index; use self::test_helpers::mock::new_leaf; use ::test_helpers::TestCandidateBuilder; @@ -31,8 +32,8 @@ use polkadot_node_subsystem::{ use polkadot_node_subsystem_test_helpers as test_helpers; use polkadot_node_subsystem_util::{database::Database, TimeoutExt}; use polkadot_primitives::{ - CandidateHash, CandidateReceipt, CoreIndex, GroupIndex, HeadData, Header, - PersistedValidationData, ValidatorId, + vstaging::node_features, CandidateHash, CandidateReceipt, CoreIndex, GroupIndex, HeadData, + Header, PersistedValidationData, ValidatorId, }; use sp_keyring::Sr25519Keyring; @@ -272,8 +273,7 @@ fn runtime_api_error_does_not_stop_the_subsystem() { // but that's fine, we're still alive let (tx, rx) = oneshot::channel(); let candidate_hash = CandidateHash(Hash::repeat_byte(33)); - let chunk_index = ChunkIndex(5); - let query_chunk = AvailabilityStoreMessage::QueryChunk(candidate_hash, chunk_index, tx); + let query_chunk = AvailabilityStoreMessage::QueryChunk(candidate_hash, 5.into(), tx); overseer_send(&mut virtual_overseer, query_chunk.into()).await; @@ -289,6 +289,7 @@ fn store_chunk_works() { test_harness(TestState::default(), store.clone(), |mut virtual_overseer| async move { let candidate_hash = CandidateHash(Hash::repeat_byte(33)); let chunk_index = ChunkIndex(5); + let validator_index = ValidatorIndex(2); let n_validators = 10; let chunk = ErasureChunk { @@ -314,14 +315,18 @@ fn store_chunk_works() { let (tx, rx) = oneshot::channel(); - let chunk_msg = - AvailabilityStoreMessage::StoreChunk { candidate_hash, chunk: chunk.clone(), tx }; + let chunk_msg = AvailabilityStoreMessage::StoreChunk { + candidate_hash, + validator_index, + chunk: chunk.clone(), + tx, + }; overseer_send(&mut virtual_overseer, chunk_msg).await; assert_eq!(rx.await.unwrap(), Ok(())); let (tx, rx) = oneshot::channel(); - let query_chunk = AvailabilityStoreMessage::QueryChunk(candidate_hash, chunk_index, tx); + let query_chunk = AvailabilityStoreMessage::QueryChunk(candidate_hash, validator_index, tx); overseer_send(&mut virtual_overseer, query_chunk).await; @@ -337,6 +342,7 @@ fn store_chunk_does_nothing_if_no_entry_already() { test_harness(TestState::default(), store.clone(), |mut virtual_overseer| async move { let candidate_hash = CandidateHash(Hash::repeat_byte(33)); let chunk_index = ChunkIndex(5); + let validator_index = ValidatorIndex(2); let chunk = ErasureChunk { chunk: vec![1, 2, 3], @@ -346,14 +352,18 @@ fn store_chunk_does_nothing_if_no_entry_already() { let (tx, rx) = oneshot::channel(); - let chunk_msg = - AvailabilityStoreMessage::StoreChunk { candidate_hash, chunk: chunk.clone(), tx }; + let chunk_msg = AvailabilityStoreMessage::StoreChunk { + candidate_hash, + validator_index, + chunk: chunk.clone(), + tx, + }; overseer_send(&mut virtual_overseer, chunk_msg).await; assert_eq!(rx.await.unwrap(), Err(())); let (tx, rx) = oneshot::channel(); - let query_chunk = AvailabilityStoreMessage::QueryChunk(candidate_hash, chunk_index, tx); + let query_chunk = AvailabilityStoreMessage::QueryChunk(candidate_hash, validator_index, tx); overseer_send(&mut virtual_overseer, query_chunk).await; @@ -368,7 +378,7 @@ fn query_chunk_checks_meta() { test_harness(TestState::default(), store.clone(), |mut virtual_overseer| async move { let candidate_hash = CandidateHash(Hash::repeat_byte(33)); - let chunk_index = ChunkIndex(5); + let validator_index = ValidatorIndex(5); let n_validators = 10; // Ensure an entry already exists. In reality this would come from watching @@ -382,7 +392,7 @@ fn query_chunk_checks_meta() { data_available: false, chunks_stored: { let mut v = bitvec::bitvec![u8, BitOrderLsb0; 0; n_validators]; - v.set(chunk_index.0 as usize, true); + v.set(validator_index.0 as usize, true); v }, state: State::Unavailable(BETimestamp(0)), @@ -392,7 +402,7 @@ fn query_chunk_checks_meta() { let (tx, rx) = oneshot::channel(); let query_chunk = - AvailabilityStoreMessage::QueryChunkAvailability(candidate_hash, chunk_index, tx); + AvailabilityStoreMessage::QueryChunkAvailability(candidate_hash, validator_index, tx); overseer_send(&mut virtual_overseer, query_chunk.into()).await; assert!(rx.await.unwrap()); @@ -400,7 +410,7 @@ fn query_chunk_checks_meta() { let (tx, rx) = oneshot::channel(); let query_chunk = AvailabilityStoreMessage::QueryChunkAvailability( candidate_hash, - ChunkIndex(chunk_index.0 + 1), + ValidatorIndex(validator_index.0 + 1), tx, ); @@ -416,8 +426,10 @@ fn store_available_data_erasure_mismatch() { let test_state = TestState::default(); test_harness(test_state.clone(), store.clone(), |mut virtual_overseer| async move { let candidate_hash = CandidateHash(Hash::repeat_byte(1)); - let chunk_index = ChunkIndex(5); + let validator_index = ValidatorIndex(5); let n_validators = 10; + let core_index = CoreIndex(8); + let node_features = NodeFeatures::EMPTY; let pov = PoV { block_data: BlockData(vec![4, 5, 6]) }; @@ -431,6 +443,8 @@ fn store_available_data_erasure_mismatch() { candidate_hash, n_validators, available_data: available_data.clone(), + core_index, + node_features, tx, // A dummy erasure root should lead to failure. expected_erasure_root: Hash::default(), @@ -441,102 +455,172 @@ fn store_available_data_erasure_mismatch() { assert!(query_available_data(&mut virtual_overseer, candidate_hash).await.is_none()); - assert!(query_chunk(&mut virtual_overseer, candidate_hash, chunk_index).await.is_none()); - - virtual_overseer - }); -} - -#[test] -fn store_block_works() { - let store = test_store(); - let test_state = TestState::default(); - test_harness(test_state.clone(), store.clone(), |mut virtual_overseer| async move { - let candidate_hash = CandidateHash(Hash::repeat_byte(1)); - let chunk_index = ChunkIndex(5); - let n_validators = 10; - - let pov = PoV { block_data: BlockData(vec![4, 5, 6]) }; - - let available_data = AvailableData { - pov: Arc::new(pov), - validation_data: test_state.persisted_validation_data.clone(), - }; - let (tx, rx) = oneshot::channel(); - - let chunks = erasure::obtain_chunks_v1(10, &available_data).unwrap(); - let mut branches = erasure::branches(chunks.as_ref()); - - let block_msg = AvailabilityStoreMessage::StoreAvailableData { - candidate_hash, - n_validators, - available_data: available_data.clone(), - tx, - expected_erasure_root: branches.root(), - }; - - virtual_overseer.send(FromOrchestra::Communication { msg: block_msg }).await; - assert_eq!(rx.await.unwrap(), Ok(())); + assert!(query_chunk(&mut virtual_overseer, candidate_hash, validator_index) + .await + .is_none()); - let pov = query_available_data(&mut virtual_overseer, candidate_hash).await.unwrap(); - assert_eq!(pov, available_data); - - let chunk = query_chunk(&mut virtual_overseer, candidate_hash, chunk_index).await.unwrap(); - - let branch = branches.nth(5).unwrap(); - let expected_chunk = ErasureChunk { - chunk: branch.1.to_vec(), - index: ChunkIndex(5), - proof: Proof::try_from(branch.0).unwrap(), - }; - - assert_eq!(chunk, expected_chunk); virtual_overseer }); } #[test] -fn store_pov_and_query_chunk_works() { - let store = test_store(); - let test_state = TestState::default(); - - test_harness(test_state.clone(), store.clone(), |mut virtual_overseer| async move { - let candidate_hash = CandidateHash(Hash::repeat_byte(1)); - let n_validators = 10; - - let pov = PoV { block_data: BlockData(vec![4, 5, 6]) }; - - let available_data = AvailableData { - pov: Arc::new(pov), - validation_data: test_state.persisted_validation_data.clone(), - }; - - let chunks_expected = - erasure::obtain_chunks_v1(n_validators as _, &available_data).unwrap(); - let branches = erasure::branches(chunks_expected.as_ref()); - - let (tx, rx) = oneshot::channel(); - let block_msg = AvailabilityStoreMessage::StoreAvailableData { - candidate_hash, - n_validators, - available_data, - tx, - expected_erasure_root: branches.root(), - }; - - virtual_overseer.send(FromOrchestra::Communication { msg: block_msg }).await; +fn store_pov_and_queries_work() { + // If the AvailabilityChunkMapping feature is not enabled, + // ValidatorIndex->ChunkIndex mapping should be 1:1 for all core indices. + { + let n_cores = 4; + for core_index in 0..n_cores { + let store = test_store(); + let test_state = TestState::default(); + + test_harness(test_state.clone(), store.clone(), |mut virtual_overseer| async move { + let node_features = NodeFeatures::EMPTY; + let candidate_hash = CandidateHash(Hash::repeat_byte(1)); + let n_validators = 10; + + let pov = PoV { block_data: BlockData(vec![4, 5, 6]) }; + let available_data = AvailableData { + pov: Arc::new(pov), + validation_data: test_state.persisted_validation_data.clone(), + }; + + let chunks = erasure::obtain_chunks_v1(n_validators as _, &available_data).unwrap(); + + let branches = erasure::branches(chunks.as_ref()); + + let (tx, rx) = oneshot::channel(); + let block_msg = AvailabilityStoreMessage::StoreAvailableData { + candidate_hash, + n_validators, + available_data: available_data.clone(), + tx, + expected_erasure_root: branches.root(), + node_features: node_features.clone(), + core_index: CoreIndex(core_index), + }; + + virtual_overseer.send(FromOrchestra::Communication { msg: block_msg }).await; + assert_eq!(rx.await.unwrap(), Ok(())); + + let pov: AvailableData = + query_available_data(&mut virtual_overseer, candidate_hash).await.unwrap(); + assert_eq!(pov, available_data); + + let query_all_chunks_res = + query_all_chunks(&mut virtual_overseer, candidate_hash).await; + assert_eq!(query_all_chunks_res.len(), chunks.len()); + + let branches: Vec<_> = branches.collect(); + + for validator_index in 0..n_validators { + let chunk = query_chunk( + &mut virtual_overseer, + candidate_hash, + ValidatorIndex(validator_index as _), + ) + .await + .unwrap(); + let branch = &branches[validator_index as usize]; + let expected_chunk = ErasureChunk { + chunk: branch.1.to_vec(), + index: validator_index.into(), + proof: Proof::try_from(branch.0.clone()).unwrap(), + }; + assert_eq!(chunk, expected_chunk); + assert_eq!(chunk, query_all_chunks_res[validator_index as usize]); + } - assert_eq!(rx.await.unwrap(), Ok(())); + virtual_overseer + }); + } + } - for i in 0..n_validators { - let chunk = query_chunk(&mut virtual_overseer, candidate_hash, ChunkIndex(i as _)) - .await - .unwrap(); + // If the AvailabilityChunkMapping feature is enabled, let's also test the + // ValidatorIndex -> ChunkIndex mapping. + { + let n_cores = 4; + for core_index in 0..n_cores { + let store = test_store(); + let test_state = TestState::default(); + + test_harness(test_state.clone(), store.clone(), |mut virtual_overseer| async move { + let mut node_features = NodeFeatures::EMPTY; + let feature_bit = node_features::FeatureIndex::AvailabilityChunkMapping; + node_features.resize((feature_bit as u8 + 1) as usize, false); + node_features.set(feature_bit as u8 as usize, true); + + let candidate_hash = CandidateHash(Hash::repeat_byte(1)); + let n_validators = 10; + + let pov = PoV { block_data: BlockData(vec![4, 5, 6]) }; + let available_data = AvailableData { + pov: Arc::new(pov), + validation_data: test_state.persisted_validation_data.clone(), + }; + + let chunks = erasure::obtain_chunks_v1(n_validators as _, &available_data).unwrap(); + + let branches = erasure::branches(chunks.as_ref()); + + let (tx, rx) = oneshot::channel(); + let block_msg = AvailabilityStoreMessage::StoreAvailableData { + candidate_hash, + n_validators, + available_data: available_data.clone(), + tx, + expected_erasure_root: branches.root(), + node_features: node_features.clone(), + core_index: CoreIndex(core_index), + }; + + virtual_overseer.send(FromOrchestra::Communication { msg: block_msg }).await; + assert_eq!(rx.await.unwrap(), Ok(())); + + let pov: AvailableData = + query_available_data(&mut virtual_overseer, candidate_hash).await.unwrap(); + assert_eq!(pov, available_data); + + let query_all_chunks_res = + query_all_chunks(&mut virtual_overseer, candidate_hash).await; + assert_eq!(query_all_chunks_res.len(), chunks.len()); + + let branches: Vec<_> = branches.collect(); + + for validator_index in 0..n_validators { + let chunk = query_chunk( + &mut virtual_overseer, + candidate_hash, + ValidatorIndex(validator_index as _), + ) + .await + .unwrap(); + let expected_chunk_index = availability_chunk_index( + Some(&node_features), + n_validators as usize, + CoreIndex(core_index), + ValidatorIndex(validator_index), + ) + .unwrap(); + let branch = &branches[expected_chunk_index.0 as usize]; + let expected_chunk = ErasureChunk { + chunk: branch.1.to_vec(), + index: expected_chunk_index, + proof: Proof::try_from(branch.0.clone()).unwrap(), + }; + assert_eq!(chunk, expected_chunk); + assert_eq!( + &chunk, + query_all_chunks_res + .iter() + .find(|c| c.index == expected_chunk_index) + .unwrap() + ); + } - assert_eq!(chunk.chunk, chunks_expected[i as usize]); + virtual_overseer + }); } - virtual_overseer - }); + } } #[test] @@ -571,6 +655,8 @@ fn query_all_chunks_works() { n_validators, available_data, tx, + core_index: CoreIndex(1), + node_features: NodeFeatures::EMPTY, expected_erasure_root: branches.root(), }; @@ -602,6 +688,7 @@ fn query_all_chunks_works() { let store_chunk_msg = AvailabilityStoreMessage::StoreChunk { candidate_hash: candidate_hash_2, chunk, + validator_index: ValidatorIndex(1), tx, }; @@ -611,29 +698,14 @@ fn query_all_chunks_works() { assert_eq!(rx.await.unwrap(), Ok(())); } - { - let (tx, rx) = oneshot::channel(); - - let msg = AvailabilityStoreMessage::QueryAllChunks(candidate_hash_1, tx); - virtual_overseer.send(FromOrchestra::Communication { msg }).await; - assert_eq!(rx.await.unwrap().len(), n_validators as usize); - } - - { - let (tx, rx) = oneshot::channel(); - - let msg = AvailabilityStoreMessage::QueryAllChunks(candidate_hash_2, tx); - virtual_overseer.send(FromOrchestra::Communication { msg }).await; - assert_eq!(rx.await.unwrap().len(), 1); - } + assert_eq!( + query_all_chunks(&mut virtual_overseer, candidate_hash_1).await.len(), + n_validators as usize + ); - { - let (tx, rx) = oneshot::channel(); + assert_eq!(query_all_chunks(&mut virtual_overseer, candidate_hash_2).await.len(), 1); + assert_eq!(query_all_chunks(&mut virtual_overseer, candidate_hash_3).await.len(), 0); - let msg = AvailabilityStoreMessage::QueryAllChunks(candidate_hash_3, tx); - virtual_overseer.send(FromOrchestra::Communication { msg }).await; - assert_eq!(rx.await.unwrap().len(), 0); - } virtual_overseer }); } @@ -663,6 +735,8 @@ fn stored_but_not_included_data_is_pruned() { n_validators, available_data: available_data.clone(), tx, + node_features: NodeFeatures::EMPTY, + core_index: CoreIndex(1), expected_erasure_root: branches.root(), }; @@ -719,6 +793,8 @@ fn stored_data_kept_until_finalized() { n_validators, available_data: available_data.clone(), tx, + node_features: NodeFeatures::EMPTY, + core_index: CoreIndex(1), expected_erasure_root: branches.root(), }; @@ -994,6 +1070,8 @@ fn forkfullness_works() { n_validators, available_data: available_data_1.clone(), tx, + node_features: NodeFeatures::EMPTY, + core_index: CoreIndex(1), expected_erasure_root: branches.root(), }; @@ -1010,6 +1088,8 @@ fn forkfullness_works() { n_validators, available_data: available_data_2.clone(), tx, + node_features: NodeFeatures::EMPTY, + core_index: CoreIndex(1), expected_erasure_root: branches.root(), }; @@ -1112,7 +1192,7 @@ async fn query_available_data( async fn query_chunk( virtual_overseer: &mut VirtualOverseer, candidate_hash: CandidateHash, - index: ChunkIndex, + index: ValidatorIndex, ) -> Option { let (tx, rx) = oneshot::channel(); @@ -1122,6 +1202,18 @@ async fn query_chunk( rx.await.unwrap() } +async fn query_all_chunks( + virtual_overseer: &mut VirtualOverseer, + candidate_hash: CandidateHash, +) -> Vec { + let (tx, rx) = oneshot::channel(); + + let msg = AvailabilityStoreMessage::QueryAllChunks(candidate_hash, tx); + virtual_overseer.send(FromOrchestra::Communication { msg }).await; + + rx.await.unwrap() +} + async fn has_all_chunks( virtual_overseer: &mut VirtualOverseer, candidate_hash: CandidateHash, @@ -1129,7 +1221,7 @@ async fn has_all_chunks( expect_present: bool, ) -> bool { for i in 0..n_validators { - if query_chunk(virtual_overseer, candidate_hash, ChunkIndex(i)).await.is_some() != + if query_chunk(virtual_overseer, candidate_hash, ValidatorIndex(i)).await.is_some() != expect_present { return false @@ -1228,8 +1320,12 @@ fn query_chunk_size_works() { let (tx, rx) = oneshot::channel(); - let chunk_msg = - AvailabilityStoreMessage::StoreChunk { candidate_hash, chunk: chunk.clone(), tx }; + let chunk_msg = AvailabilityStoreMessage::StoreChunk { + candidate_hash, + chunk: chunk.clone(), + tx, + validator_index: chunk_index.into(), + }; overseer_send(&mut virtual_overseer, chunk_msg).await; assert_eq!(rx.await.unwrap(), Ok(())); diff --git a/polkadot/node/core/backing/src/lib.rs b/polkadot/node/core/backing/src/lib.rs index 98bbd6232add..af2106b381fb 100644 --- a/polkadot/node/core/backing/src/lib.rs +++ b/polkadot/node/core/backing/src/lib.rs @@ -104,10 +104,10 @@ use polkadot_node_subsystem_util::{ Validator, }; use polkadot_primitives::{ - BackedCandidate, CandidateCommitments, CandidateHash, CandidateReceipt, + vstaging::NodeFeatures, BackedCandidate, CandidateCommitments, CandidateHash, CandidateReceipt, CommittedCandidateReceipt, CoreIndex, CoreState, ExecutorParams, Hash, Id as ParaId, - PersistedValidationData, PvfExecKind, SigningContext, ValidationCode, ValidatorId, - ValidatorIndex, ValidatorSignature, ValidityAttestation, + PersistedValidationData, PvfExecKind, SessionIndex, SigningContext, ValidationCode, + ValidatorId, ValidatorIndex, ValidatorSignature, ValidityAttestation, }; use sp_keystore::KeystorePtr; use statement_table::{ @@ -118,7 +118,7 @@ use statement_table::{ }, Config as TableConfig, Context as TableContextTrait, Table, }; -use util::vstaging::get_disabled_validators_with_fallback; +use util::{runtime::request_node_features, vstaging::get_disabled_validators_with_fallback}; mod error; @@ -204,12 +204,22 @@ where } } +struct Assignment { + /// The `ParaId` assigned to the local validator at this relay parent. + para_id: ParaId, + /// The core index above para id is scheduled on. + core_index: CoreIndex, +} + struct PerRelayParentState { prospective_parachains_mode: ProspectiveParachainsMode, /// The hash of the relay parent on top of which this job is doing it's work. parent: Hash, - /// The `ParaId` assigned to the local validator at this relay parent. - assignment: Option, + /// Session index. + session_index: SessionIndex, + /// The `ParaId` assigned to the local validator at this relay parent and the core index it's + /// scheduled on. + assignment: Option, /// The candidates that are backed by enough validators in their group, by hash. backed: HashSet, /// The table of candidates and statements under this relay-parent. @@ -495,6 +505,8 @@ async fn store_available_data( candidate_hash: CandidateHash, available_data: AvailableData, expected_erasure_root: Hash, + core_index: CoreIndex, + node_features: NodeFeatures, ) -> Result<(), Error> { let (tx, rx) = oneshot::channel(); // Important: the `av-store` subsystem will check if the erasure root of the `available_data` @@ -507,6 +519,8 @@ async fn store_available_data( n_validators, available_data, expected_erasure_root, + core_index, + node_features, tx, }) .await; @@ -530,6 +544,8 @@ async fn make_pov_available( candidate_hash: CandidateHash, validation_data: PersistedValidationData, expected_erasure_root: Hash, + core_index: CoreIndex, + node_features: NodeFeatures, ) -> Result<(), Error> { store_available_data( sender, @@ -537,6 +553,8 @@ async fn make_pov_available( candidate_hash, AvailableData { pov, validation_data }, expected_erasure_root, + core_index, + node_features, ) .await } @@ -607,6 +625,7 @@ struct BackgroundValidationParams { tx_command: mpsc::Sender<(Hash, ValidatedCandidateCommand)>, candidate: CandidateReceipt, relay_parent: Hash, + session_index: SessionIndex, persisted_validation_data: PersistedValidationData, pov: PoVData, n_validators: usize, @@ -618,12 +637,14 @@ async fn validate_and_make_available( impl overseer::CandidateBackingSenderTrait, impl Fn(BackgroundValidationResult) -> ValidatedCandidateCommand + Sync, >, + core_index: CoreIndex, ) -> Result<(), Error> { let BackgroundValidationParams { mut sender, mut tx_command, candidate, relay_parent, + session_index, persisted_validation_data, pov, n_validators, @@ -653,6 +674,10 @@ async fn validate_and_make_available( Err(e) => return Err(Error::UtilError(e)), }; + let node_features = request_node_features(relay_parent, session_index, &mut sender) + .await? + .unwrap_or(NodeFeatures::EMPTY); + let pov = match pov { PoVData::Ready(pov) => pov, PoVData::FetchFromValidator { from_validator, candidate_hash, pov_hash } => @@ -708,6 +733,8 @@ async fn validate_and_make_available( candidate.hash(), validation_data.clone(), candidate.descriptor.erasure_root, + core_index, + node_features, ) .await; @@ -1077,7 +1104,7 @@ async fn construct_per_relay_parent_state( let group_index = group_rotation_info.group_for_core(core_index, n_cores); if let Some(g) = validator_groups.get(group_index.0 as usize) { if validator.as_ref().map_or(false, |v| g.contains(&v.index())) { - assignment = Some(core_para_id); + assignment = Some(Assignment { para_id: core_para_id, core_index }); } groups.insert(core_para_id, g.clone()); } @@ -1094,6 +1121,7 @@ async fn construct_per_relay_parent_state( Ok(Some(PerRelayParentState { prospective_parachains_mode: mode, parent, + session_index, assignment, backed: HashSet::new(), table: Table::new(table_config), @@ -1719,10 +1747,11 @@ async fn background_validate_and_make_available( >, ) -> Result<(), Error> { let candidate_hash = params.candidate.hash(); + let Some(Assignment { core_index, .. }) = rp_state.assignment else { return Ok(()) }; if rp_state.awaiting_validation.insert(candidate_hash) { // spawn background task. let bg = async move { - if let Err(e) = validate_and_make_available(params).await { + if let Err(e) = validate_and_make_available(params, core_index).await { if let Error::BackgroundValidationMpsc(error) = e { gum::debug!( target: LOG_TARGET, @@ -1795,6 +1824,7 @@ async fn kick_off_validation_work( tx_command: background_validation_tx.clone(), candidate: attesting.candidate, relay_parent: rp_state.parent, + session_index: rp_state.session_index, persisted_validation_data, pov, n_validators: rp_state.table_context.validators.len(), @@ -1859,8 +1889,10 @@ async fn maybe_validate_and_import( let candidate_hash = summary.candidate; - if Some(summary.group_id) != rp_state.assignment { - return Ok(()) + match rp_state.assignment { + Some(Assignment { para_id, .. }) if para_id != summary.group_id => return Ok(()), + None => return Ok(()), + _ => {}, } let attesting = match statement.payload() { StatementWithPVD::Seconded(receipt, _) => { @@ -1947,6 +1979,7 @@ async fn validate_and_second( tx_command: background_validation_tx.clone(), candidate: candidate.clone(), relay_parent: rp_state.parent, + session_index: rp_state.session_index, persisted_validation_data, pov: PoVData::Ready(pov), n_validators: rp_state.table_context.validators.len(), @@ -2004,15 +2037,19 @@ async fn handle_second_message( } // Sanity check that candidate is from our assignment. - if Some(candidate.descriptor().para_id) != rp_state.assignment { - gum::debug!( - target: LOG_TARGET, - our_assignment = ?rp_state.assignment, - collation = ?candidate.descriptor().para_id, - "Subsystem asked to second for para outside of our assignment", - ); + match rp_state.assignment { + Some(Assignment { para_id, .. }) if para_id != candidate.descriptor().para_id => { + gum::debug!( + target: LOG_TARGET, + our_assignment = ?para_id, + collation = ?candidate.descriptor().para_id, + "Subsystem asked to second for para outside of our assignment", + ); - return Ok(()) + return Ok(()) + }, + None => return Ok(()), + _ => {}, } // If the message is a `CandidateBackingMessage::Second`, sign and dispatch a diff --git a/polkadot/node/core/bitfield-signing/src/lib.rs b/polkadot/node/core/bitfield-signing/src/lib.rs index 47d0d65d3482..c58103757d53 100644 --- a/polkadot/node/core/bitfield-signing/src/lib.rs +++ b/polkadot/node/core/bitfield-signing/src/lib.rs @@ -29,20 +29,13 @@ use futures::{ use polkadot_node_subsystem::{ jaeger, messages::{AvailabilityStoreMessage, BitfieldDistributionMessage}, - overseer, ActivatedLeaf, ChainApiError, FromOrchestra, OverseerSignal, PerLeafSpan, - SpawnedSubsystem, SubsystemError, SubsystemResult, + overseer, ActivatedLeaf, FromOrchestra, OverseerSignal, PerLeafSpan, SpawnedSubsystem, + SubsystemError, SubsystemResult, }; use polkadot_node_subsystem_util::{ - self as util, - availability_chunks::availability_chunk_index, - get_block_number, request_availability_cores, request_session_index_for_child, - request_session_info, request_validators, - runtime::{recv_runtime, request_node_features}, - Validator, -}; -use polkadot_primitives::{ - vstaging::node_features, AvailabilityBitfield, CoreState, Hash, ValidatorIndex, + self as util, request_availability_cores, runtime::recv_runtime, Validator, }; +use polkadot_primitives::{AvailabilityBitfield, CoreState, Hash, ValidatorIndex}; use sp_keystore::{Error as KeystoreError, KeystorePtr}; use std::{collections::HashMap, iter::FromIterator, time::Duration}; use wasm_timer::{Delay, Instant}; @@ -79,104 +72,18 @@ pub enum Error { #[error("Keystore failed: {0:?}")] Keystore(KeystoreError), - - #[error("Cannot find block number for given relay parent")] - BlockNumberNotFound, - - #[error("Retrieving response from Chain API unexpectedly failed with error: {0}")] - ChainApi(#[from] ChainApiError), } /// If there is a candidate pending availability, query the Availability Store /// for whether we have the availability chunk for our validator index. async fn get_core_availability( core: &CoreState, - n_validators: usize, validator_index: ValidatorIndex, sender: &Mutex<&mut impl overseer::BitfieldSigningSenderTrait>, span: &jaeger::Span, ) -> Result { if let CoreState::Occupied(core) = core { let _span = span.child("query-chunk-availability"); - let relay_parent = core.candidate_descriptor.relay_parent; - - let block_number = get_block_number::<_, Error>(*sender.lock().await, relay_parent).await; - - let block_number = match block_number { - Ok(Some(block_number)) => block_number, - Ok(None) => { - gum::warn!( - target: LOG_TARGET, - ?relay_parent, - ?core.candidate_hash, - para_id = %core.para_id(), - "Failed to get block number." - ); - - return Ok(false) - }, - Err(err) => { - gum::warn!( - target: LOG_TARGET, - ?relay_parent, - ?core.candidate_hash, - para_id = %core.para_id(), - error = ?err, - "Failed to get block number." - ); - - return Ok(false) - }, - }; - - let session_index = - recv_runtime(request_session_index_for_child(relay_parent, *sender.lock().await).await) - .await?; - - let maybe_node_features = - request_node_features(relay_parent, session_index, *sender.lock().await) - .await - .map_err(Error::from)?; - - // Init this to all zeros. It won't be used unless - // `AvailabilityChunkShuffling` is enabled. We do this to avoid querying - // the runtime API for session index and session info unless the feature is enabled. - let mut babe_randomness = [0; 32]; - - if let Some(ref node_features) = maybe_node_features { - if let Some(&true) = node_features - .get(usize::from(node_features::FeatureIndex::AvailabilityChunkShuffling as u8)) - .as_deref() - { - let Some(session_info) = recv_runtime( - request_session_info(relay_parent, session_index, *sender.lock().await).await, - ) - .await? - else { - gum::warn!( - target: LOG_TARGET, - ?relay_parent, - session_index, - ?core.candidate_hash, - para_id = %core.para_id(), - "Failed to get session info." - ); - - return Ok(false) - }; - - babe_randomness = session_info.random_seed; - } - } - - let chunk_index = availability_chunk_index( - maybe_node_features.as_ref(), - babe_randomness, - n_validators, - block_number, - core.para_id(), - validator_index, - ); let (tx, rx) = oneshot::channel(); sender @@ -184,7 +91,7 @@ async fn get_core_availability( .await .send_message(AvailabilityStoreMessage::QueryChunkAvailability( core.candidate_hash, - chunk_index, + validator_index, tx, )) .await; @@ -212,7 +119,6 @@ async fn get_core_availability( async fn construct_availability_bitfield( relay_parent: Hash, span: &jaeger::Span, - n_validators: usize, validator_idx: ValidatorIndex, sender: &mut impl overseer::BitfieldSigningSenderTrait, ) -> Result { @@ -234,7 +140,7 @@ async fn construct_availability_bitfield( let results = future::try_join_all( availability_cores .iter() - .map(|core| get_core_availability(core, n_validators, validator_idx, &sender, span)), + .map(|core| get_core_availability(core, validator_idx, &sender, span)), ) .await?; @@ -332,20 +238,13 @@ where let span_delay = span.child("delay"); let wait_until = Instant::now() + SPAWNED_TASK_DELAY; - let validators = request_validators(leaf.hash, &mut sender) - .await - .await - .map_err(|e| Error::Runtime(e.into()))? - .map_err(|e| Error::Runtime(e.into()))?; - // now do all the work we can before we need to wait for the availability store // if we're not a validator, we can just succeed effortlessly - let validator = - match Validator::new(&validators, leaf.hash, keystore.clone(), &mut sender).await { - Ok(validator) => validator, - Err(util::Error::NotAValidator) => return Ok(()), - Err(err) => return Err(Error::Util(err)), - }; + let validator = match Validator::new(leaf.hash, keystore.clone(), &mut sender).await { + Ok(validator) => validator, + Err(util::Error::NotAValidator) => return Ok(()), + Err(err) => return Err(Error::Util(err)), + }; // wait a bit before doing anything else Delay::new_at(wait_until).await?; @@ -360,7 +259,6 @@ where let bitfield = match construct_availability_bitfield( leaf.hash, &span_availability, - validators.len(), validator.index(), &mut sender, ) diff --git a/polkadot/node/core/bitfield-signing/src/tests.rs b/polkadot/node/core/bitfield-signing/src/tests.rs index ef8f40525c67..0e61e6086d28 100644 --- a/polkadot/node/core/bitfield-signing/src/tests.rs +++ b/polkadot/node/core/bitfield-signing/src/tests.rs @@ -16,10 +16,8 @@ use super::*; use futures::{executor::block_on, pin_mut, StreamExt}; -use polkadot_node_subsystem::messages::{ - AllMessages, ChainApiMessage, RuntimeApiMessage, RuntimeApiRequest, -}; -use polkadot_primitives::{vstaging::NodeFeatures, CandidateHash, OccupiedCore}; +use polkadot_node_subsystem::messages::{AllMessages, RuntimeApiMessage, RuntimeApiRequest}; +use polkadot_primitives::{CandidateHash, OccupiedCore}; use test_helpers::dummy_candidate_descriptor; fn occupied_core(para_id: u32, candidate_hash: CandidateHash) -> CoreState { @@ -45,7 +43,6 @@ fn construct_availability_bitfield_works() { let future = construct_availability_bitfield( relay_parent, &jaeger::Span::Disabled, - 10, validator_index, &mut sender, ) @@ -64,28 +61,13 @@ fn construct_availability_bitfield_works() { assert_eq!(relay_parent, rp); tx.send(Ok(vec![CoreState::Free, occupied_core(1, hash_a), occupied_core(2, hash_b)])).unwrap(); } - AllMessages::RuntimeApi( - RuntimeApiMessage::Request(_, RuntimeApiRequest::NodeFeatures(_, tx)), - ) => { - tx.send(Ok(NodeFeatures::EMPTY)).unwrap(); - } - AllMessages::RuntimeApi( - RuntimeApiMessage::Request(_, RuntimeApiRequest::SessionIndexForChild(tx)), - ) => { - tx.send(Ok(1)).unwrap(); - } AllMessages::AvailabilityStore( - AvailabilityStoreMessage::QueryChunkAvailability(c_hash, cidx, tx), + AvailabilityStoreMessage::QueryChunkAvailability(c_hash, vidx, tx), ) => { - assert_eq!(validator_index, cidx.into()); + assert_eq!(validator_index, vidx.into()); tx.send(c_hash == hash_a).unwrap(); }, - AllMessages::ChainApi( - ChainApiMessage::BlockNumber(_, tx) - ) => { - tx.send(Ok(Some(1))).unwrap(); - } o => panic!("Unknown message: {:?}", o), }, r = future => match r { diff --git a/polkadot/node/jaeger/src/spans.rs b/polkadot/node/jaeger/src/spans.rs index 4038d41344f2..5ad991824a8e 100644 --- a/polkadot/node/jaeger/src/spans.rs +++ b/polkadot/node/jaeger/src/spans.rs @@ -85,7 +85,9 @@ use parity_scale_codec::Encode; use polkadot_node_primitives::PoV; -use polkadot_primitives::{BlakeTwo256, CandidateHash, Hash, HashT, Id as ParaId, ValidatorIndex}; +use polkadot_primitives::{ + BlakeTwo256, CandidateHash, ChunkIndex, Hash, HashT, Id as ParaId, ValidatorIndex, +}; use sc_network::PeerId; use std::{fmt, sync::Arc}; @@ -338,8 +340,8 @@ impl Span { } #[inline(always)] - pub fn with_chunk_index(self, chunk_index: u32) -> Self { - self.with_string_tag("chunk-index", chunk_index) + pub fn with_chunk_index(self, chunk_index: ChunkIndex) -> Self { + self.with_string_tag("chunk-index", &chunk_index.0) } #[inline(always)] diff --git a/polkadot/node/network/availability-distribution/Cargo.toml b/polkadot/node/network/availability-distribution/Cargo.toml index f31a4c29b4aa..737da16e2637 100644 --- a/polkadot/node/network/availability-distribution/Cargo.toml +++ b/polkadot/node/network/availability-distribution/Cargo.toml @@ -19,6 +19,7 @@ polkadot-node-network-protocol = { path = "../protocol" } polkadot-node-subsystem = { path = "../../subsystem" } polkadot-node-subsystem-util = { path = "../../subsystem-util" } polkadot-node-primitives = { path = "../../primitives" } +sc-network = { path = "../../../../substrate/client/network" } sp-core = { path = "../../../../substrate/primitives/core", features = ["std"] } sp-keystore = { path = "../../../../substrate/primitives/keystore" } thiserror = "1.0.48" diff --git a/polkadot/node/network/availability-distribution/src/error.rs b/polkadot/node/network/availability-distribution/src/error.rs index c0f0b5119cfb..72a809dd1140 100644 --- a/polkadot/node/network/availability-distribution/src/error.rs +++ b/polkadot/node/network/availability-distribution/src/error.rs @@ -83,8 +83,8 @@ pub enum Error { #[error("Given validator index could not be found in current session")] InvalidValidatorIndex, - #[error("Cannot find block number for given relay parent")] - BlockNumberNotFound, + #[error("Erasure coding error: {0}")] + ErasureCoding(#[from] polkadot_erasure_coding::Error), } /// General result abbreviation type alias. @@ -108,7 +108,7 @@ pub fn log_error( JfyiError::NoSuchCachedSession { .. } | JfyiError::QueryAvailableDataResponseChannel(_) | JfyiError::QueryChunkResponseChannel(_) | - JfyiError::BlockNumberNotFound => gum::warn!(target: LOG_TARGET, error = %jfyi, ctx), + JfyiError::ErasureCoding(_) => gum::warn!(target: LOG_TARGET, error = %jfyi, ctx), JfyiError::FetchPoV(_) | JfyiError::SendResponse | JfyiError::NoSuchPoV | diff --git a/polkadot/node/network/availability-distribution/src/lib.rs b/polkadot/node/network/availability-distribution/src/lib.rs index c62ce1dd981a..ec2c01f99b01 100644 --- a/polkadot/node/network/availability-distribution/src/lib.rs +++ b/polkadot/node/network/availability-distribution/src/lib.rs @@ -18,7 +18,9 @@ use futures::{future::Either, FutureExt, StreamExt, TryFutureExt}; use sp_keystore::KeystorePtr; -use polkadot_node_network_protocol::request_response::{v1, IncomingRequestReceiver}; +use polkadot_node_network_protocol::request_response::{ + v1, v2, IncomingRequestReceiver, ReqProtocolNames, +}; use polkadot_node_subsystem::{ jaeger, messages::AvailabilityDistributionMessage, overseer, FromOrchestra, OverseerSignal, SpawnedSubsystem, SubsystemError, @@ -41,7 +43,7 @@ mod pov_requester; /// Responding to erasure chunk requests: mod responder; -use responder::{run_chunk_receiver, run_pov_receiver}; +use responder::{run_chunk_receivers, run_pov_receiver}; mod metrics; /// Prometheus `Metrics` for availability distribution. @@ -58,6 +60,8 @@ pub struct AvailabilityDistributionSubsystem { runtime: RuntimeInfo, /// Receivers to receive messages from. recvs: IncomingRequestReceivers, + /// Mapping of the req-response protocols to the full protocol names. + req_protocol_names: ReqProtocolNames, /// Prometheus metrics. metrics: Metrics, } @@ -66,8 +70,10 @@ pub struct AvailabilityDistributionSubsystem { pub struct IncomingRequestReceivers { /// Receiver for incoming PoV requests. pub pov_req_receiver: IncomingRequestReceiver, - /// Receiver for incoming availability chunk requests. - pub chunk_req_receiver: IncomingRequestReceiver, + /// Receiver for incoming v1 availability chunk requests. + pub chunk_req_v1_receiver: IncomingRequestReceiver, + /// Receiver for incoming v2 availability chunk requests. + pub chunk_req_v2_receiver: IncomingRequestReceiver, } #[overseer::subsystem(AvailabilityDistribution, error=SubsystemError, prefix=self::overseer)] @@ -85,18 +91,27 @@ impl AvailabilityDistributionSubsystem { #[overseer::contextbounds(AvailabilityDistribution, prefix = self::overseer)] impl AvailabilityDistributionSubsystem { /// Create a new instance of the availability distribution. - pub fn new(keystore: KeystorePtr, recvs: IncomingRequestReceivers, metrics: Metrics) -> Self { + pub fn new( + keystore: KeystorePtr, + recvs: IncomingRequestReceivers, + req_protocol_names: ReqProtocolNames, + metrics: Metrics, + ) -> Self { let runtime = RuntimeInfo::new(Some(keystore)); - Self { runtime, recvs, metrics } + Self { runtime, recvs, req_protocol_names, metrics } } /// Start processing work as passed on from the Overseer. async fn run(self, mut ctx: Context) -> std::result::Result<(), FatalError> { - let Self { mut runtime, recvs, metrics } = self; + let Self { mut runtime, recvs, metrics, req_protocol_names } = self; let mut spans: HashMap = HashMap::new(); - let IncomingRequestReceivers { pov_req_receiver, chunk_req_receiver } = recvs; - let mut requester = Requester::new(metrics.clone()).fuse(); + let IncomingRequestReceivers { + pov_req_receiver, + chunk_req_v1_receiver, + chunk_req_v2_receiver, + } = recvs; + let mut requester = Requester::new(req_protocol_names, metrics.clone()).fuse(); let mut warn_freq = gum::Freq::new(); { @@ -109,7 +124,13 @@ impl AvailabilityDistributionSubsystem { ctx.spawn( "chunk-receiver", - run_chunk_receiver(sender, chunk_req_receiver, metrics.clone()).boxed(), + run_chunk_receivers( + sender, + chunk_req_v1_receiver, + chunk_req_v2_receiver, + metrics.clone(), + ) + .boxed(), ) .map_err(FatalError::SpawnTask)?; } diff --git a/polkadot/node/network/availability-distribution/src/requester/fetch_task/mod.rs b/polkadot/node/network/availability-distribution/src/requester/fetch_task/mod.rs index d8e13ccc67a0..c0068fde1702 100644 --- a/polkadot/node/network/availability-distribution/src/requester/fetch_task/mod.rs +++ b/polkadot/node/network/availability-distribution/src/requester/fetch_task/mod.rs @@ -22,10 +22,12 @@ use futures::{ FutureExt, SinkExt, }; +use parity_scale_codec::Decode; use polkadot_erasure_coding::branch_hash; use polkadot_node_network_protocol::request_response::{ outgoing::{OutgoingRequest, Recipient, RequestError, Requests}, - v1::{ChunkFetchingRequest, ChunkFetchingResponse}, + v1::{self, ChunkResponse}, + v2, }; use polkadot_node_primitives::ErasureChunk; use polkadot_node_subsystem::{ @@ -37,6 +39,7 @@ use polkadot_primitives::{ AuthorityDiscoveryId, BlakeTwo256, CandidateHash, ChunkIndex, GroupIndex, Hash, HashT, OccupiedCore, SessionIndex, }; +use sc_network::ProtocolName; use crate::{ error::{FatalError, Result}, @@ -111,8 +114,8 @@ struct RunningTask { /// This vector gets drained during execution of the task (it will be empty afterwards). group: Vec, - /// The request to send. - request: ChunkFetchingRequest, + /// The request to send. We can store it as either v1 or v2, they have the same payload. + request: v2::ChunkFetchingRequest, /// Root hash, for verifying the chunks validity. erasure_root: Hash, @@ -128,6 +131,16 @@ struct RunningTask { /// Span tracking the fetching of this chunk. span: jaeger::Span, + + /// Expected chunk index. We'll validate that the remote did send us the correct chunk (only + /// important for v2 requests). + chunk_index: ChunkIndex, + + /// Full protocol name for ChunkFetchingV1. + req_v1_protocol_name: ProtocolName, + + /// Full protocol name for ChunkFetchingV2. + req_v2_protocol_name: ProtocolName, } impl FetchTaskConfig { @@ -142,12 +155,15 @@ impl FetchTaskConfig { session_info: &SessionInfo, chunk_index: ChunkIndex, span: jaeger::Span, + req_v1_protocol_name: ProtocolName, + req_v2_protocol_name: ProtocolName, ) -> Self { let span = span .child("fetch-task-config") .with_trace_id(core.candidate_hash) .with_string_tag("leaf", format!("{:?}", leaf)) .with_validator_index(session_info.our_index) + .with_chunk_index(chunk_index) .with_uint_tag("group-index", core.group_responsible.0 as u64) .with_relay_parent(core.candidate_descriptor.relay_parent) .with_string_tag("pov-hash", format!("{:?}", core.candidate_descriptor.pov_hash)) @@ -166,15 +182,18 @@ impl FetchTaskConfig { group: session_info.validator_groups.get(core.group_responsible.0 as usize) .expect("The responsible group of a candidate should be available in the corresponding session. qed.") .clone(), - request: ChunkFetchingRequest { + request: v2::ChunkFetchingRequest { candidate_hash: core.candidate_hash, - index: chunk_index, + index: session_info.our_index, }, erasure_root: core.candidate_descriptor.erasure_root, relay_parent: core.candidate_descriptor.relay_parent, metrics, sender, span, + chunk_index, + req_v1_protocol_name, + req_v2_protocol_name }; FetchTaskConfig { live_in, prepared_running: Some(prepared_running) } } @@ -272,7 +291,8 @@ impl RunningTask { count += 1; let _chunk_fetch_span = span .child("fetch-chunk-request") - .with_chunk_index(self.request.index.0) + .with_validator_index(self.request.index) + .with_chunk_index(self.chunk_index) .with_stage(jaeger::Stage::AvailabilityDistribution); // Send request: let resp = match self @@ -297,11 +317,12 @@ impl RunningTask { drop(_chunk_fetch_span); let _chunk_recombine_span = span .child("recombine-chunk") - .with_chunk_index(self.request.index.0) + .with_validator_index(self.request.index) + .with_chunk_index(self.chunk_index) .with_stage(jaeger::Stage::AvailabilityDistribution); let chunk = match resp { - ChunkFetchingResponse::Chunk(resp) => resp.recombine_into_chunk(&self.request), - ChunkFetchingResponse::NoSuchChunk => { + Some(chunk) => chunk, + None => { gum::debug!( target: LOG_TARGET, validator = ?validator, @@ -321,11 +342,12 @@ impl RunningTask { drop(_chunk_recombine_span); let _chunk_validate_and_store_span = span .child("validate-and-store-chunk") - .with_chunk_index(self.request.index.0) + .with_validator_index(self.request.index) + .with_chunk_index(self.chunk_index) .with_stage(jaeger::Stage::AvailabilityDistribution); // Data genuine? - if !self.validate_chunk(&validator, &chunk) { + if !self.validate_chunk(&validator, &chunk, self.chunk_index) { bad_validators.push(validator); continue } @@ -351,7 +373,7 @@ impl RunningTask { validator: &AuthorityDiscoveryId, nerwork_error_freq: &mut gum::Freq, canceled_freq: &mut gum::Freq, - ) -> std::result::Result { + ) -> std::result::Result, TaskError> { gum::trace!( target: LOG_TARGET, origin = ?validator, @@ -363,9 +385,13 @@ impl RunningTask { "Starting chunk request", ); - let (full_request, response_recv) = - OutgoingRequest::new(Recipient::Authority(validator.clone()), self.request); - let requests = Requests::ChunkFetchingV1(full_request); + let (full_request, response_recv) = OutgoingRequest::new_with_fallback( + Recipient::Authority(validator.clone()), + self.request, + // Fallback to v1, for backwards compatibility. + v1::ChunkFetchingRequest::from(self.request), + ); + let requests = Requests::ChunkFetching(full_request); self.sender .send(FromFetchTask::Message( @@ -379,7 +405,58 @@ impl RunningTask { .map_err(|_| TaskError::ShuttingDown)?; match response_recv.await { - Ok(resp) => Ok(resp), + Ok((bytes, protocol)) => match protocol { + _ if protocol == self.req_v2_protocol_name => + match v2::ChunkFetchingResponse::decode(&mut &bytes[..]) { + Ok(chunk_response) => Ok(Option::::from(chunk_response)), + Err(e) => { + gum::warn!( + target: LOG_TARGET, + origin = ?validator, + relay_parent = ?self.relay_parent, + group_index = ?self.group_index, + session_index = ?self.session_index, + chunk_index = ?self.request.index, + candidate_hash = ?self.request.candidate_hash, + err = ?e, + "Peer sent us invalid erasure chunk data" + ); + Err(TaskError::PeerError) + }, + }, + _ if protocol == self.req_v1_protocol_name => + match v1::ChunkFetchingResponse::decode(&mut &bytes[..]) { + Ok(chunk_response) => Ok(Option::::from(chunk_response) + .map(|c| c.recombine_into_chunk(&self.request.into()))), + Err(e) => { + gum::warn!( + target: LOG_TARGET, + origin = ?validator, + relay_parent = ?self.relay_parent, + group_index = ?self.group_index, + session_index = ?self.session_index, + chunk_index = ?self.request.index, + candidate_hash = ?self.request.candidate_hash, + err = ?e, + "Peer sent us invalid erasure chunk data" + ); + Err(TaskError::PeerError) + }, + }, + _ => { + gum::warn!( + target: LOG_TARGET, + origin = ?validator, + relay_parent = ?self.relay_parent, + group_index = ?self.group_index, + session_index = ?self.session_index, + chunk_index = ?self.request.index, + candidate_hash = ?self.request.candidate_hash, + "Peer sent us invalid erasure chunk data - unknown protocol" + ); + Err(TaskError::PeerError) + }, + }, Err(RequestError::InvalidResponse(err)) => { gum::warn!( target: LOG_TARGET, @@ -428,7 +505,23 @@ impl RunningTask { } } - fn validate_chunk(&self, validator: &AuthorityDiscoveryId, chunk: &ErasureChunk) -> bool { + fn validate_chunk( + &self, + validator: &AuthorityDiscoveryId, + chunk: &ErasureChunk, + expected_chunk_index: ChunkIndex, + ) -> bool { + if chunk.index != expected_chunk_index { + gum::warn!( + target: LOG_TARGET, + candidate_hash = ?self.request.candidate_hash, + origin = ?validator, + chunk_index = ?chunk.index, + expected_chunk_index = ?expected_chunk_index, + "Validator sent the wrong chunk", + ); + return false + } let anticipated_hash = match branch_hash(&self.erasure_root, chunk.proof(), chunk.index.0 as usize) { Ok(hash) => hash, @@ -460,6 +553,7 @@ impl RunningTask { AvailabilityStoreMessage::StoreChunk { candidate_hash: self.request.candidate_hash, chunk, + validator_index: self.request.index, tx, } .into(), diff --git a/polkadot/node/network/availability-distribution/src/requester/fetch_task/tests.rs b/polkadot/node/network/availability-distribution/src/requester/fetch_task/tests.rs index b60d269f5788..25fae37f725a 100644 --- a/polkadot/node/network/availability-distribution/src/requester/fetch_task/tests.rs +++ b/polkadot/node/network/availability-distribution/src/requester/fetch_task/tests.rs @@ -24,21 +24,26 @@ use futures::{ task::{noop_waker, Context, Poll}, Future, FutureExt, StreamExt, }; +use rstest::rstest; use sc_network::{self as network, ProtocolName}; use sp_keyring::Sr25519Keyring; -use polkadot_node_network_protocol::request_response::{v1, Recipient}; +use polkadot_node_network_protocol::request_response::{ + v1::{self, ChunkResponse}, + Protocol, Recipient, ReqProtocolNames, +}; use polkadot_node_primitives::{BlockData, PoV, Proof}; use polkadot_node_subsystem::messages::AllMessages; -use polkadot_primitives::{CandidateHash, ChunkIndex}; +use polkadot_primitives::{CandidateHash, ChunkIndex, ValidatorIndex}; use super::*; use crate::{metrics::Metrics, tests::mock::get_valid_chunk_data}; #[test] fn task_can_be_canceled() { - let (task, _rx) = get_test_running_task(); + let req_protocol_names = ReqProtocolNames::new(&Hash::repeat_byte(0xff), None); + let (task, _rx) = get_test_running_task(&req_protocol_names, 0.into(), 0.into()); let (handle, kill) = oneshot::channel(); std::mem::drop(handle); let running_task = task.run(kill); @@ -49,92 +54,129 @@ fn task_can_be_canceled() { } /// Make sure task won't accept a chunk that has is invalid. -#[test] -fn task_does_not_accept_invalid_chunk() { - let (mut task, rx) = get_test_running_task(); +#[rstest] +#[case(Protocol::ChunkFetchingV1)] +#[case(Protocol::ChunkFetchingV2)] +fn task_does_not_accept_invalid_chunk(#[case] protocol: Protocol) { + let req_protocol_names = ReqProtocolNames::new(&Hash::repeat_byte(0xff), None); + let chunk_index = ChunkIndex(1); + let validator_index = ValidatorIndex(0); + let (mut task, rx) = get_test_running_task(&req_protocol_names, validator_index, chunk_index); let validators = vec![Sr25519Keyring::Alice.public().into()]; task.group = validators; + let protocol_name = req_protocol_names.get_name(protocol); let test = TestRun { chunk_responses: { - let mut m = HashMap::new(); - m.insert( + [( Recipient::Authority(Sr25519Keyring::Alice.public().into()), - ChunkFetchingResponse::Chunk(v1::ChunkResponse { - chunk: vec![1, 2, 3], - proof: Proof::try_from(vec![vec![9, 8, 2], vec![2, 3, 4]]).unwrap(), - }), - ); - m + get_response( + protocol, + protocol_name.clone(), + Some(( + vec![1, 2, 3], + Proof::try_from(vec![vec![9, 8, 2], vec![2, 3, 4]]).unwrap(), + chunk_index, + )), + ), + )] + .into_iter() + .collect() }, valid_chunks: HashSet::new(), + req_protocol_names, }; test.run(task, rx); } -#[test] -fn task_stores_valid_chunk() { - let (mut task, rx) = get_test_running_task(); +#[rstest] +#[case(Protocol::ChunkFetchingV1)] +#[case(Protocol::ChunkFetchingV2)] +fn task_stores_valid_chunk(#[case] protocol: Protocol) { + let req_protocol_names = ReqProtocolNames::new(&Hash::repeat_byte(0xff), None); + // In order for protocol version 1 to work, the chunk index needs to be equal to the validator + // index. + let chunk_index = ChunkIndex(0); + let validator_index = + if protocol == Protocol::ChunkFetchingV1 { ValidatorIndex(0) } else { ValidatorIndex(1) }; + let (mut task, rx) = get_test_running_task(&req_protocol_names, validator_index, chunk_index); let validators = vec![Sr25519Keyring::Alice.public().into()]; let pov = PoV { block_data: BlockData(vec![45, 46, 47]) }; - let (root_hash, chunk) = get_valid_chunk_data(pov, 10, ChunkIndex(0)); + let (root_hash, chunk) = get_valid_chunk_data(pov, 10, chunk_index); task.erasure_root = root_hash; - task.request.index = chunk.index.into(); - task.group = validators; + let protocol_name = req_protocol_names.get_name(protocol); let test = TestRun { chunk_responses: { - let mut m = HashMap::new(); - m.insert( + [( Recipient::Authority(Sr25519Keyring::Alice.public().into()), - ChunkFetchingResponse::Chunk(v1::ChunkResponse { - chunk: chunk.chunk.clone(), - proof: chunk.proof, - }), - ); - m - }, - valid_chunks: { - let mut s = HashSet::new(); - s.insert(chunk.chunk); - s + get_response( + protocol, + protocol_name.clone(), + Some((chunk.chunk.clone(), chunk.proof, chunk_index)), + ), + )] + .into_iter() + .collect() }, + valid_chunks: [(chunk.chunk)].into_iter().collect(), + req_protocol_names, }; test.run(task, rx); } -#[test] -fn task_does_not_accept_wrongly_indexed_chunk() { - let (mut task, rx) = get_test_running_task(); +#[rstest] +#[case(Protocol::ChunkFetchingV1)] +#[case(Protocol::ChunkFetchingV2)] +fn task_does_not_accept_wrongly_indexed_chunk(#[case] protocol: Protocol) { + let req_protocol_names = ReqProtocolNames::new(&Hash::repeat_byte(0xff), None); + // In order for protocol version 1 to work, the chunk index needs to be equal to the validator + // index. + let chunk_index = ChunkIndex(0); + let validator_index = + if protocol == Protocol::ChunkFetchingV1 { ValidatorIndex(0) } else { ValidatorIndex(1) }; + let (mut task, rx) = get_test_running_task(&req_protocol_names, validator_index, chunk_index); + let validators = vec![Sr25519Keyring::Alice.public().into()]; let pov = PoV { block_data: BlockData(vec![45, 46, 47]) }; + let (_, other_chunk) = get_valid_chunk_data(pov.clone(), 10, ChunkIndex(3)); let (root_hash, chunk) = get_valid_chunk_data(pov, 10, ChunkIndex(0)); task.erasure_root = root_hash; - task.request.index = ChunkIndex(chunk.index.0 + 1); - + task.request.index = chunk.index.into(); task.group = validators; + let protocol_name = req_protocol_names.get_name(protocol); let test = TestRun { chunk_responses: { - let mut m = HashMap::new(); - m.insert( + [( Recipient::Authority(Sr25519Keyring::Alice.public().into()), - ChunkFetchingResponse::Chunk(v1::ChunkResponse { - chunk: chunk.chunk.clone(), - proof: chunk.proof, - }), - ); - m + get_response( + protocol, + protocol_name.clone(), + Some((other_chunk.chunk.clone(), chunk.proof, other_chunk.index)), + ), + )] + .into_iter() + .collect() }, valid_chunks: HashSet::new(), + req_protocol_names, }; test.run(task, rx); } /// Task stores chunk, if there is at least one validator having a valid chunk. -#[test] -fn task_stores_valid_chunk_if_there_is_one() { - let (mut task, rx) = get_test_running_task(); +#[rstest] +#[case(Protocol::ChunkFetchingV1)] +#[case(Protocol::ChunkFetchingV2)] +fn task_stores_valid_chunk_if_there_is_one(#[case] protocol: Protocol) { + let req_protocol_names = ReqProtocolNames::new(&Hash::repeat_byte(0xff), None); + // In order for protocol version 1 to work, the chunk index needs to be equal to the validator + // index. + let chunk_index = ChunkIndex(1); + let validator_index = + if protocol == Protocol::ChunkFetchingV1 { ValidatorIndex(1) } else { ValidatorIndex(2) }; + let (mut task, rx) = get_test_running_task(&req_protocol_names, validator_index, chunk_index); let pov = PoV { block_data: BlockData(vec![45, 46, 47]) }; let validators = [ @@ -149,41 +191,44 @@ fn task_stores_valid_chunk_if_there_is_one() { .map(|v| v.public().into()) .collect::>(); - let (root_hash, chunk) = get_valid_chunk_data(pov, 10, ChunkIndex(0)); + let (root_hash, chunk) = get_valid_chunk_data(pov, 10, chunk_index); task.erasure_root = root_hash; - task.request.index = chunk.index.into(); - task.group = validators; + let protocol_name = req_protocol_names.get_name(protocol); let test = TestRun { chunk_responses: { - let mut m = HashMap::new(); - m.insert( - Recipient::Authority(Sr25519Keyring::Alice.public().into()), - ChunkFetchingResponse::Chunk(v1::ChunkResponse { - chunk: chunk.chunk.clone(), - proof: chunk.proof, - }), - ); - m.insert( - Recipient::Authority(Sr25519Keyring::Bob.public().into()), - ChunkFetchingResponse::NoSuchChunk, - ); - m.insert( - Recipient::Authority(Sr25519Keyring::Charlie.public().into()), - ChunkFetchingResponse::Chunk(v1::ChunkResponse { - chunk: vec![1, 2, 3], - proof: Proof::try_from(vec![vec![9, 8, 2], vec![2, 3, 4]]).unwrap(), - }), - ); - - m - }, - valid_chunks: { - let mut s = HashSet::new(); - s.insert(chunk.chunk); - s + [ + ( + Recipient::Authority(Sr25519Keyring::Alice.public().into()), + get_response( + protocol, + protocol_name.clone(), + Some((chunk.chunk.clone(), chunk.proof, chunk_index)), + ), + ), + ( + Recipient::Authority(Sr25519Keyring::Bob.public().into()), + get_response(protocol, protocol_name.clone(), None), + ), + ( + Recipient::Authority(Sr25519Keyring::Charlie.public().into()), + get_response( + protocol, + protocol_name.clone(), + Some(( + vec![1, 2, 3], + Proof::try_from(vec![vec![9, 8, 2], vec![2, 3, 4]]).unwrap(), + chunk_index, + )), + ), + ), + ] + .into_iter() + .collect() }, + valid_chunks: [(chunk.chunk)].into_iter().collect(), + req_protocol_names, }; test.run(task, rx); } @@ -191,14 +236,16 @@ fn task_stores_valid_chunk_if_there_is_one() { struct TestRun { /// Response to deliver for a given validator index. /// None means, answer with `NetworkError`. - chunk_responses: HashMap, + chunk_responses: HashMap, ProtocolName)>, /// Set of chunks that should be considered valid: valid_chunks: HashSet>, + /// Request protocol names + req_protocol_names: ReqProtocolNames, } impl TestRun { fn run(self, task: RunningTask, rx: mpsc::Receiver) { - sp_tracing::try_init_simple(); + sp_tracing::init_for_tests(); let mut rx = rx.fuse(); let task = task.run_inner().fuse(); futures::pin_mut!(task); @@ -242,20 +289,41 @@ impl TestRun { let mut valid_responses = 0; for req in reqs { let req = match req { - Requests::ChunkFetchingV1(req) => req, + Requests::ChunkFetching(req) => req, _ => panic!("Unexpected request"), }; let response = self.chunk_responses.get(&req.peer).ok_or(network::RequestFailure::Refused); - if let Ok(ChunkFetchingResponse::Chunk(resp)) = &response { - if self.valid_chunks.contains(&resp.chunk) { - valid_responses += 1; + if let Ok((resp, protocol)) = response { + let chunk = if protocol == + &self.req_protocol_names.get_name(Protocol::ChunkFetchingV1) + { + Into::>::into( + v1::ChunkFetchingResponse::decode(&mut &resp[..]).unwrap(), + ) + .map(|c| c.chunk) + } else if protocol == + &self.req_protocol_names.get_name(Protocol::ChunkFetchingV2) + { + Into::>::into( + v2::ChunkFetchingResponse::decode(&mut &resp[..]).unwrap(), + ) + .map(|c| c.chunk) + } else { + unreachable!() + }; + + if let Some(chunk) = chunk { + if self.valid_chunks.contains(&chunk) { + valid_responses += 1; + } } + + req.pending_response + .send(response.cloned()) + .expect("Sending response should succeed"); } - req.pending_response - .send(response.map(|r| (r.encode(), ProtocolName::from("")))) - .expect("Sending response should succeed"); } return (valid_responses == 0) && self.valid_chunks.is_empty() }, @@ -276,8 +344,12 @@ impl TestRun { } } -/// Get a `RunningTask` filled with dummy values. -fn get_test_running_task() -> (RunningTask, mpsc::Receiver) { +/// Get a `RunningTask` filled with (mostly) dummy values. +fn get_test_running_task( + req_protocol_names: &ReqProtocolNames, + validator_index: ValidatorIndex, + chunk_index: ChunkIndex, +) -> (RunningTask, mpsc::Receiver) { let (tx, rx) = mpsc::channel(0); ( @@ -285,16 +357,45 @@ fn get_test_running_task() -> (RunningTask, mpsc::Receiver) { session_index: 0, group_index: GroupIndex(0), group: Vec::new(), - request: ChunkFetchingRequest { + request: v2::ChunkFetchingRequest { candidate_hash: CandidateHash([43u8; 32].into()), - index: ChunkIndex(0), + index: validator_index, }, erasure_root: Hash::repeat_byte(99), relay_parent: Hash::repeat_byte(71), sender: tx, metrics: Metrics::new_dummy(), span: jaeger::Span::Disabled, + req_v1_protocol_name: req_protocol_names.get_name(Protocol::ChunkFetchingV1), + req_v2_protocol_name: req_protocol_names.get_name(Protocol::ChunkFetchingV2), + chunk_index, }, rx, ) } + +/// Make a versioned ChunkFetchingResponse. +fn get_response( + protocol: Protocol, + protocol_name: ProtocolName, + chunk: Option<(Vec, Proof, ChunkIndex)>, +) -> (Vec, ProtocolName) { + ( + match protocol { + Protocol::ChunkFetchingV1 => if let Some((chunk, proof, _)) = chunk { + v1::ChunkFetchingResponse::Chunk(ChunkResponse { chunk, proof }) + } else { + v1::ChunkFetchingResponse::NoSuchChunk + } + .encode(), + Protocol::ChunkFetchingV2 => if let Some((chunk, proof, index)) = chunk { + v2::ChunkFetchingResponse::Chunk(ErasureChunk { chunk, index, proof }) + } else { + v2::ChunkFetchingResponse::NoSuchChunk + } + .encode(), + _ => unreachable!(), + }, + protocol_name, + ) +} diff --git a/polkadot/node/network/availability-distribution/src/requester/mod.rs b/polkadot/node/network/availability-distribution/src/requester/mod.rs index 40b7b10e2937..efbdceb43bdd 100644 --- a/polkadot/node/network/availability-distribution/src/requester/mod.rs +++ b/polkadot/node/network/availability-distribution/src/requester/mod.rs @@ -29,19 +29,19 @@ use futures::{ Stream, }; +use polkadot_node_network_protocol::request_response::{v1, v2, IsRequest, ReqProtocolNames}; use polkadot_node_subsystem::{ jaeger, messages::{ChainApiMessage, RuntimeApiMessage}, overseer, ActivatedLeaf, ActiveLeavesUpdate, }; use polkadot_node_subsystem_util::{ - availability_chunks::ChunkIndexCacheRegistry, - get_block_number, - runtime::{get_occupied_cores, request_node_features, RuntimeInfo}, + availability_chunks::availability_chunk_index, + runtime::{get_occupied_cores, RuntimeInfo}, }; -use polkadot_primitives::{CandidateHash, Hash, OccupiedCore, SessionIndex}; +use polkadot_primitives::{CandidateHash, CoreIndex, Hash, OccupiedCore, SessionIndex}; -use super::{error::Error, FatalError, Metrics, Result, LOG_TARGET}; +use super::{FatalError, Metrics, Result, LOG_TARGET}; #[cfg(test)] mod tests; @@ -79,8 +79,8 @@ pub struct Requester { /// Prometheus Metrics metrics: Metrics, - /// Cache of our chunk indices based on the relay parent block and core index. - chunk_indices: ChunkIndexCacheRegistry, + /// Mapping of the req-response protocols to the full protocol names. + req_protocol_names: ReqProtocolNames, } #[overseer::contextbounds(AvailabilityDistribution, prefix = self::overseer)] @@ -92,7 +92,7 @@ impl Requester { /// /// You must feed it with `ActiveLeavesUpdate` via `update_fetching_heads` and make it progress /// by advancing the stream. - pub fn new(metrics: Metrics) -> Self { + pub fn new(req_protocol_names: ReqProtocolNames, metrics: Metrics) -> Self { let (tx, rx) = mpsc::channel(1); Requester { fetches: HashMap::new(), @@ -100,9 +100,7 @@ impl Requester { tx, rx, metrics, - // Candidates shouldn't be pending availability for many blocks, so keep our index for - // the last two relay parents. - chunk_indices: ChunkIndexCacheRegistry::new(2), + req_protocol_names, } } @@ -210,10 +208,10 @@ impl Requester { runtime: &mut RuntimeInfo, leaf: Hash, leaf_session_index: SessionIndex, - cores: impl IntoIterator, + cores: impl IntoIterator, span: jaeger::Span, ) -> Result<()> { - for core in cores { + for (core_index, core) in cores { let mut span = span .child("check-fetch-candidate") .with_trace_id(core.candidate_hash) @@ -229,12 +227,6 @@ impl Requester { span.add_string_tag("already-requested-chunk", "false"); let tx = self.tx.clone(); let metrics = self.metrics.clone(); - let block_number = get_block_number::<_, Error>( - context.sender(), - core.candidate_descriptor.relay_parent, - ) - .await? - .ok_or(Error::BlockNumberNotFound)?; let session_info = self .session_cache @@ -264,33 +256,12 @@ impl Requester { acc = acc.saturating_add(group.len()); acc }); - - let chunk_index = if let Some(chunk_index) = - self.chunk_indices.query_cache_for_validator( - block_number, - session_info.session_index, - core.para_id(), - session_info.our_index, - ) { - chunk_index - } else { - let maybe_node_features = request_node_features( - core.candidate_descriptor.relay_parent, - session_info.session_index, - context.sender(), - ) - .await?; - - self.chunk_indices.populate_for_validator( - maybe_node_features, - session_info.random_seed, - n_validators, - block_number, - session_info.session_index, - core.para_id(), - session_info.our_index, - ) - }; + let chunk_index = availability_chunk_index( + session_info.node_features.as_ref(), + n_validators, + core_index, + session_info.our_index, + )?; let task_cfg = FetchTaskConfig::new( leaf, @@ -300,6 +271,8 @@ impl Requester { session_info, chunk_index, span, + self.req_protocol_names.get_name(v1::ChunkFetchingRequest::PROTOCOL), + self.req_protocol_names.get_name(v2::ChunkFetchingRequest::PROTOCOL), ); self.fetches diff --git a/polkadot/node/network/availability-distribution/src/requester/session_cache.rs b/polkadot/node/network/availability-distribution/src/requester/session_cache.rs index 0bc5aaf63468..d241e9be29c0 100644 --- a/polkadot/node/network/availability-distribution/src/requester/session_cache.rs +++ b/polkadot/node/network/availability-distribution/src/requester/session_cache.rs @@ -20,8 +20,10 @@ use rand::{seq::SliceRandom, thread_rng}; use schnellru::{ByLength, LruMap}; use polkadot_node_subsystem::overseer; -use polkadot_node_subsystem_util::runtime::RuntimeInfo; -use polkadot_primitives::{AuthorityDiscoveryId, GroupIndex, Hash, SessionIndex, ValidatorIndex}; +use polkadot_node_subsystem_util::runtime::{request_node_features, RuntimeInfo}; +use polkadot_primitives::{ + vstaging::NodeFeatures, AuthorityDiscoveryId, GroupIndex, Hash, SessionIndex, ValidatorIndex, +}; use crate::{ error::{Error, Result}, @@ -63,8 +65,8 @@ pub struct SessionInfo { /// `None`, if we are not in fact part of any group. pub our_group: Option, - /// Per-session randomness gathered from BABE. - pub random_seed: [u8; 32], + /// Node features. + pub node_features: Option, } /// Report of bad validators. @@ -173,6 +175,9 @@ impl SessionCache { .get_session_info_by_index(ctx.sender(), relay_parent, session_index) .await?; + let node_features = + request_node_features(relay_parent, session_index, ctx.sender()).await?; + let discovery_keys = info.session_info.discovery_keys.clone(); let mut validator_groups = info.session_info.validator_groups.clone(); @@ -205,7 +210,7 @@ impl SessionCache { our_index, session_index, our_group, - random_seed: info.session_info.random_seed, + node_features, }; return Ok(Some(info)) } diff --git a/polkadot/node/network/availability-distribution/src/requester/tests.rs b/polkadot/node/network/availability-distribution/src/requester/tests.rs index 8e112de8666a..09567a8f87d3 100644 --- a/polkadot/node/network/availability-distribution/src/requester/tests.rs +++ b/polkadot/node/network/availability-distribution/src/requester/tests.rs @@ -17,7 +17,7 @@ use futures::FutureExt; use std::{collections::HashMap, future::Future}; -use polkadot_node_network_protocol::jaeger; +use polkadot_node_network_protocol::{jaeger, request_response::ReqProtocolNames}; use polkadot_node_primitives::{BlockData, ErasureChunk, PoV}; use polkadot_node_subsystem_util::runtime::RuntimeInfo; use polkadot_primitives::{ @@ -41,7 +41,7 @@ use polkadot_node_subsystem_test_helpers::{ use crate::tests::{ mock::{get_valid_chunk_data, make_session_info, OccupiedCoreBuilder}, - node_features_with_shuffling, + node_features_with_mapping_enabled, }; use super::Requester; @@ -124,7 +124,7 @@ fn spawn_virtual_overseer( .expect("Receiver should be alive."); }, RuntimeApiRequest::NodeFeatures(_, tx) => { - tx.send(Ok(node_features_with_shuffling())) + tx.send(Ok(node_features_with_mapping_enabled())) .expect("Receiver should be alive."); }, RuntimeApiRequest::AvailabilityCores(tx) => { @@ -176,15 +176,6 @@ fn spawn_virtual_overseer( .send(Ok(ancestors)) .expect("Receiver is expected to be alive"); }, - AllMessages::ChainApi(ChainApiMessage::BlockNumber(hash, response_channel)) => { - response_channel - .send(Ok(test_state - .relay_chain - .iter() - .position(|h| *h == hash) - .map(|pos| pos as u32))) - .expect("Receiver is expected to be alive"); - }, msg => panic!("Unexpected overseer message: {:?}", msg), } } @@ -210,7 +201,8 @@ fn test_harness>( #[test] fn check_ancestry_lookup_in_same_session() { let test_state = TestState::new(); - let mut requester = Requester::new(Default::default()); + let mut requester = + Requester::new(ReqProtocolNames::new(&Hash::repeat_byte(0xff), None), Default::default()); let keystore = make_ferdie_keystore(); let mut runtime = RuntimeInfo::new(Some(keystore)); @@ -277,7 +269,8 @@ fn check_ancestry_lookup_in_same_session() { #[test] fn check_ancestry_lookup_in_different_sessions() { let mut test_state = TestState::new(); - let mut requester = Requester::new(Default::default()); + let mut requester = + Requester::new(ReqProtocolNames::new(&Hash::repeat_byte(0xff), None), Default::default()); let keystore = make_ferdie_keystore(); let mut runtime = RuntimeInfo::new(Some(keystore)); diff --git a/polkadot/node/network/availability-distribution/src/responder.rs b/polkadot/node/network/availability-distribution/src/responder.rs index 5d768f84a567..2c1885d27727 100644 --- a/polkadot/node/network/availability-distribution/src/responder.rs +++ b/polkadot/node/network/availability-distribution/src/responder.rs @@ -18,16 +18,17 @@ use std::sync::Arc; -use futures::channel::oneshot; +use futures::{channel::oneshot, select, FutureExt}; use fatality::Nested; +use parity_scale_codec::{Decode, Encode}; use polkadot_node_network_protocol::{ - request_response::{v1, IncomingRequest, IncomingRequestReceiver}, + request_response::{v1, v2, IncomingRequest, IncomingRequestReceiver, IsRequest}, UnifiedReputationChange as Rep, }; use polkadot_node_primitives::{AvailableData, ErasureChunk}; use polkadot_node_subsystem::{jaeger, messages::AvailabilityStoreMessage, SubsystemSender}; -use polkadot_primitives::{CandidateHash, ChunkIndex}; +use polkadot_primitives::{CandidateHash, ValidatorIndex}; use crate::{ error::{JfyiError, Result}, @@ -66,33 +67,66 @@ pub async fn run_pov_receiver( } /// Receiver task to be forked as a separate task to handle chunk requests. -pub async fn run_chunk_receiver( +pub async fn run_chunk_receivers( mut sender: Sender, - mut receiver: IncomingRequestReceiver, + mut receiver_v1: IncomingRequestReceiver, + mut receiver_v2: IncomingRequestReceiver, metrics: Metrics, ) where Sender: SubsystemSender, { + let make_resp_v1 = |chunk: Option| match chunk { + None => v1::ChunkFetchingResponse::NoSuchChunk, + Some(chunk) => v1::ChunkFetchingResponse::Chunk(chunk.into()), + }; + + let make_resp_v2 = |chunk: Option| match chunk { + None => v2::ChunkFetchingResponse::NoSuchChunk, + Some(chunk) => v2::ChunkFetchingResponse::Chunk(chunk.into()), + }; + loop { - match receiver.recv(|| vec![COST_INVALID_REQUEST]).await.into_nested() { - Ok(Ok(msg)) => { - answer_chunk_request_log(&mut sender, msg, &metrics).await; - }, - Err(fatal) => { - gum::debug!( - target: LOG_TARGET, - error = ?fatal, - "Shutting down chunk receiver." - ); - return - }, - Ok(Err(jfyi)) => { - gum::debug!( - target: LOG_TARGET, - error = ?jfyi, - "Error decoding incoming chunk request." - ); + select! { + res = receiver_v1.recv(|| vec![COST_INVALID_REQUEST]).fuse() => match res.into_nested() { + Ok(Ok(msg)) => { + answer_chunk_request_log(&mut sender, msg, make_resp_v1, &metrics).await; + }, + Err(fatal) => { + gum::debug!( + target: LOG_TARGET, + error = ?fatal, + "Shutting down chunk receiver." + ); + return + }, + Ok(Err(jfyi)) => { + gum::debug!( + target: LOG_TARGET, + error = ?jfyi, + "Error decoding incoming chunk request." + ); + } }, + res = receiver_v2.recv(|| vec![COST_INVALID_REQUEST]).fuse() => match res.into_nested() { + Ok(Ok(msg)) => { + answer_chunk_request_log(&mut sender, msg.into(), make_resp_v2, &metrics).await; + }, + Err(fatal) => { + gum::debug!( + target: LOG_TARGET, + error = ?fatal, + "Shutting down chunk receiver." + ); + return + }, + Ok(Err(jfyi)) => { + gum::debug!( + target: LOG_TARGET, + error = ?jfyi, + "Error decoding incoming chunk request." + ); + } + } } } } @@ -124,15 +158,18 @@ pub async fn answer_pov_request_log( /// Variant of `answer_chunk_request` that does Prometheus metric and logging on errors. /// /// Any errors of `answer_request` will simply be logged. -pub async fn answer_chunk_request_log( +pub async fn answer_chunk_request_log( sender: &mut Sender, - req: IncomingRequest, + req: IncomingRequest, + make_response: MakeResp, metrics: &Metrics, -) -> () -where +) where + Req: IsRequest + Decode + Encode + Into, + Req::Response: Encode, Sender: SubsystemSender, + MakeResp: Fn(Option) -> Req::Response, { - let res = answer_chunk_request(sender, req).await; + let res = answer_chunk_request(sender, req, make_response).await; match res { Ok(result) => metrics.on_served_chunk(if result { SUCCEEDED } else { NOT_FOUND }), Err(err) => { @@ -177,39 +214,46 @@ where /// Answer an incoming chunk request by querying the av store. /// /// Returns: `Ok(true)` if chunk was found and served. -pub async fn answer_chunk_request( +pub async fn answer_chunk_request( sender: &mut Sender, - req: IncomingRequest, + req: IncomingRequest, + make_response: MakeResp, ) -> Result where Sender: SubsystemSender, + Req: IsRequest + Decode + Encode + Into, + Req::Response: Encode, + MakeResp: Fn(Option) -> Req::Response, { - let span = jaeger::Span::new(req.payload.candidate_hash, "answer-chunk-request"); + // V1 and V2 requests have the same payload, so decoding into either one will work. It's the + // responses that differ, hence the `MakeResp` generic. + let payload: v1::ChunkFetchingRequest = req.payload.into(); + let span = jaeger::Span::new(payload.candidate_hash, "answer-chunk-request"); let _child_span = span .child("answer-chunk-request") - .with_trace_id(req.payload.candidate_hash) - .with_chunk_index(req.payload.index.0); + .with_trace_id(payload.candidate_hash) + .with_validator_index(payload.index); - let chunk = query_chunk(sender, req.payload.candidate_hash, req.payload.index).await?; + let chunk = query_chunk(sender, payload.candidate_hash, payload.index).await?; let result = chunk.is_some(); gum::trace!( target: LOG_TARGET, - hash = ?req.payload.candidate_hash, - index = ?req.payload.index, + hash = ?payload.candidate_hash, + index = ?payload.index, peer = ?req.peer, has_data = ?chunk.is_some(), "Serving chunk", ); - let response = match chunk { - None => v1::ChunkFetchingResponse::NoSuchChunk, - Some(chunk) => v1::ChunkFetchingResponse::Chunk(chunk.into()), - }; + let response = make_response(chunk); + + req.pending_response + .send_response(response) + .map_err(|_| JfyiError::SendResponse)?; - req.send_response(response).map_err(|_| JfyiError::SendResponse)?; Ok(result) } @@ -217,20 +261,22 @@ where async fn query_chunk( sender: &mut Sender, candidate_hash: CandidateHash, - chunk_index: ChunkIndex, + validator_index: ValidatorIndex, ) -> std::result::Result, JfyiError> where Sender: SubsystemSender, { let (tx, rx) = oneshot::channel(); sender - .send_message(AvailabilityStoreMessage::QueryChunk(candidate_hash, chunk_index, tx).into()) + .send_message( + AvailabilityStoreMessage::QueryChunk(candidate_hash, validator_index, tx).into(), + ) .await; let result = rx.await.map_err(|e| { gum::trace!( target: LOG_TARGET, - ?chunk_index, + ?validator_index, ?candidate_hash, error = ?e, "Error retrieving chunk", diff --git a/polkadot/node/network/availability-distribution/src/tests/mod.rs b/polkadot/node/network/availability-distribution/src/tests/mod.rs index 69c2e8c1978d..7c8321237089 100644 --- a/polkadot/node/network/availability-distribution/src/tests/mod.rs +++ b/polkadot/node/network/availability-distribution/src/tests/mod.rs @@ -19,7 +19,9 @@ use std::collections::HashSet; use futures::{executor, future, Future}; use rstest::rstest; -use polkadot_node_network_protocol::request_response::{IncomingRequest, ReqProtocolNames}; +use polkadot_node_network_protocol::request_response::{ + IncomingRequest, Protocol, ReqProtocolNames, +}; use polkadot_primitives::{ vstaging::{node_features, NodeFeatures}, CoreState, Hash, @@ -39,38 +41,45 @@ pub(crate) mod mock; fn test_harness>( keystore: KeystorePtr, + req_protocol_names: ReqProtocolNames, test_fx: impl FnOnce(TestHarness) -> T, -) { - sp_tracing::try_init_simple(); +) -> std::result::Result<(), FatalError> { + sp_tracing::init_for_tests(); let pool = sp_core::testing::TaskExecutor::new(); let (context, virtual_overseer) = test_helpers::make_subsystem_context(pool.clone()); - let genesis_hash = Hash::repeat_byte(0xff); - let req_protocol_names = ReqProtocolNames::new(&genesis_hash, None); let (pov_req_receiver, pov_req_cfg) = IncomingRequest::get_config_receiver(&req_protocol_names); - let (chunk_req_receiver, chunk_req_cfg) = + let (chunk_req_v1_receiver, chunk_req_v1_cfg) = + IncomingRequest::get_config_receiver(&req_protocol_names); + let (chunk_req_v2_receiver, chunk_req_v2_cfg) = IncomingRequest::get_config_receiver(&req_protocol_names); let subsystem = AvailabilityDistributionSubsystem::new( keystore, - IncomingRequestReceivers { pov_req_receiver, chunk_req_receiver }, + IncomingRequestReceivers { pov_req_receiver, chunk_req_v1_receiver, chunk_req_v2_receiver }, + req_protocol_names, Default::default(), ); let subsystem = subsystem.run(context); - let test_fut = test_fx(TestHarness { virtual_overseer, pov_req_cfg, chunk_req_cfg, pool }); + let test_fut = test_fx(TestHarness { + virtual_overseer, + pov_req_cfg, + chunk_req_v1_cfg, + chunk_req_v2_cfg, + pool, + }); futures::pin_mut!(test_fut); futures::pin_mut!(subsystem); - executor::block_on(future::join(test_fut, subsystem)).1.unwrap(); + executor::block_on(future::join(test_fut, subsystem)).1 } -pub fn node_features_with_shuffling() -> NodeFeatures { +pub fn node_features_with_mapping_enabled() -> NodeFeatures { let mut node_features = NodeFeatures::new(); - node_features - .resize(node_features::FeatureIndex::AvailabilityChunkShuffling as usize + 1, false); - node_features.set(node_features::FeatureIndex::AvailabilityChunkShuffling as u8 as usize, true); + node_features.resize(node_features::FeatureIndex::AvailabilityChunkMapping as usize + 1, false); + node_features.set(node_features::FeatureIndex::AvailabilityChunkMapping as u8 as usize, true); node_features } @@ -78,26 +87,61 @@ pub fn node_features_with_shuffling() -> NodeFeatures { /// /// Exceptional cases are tested as unit tests in `fetch_task`. #[rstest] -#[case(NodeFeatures::EMPTY)] -#[case(node_features_with_shuffling())] -fn check_basic(#[case] node_features: NodeFeatures) { - let state = TestState::new(node_features); - test_harness(state.keystore.clone(), move |harness| state.run(harness)); +#[case(NodeFeatures::EMPTY, Protocol::ChunkFetchingV1)] +#[case(NodeFeatures::EMPTY, Protocol::ChunkFetchingV2)] +#[case(node_features_with_mapping_enabled(), Protocol::ChunkFetchingV1)] +#[case(node_features_with_mapping_enabled(), Protocol::ChunkFetchingV2)] +fn check_basic(#[case] node_features: NodeFeatures, #[case] chunk_resp_protocol: Protocol) { + let req_protocol_names = ReqProtocolNames::new(&Hash::repeat_byte(0xff), None); + let state = + TestState::new(node_features.clone(), req_protocol_names.clone(), chunk_resp_protocol); + + if node_features == node_features_with_mapping_enabled() && + chunk_resp_protocol == Protocol::ChunkFetchingV1 + { + // For this specific case, chunk fetching is not possible, because the ValidatorIndex is not + // equal to the ChunkIndex and the peer does not send back the actual ChunkIndex. + let _ = test_harness(state.keystore.clone(), req_protocol_names, move |harness| { + state.run_assert_timeout(harness) + }); + } else { + test_harness(state.keystore.clone(), req_protocol_names, move |harness| state.run(harness)) + .unwrap(); + } } /// Check whether requester tries all validators in group. #[rstest] -#[case(NodeFeatures::EMPTY)] -#[case(node_features_with_shuffling())] -fn check_fetch_tries_all(#[case] node_features: NodeFeatures) { - let mut state = TestState::new(node_features); +#[case(NodeFeatures::EMPTY, Protocol::ChunkFetchingV1)] +#[case(NodeFeatures::EMPTY, Protocol::ChunkFetchingV2)] +#[case(node_features_with_mapping_enabled(), Protocol::ChunkFetchingV1)] +#[case(node_features_with_mapping_enabled(), Protocol::ChunkFetchingV2)] +fn check_fetch_tries_all( + #[case] node_features: NodeFeatures, + #[case] chunk_resp_protocol: Protocol, +) { + let req_protocol_names = ReqProtocolNames::new(&Hash::repeat_byte(0xff), None); + let mut state = + TestState::new(node_features.clone(), req_protocol_names.clone(), chunk_resp_protocol); for (_, v) in state.chunks.iter_mut() { // 4 validators in group, so this should still succeed: v.push(None); v.push(None); v.push(None); } - test_harness(state.keystore.clone(), move |harness| state.run(harness)); + + if node_features == node_features_with_mapping_enabled() && + chunk_resp_protocol == Protocol::ChunkFetchingV1 + { + // For this specific case, chunk fetching is not possible, because the ValidatorIndex is not + // equal to the ChunkIndex and the peer does not send back the actual ChunkIndex. + let _ = test_harness(state.keystore.clone(), req_protocol_names, move |harness| { + state.run_assert_timeout(harness) + }); + } else { + test_harness(state.keystore.clone(), req_protocol_names, move |harness| state.run(harness)) + .unwrap(); + } } /// Check whether requester tries all validators in group @@ -105,10 +149,14 @@ fn check_fetch_tries_all(#[case] node_features: NodeFeatures) { /// Check that requester will retry the fetch on error on the next block still pending /// availability. #[rstest] -#[case(NodeFeatures::EMPTY)] -#[case(node_features_with_shuffling())] -fn check_fetch_retry(#[case] node_features: NodeFeatures) { - let mut state = TestState::new(node_features); +#[case(NodeFeatures::EMPTY, Protocol::ChunkFetchingV1)] +#[case(NodeFeatures::EMPTY, Protocol::ChunkFetchingV2)] +#[case(node_features_with_mapping_enabled(), Protocol::ChunkFetchingV1)] +#[case(node_features_with_mapping_enabled(), Protocol::ChunkFetchingV2)] +fn check_fetch_retry(#[case] node_features: NodeFeatures, #[case] chunk_resp_protocol: Protocol) { + let req_protocol_names = ReqProtocolNames::new(&Hash::repeat_byte(0xff), None); + let mut state = + TestState::new(node_features.clone(), req_protocol_names.clone(), chunk_resp_protocol); state .cores .insert(state.relay_chain[2], state.cores.get(&state.relay_chain[1]).unwrap().clone()); @@ -139,5 +187,17 @@ fn check_fetch_retry(#[case] node_features: NodeFeatures) { v.push(None); v.push(None); } - test_harness(state.keystore.clone(), move |harness| state.run(harness)); + + if node_features == node_features_with_mapping_enabled() && + chunk_resp_protocol == Protocol::ChunkFetchingV1 + { + // For this specific case, chunk fetching is not possible, because the ValidatorIndex is not + // equal to the ChunkIndex and the peer does not send back the actual ChunkIndex. + let _ = test_harness(state.keystore.clone(), req_protocol_names, move |harness| { + state.run_assert_timeout(harness) + }); + } else { + test_harness(state.keystore.clone(), req_protocol_names, move |harness| state.run(harness)) + .unwrap(); + } } diff --git a/polkadot/node/network/availability-distribution/src/tests/state.rs b/polkadot/node/network/availability-distribution/src/tests/state.rs index 0fa5f5b8998f..b35af1c1ce1f 100644 --- a/polkadot/node/network/availability-distribution/src/tests/state.rs +++ b/polkadot/node/network/availability-distribution/src/tests/state.rs @@ -19,7 +19,7 @@ use std::{ time::Duration, }; -use network::ProtocolName; +use network::{request_responses::OutgoingResponse, ProtocolName, RequestFailure}; use polkadot_node_subsystem_test_helpers::TestSubsystemContextHandle; use polkadot_node_subsystem_util::{availability_chunks::availability_chunk_index, TimeoutExt}; @@ -35,7 +35,7 @@ use sp_core::{testing::TaskExecutor, traits::SpawnNamed}; use sp_keystore::KeystorePtr; use polkadot_node_network_protocol::request_response::{ - v1, IncomingRequest, OutgoingRequest, Requests, + v1, v2, IncomingRequest, OutgoingRequest, Protocol, ReqProtocolNames, Requests, }; use polkadot_node_primitives::ErasureChunk; use polkadot_node_subsystem::{ @@ -47,8 +47,8 @@ use polkadot_node_subsystem::{ }; use polkadot_node_subsystem_test_helpers as test_helpers; use polkadot_primitives::{ - vstaging::NodeFeatures, CandidateHash, ChunkIndex, CoreState, ExecutorParams, GroupIndex, Hash, - Id as ParaId, ScheduledCore, SessionInfo, ValidatorIndex, + vstaging::NodeFeatures, CandidateHash, ChunkIndex, CoreIndex, CoreState, ExecutorParams, + GroupIndex, Hash, Id as ParaId, ScheduledCore, SessionInfo, ValidatorIndex, }; use test_helpers::mock::{make_ferdie_keystore, new_leaf}; @@ -59,7 +59,8 @@ type VirtualOverseer = test_helpers::TestSubsystemContextHandle>>, + pub chunks: HashMap<(CandidateHash, ValidatorIndex), Vec>>, /// All chunks that are valid and should be accepted. - pub valid_chunks: HashSet<(CandidateHash, ChunkIndex)>, + pub valid_chunks: HashSet<(CandidateHash, ValidatorIndex)>, pub session_info: SessionInfo, /// Cores per relay chain block. pub cores: HashMap>, pub keystore: KeystorePtr, pub node_features: NodeFeatures, + pub chunk_response_protocol: Protocol, + pub req_protocol_names: ReqProtocolNames, + pub our_chunk_index: ChunkIndex, } impl TestState { /// Initialize a default test state. - pub fn new(node_features: NodeFeatures) -> Self { + pub fn new( + node_features: NodeFeatures, + req_protocol_names: ReqProtocolNames, + chunk_response_protocol: Protocol, + ) -> Self { let relay_chain: Vec<_> = (1u8..10).map(Hash::repeat_byte).collect(); let chain_a = ParaId::from(1); let chain_b = ParaId::from(2); @@ -99,6 +107,14 @@ impl TestState { let session_info = make_session_info(); + let our_chunk_index = availability_chunk_index( + Some(&node_features), + session_info.validators.len(), + CoreIndex(1), + ValidatorIndex(0), + ) + .unwrap(); + let (cores, chunks) = { let mut cores = HashMap::new(); let mut chunks = HashMap::new(); @@ -116,17 +132,7 @@ impl TestState { advanced.next(); relay_chain.iter().zip(advanced) }; - for (block_number, (relay_parent, relay_child)) in heads.enumerate() { - let our_chunk_index = availability_chunk_index( - Some(&node_features), - session_info.random_seed, - session_info.validators.len(), - block_number as u32, - // Use the second para id, as we're in the first group and hold the entire POV. - 2.into(), - ValidatorIndex(0), - ); - + for (relay_parent, relay_child) in heads { let (p_cores, p_chunks): (Vec<_>, Vec<_>) = chain_ids .iter() .enumerate() @@ -147,7 +153,7 @@ impl TestState { let mut chunks_other_groups = p_chunks.into_iter(); chunks_other_groups.next(); for (candidate, chunk) in chunks_other_groups { - chunks.insert((candidate, chunk.index), vec![Some(chunk)]); + chunks.insert((candidate, ValidatorIndex(0)), vec![Some(chunk)]); } } (cores, chunks) @@ -160,16 +166,26 @@ impl TestState { cores, keystore, node_features, + chunk_response_protocol, + req_protocol_names, + our_chunk_index, } } /// Run, but fail after some timeout. pub async fn run(self, harness: TestHarness) { // Make sure test won't run forever. - let f = self.run_inner(harness).timeout(Duration::from_secs(10)); + let f = self.run_inner(harness).timeout(Duration::from_secs(5)); assert!(f.await.is_some(), "Test ran into timeout"); } + /// Run, and assert an expected timeout. + pub async fn run_assert_timeout(self, harness: TestHarness) { + // Make sure test won't run forever. + let f = self.run_inner(harness).timeout(Duration::from_secs(5)); + assert!(f.await.is_none(), "Test should have run into timeout"); + } + /// Run tests with the given mock values in `TestState`. /// /// This will simply advance through the simulated chain and examines whether the subsystem @@ -227,25 +243,51 @@ impl TestState { )) => { for req in reqs { // Forward requests: - let in_req = to_incoming_req(&harness.pool, req); - harness - .chunk_req_cfg - .inbound_queue - .as_mut() - .unwrap() - .send(in_req.into_raw()) - .await - .unwrap(); + match self.chunk_response_protocol { + Protocol::ChunkFetchingV1 => { + let in_req = to_incoming_req_v1( + &harness.pool, + req, + self.req_protocol_names.get_name(Protocol::ChunkFetchingV1), + ); + + harness + .chunk_req_v1_cfg + .inbound_queue + .as_mut() + .unwrap() + .send(in_req.into_raw()) + .await + .unwrap(); + }, + Protocol::ChunkFetchingV2 => { + let in_req = to_incoming_req_v2( + &harness.pool, + req, + self.req_protocol_names.get_name(Protocol::ChunkFetchingV2), + ); + + harness + .chunk_req_v2_cfg + .inbound_queue + .as_mut() + .unwrap() + .send(in_req.into_raw()) + .await + .unwrap(); + }, + _ => panic!("Unexpected protocol"), + } } }, AllMessages::AvailabilityStore(AvailabilityStoreMessage::QueryChunk( candidate_hash, - chunk_index, + validator_index, tx, )) => { let chunk = self .chunks - .get_mut(&(candidate_hash, chunk_index)) + .get_mut(&(candidate_hash, validator_index)) .and_then(Vec::pop) .flatten(); tx.send(chunk).expect("Receiver is expected to be alive"); @@ -253,13 +295,16 @@ impl TestState { AllMessages::AvailabilityStore(AvailabilityStoreMessage::StoreChunk { candidate_hash, chunk, + validator_index, tx, .. }) => { assert!( - self.valid_chunks.contains(&(candidate_hash, chunk.index)), + self.valid_chunks.contains(&(candidate_hash, validator_index)), "Only valid chunks should ever get stored." ); + assert_eq!(self.our_chunk_index, chunk.index); + tx.send(Ok(())).expect("Receiver is expected to be alive"); gum::trace!(target: LOG_TARGET, "'Stored' fetched chunk."); remaining_stores -= 1; @@ -300,15 +345,6 @@ impl TestState { .unwrap_or_default(); response_channel.send(Ok(ancestors)).expect("Receiver is expected to be alive"); }, - AllMessages::ChainApi(ChainApiMessage::BlockNumber(hash, response_channel)) => { - response_channel - .send(Ok(self - .relay_chain - .iter() - .position(|h| *h == hash) - .map(|pos| pos as u32))) - .expect("Receiver is expected to be alive"); - }, _ => { panic!("Received unexpected message") @@ -336,30 +372,47 @@ async fn overseer_recv(rx: &mut mpsc::UnboundedReceiver) -> AllMess rx.next().await.expect("Test subsystem no longer live") } -fn to_incoming_req( +fn to_incoming_req_v1( executor: &TaskExecutor, outgoing: Requests, + protocol_name: ProtocolName, ) -> IncomingRequest { match outgoing { - Requests::ChunkFetchingV1(OutgoingRequest { payload, pending_response, .. }) => { - let (tx, rx): (oneshot::Sender, oneshot::Receiver<_>) = - oneshot::channel(); - executor.spawn( - "message-forwarding", - None, - async { - let response = rx.await; - let payload = response.expect("Unexpected canceled request").result; - pending_response - .send( - payload - .map_err(|_| network::RequestFailure::Refused) - .map(|r| (r, ProtocolName::from(""))), - ) - .expect("Sending response is expected to work"); - } - .boxed(), - ); + Requests::ChunkFetching(OutgoingRequest { + pending_response, + fallback_request: Some((fallback_request, fallback_protocol)), + .. + }) => { + assert_eq!(fallback_protocol, Protocol::ChunkFetchingV1); + + let tx = spawn_message_forwarding(executor, protocol_name, pending_response); + + IncomingRequest::new( + // We don't really care: + network::PeerId::random(), + fallback_request, + tx, + ) + }, + _ => panic!("Unexpected request!"), + } +} + +fn to_incoming_req_v2( + executor: &TaskExecutor, + outgoing: Requests, + protocol_name: ProtocolName, +) -> IncomingRequest { + match outgoing { + Requests::ChunkFetching(OutgoingRequest { + payload, + pending_response, + fallback_request: Some((_, fallback_protocol)), + .. + }) => { + assert_eq!(fallback_protocol, Protocol::ChunkFetchingV1); + + let tx = spawn_message_forwarding(executor, protocol_name, pending_response); IncomingRequest::new( // We don't really care: @@ -371,3 +424,26 @@ fn to_incoming_req( _ => panic!("Unexpected request!"), } } + +fn spawn_message_forwarding( + executor: &TaskExecutor, + protocol_name: ProtocolName, + pending_response: oneshot::Sender, ProtocolName), RequestFailure>>, +) -> oneshot::Sender { + let (tx, rx): (oneshot::Sender, oneshot::Receiver<_>) = + oneshot::channel(); + executor.spawn( + "message-forwarding", + None, + async { + let response = rx.await; + let payload = response.expect("Unexpected canceled request").result; + pending_response + .send(payload.map_err(|_| RequestFailure::Refused).map(|r| (r, protocol_name))) + .expect("Sending response is expected to work"); + } + .boxed(), + ); + + tx +} diff --git a/polkadot/node/network/availability-recovery/src/error.rs b/polkadot/node/network/availability-recovery/src/error.rs index 2aa1f07465e7..9cf2c63cbfa5 100644 --- a/polkadot/node/network/availability-recovery/src/error.rs +++ b/polkadot/node/network/availability-recovery/src/error.rs @@ -20,7 +20,7 @@ use crate::LOG_TARGET; use fatality::{fatality, Nested}; use futures::channel::oneshot; use polkadot_node_network_protocol::request_response::incoming; -use polkadot_node_subsystem::{ChainApiError, RecoveryError, SubsystemError}; +use polkadot_node_subsystem::{RecoveryError, SubsystemError}; use polkadot_primitives::Hash; /// Error type used by the Availability Recovery subsystem. @@ -58,12 +58,6 @@ pub enum Error { #[error(transparent)] Oneshot(#[from] oneshot::Canceled), - #[error("Retrieving response from Chain API unexpectedly failed with error: {0}")] - ChainApi(#[from] ChainApiError), - - #[error("Cannot find block number for given relay parent")] - BlockNumberNotFound, - #[fatal(forward)] #[error("Error during recovery: {0}")] Recovery(#[from] RecoveryError), diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index f8ddae9f4577..f2bf2ccb2b6e 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -34,6 +34,7 @@ use futures::{ stream::{FuturesUnordered, StreamExt}, task::{Context, Poll}, }; +use sc_network::ProtocolName; use schnellru::{ByLength, LruMap}; use task::{ FetchChunks, FetchChunksParams, FetchFull, FetchFullParams, FetchSystematicChunks, @@ -48,10 +49,12 @@ use task::{RecoveryParams, RecoveryStrategy, RecoveryTask}; use error::{log_error, Error, FatalError, Result}; use polkadot_node_network_protocol::{ - request_response::{v1 as request_v1, IncomingRequestReceiver}, + request_response::{ + v1 as request_v1, v2 as request_v2, IncomingRequestReceiver, IsRequest, ReqProtocolNames, + }, UnifiedReputationChange as Rep, }; -use polkadot_node_primitives::{AvailableData, ErasureChunk}; +use polkadot_node_primitives::AvailableData; use polkadot_node_subsystem::{ errors::RecoveryError, jaeger, @@ -60,12 +63,12 @@ use polkadot_node_subsystem::{ SubsystemContext, SubsystemError, }; use polkadot_node_subsystem_util::{ - availability_chunks::ChunkIndexCacheRegistry, get_block_number, request_session_info, - runtime::request_node_features, + availability_chunks::availability_chunk_indices, + runtime::{ExtendedSessionInfo, RuntimeInfo}, }; use polkadot_primitives::{ - BlockNumber, CandidateHash, CandidateReceipt, ChunkIndex, GroupIndex, Hash, SessionIndex, - SessionInfo, ValidatorIndex, + vstaging::node_features, BlockNumber, CandidateHash, CandidateReceipt, ChunkIndex, CoreIndex, + GroupIndex, Hash, SessionIndex, ValidatorIndex, }; mod error; @@ -91,20 +94,27 @@ const SMALL_POV_LIMIT: usize = 128 * 1024; #[derive(Clone, PartialEq)] /// The strategy we use to recover the PoV. -pub enum RecoveryStrategyKind { - /// We always try the backing group first, then fallback to validator chunks. - BackersFirstAlways, +enum RecoveryStrategyKind { /// We try the backing group first if PoV size is lower than specified, then fallback to /// validator chunks. BackersFirstIfSizeLower(usize), /// We try the backing group first if PoV size is lower than specified, then fallback to - /// systematic chunks. + /// systematic chunks. Regular chunk recovery as a last resort. BackersFirstIfSizeLowerThenSystematicChunks(usize), + + /// The following variants are only helpful for integration tests. + /// + /// We always try the backing group first, then fallback to validator chunks. + #[allow(dead_code)] + BackersFirstAlways, /// We always recover using validator chunks. + #[allow(dead_code)] ChunksAlways, /// First try the backing group. Then systematic chunks. + #[allow(dead_code)] BackersThenSystematicChunks, /// Always recover using systematic chunks, fall back to regular chunks. + #[allow(dead_code)] SystematicChunks, } @@ -123,6 +133,10 @@ pub struct AvailabilityRecoverySubsystem { metrics: Metrics, /// The type of check to perform after available data was recovered. post_recovery_check: PostRecoveryCheck, + /// Full protocol name for ChunkFetchingV1. + req_v1_protocol_name: ProtocolName, + /// Full protocol name for ChunkFetchingV2. + req_v2_protocol_name: ProtocolName, } #[derive(Clone, PartialEq, Debug)] @@ -139,7 +153,7 @@ pub enum ErasureTask { /// Reconstructs `AvailableData` from chunks given `n_validators`. Reconstruct( usize, - BTreeMap, + BTreeMap>, oneshot::Sender>, ), /// Re-encode `AvailableData` into erasure chunks in order to verify the provided root hash of @@ -282,8 +296,8 @@ struct State { /// An LRU cache of recently recovered data. availability_lru: LruMap, - /// Cache of the chunk indices shuffle based on the relay parent block. - chunk_indices: ChunkIndexCacheRegistry, + /// Cached runtime info. + runtime_info: RuntimeInfo, } impl Default for State { @@ -292,7 +306,7 @@ impl Default for State { ongoing_recoveries: FuturesUnordered::new(), live_block: (0, Hash::default()), availability_lru: LruMap::new(ByLength::new(LRU_SIZE)), - chunk_indices: ChunkIndexCacheRegistry::new(LRU_SIZE), + runtime_info: RuntimeInfo::new(None), } } } @@ -332,27 +346,11 @@ async fn handle_signal(state: &mut State, signal: OverseerSignal) -> bool { async fn launch_recovery_task( state: &mut State, ctx: &mut Context, - session_info: SessionInfo, - receipt: CandidateReceipt, response_sender: oneshot::Sender, - metrics: &Metrics, recovery_strategies: VecDeque::Sender>>>, - bypass_availability_store: bool, - post_recovery_check: PostRecoveryCheck, + params: RecoveryParams, ) -> Result<()> { - let candidate_hash = receipt.hash(); - let params = RecoveryParams { - validator_authority_keys: session_info.discovery_keys.clone(), - n_validators: session_info.validators.len(), - threshold: recovery_threshold(session_info.validators.len())?, - candidate_hash, - erasure_root: receipt.descriptor.erasure_root, - metrics: metrics.clone(), - bypass_availability_store, - post_recovery_check, - pov_hash: receipt.descriptor.pov_hash, - }; - + let candidate_hash = params.candidate_hash; let recovery_task = RecoveryTask::new(ctx.sender().clone(), params, recovery_strategies); let (remote, remote_handle) = recovery_task.run().remote_handle(); @@ -381,7 +379,9 @@ async fn handle_recover( recovery_strategy_kind: RecoveryStrategyKind, bypass_availability_store: bool, post_recovery_check: PostRecoveryCheck, - maybe_block_number: Option, + maybe_core_index: Option, + req_v1_protocol_name: ProtocolName, + req_v2_protocol_name: ProtocolName, ) -> Result<()> { let candidate_hash = receipt.hash(); @@ -402,59 +402,17 @@ async fn handle_recover( } let _span = span.child("not-cached"); - let session_info = request_session_info(state.live_block.1, session_index, ctx.sender()) - .await - .await??; + let session_info_res = state + .runtime_info + .get_session_info_by_index(ctx.sender(), state.live_block.1, session_index) + .await; let _span = span.child("session-info-ctx-received"); - match session_info { - Some(session_info) => { - let block_number = if let Some(block_number) = maybe_block_number { - block_number - } else { - get_block_number::<_, Error>(ctx.sender(), receipt.descriptor.relay_parent) - .await? - .ok_or(Error::BlockNumberNotFound)? - }; - - let chunk_indices = if let Some(chunk_indices) = state - .chunk_indices - .query_cache_for_para(block_number, session_index, receipt.descriptor.para_id) - { - chunk_indices - } else { - let maybe_node_features = request_node_features( - receipt.descriptor.relay_parent, - session_index, - ctx.sender(), - ) - .await - .map_err(Error::RequestNodeFeatures)?; - - state.chunk_indices.populate_for_para( - maybe_node_features, - session_info.random_seed, - session_info.validators.len(), - block_number, - session_index, - receipt.descriptor.para_id, - ) - }; - - let chunk_indices: VecDeque<_> = chunk_indices - .iter() - .enumerate() - .map(|(v_index, c_index)| { - ( - *c_index, - ValidatorIndex( - u32::try_from(v_index).expect("validator count should not exceed u32"), - ), - ) - }) - .collect(); - + match session_info_res { + Ok(ExtendedSessionInfo { session_info, node_features, .. }) => { let mut backer_group = None; + let n_validators = session_info.validators.len(); + let systematic_threshold = systematic_recovery_threshold(n_validators)?; let mut recovery_strategies: VecDeque< Box::Sender>>, > = VecDeque::with_capacity(2); @@ -472,8 +430,7 @@ async fn handle_recover( let chunk_size: Result> = query_chunk_size(ctx, candidate_hash).await; if let Ok(Some(chunk_size)) = chunk_size { - let pov_size_estimate = - chunk_size.saturating_mul(session_info.validators.len()) / 3; + let pov_size_estimate = chunk_size * systematic_threshold; small_pov_size = pov_size_estimate < small_pov_limit; if small_pov_size { @@ -514,54 +471,95 @@ async fn handle_recover( } } - if matches!( - recovery_strategy_kind, - RecoveryStrategyKind::BackersThenSystematicChunks | - RecoveryStrategyKind::SystematicChunks | - RecoveryStrategyKind::BackersFirstIfSizeLowerThenSystematicChunks(_) - ) { - let systematic_threshold = - systematic_recovery_threshold(session_info.validators.len())?; - - // Only get the validators according to the threshold. - let validators = chunk_indices - .clone() - .into_iter() - .filter(|(c_index, _)| { - usize::try_from(c_index.0) - .expect("usize is at least u32 bytes on all modern targets.") < - systematic_threshold - }) - .collect(); - - recovery_strategies.push_back(Box::new(FetchSystematicChunks::new( - FetchSystematicChunksParams { - validators, - backers: backer_group.map(|v| v.to_vec()).unwrap_or_else(|| vec![]), - erasure_task_tx: erasure_task_tx.clone(), - }, - ))); + // We can only attempt systematic recovery if we received the core index of the + // candidate. + if let Some(core_index) = maybe_core_index { + if matches!( + recovery_strategy_kind, + RecoveryStrategyKind::BackersThenSystematicChunks | + RecoveryStrategyKind::SystematicChunks | + RecoveryStrategyKind::BackersFirstIfSizeLowerThenSystematicChunks(_) + ) { + let chunk_indices = + availability_chunk_indices(Some(node_features), n_validators, core_index)?; + + let chunk_indices: VecDeque<_> = chunk_indices + .iter() + .enumerate() + .map(|(v_index, c_index)| { + ( + *c_index, + ValidatorIndex( + u32::try_from(v_index) + .expect("validator count should not exceed u32"), + ), + ) + }) + .collect(); + + // Only get the validators according to the threshold. + let validators = chunk_indices + .clone() + .into_iter() + .filter(|(c_index, _)| { + usize::try_from(c_index.0) + .expect("usize is at least u32 bytes on all modern targets.") < + systematic_threshold + }) + .collect(); + + recovery_strategies.push_back(Box::new(FetchSystematicChunks::new( + FetchSystematicChunksParams { + validators, + backers: backer_group.map(|v| v.to_vec()).unwrap_or_else(|| vec![]), + erasure_task_tx: erasure_task_tx.clone(), + }, + ))); + } } recovery_strategies.push_back(Box::new(FetchChunks::new(FetchChunksParams { - validators: chunk_indices.clone(), + n_validators: session_info.validators.len(), erasure_task_tx, }))); + let session_info = session_info.clone(); + + let n_validators = session_info.validators.len(); + + let chunk_mapping_enabled = if let Some(&true) = node_features + .get(usize::from(node_features::FeatureIndex::AvailabilityChunkMapping as u8)) + .as_deref() + { + true + } else { + false + }; + launch_recovery_task( state, ctx, - session_info, - receipt, response_sender, - metrics, recovery_strategies, - bypass_availability_store, - post_recovery_check, + RecoveryParams { + validator_authority_keys: session_info.discovery_keys.clone(), + n_validators, + threshold: recovery_threshold(n_validators)?, + systematic_threshold, + candidate_hash, + erasure_root: receipt.descriptor.erasure_root, + metrics: metrics.clone(), + bypass_availability_store, + post_recovery_check, + pov_hash: receipt.descriptor.pov_hash, + req_v1_protocol_name, + req_v2_protocol_name, + chunk_mapping_enabled, + }, ) .await }, - None => { + Err(_) => { response_sender .send(Err(RecoveryError::Unavailable)) .map_err(|_| Error::CanceledResponseSender)?; @@ -604,6 +602,7 @@ impl AvailabilityRecoverySubsystem { /// instead of reencoding the available data. pub fn for_collator( req_receiver: IncomingRequestReceiver, + req_protocol_names: &ReqProtocolNames, metrics: Metrics, ) -> Self { Self { @@ -612,57 +611,23 @@ impl AvailabilityRecoverySubsystem { post_recovery_check: PostRecoveryCheck::PovHash, req_receiver, metrics, + req_v1_protocol_name: req_protocol_names + .get_name(request_v1::ChunkFetchingRequest::PROTOCOL), + req_v2_protocol_name: req_protocol_names + .get_name(request_v2::ChunkFetchingRequest::PROTOCOL), } } - /// Create a new instance of `AvailabilityRecoverySubsystem` which starts with a fast path to - /// request data from backers. - pub fn with_fast_path( - req_receiver: IncomingRequestReceiver, - metrics: Metrics, - ) -> Self { - Self { - recovery_strategy_kind: RecoveryStrategyKind::BackersFirstAlways, - bypass_availability_store: false, - post_recovery_check: PostRecoveryCheck::Reencode, - req_receiver, - metrics, - } - } - - /// Create a new instance of `AvailabilityRecoverySubsystem` which requests only chunks - pub fn with_chunks_only( - req_receiver: IncomingRequestReceiver, - metrics: Metrics, - ) -> Self { - Self { - recovery_strategy_kind: RecoveryStrategyKind::ChunksAlways, - bypass_availability_store: false, - post_recovery_check: PostRecoveryCheck::Reencode, - req_receiver, - metrics, - } - } - - /// Create a new instance of `AvailabilityRecoverySubsystem` which requests chunks if PoV is - /// above a threshold. - pub fn with_chunks_if_pov_large( - req_receiver: IncomingRequestReceiver, - metrics: Metrics, - ) -> Self { - Self { - recovery_strategy_kind: RecoveryStrategyKind::BackersFirstIfSizeLower(SMALL_POV_LIMIT), - bypass_availability_store: false, - post_recovery_check: PostRecoveryCheck::Reencode, - req_receiver, - metrics, - } - } - - /// Create a new instance of `AvailabilityRecoverySubsystem` which requests systematic chunks if - /// PoV is above a threshold. - pub fn with_systematic_chunks_if_pov_large( + /// Create an optimised new instance of `AvailabilityRecoverySubsystem` suitable for validator + /// nodes, which: + /// - for small POVs (over 128Kib), it attempts full recovery from backers, if backing group + /// supplied. + /// - for large POVs, attempts systematic recovery, if core_index supplied and + /// AvailabilityChunkMapping node feature is enabled. + /// - as a last resort, attempt regular chunk recovery from all validators. + pub fn for_validator( req_receiver: IncomingRequestReceiver, + req_protocol_names: &ReqProtocolNames, metrics: Metrics, ) -> Self { Self { @@ -672,36 +637,32 @@ impl AvailabilityRecoverySubsystem { post_recovery_check: PostRecoveryCheck::Reencode, req_receiver, metrics, + req_v1_protocol_name: req_protocol_names + .get_name(request_v1::ChunkFetchingRequest::PROTOCOL), + req_v2_protocol_name: req_protocol_names + .get_name(request_v2::ChunkFetchingRequest::PROTOCOL), } } - /// Create a new instance of `AvailabilityRecoverySubsystem` which first requests full data - /// from backers, with a fallback to recover from systematic chunks. - pub fn with_fast_path_then_systematic_chunks( - req_receiver: IncomingRequestReceiver, - metrics: Metrics, - ) -> Self { - Self { - recovery_strategy_kind: RecoveryStrategyKind::BackersThenSystematicChunks, - bypass_availability_store: false, - post_recovery_check: PostRecoveryCheck::Reencode, - req_receiver, - metrics, - } - } - - /// Create a new instance of `AvailabilityRecoverySubsystem` which first attempts to request - /// systematic chunks, with a fallback to requesting regular chunks. - pub fn with_systematic_chunks( + /// Customise the recovery strategy kind + /// Currently only useful for tests. + #[cfg(test)] + fn with_recovery_strategy_kind( req_receiver: IncomingRequestReceiver, + req_protocol_names: &ReqProtocolNames, metrics: Metrics, + recovery_strategy_kind: RecoveryStrategyKind, ) -> Self { Self { - recovery_strategy_kind: RecoveryStrategyKind::SystematicChunks, + recovery_strategy_kind, bypass_availability_store: false, post_recovery_check: PostRecoveryCheck::Reencode, req_receiver, metrics, + req_v1_protocol_name: req_protocol_names + .get_name(request_v1::ChunkFetchingRequest::PROTOCOL), + req_v2_protocol_name: req_protocol_names + .get_name(request_v2::ChunkFetchingRequest::PROTOCOL), } } @@ -714,6 +675,8 @@ impl AvailabilityRecoverySubsystem { recovery_strategy_kind, bypass_availability_store, post_recovery_check, + req_v1_protocol_name, + req_v2_protocol_name, } = self; let (erasure_task_tx, erasure_task_rx) = futures::channel::mpsc::channel(16); @@ -783,7 +746,7 @@ impl AvailabilityRecoverySubsystem { receipt, session_index, maybe_backing_group, - maybe_block_number, + maybe_core_index, response_sender, ) } => handle_recover( @@ -798,7 +761,9 @@ impl AvailabilityRecoverySubsystem { recovery_strategy_kind.clone(), bypass_availability_store, post_recovery_check.clone(), - maybe_block_number + maybe_core_index, + req_v1_protocol_name.clone(), + req_v2_protocol_name.clone(), ).await } }, @@ -919,7 +884,7 @@ async fn erasure_task_thread( n_validators, chunks.iter().map(|(c_index, chunk)| { ( - &chunk.chunk[..], + &chunk[..], usize::try_from(c_index.0) .expect("usize is at least u32 bytes on all modern targets."), ) diff --git a/polkadot/node/network/availability-recovery/src/task.rs b/polkadot/node/network/availability-recovery/src/task.rs deleted file mode 100644 index 22c2f2c7bb14..000000000000 --- a/polkadot/node/network/availability-recovery/src/task.rs +++ /dev/null @@ -1,2223 +0,0 @@ -// Copyright (C) Parity Technologies (UK) Ltd. -// This file is part of Polkadot. - -// Polkadot is free software: you can redistribute it and/or modify -// it under the terms of the GNU General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. - -// Polkadot is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU General Public License for more details. - -// You should have received a copy of the GNU General Public License -// along with Polkadot. If not, see . - -//! Recovery task and associated strategies. - -#![warn(missing_docs)] - -use crate::{ - futures_undead::FuturesUndead, metrics::Metrics, ErasureTask, PostRecoveryCheck, LOG_TARGET, -}; -use futures::{channel::oneshot, SinkExt}; -use parity_scale_codec::Encode; -use polkadot_erasure_coding::branch_hash; -#[cfg(not(test))] -use polkadot_node_network_protocol::request_response::CHUNK_REQUEST_TIMEOUT; -use polkadot_node_network_protocol::request_response::{ - self as req_res, outgoing::RequestError, OutgoingRequest, Recipient, Requests, -}; -use polkadot_node_primitives::{AvailableData, ErasureChunk}; -use polkadot_node_subsystem::{ - messages::{AvailabilityStoreMessage, NetworkBridgeTxMessage}, - overseer, RecoveryError, -}; -use polkadot_primitives::{ - AuthorityDiscoveryId, BlakeTwo256, CandidateHash, ChunkIndex, Hash, HashT, ValidatorIndex, -}; -use rand::seq::SliceRandom; -use sc_network::{IfDisconnected, OutboundFailure, RequestFailure}; -use std::{ - collections::{BTreeMap, HashMap, VecDeque}, - time::Duration, -}; - -// How many parallel recovery tasks should be running at once. -const N_PARALLEL: usize = 50; - -/// Time after which we consider a request to have failed -/// -/// and we should try more peers. Note in theory the request times out at the network level, -/// measurements have shown, that in practice requests might actually take longer to fail in -/// certain occasions. (The very least, authority discovery is not part of the timeout.) -/// -/// For the time being this value is the same as the timeout on the networking layer, but as this -/// timeout is more soft than the networking one, it might make sense to pick different values as -/// well. -#[cfg(not(test))] -const TIMEOUT_START_NEW_REQUESTS: Duration = CHUNK_REQUEST_TIMEOUT; -#[cfg(test)] -const TIMEOUT_START_NEW_REQUESTS: Duration = Duration::from_millis(100); - -/// The maximum number of times systematic chunk recovery will try making a request for a given -/// (validator,chunk) pair, if the error was not fatal. Added so that we don't get stuck in an -/// infinite retry loop. -pub const SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT: u32 = 2; -/// The maximum number of times regular chunk recovery will try making a request for a given -/// (validator,chunk) pair, if the error was not fatal. Added so that we don't get stuck in an -/// infinite retry loop. -pub const REGULAR_CHUNKS_REQ_RETRY_LIMIT: u32 = 5; - -const fn is_unavailable( - received_chunks: usize, - requesting_chunks: usize, - unrequested_validators: usize, - threshold: usize, -) -> bool { - received_chunks + requesting_chunks + unrequested_validators < threshold -} - -/// Check validity of a chunk. -fn is_chunk_valid(params: &RecoveryParams, chunk: &ErasureChunk) -> bool { - let anticipated_hash = - match branch_hash(¶ms.erasure_root, chunk.proof(), chunk.index.0 as usize) { - Ok(hash) => hash, - Err(e) => { - gum::debug!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - chunk_index = ?chunk.index, - error = ?e, - "Invalid Merkle proof", - ); - return false - }, - }; - let erasure_chunk_hash = BlakeTwo256::hash(&chunk.chunk); - if anticipated_hash != erasure_chunk_hash { - gum::debug!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - chunk_index = ?chunk.index, - "Merkle proof mismatch" - ); - return false - } - true -} - -#[async_trait::async_trait] -/// Common trait for runnable recovery strategies. -pub trait RecoveryStrategy: Send { - /// Main entry point of the strategy. - async fn run( - mut self: Box, - state: &mut State, - sender: &mut Sender, - common_params: &RecoveryParams, - ) -> Result; - - /// Return the name of the strategy for logging purposes. - fn display_name(&self) -> &'static str; - - /// Return the strategy type for use as a metric label. - fn strategy_type(&self) -> &'static str; -} - -/// Recovery parameters common to all strategies in a `RecoveryTask`. -#[derive(Clone)] -pub struct RecoveryParams { - /// Discovery ids of `validators`. - pub validator_authority_keys: Vec, - - /// Number of validators. - pub n_validators: usize, - - /// The number of chunks needed. - pub threshold: usize, - - /// A hash of the relevant candidate. - pub candidate_hash: CandidateHash, - - /// The root of the erasure encoding of the candidate. - pub erasure_root: Hash, - - /// Metrics to report. - pub metrics: Metrics, - - /// Do not request data from availability-store. Useful for collators. - pub bypass_availability_store: bool, - - /// The type of check to perform after available data was recovered. - pub post_recovery_check: PostRecoveryCheck, - - /// The blake2-256 hash of the PoV. - pub pov_hash: Hash, -} - -/// Utility type used for recording the result of requesting a chunk from a validator. -pub enum ErrorRecord { - NonFatal(u32), - Fatal, -} - -/// Intermediate/common data that must be passed between `RecoveryStrategy`s belonging to the -/// same `RecoveryTask`. -pub struct State { - /// Chunks received so far. - /// This MUST be a `BTreeMap` in order for systematic recovery to work (the algorithm assumes - /// that chunks are ordered by their index). If we ever switch this to some non-ordered - /// collection, we need to add a sort step to the systematic recovery. - received_chunks: BTreeMap, - - /// A record of errors returned when requesting a chunk from a validator. - recorded_errors: HashMap<(ChunkIndex, ValidatorIndex), ErrorRecord>, -} - -impl State { - fn new() -> Self { - Self { received_chunks: BTreeMap::new(), recorded_errors: HashMap::new() } - } - - fn insert_chunk(&mut self, chunk_index: ChunkIndex, chunk: ErasureChunk) { - self.received_chunks.insert(chunk_index, chunk); - } - - fn chunk_count(&self) -> usize { - self.received_chunks.len() - } - - fn record_error_fatal(&mut self, chunk_index: ChunkIndex, validator_index: ValidatorIndex) { - self.recorded_errors.insert((chunk_index, validator_index), ErrorRecord::Fatal); - } - - fn record_error_non_fatal(&mut self, chunk_index: ChunkIndex, validator_index: ValidatorIndex) { - self.recorded_errors - .entry((chunk_index, validator_index)) - .and_modify(|record| { - if let ErrorRecord::NonFatal(ref mut count) = record { - *count = count.saturating_add(1); - } - }) - .or_insert(ErrorRecord::NonFatal(1)); - } - - fn can_retry_request( - &self, - chunk_index: ChunkIndex, - validator_index: ValidatorIndex, - retry_threshold: u32, - ) -> bool { - match self.recorded_errors.get(&(chunk_index, validator_index)) { - None => true, - Some(entry) => match entry { - ErrorRecord::Fatal => false, - ErrorRecord::NonFatal(count) if *count < retry_threshold => true, - ErrorRecord::NonFatal(_) => false, - }, - } - } - - /// Retrieve the local chunks held in the av-store (either 0 or 1). - async fn populate_from_av_store( - &mut self, - params: &RecoveryParams, - sender: &mut Sender, - ) -> Vec { - let (tx, rx) = oneshot::channel(); - sender - .send_message(AvailabilityStoreMessage::QueryAllChunks(params.candidate_hash, tx)) - .await; - - match rx.await { - Ok(chunks) => { - // This should either be length 1 or 0. If we had the whole data, - // we wouldn't have reached this stage. - let chunk_indices: Vec<_> = chunks.iter().map(|c| c.index).collect(); - - for chunk in chunks { - if is_chunk_valid(params, &chunk) { - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - chunk_index = ?chunk.index, - "Found valid chunk on disk" - ); - self.insert_chunk(chunk.index, chunk); - } else { - gum::error!( - target: LOG_TARGET, - "Loaded invalid chunk from disk! Disk/Db corruption _very_ likely - please fix ASAP!" - ); - }; - } - - chunk_indices - }, - Err(oneshot::Canceled) => { - gum::warn!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - "Failed to reach the availability store" - ); - - vec![] - }, - } - } - - /// Launch chunk requests in parallel, according to the parameters. - async fn launch_parallel_chunk_requests( - &mut self, - strategy_type: &str, - params: &RecoveryParams, - sender: &mut Sender, - desired_requests_count: usize, - validators: &mut VecDeque<(ChunkIndex, ValidatorIndex)>, - requesting_chunks: &mut FuturesUndead<( - ChunkIndex, - ValidatorIndex, - Result, RequestError>, - )>, - ) where - Sender: overseer::AvailabilityRecoverySenderTrait, - { - let candidate_hash = ¶ms.candidate_hash; - let already_requesting_count = requesting_chunks.len(); - - let to_launch = desired_requests_count - already_requesting_count; - let mut requests = Vec::with_capacity(to_launch); - - gum::trace!( - target: LOG_TARGET, - ?candidate_hash, - "Attempting to launch {} requests", - to_launch - ); - - while requesting_chunks.len() < desired_requests_count { - if let Some((chunk_index, validator_index)) = validators.pop_back() { - let validator = params.validator_authority_keys[validator_index.0 as usize].clone(); - gum::trace!( - target: LOG_TARGET, - ?validator, - ?validator_index, - ?chunk_index, - ?candidate_hash, - "Requesting chunk", - ); - - // Request data. - let raw_request = req_res::v1::ChunkFetchingRequest { - candidate_hash: params.candidate_hash, - index: chunk_index, - }; - - let (req, res) = OutgoingRequest::new(Recipient::Authority(validator), raw_request); - requests.push(Requests::ChunkFetchingV1(req)); - - params.metrics.on_chunk_request_issued(strategy_type); - let timer = params.metrics.time_chunk_request(strategy_type); - - requesting_chunks.push(Box::pin(async move { - let _timer = timer; - let res = match res.await { - Ok(req_res::v1::ChunkFetchingResponse::Chunk(chunk)) => - Ok(Some(chunk.recombine_into_chunk(&raw_request))), - Ok(req_res::v1::ChunkFetchingResponse::NoSuchChunk) => Ok(None), - Err(e) => Err(e), - }; - - (chunk_index, validator_index, res) - })); - } else { - break - } - } - - if requests.len() != 0 { - sender - .send_message(NetworkBridgeTxMessage::SendRequests( - requests, - IfDisconnected::TryConnect, - )) - .await; - } - } - - /// Wait for a sufficient amount of chunks to reconstruct according to the provided `params`. - async fn wait_for_chunks( - &mut self, - strategy_type: &str, - params: &RecoveryParams, - retry_threshold: u32, - validators: &mut VecDeque<(ChunkIndex, ValidatorIndex)>, - requesting_chunks: &mut FuturesUndead<( - ChunkIndex, - ValidatorIndex, - Result, RequestError>, - )>, - // If supplied, these validators will be used as a backup for requesting chunks. They - // should hold all chunks. Each of them will only be used to query one chunk. - backup_validators: &mut Vec, - // Function that returns `true` when this strategy can conclude. Either if we got enough - // chunks or if it's impossible. - can_conclude: impl Fn( - // Number of validators left in the queue - usize, - // Number of in flight requests - usize, - // Number of valid chunks received so far - usize, - // Number of valid chunks received in this iteration - usize, - ) -> bool, - ) -> (usize, usize) { - let metrics = ¶ms.metrics; - - let mut total_received_responses = 0; - let mut error_count = 0; - - // Wait for all current requests to conclude or time-out, or until we reach enough chunks. - // We also declare requests undead, once `TIMEOUT_START_NEW_REQUESTS` is reached and will - // return in that case for `launch_parallel_requests` to fill up slots again. - while let Some(res) = requesting_chunks.next_with_timeout(TIMEOUT_START_NEW_REQUESTS).await - { - total_received_responses += 1; - - let (chunk_index, validator_index, request_result) = res; - - let mut is_error = false; - - match request_result { - Ok(Some(chunk)) => - if is_chunk_valid(params, &chunk) { - metrics.on_chunk_request_succeeded(strategy_type); - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - ?chunk_index, - ?validator_index, - "Received valid chunk", - ); - self.insert_chunk(chunk.index, chunk); - } else { - metrics.on_chunk_request_invalid(strategy_type); - error_count += 1; - // Record that we got an invalid chunk so that subsequent strategies don't - // try requesting this again. - self.record_error_fatal(chunk_index, validator_index); - is_error = true; - }, - Ok(None) => { - metrics.on_chunk_request_no_such_chunk(strategy_type); - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - ?chunk_index, - ?validator_index, - "Validator did not have the requested chunk", - ); - error_count += 1; - // Record that the validator did not have this chunk so that subsequent - // strategies don't try requesting this again. - self.record_error_fatal(chunk_index, validator_index); - is_error = true; - }, - Err(err) => { - error_count += 1; - - gum::trace!( - target: LOG_TARGET, - candidate_hash= ?params.candidate_hash, - ?err, - ?chunk_index, - ?validator_index, - "Failure requesting chunk", - ); - - is_error = true; - - match err { - RequestError::InvalidResponse(_) => { - metrics.on_chunk_request_invalid(strategy_type); - - gum::debug!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - ?err, - ?chunk_index, - ?validator_index, - "Chunk fetching response was invalid", - ); - - // Record that we got an invalid chunk so that this or subsequent - // strategies don't try requesting this again. - self.record_error_fatal(chunk_index, validator_index); - }, - RequestError::NetworkError(err) => { - // No debug logs on general network errors - that became very spammy - // occasionally. - if let RequestFailure::Network(OutboundFailure::Timeout) = err { - metrics.on_chunk_request_timeout(strategy_type); - } else { - metrics.on_chunk_request_error(strategy_type); - } - - // Record that we got a non-fatal error so that this or subsequent - // strategies will retry requesting this only a limited number of times. - self.record_error_non_fatal(chunk_index, validator_index); - }, - RequestError::Canceled(_) => { - metrics.on_chunk_request_error(strategy_type); - - // Record that we got a non-fatal error so that this or subsequent - // strategies will retry requesting this only a limited number of times. - self.record_error_non_fatal(chunk_index, validator_index); - }, - } - }, - } - - if is_error && !self.received_chunks.contains_key(&chunk_index) { - // First, see if we can retry the request. - if self.can_retry_request(chunk_index, validator_index, retry_threshold) { - validators.push_front((chunk_index, validator_index)); - } else { - // Otherwise, try requesting from a backer as a backup, if we've not already - // requested the same chunk from it. - - let position = backup_validators - .iter() - .position(|v| !self.recorded_errors.contains_key(&(chunk_index, *v))); - if let Some(position) = position { - let backer = backup_validators.swap_remove(position); - validators.push_front((chunk_index, backer)); - println!("There"); - } else { - println!("here"); - } - } - } - - if can_conclude( - validators.len(), - requesting_chunks.total_len(), - self.chunk_count(), - total_received_responses - error_count, - ) { - gum::debug!( - target: LOG_TARGET, - validators_len = validators.len(), - candidate_hash = ?params.candidate_hash, - received_chunks_count = ?self.chunk_count(), - requested_chunks_count = ?requesting_chunks.len(), - threshold = ?params.threshold, - "Can conclude availability recovery strategy", - ); - break - } - } - - (total_received_responses, error_count) - } -} - -/// A stateful reconstruction of availability data in reference to -/// a candidate hash. -pub struct RecoveryTask { - sender: Sender, - params: RecoveryParams, - strategies: VecDeque>>, - state: State, -} - -impl RecoveryTask -where - Sender: overseer::AvailabilityRecoverySenderTrait, -{ - /// Instantiate a new recovery task. - pub fn new( - sender: Sender, - params: RecoveryParams, - strategies: VecDeque>>, - ) -> Self { - Self { sender, params, strategies, state: State::new() } - } - - async fn in_availability_store(&mut self) -> Option { - if !self.params.bypass_availability_store { - let (tx, rx) = oneshot::channel(); - self.sender - .send_message(AvailabilityStoreMessage::QueryAvailableData( - self.params.candidate_hash, - tx, - )) - .await; - - match rx.await { - Ok(Some(data)) => return Some(data), - Ok(None) => {}, - Err(oneshot::Canceled) => { - gum::warn!( - target: LOG_TARGET, - candidate_hash = ?self.params.candidate_hash, - "Failed to reach the availability store", - ) - }, - } - } - - None - } - - /// Run this recovery task to completion. It will loop through the configured strategies - /// in-order and return whenever the first one recovers the full `AvailableData`. - pub async fn run(mut self) -> Result { - if let Some(data) = self.in_availability_store().await { - return Ok(data) - } - - self.params.metrics.on_recovery_started(); - - let _timer = self.params.metrics.time_full_recovery(); - - while let Some(current_strategy) = self.strategies.pop_front() { - let display_name = current_strategy.display_name(); - let strategy_type = current_strategy.strategy_type(); - - gum::debug!( - target: LOG_TARGET, - candidate_hash = ?self.params.candidate_hash, - "Starting `{}` strategy", - display_name - ); - - let res = current_strategy.run(&mut self.state, &mut self.sender, &self.params).await; - - match res { - Err(RecoveryError::Unavailable) => - if self.strategies.front().is_some() { - gum::debug!( - target: LOG_TARGET, - candidate_hash = ?self.params.candidate_hash, - "Recovery strategy `{}` did not conclude. Trying the next one.", - display_name - ); - continue - }, - Err(err) => { - match &err { - RecoveryError::Invalid => - self.params.metrics.on_recovery_invalid(strategy_type), - _ => self.params.metrics.on_recovery_failed(strategy_type), - } - return Err(err) - }, - Ok(data) => { - self.params.metrics.on_recovery_succeeded(strategy_type, data.encoded_size()); - return Ok(data) - }, - } - } - - // We have no other strategies to try. - gum::warn!( - target: LOG_TARGET, - candidate_hash = ?self.params.candidate_hash, - "Recovery of available data failed.", - ); - - self.params.metrics.on_recovery_failed("all"); - - Err(RecoveryError::Unavailable) - } -} - -/// `RecoveryStrategy` that sequentially tries to fetch the full `AvailableData` from -/// already-connected validators in the configured validator set. -pub struct FetchFull { - params: FetchFullParams, -} - -pub struct FetchFullParams { - /// Validators that will be used for fetching the data. - pub validators: Vec, - /// Channel to the erasure task handler. - pub erasure_task_tx: futures::channel::mpsc::Sender, -} - -impl FetchFull { - /// Create a new `FetchFull` recovery strategy. - pub fn new(mut params: FetchFullParams) -> Self { - params.validators.shuffle(&mut rand::thread_rng()); - Self { params } - } -} - -#[async_trait::async_trait] -impl RecoveryStrategy for FetchFull { - fn display_name(&self) -> &'static str { - "Full recovery from backers" - } - - fn strategy_type(&self) -> &'static str { - "full_from_backers" - } - - async fn run( - mut self: Box, - _: &mut State, - sender: &mut Sender, - common_params: &RecoveryParams, - ) -> Result { - let strategy_type = RecoveryStrategy::::strategy_type(&*self); - - loop { - // Pop the next validator. - let validator_index = - self.params.validators.pop().ok_or_else(|| RecoveryError::Unavailable)?; - - // Request data. - let (req, response) = OutgoingRequest::new( - Recipient::Authority( - common_params.validator_authority_keys[validator_index.0 as usize].clone(), - ), - req_res::v1::AvailableDataFetchingRequest { - candidate_hash: common_params.candidate_hash, - }, - ); - - sender - .send_message(NetworkBridgeTxMessage::SendRequests( - vec![Requests::AvailableDataFetchingV1(req)], - IfDisconnected::ImmediateError, - )) - .await; - - common_params.metrics.on_full_request_issued(); - - match response.await { - Ok(req_res::v1::AvailableDataFetchingResponse::AvailableData(data)) => { - let recovery_duration = - common_params.metrics.time_erasure_recovery(strategy_type); - let maybe_data = match common_params.post_recovery_check { - PostRecoveryCheck::Reencode => { - let (reencode_tx, reencode_rx) = oneshot::channel(); - self.params - .erasure_task_tx - .send(ErasureTask::Reencode( - common_params.n_validators, - common_params.erasure_root, - data, - reencode_tx, - )) - .await - .map_err(|_| RecoveryError::ChannelClosed)?; - - reencode_rx.await.map_err(|_| RecoveryError::ChannelClosed)? - }, - PostRecoveryCheck::PovHash => - (data.pov.hash() == common_params.pov_hash).then_some(data), - }; - - match maybe_data { - Some(data) => { - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?common_params.candidate_hash, - "Received full data", - ); - - common_params.metrics.on_full_request_succeeded(); - return Ok(data) - }, - None => { - common_params.metrics.on_full_request_invalid(); - recovery_duration.map(|rd| rd.stop_and_discard()); - - gum::debug!( - target: LOG_TARGET, - candidate_hash = ?common_params.candidate_hash, - ?validator_index, - "Invalid data response", - ); - - // it doesn't help to report the peer with req/res. - // we'll try the next backer. - }, - } - }, - Ok(req_res::v1::AvailableDataFetchingResponse::NoSuchData) => { - common_params.metrics.on_full_request_no_such_data(); - }, - Err(e) => { - match &e { - RequestError::Canceled(_) => common_params.metrics.on_full_request_error(), - RequestError::InvalidResponse(_) => - common_params.metrics.on_full_request_invalid(), - RequestError::NetworkError(req_failure) => { - if let RequestFailure::Network(OutboundFailure::Timeout) = req_failure { - common_params.metrics.on_full_request_timeout(); - } else { - common_params.metrics.on_full_request_error(); - } - }, - }; - gum::debug!( - target: LOG_TARGET, - candidate_hash = ?common_params.candidate_hash, - ?validator_index, - err = ?e, - "Error fetching full available data." - ); - }, - } - } - } -} - -/// `RecoveryStrategy` that attempts to recover the systematic chunks from the validators that -/// hold them, in order to bypass the erasure code reconstruction step, which is costly. -pub struct FetchSystematicChunks { - /// Systematic recovery threshold. - threshold: usize, - /// Validators that hold the systematic chunks. - validators: VecDeque<(ChunkIndex, ValidatorIndex)>, - /// Backers. to be used as a backup. - backers: Vec, - /// Collection of in-flight requests. - requesting_chunks: - FuturesUndead<(ChunkIndex, ValidatorIndex, Result, RequestError>)>, - /// Channel to the erasure task handler. - erasure_task_tx: futures::channel::mpsc::Sender, -} - -/// Parameters needed for fetching systematic chunks. -pub struct FetchSystematicChunksParams { - /// Validators that hold the systematic chunks. - pub validators: VecDeque<(ChunkIndex, ValidatorIndex)>, - /// Validators in the backing group, to be used as a backup for requesting systematic chunks. - pub backers: Vec, - /// Channel to the erasure task handler. - pub erasure_task_tx: futures::channel::mpsc::Sender, -} - -impl FetchSystematicChunks { - /// Instantiate a new systematic chunks strategy. - pub fn new(params: FetchSystematicChunksParams) -> Self { - Self { - threshold: params.validators.len(), - validators: params.validators, - backers: params.backers, - requesting_chunks: FuturesUndead::new(), - erasure_task_tx: params.erasure_task_tx, - } - } - - fn is_unavailable( - unrequested_validators: usize, - in_flight_requests: usize, - systematic_chunk_count: usize, - threshold: usize, - ) -> bool { - is_unavailable( - systematic_chunk_count, - in_flight_requests, - unrequested_validators, - threshold, - ) - } - - /// Desired number of parallel requests. - /// - /// For the given threshold (total required number of chunks) get the desired number of - /// requests we want to have running in parallel at this time. - fn get_desired_request_count(&self, chunk_count: usize, threshold: usize) -> usize { - // Upper bound for parallel requests. - let max_requests_boundary = std::cmp::min(N_PARALLEL, threshold); - // How many chunks are still needed? - let remaining_chunks = threshold.saturating_sub(chunk_count); - // Actual number of requests we want to have in flight in parallel: - // We don't have to make up for any error rate, as an error fetching a systematic chunk - // results in failure of the entire strategy. - std::cmp::min(max_requests_boundary, remaining_chunks) - } - - async fn attempt_systematic_recovery( - &mut self, - state: &mut State, - common_params: &RecoveryParams, - ) -> Result { - let strategy_type = RecoveryStrategy::::strategy_type(self); - let recovery_duration = common_params.metrics.time_erasure_recovery(strategy_type); - let reconstruct_duration = common_params.metrics.time_erasure_reconstruct(strategy_type); - let chunks = state - .received_chunks - .range( - ChunkIndex(0).. - ChunkIndex( - u32::try_from(self.threshold) - .expect("validator count should not exceed u32"), - ), - ) - .map(|(_, chunk)| &chunk.chunk[..]) - .collect::>(); - - let available_data = polkadot_erasure_coding::reconstruct_from_systematic_v1( - common_params.n_validators, - chunks, - ); - - match available_data { - Ok(data) => { - drop(reconstruct_duration); - - // Send request to re-encode the chunks and check merkle root. - let (reencode_tx, reencode_rx) = oneshot::channel(); - self.erasure_task_tx - .send(ErasureTask::Reencode( - common_params.n_validators, - common_params.erasure_root, - data, - reencode_tx, - )) - .await - .map_err(|_| RecoveryError::ChannelClosed)?; - - let reencode_response = - reencode_rx.await.map_err(|_| RecoveryError::ChannelClosed)?; - - if let Some(data) = reencode_response { - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?common_params.candidate_hash, - erasure_root = ?common_params.erasure_root, - "Recovery from systematic chunks complete", - ); - - Ok(data) - } else { - recovery_duration.map(|rd| rd.stop_and_discard()); - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?common_params.candidate_hash, - erasure_root = ?common_params.erasure_root, - "Systematic data recovery error - root mismatch", - ); - - Err(RecoveryError::Invalid) - } - }, - Err(err) => { - reconstruct_duration.map(|rd| rd.stop_and_discard()); - recovery_duration.map(|rd| rd.stop_and_discard()); - - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?common_params.candidate_hash, - erasure_root = ?common_params.erasure_root, - ?err, - "Systematic data recovery error", - ); - - Err(RecoveryError::Invalid) - }, - } - } -} - -#[async_trait::async_trait] -impl RecoveryStrategy - for FetchSystematicChunks -{ - fn display_name(&self) -> &'static str { - "Fetch systematic chunks" - } - - fn strategy_type(&self) -> &'static str { - "systematic_chunks" - } - - async fn run( - mut self: Box, - state: &mut State, - sender: &mut Sender, - common_params: &RecoveryParams, - ) -> Result { - // First query the store for any chunks we've got. - if !common_params.bypass_availability_store { - let local_chunk_indices = state.populate_from_av_store(common_params, sender).await; - - for our_c_index in &local_chunk_indices { - // If we are among the systematic validators but hold an invalid chunk, we cannot - // perform the systematic recovery. Fall through to the next strategy. - if self.validators.iter().any(|(c_index, _)| c_index == our_c_index) && - !state.received_chunks.contains_key(our_c_index) - { - gum::debug!( - target: LOG_TARGET, - candidate_hash = ?common_params.candidate_hash, - erasure_root = ?common_params.erasure_root, - requesting = %self.requesting_chunks.len(), - total_requesting = %self.requesting_chunks.total_len(), - n_validators = %common_params.n_validators, - chunk_index = ?our_c_index, - "Systematic chunk recovery is not possible. We are among the systematic validators but hold an invalid chunk", - ); - return Err(RecoveryError::Unavailable) - } - } - } - - // Instead of counting the chunks we already have, perform the difference after we remove - // them from the queue. - let mut systematic_chunk_count = self.validators.len(); - - // No need to query the validators that have the chunks we already received or that we know - // don't have the data from previous strategies. - self.validators.retain(|(c_index, v_index)| { - !state.received_chunks.contains_key(c_index) && - state.can_retry_request(*c_index, *v_index, SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT) - }); - - systematic_chunk_count -= self.validators.len(); - - // Safe to `take` here, as we're consuming `self` anyway and we're not using the - // `validators` field in other methods. - let mut validators_queue: VecDeque<_> = std::mem::take(&mut self.validators); - - loop { - // If received_chunks has `systematic_chunk_threshold` entries, attempt to recover the - // data. - if systematic_chunk_count >= self.threshold { - return self.attempt_systematic_recovery::(state, common_params).await - } - - if Self::is_unavailable( - validators_queue.len(), - self.requesting_chunks.total_len(), - systematic_chunk_count, - self.threshold, - ) { - gum::debug!( - target: LOG_TARGET, - candidate_hash = ?common_params.candidate_hash, - erasure_root = ?common_params.erasure_root, - %systematic_chunk_count, - requesting = %self.requesting_chunks.len(), - total_requesting = %self.requesting_chunks.total_len(), - n_validators = %common_params.n_validators, - systematic_threshold = ?self.threshold, - "Data recovery from systematic chunks is not possible", - ); - - return Err(RecoveryError::Unavailable) - } - - let desired_requests_count = - self.get_desired_request_count(systematic_chunk_count, self.threshold); - let already_requesting_count = self.requesting_chunks.len(); - gum::debug!( - target: LOG_TARGET, - ?common_params.candidate_hash, - ?desired_requests_count, - total_received = ?systematic_chunk_count, - systematic_threshold = ?self.threshold, - ?already_requesting_count, - "Requesting systematic availability chunks for a candidate", - ); - - let strategy_type = RecoveryStrategy::::strategy_type(&*self); - - state - .launch_parallel_chunk_requests( - strategy_type, - common_params, - sender, - desired_requests_count, - &mut validators_queue, - &mut self.requesting_chunks, - ) - .await; - - let (total_responses, error_count) = state - .wait_for_chunks( - strategy_type, - common_params, - SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT, - &mut validators_queue, - &mut self.requesting_chunks, - &mut self.backers, - |unrequested_validators, - in_flight_reqs, - // Don't use this chunk count, as it may contain non-systematic chunks. - _chunk_count, - success_responses| { - let chunk_count = systematic_chunk_count + success_responses; - let is_unavailable = Self::is_unavailable( - unrequested_validators, - in_flight_reqs, - chunk_count, - self.threshold, - ); - - chunk_count >= self.threshold || is_unavailable - }, - ) - .await; - - systematic_chunk_count += total_responses - error_count; - } - } -} - -/// `RecoveryStrategy` that requests chunks from validators, in parallel. -pub struct FetchChunks { - /// How many requests have been unsuccessful so far. - error_count: usize, - /// Total number of responses that have been received, including failed ones. - total_received_responses: usize, - /// The collection of chunk indices and the respective validators holding the chunks. - validators: VecDeque<(ChunkIndex, ValidatorIndex)>, - /// Collection of in-flight requests. - requesting_chunks: - FuturesUndead<(ChunkIndex, ValidatorIndex, Result, RequestError>)>, - /// Channel to the erasure task handler. - erasure_task_tx: futures::channel::mpsc::Sender, -} - -/// Parameters specific to the `FetchChunks` strategy. -pub struct FetchChunksParams { - /// The collection of chunk indices and the respective validators holding the chunks. - pub validators: VecDeque<(ChunkIndex, ValidatorIndex)>, - /// Channel to the erasure task handler. - pub erasure_task_tx: futures::channel::mpsc::Sender, -} - -impl FetchChunks { - /// Instantiate a new strategy. - pub fn new(mut params: FetchChunksParams) -> Self { - // Shuffle the validators to make sure that we don't request chunks from the same - // validators over and over. - params.validators.make_contiguous().shuffle(&mut rand::thread_rng()); - - Self { - error_count: 0, - total_received_responses: 0, - validators: params.validators, - requesting_chunks: FuturesUndead::new(), - erasure_task_tx: params.erasure_task_tx, - } - } - - fn is_unavailable( - unrequested_validators: usize, - in_flight_requests: usize, - chunk_count: usize, - threshold: usize, - ) -> bool { - is_unavailable(chunk_count, in_flight_requests, unrequested_validators, threshold) - } - - /// Desired number of parallel requests. - /// - /// For the given threshold (total required number of chunks) get the desired number of - /// requests we want to have running in parallel at this time. - fn get_desired_request_count(&self, chunk_count: usize, threshold: usize) -> usize { - // Upper bound for parallel requests. - // We want to limit this, so requests can be processed within the timeout and we limit the - // following feedback loop: - // 1. Requests fail due to timeout - // 2. We request more chunks to make up for it - // 3. Bandwidth is spread out even more, so we get even more timeouts - // 4. We request more chunks to make up for it ... - let max_requests_boundary = std::cmp::min(N_PARALLEL, threshold); - // How many chunks are still needed? - let remaining_chunks = threshold.saturating_sub(chunk_count); - // What is the current error rate, so we can make up for it? - let inv_error_rate = - self.total_received_responses.checked_div(self.error_count).unwrap_or(0); - // Actual number of requests we want to have in flight in parallel: - std::cmp::min( - max_requests_boundary, - remaining_chunks + remaining_chunks.checked_div(inv_error_rate).unwrap_or(0), - ) - } - - async fn attempt_recovery( - &mut self, - state: &mut State, - common_params: &RecoveryParams, - ) -> Result { - let recovery_duration = common_params - .metrics - .time_erasure_recovery(RecoveryStrategy::::strategy_type(self)); - - // Send request to reconstruct available data from chunks. - let (avilable_data_tx, available_data_rx) = oneshot::channel(); - self.erasure_task_tx - .send(ErasureTask::Reconstruct( - common_params.n_validators, - // Safe to leave an empty vec in place, as we're stopping the recovery process if - // this reconstruct fails. - std::mem::take(&mut state.received_chunks), - avilable_data_tx, - )) - .await - .map_err(|_| RecoveryError::ChannelClosed)?; - - let available_data_response = - available_data_rx.await.map_err(|_| RecoveryError::ChannelClosed)?; - - match available_data_response { - Ok(data) => { - let maybe_data = match common_params.post_recovery_check { - PostRecoveryCheck::Reencode => { - // Send request to re-encode the chunks and check merkle root. - let (reencode_tx, reencode_rx) = oneshot::channel(); - self.erasure_task_tx - .send(ErasureTask::Reencode( - common_params.n_validators, - common_params.erasure_root, - data, - reencode_tx, - )) - .await - .map_err(|_| RecoveryError::ChannelClosed)?; - - reencode_rx.await.map_err(|_| RecoveryError::ChannelClosed)?.or_else(|| { - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?common_params.candidate_hash, - erasure_root = ?common_params.erasure_root, - "Data recovery error - root mismatch", - ); - None - }) - }, - PostRecoveryCheck::PovHash => - (data.pov.hash() == common_params.pov_hash).then_some(data).or_else(|| { - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?common_params.candidate_hash, - pov_hash = ?common_params.pov_hash, - "Data recovery error - PoV hash mismatch", - ); - None - }), - }; - - if let Some(data) = maybe_data { - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?common_params.candidate_hash, - erasure_root = ?common_params.erasure_root, - "Data recovery from chunks complete", - ); - - Ok(data) - } else { - recovery_duration.map(|rd| rd.stop_and_discard()); - - Err(RecoveryError::Invalid) - } - }, - Err(err) => { - recovery_duration.map(|rd| rd.stop_and_discard()); - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?common_params.candidate_hash, - erasure_root = ?common_params.erasure_root, - ?err, - "Data recovery error", - ); - - Err(RecoveryError::Invalid) - }, - } - } -} - -#[async_trait::async_trait] -impl RecoveryStrategy for FetchChunks { - fn display_name(&self) -> &'static str { - "Fetch chunks" - } - - fn strategy_type(&self) -> &'static str { - "regular_chunks" - } - - async fn run( - mut self: Box, - state: &mut State, - sender: &mut Sender, - common_params: &RecoveryParams, - ) -> Result { - // First query the store for any chunks we've got. - if !common_params.bypass_availability_store { - let local_chunk_indices = state.populate_from_av_store(common_params, sender).await; - self.validators.retain(|(c_index, _)| !local_chunk_indices.contains(c_index)); - } - - // No need to query the validators that have the chunks we already received or that we know - // don't have the data from previous strategies. - self.validators.retain(|(c_index, v_index)| { - !state.received_chunks.contains_key(c_index) && - state.can_retry_request(*c_index, *v_index, REGULAR_CHUNKS_REQ_RETRY_LIMIT) - }); - - // Safe to `take` here, as we're consuming `self` anyway and we're not using the - // `validators` field in other methods. - let mut validators_queue = std::mem::take(&mut self.validators); - - loop { - // If received_chunks has more than threshold entries, attempt to recover the data. - // If that fails, or a re-encoding of it doesn't match the expected erasure root, - // return Err(RecoveryError::Invalid). - // Do this before requesting any chunks because we may have enough of them coming from - // past RecoveryStrategies. - if state.chunk_count() >= common_params.threshold { - return self.attempt_recovery::(state, common_params).await - } - - if Self::is_unavailable( - validators_queue.len(), - self.requesting_chunks.total_len(), - state.chunk_count(), - common_params.threshold, - ) { - gum::debug!( - target: LOG_TARGET, - candidate_hash = ?common_params.candidate_hash, - erasure_root = ?common_params.erasure_root, - received = %state.chunk_count(), - requesting = %self.requesting_chunks.len(), - total_requesting = %self.requesting_chunks.total_len(), - n_validators = %common_params.n_validators, - "Data recovery from chunks is not possible", - ); - - return Err(RecoveryError::Unavailable) - } - - let desired_requests_count = - self.get_desired_request_count(state.chunk_count(), common_params.threshold); - let already_requesting_count = self.requesting_chunks.len(); - gum::debug!( - target: LOG_TARGET, - ?common_params.candidate_hash, - ?desired_requests_count, - error_count= ?self.error_count, - total_received = ?self.total_received_responses, - threshold = ?common_params.threshold, - ?already_requesting_count, - "Requesting availability chunks for a candidate", - ); - - let strategy_type = RecoveryStrategy::::strategy_type(&*self); - - state - .launch_parallel_chunk_requests( - strategy_type, - common_params, - sender, - desired_requests_count, - &mut validators_queue, - &mut self.requesting_chunks, - ) - .await; - - let (total_responses, error_count) = state - .wait_for_chunks( - strategy_type, - common_params, - REGULAR_CHUNKS_REQ_RETRY_LIMIT, - &mut validators_queue, - &mut self.requesting_chunks, - &mut vec![], - |unrequested_validators, in_flight_reqs, chunk_count, _success_responses| { - chunk_count >= common_params.threshold || - Self::is_unavailable( - unrequested_validators, - in_flight_reqs, - chunk_count, - common_params.threshold, - ) - }, - ) - .await; - - self.total_received_responses += total_responses; - self.error_count += error_count; - } - } -} - -#[cfg(test)] -mod tests { - use super::{super::tests::*, *}; - use assert_matches::assert_matches; - use futures::{ - channel::mpsc::UnboundedReceiver, executor, future, Future, FutureExt, StreamExt, - }; - use parity_scale_codec::Error as DecodingError; - use polkadot_erasure_coding::{recovery_threshold, systematic_recovery_threshold}; - use polkadot_node_primitives::{BlockData, PoV}; - use polkadot_node_subsystem::{AllMessages, TimeoutExt}; - use polkadot_node_subsystem_test_helpers::{ - derive_erasure_chunks_with_proofs_and_root, sender_receiver, TestSubsystemSender, - }; - use polkadot_primitives::{HeadData, PersistedValidationData}; - use polkadot_primitives_test_helpers::dummy_hash; - use sp_keyring::Sr25519Keyring; - use std::sync::Arc; - - const TIMEOUT: Duration = Duration::from_secs(1); - - impl Default for RecoveryParams { - fn default() -> Self { - let validators = vec![ - Sr25519Keyring::Ferdie, - Sr25519Keyring::Alice, - Sr25519Keyring::Bob, - Sr25519Keyring::Charlie, - Sr25519Keyring::Dave, - Sr25519Keyring::One, - Sr25519Keyring::Two, - ]; - - Self { - validator_authority_keys: validator_authority_id(&validators), - n_validators: validators.len(), - threshold: recovery_threshold(validators.len()).unwrap(), - candidate_hash: CandidateHash(dummy_hash()), - erasure_root: dummy_hash(), - metrics: Metrics::new_dummy(), - bypass_availability_store: false, - post_recovery_check: PostRecoveryCheck::Reencode, - pov_hash: dummy_hash(), - } - } - } - - impl RecoveryParams { - fn create_chunks(&mut self) -> Vec { - let available_data = dummy_available_data(); - let (chunks, erasure_root) = derive_erasure_chunks_with_proofs_and_root( - self.n_validators, - &available_data, - |_, _| {}, - ); - - self.erasure_root = erasure_root; - self.pov_hash = available_data.pov.hash(); - - chunks - } - } - - fn dummy_available_data() -> AvailableData { - let validation_data = PersistedValidationData { - parent_head: HeadData(vec![7, 8, 9]), - relay_parent_number: Default::default(), - max_pov_size: 1024, - relay_parent_storage_root: Default::default(), - }; - - AvailableData { - validation_data, - pov: Arc::new(PoV { block_data: BlockData(vec![42; 64]) }), - } - } - - fn test_harness, TestFut: Future>( - receiver_future: impl FnOnce(UnboundedReceiver) -> RecvFut, - test: impl FnOnce(TestSubsystemSender) -> TestFut, - ) { - let (sender, receiver) = sender_receiver(); - - let test_fut = test(sender); - let receiver_future = receiver_future(receiver); - - futures::pin_mut!(test_fut); - futures::pin_mut!(receiver_future); - - executor::block_on(future::join(test_fut, receiver_future)).1 - } - - #[test] - fn test_recorded_errors() { - let retry_threshold = 2; - let mut state = State::new(); - - assert!(state.can_retry_request(0.into(), 0.into(), retry_threshold)); - assert!(state.can_retry_request(0.into(), 0.into(), 0)); - state.record_error_non_fatal(0.into(), 0.into()); - assert!(state.can_retry_request(0.into(), 0.into(), retry_threshold)); - state.record_error_non_fatal(0.into(), 0.into()); - assert!(!state.can_retry_request(0.into(), 0.into(), retry_threshold)); - state.record_error_non_fatal(0.into(), 0.into()); - assert!(!state.can_retry_request(0.into(), 0.into(), retry_threshold)); - - assert!(state.can_retry_request(0.into(), 0.into(), 5)); - - state.record_error_fatal(1.into(), 1.into()); - assert!(!state.can_retry_request(1.into(), 1.into(), retry_threshold)); - state.record_error_non_fatal(1.into(), 1.into()); - assert!(!state.can_retry_request(1.into(), 1.into(), retry_threshold)); - - assert!(state.can_retry_request(4.into(), 4.into(), 0)); - assert!(state.can_retry_request(4.into(), 4.into(), retry_threshold)); - } - - #[test] - fn test_populate_from_av_store() { - let params = RecoveryParams::default(); - - // Failed to reach the av store - { - let params = params.clone(); - let candidate_hash = params.candidate_hash; - let mut state = State::new(); - - test_harness( - |mut receiver: UnboundedReceiver| async move { - assert_matches!( - receiver.next().timeout(TIMEOUT).await.unwrap().unwrap(), - AllMessages::AvailabilityStore(AvailabilityStoreMessage::QueryAllChunks(hash, tx)) => { - assert_eq!(hash, candidate_hash); - drop(tx); - }); - }, - |mut sender| async move { - let local_chunk_indices = - state.populate_from_av_store(¶ms, &mut sender).await; - - assert_eq!(state.chunk_count(), 0); - assert_eq!(local_chunk_indices.len(), 0); - }, - ); - } - - // Found invalid chunk - { - let mut params = params.clone(); - let candidate_hash = params.candidate_hash; - let mut state = State::new(); - let chunks = params.create_chunks(); - - test_harness( - |mut receiver: UnboundedReceiver| async move { - assert_matches!( - receiver.next().timeout(TIMEOUT).await.unwrap().unwrap(), - AllMessages::AvailabilityStore(AvailabilityStoreMessage::QueryAllChunks(hash, tx)) => { - assert_eq!(hash, candidate_hash); - let mut chunk = chunks[0].clone(); - chunk.index = 3.into(); - tx.send(vec![chunk]).unwrap(); - }); - }, - |mut sender| async move { - let local_chunk_indices = - state.populate_from_av_store(¶ms, &mut sender).await; - - assert_eq!(state.chunk_count(), 0); - assert_eq!(local_chunk_indices.len(), 1); - }, - ); - } - - // Found valid chunk - { - let mut params = params.clone(); - let candidate_hash = params.candidate_hash; - let mut state = State::new(); - let chunks = params.create_chunks(); - - test_harness( - |mut receiver: UnboundedReceiver| async move { - assert_matches!( - receiver.next().timeout(TIMEOUT).await.unwrap().unwrap(), - AllMessages::AvailabilityStore(AvailabilityStoreMessage::QueryAllChunks(hash, tx)) => { - assert_eq!(hash, candidate_hash); - tx.send(vec![chunks[1].clone()]).unwrap(); - }); - }, - |mut sender| async move { - let local_chunk_indices = - state.populate_from_av_store(¶ms, &mut sender).await; - - assert_eq!(state.chunk_count(), 1); - assert_eq!(local_chunk_indices.len(), 1); - }, - ); - } - } - - #[test] - fn test_launch_parallel_chunk_requests() { - let params = RecoveryParams::default(); - - // No validators to request from. - { - let params = params.clone(); - let mut state = State::new(); - let mut ongoing_reqs = FuturesUndead::new(); - let mut validators = VecDeque::new(); - - test_harness( - |mut receiver: UnboundedReceiver| async move { - // Shouldn't send any requests. - assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); - }, - |mut sender| async move { - state - .launch_parallel_chunk_requests( - "regular", - ¶ms, - &mut sender, - 3, - &mut validators, - &mut ongoing_reqs, - ) - .await; - - assert_eq!(ongoing_reqs.total_len(), 0); - }, - ); - } - - // Has validators but no need to request more. - { - let params = params.clone(); - let mut state = State::new(); - let mut ongoing_reqs = FuturesUndead::new(); - let mut validators = VecDeque::new(); - validators.push_back((ChunkIndex(1), ValidatorIndex(1))); - - test_harness( - |mut receiver: UnboundedReceiver| async move { - // Shouldn't send any requests. - assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); - }, - |mut sender| async move { - state - .launch_parallel_chunk_requests( - "regular", - ¶ms, - &mut sender, - 0, - &mut validators, - &mut ongoing_reqs, - ) - .await; - - assert_eq!(ongoing_reqs.total_len(), 0); - }, - ); - } - - // Has validators but no need to request more. - { - let params = params.clone(); - let mut state = State::new(); - let mut ongoing_reqs = FuturesUndead::new(); - ongoing_reqs.push(async { todo!() }.boxed()); - ongoing_reqs.soft_cancel(); - let mut validators = VecDeque::new(); - validators.push_back((ChunkIndex(1), ValidatorIndex(1))); - - test_harness( - |mut receiver: UnboundedReceiver| async move { - // Shouldn't send any requests. - assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); - }, - |mut sender| async move { - state - .launch_parallel_chunk_requests( - "regular", - ¶ms, - &mut sender, - 0, - &mut validators, - &mut ongoing_reqs, - ) - .await; - - assert_eq!(ongoing_reqs.total_len(), 1); - assert_eq!(ongoing_reqs.len(), 0); - }, - ); - } - - // Needs to request more. - { - let params = params.clone(); - let mut state = State::new(); - let mut ongoing_reqs = FuturesUndead::new(); - ongoing_reqs.push(async { todo!() }.boxed()); - ongoing_reqs.soft_cancel(); - ongoing_reqs.push(async { todo!() }.boxed()); - let mut validators = (0..3).map(|i| (ChunkIndex(i), ValidatorIndex(i))).collect(); - - test_harness( - |mut receiver: UnboundedReceiver| async move { - assert_matches!( - receiver.next().timeout(TIMEOUT).await.unwrap().unwrap(), - AllMessages::NetworkBridgeTx(NetworkBridgeTxMessage::SendRequests(requests, _)) if requests.len() == 3 - ); - }, - |mut sender| async move { - state - .launch_parallel_chunk_requests( - "regular", - ¶ms, - &mut sender, - 10, - &mut validators, - &mut ongoing_reqs, - ) - .await; - - assert_eq!(ongoing_reqs.total_len(), 5); - assert_eq!(ongoing_reqs.len(), 4); - }, - ); - } - } - - #[test] - fn test_wait_for_chunks() { - let params = RecoveryParams::default(); - let retry_threshold = 2; - - // No ongoing requests. - { - let params = params.clone(); - let mut state = State::new(); - let mut ongoing_reqs: FuturesUndead<( - ChunkIndex, - ValidatorIndex, - Result, RequestError>, - )> = FuturesUndead::new(); - let mut validators = VecDeque::new(); - - test_harness( - |mut receiver: UnboundedReceiver| async move { - // Shouldn't send any requests. - assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); - }, - |_| async move { - let (total_responses, error_count) = state - .wait_for_chunks( - "regular", - ¶ms, - retry_threshold, - &mut validators, - &mut ongoing_reqs, - &mut vec![], - |_, _, _, _| false, - ) - .await; - assert_eq!(total_responses, 0); - assert_eq!(error_count, 0); - assert_eq!(state.chunk_count(), 0); - }, - ); - } - - // Complex scenario. - { - let mut params = params.clone(); - let chunks = params.create_chunks(); - let mut state = State::new(); - let mut ongoing_reqs = FuturesUndead::new(); - ongoing_reqs - .push(future::ready((0.into(), 0.into(), Ok(Some(chunks[0].clone())))).boxed()); - ongoing_reqs.soft_cancel(); - ongoing_reqs - .push(future::ready((1.into(), 1.into(), Ok(Some(chunks[1].clone())))).boxed()); - ongoing_reqs.push(future::ready((2.into(), 2.into(), Ok(None))).boxed()); - ongoing_reqs.push( - future::ready(( - 3.into(), - 3.into(), - Err(RequestError::from(DecodingError::from("err"))), - )) - .boxed(), - ); - ongoing_reqs.push( - future::ready(( - 4.into(), - 4.into(), - Err(RequestError::NetworkError(RequestFailure::NotConnected)), - )) - .boxed(), - ); - - let mut validators = - (5..=params.n_validators as u32).map(|i| (i.into(), i.into())).collect(); - - test_harness( - |mut receiver: UnboundedReceiver| async move { - // Shouldn't send any requests. - assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); - }, - |_| async move { - let (total_responses, error_count) = state - .wait_for_chunks( - "regular", - ¶ms, - retry_threshold, - &mut validators, - &mut ongoing_reqs, - &mut vec![], - |_, _, _, _| false, - ) - .await; - assert_eq!(total_responses, 5); - assert_eq!(error_count, 3); - assert_eq!(state.chunk_count(), 2); - - let expected_validators: VecDeque<_> = - (4..=params.n_validators as u32).map(|i| (i.into(), i.into())).collect(); - - assert_eq!(validators, expected_validators); - - // This time we'll go over the recoverable error threshold. - ongoing_reqs.push( - future::ready(( - 4.into(), - 4.into(), - Err(RequestError::NetworkError(RequestFailure::NotConnected)), - )) - .boxed(), - ); - - let (total_responses, error_count) = state - .wait_for_chunks( - "regular", - ¶ms, - retry_threshold, - &mut validators, - &mut ongoing_reqs, - &mut vec![], - |_, _, _, _| false, - ) - .await; - assert_eq!(total_responses, 1); - assert_eq!(error_count, 1); - assert_eq!(state.chunk_count(), 2); - - validators.pop_front(); - let expected_validators: VecDeque<_> = - (5..=params.n_validators as u32).map(|i| (i.into(), i.into())).collect(); - - assert_eq!(validators, expected_validators); - - // Check that can_conclude returning true terminates the loop. - let (total_responses, error_count) = state - .wait_for_chunks( - "regular", - ¶ms, - retry_threshold, - &mut validators, - &mut ongoing_reqs, - &mut vec![], - |_, _, _, _| true, - ) - .await; - assert_eq!(total_responses, 0); - assert_eq!(error_count, 0); - assert_eq!(state.chunk_count(), 2); - - assert_eq!(validators, expected_validators); - }, - ); - } - - // Complex scenario with backups in the backing group. - { - let mut params = params.clone(); - let chunks = params.create_chunks(); - let mut state = State::new(); - let mut ongoing_reqs = FuturesUndead::new(); - ongoing_reqs - .push(future::ready((0.into(), 0.into(), Ok(Some(chunks[0].clone())))).boxed()); - ongoing_reqs.soft_cancel(); - ongoing_reqs - .push(future::ready((1.into(), 1.into(), Ok(Some(chunks[1].clone())))).boxed()); - ongoing_reqs.push(future::ready((2.into(), 2.into(), Ok(None))).boxed()); - ongoing_reqs.push( - future::ready(( - 3.into(), - 3.into(), - Err(RequestError::from(DecodingError::from("err"))), - )) - .boxed(), - ); - ongoing_reqs.push( - future::ready(( - 4.into(), - 4.into(), - Err(RequestError::NetworkError(RequestFailure::NotConnected)), - )) - .boxed(), - ); - - let mut validators = - (5..=params.n_validators as u32).map(|i| (i.into(), i.into())).collect(); - let mut backup_backers = vec![ - 2.into(), - 0.into(), - 4.into(), - 3.into(), - (params.n_validators as u32 + 1).into(), - (params.n_validators as u32 + 2).into(), - ]; - - test_harness( - |mut receiver: UnboundedReceiver| async move { - // Shouldn't send any requests. - assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); - }, - |_| async move { - let (total_responses, error_count) = state - .wait_for_chunks( - "regular", - ¶ms, - retry_threshold, - &mut validators, - &mut ongoing_reqs, - &mut backup_backers, - |_, _, _, _| false, - ) - .await; - assert_eq!(total_responses, 5); - assert_eq!(error_count, 3); - assert_eq!(state.chunk_count(), 2); - - let mut expected_validators: VecDeque<_> = - (5..=params.n_validators as u32).map(|i| (i.into(), i.into())).collect(); - // We picked a backer as a backup for chunks 2 and 3. - expected_validators.push_front((2.into(), 0.into())); - expected_validators.push_front((3.into(), 2.into())); - expected_validators.push_front((4.into(), 4.into())); - - assert_eq!(validators, expected_validators); - - // This time we'll go over the recoverable error threshold for chunk 4. - ongoing_reqs.push( - future::ready(( - 4.into(), - 4.into(), - Err(RequestError::NetworkError(RequestFailure::NotConnected)), - )) - .boxed(), - ); - - validators.pop_front(); - - let (total_responses, error_count) = state - .wait_for_chunks( - "regular", - ¶ms, - retry_threshold, - &mut validators, - &mut ongoing_reqs, - &mut backup_backers, - |_, _, _, _| false, - ) - .await; - assert_eq!(total_responses, 1); - assert_eq!(error_count, 1); - assert_eq!(state.chunk_count(), 2); - - expected_validators.pop_front(); - expected_validators - .push_front((4.into(), (params.n_validators as u32 + 1).into())); - - assert_eq!(validators, expected_validators); - }, - ); - } - } - - #[test] - fn test_recovery_strategy_run() { - let params = RecoveryParams::default(); - - struct GoodStrategy; - #[async_trait::async_trait] - impl RecoveryStrategy for GoodStrategy { - fn display_name(&self) -> &'static str { - "GoodStrategy" - } - - fn strategy_type(&self) -> &'static str { - "good_strategy" - } - - async fn run( - mut self: Box, - _state: &mut State, - _sender: &mut Sender, - _common_params: &RecoveryParams, - ) -> Result { - Ok(dummy_available_data()) - } - } - - struct UnavailableStrategy; - #[async_trait::async_trait] - impl RecoveryStrategy - for UnavailableStrategy - { - fn display_name(&self) -> &'static str { - "UnavailableStrategy" - } - - fn strategy_type(&self) -> &'static str { - "unavailable_strategy" - } - - async fn run( - mut self: Box, - _state: &mut State, - _sender: &mut Sender, - _common_params: &RecoveryParams, - ) -> Result { - Err(RecoveryError::Unavailable) - } - } - - struct InvalidStrategy; - #[async_trait::async_trait] - impl RecoveryStrategy - for InvalidStrategy - { - fn display_name(&self) -> &'static str { - "InvalidStrategy" - } - - fn strategy_type(&self) -> &'static str { - "invalid_strategy" - } - - async fn run( - mut self: Box, - _state: &mut State, - _sender: &mut Sender, - _common_params: &RecoveryParams, - ) -> Result { - Err(RecoveryError::Invalid) - } - } - - // No recovery strategies. - { - let mut params = params.clone(); - let strategies = VecDeque::new(); - params.bypass_availability_store = true; - - test_harness( - |mut receiver: UnboundedReceiver| async move { - // Shouldn't send any requests. - assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); - }, - |sender| async move { - let task = RecoveryTask::new(sender, params, strategies); - - assert_eq!(task.run().await.unwrap_err(), RecoveryError::Unavailable); - }, - ); - } - - // If we have the data in av-store, returns early. - { - let params = params.clone(); - let strategies = VecDeque::new(); - let candidate_hash = params.candidate_hash; - - test_harness( - |mut receiver: UnboundedReceiver| async move { - assert_matches!( - receiver.next().timeout(TIMEOUT).await.unwrap().unwrap(), - AllMessages::AvailabilityStore(AvailabilityStoreMessage::QueryAvailableData(hash, tx)) => { - assert_eq!(hash, candidate_hash); - tx.send(Some(dummy_available_data())).unwrap(); - }); - }, - |sender| async move { - let task = RecoveryTask::new(sender, params, strategies); - - assert_eq!(task.run().await.unwrap(), dummy_available_data()); - }, - ); - } - - // Strategy returning `RecoveryError::Invalid`` will short-circuit the entire task. - { - let mut params = params.clone(); - params.bypass_availability_store = true; - let mut strategies: VecDeque>> = - VecDeque::new(); - strategies.push_back(Box::new(InvalidStrategy)); - strategies.push_back(Box::new(GoodStrategy)); - - test_harness( - |mut receiver: UnboundedReceiver| async move { - // Shouldn't send any requests. - assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); - }, - |sender| async move { - let task = RecoveryTask::new(sender, params, strategies); - - assert_eq!(task.run().await.unwrap_err(), RecoveryError::Invalid); - }, - ); - } - - // Strategy returning `Unavailable` will fall back to the next one. - { - let params = params.clone(); - let candidate_hash = params.candidate_hash; - let mut strategies: VecDeque>> = - VecDeque::new(); - strategies.push_back(Box::new(UnavailableStrategy)); - strategies.push_back(Box::new(GoodStrategy)); - - test_harness( - |mut receiver: UnboundedReceiver| async move { - assert_matches!( - receiver.next().timeout(TIMEOUT).await.unwrap().unwrap(), - AllMessages::AvailabilityStore(AvailabilityStoreMessage::QueryAvailableData(hash, tx)) => { - assert_eq!(hash, candidate_hash); - tx.send(Some(dummy_available_data())).unwrap(); - }); - }, - |sender| async move { - let task = RecoveryTask::new(sender, params, strategies); - - assert_eq!(task.run().await.unwrap(), dummy_available_data()); - }, - ); - } - - // More complex scenario. - { - let params = params.clone(); - let candidate_hash = params.candidate_hash; - let mut strategies: VecDeque>> = - VecDeque::new(); - strategies.push_back(Box::new(UnavailableStrategy)); - strategies.push_back(Box::new(UnavailableStrategy)); - strategies.push_back(Box::new(GoodStrategy)); - strategies.push_back(Box::new(InvalidStrategy)); - - test_harness( - |mut receiver: UnboundedReceiver| async move { - assert_matches!( - receiver.next().timeout(TIMEOUT).await.unwrap().unwrap(), - AllMessages::AvailabilityStore(AvailabilityStoreMessage::QueryAvailableData(hash, tx)) => { - assert_eq!(hash, candidate_hash); - tx.send(Some(dummy_available_data())).unwrap(); - }); - }, - |sender| async move { - let task = RecoveryTask::new(sender, params, strategies); - - assert_eq!(task.run().await.unwrap(), dummy_available_data()); - }, - ); - } - } - - #[test] - fn test_is_unavailable() { - assert_eq!(is_unavailable(0, 0, 0, 0), false); - assert_eq!(is_unavailable(2, 2, 2, 0), false); - // Already reached the threshold. - assert_eq!(is_unavailable(3, 0, 10, 3), false); - assert_eq!(is_unavailable(3, 2, 0, 3), false); - assert_eq!(is_unavailable(3, 2, 10, 3), false); - // It's still possible to reach the threshold - assert_eq!(is_unavailable(0, 0, 10, 3), false); - assert_eq!(is_unavailable(0, 0, 3, 3), false); - assert_eq!(is_unavailable(1, 1, 1, 3), false); - // Not possible to reach the threshold - assert_eq!(is_unavailable(0, 0, 0, 3), true); - assert_eq!(is_unavailable(2, 3, 2, 10), true); - } - - #[test] - fn test_get_desired_request_count() { - // Systematic chunk recovery - { - let num_validators = 100; - let threshold = systematic_recovery_threshold(num_validators).unwrap(); - let (erasure_task_tx, _erasure_task_rx) = futures::channel::mpsc::channel(16); - - let systematic_chunks_task = FetchChunks::new(FetchChunksParams { - validators: (0..100u32).map(|i| (i.into(), i.into())).collect(), - erasure_task_tx, - }); - assert_eq!(systematic_chunks_task.get_desired_request_count(0, threshold), threshold); - assert_eq!( - systematic_chunks_task.get_desired_request_count(5, threshold), - threshold - 5 - ); - assert_eq!( - systematic_chunks_task.get_desired_request_count(num_validators * 2, threshold), - 0 - ); - assert_eq!( - systematic_chunks_task.get_desired_request_count(0, N_PARALLEL * 2), - N_PARALLEL - ); - assert_eq!( - systematic_chunks_task.get_desired_request_count(N_PARALLEL, N_PARALLEL + 2), - 2 - ); - } - - // Regular chunk recovery - { - let num_validators = 100; - let threshold = recovery_threshold(num_validators).unwrap(); - let (erasure_task_tx, _erasure_task_rx) = futures::channel::mpsc::channel(16); - - let mut fetch_chunks_task = FetchChunks::new(FetchChunksParams { - validators: (0..100u32).map(|i| (i.into(), i.into())).collect(), - erasure_task_tx, - }); - assert_eq!(fetch_chunks_task.get_desired_request_count(0, threshold), threshold); - fetch_chunks_task.error_count = 1; - fetch_chunks_task.total_received_responses = 1; - // We saturate at threshold (34): - assert_eq!(fetch_chunks_task.get_desired_request_count(0, threshold), threshold); - - // We saturate at the parallel limit. - assert_eq!(fetch_chunks_task.get_desired_request_count(0, N_PARALLEL + 2), N_PARALLEL); - - fetch_chunks_task.total_received_responses = 2; - // With given error rate - still saturating: - assert_eq!(fetch_chunks_task.get_desired_request_count(1, threshold), threshold); - fetch_chunks_task.total_received_responses = 10; - // error rate: 1/10 - // remaining chunks needed: threshold (34) - 9 - // expected: 24 * (1+ 1/10) = (next greater integer) = 27 - assert_eq!(fetch_chunks_task.get_desired_request_count(9, threshold), 27); - // We saturate at the parallel limit. - assert_eq!(fetch_chunks_task.get_desired_request_count(9, N_PARALLEL + 9), N_PARALLEL); - - fetch_chunks_task.error_count = 0; - // With error count zero - we should fetch exactly as needed: - assert_eq!(fetch_chunks_task.get_desired_request_count(10, threshold), threshold - 10); - } - } -} diff --git a/polkadot/node/network/availability-recovery/src/task/mod.rs b/polkadot/node/network/availability-recovery/src/task/mod.rs new file mode 100644 index 000000000000..d5d6b25eb51b --- /dev/null +++ b/polkadot/node/network/availability-recovery/src/task/mod.rs @@ -0,0 +1,192 @@ +// Copyright (C) Parity Technologies (UK) Ltd. +// This file is part of Polkadot. + +// Polkadot is free software: you can redistribute it and/or modify +// it under the terms of the GNU General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. + +// Polkadot is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU General Public License for more details. + +// You should have received a copy of the GNU General Public License +// along with Polkadot. If not, see . + +//! Main recovery task logic. Runs recovery strategies. + +#![warn(missing_docs)] + +mod strategy; + +pub use self::strategy::{ + FetchChunks, FetchChunksParams, FetchFull, FetchFullParams, FetchSystematicChunks, + FetchSystematicChunksParams, RecoveryStrategy, State, REGULAR_CHUNKS_REQ_RETRY_LIMIT, + SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT, +}; + +use crate::{metrics::Metrics, PostRecoveryCheck, LOG_TARGET}; + +use parity_scale_codec::Encode; +use polkadot_node_primitives::AvailableData; +use polkadot_node_subsystem::{messages::AvailabilityStoreMessage, overseer, RecoveryError}; +use polkadot_primitives::{AuthorityDiscoveryId, CandidateHash, Hash}; +use sc_network::ProtocolName; + +use futures::channel::oneshot; +use std::collections::VecDeque; + +/// Recovery parameters common to all strategies in a `RecoveryTask`. +#[derive(Clone)] +pub struct RecoveryParams { + /// Discovery ids of `validators`. + pub validator_authority_keys: Vec, + + /// Number of validators. + pub n_validators: usize, + + /// The number of regular chunks needed. + pub threshold: usize, + + /// The number of systematic chunks needed. + pub systematic_threshold: usize, + + /// A hash of the relevant candidate. + pub candidate_hash: CandidateHash, + + /// The root of the erasure encoding of the candidate. + pub erasure_root: Hash, + + /// Metrics to report. + pub metrics: Metrics, + + /// Do not request data from availability-store. Useful for collators. + pub bypass_availability_store: bool, + + /// The type of check to perform after available data was recovered. + pub post_recovery_check: PostRecoveryCheck, + + /// The blake2-256 hash of the PoV. + pub pov_hash: Hash, + + /// Protocol name for ChunkFetchingV1. + pub req_v1_protocol_name: ProtocolName, + + /// Protocol name for ChunkFetchingV2. + pub req_v2_protocol_name: ProtocolName, + + /// Whether or not chunk mapping is enabled. + pub chunk_mapping_enabled: bool, +} + +/// A stateful reconstruction of availability data in reference to +/// a candidate hash. +pub struct RecoveryTask { + sender: Sender, + params: RecoveryParams, + strategies: VecDeque>>, + state: State, +} + +impl RecoveryTask +where + Sender: overseer::AvailabilityRecoverySenderTrait, +{ + /// Instantiate a new recovery task. + pub fn new( + sender: Sender, + params: RecoveryParams, + strategies: VecDeque>>, + ) -> Self { + Self { sender, params, strategies, state: State::new() } + } + + async fn in_availability_store(&mut self) -> Option { + if !self.params.bypass_availability_store { + let (tx, rx) = oneshot::channel(); + self.sender + .send_message(AvailabilityStoreMessage::QueryAvailableData( + self.params.candidate_hash, + tx, + )) + .await; + + match rx.await { + Ok(Some(data)) => return Some(data), + Ok(None) => {}, + Err(oneshot::Canceled) => { + gum::warn!( + target: LOG_TARGET, + candidate_hash = ?self.params.candidate_hash, + "Failed to reach the availability store", + ) + }, + } + } + + None + } + + /// Run this recovery task to completion. It will loop through the configured strategies + /// in-order and return whenever the first one recovers the full `AvailableData`. + pub async fn run(mut self) -> Result { + if let Some(data) = self.in_availability_store().await { + return Ok(data) + } + + self.params.metrics.on_recovery_started(); + + let _timer = self.params.metrics.time_full_recovery(); + + while let Some(current_strategy) = self.strategies.pop_front() { + let display_name = current_strategy.display_name(); + let strategy_type = current_strategy.strategy_type(); + + gum::debug!( + target: LOG_TARGET, + candidate_hash = ?self.params.candidate_hash, + "Starting `{}` strategy", + display_name + ); + + let res = current_strategy.run(&mut self.state, &mut self.sender, &self.params).await; + + match res { + Err(RecoveryError::Unavailable) => + if self.strategies.front().is_some() { + gum::debug!( + target: LOG_TARGET, + candidate_hash = ?self.params.candidate_hash, + "Recovery strategy `{}` did not conclude. Trying the next one.", + display_name + ); + continue + }, + Err(err) => { + match &err { + RecoveryError::Invalid => + self.params.metrics.on_recovery_invalid(strategy_type), + _ => self.params.metrics.on_recovery_failed(strategy_type), + } + return Err(err) + }, + Ok(data) => { + self.params.metrics.on_recovery_succeeded(strategy_type, data.encoded_size()); + return Ok(data) + }, + } + } + + // We have no other strategies to try. + gum::warn!( + target: LOG_TARGET, + candidate_hash = ?self.params.candidate_hash, + "Recovery of available data failed.", + ); + + self.params.metrics.on_recovery_failed("all"); + + Err(RecoveryError::Unavailable) + } +} diff --git a/polkadot/node/network/availability-recovery/src/task/strategy/chunks.rs b/polkadot/node/network/availability-recovery/src/task/strategy/chunks.rs new file mode 100644 index 000000000000..67a7d7876157 --- /dev/null +++ b/polkadot/node/network/availability-recovery/src/task/strategy/chunks.rs @@ -0,0 +1,340 @@ +// Copyright (C) Parity Technologies (UK) Ltd. +// This file is part of Polkadot. + +// Polkadot is free software: you can redistribute it and/or modify +// it under the terms of the GNU General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. + +// Polkadot is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU General Public License for more details. + +// You should have received a copy of the GNU General Public License +// along with Polkadot. If not, see . + +use crate::{ + futures_undead::FuturesUndead, + task::{ + strategy::{ + do_post_recovery_check, is_unavailable, OngoingRequests, N_PARALLEL, + REGULAR_CHUNKS_REQ_RETRY_LIMIT, + }, + RecoveryParams, State, + }, + ErasureTask, RecoveryStrategy, LOG_TARGET, +}; + +use polkadot_node_primitives::AvailableData; +use polkadot_node_subsystem::{overseer, RecoveryError}; +use polkadot_primitives::ValidatorIndex; + +use futures::{channel::oneshot, SinkExt}; +use rand::seq::SliceRandom; +use std::collections::VecDeque; + +/// Parameters specific to the `FetchChunks` strategy. +pub struct FetchChunksParams { + pub n_validators: usize, + /// Channel to the erasure task handler. + pub erasure_task_tx: futures::channel::mpsc::Sender, +} + +/// `RecoveryStrategy` that requests chunks from validators, in parallel. +pub struct FetchChunks { + /// How many requests have been unsuccessful so far. + error_count: usize, + /// Total number of responses that have been received, including failed ones. + total_received_responses: usize, + /// A shuffled array of validator indices. + validators: VecDeque, + /// Collection of in-flight requests. + requesting_chunks: OngoingRequests, + /// Channel to the erasure task handler. + erasure_task_tx: futures::channel::mpsc::Sender, +} + +impl FetchChunks { + /// Instantiate a new strategy. + pub fn new(params: FetchChunksParams) -> Self { + // Shuffle the validators to make sure that we don't request chunks from the same + // validators over and over. + let mut validators: VecDeque = + (0..params.n_validators).map(|i| ValidatorIndex(i as u32)).collect(); + validators.make_contiguous().shuffle(&mut rand::thread_rng()); + + Self { + error_count: 0, + total_received_responses: 0, + validators, + requesting_chunks: FuturesUndead::new(), + erasure_task_tx: params.erasure_task_tx, + } + } + + fn is_unavailable( + unrequested_validators: usize, + in_flight_requests: usize, + chunk_count: usize, + threshold: usize, + ) -> bool { + is_unavailable(chunk_count, in_flight_requests, unrequested_validators, threshold) + } + + /// Desired number of parallel requests. + /// + /// For the given threshold (total required number of chunks) get the desired number of + /// requests we want to have running in parallel at this time. + fn get_desired_request_count(&self, chunk_count: usize, threshold: usize) -> usize { + // Upper bound for parallel requests. + // We want to limit this, so requests can be processed within the timeout and we limit the + // following feedback loop: + // 1. Requests fail due to timeout + // 2. We request more chunks to make up for it + // 3. Bandwidth is spread out even more, so we get even more timeouts + // 4. We request more chunks to make up for it ... + let max_requests_boundary = std::cmp::min(N_PARALLEL, threshold); + // How many chunks are still needed? + let remaining_chunks = threshold.saturating_sub(chunk_count); + // What is the current error rate, so we can make up for it? + let inv_error_rate = + self.total_received_responses.checked_div(self.error_count).unwrap_or(0); + // Actual number of requests we want to have in flight in parallel: + std::cmp::min( + max_requests_boundary, + remaining_chunks + remaining_chunks.checked_div(inv_error_rate).unwrap_or(0), + ) + } + + async fn attempt_recovery( + &mut self, + state: &mut State, + common_params: &RecoveryParams, + ) -> Result { + let recovery_duration = common_params + .metrics + .time_erasure_recovery(RecoveryStrategy::::strategy_type(self)); + + // Send request to reconstruct available data from chunks. + let (avilable_data_tx, available_data_rx) = oneshot::channel(); + self.erasure_task_tx + .send(ErasureTask::Reconstruct( + common_params.n_validators, + // Safe to leave an empty vec in place, as we're stopping the recovery process if + // this reconstruct fails. + std::mem::take(&mut state.received_chunks) + .into_iter() + .map(|(c_index, chunk)| (c_index, chunk.chunk)) + .collect(), + avilable_data_tx, + )) + .await + .map_err(|_| RecoveryError::ChannelClosed)?; + + let available_data_response = + available_data_rx.await.map_err(|_| RecoveryError::ChannelClosed)?; + + match available_data_response { + // Attempt post-recovery check. + Ok(data) => do_post_recovery_check(common_params, data, &mut self.erasure_task_tx) + .await + .map_err(|e| { + recovery_duration.map(|rd| rd.stop_and_discard()); + e + }) + .map(|data| { + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + erasure_root = ?common_params.erasure_root, + "Data recovery from chunks complete", + ); + data + }), + Err(err) => { + recovery_duration.map(|rd| rd.stop_and_discard()); + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + erasure_root = ?common_params.erasure_root, + ?err, + "Data recovery error", + ); + + Err(RecoveryError::Invalid) + }, + } + } +} + +#[async_trait::async_trait] +impl RecoveryStrategy for FetchChunks { + fn display_name(&self) -> &'static str { + "Fetch chunks" + } + + fn strategy_type(&self) -> &'static str { + "regular_chunks" + } + + async fn run( + mut self: Box, + state: &mut State, + sender: &mut Sender, + common_params: &RecoveryParams, + ) -> Result { + // First query the store for any chunks we've got. + if !common_params.bypass_availability_store { + let local_chunk_indices = state.populate_from_av_store(common_params, sender).await; + self.validators.retain(|validator_index| { + !local_chunk_indices.iter().any(|(v_index, _)| v_index == validator_index) + }); + } + + // No need to query the validators that have the chunks we already received or that we know + // don't have the data from previous strategies. + self.validators.retain(|v_index| { + !state.received_chunks.values().any(|c| v_index == &c.validator_index) && + state.can_retry_request( + &(common_params.validator_authority_keys[v_index.0 as usize].clone(), *v_index), + REGULAR_CHUNKS_REQ_RETRY_LIMIT, + ) + }); + + // Safe to `take` here, as we're consuming `self` anyway and we're not using the + // `validators` field in other methods. + let mut validators_queue: VecDeque<_> = std::mem::take(&mut self.validators) + .into_iter() + .map(|validator_index| { + ( + common_params.validator_authority_keys[validator_index.0 as usize].clone(), + validator_index, + ) + }) + .collect(); + + loop { + // If received_chunks has more than threshold entries, attempt to recover the data. + // If that fails, or a re-encoding of it doesn't match the expected erasure root, + // return Err(RecoveryError::Invalid). + // Do this before requesting any chunks because we may have enough of them coming from + // past RecoveryStrategies. + if state.chunk_count() >= common_params.threshold { + return self.attempt_recovery::(state, common_params).await + } + + if Self::is_unavailable( + validators_queue.len(), + self.requesting_chunks.total_len(), + state.chunk_count(), + common_params.threshold, + ) { + gum::debug!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + erasure_root = ?common_params.erasure_root, + received = %state.chunk_count(), + requesting = %self.requesting_chunks.len(), + total_requesting = %self.requesting_chunks.total_len(), + n_validators = %common_params.n_validators, + "Data recovery from chunks is not possible", + ); + + return Err(RecoveryError::Unavailable) + } + + let desired_requests_count = + self.get_desired_request_count(state.chunk_count(), common_params.threshold); + let already_requesting_count = self.requesting_chunks.len(); + gum::debug!( + target: LOG_TARGET, + ?common_params.candidate_hash, + ?desired_requests_count, + error_count= ?self.error_count, + total_received = ?self.total_received_responses, + threshold = ?common_params.threshold, + ?already_requesting_count, + "Requesting availability chunks for a candidate", + ); + + let strategy_type = RecoveryStrategy::::strategy_type(&*self); + + state + .launch_parallel_chunk_requests( + strategy_type, + common_params, + sender, + desired_requests_count, + &mut validators_queue, + &mut self.requesting_chunks, + ) + .await; + + let (total_responses, error_count) = state + .wait_for_chunks( + strategy_type, + common_params, + REGULAR_CHUNKS_REQ_RETRY_LIMIT, + &mut validators_queue, + &mut self.requesting_chunks, + &mut vec![], + |unrequested_validators, + in_flight_reqs, + chunk_count, + _systematic_chunk_count| { + chunk_count >= common_params.threshold || + Self::is_unavailable( + unrequested_validators, + in_flight_reqs, + chunk_count, + common_params.threshold, + ) + }, + ) + .await; + + self.total_received_responses += total_responses; + self.error_count += error_count; + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use polkadot_erasure_coding::recovery_threshold; + + #[test] + fn test_get_desired_request_count() { + let num_validators = 100; + let threshold = recovery_threshold(num_validators).unwrap(); + let (erasure_task_tx, _erasure_task_rx) = futures::channel::mpsc::channel(16); + + let mut fetch_chunks_task = + FetchChunks::new(FetchChunksParams { n_validators: num_validators, erasure_task_tx }); + assert_eq!(fetch_chunks_task.get_desired_request_count(0, threshold), threshold); + fetch_chunks_task.error_count = 1; + fetch_chunks_task.total_received_responses = 1; + // We saturate at threshold (34): + assert_eq!(fetch_chunks_task.get_desired_request_count(0, threshold), threshold); + + // We saturate at the parallel limit. + assert_eq!(fetch_chunks_task.get_desired_request_count(0, N_PARALLEL + 2), N_PARALLEL); + + fetch_chunks_task.total_received_responses = 2; + // With given error rate - still saturating: + assert_eq!(fetch_chunks_task.get_desired_request_count(1, threshold), threshold); + fetch_chunks_task.total_received_responses = 10; + // error rate: 1/10 + // remaining chunks needed: threshold (34) - 9 + // expected: 24 * (1+ 1/10) = (next greater integer) = 27 + assert_eq!(fetch_chunks_task.get_desired_request_count(9, threshold), 27); + // We saturate at the parallel limit. + assert_eq!(fetch_chunks_task.get_desired_request_count(9, N_PARALLEL + 9), N_PARALLEL); + + fetch_chunks_task.error_count = 0; + // With error count zero - we should fetch exactly as needed: + assert_eq!(fetch_chunks_task.get_desired_request_count(10, threshold), threshold - 10); + } +} diff --git a/polkadot/node/network/availability-recovery/src/task/strategy/full.rs b/polkadot/node/network/availability-recovery/src/task/strategy/full.rs new file mode 100644 index 000000000000..5b9b2b5a7440 --- /dev/null +++ b/polkadot/node/network/availability-recovery/src/task/strategy/full.rs @@ -0,0 +1,175 @@ +// Copyright (C) Parity Technologies (UK) Ltd. +// This file is part of Polkadot. + +// Polkadot is free software: you can redistribute it and/or modify +// it under the terms of the GNU General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. + +// Polkadot is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU General Public License for more details. + +// You should have received a copy of the GNU General Public License +// along with Polkadot. If not, see . + +use crate::{ + task::{RecoveryParams, RecoveryStrategy, State}, + ErasureTask, PostRecoveryCheck, LOG_TARGET, +}; + +use polkadot_node_network_protocol::request_response::{ + self as req_res, outgoing::RequestError, OutgoingRequest, Recipient, Requests, +}; +use polkadot_node_primitives::AvailableData; +use polkadot_node_subsystem::{messages::NetworkBridgeTxMessage, overseer, RecoveryError}; +use polkadot_primitives::ValidatorIndex; +use sc_network::{IfDisconnected, OutboundFailure, RequestFailure}; + +use futures::{channel::oneshot, SinkExt}; +use rand::seq::SliceRandom; + +/// Parameters specific to the `FetchFull` strategy. +pub struct FetchFullParams { + /// Validators that will be used for fetching the data. + pub validators: Vec, + /// Channel to the erasure task handler. + pub erasure_task_tx: futures::channel::mpsc::Sender, +} + +/// `RecoveryStrategy` that sequentially tries to fetch the full `AvailableData` from +/// already-connected validators in the configured validator set. +pub struct FetchFull { + params: FetchFullParams, +} + +impl FetchFull { + /// Create a new `FetchFull` recovery strategy. + pub fn new(mut params: FetchFullParams) -> Self { + params.validators.shuffle(&mut rand::thread_rng()); + Self { params } + } +} + +#[async_trait::async_trait] +impl RecoveryStrategy for FetchFull { + fn display_name(&self) -> &'static str { + "Full recovery from backers" + } + + fn strategy_type(&self) -> &'static str { + "full_from_backers" + } + + async fn run( + mut self: Box, + _: &mut State, + sender: &mut Sender, + common_params: &RecoveryParams, + ) -> Result { + let strategy_type = RecoveryStrategy::::strategy_type(&*self); + + loop { + // Pop the next validator. + let validator_index = + self.params.validators.pop().ok_or_else(|| RecoveryError::Unavailable)?; + + // Request data. + let (req, response) = OutgoingRequest::new( + Recipient::Authority( + common_params.validator_authority_keys[validator_index.0 as usize].clone(), + ), + req_res::v1::AvailableDataFetchingRequest { + candidate_hash: common_params.candidate_hash, + }, + ); + + sender + .send_message(NetworkBridgeTxMessage::SendRequests( + vec![Requests::AvailableDataFetchingV1(req)], + IfDisconnected::ImmediateError, + )) + .await; + + common_params.metrics.on_full_request_issued(); + + match response.await { + Ok(req_res::v1::AvailableDataFetchingResponse::AvailableData(data)) => { + let recovery_duration = + common_params.metrics.time_erasure_recovery(strategy_type); + let maybe_data = match common_params.post_recovery_check { + PostRecoveryCheck::Reencode => { + let (reencode_tx, reencode_rx) = oneshot::channel(); + self.params + .erasure_task_tx + .send(ErasureTask::Reencode( + common_params.n_validators, + common_params.erasure_root, + data, + reencode_tx, + )) + .await + .map_err(|_| RecoveryError::ChannelClosed)?; + + reencode_rx.await.map_err(|_| RecoveryError::ChannelClosed)? + }, + PostRecoveryCheck::PovHash => + (data.pov.hash() == common_params.pov_hash).then_some(data), + }; + + match maybe_data { + Some(data) => { + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + "Received full data", + ); + + common_params.metrics.on_full_request_succeeded(); + return Ok(data) + }, + None => { + common_params.metrics.on_full_request_invalid(); + recovery_duration.map(|rd| rd.stop_and_discard()); + + gum::debug!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + ?validator_index, + "Invalid data response", + ); + + // it doesn't help to report the peer with req/res. + // we'll try the next backer. + }, + } + }, + Ok(req_res::v1::AvailableDataFetchingResponse::NoSuchData) => { + common_params.metrics.on_full_request_no_such_data(); + }, + Err(e) => { + match &e { + RequestError::Canceled(_) => common_params.metrics.on_full_request_error(), + RequestError::InvalidResponse(_) => + common_params.metrics.on_full_request_invalid(), + RequestError::NetworkError(req_failure) => { + if let RequestFailure::Network(OutboundFailure::Timeout) = req_failure { + common_params.metrics.on_full_request_timeout(); + } else { + common_params.metrics.on_full_request_error(); + } + }, + }; + gum::debug!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + ?validator_index, + err = ?e, + "Error fetching full available data." + ); + }, + } + } + } +} diff --git a/polkadot/node/network/availability-recovery/src/task/strategy/mod.rs b/polkadot/node/network/availability-recovery/src/task/strategy/mod.rs new file mode 100644 index 000000000000..1b24265844de --- /dev/null +++ b/polkadot/node/network/availability-recovery/src/task/strategy/mod.rs @@ -0,0 +1,1534 @@ +// Copyright (C) Parity Technologies (UK) Ltd. +// This file is part of Polkadot. + +// Polkadot is free software: you can redistribute it and/or modify +// it under the terms of the GNU General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. + +// Polkadot is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU General Public License for more details. + +// You should have received a copy of the GNU General Public License +// along with Polkadot. If not, see . + +//! Recovery strategies. + +mod chunks; +mod full; +mod systematic; + +pub use self::{ + chunks::{FetchChunks, FetchChunksParams}, + full::{FetchFull, FetchFullParams}, + systematic::{FetchSystematicChunks, FetchSystematicChunksParams}, +}; +use crate::{ + futures_undead::FuturesUndead, ErasureTask, PostRecoveryCheck, RecoveryParams, LOG_TARGET, +}; + +use futures::{ + channel::{mpsc, oneshot}, + SinkExt, +}; +use parity_scale_codec::Decode; +use polkadot_erasure_coding::branch_hash; +#[cfg(not(test))] +use polkadot_node_network_protocol::request_response::CHUNK_REQUEST_TIMEOUT; +use polkadot_node_network_protocol::request_response::{ + self as req_res, outgoing::RequestError, OutgoingRequest, Recipient, Requests, +}; +use polkadot_node_primitives::{AvailableData, ErasureChunk}; +use polkadot_node_subsystem::{ + messages::{AvailabilityStoreMessage, NetworkBridgeTxMessage}, + overseer, RecoveryError, +}; +use polkadot_primitives::{AuthorityDiscoveryId, BlakeTwo256, ChunkIndex, HashT, ValidatorIndex}; +use sc_network::{IfDisconnected, OutboundFailure, RequestFailure}; +use std::{ + collections::{BTreeMap, HashMap, VecDeque}, + time::Duration, +}; + +// How many parallel chunk fetching requests should be running at once. +const N_PARALLEL: usize = 50; + +/// Time after which we consider a request to have failed +/// +/// and we should try more peers. Note in theory the request times out at the network level, +/// measurements have shown, that in practice requests might actually take longer to fail in +/// certain occasions. (The very least, authority discovery is not part of the timeout.) +/// +/// For the time being this value is the same as the timeout on the networking layer, but as this +/// timeout is more soft than the networking one, it might make sense to pick different values as +/// well. +#[cfg(not(test))] +const TIMEOUT_START_NEW_REQUESTS: Duration = CHUNK_REQUEST_TIMEOUT; +#[cfg(test)] +const TIMEOUT_START_NEW_REQUESTS: Duration = Duration::from_millis(100); + +/// The maximum number of times systematic chunk recovery will try making a request for a given +/// (validator,chunk) pair, if the error was not fatal. Added so that we don't get stuck in an +/// infinite retry loop. +pub const SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT: u32 = 2; +/// The maximum number of times regular chunk recovery will try making a request for a given +/// (validator,chunk) pair, if the error was not fatal. Added so that we don't get stuck in an +/// infinite retry loop. +pub const REGULAR_CHUNKS_REQ_RETRY_LIMIT: u32 = 5; + +// Helpful type alias for tracking ongoing chunk requests. +type OngoingRequests = FuturesUndead<( + AuthorityDiscoveryId, + ValidatorIndex, + Result, RequestError>, +)>; + +const fn is_unavailable( + received_chunks: usize, + requesting_chunks: usize, + unrequested_validators: usize, + threshold: usize, +) -> bool { + received_chunks + requesting_chunks + unrequested_validators < threshold +} + +/// Check validity of a chunk. +fn is_chunk_valid(params: &RecoveryParams, chunk: &ErasureChunk) -> bool { + let anticipated_hash = + match branch_hash(¶ms.erasure_root, chunk.proof(), chunk.index.0 as usize) { + Ok(hash) => hash, + Err(e) => { + gum::debug!( + target: LOG_TARGET, + candidate_hash = ?params.candidate_hash, + chunk_index = ?chunk.index, + error = ?e, + "Invalid Merkle proof", + ); + return false + }, + }; + let erasure_chunk_hash = BlakeTwo256::hash(&chunk.chunk); + if anticipated_hash != erasure_chunk_hash { + gum::debug!( + target: LOG_TARGET, + candidate_hash = ?params.candidate_hash, + chunk_index = ?chunk.index, + "Merkle proof mismatch" + ); + return false + } + true +} + +/// Perform the validity checks after recovery. +async fn do_post_recovery_check( + params: &RecoveryParams, + data: AvailableData, + erasure_task_tx: &mut mpsc::Sender, +) -> Result { + match params.post_recovery_check { + PostRecoveryCheck::Reencode => { + // Send request to re-encode the chunks and check merkle root. + let (reencode_tx, reencode_rx) = oneshot::channel(); + erasure_task_tx + .send(ErasureTask::Reencode( + params.n_validators, + params.erasure_root, + data, + reencode_tx, + )) + .await + .map_err(|_| RecoveryError::ChannelClosed)?; + + reencode_rx.await.map_err(|_| RecoveryError::ChannelClosed)?.ok_or_else(|| { + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?params.candidate_hash, + erasure_root = ?params.erasure_root, + "Data recovery error - root mismatch", + ); + RecoveryError::Invalid + }) + }, + PostRecoveryCheck::PovHash => { + let pov = data.pov.clone(); + (pov.hash() == params.pov_hash).then_some(data).ok_or_else(|| { + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?params.candidate_hash, + expected_pov_hash = ?params.pov_hash, + actual_pov_hash = ?pov.hash(), + "Data recovery error - PoV hash mismatch", + ); + RecoveryError::Invalid + }) + }, + } +} + +#[async_trait::async_trait] +/// Common trait for runnable recovery strategies. +pub trait RecoveryStrategy: Send { + /// Main entry point of the strategy. + async fn run( + mut self: Box, + state: &mut State, + sender: &mut Sender, + common_params: &RecoveryParams, + ) -> Result; + + /// Return the name of the strategy for logging purposes. + fn display_name(&self) -> &'static str; + + /// Return the strategy type for use as a metric label. + fn strategy_type(&self) -> &'static str; +} + +/// Utility type used for recording the result of requesting a chunk from a validator. +enum ErrorRecord { + NonFatal(u32), + Fatal, +} + +/// Helper struct used for the `received_chunks` mapping. +/// Compared to `ErasureChunk`, it doesn't need to hold the `ChunkIndex` (because it's the key used +/// for the map) and proof, but needs to hold the `ValidatorIndex` instead. +struct Chunk { + /// The erasure-encoded chunk of data belonging to the candidate block. + chunk: Vec, + /// The validator index that corresponds to this chunk. Not always the same as the chunk index. + validator_index: ValidatorIndex, +} + +/// Intermediate/common data that must be passed between `RecoveryStrategy`s belonging to the +/// same `RecoveryTask`. +pub struct State { + /// Chunks received so far. + /// This MUST be a `BTreeMap` in order for systematic recovery to work (the algorithm assumes + /// that chunks are ordered by their index). If we ever switch this to some non-ordered + /// collection, we need to add a sort step to the systematic recovery. + received_chunks: BTreeMap, + + /// A record of errors returned when requesting a chunk from a validator. + recorded_errors: HashMap<(AuthorityDiscoveryId, ValidatorIndex), ErrorRecord>, +} + +impl State { + pub fn new() -> Self { + Self { received_chunks: BTreeMap::new(), recorded_errors: HashMap::new() } + } + + fn insert_chunk(&mut self, chunk_index: ChunkIndex, chunk: Chunk) { + self.received_chunks.insert(chunk_index, chunk); + } + + fn chunk_count(&self) -> usize { + self.received_chunks.len() + } + + fn systematic_chunk_count(&self, systematic_threshold: usize) -> usize { + self.received_chunks + .range(ChunkIndex(0)..ChunkIndex(systematic_threshold as u32)) + .count() + } + + fn record_error_fatal( + &mut self, + authority_id: AuthorityDiscoveryId, + validator_index: ValidatorIndex, + ) { + self.recorded_errors.insert((authority_id, validator_index), ErrorRecord::Fatal); + } + + fn record_error_non_fatal( + &mut self, + authority_id: AuthorityDiscoveryId, + validator_index: ValidatorIndex, + ) { + self.recorded_errors + .entry((authority_id, validator_index)) + .and_modify(|record| { + if let ErrorRecord::NonFatal(ref mut count) = record { + *count = count.saturating_add(1); + } + }) + .or_insert(ErrorRecord::NonFatal(1)); + } + + fn can_retry_request( + &self, + key: &(AuthorityDiscoveryId, ValidatorIndex), + retry_threshold: u32, + ) -> bool { + match self.recorded_errors.get(key) { + None => true, + Some(entry) => match entry { + ErrorRecord::Fatal => false, + ErrorRecord::NonFatal(count) if *count < retry_threshold => true, + ErrorRecord::NonFatal(_) => false, + }, + } + } + + /// Retrieve the local chunks held in the av-store (should be either 0 or 1). + async fn populate_from_av_store( + &mut self, + params: &RecoveryParams, + sender: &mut Sender, + ) -> Vec<(ValidatorIndex, ChunkIndex)> { + let (tx, rx) = oneshot::channel(); + sender + .send_message(AvailabilityStoreMessage::QueryAllChunks(params.candidate_hash, tx)) + .await; + + match rx.await { + Ok(chunks) => { + // This should either be length 1 or 0. If we had the whole data, + // we wouldn't have reached this stage. + let chunk_indices: Vec<_> = chunks + .iter() + .map(|(validator_index, chunk)| (*validator_index, chunk.index)) + .collect(); + + for (validator_index, chunk) in chunks { + if is_chunk_valid(params, &chunk) { + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?params.candidate_hash, + chunk_index = ?chunk.index, + "Found valid chunk on disk" + ); + self.insert_chunk( + chunk.index, + Chunk { chunk: chunk.chunk, validator_index }, + ); + } else { + gum::error!( + target: LOG_TARGET, + "Loaded invalid chunk from disk! Disk/Db corruption _very_ likely - please fix ASAP!" + ); + }; + } + + chunk_indices + }, + Err(oneshot::Canceled) => { + gum::warn!( + target: LOG_TARGET, + candidate_hash = ?params.candidate_hash, + "Failed to reach the availability store" + ); + + vec![] + }, + } + } + + /// Launch chunk requests in parallel, according to the parameters. + async fn launch_parallel_chunk_requests( + &mut self, + strategy_type: &str, + params: &RecoveryParams, + sender: &mut Sender, + desired_requests_count: usize, + validators: &mut VecDeque<(AuthorityDiscoveryId, ValidatorIndex)>, + requesting_chunks: &mut OngoingRequests, + ) where + Sender: overseer::AvailabilityRecoverySenderTrait, + { + let candidate_hash = params.candidate_hash; + let already_requesting_count = requesting_chunks.len(); + + let to_launch = desired_requests_count - already_requesting_count; + let mut requests = Vec::with_capacity(to_launch); + + gum::trace!( + target: LOG_TARGET, + ?candidate_hash, + "Attempting to launch {} requests", + to_launch + ); + + while requesting_chunks.len() < desired_requests_count { + if let Some((authority_id, validator_index)) = validators.pop_back() { + gum::trace!( + target: LOG_TARGET, + ?authority_id, + ?validator_index, + ?candidate_hash, + "Requesting chunk", + ); + + // Request data. + let raw_request_v2 = + req_res::v2::ChunkFetchingRequest { candidate_hash, index: validator_index }; + let raw_request_v1 = req_res::v1::ChunkFetchingRequest::from(raw_request_v2); + + let (req, res) = OutgoingRequest::new_with_fallback( + Recipient::Authority(authority_id.clone()), + raw_request_v2, + raw_request_v1, + ); + requests.push(Requests::ChunkFetching(req)); + + params.metrics.on_chunk_request_issued(strategy_type); + let timer = params.metrics.time_chunk_request(strategy_type); + let v1_protocol_name = params.req_v1_protocol_name.clone(); + let v2_protocol_name = params.req_v2_protocol_name.clone(); + + let chunk_mapping_enabled = params.chunk_mapping_enabled; + let authority_id_clone = authority_id.clone(); + + requesting_chunks.push(Box::pin(async move { + let _timer = timer; + let res = match res.await { + Ok((bytes, protocol)) => + if v2_protocol_name == protocol { + match req_res::v2::ChunkFetchingResponse::decode(&mut &bytes[..]) { + Ok(req_res::v2::ChunkFetchingResponse::Chunk(chunk)) => + Ok(Some(chunk.into())), + Ok(req_res::v2::ChunkFetchingResponse::NoSuchChunk) => Ok(None), + Err(e) => Err(RequestError::InvalidResponse(e)), + } + } else if v1_protocol_name == protocol { + // V1 protocol version must not be used when chunk mapping node + // feature is enabled, because we can't know the real index of the + // returned chunk. + // This case should never be reached as long as the + // `AvailabilityChunkMapping` feature is only enabled after the + // v1 version is removed. Still, log this. + if chunk_mapping_enabled { + gum::warn!( + target: LOG_TARGET, + ?candidate_hash, + authority_id = ?authority_id_clone, + "Another validator is responding on /req_chunk/1 protocol while the availability chunk \ + mapping feature is enabled in the runtime. All validators must switch to /req_chunk/2." + ); + } + + match req_res::v1::ChunkFetchingResponse::decode(&mut &bytes[..]) { + Ok(req_res::v1::ChunkFetchingResponse::Chunk(chunk)) => + Ok(Some(chunk.recombine_into_chunk(&raw_request_v1))), + Ok(req_res::v1::ChunkFetchingResponse::NoSuchChunk) => Ok(None), + Err(e) => Err(RequestError::InvalidResponse(e)), + } + } else { + Err(RequestError::NetworkError(RequestFailure::UnknownProtocol)) + }, + + Err(e) => Err(e), + }; + + (authority_id, validator_index, res) + })); + } else { + break + } + } + + if requests.len() != 0 { + sender + .send_message(NetworkBridgeTxMessage::SendRequests( + requests, + IfDisconnected::TryConnect, + )) + .await; + } + } + + /// Wait for a sufficient amount of chunks to reconstruct according to the provided `params`. + async fn wait_for_chunks( + &mut self, + strategy_type: &str, + params: &RecoveryParams, + retry_threshold: u32, + validators: &mut VecDeque<(AuthorityDiscoveryId, ValidatorIndex)>, + requesting_chunks: &mut OngoingRequests, + // If supplied, these validators will be used as a backup for requesting chunks. They + // should hold all chunks. Each of them will only be used to query one chunk. + backup_validators: &mut Vec, + // Function that returns `true` when this strategy can conclude. Either if we got enough + // chunks or if it's impossible. + mut can_conclude: impl FnMut( + // Number of validators left in the queue + usize, + // Number of in flight requests + usize, + // Number of valid chunks received so far + usize, + // Number of valid systematic chunks received so far + usize, + ) -> bool, + ) -> (usize, usize) { + let metrics = ¶ms.metrics; + + let mut total_received_responses = 0; + let mut error_count = 0; + + // Wait for all current requests to conclude or time-out, or until we reach enough chunks. + // We also declare requests undead, once `TIMEOUT_START_NEW_REQUESTS` is reached and will + // return in that case for `launch_parallel_requests` to fill up slots again. + while let Some(res) = requesting_chunks.next_with_timeout(TIMEOUT_START_NEW_REQUESTS).await + { + total_received_responses += 1; + + let (authority_id, validator_index, request_result) = res; + + let mut is_error = false; + + match request_result { + Ok(Some(chunk)) => + if is_chunk_valid(params, &chunk) { + metrics.on_chunk_request_succeeded(strategy_type); + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?params.candidate_hash, + ?authority_id, + ?validator_index, + "Received valid chunk", + ); + self.insert_chunk( + chunk.index, + Chunk { chunk: chunk.chunk, validator_index }, + ); + } else { + metrics.on_chunk_request_invalid(strategy_type); + error_count += 1; + // Record that we got an invalid chunk so that subsequent strategies don't + // try requesting this again. + self.record_error_fatal(authority_id.clone(), validator_index); + is_error = true; + }, + Ok(None) => { + metrics.on_chunk_request_no_such_chunk(strategy_type); + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?params.candidate_hash, + ?authority_id, + ?validator_index, + "Validator did not have the chunk", + ); + error_count += 1; + // Record that the validator did not have this chunk so that subsequent + // strategies don't try requesting this again. + self.record_error_fatal(authority_id.clone(), validator_index); + is_error = true; + }, + Err(err) => { + error_count += 1; + + gum::trace!( + target: LOG_TARGET, + candidate_hash= ?params.candidate_hash, + ?err, + ?authority_id, + ?validator_index, + "Failure requesting chunk", + ); + + is_error = true; + + match err { + RequestError::InvalidResponse(_) => { + metrics.on_chunk_request_invalid(strategy_type); + + gum::debug!( + target: LOG_TARGET, + candidate_hash = ?params.candidate_hash, + ?err, + ?authority_id, + ?validator_index, + "Chunk fetching response was invalid", + ); + + // Record that we got an invalid chunk so that this or + // subsequent strategies don't try requesting this again. + self.record_error_fatal(authority_id.clone(), validator_index); + }, + RequestError::NetworkError(err) => { + // No debug logs on general network errors - that became very + // spammy occasionally. + if let RequestFailure::Network(OutboundFailure::Timeout) = err { + metrics.on_chunk_request_timeout(strategy_type); + } else { + metrics.on_chunk_request_error(strategy_type); + } + + // Record that we got a non-fatal error so that this or + // subsequent strategies will retry requesting this only a + // limited number of times. + self.record_error_non_fatal(authority_id.clone(), validator_index); + }, + RequestError::Canceled(_) => { + metrics.on_chunk_request_error(strategy_type); + + // Record that we got a non-fatal error so that this or + // subsequent strategies will retry requesting this only a + // limited number of times. + self.record_error_non_fatal(authority_id.clone(), validator_index); + }, + } + }, + } + + if is_error { + // First, see if we can retry the request. + if self.can_retry_request(&(authority_id.clone(), validator_index), retry_threshold) + { + validators.push_front((authority_id, validator_index)); + } else { + // Otherwise, try requesting from a backer as a backup, if we've not already + // requested the same chunk from it. + + let position = backup_validators.iter().position(|v| { + !self.recorded_errors.contains_key(&(v.clone(), validator_index)) + }); + if let Some(position) = position { + // Use swap_remove because it's faster and we don't care about order here. + let backer = backup_validators.swap_remove(position); + validators.push_front((backer, validator_index)); + } + } + } + + if can_conclude( + validators.len(), + requesting_chunks.total_len(), + self.chunk_count(), + self.systematic_chunk_count(params.systematic_threshold), + ) { + gum::debug!( + target: LOG_TARGET, + validators_len = validators.len(), + candidate_hash = ?params.candidate_hash, + received_chunks_count = ?self.chunk_count(), + requested_chunks_count = ?requesting_chunks.len(), + threshold = ?params.threshold, + "Can conclude availability recovery strategy", + ); + break + } + } + + (total_received_responses, error_count) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::{tests::*, Metrics, RecoveryStrategy, RecoveryTask}; + use assert_matches::assert_matches; + use futures::{ + channel::mpsc::UnboundedReceiver, executor, future, Future, FutureExt, StreamExt, + }; + use parity_scale_codec::Error as DecodingError; + use polkadot_erasure_coding::{recovery_threshold, systematic_recovery_threshold}; + use polkadot_node_network_protocol::request_response::Protocol; + use polkadot_node_primitives::{BlockData, PoV}; + use polkadot_node_subsystem::{AllMessages, TimeoutExt}; + use polkadot_node_subsystem_test_helpers::{ + derive_erasure_chunks_with_proofs_and_root, sender_receiver, TestSubsystemSender, + }; + use polkadot_primitives::{CandidateHash, HeadData, PersistedValidationData}; + use polkadot_primitives_test_helpers::dummy_hash; + use sp_keyring::Sr25519Keyring; + use std::sync::Arc; + + const TIMEOUT: Duration = Duration::from_secs(1); + + impl Default for RecoveryParams { + fn default() -> Self { + let validators = vec![ + Sr25519Keyring::Ferdie, + Sr25519Keyring::Alice.into(), + Sr25519Keyring::Bob.into(), + Sr25519Keyring::Charlie, + Sr25519Keyring::Dave, + Sr25519Keyring::One, + Sr25519Keyring::Two, + ]; + + Self { + validator_authority_keys: validator_authority_id(&validators), + n_validators: validators.len(), + threshold: recovery_threshold(validators.len()).unwrap(), + systematic_threshold: systematic_recovery_threshold(validators.len()).unwrap(), + candidate_hash: CandidateHash(dummy_hash()), + erasure_root: dummy_hash(), + metrics: Metrics::new_dummy(), + bypass_availability_store: false, + post_recovery_check: PostRecoveryCheck::Reencode, + pov_hash: dummy_hash(), + req_v1_protocol_name: "/req_chunk/1".into(), + req_v2_protocol_name: "/req_chunk/2".into(), + chunk_mapping_enabled: true, + } + } + } + + impl RecoveryParams { + fn create_chunks(&mut self) -> Vec { + let available_data = dummy_available_data(); + let (chunks, erasure_root) = derive_erasure_chunks_with_proofs_and_root( + self.n_validators, + &available_data, + |_, _| {}, + ); + + self.erasure_root = erasure_root; + self.pov_hash = available_data.pov.hash(); + + chunks + } + } + + fn dummy_available_data() -> AvailableData { + let validation_data = PersistedValidationData { + parent_head: HeadData(vec![7, 8, 9]), + relay_parent_number: Default::default(), + max_pov_size: 1024, + relay_parent_storage_root: Default::default(), + }; + + AvailableData { + validation_data, + pov: Arc::new(PoV { block_data: BlockData(vec![42; 64]) }), + } + } + + fn test_harness, TestFut: Future>( + receiver_future: impl FnOnce(UnboundedReceiver) -> RecvFut, + test: impl FnOnce(TestSubsystemSender) -> TestFut, + ) { + let (sender, receiver) = sender_receiver(); + + let test_fut = test(sender); + let receiver_future = receiver_future(receiver); + + futures::pin_mut!(test_fut); + futures::pin_mut!(receiver_future); + + executor::block_on(future::join(test_fut, receiver_future)).1 + } + + #[test] + fn test_recorded_errors() { + let retry_threshold = 2; + let mut state = State::new(); + + let alice = Sr25519Keyring::Alice.public(); + let bob = Sr25519Keyring::Bob.public(); + let eve = Sr25519Keyring::Eve.public(); + + assert!(state.can_retry_request(&(alice.into(), 0.into()), retry_threshold)); + assert!(state.can_retry_request(&(alice.into(), 0.into()), 0)); + state.record_error_non_fatal(alice.into(), 0.into()); + assert!(state.can_retry_request(&(alice.into(), 0.into()), retry_threshold)); + state.record_error_non_fatal(alice.into(), 0.into()); + assert!(!state.can_retry_request(&(alice.into(), 0.into()), retry_threshold)); + state.record_error_non_fatal(alice.into(), 0.into()); + assert!(!state.can_retry_request(&(alice.into(), 0.into()), retry_threshold)); + + assert!(state.can_retry_request(&(alice.into(), 0.into()), 5)); + + state.record_error_fatal(bob.into(), 1.into()); + assert!(!state.can_retry_request(&(bob.into(), 1.into()), retry_threshold)); + state.record_error_non_fatal(bob.into(), 1.into()); + assert!(!state.can_retry_request(&(bob.into(), 1.into()), retry_threshold)); + + assert!(state.can_retry_request(&(eve.into(), 4.into()), 0)); + assert!(state.can_retry_request(&(eve.into(), 4.into()), retry_threshold)); + } + + #[test] + fn test_populate_from_av_store() { + let params = RecoveryParams::default(); + + // Failed to reach the av store + { + let params = params.clone(); + let candidate_hash = params.candidate_hash; + let mut state = State::new(); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + assert_matches!( + receiver.next().timeout(TIMEOUT).await.unwrap().unwrap(), + AllMessages::AvailabilityStore(AvailabilityStoreMessage::QueryAllChunks(hash, tx)) => { + assert_eq!(hash, candidate_hash); + drop(tx); + }); + }, + |mut sender| async move { + let local_chunk_indices = + state.populate_from_av_store(¶ms, &mut sender).await; + + assert_eq!(state.chunk_count(), 0); + assert_eq!(local_chunk_indices.len(), 0); + }, + ); + } + + // Found invalid chunk + { + let mut params = params.clone(); + let candidate_hash = params.candidate_hash; + let mut state = State::new(); + let chunks = params.create_chunks(); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + assert_matches!( + receiver.next().timeout(TIMEOUT).await.unwrap().unwrap(), + AllMessages::AvailabilityStore(AvailabilityStoreMessage::QueryAllChunks(hash, tx)) => { + assert_eq!(hash, candidate_hash); + let mut chunk = chunks[0].clone(); + chunk.index = 3.into(); + tx.send(vec![(2.into(), chunk)]).unwrap(); + }); + }, + |mut sender| async move { + let local_chunk_indices = + state.populate_from_av_store(¶ms, &mut sender).await; + + assert_eq!(state.chunk_count(), 0); + assert_eq!(local_chunk_indices.len(), 1); + }, + ); + } + + // Found valid chunk + { + let mut params = params.clone(); + let candidate_hash = params.candidate_hash; + let mut state = State::new(); + let chunks = params.create_chunks(); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + assert_matches!( + receiver.next().timeout(TIMEOUT).await.unwrap().unwrap(), + AllMessages::AvailabilityStore(AvailabilityStoreMessage::QueryAllChunks(hash, tx)) => { + assert_eq!(hash, candidate_hash); + tx.send(vec![(4.into(), chunks[1].clone())]).unwrap(); + }); + }, + |mut sender| async move { + let local_chunk_indices = + state.populate_from_av_store(¶ms, &mut sender).await; + + assert_eq!(state.chunk_count(), 1); + assert_eq!(local_chunk_indices.len(), 1); + }, + ); + } + } + + #[test] + fn test_launch_parallel_chunk_requests() { + let params = RecoveryParams::default(); + let alice: AuthorityDiscoveryId = Sr25519Keyring::Alice.public().into(); + let bob: AuthorityDiscoveryId = Sr25519Keyring::Bob.public().into(); + let eve: AuthorityDiscoveryId = Sr25519Keyring::Eve.public().into(); + + // No validators to request from. + { + let params = params.clone(); + let mut state = State::new(); + let mut ongoing_reqs = OngoingRequests::new(); + let mut validators = VecDeque::new(); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + // Shouldn't send any requests. + assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); + }, + |mut sender| async move { + state + .launch_parallel_chunk_requests( + "regular", + ¶ms, + &mut sender, + 3, + &mut validators, + &mut ongoing_reqs, + ) + .await; + + assert_eq!(ongoing_reqs.total_len(), 0); + }, + ); + } + + // Has validators but no need to request more. + { + let params = params.clone(); + let mut state = State::new(); + let mut ongoing_reqs = OngoingRequests::new(); + let mut validators = VecDeque::new(); + validators.push_back((alice.clone(), ValidatorIndex(1))); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + // Shouldn't send any requests. + assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); + }, + |mut sender| async move { + state + .launch_parallel_chunk_requests( + "regular", + ¶ms, + &mut sender, + 0, + &mut validators, + &mut ongoing_reqs, + ) + .await; + + assert_eq!(ongoing_reqs.total_len(), 0); + }, + ); + } + + // Has validators but no need to request more. + { + let params = params.clone(); + let mut state = State::new(); + let mut ongoing_reqs = OngoingRequests::new(); + ongoing_reqs.push(async { todo!() }.boxed()); + ongoing_reqs.soft_cancel(); + let mut validators = VecDeque::new(); + validators.push_back((alice.clone(), ValidatorIndex(1))); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + // Shouldn't send any requests. + assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); + }, + |mut sender| async move { + state + .launch_parallel_chunk_requests( + "regular", + ¶ms, + &mut sender, + 0, + &mut validators, + &mut ongoing_reqs, + ) + .await; + + assert_eq!(ongoing_reqs.total_len(), 1); + assert_eq!(ongoing_reqs.len(), 0); + }, + ); + } + + // Needs to request more. + { + let params = params.clone(); + let mut state = State::new(); + let mut ongoing_reqs = OngoingRequests::new(); + ongoing_reqs.push(async { todo!() }.boxed()); + ongoing_reqs.soft_cancel(); + ongoing_reqs.push(async { todo!() }.boxed()); + let mut validators = VecDeque::new(); + validators.push_back((alice.clone(), 0.into())); + validators.push_back((bob, 1.into())); + validators.push_back((eve, 2.into())); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + assert_matches!( + receiver.next().timeout(TIMEOUT).await.unwrap().unwrap(), + AllMessages::NetworkBridgeTx(NetworkBridgeTxMessage::SendRequests(requests, _)) if requests.len() +== 3 ); + }, + |mut sender| async move { + state + .launch_parallel_chunk_requests( + "regular", + ¶ms, + &mut sender, + 10, + &mut validators, + &mut ongoing_reqs, + ) + .await; + + assert_eq!(ongoing_reqs.total_len(), 5); + assert_eq!(ongoing_reqs.len(), 4); + }, + ); + } + + // Check network protocol versioning. + { + let params = params.clone(); + let mut state = State::new(); + let mut ongoing_reqs = OngoingRequests::new(); + let mut validators = VecDeque::new(); + validators.push_back((alice, 0.into())); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + match receiver.next().timeout(TIMEOUT).await.unwrap().unwrap() { + AllMessages::NetworkBridgeTx(NetworkBridgeTxMessage::SendRequests( + mut requests, + _, + )) => { + assert_eq!(requests.len(), 1); + // By default, we should use the new protocol version with a fallback on + // the older one. + let (protocol, request) = requests.remove(0).encode_request(); + assert_eq!(protocol, Protocol::ChunkFetchingV2); + assert_eq!( + request.fallback_request.unwrap().1, + Protocol::ChunkFetchingV1 + ); + }, + _ => unreachable!(), + } + }, + |mut sender| async move { + state + .launch_parallel_chunk_requests( + "regular", + ¶ms, + &mut sender, + 10, + &mut validators, + &mut ongoing_reqs, + ) + .await; + + assert_eq!(ongoing_reqs.total_len(), 1); + assert_eq!(ongoing_reqs.len(), 1); + }, + ); + } + } + + #[test] + fn test_wait_for_chunks() { + let params = RecoveryParams::default(); + let retry_threshold = 2; + + // No ongoing requests. + { + let params = params.clone(); + let mut state = State::new(); + let mut ongoing_reqs = OngoingRequests::new(); + let mut validators = VecDeque::new(); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + // Shouldn't send any requests. + assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); + }, + |_| async move { + let (total_responses, error_count) = state + .wait_for_chunks( + "regular", + ¶ms, + retry_threshold, + &mut validators, + &mut ongoing_reqs, + &mut vec![], + |_, _, _, _| false, + ) + .await; + assert_eq!(total_responses, 0); + assert_eq!(error_count, 0); + assert_eq!(state.chunk_count(), 0); + }, + ); + } + + // Complex scenario. + { + let mut params = params.clone(); + let chunks = params.create_chunks(); + let mut state = State::new(); + let mut ongoing_reqs = OngoingRequests::new(); + ongoing_reqs.push( + future::ready(( + params.validator_authority_keys[0].clone(), + 0.into(), + Ok(Some(chunks[0].clone())), + )) + .boxed(), + ); + ongoing_reqs.soft_cancel(); + ongoing_reqs.push( + future::ready(( + params.validator_authority_keys[1].clone(), + 1.into(), + Ok(Some(chunks[1].clone())), + )) + .boxed(), + ); + ongoing_reqs.push( + future::ready((params.validator_authority_keys[2].clone(), 2.into(), Ok(None))) + .boxed(), + ); + ongoing_reqs.push( + future::ready(( + params.validator_authority_keys[3].clone(), + 3.into(), + Err(RequestError::from(DecodingError::from("err"))), + )) + .boxed(), + ); + ongoing_reqs.push( + future::ready(( + params.validator_authority_keys[4].clone(), + 4.into(), + Err(RequestError::NetworkError(RequestFailure::NotConnected)), + )) + .boxed(), + ); + + let mut validators: VecDeque<_> = (5..params.n_validators as u32) + .map(|i| (params.validator_authority_keys[i as usize].clone(), i.into())) + .collect(); + validators.push_back(( + Sr25519Keyring::AliceStash.public().into(), + ValidatorIndex(params.n_validators as u32), + )); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + // Shouldn't send any requests. + assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); + }, + |_| async move { + let (total_responses, error_count) = state + .wait_for_chunks( + "regular", + ¶ms, + retry_threshold, + &mut validators, + &mut ongoing_reqs, + &mut vec![], + |_, _, _, _| false, + ) + .await; + assert_eq!(total_responses, 5); + assert_eq!(error_count, 3); + assert_eq!(state.chunk_count(), 2); + + let mut expected_validators: VecDeque<_> = (4..params.n_validators as u32) + .map(|i| (params.validator_authority_keys[i as usize].clone(), i.into())) + .collect(); + expected_validators.push_back(( + Sr25519Keyring::AliceStash.public().into(), + ValidatorIndex(params.n_validators as u32), + )); + + assert_eq!(validators, expected_validators); + + // This time we'll go over the recoverable error threshold. + ongoing_reqs.push( + future::ready(( + params.validator_authority_keys[4].clone(), + 4.into(), + Err(RequestError::NetworkError(RequestFailure::NotConnected)), + )) + .boxed(), + ); + + let (total_responses, error_count) = state + .wait_for_chunks( + "regular", + ¶ms, + retry_threshold, + &mut validators, + &mut ongoing_reqs, + &mut vec![], + |_, _, _, _| false, + ) + .await; + assert_eq!(total_responses, 1); + assert_eq!(error_count, 1); + assert_eq!(state.chunk_count(), 2); + + validators.pop_front(); + let mut expected_validators: VecDeque<_> = (5..params.n_validators as u32) + .map(|i| (params.validator_authority_keys[i as usize].clone(), i.into())) + .collect(); + expected_validators.push_back(( + Sr25519Keyring::AliceStash.public().into(), + ValidatorIndex(params.n_validators as u32), + )); + + assert_eq!(validators, expected_validators); + + // Check that can_conclude returning true terminates the loop. + let (total_responses, error_count) = state + .wait_for_chunks( + "regular", + ¶ms, + retry_threshold, + &mut validators, + &mut ongoing_reqs, + &mut vec![], + |_, _, _, _| true, + ) + .await; + assert_eq!(total_responses, 0); + assert_eq!(error_count, 0); + assert_eq!(state.chunk_count(), 2); + + assert_eq!(validators, expected_validators); + }, + ); + } + + // Complex scenario with backups in the backing group. + { + let mut params = params.clone(); + let chunks = params.create_chunks(); + let mut state = State::new(); + let mut ongoing_reqs = OngoingRequests::new(); + ongoing_reqs.push( + future::ready(( + params.validator_authority_keys[0].clone(), + 0.into(), + Ok(Some(chunks[0].clone())), + )) + .boxed(), + ); + ongoing_reqs.soft_cancel(); + ongoing_reqs.push( + future::ready(( + params.validator_authority_keys[1].clone(), + 1.into(), + Ok(Some(chunks[1].clone())), + )) + .boxed(), + ); + ongoing_reqs.push( + future::ready((params.validator_authority_keys[2].clone(), 2.into(), Ok(None))) + .boxed(), + ); + ongoing_reqs.push( + future::ready(( + params.validator_authority_keys[3].clone(), + 3.into(), + Err(RequestError::from(DecodingError::from("err"))), + )) + .boxed(), + ); + ongoing_reqs.push( + future::ready(( + params.validator_authority_keys[4].clone(), + 4.into(), + Err(RequestError::NetworkError(RequestFailure::NotConnected)), + )) + .boxed(), + ); + + let mut validators: VecDeque<_> = (5..params.n_validators as u32) + .map(|i| (params.validator_authority_keys[i as usize].clone(), i.into())) + .collect(); + validators.push_back(( + Sr25519Keyring::Eve.public().into(), + ValidatorIndex(params.n_validators as u32), + )); + + let mut backup_backers = vec![ + params.validator_authority_keys[2].clone(), + params.validator_authority_keys[0].clone(), + params.validator_authority_keys[4].clone(), + params.validator_authority_keys[3].clone(), + Sr25519Keyring::AliceStash.public().into(), + Sr25519Keyring::BobStash.public().into(), + ]; + + test_harness( + |mut receiver: UnboundedReceiver| async move { + // Shouldn't send any requests. + assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); + }, + |_| async move { + let (total_responses, error_count) = state + .wait_for_chunks( + "regular", + ¶ms, + retry_threshold, + &mut validators, + &mut ongoing_reqs, + &mut backup_backers, + |_, _, _, _| false, + ) + .await; + assert_eq!(total_responses, 5); + assert_eq!(error_count, 3); + assert_eq!(state.chunk_count(), 2); + + let mut expected_validators: VecDeque<_> = (5..params.n_validators as u32) + .map(|i| (params.validator_authority_keys[i as usize].clone(), i.into())) + .collect(); + expected_validators.push_back(( + Sr25519Keyring::Eve.public().into(), + ValidatorIndex(params.n_validators as u32), + )); + // We picked a backer as a backup for chunks 2 and 3. + expected_validators + .push_front((params.validator_authority_keys[0].clone(), 2.into())); + expected_validators + .push_front((params.validator_authority_keys[2].clone(), 3.into())); + expected_validators + .push_front((params.validator_authority_keys[4].clone(), 4.into())); + + assert_eq!(validators, expected_validators); + + // This time we'll go over the recoverable error threshold for chunk 4. + ongoing_reqs.push( + future::ready(( + params.validator_authority_keys[4].clone(), + 4.into(), + Err(RequestError::NetworkError(RequestFailure::NotConnected)), + )) + .boxed(), + ); + + validators.pop_front(); + + let (total_responses, error_count) = state + .wait_for_chunks( + "regular", + ¶ms, + retry_threshold, + &mut validators, + &mut ongoing_reqs, + &mut backup_backers, + |_, _, _, _| false, + ) + .await; + assert_eq!(total_responses, 1); + assert_eq!(error_count, 1); + assert_eq!(state.chunk_count(), 2); + + expected_validators.pop_front(); + expected_validators + .push_front((Sr25519Keyring::AliceStash.public().into(), 4.into())); + + assert_eq!(validators, expected_validators); + }, + ); + } + } + + #[test] + fn test_recovery_strategy_run() { + let params = RecoveryParams::default(); + + struct GoodStrategy; + #[async_trait::async_trait] + impl RecoveryStrategy for GoodStrategy { + fn display_name(&self) -> &'static str { + "GoodStrategy" + } + + fn strategy_type(&self) -> &'static str { + "good_strategy" + } + + async fn run( + mut self: Box, + _state: &mut State, + _sender: &mut Sender, + _common_params: &RecoveryParams, + ) -> Result { + Ok(dummy_available_data()) + } + } + + struct UnavailableStrategy; + #[async_trait::async_trait] + impl RecoveryStrategy + for UnavailableStrategy + { + fn display_name(&self) -> &'static str { + "UnavailableStrategy" + } + + fn strategy_type(&self) -> &'static str { + "unavailable_strategy" + } + + async fn run( + mut self: Box, + _state: &mut State, + _sender: &mut Sender, + _common_params: &RecoveryParams, + ) -> Result { + Err(RecoveryError::Unavailable) + } + } + + struct InvalidStrategy; + #[async_trait::async_trait] + impl RecoveryStrategy + for InvalidStrategy + { + fn display_name(&self) -> &'static str { + "InvalidStrategy" + } + + fn strategy_type(&self) -> &'static str { + "invalid_strategy" + } + + async fn run( + mut self: Box, + _state: &mut State, + _sender: &mut Sender, + _common_params: &RecoveryParams, + ) -> Result { + Err(RecoveryError::Invalid) + } + } + + // No recovery strategies. + { + let mut params = params.clone(); + let strategies = VecDeque::new(); + params.bypass_availability_store = true; + + test_harness( + |mut receiver: UnboundedReceiver| async move { + // Shouldn't send any requests. + assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); + }, + |sender| async move { + let task = RecoveryTask::new(sender, params, strategies); + + assert_eq!(task.run().await.unwrap_err(), RecoveryError::Unavailable); + }, + ); + } + + // If we have the data in av-store, returns early. + { + let params = params.clone(); + let strategies = VecDeque::new(); + let candidate_hash = params.candidate_hash; + + test_harness( + |mut receiver: UnboundedReceiver| async move { + assert_matches!( + receiver.next().timeout(TIMEOUT).await.unwrap().unwrap(), + AllMessages::AvailabilityStore(AvailabilityStoreMessage::QueryAvailableData(hash, tx)) => { + assert_eq!(hash, candidate_hash); + tx.send(Some(dummy_available_data())).unwrap(); + }); + }, + |sender| async move { + let task = RecoveryTask::new(sender, params, strategies); + + assert_eq!(task.run().await.unwrap(), dummy_available_data()); + }, + ); + } + + // Strategy returning `RecoveryError::Invalid`` will short-circuit the entire task. + { + let mut params = params.clone(); + params.bypass_availability_store = true; + let mut strategies: VecDeque>> = + VecDeque::new(); + strategies.push_back(Box::new(InvalidStrategy)); + strategies.push_back(Box::new(GoodStrategy)); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + // Shouldn't send any requests. + assert!(receiver.next().timeout(TIMEOUT).await.unwrap().is_none()); + }, + |sender| async move { + let task = RecoveryTask::new(sender, params, strategies); + + assert_eq!(task.run().await.unwrap_err(), RecoveryError::Invalid); + }, + ); + } + + // Strategy returning `Unavailable` will fall back to the next one. + { + let params = params.clone(); + let candidate_hash = params.candidate_hash; + let mut strategies: VecDeque>> = + VecDeque::new(); + strategies.push_back(Box::new(UnavailableStrategy)); + strategies.push_back(Box::new(GoodStrategy)); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + assert_matches!( + receiver.next().timeout(TIMEOUT).await.unwrap().unwrap(), + AllMessages::AvailabilityStore(AvailabilityStoreMessage::QueryAvailableData(hash, tx)) => { + assert_eq!(hash, candidate_hash); + tx.send(Some(dummy_available_data())).unwrap(); + }); + }, + |sender| async move { + let task = RecoveryTask::new(sender, params, strategies); + + assert_eq!(task.run().await.unwrap(), dummy_available_data()); + }, + ); + } + + // More complex scenario. + { + let params = params.clone(); + let candidate_hash = params.candidate_hash; + let mut strategies: VecDeque>> = + VecDeque::new(); + strategies.push_back(Box::new(UnavailableStrategy)); + strategies.push_back(Box::new(UnavailableStrategy)); + strategies.push_back(Box::new(GoodStrategy)); + strategies.push_back(Box::new(InvalidStrategy)); + + test_harness( + |mut receiver: UnboundedReceiver| async move { + assert_matches!( + receiver.next().timeout(TIMEOUT).await.unwrap().unwrap(), + AllMessages::AvailabilityStore(AvailabilityStoreMessage::QueryAvailableData(hash, tx)) => { + assert_eq!(hash, candidate_hash); + tx.send(Some(dummy_available_data())).unwrap(); + }); + }, + |sender| async move { + let task = RecoveryTask::new(sender, params, strategies); + + assert_eq!(task.run().await.unwrap(), dummy_available_data()); + }, + ); + } + } + + #[test] + fn test_is_unavailable() { + assert_eq!(is_unavailable(0, 0, 0, 0), false); + assert_eq!(is_unavailable(2, 2, 2, 0), false); + // Already reached the threshold. + assert_eq!(is_unavailable(3, 0, 10, 3), false); + assert_eq!(is_unavailable(3, 2, 0, 3), false); + assert_eq!(is_unavailable(3, 2, 10, 3), false); + // It's still possible to reach the threshold + assert_eq!(is_unavailable(0, 0, 10, 3), false); + assert_eq!(is_unavailable(0, 0, 3, 3), false); + assert_eq!(is_unavailable(1, 1, 1, 3), false); + // Not possible to reach the threshold + assert_eq!(is_unavailable(0, 0, 0, 3), true); + assert_eq!(is_unavailable(2, 3, 2, 10), true); + } +} diff --git a/polkadot/node/network/availability-recovery/src/task/strategy/systematic.rs b/polkadot/node/network/availability-recovery/src/task/strategy/systematic.rs new file mode 100644 index 000000000000..65cac11fd5f0 --- /dev/null +++ b/polkadot/node/network/availability-recovery/src/task/strategy/systematic.rs @@ -0,0 +1,350 @@ +// Copyright (C) Parity Technologies (UK) Ltd. +// This file is part of Polkadot. + +// Polkadot is free software: you can redistribute it and/or modify +// it under the terms of the GNU General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. + +// Polkadot is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU General Public License for more details. + +// You should have received a copy of the GNU General Public License +// along with Polkadot. If not, see . + +use crate::{ + futures_undead::FuturesUndead, + task::{ + strategy::{ + do_post_recovery_check, is_unavailable, OngoingRequests, N_PARALLEL, + SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT, + }, + RecoveryParams, RecoveryStrategy, State, + }, + ErasureTask, LOG_TARGET, +}; + +use polkadot_node_primitives::AvailableData; +use polkadot_node_subsystem::{overseer, RecoveryError}; +use polkadot_primitives::{ChunkIndex, ValidatorIndex}; + +use std::collections::VecDeque; + +/// Parameters needed for fetching systematic chunks. +pub struct FetchSystematicChunksParams { + /// Validators that hold the systematic chunks. + pub validators: Vec<(ChunkIndex, ValidatorIndex)>, + /// Validators in the backing group, to be used as a backup for requesting systematic chunks. + pub backers: Vec, + /// Channel to the erasure task handler. + pub erasure_task_tx: futures::channel::mpsc::Sender, +} + +/// `RecoveryStrategy` that attempts to recover the systematic chunks from the validators that +/// hold them, in order to bypass the erasure code reconstruction step, which is costly. +pub struct FetchSystematicChunks { + /// Systematic recovery threshold. + threshold: usize, + /// Validators that hold the systematic chunks. + validators: Vec<(ChunkIndex, ValidatorIndex)>, + /// Backers. to be used as a backup. + backers: Vec, + /// Collection of in-flight requests. + requesting_chunks: OngoingRequests, + /// Channel to the erasure task handler. + erasure_task_tx: futures::channel::mpsc::Sender, +} + +impl FetchSystematicChunks { + /// Instantiate a new systematic chunks strategy. + pub fn new(params: FetchSystematicChunksParams) -> Self { + Self { + threshold: params.validators.len(), + validators: params.validators, + backers: params.backers, + requesting_chunks: FuturesUndead::new(), + erasure_task_tx: params.erasure_task_tx, + } + } + + fn is_unavailable( + unrequested_validators: usize, + in_flight_requests: usize, + systematic_chunk_count: usize, + threshold: usize, + ) -> bool { + is_unavailable( + systematic_chunk_count, + in_flight_requests, + unrequested_validators, + threshold, + ) + } + + /// Desired number of parallel requests. + /// + /// For the given threshold (total required number of chunks) get the desired number of + /// requests we want to have running in parallel at this time. + fn get_desired_request_count(&self, chunk_count: usize, threshold: usize) -> usize { + // Upper bound for parallel requests. + let max_requests_boundary = std::cmp::min(N_PARALLEL, threshold); + // How many chunks are still needed? + let remaining_chunks = threshold.saturating_sub(chunk_count); + // Actual number of requests we want to have in flight in parallel: + // We don't have to make up for any error rate, as an error fetching a systematic chunk + // results in failure of the entire strategy. + std::cmp::min(max_requests_boundary, remaining_chunks) + } + + async fn attempt_systematic_recovery( + &mut self, + state: &mut State, + common_params: &RecoveryParams, + ) -> Result { + let strategy_type = RecoveryStrategy::::strategy_type(self); + let recovery_duration = common_params.metrics.time_erasure_recovery(strategy_type); + let reconstruct_duration = common_params.metrics.time_erasure_reconstruct(strategy_type); + let chunks = state + .received_chunks + .range( + ChunkIndex(0).. + ChunkIndex( + u32::try_from(self.threshold) + .expect("validator count should not exceed u32"), + ), + ) + .map(|(_, chunk)| &chunk.chunk[..]) + .collect::>(); + + let available_data = polkadot_erasure_coding::reconstruct_from_systematic_v1( + common_params.n_validators, + chunks, + ); + + match available_data { + Ok(data) => { + drop(reconstruct_duration); + + // Attempt post-recovery check. + do_post_recovery_check(common_params, data, &mut self.erasure_task_tx) + .await + .map_err(|e| { + recovery_duration.map(|rd| rd.stop_and_discard()); + e + }) + .map(|data| { + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + erasure_root = ?common_params.erasure_root, + "Data recovery from systematic chunks complete", + ); + data + }) + }, + Err(err) => { + reconstruct_duration.map(|rd| rd.stop_and_discard()); + recovery_duration.map(|rd| rd.stop_and_discard()); + + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + erasure_root = ?common_params.erasure_root, + ?err, + "Systematic data recovery error", + ); + + Err(RecoveryError::Invalid) + }, + } + } +} + +#[async_trait::async_trait] +impl RecoveryStrategy + for FetchSystematicChunks +{ + fn display_name(&self) -> &'static str { + "Fetch systematic chunks" + } + + fn strategy_type(&self) -> &'static str { + "systematic_chunks" + } + + async fn run( + mut self: Box, + state: &mut State, + sender: &mut Sender, + common_params: &RecoveryParams, + ) -> Result { + // First query the store for any chunks we've got. + if !common_params.bypass_availability_store { + let local_chunk_indices = state.populate_from_av_store(common_params, sender).await; + + for (_, our_c_index) in &local_chunk_indices { + // If we are among the systematic validators but hold an invalid chunk, we cannot + // perform the systematic recovery. Fall through to the next strategy. + if self.validators.iter().any(|(c_index, _)| c_index == our_c_index) && + !state.received_chunks.contains_key(our_c_index) + { + gum::debug!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + erasure_root = ?common_params.erasure_root, + requesting = %self.requesting_chunks.len(), + total_requesting = %self.requesting_chunks.total_len(), + n_validators = %common_params.n_validators, + chunk_index = ?our_c_index, + "Systematic chunk recovery is not possible. We are among the systematic validators but hold an invalid chunk", + ); + return Err(RecoveryError::Unavailable) + } + } + } + + // No need to query the validators that have the chunks we already received or that we know + // don't have the data from previous strategies. + self.validators.retain(|(c_index, v_index)| { + !state.received_chunks.contains_key(c_index) && + state.can_retry_request( + &(common_params.validator_authority_keys[v_index.0 as usize].clone(), *v_index), + SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT, + ) + }); + + let mut systematic_chunk_count = state + .received_chunks + .range(ChunkIndex(0)..ChunkIndex(self.threshold as u32)) + .count(); + + // Safe to `take` here, as we're consuming `self` anyway and we're not using the + // `validators` or `backers` fields in other methods. + let mut validators_queue: VecDeque<_> = std::mem::take(&mut self.validators) + .into_iter() + .map(|(_, validator_index)| { + ( + common_params.validator_authority_keys[validator_index.0 as usize].clone(), + validator_index, + ) + }) + .collect(); + let mut backers: Vec<_> = std::mem::take(&mut self.backers) + .into_iter() + .map(|validator_index| { + common_params.validator_authority_keys[validator_index.0 as usize].clone() + }) + .collect(); + + loop { + // If received_chunks has `systematic_chunk_threshold` entries, attempt to recover the + // data. + if systematic_chunk_count >= self.threshold { + return self.attempt_systematic_recovery::(state, common_params).await + } + + if Self::is_unavailable( + validators_queue.len(), + self.requesting_chunks.total_len(), + systematic_chunk_count, + self.threshold, + ) { + gum::debug!( + target: LOG_TARGET, + candidate_hash = ?common_params.candidate_hash, + erasure_root = ?common_params.erasure_root, + %systematic_chunk_count, + requesting = %self.requesting_chunks.len(), + total_requesting = %self.requesting_chunks.total_len(), + n_validators = %common_params.n_validators, + systematic_threshold = ?self.threshold, + "Data recovery from systematic chunks is not possible", + ); + + return Err(RecoveryError::Unavailable) + } + + let desired_requests_count = + self.get_desired_request_count(systematic_chunk_count, self.threshold); + let already_requesting_count = self.requesting_chunks.len(); + gum::debug!( + target: LOG_TARGET, + ?common_params.candidate_hash, + ?desired_requests_count, + total_received = ?systematic_chunk_count, + systematic_threshold = ?self.threshold, + ?already_requesting_count, + "Requesting systematic availability chunks for a candidate", + ); + + let strategy_type = RecoveryStrategy::::strategy_type(&*self); + + state + .launch_parallel_chunk_requests( + strategy_type, + common_params, + sender, + desired_requests_count, + &mut validators_queue, + &mut self.requesting_chunks, + ) + .await; + + let _ = state + .wait_for_chunks( + strategy_type, + common_params, + SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT, + &mut validators_queue, + &mut self.requesting_chunks, + &mut backers, + |unrequested_validators, + in_flight_reqs, + // Don't use this chunk count, as it may contain non-systematic chunks. + _chunk_count, + new_systematic_chunk_count| { + systematic_chunk_count = new_systematic_chunk_count; + + let is_unavailable = Self::is_unavailable( + unrequested_validators, + in_flight_reqs, + systematic_chunk_count, + self.threshold, + ); + + systematic_chunk_count >= self.threshold || is_unavailable + }, + ) + .await; + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use polkadot_erasure_coding::systematic_recovery_threshold; + + #[test] + fn test_get_desired_request_count() { + let num_validators = 100; + let threshold = systematic_recovery_threshold(num_validators).unwrap(); + let (erasure_task_tx, _erasure_task_rx) = futures::channel::mpsc::channel(16); + + let systematic_chunks_task = FetchSystematicChunks::new(FetchSystematicChunksParams { + validators: vec![(1.into(), 1.into()); num_validators], + backers: vec![], + erasure_task_tx, + }); + assert_eq!(systematic_chunks_task.get_desired_request_count(0, threshold), threshold); + assert_eq!(systematic_chunks_task.get_desired_request_count(5, threshold), threshold - 5); + assert_eq!( + systematic_chunks_task.get_desired_request_count(num_validators * 2, threshold), + 0 + ); + assert_eq!(systematic_chunks_task.get_desired_request_count(0, N_PARALLEL * 2), N_PARALLEL); + assert_eq!(systematic_chunks_task.get_desired_request_count(N_PARALLEL, N_PARALLEL + 2), 2); + } +} diff --git a/polkadot/node/network/availability-recovery/src/tests.rs b/polkadot/node/network/availability-recovery/src/tests.rs index 84f104a2f7bb..7fd5e3ccd046 100644 --- a/polkadot/node/network/availability-recovery/src/tests.rs +++ b/polkadot/node/network/availability-recovery/src/tests.rs @@ -17,7 +17,7 @@ use crate::task::{REGULAR_CHUNKS_REQ_RETRY_LIMIT, SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT}; use super::*; -use std::{sync::Arc, time::Duration}; +use std::{result::Result, sync::Arc, time::Duration}; use assert_matches::assert_matches; use futures::{executor, future}; @@ -26,13 +26,14 @@ use rstest::rstest; use parity_scale_codec::Encode; use polkadot_node_network_protocol::request_response::{ - self as req_res, v1::AvailableDataFetchingRequest, IncomingRequest, Protocol, Recipient, - ReqProtocolNames, Requests, + self as req_res, + v1::{AvailableDataFetchingRequest, ChunkResponse}, + IncomingRequest, Protocol, Recipient, ReqProtocolNames, Requests, }; -use polkadot_node_primitives::{BlockData, PoV, Proof}; +use polkadot_node_primitives::{BlockData, ErasureChunk, PoV, Proof}; use polkadot_node_subsystem::messages::{ - AllMessages, ChainApiMessage, NetworkBridgeTxMessage, RuntimeApiMessage, RuntimeApiRequest, + AllMessages, NetworkBridgeTxMessage, RuntimeApiMessage, RuntimeApiRequest, }; use polkadot_node_subsystem_test_helpers::{ derive_erasure_chunks_with_proofs_and_root, make_subsystem_context, mock::new_leaf, @@ -41,7 +42,8 @@ use polkadot_node_subsystem_test_helpers::{ use polkadot_node_subsystem_util::TimeoutExt; use polkadot_primitives::{ vstaging::{node_features, NodeFeatures}, - AuthorityDiscoveryId, Hash, HeadData, IndexedVec, PersistedValidationData, ValidatorId, + AuthorityDiscoveryId, ExecutorParams, Hash, HeadData, IndexedVec, PersistedValidationData, + SessionInfo, ValidatorId, }; use polkadot_primitives_test_helpers::{dummy_candidate_receipt, dummy_hash}; use sc_network::{IfDisconnected, OutboundFailure, ProtocolName, RequestFailure}; @@ -49,6 +51,92 @@ use sp_keyring::Sr25519Keyring; type VirtualOverseer = TestSubsystemContextHandle; +// Implement some helper constructors for the AvailabilityRecoverySubsystem + +/// Create a new instance of `AvailabilityRecoverySubsystem` which starts with a fast path to +/// request data from backers. +fn with_fast_path( + req_receiver: IncomingRequestReceiver, + req_protocol_names: &ReqProtocolNames, + metrics: Metrics, +) -> AvailabilityRecoverySubsystem { + AvailabilityRecoverySubsystem::with_recovery_strategy_kind( + req_receiver, + req_protocol_names, + metrics, + RecoveryStrategyKind::BackersFirstAlways, + ) +} + +/// Create a new instance of `AvailabilityRecoverySubsystem` which requests only chunks +fn with_chunks_only( + req_receiver: IncomingRequestReceiver, + req_protocol_names: &ReqProtocolNames, + metrics: Metrics, +) -> AvailabilityRecoverySubsystem { + AvailabilityRecoverySubsystem::with_recovery_strategy_kind( + req_receiver, + req_protocol_names, + metrics, + RecoveryStrategyKind::ChunksAlways, + ) +} + +/// Create a new instance of `AvailabilityRecoverySubsystem` which requests chunks if PoV is +/// above a threshold. +fn with_chunks_if_pov_large( + req_receiver: IncomingRequestReceiver, + req_protocol_names: &ReqProtocolNames, + metrics: Metrics, +) -> AvailabilityRecoverySubsystem { + AvailabilityRecoverySubsystem::with_recovery_strategy_kind( + req_receiver, + req_protocol_names, + metrics, + RecoveryStrategyKind::BackersFirstIfSizeLower(SMALL_POV_LIMIT), + ) +} + +/// Create a new instance of `AvailabilityRecoverySubsystem` which requests systematic chunks if +/// PoV is above a threshold. +fn with_systematic_chunks_if_pov_large( + req_receiver: IncomingRequestReceiver, + req_protocol_names: &ReqProtocolNames, + metrics: Metrics, +) -> AvailabilityRecoverySubsystem { + AvailabilityRecoverySubsystem::for_validator(req_receiver, req_protocol_names, metrics) +} + +/// Create a new instance of `AvailabilityRecoverySubsystem` which first requests full data +/// from backers, with a fallback to recover from systematic chunks. +fn with_fast_path_then_systematic_chunks( + req_receiver: IncomingRequestReceiver, + req_protocol_names: &ReqProtocolNames, + metrics: Metrics, +) -> AvailabilityRecoverySubsystem { + AvailabilityRecoverySubsystem::with_recovery_strategy_kind( + req_receiver, + req_protocol_names, + metrics, + RecoveryStrategyKind::BackersThenSystematicChunks, + ) +} + +/// Create a new instance of `AvailabilityRecoverySubsystem` which first attempts to request +/// systematic chunks, with a fallback to requesting regular chunks. +fn with_systematic_chunks( + req_receiver: IncomingRequestReceiver, + req_protocol_names: &ReqProtocolNames, + metrics: Metrics, +) -> AvailabilityRecoverySubsystem { + AvailabilityRecoverySubsystem::with_recovery_strategy_kind( + req_receiver, + req_protocol_names, + metrics, + RecoveryStrategyKind::SystematicChunks, + ) +} + // Deterministic genesis hash for protocol names const GENESIS_HASH: Hash = Hash::repeat_byte(0xff); @@ -162,15 +250,110 @@ struct TestState { current: Hash, candidate: CandidateReceipt, session_index: SessionIndex, + core_index: CoreIndex, + node_features: NodeFeatures, persisted_validation_data: PersistedValidationData, available_data: AvailableData, - chunks: Vec, - invalid_chunks: Vec, + chunks: IndexedVec, + invalid_chunks: IndexedVec, } impl TestState { + fn new(node_features: NodeFeatures) -> Self { + let validators = vec![ + Sr25519Keyring::Ferdie, // <- this node, role: validator + Sr25519Keyring::Alice, + Sr25519Keyring::Bob, + Sr25519Keyring::Charlie, + Sr25519Keyring::Dave, + Sr25519Keyring::One, + Sr25519Keyring::Two, + ]; + + let validator_public = validator_pubkeys(&validators); + let validator_authority_id = validator_authority_id(&validators); + let validator_groups = vec![ + vec![1.into(), 0.into(), 3.into(), 4.into()], + vec![5.into(), 6.into()], + vec![2.into()], + ]; + + let current = Hash::repeat_byte(1); + + let mut candidate = dummy_candidate_receipt(dummy_hash()); + + let session_index = 10; + + let persisted_validation_data = PersistedValidationData { + parent_head: HeadData(vec![7, 8, 9]), + relay_parent_number: Default::default(), + max_pov_size: 1024, + relay_parent_storage_root: Default::default(), + }; + + let pov = PoV { block_data: BlockData(vec![42; 64]) }; + + let available_data = AvailableData { + validation_data: persisted_validation_data.clone(), + pov: Arc::new(pov), + }; + + let core_index = CoreIndex(2); + + let (chunks, erasure_root) = derive_erasure_chunks_with_proofs_and_root( + validators.len(), + &available_data, + |_, _| {}, + ); + let chunks = map_chunks(chunks, &node_features, validators.len(), core_index); + + // Mess around: + let invalid_chunks = chunks + .iter() + .cloned() + .map(|mut chunk| { + if chunk.chunk.len() >= 2 && chunk.chunk[0] != chunk.chunk[1] { + chunk.chunk[0] = chunk.chunk[1]; + } else if chunk.chunk.len() >= 1 { + chunk.chunk[0] = !chunk.chunk[0]; + } else { + chunk.proof = Proof::dummy_proof(); + } + chunk + }) + .collect(); + debug_assert_ne!(chunks, invalid_chunks); + + candidate.descriptor.erasure_root = erasure_root; + candidate.descriptor.relay_parent = Hash::repeat_byte(10); + candidate.descriptor.pov_hash = Hash::repeat_byte(3); + + Self { + validators, + validator_public, + validator_authority_id, + validator_groups: IndexedVec::>::try_from( + validator_groups, + ) + .unwrap(), + current, + candidate, + session_index, + core_index, + node_features, + persisted_validation_data, + available_data, + chunks, + invalid_chunks, + } + } + + fn with_empty_node_features() -> Self { + Self::new(NodeFeatures::EMPTY) + } + fn threshold(&self) -> usize { recovery_threshold(self.validators.len()).unwrap() } @@ -213,32 +396,24 @@ impl TestState { }))).unwrap(); } ); - } - - async fn test_runtime_api_node_features(&self, virtual_overseer: &mut VirtualOverseer) { - let mut node_features = NodeFeatures::new(); - node_features - .resize(node_features::FeatureIndex::AvailabilityChunkShuffling as usize + 1, false); - node_features - .set(node_features::FeatureIndex::AvailabilityChunkShuffling as u8 as usize, true); - assert_matches!( overseer_recv(virtual_overseer).await, AllMessages::RuntimeApi(RuntimeApiMessage::Request( - _relay_parent, - RuntimeApiRequest::NodeFeatures( - _, + relay_parent, + RuntimeApiRequest::SessionExecutorParams( + session_index, tx, ) )) => { - tx.send(Ok( - node_features - )).unwrap(); + assert_eq!(relay_parent, self.current); + assert_eq!(session_index, self.session_index); + + tx.send(Ok(Some(ExecutorParams::new()))).unwrap(); } ); } - async fn test_runtime_api_empty_node_features(&self, virtual_overseer: &mut VirtualOverseer) { + async fn test_runtime_api_node_features(&self, virtual_overseer: &mut VirtualOverseer) { assert_matches!( overseer_recv(virtual_overseer).await, AllMessages::RuntimeApi(RuntimeApiMessage::Request( @@ -249,27 +424,12 @@ impl TestState { ) )) => { tx.send(Ok( - NodeFeatures::EMPTY + self.node_features.clone() )).unwrap(); } ); } - async fn respond_to_block_number_query( - &self, - virtual_overseer: &mut VirtualOverseer, - block_number: BlockNumber, - ) { - assert_matches!( - overseer_recv(virtual_overseer).await, - AllMessages::ChainApi( - ChainApiMessage::BlockNumber(_, tx) - ) => { - let _ = tx.send(Ok(Some(block_number))); - } - ) - } - async fn respond_to_available_data_query( &self, virtual_overseer: &mut VirtualOverseer, @@ -293,16 +453,19 @@ impl TestState { async fn respond_to_query_all_request( &self, virtual_overseer: &mut VirtualOverseer, - send_chunk: impl Fn(usize) -> bool, + send_chunk: impl Fn(ValidatorIndex) -> bool, ) { assert_matches!( overseer_recv(virtual_overseer).await, AllMessages::AvailabilityStore( AvailabilityStoreMessage::QueryAllChunks(_, tx) ) => { - let v = self.chunks.iter() - .filter(|c| send_chunk(c.index.0 as usize)) - .cloned() + let v = self.chunks.iter().enumerate() + .filter_map(|(val_idx, c)| if send_chunk(ValidatorIndex(val_idx as u32)) { + Some((ValidatorIndex(val_idx as u32), c.clone())) + } else { + None + }) .collect(); let _ = tx.send(v); @@ -313,16 +476,19 @@ impl TestState { async fn respond_to_query_all_request_invalid( &self, virtual_overseer: &mut VirtualOverseer, - send_chunk: impl Fn(usize) -> bool, + send_chunk: impl Fn(ValidatorIndex) -> bool, ) { assert_matches!( overseer_recv(virtual_overseer).await, AllMessages::AvailabilityStore( AvailabilityStoreMessage::QueryAllChunks(_, tx) ) => { - let v = self.invalid_chunks.iter() - .filter(|c| send_chunk(c.index.0 as usize)) - .cloned() + let v = self.invalid_chunks.iter().enumerate() + .filter_map(|(val_idx, c)| if send_chunk(ValidatorIndex(val_idx as u32)) { + Some((ValidatorIndex(val_idx as u32), c.clone())) + } else { + None + }) .collect(); let _ = tx.send(v); @@ -330,15 +496,16 @@ impl TestState { ) } - async fn test_chunk_requests( + async fn test_chunk_requests_inner( &self, req_protocol_names: &ReqProtocolNames, candidate_hash: CandidateHash, virtual_overseer: &mut VirtualOverseer, n: usize, - mut who_has: impl FnMut(usize) -> Has, + mut who_has: impl FnMut(ValidatorIndex) -> Has, systematic_recovery: bool, - ) -> Vec, ProtocolName), RequestFailure>>> { + protocol: Protocol, + ) -> Vec, ProtocolName), RequestFailure>>> { // arbitrary order. let mut i = 0; let mut senders = Vec::new(); @@ -356,18 +523,19 @@ impl TestState { i += 1; assert_matches!( req, - Requests::ChunkFetchingV1(req) => { + Requests::ChunkFetching(req) => { assert_eq!(req.payload.candidate_hash, candidate_hash); - let chunk_index = req.payload.index.0 as usize; + let validator_index = req.payload.index; + let chunk = self.chunks.get(validator_index).unwrap().clone(); if systematic_recovery { - assert!(chunk_index <= self.systematic_threshold(), "requsted non-systematic chunk"); + assert!(chunk.index.0 as usize <= self.systematic_threshold(), "requested non-systematic chunk"); } - let available_data = match who_has(chunk_index) { + let available_data = match who_has(validator_index) { Has::No => Ok(None), - Has::Yes => Ok(Some(self.chunks[chunk_index].clone().into())), + Has::Yes => Ok(Some(chunk)), Has::NetworkError(e) => Err(e), Has::DoesNotReturn => { senders.push(req.pending_response); @@ -375,11 +543,29 @@ impl TestState { } }; - let _ = req.pending_response.send( + req.pending_response.send( available_data.map(|r| - (req_res::v1::ChunkFetchingResponse::from(r).encode(), req_protocol_names.get_name(Protocol::ChunkFetchingV1)) + ( + match protocol { + Protocol::ChunkFetchingV1 => + match r { + None => req_res::v1::ChunkFetchingResponse::NoSuchChunk, + Some(c) => req_res::v1::ChunkFetchingResponse::Chunk( + ChunkResponse { + chunk: c.chunk, + proof: c.proof + } + ) + }.encode(), + Protocol::ChunkFetchingV2 => + req_res::v2::ChunkFetchingResponse::from(r).encode(), + + _ => unreachable!() + }, + req_protocol_names.get_name(protocol) + ) ) - ); + ).unwrap(); } ) } @@ -389,6 +575,49 @@ impl TestState { senders } + async fn test_chunk_requests( + &self, + req_protocol_names: &ReqProtocolNames, + candidate_hash: CandidateHash, + virtual_overseer: &mut VirtualOverseer, + n: usize, + who_has: impl FnMut(ValidatorIndex) -> Has, + systematic_recovery: bool, + ) -> Vec, ProtocolName), RequestFailure>>> { + self.test_chunk_requests_inner( + req_protocol_names, + candidate_hash, + virtual_overseer, + n, + who_has, + systematic_recovery, + Protocol::ChunkFetchingV2, + ) + .await + } + + // Use legacy network protocol version. + async fn test_chunk_requests_v1( + &self, + req_protocol_names: &ReqProtocolNames, + candidate_hash: CandidateHash, + virtual_overseer: &mut VirtualOverseer, + n: usize, + who_has: impl FnMut(ValidatorIndex) -> Has, + systematic_recovery: bool, + ) -> Vec, ProtocolName), RequestFailure>>> { + self.test_chunk_requests_inner( + req_protocol_names, + candidate_hash, + virtual_overseer, + n, + who_has, + systematic_recovery, + Protocol::ChunkFetchingV1, + ) + .await + } + async fn test_full_data_requests( &self, req_protocol_names: &ReqProtocolNames, @@ -396,7 +625,7 @@ impl TestState { virtual_overseer: &mut VirtualOverseer, who_has: impl Fn(usize) -> Has, group_index: GroupIndex, - ) -> Vec, ProtocolName), RequestFailure>>> { + ) -> Vec, ProtocolName), RequestFailure>>> { let mut senders = Vec::new(); let expected_validators = self.validator_groups.get(group_index).unwrap(); for _ in 0..expected_validators.len() { @@ -450,6 +679,19 @@ impl TestState { } } +impl Default for TestState { + fn default() -> Self { + // Enable the chunk mapping node feature. + let mut node_features = NodeFeatures::new(); + node_features + .resize(node_features::FeatureIndex::AvailabilityChunkMapping as usize + 1, false); + node_features + .set(node_features::FeatureIndex::AvailabilityChunkMapping as u8 as usize, true); + + Self::new(node_features) + } +} + fn validator_pubkeys(val_ids: &[Sr25519Keyring]) -> IndexedVec { val_ids.iter().map(|v| v.public().into()).collect() } @@ -458,89 +700,20 @@ pub fn validator_authority_id(val_ids: &[Sr25519Keyring]) -> Vec Self { - let validators = vec![ - Sr25519Keyring::Ferdie, // <- this node, role: validator - Sr25519Keyring::Alice, - Sr25519Keyring::Bob, - Sr25519Keyring::Charlie, - Sr25519Keyring::Dave, - Sr25519Keyring::One, - Sr25519Keyring::Two, - ]; - - let validator_public = validator_pubkeys(&validators); - let validator_authority_id = validator_authority_id(&validators); - let validator_groups = vec![ - vec![1.into(), 0.into(), 3.into(), 4.into()], - vec![5.into(), 6.into()], - vec![2.into()], - ]; - - let current = Hash::repeat_byte(1); - - let mut candidate = dummy_candidate_receipt(dummy_hash()); - - let session_index = 10; - - let persisted_validation_data = PersistedValidationData { - parent_head: HeadData(vec![7, 8, 9]), - relay_parent_number: Default::default(), - max_pov_size: 1024, - relay_parent_storage_root: Default::default(), - }; - - let pov = PoV { block_data: BlockData(vec![42; 64]) }; - - let available_data = AvailableData { - validation_data: persisted_validation_data.clone(), - pov: Arc::new(pov), - }; - - let (chunks, erasure_root) = derive_erasure_chunks_with_proofs_and_root( - validators.len(), - &available_data, - |_, _| {}, - ); - // Mess around: - let invalid_chunks = chunks - .iter() - .cloned() - .map(|mut chunk| { - if chunk.chunk.len() >= 2 && chunk.chunk[0] != chunk.chunk[1] { - chunk.chunk[0] = chunk.chunk[1]; - } else if chunk.chunk.len() >= 1 { - chunk.chunk[0] = !chunk.chunk[0]; - } else { - chunk.proof = Proof::dummy_proof(); - } - chunk - }) - .collect(); - debug_assert_ne!(chunks, invalid_chunks); - - candidate.descriptor.erasure_root = erasure_root; - candidate.descriptor.relay_parent = Hash::repeat_byte(10); - candidate.descriptor.pov_hash = Hash::repeat_byte(3); - - Self { - validators, - validator_public, - validator_authority_id, - validator_groups: IndexedVec::>::try_from( - validator_groups, - ) - .unwrap(), - current, - candidate, - session_index, - persisted_validation_data, - available_data, - chunks, - invalid_chunks, - } - } +/// Map the chunks to the validators according to the availability chunk mapping algorithm. +fn map_chunks( + chunks: Vec, + node_features: &NodeFeatures, + n_validators: usize, + core_index: CoreIndex, +) -> IndexedVec { + let chunk_indices = + availability_chunk_indices(Some(node_features), n_validators, core_index).unwrap(); + + (0..n_validators) + .map(|val_idx| chunks[chunk_indices[val_idx].0 as usize].clone()) + .collect::>() + .into() } #[rstest] @@ -551,15 +724,17 @@ fn availability_is_recovered_from_chunks_if_no_group_provided(#[case] systematic let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); let (subsystem, threshold) = match systematic_recovery { true => ( - AvailabilityRecoverySubsystem::with_fast_path_then_systematic_chunks( + with_fast_path_then_systematic_chunks( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), test_state.systematic_threshold(), ), false => ( - AvailabilityRecoverySubsystem::with_fast_path( + with_fast_path( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), test_state.threshold(), @@ -584,14 +759,14 @@ fn availability_is_recovered_from_chunks_if_no_group_provided(#[case] systematic test_state.candidate.clone(), test_state.session_index, None, - None, + Some(test_state.core_index), tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -626,15 +801,12 @@ fn availability_is_recovered_from_chunks_if_no_group_provided(#[case] systematic new_candidate.clone(), test_state.session_index, None, - None, + Some(test_state.core_index), tx, ), ) .await; - test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; @@ -681,15 +853,17 @@ fn availability_is_recovered_from_chunks_even_if_backing_group_supplied_if_chunk let test_state = TestState::default(); let (subsystem, threshold) = match systematic_recovery { true => ( - AvailabilityRecoverySubsystem::with_systematic_chunks( + with_systematic_chunks( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), test_state.systematic_threshold(), ), false => ( - AvailabilityRecoverySubsystem::with_chunks_only( + with_chunks_only( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), test_state.threshold(), @@ -714,14 +888,14 @@ fn availability_is_recovered_from_chunks_even_if_backing_group_supplied_if_chunk test_state.candidate.clone(), test_state.session_index, Some(GroupIndex(0)), - None, + Some(test_state.core_index), tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -756,15 +930,12 @@ fn availability_is_recovered_from_chunks_even_if_backing_group_supplied_if_chunk new_candidate.clone(), test_state.session_index, Some(GroupIndex(1)), - None, + Some(test_state.core_index), tx, ), ) .await; - test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; @@ -821,12 +992,14 @@ fn bad_merkle_path_leads_to_recovery_error(#[case] systematic_recovery: bool) { let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); let mut test_state = TestState::default(); let subsystem = match systematic_recovery { - true => AvailabilityRecoverySubsystem::with_systematic_chunks( + true => with_systematic_chunks( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), - false => AvailabilityRecoverySubsystem::with_chunks_only( + false => with_chunks_only( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), }; @@ -849,14 +1022,14 @@ fn bad_merkle_path_leads_to_recovery_error(#[case] systematic_recovery: bool) { test_state.candidate.clone(), test_state.session_index, None, - None, + Some(test_state.core_index), tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -907,12 +1080,14 @@ fn wrong_chunk_index_leads_to_recovery_error(#[case] systematic_recovery: bool) let mut test_state = TestState::default(); let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); let subsystem = match systematic_recovery { - true => AvailabilityRecoverySubsystem::with_systematic_chunks( + true => with_systematic_chunks( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), - false => AvailabilityRecoverySubsystem::with_chunks_only( + false => with_chunks_only( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), }; @@ -935,14 +1110,14 @@ fn wrong_chunk_index_leads_to_recovery_error(#[case] systematic_recovery: bool) test_state.candidate.clone(), test_state.session_index, None, - None, + Some(test_state.core_index), tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -951,10 +1126,13 @@ fn wrong_chunk_index_leads_to_recovery_error(#[case] systematic_recovery: bool) test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; // Chunks should fail the index check as they don't have the correct index. - let first_chunk = test_state.chunks[0].clone(); - test_state.chunks[0] = test_state.chunks[1].clone(); + + // *(test_state.chunks.get_mut(0.into()).unwrap()) = + // test_state.chunks.get(1.into()).unwrap().clone(); + let first_chunk = test_state.chunks.get(0.into()).unwrap().clone(); for c_index in 1..test_state.chunks.len() { - test_state.chunks[c_index] = first_chunk.clone(); + *(test_state.chunks.get_mut(ValidatorIndex(c_index as u32)).unwrap()) = + first_chunk.clone(); } if systematic_recovery { @@ -965,7 +1143,8 @@ fn wrong_chunk_index_leads_to_recovery_error(#[case] systematic_recovery: bool) &mut virtual_overseer, test_state.systematic_threshold(), |_| Has::Yes, - systematic_recovery, + // We set this to false, as we know we will be requesting the wrong indices. + false, ) .await; @@ -997,15 +1176,17 @@ fn invalid_erasure_coding_leads_to_invalid_error(#[case] systematic_recovery: bo let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); let (subsystem, threshold) = match systematic_recovery { true => ( - AvailabilityRecoverySubsystem::with_fast_path_then_systematic_chunks( + with_fast_path_then_systematic_chunks( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), test_state.systematic_threshold(), ), false => ( - AvailabilityRecoverySubsystem::with_fast_path( + with_fast_path( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), test_state.threshold(), @@ -1024,7 +1205,12 @@ fn invalid_erasure_coding_leads_to_invalid_error(#[case] systematic_recovery: bo |i, chunk| *chunk = vec![i as u8; 32], ); - test_state.chunks = bad_chunks; + test_state.chunks = map_chunks( + bad_chunks, + &test_state.node_features, + test_state.validators.len(), + test_state.core_index, + ); test_state.candidate.descriptor.erasure_root = bad_erasure_root; let candidate_hash = test_state.candidate.hash(); @@ -1046,14 +1232,14 @@ fn invalid_erasure_coding_leads_to_invalid_error(#[case] systematic_recovery: bo test_state.candidate.clone(), test_state.session_index, None, - None, + Some(test_state.core_index), tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; @@ -1082,6 +1268,7 @@ fn invalid_pov_hash_leads_to_invalid_error() { let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); let subsystem = AvailabilityRecoverySubsystem::for_collator( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ); @@ -1109,14 +1296,14 @@ fn invalid_pov_hash_leads_to_invalid_error() { test_state.candidate.clone(), test_state.session_index, None, - None, + Some(test_state.core_index), tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; test_state @@ -1135,14 +1322,13 @@ fn invalid_pov_hash_leads_to_invalid_error() { }); } -#[rstest] -#[case(Some(1))] -#[case(None)] -fn fast_path_backing_group_recovers(#[case] relay_parent_block_number: Option) { +#[test] +fn fast_path_backing_group_recovers() { let test_state = TestState::default(); let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); - let subsystem = AvailabilityRecoverySubsystem::with_fast_path( + let subsystem = with_fast_path( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ); @@ -1164,16 +1350,13 @@ fn fast_path_backing_group_recovers(#[case] relay_parent_block_number: Option ( - AvailabilityRecoverySubsystem::with_systematic_chunks_if_pov_large( + with_systematic_chunks_if_pov_large( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), test_state.systematic_threshold(), ), (false, false) => ( - AvailabilityRecoverySubsystem::with_chunks_if_pov_large( + with_chunks_if_pov_large( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), test_state.threshold(), @@ -1231,6 +1416,7 @@ fn recovers_from_only_chunks_if_pov_large( ( AvailabilityRecoverySubsystem::for_collator( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), test_state.threshold(), @@ -1257,14 +1443,14 @@ fn recovers_from_only_chunks_if_pov_large( test_state.candidate.clone(), test_state.session_index, Some(GroupIndex(0)), - None, + Some(test_state.core_index), tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -1310,15 +1496,12 @@ fn recovers_from_only_chunks_if_pov_large( new_candidate.clone(), test_state.session_index, Some(GroupIndex(1)), - None, + Some(test_state.core_index), tx, ), ) .await; - test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - assert_matches!( overseer_recv(&mut virtual_overseer).await, AllMessages::AvailabilityStore( @@ -1389,19 +1572,22 @@ fn fast_path_backing_group_recovers_if_pov_small( let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); let subsystem = match (systematic_recovery, for_collator) { - (true, false) => AvailabilityRecoverySubsystem::with_systematic_chunks_if_pov_large( + (true, false) => with_systematic_chunks_if_pov_large( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), - (false, false) => AvailabilityRecoverySubsystem::with_chunks_if_pov_large( + (false, false) => with_chunks_if_pov_large( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), (false, true) => { test_state.candidate.descriptor.pov_hash = test_state.available_data.pov.hash(); AvailabilityRecoverySubsystem::for_collator( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ) }, @@ -1426,14 +1612,14 @@ fn fast_path_backing_group_recovers_if_pov_small( test_state.candidate.clone(), test_state.session_index, Some(GroupIndex(0)), - None, + Some(test_state.core_index), tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -1481,15 +1667,17 @@ fn no_answers_in_fast_path_causes_chunk_requests(#[case] systematic_recovery: bo let (subsystem, threshold) = match systematic_recovery { true => ( - AvailabilityRecoverySubsystem::with_fast_path_then_systematic_chunks( + with_fast_path_then_systematic_chunks( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), test_state.systematic_threshold(), ), false => ( - AvailabilityRecoverySubsystem::with_fast_path( + with_fast_path( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), test_state.threshold(), @@ -1514,14 +1702,14 @@ fn no_answers_in_fast_path_causes_chunk_requests(#[case] systematic_recovery: bo test_state.candidate.clone(), test_state.session_index, Some(GroupIndex(0)), - None, + Some(test_state.core_index), tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -1571,12 +1759,14 @@ fn task_canceled_when_receivers_dropped(#[case] systematic_recovery: bool) { let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); let subsystem = match systematic_recovery { - true => AvailabilityRecoverySubsystem::with_systematic_chunks( + true => with_systematic_chunks( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), - false => AvailabilityRecoverySubsystem::with_chunks_only( + false => with_chunks_only( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), }; @@ -1599,14 +1789,14 @@ fn task_canceled_when_receivers_dropped(#[case] systematic_recovery: bool) { test_state.candidate.clone(), test_state.session_index, None, - None, + Some(test_state.core_index), tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; for _ in 0..test_state.validators.len() { @@ -1627,12 +1817,14 @@ fn chunks_retry_until_all_nodes_respond(#[case] systematic_recovery: bool) { let test_state = TestState::default(); let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); let subsystem = match systematic_recovery { - true => AvailabilityRecoverySubsystem::with_systematic_chunks( + true => with_systematic_chunks( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), - false => AvailabilityRecoverySubsystem::with_chunks_only( + false => with_chunks_only( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), }; @@ -1655,14 +1847,14 @@ fn chunks_retry_until_all_nodes_respond(#[case] systematic_recovery: bool) { test_state.candidate.clone(), test_state.session_index, None, - None, + Some(test_state.core_index), tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -1720,8 +1912,9 @@ fn chunks_retry_until_all_nodes_respond(#[case] systematic_recovery: bool) { fn network_bridge_not_returning_responses_wont_stall_retrieval() { let test_state = TestState::default(); let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); - let subsystem = AvailabilityRecoverySubsystem::with_chunks_only( + let subsystem = with_chunks_only( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ); @@ -1743,14 +1936,14 @@ fn network_bridge_not_returning_responses_wont_stall_retrieval() { test_state.candidate.clone(), test_state.session_index, Some(GroupIndex(0)), - None, + Some(test_state.core_index), tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -1810,12 +2003,14 @@ fn all_not_returning_requests_still_recovers_on_return(#[case] systematic_recove let test_state = TestState::default(); let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); let subsystem = match systematic_recovery { - true => AvailabilityRecoverySubsystem::with_systematic_chunks( + true => with_systematic_chunks( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), - false => AvailabilityRecoverySubsystem::with_chunks_only( + false => with_chunks_only( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), }; @@ -1838,14 +2033,14 @@ fn all_not_returning_requests_still_recovers_on_return(#[case] systematic_recove test_state.candidate.clone(), test_state.session_index, None, - None, + Some(test_state.core_index), tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -1920,12 +2115,14 @@ fn returns_early_if_we_have_the_data(#[case] systematic_recovery: bool) { let test_state = TestState::default(); let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); let subsystem = match systematic_recovery { - true => AvailabilityRecoverySubsystem::with_systematic_chunks( + true => with_systematic_chunks( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), - false => AvailabilityRecoverySubsystem::with_chunks_only( + false => with_chunks_only( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), }; @@ -1948,14 +2145,14 @@ fn returns_early_if_we_have_the_data(#[case] systematic_recovery: bool) { test_state.candidate.clone(), test_state.session_index, None, - None, + Some(test_state.core_index), tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; test_state.respond_to_available_data_query(&mut virtual_overseer, true).await; @@ -1968,8 +2165,9 @@ fn returns_early_if_we_have_the_data(#[case] systematic_recovery: bool) { fn returns_early_if_present_in_the_subsystem_cache() { let test_state = TestState::default(); let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); - let subsystem = AvailabilityRecoverySubsystem::with_fast_path( + let subsystem = with_fast_path( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ); @@ -1991,14 +2189,14 @@ fn returns_early_if_present_in_the_subsystem_cache() { test_state.candidate.clone(), test_state.session_index, Some(GroupIndex(0)), - None, + Some(test_state.core_index), tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; let candidate_hash = test_state.candidate.hash(); @@ -2032,7 +2230,7 @@ fn returns_early_if_present_in_the_subsystem_cache() { test_state.candidate.clone(), test_state.session_index, Some(GroupIndex(0)), - None, + Some(test_state.core_index), tx, ), ) @@ -2051,15 +2249,17 @@ fn does_not_query_local_validator(#[case] systematic_recovery: bool) { let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); let (subsystem, threshold) = match systematic_recovery { true => ( - AvailabilityRecoverySubsystem::with_systematic_chunks( + with_systematic_chunks( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), test_state.systematic_threshold(), ), false => ( - AvailabilityRecoverySubsystem::with_chunks_only( + with_chunks_only( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), test_state.threshold(), @@ -2084,17 +2284,19 @@ fn does_not_query_local_validator(#[case] systematic_recovery: bool) { test_state.candidate.clone(), test_state.session_index, None, - None, + Some(test_state.core_index), tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; - test_state.respond_to_query_all_request(&mut virtual_overseer, |i| i == 0).await; + test_state + .respond_to_query_all_request(&mut virtual_overseer, |i| i.0 == 0) + .await; let candidate_hash = test_state.candidate.hash(); @@ -2105,7 +2307,7 @@ fn does_not_query_local_validator(#[case] systematic_recovery: bool) { candidate_hash, &mut virtual_overseer, threshold - 1, - |i| if i == 0 { panic!("requested from local validator") } else { Has::Yes }, + |i| if i.0 == 0 { panic!("requested from local validator") } else { Has::Yes }, systematic_recovery, ) .await; @@ -2122,12 +2324,14 @@ fn invalid_local_chunk(#[case] systematic_recovery: bool) { let test_state = TestState::default(); let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); let subsystem = match systematic_recovery { - true => AvailabilityRecoverySubsystem::with_systematic_chunks( + true => with_systematic_chunks( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), - false => AvailabilityRecoverySubsystem::with_chunks_only( + false => with_chunks_only( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), }; @@ -2150,27 +2354,39 @@ fn invalid_local_chunk(#[case] systematic_recovery: bool) { test_state.candidate.clone(), test_state.session_index, None, - None, + Some(test_state.core_index), tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; + + let validator_index_for_first_chunk = test_state + .chunks + .iter() + .enumerate() + .find_map(|(val_idx, chunk)| if chunk.index.0 == 0 { Some(val_idx) } else { None }) + .unwrap() as u32; + test_state - .respond_to_query_all_request_invalid(&mut virtual_overseer, |i| i == 0) + .respond_to_query_all_request_invalid(&mut virtual_overseer, |i| { + i.0 == validator_index_for_first_chunk + }) .await; let candidate_hash = test_state.candidate.hash(); // If systematic recovery detects invalid local chunk, it'll directly go to regular - // recovery. + // recovery, if we were the one holding an invalid chunk. if systematic_recovery { test_state - .respond_to_query_all_request_invalid(&mut virtual_overseer, |i| i == 0) + .respond_to_query_all_request_invalid(&mut virtual_overseer, |i| { + i.0 == validator_index_for_first_chunk + }) .await; } @@ -2180,7 +2396,13 @@ fn invalid_local_chunk(#[case] systematic_recovery: bool) { candidate_hash, &mut virtual_overseer, test_state.threshold(), - |i| if i == 0 { panic!("requested from local validator") } else { Has::Yes }, + |i| { + if i.0 == validator_index_for_first_chunk { + panic!("requested from local validator") + } else { + Has::Yes + } + }, false, ) .await; @@ -2197,8 +2419,9 @@ fn systematic_chunks_are_not_requested_again_in_regular_recovery() { for _ in 0..TestState::default().chunks.len() { let test_state = TestState::default(); let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); - let subsystem = AvailabilityRecoverySubsystem::with_systematic_chunks( + let subsystem = with_systematic_chunks( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ); @@ -2220,25 +2443,32 @@ fn systematic_chunks_are_not_requested_again_in_regular_recovery() { test_state.candidate.clone(), test_state.session_index, None, - None, + Some(test_state.core_index), tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + let validator_index_for_first_chunk = test_state + .chunks + .iter() + .enumerate() + .find_map(|(val_idx, chunk)| if chunk.index.0 == 0 { Some(val_idx) } else { None }) + .unwrap() as u32; + test_state .test_chunk_requests( &req_protocol_names, test_state.candidate.hash(), &mut virtual_overseer, test_state.systematic_threshold(), - |i| if i == 0 { Has::No } else { Has::Yes }, + |i| if i.0 == validator_index_for_first_chunk { Has::No } else { Has::Yes }, true, ) .await; @@ -2252,8 +2482,10 @@ fn systematic_chunks_are_not_requested_again_in_regular_recovery() { test_state.candidate.hash(), &mut virtual_overseer, 1, - |i: usize| { - if i < test_state.systematic_threshold() { + |i| { + if (test_state.chunks.get(i).unwrap().index.0 as usize) < + test_state.systematic_threshold() + { panic!("Already requested") } else { Has::Yes @@ -2274,16 +2506,24 @@ fn systematic_chunks_are_not_requested_again_in_regular_recovery() { #[case(true, false)] #[case(false, true)] #[case(false, false)] -fn chunk_indices_are_shuffled(#[case] systematic_recovery: bool, #[case] shuffling_enabled: bool) { - let test_state = TestState::default(); +fn chunk_indices_are_mapped_to_different_validators( + #[case] systematic_recovery: bool, + #[case] mapping_enabled: bool, +) { let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); + let test_state = match mapping_enabled { + true => TestState::default(), + false => TestState::with_empty_node_features(), + }; let subsystem = match systematic_recovery { - true => AvailabilityRecoverySubsystem::with_systematic_chunks( + true => with_systematic_chunks( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), - false => AvailabilityRecoverySubsystem::with_chunks_only( + false => with_chunks_only( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), }; @@ -2306,25 +2546,20 @@ fn chunk_indices_are_shuffled(#[case] systematic_recovery: bool, #[case] shuffli test_state.candidate.clone(), test_state.session_index, None, - None, + Some(test_state.core_index), tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; - if shuffling_enabled { - test_state.test_runtime_api_node_features(&mut virtual_overseer).await; - } else { - test_state.test_runtime_api_empty_node_features(&mut virtual_overseer).await; - } + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; - let mut chunk_indices: Vec<(usize, usize)> = vec![]; + let mut chunk_indices: Vec<(u32, u32)> = vec![]; assert_matches!( overseer_recv(&mut virtual_overseer).await, @@ -2337,31 +2572,24 @@ fn chunk_indices_are_shuffled(#[case] systematic_recovery: bool, #[case] shuffli for req in requests { assert_matches!( req, - Requests::ChunkFetchingV1(req) => { + Requests::ChunkFetching(req) => { assert_eq!(req.payload.candidate_hash, test_state.candidate.hash()); - let chunk_index = req.payload.index.0 as usize; - let validator_index = test_state.validator_authority_id.iter().enumerate().find(|(_, id)| { - if let Recipient::Authority(auth_id) = &req.peer { - if *id == auth_id { - return true - } - } - false - }).expect("validator not found").0; + let validator_index = req.payload.index; + let chunk_index = test_state.chunks.get(validator_index).unwrap().index; if systematic_recovery { - assert!(chunk_index <= test_state.systematic_threshold(), "requsted non-systematic chunk"); + assert!((chunk_index.0 as usize) <= test_state.systematic_threshold(), "requested non-systematic chunk"); } - chunk_indices.push((chunk_index, validator_index)); + chunk_indices.push((chunk_index.0, validator_index.0)); } ) } } ); - if shuffling_enabled { + if mapping_enabled { assert!(!chunk_indices.iter().any(|(c_index, v_index)| c_index == v_index)); } else { assert!(chunk_indices.iter().all(|(c_index, v_index)| c_index == v_index)); @@ -2395,20 +2623,23 @@ fn number_of_request_retries_is_bounded( &test_state.available_data, |_, _| {}, ); - test_state.chunks = chunks; + test_state.chunks = + map_chunks(chunks, &test_state.node_features, n_validators, test_state.core_index); test_state.candidate.descriptor.erasure_root = erasure_root; let (subsystem, retry_limit) = match systematic_recovery { false => ( - AvailabilityRecoverySubsystem::with_chunks_only( + with_chunks_only( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), REGULAR_CHUNKS_REQ_RETRY_LIMIT, ), true => ( - AvailabilityRecoverySubsystem::with_systematic_chunks( + with_systematic_chunks( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT, @@ -2433,14 +2664,14 @@ fn number_of_request_retries_is_bounded( test_state.candidate.clone(), test_state.session_index, None, - None, + Some(test_state.core_index), tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; @@ -2497,23 +2728,117 @@ fn number_of_request_retries_is_bounded( }); } +#[test] +fn systematic_recovery_retries_from_backers() { + let test_state = TestState::default(); + let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); + let subsystem = with_systematic_chunks( + request_receiver(&req_protocol_names), + &req_protocol_names, + Metrics::new_dummy(), + ); + + test_harness(subsystem, |mut virtual_overseer| async move { + overseer_signal( + &mut virtual_overseer, + OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( + test_state.current, + 1, + ))), + ) + .await; + + let (tx, rx) = oneshot::channel(); + let group_index = GroupIndex(2); + let group_size = test_state.validator_groups.get(group_index).unwrap().len(); + + overseer_send( + &mut virtual_overseer, + AvailabilityRecoveryMessage::RecoverAvailableData( + test_state.candidate.clone(), + test_state.session_index, + Some(group_index), + Some(test_state.core_index), + tx, + ), + ) + .await; + + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; + + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; + test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; + test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + + let mut cnt = 0; + + test_state + .test_chunk_requests( + &req_protocol_names, + test_state.candidate.hash(), + &mut virtual_overseer, + test_state.systematic_threshold(), + |_| { + let res = if cnt < group_size { Has::timeout() } else { Has::Yes }; + cnt += 1; + res + }, + true, + ) + .await; + + // Exhaust retries. + for _ in 0..(SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT - 1) { + test_state + .test_chunk_requests( + &req_protocol_names, + test_state.candidate.hash(), + &mut virtual_overseer, + group_size, + |_| Has::No, + true, + ) + .await; + } + + // Now, final chance is to try from a backer. + test_state + .test_chunk_requests( + &req_protocol_names, + test_state.candidate.hash(), + &mut virtual_overseer, + group_size, + |_| Has::Yes, + true, + ) + .await; + + assert_eq!(rx.await.unwrap().unwrap(), test_state.available_data); + virtual_overseer + }); +} + #[rstest] -#[case(false)] #[case(true)] -fn block_number_not_requested_if_provided(#[case] systematic_recovery: bool) { - let test_state = TestState::default(); +#[case(false)] +fn test_legacy_network_protocol_with_mapping_disabled(#[case] systematic_recovery: bool) { + // In this case, when the mapping is disabled, recovery will work with both v2 and v1 requests, + // under the assumption that ValidatorIndex is always equal to ChunkIndex. + let test_state = TestState::with_empty_node_features(); let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); let (subsystem, threshold) = match systematic_recovery { true => ( - AvailabilityRecoverySubsystem::with_systematic_chunks( + with_systematic_chunks( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), test_state.systematic_threshold(), ), false => ( - AvailabilityRecoverySubsystem::with_chunks_only( + with_fast_path( request_receiver(&req_protocol_names), + &req_protocol_names, Metrics::new_dummy(), ), test_state.threshold(), @@ -2538,21 +2863,25 @@ fn block_number_not_requested_if_provided(#[case] systematic_recovery: bool) { test_state.candidate.clone(), test_state.session_index, None, - Some(1), + Some(test_state.core_index), tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; + + let candidate_hash = test_state.candidate.hash(); + test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; test_state - .test_chunk_requests( + .test_chunk_requests_v1( &req_protocol_names, - test_state.candidate.hash(), + candidate_hash, &mut virtual_overseer, threshold, |_| Has::Yes, @@ -2560,19 +2889,37 @@ fn block_number_not_requested_if_provided(#[case] systematic_recovery: bool) { ) .await; + // Recovered data should match the original one. assert_eq!(rx.await.unwrap().unwrap(), test_state.available_data); virtual_overseer }); } -#[test] -fn systematic_recovery_retries_from_backers() { +#[rstest] +#[case(true)] +#[case(false)] +fn test_legacy_network_protocol_with_mapping_enabled(#[case] systematic_recovery: bool) { + // In this case, when the mapping is enabled, we MUST only use v2. Recovery should fail for v1. let test_state = TestState::default(); let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); - let subsystem = AvailabilityRecoverySubsystem::with_systematic_chunks( - request_receiver(&req_protocol_names), - Metrics::new_dummy(), - ); + let (subsystem, threshold) = match systematic_recovery { + true => ( + with_systematic_chunks( + request_receiver(&req_protocol_names), + &req_protocol_names, + Metrics::new_dummy(), + ), + test_state.systematic_threshold(), + ), + false => ( + with_fast_path( + request_receiver(&req_protocol_names), + &req_protocol_names, + Metrics::new_dummy(), + ), + test_state.threshold(), + ), + }; test_harness(subsystem, |mut virtual_overseer| async move { overseer_signal( @@ -2585,71 +2932,118 @@ fn systematic_recovery_retries_from_backers() { .await; let (tx, rx) = oneshot::channel(); - let group_index = GroupIndex(2); - let group_size = test_state.validator_groups.get(group_index).unwrap().len(); overseer_send( &mut virtual_overseer, AvailabilityRecoveryMessage::RecoverAvailableData( test_state.candidate.clone(), test_state.session_index, - Some(group_index), None, + Some(test_state.core_index), tx, ), ) .await; test_state.test_runtime_api_session_info(&mut virtual_overseer).await; - test_state.respond_to_block_number_query(&mut virtual_overseer, 1).await; + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; + + let candidate_hash = test_state.candidate.hash(); + test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; - let mut cnt = 0; + if systematic_recovery { + test_state + .test_chunk_requests_v1( + &req_protocol_names, + candidate_hash, + &mut virtual_overseer, + threshold, + |_| Has::Yes, + systematic_recovery, + ) + .await; + + // Systematic recovery failed, trying regular recovery. + test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + } test_state - .test_chunk_requests( + .test_chunk_requests_v1( &req_protocol_names, - test_state.candidate.hash(), + candidate_hash, &mut virtual_overseer, - test_state.systematic_threshold(), - |_| { - let res = if cnt < group_size { Has::timeout() } else { Has::Yes }; - cnt += 1; - res - }, - true, + test_state.validators.len() - test_state.threshold(), + |_| Has::Yes, + false, ) .await; - // Exhaust retries. - for _ in 0..(SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT - 1) { - test_state - .test_chunk_requests( - &req_protocol_names, - test_state.candidate.hash(), - &mut virtual_overseer, - group_size, - |_| Has::No, - true, - ) - .await; - } + assert_eq!(rx.await.unwrap().unwrap_err(), RecoveryError::Unavailable); + virtual_overseer + }); +} - // Now, final chance is to try from a backer. +#[test] +fn test_systematic_recovery_skipped_if_no_core_index() { + let test_state = TestState::default(); + let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); + let subsystem = with_systematic_chunks( + request_receiver(&req_protocol_names), + &req_protocol_names, + Metrics::new_dummy(), + ); + + test_harness(subsystem, |mut virtual_overseer| async move { + overseer_signal( + &mut virtual_overseer, + OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( + test_state.current, + 1, + ))), + ) + .await; + + let (tx, rx) = oneshot::channel(); + + overseer_send( + &mut virtual_overseer, + AvailabilityRecoveryMessage::RecoverAvailableData( + test_state.candidate.clone(), + test_state.session_index, + None, + None, + tx, + ), + ) + .await; + + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; + + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; + + let candidate_hash = test_state.candidate.hash(); + + test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; + test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + + // Systematic recovery not possible without core index, falling back to regular recovery. test_state .test_chunk_requests( &req_protocol_names, - test_state.candidate.hash(), + candidate_hash, &mut virtual_overseer, - group_size, - |_| Has::Yes, - true, + test_state.validators.len() - test_state.threshold(), + |_| Has::No, + false, ) .await; - assert_eq!(rx.await.unwrap().unwrap(), test_state.available_data); + // Make it fail, in order to assert that indeed regular recovery was attempted. If it were + // systematic recovery, we would have had one more attempt for regular reconstruction. + assert_eq!(rx.await.unwrap().unwrap_err(), RecoveryError::Unavailable); virtual_overseer }); } diff --git a/polkadot/node/network/bridge/src/tx/mod.rs b/polkadot/node/network/bridge/src/tx/mod.rs index d5be6f01c337..7b6dea748572 100644 --- a/polkadot/node/network/bridge/src/tx/mod.rs +++ b/polkadot/node/network/bridge/src/tx/mod.rs @@ -301,7 +301,15 @@ where for req in reqs { match req { - Requests::ChunkFetchingV1(_) => metrics.on_message("chunk_fetching_v1"), + Requests::ChunkFetching(ref req) => { + // This is not the actual request that will succeed, as we don't know yet + // what that will be. It's only the primary request we tried. + if req.fallback_request.is_some() { + metrics.on_message("chunk_fetching_v2") + } else { + metrics.on_message("chunk_fetching_v1") + } + }, Requests::AvailableDataFetchingV1(_) => metrics.on_message("available_data_fetching_v1"), Requests::CollationFetchingV1(_) => metrics.on_message("collation_fetching_v1"), diff --git a/polkadot/node/network/protocol/src/request_response/mod.rs b/polkadot/node/network/protocol/src/request_response/mod.rs index a67d83aff0c9..68f98d11c231 100644 --- a/polkadot/node/network/protocol/src/request_response/mod.rs +++ b/polkadot/node/network/protocol/src/request_response/mod.rs @@ -96,6 +96,10 @@ pub enum Protocol { /// Protocol for requesting candidates with attestations in statement distribution /// when async backing is enabled. AttestedCandidateV2, + + /// Protocol for chunk fetching version 2, used by availability distribution and availability + /// recovery. + ChunkFetchingV2, } /// Minimum bandwidth we expect for validators - 500Mbit/s is the recommendation, so approximately @@ -207,7 +211,7 @@ impl Protocol { let name = req_protocol_names.get_name(self); let legacy_names = self.get_legacy_name().into_iter().map(Into::into).collect(); match self { - Protocol::ChunkFetchingV1 => RequestResponseConfig { + Protocol::ChunkFetchingV1 | Protocol::ChunkFetchingV2 => RequestResponseConfig { name, fallback_names: legacy_names, max_request_size: 1_000, @@ -290,7 +294,7 @@ impl Protocol { // times (due to network delays), 100 seems big enough to accomodate for "bursts", // assuming we can service requests relatively quickly, which would need to be measured // as well. - Protocol::ChunkFetchingV1 => 100, + Protocol::ChunkFetchingV1 | Protocol::ChunkFetchingV2 => 100, // 10 seems reasonable, considering group sizes of max 10 validators. Protocol::CollationFetchingV1 | Protocol::CollationFetchingV2 => 10, // 10 seems reasonable, considering group sizes of max 10 validators. @@ -360,6 +364,7 @@ impl Protocol { // Introduced after legacy names became legacy. Protocol::AttestedCandidateV2 => None, Protocol::CollationFetchingV2 => None, + Protocol::ChunkFetchingV2 => None, } } } @@ -410,6 +415,7 @@ impl ReqProtocolNames { }; let short_name = match protocol { + // V1: Protocol::ChunkFetchingV1 => "/req_chunk/1", Protocol::CollationFetchingV1 => "/req_collation/1", Protocol::PoVFetchingV1 => "/req_pov/1", @@ -417,8 +423,10 @@ impl ReqProtocolNames { Protocol::StatementFetchingV1 => "/req_statement/1", Protocol::DisputeSendingV1 => "/send_dispute/1", + // V2: Protocol::CollationFetchingV2 => "/req_collation/2", Protocol::AttestedCandidateV2 => "/req_attested_candidate/2", + Protocol::ChunkFetchingV2 => "/req_chunk/2", }; format!("{}{}", prefix, short_name).into() diff --git a/polkadot/node/network/protocol/src/request_response/outgoing.rs b/polkadot/node/network/protocol/src/request_response/outgoing.rs index 88439ad40367..616b133a142b 100644 --- a/polkadot/node/network/protocol/src/request_response/outgoing.rs +++ b/polkadot/node/network/protocol/src/request_response/outgoing.rs @@ -30,7 +30,7 @@ use super::{v1, v2, IsRequest, Protocol}; #[derive(Debug)] pub enum Requests { /// Request an availability chunk from a node. - ChunkFetchingV1(OutgoingRequest), + ChunkFetching(OutgoingRequest), /// Fetch a collation from a collator which previously announced it. CollationFetchingV1(OutgoingRequest), /// Fetch a PoV from a validator which previously sent out a seconded statement. @@ -59,7 +59,7 @@ impl Requests { /// contained in the `enum`. pub fn encode_request(self) -> (Protocol, OutgoingRequest>) { match self { - Self::ChunkFetchingV1(r) => r.encode_request(), + Self::ChunkFetching(r) => r.encode_request(), Self::CollationFetchingV1(r) => r.encode_request(), Self::CollationFetchingV2(r) => r.encode_request(), Self::PoVFetchingV1(r) => r.encode_request(), @@ -164,24 +164,20 @@ where /// /// Returns a raw `Vec` response over the channel. Use the associated `ProtocolName` to know /// which request was the successful one and appropriately decode the response. - // WARNING: This is commented for now because it's not used yet. - // If you need it, make sure to test it. You may need to enable the V1 substream upgrade - // protocol, unless libp2p was in the meantime updated to a version that fixes the problem - // described in https://github.com/libp2p/rust-libp2p/issues/5074 - // pub fn new_with_fallback( - // peer: Recipient, - // payload: Req, - // fallback_request: FallbackReq, - // ) -> (Self, impl Future, ProtocolName)>>) { - // let (tx, rx) = oneshot::channel(); - // let r = Self { - // peer, - // payload, - // pending_response: tx, - // fallback_request: Some((fallback_request, FallbackReq::PROTOCOL)), - // }; - // (r, async { Ok(rx.await??) }) - // } + pub fn new_with_fallback( + peer: Recipient, + payload: Req, + fallback_request: FallbackReq, + ) -> (Self, impl Future, ProtocolName)>>) { + let (tx, rx) = oneshot::channel(); + let r = Self { + peer, + payload, + pending_response: tx, + fallback_request: Some((fallback_request, FallbackReq::PROTOCOL)), + }; + (r, async { Ok(rx.await??) }) + } /// Encode a request into a `Vec`. /// diff --git a/polkadot/node/network/protocol/src/request_response/v1.rs b/polkadot/node/network/protocol/src/request_response/v1.rs index ec8e9ef4b277..fc90f76c371a 100644 --- a/polkadot/node/network/protocol/src/request_response/v1.rs +++ b/polkadot/node/network/protocol/src/request_response/v1.rs @@ -22,7 +22,7 @@ use polkadot_node_primitives::{ AvailableData, DisputeMessage, ErasureChunk, PoV, Proof, UncheckedDisputeMessage, }; use polkadot_primitives::{ - CandidateHash, CandidateReceipt, ChunkIndex, CommittedCandidateReceipt, Hash, Id as ParaId, + CandidateHash, CandidateReceipt, CommittedCandidateReceipt, Hash, Id as ParaId, ValidatorIndex, }; use super::{IsRequest, Protocol}; @@ -32,8 +32,9 @@ use super::{IsRequest, Protocol}; pub struct ChunkFetchingRequest { /// Hash of candidate we want a chunk for. pub candidate_hash: CandidateHash, - /// The index of the chunk to fetch. - pub index: ChunkIndex, + /// The validator index we are requesting from. This must be identical to the index of the + /// chunk we'll receive. For v2, this may not be the case. + pub index: ValidatorIndex, } /// Receive a requested erasure chunk. @@ -56,6 +57,15 @@ impl From> for ChunkFetchingResponse { } } +impl From for Option { + fn from(x: ChunkFetchingResponse) -> Self { + match x { + ChunkFetchingResponse::Chunk(c) => Some(c), + ChunkFetchingResponse::NoSuchChunk => None, + } + } +} + /// Skimmed down variant of `ErasureChunk`. /// /// Instead of transmitting a full `ErasureChunk` we transmit `ChunkResponse` in @@ -79,7 +89,7 @@ impl From for ChunkResponse { impl ChunkResponse { /// Re-build an `ErasureChunk` from response and request. pub fn recombine_into_chunk(self, req: &ChunkFetchingRequest) -> ErasureChunk { - ErasureChunk { chunk: self.chunk, proof: self.proof, index: req.index } + ErasureChunk { chunk: self.chunk, proof: self.proof, index: req.index.into() } } } diff --git a/polkadot/node/network/protocol/src/request_response/v2.rs b/polkadot/node/network/protocol/src/request_response/v2.rs index 6b90c579237f..7e1a2d989168 100644 --- a/polkadot/node/network/protocol/src/request_response/v2.rs +++ b/polkadot/node/network/protocol/src/request_response/v2.rs @@ -18,12 +18,13 @@ use parity_scale_codec::{Decode, Encode}; +use polkadot_node_primitives::ErasureChunk; use polkadot_primitives::{ CandidateHash, CommittedCandidateReceipt, Hash, Id as ParaId, PersistedValidationData, - UncheckedSignedStatement, + UncheckedSignedStatement, ValidatorIndex, }; -use super::{IsRequest, Protocol}; +use super::{v1, IsRequest, Protocol}; use crate::v2::StatementFilter; /// Request a candidate with statements. @@ -78,3 +79,60 @@ impl IsRequest for CollationFetchingRequest { type Response = CollationFetchingResponse; const PROTOCOL: Protocol = Protocol::CollationFetchingV2; } + +/// Request an availability chunk. +#[derive(Debug, Copy, Clone, Encode, Decode)] +pub struct ChunkFetchingRequest { + /// Hash of candidate we want a chunk for. + pub candidate_hash: CandidateHash, + /// The validator index we are requesting from. This may not be identical to the index of the + /// chunk we'll receive. It's up to the caller to decide whether they need to validate they got + /// the chunk they were expecting. + pub index: ValidatorIndex, +} + +/// Receive a requested erasure chunk. +#[derive(Debug, Clone, Encode, Decode)] +pub enum ChunkFetchingResponse { + /// The requested chunk data. + #[codec(index = 0)] + Chunk(ErasureChunk), + /// Node was not in possession of the requested chunk. + #[codec(index = 1)] + NoSuchChunk, +} + +impl From> for ChunkFetchingResponse { + fn from(x: Option) -> Self { + match x { + Some(c) => ChunkFetchingResponse::Chunk(c), + None => ChunkFetchingResponse::NoSuchChunk, + } + } +} + +impl From for Option { + fn from(x: ChunkFetchingResponse) -> Self { + match x { + ChunkFetchingResponse::Chunk(c) => Some(c), + ChunkFetchingResponse::NoSuchChunk => None, + } + } +} + +impl From for ChunkFetchingRequest { + fn from(v1::ChunkFetchingRequest { candidate_hash, index }: v1::ChunkFetchingRequest) -> Self { + Self { candidate_hash, index } + } +} + +impl From for v1::ChunkFetchingRequest { + fn from(ChunkFetchingRequest { candidate_hash, index }: ChunkFetchingRequest) -> Self { + Self { candidate_hash, index } + } +} + +impl IsRequest for ChunkFetchingRequest { + type Response = ChunkFetchingResponse; + const PROTOCOL: Protocol = Protocol::ChunkFetchingV2; +} diff --git a/polkadot/node/service/src/lib.rs b/polkadot/node/service/src/lib.rs index ff70dbde7349..21c9178f6b4e 100644 --- a/polkadot/node/service/src/lib.rs +++ b/polkadot/node/service/src/lib.rs @@ -869,7 +869,9 @@ pub fn new_full( let (pov_req_receiver, cfg) = IncomingRequest::get_config_receiver(&req_protocol_names); net_config.add_request_response_protocol(cfg); - let (chunk_req_receiver, cfg) = IncomingRequest::get_config_receiver(&req_protocol_names); + let (chunk_req_v1_receiver, cfg) = IncomingRequest::get_config_receiver(&req_protocol_names); + net_config.add_request_response_protocol(cfg); + let (chunk_req_v2_receiver, cfg) = IncomingRequest::get_config_receiver(&req_protocol_names); net_config.add_request_response_protocol(cfg); let (collation_req_v1_receiver, cfg) = IncomingRequest::get_config_receiver(&req_protocol_names); @@ -1074,7 +1076,8 @@ pub fn new_full( sync_service: sync_service.clone(), authority_discovery_service, pov_req_receiver, - chunk_req_receiver, + chunk_req_v1_receiver, + chunk_req_v2_receiver, collation_req_v1_receiver, collation_req_v2_receiver, available_data_req_receiver, diff --git a/polkadot/node/service/src/overseer.rs b/polkadot/node/service/src/overseer.rs index 701161e210d3..4e1c3b598acb 100644 --- a/polkadot/node/service/src/overseer.rs +++ b/polkadot/node/service/src/overseer.rs @@ -100,8 +100,10 @@ where pub authority_discovery_service: AuthorityDiscoveryService, /// POV request receiver. pub pov_req_receiver: IncomingRequestReceiver, - /// Erasure chunks request receiver. - pub chunk_req_receiver: IncomingRequestReceiver, + /// Erasure chunks request receiver for v1. + pub chunk_req_v1_receiver: IncomingRequestReceiver, + /// Erasure chunks request receiver for v2. + pub chunk_req_v2_receiver: IncomingRequestReceiver, /// Collations request receiver for network protocol v1. pub collation_req_v1_receiver: IncomingRequestReceiver, /// Collations request receiver for network protocol v2. @@ -156,7 +158,8 @@ pub fn prepared_overseer_builder( sync_service, authority_discovery_service, pov_req_receiver, - chunk_req_receiver, + chunk_req_v1_receiver, + chunk_req_v2_receiver, collation_req_v1_receiver, collation_req_v2_receiver, available_data_req_receiver, @@ -238,7 +241,7 @@ where network_service.clone(), authority_discovery_service.clone(), network_bridge_metrics.clone(), - req_protocol_names, + req_protocol_names.clone(), peerset_protocol_names.clone(), notification_sinks.clone(), )) @@ -253,11 +256,17 @@ where )) .availability_distribution(AvailabilityDistributionSubsystem::new( keystore.clone(), - IncomingRequestReceivers { pov_req_receiver, chunk_req_receiver }, + IncomingRequestReceivers { + pov_req_receiver, + chunk_req_v1_receiver, + chunk_req_v2_receiver, + }, + req_protocol_names.clone(), Metrics::register(registry)?, )) - .availability_recovery(AvailabilityRecoverySubsystem::with_systematic_chunks_if_pov_large( + .availability_recovery(AvailabilityRecoverySubsystem::for_validator( available_data_req_receiver, + &req_protocol_names, Metrics::register(registry)?, )) .availability_store(AvailabilityStoreSubsystem::new( diff --git a/polkadot/node/subsystem-types/src/messages.rs b/polkadot/node/subsystem-types/src/messages.rs index ef4006c9cf9e..dae51e622d54 100644 --- a/polkadot/node/subsystem-types/src/messages.rs +++ b/polkadot/node/subsystem-types/src/messages.rs @@ -45,7 +45,7 @@ use polkadot_primitives::{ async_backing, slashing, vstaging::{ApprovalVotingParams, NodeFeatures}, AuthorityDiscoveryId, BackedCandidate, BlockNumber, CandidateEvent, CandidateHash, - CandidateIndex, CandidateReceipt, ChunkIndex, CollatorId, CommittedCandidateReceipt, CoreState, + CandidateIndex, CandidateReceipt, CollatorId, CommittedCandidateReceipt, CoreIndex, CoreState, DisputeState, ExecutorParams, GroupIndex, GroupRotationInfo, Hash, Header as BlockHeader, Id as ParaId, InboundDownwardMessage, InboundHrmpMessage, MultiDisputeStatementSet, OccupiedCoreAssumption, PersistedValidationData, PvfCheckStatement, PvfExecKind, SessionIndex, @@ -472,8 +472,9 @@ pub enum AvailabilityRecoveryMessage { RecoverAvailableData( CandidateReceipt, SessionIndex, - Option, // Optional backing group to request from first. - Option, // Optional block number of the relay parent of the candidate. + Option, // Optional backing group to request from first. + Option, /* Optional core index that the candidate was occupying. Needed for + * systematic recovery. */ oneshot::Sender>, ), } @@ -503,20 +504,20 @@ pub enum AvailabilityStoreMessage { QueryDataAvailability(CandidateHash, oneshot::Sender), /// Query an `ErasureChunk` from the AV store by the candidate hash and validator index. - QueryChunk(CandidateHash, ChunkIndex, oneshot::Sender>), + QueryChunk(CandidateHash, ValidatorIndex, oneshot::Sender>), /// Get the size of an `ErasureChunk` from the AV store by the candidate hash. QueryChunkSize(CandidateHash, oneshot::Sender>), /// Query all chunks that we have for the given candidate hash. - QueryAllChunks(CandidateHash, oneshot::Sender>), + QueryAllChunks(CandidateHash, oneshot::Sender>), /// Query whether an `ErasureChunk` exists within the AV Store. /// /// This is useful in cases like bitfield signing, when existence /// matters, but we don't want to necessarily pass around large /// quantities of data to get a single bit of information. - QueryChunkAvailability(CandidateHash, ChunkIndex, oneshot::Sender), + QueryChunkAvailability(CandidateHash, ValidatorIndex, oneshot::Sender), /// Store an `ErasureChunk` in the AV store. /// @@ -524,6 +525,8 @@ pub enum AvailabilityStoreMessage { StoreChunk { /// A hash of the candidate this chunk belongs to. candidate_hash: CandidateHash, + /// Validator index. May not be equal to the chunk index. + validator_index: ValidatorIndex, /// The chunk itself. chunk: ErasureChunk, /// Sending side of the channel to send result to. @@ -543,6 +546,11 @@ pub enum AvailabilityStoreMessage { available_data: AvailableData, /// Erasure root we expect to get after chunking. expected_erasure_root: Hash, + /// Core index that the candidate will begin occupying (that the para is scheduled on). + core_index: CoreIndex, + /// Node features at the candidate relay parent. Used for computing the validator->chunk + /// mapping. + node_features: NodeFeatures, /// Sending side of the channel to send result to. tx: oneshot::Sender>, }, diff --git a/polkadot/node/subsystem-util/Cargo.toml b/polkadot/node/subsystem-util/Cargo.toml index 68e057e260ea..10494861c9d3 100644 --- a/polkadot/node/subsystem-util/Cargo.toml +++ b/polkadot/node/subsystem-util/Cargo.toml @@ -25,6 +25,7 @@ gum = { package = "tracing-gum", path = "../gum" } derive_more = "0.99.17" schnellru = "0.2.1" +erasure-coding = { package = "polkadot-erasure-coding", path = "../../erasure-coding" } polkadot-node-subsystem = { path = "../subsystem" } polkadot-node-subsystem-types = { path = "../subsystem-types" } polkadot-node-jaeger = { path = "../jaeger" } diff --git a/polkadot/node/subsystem-util/src/availability_chunks.rs b/polkadot/node/subsystem-util/src/availability_chunks.rs index a5c7945b7d96..337d08a68f5b 100644 --- a/polkadot/node/subsystem-util/src/availability_chunks.rs +++ b/polkadot/node/subsystem-util/src/availability_chunks.rs @@ -14,211 +14,66 @@ // You should have received a copy of the GNU General Public License // along with Polkadot. If not, see . -use polkadot_node_primitives::BabeRandomness; +use erasure_coding::systematic_recovery_threshold; use polkadot_primitives::{ vstaging::{node_features, NodeFeatures}, - BlockNumber, ChunkIndex, Id as ParaId, SessionIndex, ValidatorIndex, + ChunkIndex, CoreIndex, ValidatorIndex, }; -use rand::{seq::SliceRandom, Rng, SeedableRng}; -use rand_chacha::ChaCha8Rng; -use schnellru::{ByLength, LruMap}; - -/// Object used for holding and computing assigned chunk indices for validators. -pub struct ChunkIndexCacheRegistry( - LruMap<(BlockNumber, SessionIndex), (Vec, Option)>, -); - -impl ChunkIndexCacheRegistry { - /// Initialize with the cache capacity. - pub fn new(capacity: u32) -> Self { - Self(LruMap::new(ByLength::new(capacity))) - } +/// Compute the per-validator availability chunk index. +/// WARNING: THIS FUNCTION IS CRITICAL TO PARACHAIN CONSENSUS. +/// Any modification to the output of the function needs to be coordinated via the runtime. +/// It's best to use minimal/no external dependencies. +pub fn availability_chunk_index( + maybe_node_features: Option<&NodeFeatures>, + n_validators: usize, + core_index: CoreIndex, + validator_index: ValidatorIndex, +) -> Result { + if let Some(features) = maybe_node_features { + if let Some(&true) = features + .get(usize::from(node_features::FeatureIndex::AvailabilityChunkMapping as u8)) + .as_deref() + { + let systematic_threshold = systematic_recovery_threshold(n_validators)? as u32; + let core_start_pos = core_index.0 * systematic_threshold; - /// Return the per-validator chunk index if present in the cache. - pub fn query_cache_for_validator( - &mut self, - block_number: BlockNumber, - session_index: SessionIndex, - para_id: ParaId, - validator_index: ValidatorIndex, - ) -> Option { - if let Some((shuffle, maybe_node_features)) = self.0.get(&(block_number, session_index)) { - Some(Self::chunk_index_for_validator( - maybe_node_features.as_ref(), - shuffle, - para_id, - validator_index, - )) - } else { - None + return Ok(ChunkIndex((core_start_pos + validator_index.0) % n_validators as u32)) } } - /// Return the per-para chunk index vector if present in the cache. - pub fn query_cache_for_para( - &mut self, - block_number: BlockNumber, - session_index: SessionIndex, - para_id: ParaId, - ) -> Option> { - if let Some((shuffle, maybe_node_features)) = self.0.get(&(block_number, session_index)) { - let core_start_index = - Self::para_start_index(maybe_node_features.as_ref(), shuffle.len(), para_id); + Ok(validator_index.into()) +} - let chunk_indices = shuffle - .clone() +/// Compute the per-core availability chunk indices. Item on position i corresponds to the i-th +/// validator. +/// WARNING: THIS FUNCTION IS CRITICAL TO PARACHAIN CONSENSUS. +/// Any modification to the output of the function needs to be coordinated via the runtime. +/// It's best to use minimal/no external dependencies. +pub fn availability_chunk_indices( + maybe_node_features: Option<&NodeFeatures>, + n_validators: usize, + core_index: CoreIndex, +) -> Result, erasure_coding::Error> { + let identity = (0..n_validators).map(|index| ChunkIndex(index as u32)); + if let Some(features) = maybe_node_features { + if let Some(&true) = features + .get(usize::from(node_features::FeatureIndex::AvailabilityChunkMapping as u8)) + .as_deref() + { + let systematic_threshold = systematic_recovery_threshold(n_validators)? as u32; + let core_start_pos = core_index.0 * systematic_threshold; + + return Ok(identity .into_iter() .cycle() - .skip(core_start_index) - .take(shuffle.len()) - .collect(); - - Some(chunk_indices) - } else { - None + .skip(core_start_pos as usize) + .take(n_validators) + .collect()) } } - /// Return and populate the cache with the per-validator chunk index. - /// Should only be called if `query_cache_for_validator` returns `None`. - pub fn populate_for_validator( - &mut self, - maybe_node_features: Option, - babe_randomness: BabeRandomness, - n_validators: usize, - block_number: BlockNumber, - session_index: SessionIndex, - para_id: ParaId, - validator_index: ValidatorIndex, - ) -> ChunkIndex { - let shuffle = Self::get_shuffle( - maybe_node_features.as_ref(), - block_number, - babe_randomness, - n_validators, - ); - self.0.insert((block_number, session_index), (shuffle, maybe_node_features)); - - self.query_cache_for_validator(block_number, session_index, para_id, validator_index) - .expect("We just inserted the entry.") - } - - /// Return and populate the cache with the per-para chunk index vector. - /// Should only be called if `query_cache_for_para` returns `None`. - pub fn populate_for_para( - &mut self, - maybe_node_features: Option, - babe_randomness: BabeRandomness, - n_validators: usize, - block_number: BlockNumber, - session_index: SessionIndex, - para_id: ParaId, - ) -> Vec { - let shuffle = Self::get_shuffle( - maybe_node_features.as_ref(), - block_number, - babe_randomness, - n_validators, - ); - self.0.insert((block_number, session_index), (shuffle, maybe_node_features)); - - self.query_cache_for_para(block_number, session_index, para_id) - .expect("We just inserted the entry.") - } - - fn get_shuffle( - maybe_node_features: Option<&NodeFeatures>, - block_number: BlockNumber, - mut babe_randomness: BabeRandomness, - n_validators: usize, - ) -> Vec { - let mut indices: Vec<_> = (0..n_validators) - .map(|i| ChunkIndex(u32::try_from(i).expect("validator count should not exceed u32"))) - .collect(); - - if let Some(features) = maybe_node_features { - if let Some(&true) = features - .get(usize::from(node_features::FeatureIndex::AvailabilityChunkShuffling as u8)) - .as_deref() - { - let block_number_bytes = block_number.to_be_bytes(); - for i in 0..32 { - babe_randomness[i] ^= block_number_bytes[i % block_number_bytes.len()]; - } - - let mut rng: ChaCha8Rng = SeedableRng::from_seed(babe_randomness); - - indices.shuffle(&mut rng); - } - } - - indices - } - - /// Return the availability chunk start index for this para. - fn para_start_index( - maybe_node_features: Option<&NodeFeatures>, - n_validators: usize, - para_id: ParaId, - ) -> usize { - if let Some(features) = maybe_node_features { - if let Some(&true) = features - .get(usize::from(node_features::FeatureIndex::AvailabilityChunkShuffling as u8)) - .as_deref() - { - let mut rng: ChaCha8Rng = - SeedableRng::from_seed( - u32::from(para_id).to_be_bytes().repeat(8).try_into().expect( - "vector of 32 bytes is safe to cast to array of 32 bytes. qed.", - ), - ); - return rng.gen_range(0..n_validators) - } - } - - 0 - } - - fn chunk_index_for_validator( - maybe_node_features: Option<&NodeFeatures>, - shuffle: &Vec, - para_id: ParaId, - validator_index: ValidatorIndex, - ) -> ChunkIndex { - let core_start_index = Self::para_start_index(maybe_node_features, shuffle.len(), para_id); - - let chunk_index = shuffle[(core_start_index + - usize::try_from(validator_index.0) - .expect("usize is at least u32 bytes on all modern targets.")) % - shuffle.len()]; - chunk_index - } -} - -/// Compute the per-validator availability chunk index. -/// It's preferred to use the `ChunkIndexCacheRegistry` if you also need a cache. -pub fn availability_chunk_index( - maybe_node_features: Option<&NodeFeatures>, - babe_randomness: BabeRandomness, - n_validators: usize, - block_number: BlockNumber, - para_id: ParaId, - validator_index: ValidatorIndex, -) -> ChunkIndex { - let shuffle = ChunkIndexCacheRegistry::get_shuffle( - maybe_node_features, - block_number, - babe_randomness, - n_validators, - ); - - ChunkIndexCacheRegistry::chunk_index_for_validator( - maybe_node_features, - &shuffle, - para_id, - validator_index, - ) + Ok(identity.collect()) } #[cfg(test)] @@ -226,196 +81,150 @@ mod tests { use super::*; use std::collections::HashSet; - pub fn node_features_with_shuffling() -> NodeFeatures { + pub fn node_features_with_mapping_enabled() -> NodeFeatures { let mut node_features = NodeFeatures::new(); node_features - .resize(node_features::FeatureIndex::AvailabilityChunkShuffling as usize + 1, false); + .resize(node_features::FeatureIndex::AvailabilityChunkMapping as usize + 1, false); + node_features + .set(node_features::FeatureIndex::AvailabilityChunkMapping as u8 as usize, true); + node_features + } + + pub fn node_features_with_other_bits_enabled() -> NodeFeatures { + let mut node_features = NodeFeatures::new(); + node_features.resize(node_features::FeatureIndex::FirstUnassigned as usize + 1, true); node_features - .set(node_features::FeatureIndex::AvailabilityChunkShuffling as u8 as usize, true); + .set(node_features::FeatureIndex::AvailabilityChunkMapping as u8 as usize, false); node_features } #[test] fn test_availability_chunk_indices() { - let block_number = 89; - let n_validators = 11u32; - let babe_randomness = [12u8; 32]; - let session_index = 0; - let n_paras = 5u32; + let n_validators = 20u32; + let n_cores = 15u32; - // Test the `_for_validator` methods + // If the mapping feature is not enabled, it should always be the identity vector. { - let para_id = 2.into(); - let mut index_registry = ChunkIndexCacheRegistry::new(2); - - for validator in 0..n_validators { - assert!(index_registry - .query_cache_for_validator( - block_number, - session_index, - para_id, - validator.into() - ) - .is_none()); - } - - for validator in 0..n_validators { - // Check that if the node feature is not set, we'll always return the validator - // index. - let chunk_index = index_registry.populate_for_validator( - None, - babe_randomness, - n_validators as usize, - block_number, - session_index, - para_id, - validator.into(), - ); - assert_eq!( - index_registry - .query_cache_for_validator( - block_number, - session_index, - para_id, - validator.into() - ) - .unwrap(), - chunk_index - ); - assert_eq!(chunk_index.0, validator); - assert_eq!( - chunk_index, - availability_chunk_index( - None, - babe_randomness, + for node_features in + [None, Some(NodeFeatures::EMPTY), Some(node_features_with_other_bits_enabled())] + { + for core_index in 0..n_cores { + let indices = availability_chunk_indices( + node_features.as_ref(), n_validators as usize, - block_number, - para_id, - validator.into(), + CoreIndex(core_index), ) - ); - - // Check for when the node feature is set. - let chunk_index = index_registry.populate_for_validator( - Some(node_features_with_shuffling()), - babe_randomness, - n_validators as usize, - block_number, - session_index, - para_id, - validator.into(), - ); - assert_eq!( - index_registry - .query_cache_for_validator( - block_number, - session_index, - para_id, - validator.into() + .unwrap(); + + for validator_index in 0..n_validators { + assert_eq!( + indices[validator_index as usize], + availability_chunk_index( + node_features.as_ref(), + n_validators as usize, + CoreIndex(core_index), + ValidatorIndex(validator_index) + ) + .unwrap() ) - .unwrap(), - chunk_index - ); - assert_ne!(chunk_index.0, validator); - assert_eq!( - chunk_index, - availability_chunk_index( - Some(&node_features_with_shuffling()), - babe_randomness, - n_validators as usize, - block_number, - para_id, - validator.into(), - ) - ); + } + + assert_eq!( + indices, + (0..n_validators).map(|i| ChunkIndex(i)).collect::>() + ); + } } } - // Test the `_for_para` methods + // Test when mapping feature is enabled. { - let mut index_registry = ChunkIndexCacheRegistry::new(2); - - for para in 0..n_paras { - assert!(index_registry - .query_cache_for_para(block_number, session_index, para.into()) - .is_none()); - } + let node_features = node_features_with_mapping_enabled(); + let mut previous_indices = None; - for para in 0..n_paras { - // Check that if the node feature is not set, we'll always return the identity - // vector. - let chunk_indices = index_registry.populate_for_para( - None, - babe_randomness, + for core_index in 0..n_cores { + let indices = availability_chunk_indices( + Some(&node_features), n_validators as usize, - block_number, - session_index, - para.into(), - ); - assert_eq!( - index_registry - .query_cache_for_para(block_number, session_index, para.into()) - .unwrap(), - chunk_indices - ); - assert_eq!( - chunk_indices, - (0..n_validators).map(|i| ChunkIndex(i)).collect::>() - ); + CoreIndex(core_index), + ) + .unwrap(); - for validator in 0..n_validators { + for validator_index in 0..n_validators { assert_eq!( + indices[validator_index as usize], availability_chunk_index( - None, - babe_randomness, + Some(&node_features), n_validators as usize, - block_number, - para.into(), - validator.into(), - ), - chunk_indices[validator as usize] - ); + CoreIndex(core_index), + ValidatorIndex(validator_index) + ) + .unwrap() + ) } - // Check for when the node feature is set. - let chunk_indices = index_registry.populate_for_para( - Some(node_features_with_shuffling()), - babe_randomness, - n_validators as usize, - block_number, - session_index, - para.into(), - ); - assert_eq!( - index_registry - .query_cache_for_para(block_number, session_index, para.into()) - .unwrap(), - chunk_indices - ); - assert_eq!(chunk_indices.len(), n_validators as usize); - assert_ne!( - chunk_indices, - (0..n_validators).map(|i| ChunkIndex(i)).collect::>() - ); + // Check that it's not equal to the previous core's indices. + if let Some(previous_indices) = previous_indices { + assert_ne!(previous_indices, indices); + } + + previous_indices = Some(indices.clone()); + + // Check that it's indeed a permutation. assert_eq!( - chunk_indices.iter().collect::>().len(), - n_validators as usize + (0..n_validators).map(|i| ChunkIndex(i)).collect::>(), + indices.into_iter().collect::>() ); - - for validator in 0..n_validators { - assert_eq!( - availability_chunk_index( - Some(&node_features_with_shuffling()), - babe_randomness, - n_validators as usize, - block_number, - para.into(), - validator.into(), - ), - chunk_indices[validator as usize] - ); - } } } } + + #[test] + // This is just a dummy test that checks the mapping against some hardcoded outputs, to prevent + // accidental changes to the algorithms. + fn prevent_changes_to_mapping() { + let n_validators = 7; + let node_features = node_features_with_mapping_enabled(); + + assert_eq!( + availability_chunk_indices(Some(&node_features), n_validators, CoreIndex(0)) + .unwrap() + .into_iter() + .map(|i| i.0) + .collect::>(), + vec![0, 1, 2, 3, 4, 5, 6] + ); + assert_eq!( + availability_chunk_indices(Some(&node_features), n_validators, CoreIndex(1)) + .unwrap() + .into_iter() + .map(|i| i.0) + .collect::>(), + vec![2, 3, 4, 5, 6, 0, 1] + ); + assert_eq!( + availability_chunk_indices(Some(&node_features), n_validators, CoreIndex(2)) + .unwrap() + .into_iter() + .map(|i| i.0) + .collect::>(), + vec![4, 5, 6, 0, 1, 2, 3] + ); + assert_eq!( + availability_chunk_indices(Some(&node_features), n_validators, CoreIndex(3)) + .unwrap() + .into_iter() + .map(|i| i.0) + .collect::>(), + vec![6, 0, 1, 2, 3, 4, 5] + ); + assert_eq!( + availability_chunk_indices(Some(&node_features), n_validators, CoreIndex(4)) + .unwrap() + .into_iter() + .map(|i| i.0) + .collect::>(), + vec![1, 2, 3, 4, 5, 6, 0] + ); + } } diff --git a/polkadot/node/subsystem-util/src/lib.rs b/polkadot/node/subsystem-util/src/lib.rs index 6633b64be3fa..09b0dc98efc6 100644 --- a/polkadot/node/subsystem-util/src/lib.rs +++ b/polkadot/node/subsystem-util/src/lib.rs @@ -32,9 +32,8 @@ pub use overseer::{ use polkadot_node_subsystem::{ errors::{RuntimeApiError, SubsystemError}, messages::{RuntimeApiMessage, RuntimeApiRequest, RuntimeApiSender}, - overseer, ChainApiError, SubsystemSender, + overseer, SubsystemSender, }; -use polkadot_node_subsystem_types::messages::ChainApiMessage; pub use polkadot_node_metrics::{metrics, Metronome}; @@ -42,7 +41,7 @@ use futures::channel::{mpsc, oneshot}; use parity_scale_codec::Encode; use polkadot_primitives::{ - slashing, AsyncBackingParams, AuthorityDiscoveryId, BlockNumber, CandidateEvent, CandidateHash, + slashing, AsyncBackingParams, AuthorityDiscoveryId, CandidateEvent, CandidateHash, CommittedCandidateReceipt, CoreState, EncodeAs, ExecutorParams, GroupIndex, GroupRotationInfo, Hash, Id as ParaId, OccupiedCoreAssumption, PersistedValidationData, ScrapedOnChainVotes, SessionIndex, SessionInfo, Signed, SigningContext, ValidationCode, ValidationCodeHash, @@ -460,18 +459,21 @@ pub struct Validator { impl Validator { /// Get a struct representing this node's validator if this node is in fact a validator in the /// context of the given block. - pub async fn new( - validators: &[ValidatorId], - parent: Hash, - keystore: KeystorePtr, - sender: &mut S, - ) -> Result + pub async fn new(parent: Hash, keystore: KeystorePtr, sender: &mut S) -> Result where S: SubsystemSender, { - let session_index = request_session_index_for_child(parent, sender).await.await??; + // Note: request_validators and request_session_index_for_child do not and cannot + // run concurrently: they both have a mutable handle to the same sender. + // However, each of them returns a oneshot::Receiver, and those are resolved concurrently. + let (validators, session_index) = futures::try_join!( + request_validators(parent, sender).await, + request_session_index_for_child(parent, sender).await, + )?; + + let signing_context = SigningContext { session_index: session_index?, parent_hash: parent }; - let signing_context = SigningContext { session_index, parent_hash: parent }; + let validators = validators?; // TODO: https://github.com/paritytech/polkadot-sdk/issues/1940 // When `DisabledValidators` is released remove this and add a @@ -529,18 +531,3 @@ impl Validator { Signed::sign(&keystore, payload, &self.signing_context, self.index, &self.key) } } - -/// Get the block number by hash. -pub async fn get_block_number( - sender: &mut Sender, - relay_parent: Hash, -) -> Result, E> -where - Sender: overseer::SubsystemSender, - E: From + From, -{ - let (tx, rx) = oneshot::channel(); - sender.send_message(ChainApiMessage::BlockNumber(relay_parent, tx)).await; - - rx.await?.map_err(Into::into) -} diff --git a/polkadot/node/subsystem-util/src/runtime/mod.rs b/polkadot/node/subsystem-util/src/runtime/mod.rs index 481625acb321..60dd379cec93 100644 --- a/polkadot/node/subsystem-util/src/runtime/mod.rs +++ b/polkadot/node/subsystem-util/src/runtime/mod.rs @@ -32,10 +32,10 @@ use polkadot_node_subsystem_types::UnpinHandle; use polkadot_primitives::{ slashing, vstaging::{node_features::FeatureIndex, NodeFeatures}, - AsyncBackingParams, CandidateEvent, CandidateHash, CoreState, EncodeAs, ExecutorParams, - GroupIndex, GroupRotationInfo, Hash, IndexedVec, OccupiedCore, ScrapedOnChainVotes, - SessionIndex, SessionInfo, Signed, SigningContext, UncheckedSigned, ValidationCode, - ValidationCodeHash, ValidatorId, ValidatorIndex, LEGACY_MIN_BACKING_VOTES, + AsyncBackingParams, CandidateEvent, CandidateHash, CoreIndex, CoreState, EncodeAs, + ExecutorParams, GroupIndex, GroupRotationInfo, Hash, IndexedVec, OccupiedCore, + ScrapedOnChainVotes, SessionIndex, SessionInfo, Signed, SigningContext, UncheckedSigned, + ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, LEGACY_MIN_BACKING_VOTES, }; use crate::{ @@ -349,7 +349,7 @@ where pub async fn get_occupied_cores( sender: &mut Sender, relay_parent: Hash, -) -> Result> +) -> Result> where Sender: overseer::SubsystemSender, { @@ -357,9 +357,10 @@ where Ok(cores .into_iter() - .filter_map(|core_state| { + .enumerate() + .filter_map(|(core_index, core_state)| { if let CoreState::Occupied(occupied) = core_state { - Some(occupied) + Some((CoreIndex(core_index as u32), occupied)) } else { None } diff --git a/polkadot/primitives/src/v6/mod.rs b/polkadot/primitives/src/v6/mod.rs index a8b628abe8cf..38d82cca7bda 100644 --- a/polkadot/primitives/src/v6/mod.rs +++ b/polkadot/primitives/src/v6/mod.rs @@ -1687,6 +1687,14 @@ where self.0.get(index.type_index()) } + /// Returns a mutable reference to an element indexed using `K`. + pub fn get_mut(&mut self, index: K) -> Option<&mut V> + where + K: TypeIndex, + { + self.0.get_mut(index.type_index()) + } + /// Returns number of elements in vector. pub fn len(&self) -> usize { self.0.len() diff --git a/polkadot/primitives/src/vstaging/mod.rs b/polkadot/primitives/src/vstaging/mod.rs index 834f3f5daf2a..96681a34bc43 100644 --- a/polkadot/primitives/src/vstaging/mod.rs +++ b/polkadot/primitives/src/vstaging/mod.rs @@ -61,12 +61,13 @@ pub mod node_features { /// A feature index used to indentify a bit into the node_features array stored /// in the HostConfiguration. #[repr(u8)] + #[derive(Clone, Copy)] pub enum FeatureIndex { /// Tells if tranch0 assignments could be sent in a single certificate. /// Reserved for: `` EnableAssignmentsV2 = 0, - /// Index of the availability chunk shuffling feature bit. - AvailabilityChunkShuffling = 1, + /// Index of the availability chunk mapping feature bit. + AvailabilityChunkMapping = 1, /// First unassigned feature bit. /// Every time a new feature flag is assigned it should take this value. /// and this should be incremented. diff --git a/prdoc/pr_1644.prdoc b/prdoc/pr_1644.prdoc new file mode 100644 index 000000000000..125ce63f13ac --- /dev/null +++ b/prdoc/pr_1644.prdoc @@ -0,0 +1,8 @@ +title: Add availability-recovery from systematic chunks + +doc: + - audience: Node Dev + description: | + ... + +crates: [ ] diff --git a/substrate/client/network/src/service.rs b/substrate/client/network/src/service.rs index 47e23337633b..e40008f83321 100644 --- a/substrate/client/network/src/service.rs +++ b/substrate/client/network/src/service.rs @@ -467,7 +467,7 @@ where crate::MAX_CONNECTIONS_ESTABLISHED_INCOMING, )), ) - .substream_upgrade_protocol_override(upgrade::Version::V1Lazy) + .substream_upgrade_protocol_override(upgrade::Version::V1) .notify_handler_buffer_size(NonZeroUsize::new(32).expect("32 != 0; qed")) // NOTE: 24 is somewhat arbitrary and should be tuned in the future if necessary. // See From 4530caa232efae1eb8816be4c49afdbf974836dc Mon Sep 17 00:00:00 2001 From: alindima Date: Thu, 18 Jan 2024 11:23:15 +0200 Subject: [PATCH 081/126] fix av-store tests --- polkadot/node/core/av-store/src/tests.rs | 64 ++++++++++++++++++++---- 1 file changed, 53 insertions(+), 11 deletions(-) diff --git a/polkadot/node/core/av-store/src/tests.rs b/polkadot/node/core/av-store/src/tests.rs index 14c4297c2657..3061044dfabb 100644 --- a/polkadot/node/core/av-store/src/tests.rs +++ b/polkadot/node/core/av-store/src/tests.rs @@ -472,6 +472,7 @@ fn store_pov_and_queries_work() { for core_index in 0..n_cores { let store = test_store(); let test_state = TestState::default(); + let core_index = CoreIndex(core_index); test_harness(test_state.clone(), store.clone(), |mut virtual_overseer| async move { let node_features = NodeFeatures::EMPTY; @@ -494,9 +495,9 @@ fn store_pov_and_queries_work() { n_validators, available_data: available_data.clone(), tx, + core_index, expected_erasure_root: branches.root(), node_features: node_features.clone(), - core_index: CoreIndex(core_index), }; virtual_overseer.send(FromOrchestra::Communication { msg: block_msg }).await; @@ -506,8 +507,17 @@ fn store_pov_and_queries_work() { query_available_data(&mut virtual_overseer, candidate_hash).await.unwrap(); assert_eq!(pov, available_data); - let query_all_chunks_res = - query_all_chunks(&mut virtual_overseer, candidate_hash).await; + let query_all_chunks_res = query_all_chunks( + &mut virtual_overseer, + availability_chunk_indices( + Some(&node_features), + n_validators as usize, + core_index, + ) + .unwrap(), + candidate_hash, + ) + .await; assert_eq!(query_all_chunks_res.len(), chunks.len()); let branches: Vec<_> = branches.collect(); @@ -561,6 +571,7 @@ fn store_pov_and_queries_work() { let chunks = erasure::obtain_chunks_v1(n_validators as _, &available_data).unwrap(); let branches = erasure::branches(chunks.as_ref()); + let core_index = CoreIndex(core_index); let (tx, rx) = oneshot::channel(); let block_msg = AvailabilityStoreMessage::StoreAvailableData { @@ -568,9 +579,9 @@ fn store_pov_and_queries_work() { n_validators, available_data: available_data.clone(), tx, + core_index, expected_erasure_root: branches.root(), node_features: node_features.clone(), - core_index: CoreIndex(core_index), }; virtual_overseer.send(FromOrchestra::Communication { msg: block_msg }).await; @@ -580,8 +591,17 @@ fn store_pov_and_queries_work() { query_available_data(&mut virtual_overseer, candidate_hash).await.unwrap(); assert_eq!(pov, available_data); - let query_all_chunks_res = - query_all_chunks(&mut virtual_overseer, candidate_hash).await; + let query_all_chunks_res = query_all_chunks( + &mut virtual_overseer, + availability_chunk_indices( + Some(&node_features), + n_validators as usize, + core_index, + ) + .unwrap(), + candidate_hash, + ) + .await; assert_eq!(query_all_chunks_res.len(), chunks.len()); let branches: Vec<_> = branches.collect(); @@ -597,7 +617,7 @@ fn store_pov_and_queries_work() { let expected_chunk_index = availability_chunk_index( Some(&node_features), n_validators as usize, - CoreIndex(core_index), + core_index, ValidatorIndex(validator_index), ) .unwrap(); @@ -698,13 +718,28 @@ fn query_all_chunks_works() { assert_eq!(rx.await.unwrap(), Ok(())); } + let chunk_indices = + availability_chunk_indices(None, n_validators as usize, CoreIndex(0)).unwrap(); + assert_eq!( - query_all_chunks(&mut virtual_overseer, candidate_hash_1).await.len(), + query_all_chunks(&mut virtual_overseer, chunk_indices.clone(), candidate_hash_1) + .await + .len(), n_validators as usize ); - assert_eq!(query_all_chunks(&mut virtual_overseer, candidate_hash_2).await.len(), 1); - assert_eq!(query_all_chunks(&mut virtual_overseer, candidate_hash_3).await.len(), 0); + assert_eq!( + query_all_chunks(&mut virtual_overseer, chunk_indices.clone(), candidate_hash_2) + .await + .len(), + 1 + ); + assert_eq!( + query_all_chunks(&mut virtual_overseer, chunk_indices.clone(), candidate_hash_3) + .await + .len(), + 0 + ); virtual_overseer }); @@ -1204,6 +1239,7 @@ async fn query_chunk( async fn query_all_chunks( virtual_overseer: &mut VirtualOverseer, + chunk_mapping: Vec, candidate_hash: CandidateHash, ) -> Vec { let (tx, rx) = oneshot::channel(); @@ -1211,7 +1247,13 @@ async fn query_all_chunks( let msg = AvailabilityStoreMessage::QueryAllChunks(candidate_hash, tx); virtual_overseer.send(FromOrchestra::Communication { msg }).await; - rx.await.unwrap() + let resp = rx.await.unwrap(); + resp.into_iter() + .map(|(val_idx, chunk)| { + assert_eq!(chunk.index, chunk_mapping[val_idx.0 as usize]); + chunk + }) + .collect() } async fn has_all_chunks( From 1ee008c79beec92aa5d89c7f17c81939c551e1b9 Mon Sep 17 00:00:00 2001 From: alindima Date: Thu, 18 Jan 2024 13:29:21 +0200 Subject: [PATCH 082/126] don't allow systematic recovery if chunk mapping is disabled --- .../network/availability-recovery/src/lib.rs | 23 ++++--- .../availability-recovery/src/tests.rs | 69 ++++++++++++++++++- 2 files changed, 78 insertions(+), 14 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index f2bf2ccb2b6e..122147f681c4 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -471,15 +471,25 @@ async fn handle_recover( } } + let chunk_mapping_enabled = if let Some(&true) = node_features + .get(usize::from(node_features::FeatureIndex::AvailabilityChunkMapping as u8)) + .as_deref() + { + true + } else { + false + }; + // We can only attempt systematic recovery if we received the core index of the - // candidate. + // candidate and chunk mapping is enabled. if let Some(core_index) = maybe_core_index { if matches!( recovery_strategy_kind, RecoveryStrategyKind::BackersThenSystematicChunks | RecoveryStrategyKind::SystematicChunks | RecoveryStrategyKind::BackersFirstIfSizeLowerThenSystematicChunks(_) - ) { + ) && chunk_mapping_enabled + { let chunk_indices = availability_chunk_indices(Some(node_features), n_validators, core_index)?; @@ -527,15 +537,6 @@ async fn handle_recover( let n_validators = session_info.validators.len(); - let chunk_mapping_enabled = if let Some(&true) = node_features - .get(usize::from(node_features::FeatureIndex::AvailabilityChunkMapping as u8)) - .as_deref() - { - true - } else { - false - }; - launch_recovery_task( state, ctx, diff --git a/polkadot/node/network/availability-recovery/src/tests.rs b/polkadot/node/network/availability-recovery/src/tests.rs index 7fd5e3ccd046..6d8e3598b30c 100644 --- a/polkadot/node/network/availability-recovery/src/tests.rs +++ b/polkadot/node/network/availability-recovery/src/tests.rs @@ -2578,7 +2578,7 @@ fn chunk_indices_are_mapped_to_different_validators( let validator_index = req.payload.index; let chunk_index = test_state.chunks.get(validator_index).unwrap().index; - if systematic_recovery { + if systematic_recovery && mapping_enabled { assert!((chunk_index.0 as usize) <= test_state.systematic_threshold(), "requested non-systematic chunk"); } @@ -2823,7 +2823,8 @@ fn systematic_recovery_retries_from_backers() { #[case(false)] fn test_legacy_network_protocol_with_mapping_disabled(#[case] systematic_recovery: bool) { // In this case, when the mapping is disabled, recovery will work with both v2 and v1 requests, - // under the assumption that ValidatorIndex is always equal to ChunkIndex. + // under the assumption that ValidatorIndex is always equal to ChunkIndex. However, systematic + // recovery will not be possible, it will fall back to regular recovery. let test_state = TestState::with_empty_node_features(); let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); let (subsystem, threshold) = match systematic_recovery { @@ -2885,7 +2886,7 @@ fn test_legacy_network_protocol_with_mapping_disabled(#[case] systematic_recover &mut virtual_overseer, threshold, |_| Has::Yes, - systematic_recovery, + false, ) .await; @@ -3047,3 +3048,65 @@ fn test_systematic_recovery_skipped_if_no_core_index() { virtual_overseer }); } + +#[test] +fn test_systematic_recovery_skipped_if_mapping_disabled() { + let test_state = TestState::with_empty_node_features(); + let req_protocol_names = ReqProtocolNames::new(&GENESIS_HASH, None); + let subsystem = AvailabilityRecoverySubsystem::for_validator( + request_receiver(&req_protocol_names), + &req_protocol_names, + Metrics::new_dummy(), + ); + + test_harness(subsystem, |mut virtual_overseer| async move { + overseer_signal( + &mut virtual_overseer, + OverseerSignal::ActiveLeaves(ActiveLeavesUpdate::start_work(new_leaf( + test_state.current, + 1, + ))), + ) + .await; + + let (tx, rx) = oneshot::channel(); + + overseer_send( + &mut virtual_overseer, + AvailabilityRecoveryMessage::RecoverAvailableData( + test_state.candidate.clone(), + test_state.session_index, + None, + Some(test_state.core_index), + tx, + ), + ) + .await; + + test_state.test_runtime_api_session_info(&mut virtual_overseer).await; + + test_state.test_runtime_api_node_features(&mut virtual_overseer).await; + + let candidate_hash = test_state.candidate.hash(); + + test_state.respond_to_available_data_query(&mut virtual_overseer, false).await; + test_state.respond_to_query_all_request(&mut virtual_overseer, |_| false).await; + + // Systematic recovery not possible without core index, falling back to regular recovery. + test_state + .test_chunk_requests( + &req_protocol_names, + candidate_hash, + &mut virtual_overseer, + test_state.validators.len() - test_state.threshold(), + |_| Has::No, + false, + ) + .await; + + // Make it fail, in order to assert that indeed regular recovery was attempted. If it were + // systematic recovery, we would have had one more attempt for regular reconstruction. + assert_eq!(rx.await.unwrap().unwrap_err(), RecoveryError::Unavailable); + virtual_overseer + }); +} From 46486db0accd6ac8c98739d65bd9184f9641278a Mon Sep 17 00:00:00 2001 From: alindima Date: Thu, 18 Jan 2024 13:32:01 +0200 Subject: [PATCH 083/126] launch systematic recovery during approval voting --- polkadot/node/core/approval-voting/src/lib.rs | 22 ++++++++++++++----- 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/polkadot/node/core/approval-voting/src/lib.rs b/polkadot/node/core/approval-voting/src/lib.rs index 3b63a7e938c5..4bd2ddfc0b1e 100644 --- a/polkadot/node/core/approval-voting/src/lib.rs +++ b/polkadot/node/core/approval-voting/src/lib.rs @@ -55,9 +55,9 @@ use polkadot_node_subsystem_util::{ }; use polkadot_primitives::{ vstaging::{ApprovalVoteMultipleCandidates, ApprovalVotingParams}, - BlockNumber, CandidateHash, CandidateIndex, CandidateReceipt, DisputeStatement, ExecutorParams, - GroupIndex, Hash, PvfExecKind, SessionIndex, SessionInfo, ValidDisputeStatementKind, - ValidatorId, ValidatorIndex, ValidatorPair, ValidatorSignature, + BlockNumber, CandidateHash, CandidateIndex, CandidateReceipt, CoreIndex, DisputeStatement, + ExecutorParams, GroupIndex, Hash, PvfExecKind, SessionIndex, SessionInfo, + ValidDisputeStatementKind, ValidatorId, ValidatorIndex, ValidatorPair, ValidatorSignature, }; use sc_keystore::LocalKeystore; use sp_application_crypto::Pair; @@ -898,6 +898,7 @@ enum Action { candidate: CandidateReceipt, backing_group: GroupIndex, distribute_assignment: bool, + core_index: Option, }, NoteApprovedInChainSelection(Hash), IssueApproval(CandidateHash, ApprovalVoteRequest), @@ -1157,6 +1158,7 @@ async fn handle_actions( candidate, backing_group, distribute_assignment, + core_index, } => { // Don't launch approval work if the node is syncing. if let Mode::Syncing(_) = *mode { @@ -1213,6 +1215,7 @@ async fn handle_actions( block_hash, backing_group, executor_params, + core_index, &launch_approval_span, ) .await @@ -2948,6 +2951,11 @@ async fn process_wakeup( "Launching approval work.", ); + let candidate_core_index = block_entry + .candidates() + .iter() + .find_map(|(core_index, h)| (h == &candidate_hash).then_some(*core_index)); + if let Some(claimed_core_indices) = get_assignment_core_indices(&indirect_cert.cert.kind, &candidate_hash, &block_entry) { @@ -2960,7 +2968,6 @@ async fn process_wakeup( true }; db.write_block_entry(block_entry.clone()); - actions.push(Action::LaunchApproval { claimed_candidate_indices, candidate_hash, @@ -2972,10 +2979,12 @@ async fn process_wakeup( candidate: candidate_receipt, backing_group, distribute_assignment, + core_index: candidate_core_index, }); }, Err(err) => { - // Never happens, it should only happen if no cores are claimed, which is a bug. + // Never happens, it should only happen if no cores are claimed, which is a + // bug. gum::warn!( target: LOG_TARGET, block_hash = ?relay_block, @@ -3030,6 +3039,7 @@ async fn launch_approval( block_hash: Hash, backing_group: GroupIndex, executor_params: ExecutorParams, + core_index: Option, span: &jaeger::Span, ) -> SubsystemResult> { let (a_tx, a_rx) = oneshot::channel(); @@ -3076,7 +3086,7 @@ async fn launch_approval( candidate.clone(), session_index, Some(backing_group), - None, + core_index, a_tx, )) .await; From 199bc88839072d1779e94fad44590861f94aab62 Mon Sep 17 00:00:00 2001 From: alindima Date: Thu, 18 Jan 2024 17:50:05 +0200 Subject: [PATCH 084/126] add zombienet test for network-level compatibility --- .gitlab/pipeline/zombienet/polkadot.yml | 8 ++++ ...-chunk-fetching-network-compatibility.toml | 48 +++++++++++++++++++ ...chunk-fetching-network-compatibility.zndsl | 48 +++++++++++++++++++ 3 files changed, 104 insertions(+) create mode 100644 polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.toml create mode 100644 polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.zndsl diff --git a/.gitlab/pipeline/zombienet/polkadot.yml b/.gitlab/pipeline/zombienet/polkadot.yml index 7f5d424ec1b6..9fed0ab227bf 100644 --- a/.gitlab/pipeline/zombienet/polkadot.yml +++ b/.gitlab/pipeline/zombienet/polkadot.yml @@ -150,6 +150,14 @@ zombienet-polkadot-functional-0010-validator-disabling: --local-dir="${LOCAL_DIR}/functional" --test="0010-validator-disabling.zndsl" +zombienet-polkadot-functional-0011-chunk-fetching-network-compatibility: + extends: + - .zombienet-polkadot-common + script: + - /home/nonroot/zombie-net/scripts/ci/run-test-local-env-manager.sh + --local-dir="${LOCAL_DIR}/functional" + --test="0011-chunk-fetching-network-compatibility.zndsl" + zombienet-polkadot-smoke-0001-parachains-smoke-test: extends: - .zombienet-polkadot-common diff --git a/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.toml b/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.toml new file mode 100644 index 000000000000..3c1f3f06ca14 --- /dev/null +++ b/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.toml @@ -0,0 +1,48 @@ +[settings] +timeout = 1000 +bootnode = true + +[relaychain.genesis.runtimeGenesis.patch.configuration.config] + max_validators_per_core = 2 + needed_approvals = 4 + +[relaychain] +default_image = "{{ZOMBIENET_INTEGRATION_TEST_IMAGE}}" +chain = "rococo-local" +default_command = "polkadot" + +[relaychain.default_resources] +limits = { memory = "4G", cpu = "2" } +requests = { memory = "2G", cpu = "1" } + + [[relaychain.node_groups]] + # Fix this to an image that doesn't speak /req_chunk/2 protocol. + image = "{{PARACHAINS_IMAGE_NAME}}:v1.0.0" + name = "old" + count = 2 + args = ["-lparachain=debug,parachain::availability-recovery=trace,parachain::availability-distribution=trace"] + + [[relaychain.node_groups]] + name = "new" + count = 2 + args = ["-lparachain=debug,parachain::availability-recovery=trace,parachain::availability-distribution=trace,sub-libp2p=trace"] + +{% for id in range(2000,2002) %} +[[parachains]] +id = {{id}} +addToGenesis = true +cumulus_based = true +chain = "glutton-westend-local-{{id}}" + + [parachains.genesis.runtimeGenesis.patch.glutton] + compute = "50000000" + storage = "131000000" + trashDataCount = 5120 + + [parachains.collator] + name = "collator" + image = "{{CUMULUS_IMAGE}}" + command = "polkadot-parachain" + args = ["-lparachain=debug"] + +{% endfor %} diff --git a/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.zndsl b/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.zndsl new file mode 100644 index 000000000000..7620eed4b2f1 --- /dev/null +++ b/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.zndsl @@ -0,0 +1,48 @@ +Description: Validators preserve backwards compatibility with peers speaking an older version of the /req_chunk protocol +Network: ./0011-chunk-fetching-network-compatibility.toml +Creds: config + +# Check authority status. +new: reports node_roles is 4 +old: reports node_roles is 4 + +# Ensure parachains are registered. +new: parachain 2000 is registered within 60 seconds +old: parachain 2000 is registered within 60 seconds +old: parachain 2001 is registered within 60 seconds +new: parachain 2001 is registered within 60 seconds + +# Ensure parachains made progress and approval checking works. +new: parachain 2000 block height is at least 10 within 300 seconds +old: parachain 2000 block height is at least 10 within 300 seconds +new: parachain 2001 block height is at least 10 within 300 seconds +old: parachain 2001 block height is at least 10 within 300 seconds + +new: reports substrate_block_height{status="finalized"} is at least 30 within 400 seconds +old: reports substrate_block_height{status="finalized"} is at least 30 within 400 seconds + +new: reports polkadot_parachain_approval_checking_finality_lag < 3 +old: reports polkadot_parachain_approval_checking_finality_lag < 3 + +new: reports polkadot_parachain_approvals_no_shows_total < 3 within 10 seconds +old: reports polkadot_parachain_approvals_no_shows_total < 3 within 10 seconds + + +# Ensure we used the fallback network request. +new: log line contains "Trying the fallback protocol" within 10 seconds + +# Ensure that there are no failed recoveries and that we've only made regular chunk recoveries. +new: reports polkadot_parachain_availability_recovery_recoveries_finished{result="failure"} is 0 within 10 seconds +old: reports polkadot_parachain_availability_recovery_recoveries_finished{result="failure"} is 0 within 10 seconds +new: reports polkadot_parachain_availability_recovery_recoveries_finished{result="success"} is at least 5 within 10 seconds +old: reports polkadot_parachain_availability_recovery_recoveries_finished{result="success"} is at least 5 within 10 seconds + +# Ensure availability-distribution worked fine +new: reports polkadot_parachain_fetched_chunks_total{success="succeeded"} is at least 5 within 10 seconds +old: reports polkadot_parachain_fetched_chunks_total{success="succeeded"} is at least 5 within 10 seconds + +new: reports polkadot_parachain_fetched_chunks_total{success="failed"} is 0 within 10 seconds +old: reports polkadot_parachain_fetched_chunks_total{success="failed"} is 0 within 10 seconds + +new: reports polkadot_parachain_fetched_chunks_total{success="not-found"} is 0 within 10 seconds +old: reports polkadot_parachain_fetched_chunks_total{success="not-found"} is 0 within 10 seconds From d923269b122ca35644ff18f5e2c00ec4d61a87bf Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 19 Jan 2024 13:22:10 +0200 Subject: [PATCH 085/126] add systematic recovery to subsystem-bench --- Cargo.lock | 45 +++++++++--- .../network/availability-recovery/src/lib.rs | 10 +-- polkadot/node/subsystem-bench/Cargo.toml | 1 + .../subsystem-bench/src/availability/cli.rs | 22 ++++-- .../subsystem-bench/src/availability/mod.rs | 71 +++++++++++++------ .../subsystem-bench/src/core/mock/av_store.rs | 67 ++++++++++++----- .../src/core/mock/network_bridge.rs | 52 +++++++++----- .../src/core/mock/runtime_api.rs | 32 ++++++++- 8 files changed, 219 insertions(+), 81 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index adc04e39c46e..ae27cf76f540 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2887,8 +2887,8 @@ version = "7.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9ab77dbd8adecaf3f0db40581631b995f312a8a5ae3aa9993188bb8f23d83a5b" dependencies = [ - "strum", - "strum_macros", + "strum 0.24.1", + "strum_macros 0.24.3", "unicode-width", ] @@ -9826,7 +9826,7 @@ dependencies = [ "sp-runtime", "sp-std 8.0.0", "sp-tracing 10.0.0", - "strum", + "strum 0.24.1", ] [[package]] @@ -12885,7 +12885,7 @@ dependencies = [ "rand_chacha 0.3.1", "sc-authority-discovery", "sc-network", - "strum", + "strum 0.24.1", "thiserror", "tracing-gum", ] @@ -13600,6 +13600,7 @@ dependencies = [ "sp-core", "sp-keyring", "sp-keystore", + "strum 0.25.0", "substrate-prometheus-endpoint", "tokio", "tracing-gum", @@ -18231,7 +18232,7 @@ dependencies = [ "sp-mmr-primitives", "sp-runtime", "sp-std 8.0.0", - "strum", + "strum 0.24.1", "w3f-bls", ] @@ -18512,7 +18513,7 @@ version = "24.0.0" dependencies = [ "sp-core", "sp-runtime", - "strum", + "strum 0.24.1", ] [[package]] @@ -19054,7 +19055,7 @@ checksum = "08615eea740067d9899969bc2891c68a19c315cb1f66640af9a9ecb91b13bcab" dependencies = [ "lazy_static", "maplit", - "strum", + "strum 0.24.1", ] [[package]] @@ -19397,7 +19398,16 @@ version = "0.24.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "063e6045c0e62079840579a7e47a355ae92f60eb74daaf156fb1e84ba164e63f" dependencies = [ - "strum_macros", + "strum_macros 0.24.3", +] + +[[package]] +name = "strum" +version = "0.25.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "290d54ea6f91c969195bdbcd7442c8c2a2ba87da8bf60a7ee86a235d4bc1e125" +dependencies = [ + "strum_macros 0.25.3", ] [[package]] @@ -19413,6 +19423,19 @@ dependencies = [ "syn 1.0.109", ] +[[package]] +name = "strum_macros" +version = "0.25.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "23dc1fa9ac9c169a78ba62f0b841814b7abae11bdd047b9c58f893439e309ea0" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "rustversion", + "syn 2.0.48", +] + [[package]] name = "subkey" version = "3.0.0" @@ -19676,7 +19699,7 @@ dependencies = [ "filetime", "parity-wasm", "sp-maybe-compressed-blob", - "strum", + "strum 0.24.1", "tempfile", "toml 0.8.8", "walkdir", @@ -21123,8 +21146,8 @@ checksum = "fc942673e7684671f0c5708fc18993569d184265fd5223bb51fc8e5b9b6cfd52" dependencies = [ "anyhow", "libc", - "strum", - "strum_macros", + "strum 0.24.1", + "strum_macros 0.24.3", "tempfile", "thiserror", "wasm-opt-cxx-sys", diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index 122147f681c4..e937ac200d0e 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -94,7 +94,7 @@ const SMALL_POV_LIMIT: usize = 128 * 1024; #[derive(Clone, PartialEq)] /// The strategy we use to recover the PoV. -enum RecoveryStrategyKind { +pub enum RecoveryStrategyKind { /// We try the backing group first if PoV size is lower than specified, then fallback to /// validator chunks. BackersFirstIfSizeLower(usize), @@ -141,7 +141,7 @@ pub struct AvailabilityRecoverySubsystem { #[derive(Clone, PartialEq, Debug)] /// The type of check to perform after available data was recovered. -pub enum PostRecoveryCheck { +enum PostRecoveryCheck { /// Reencode the data and check erasure root. For validators. Reencode, /// Only check the pov hash. For collators only. @@ -149,7 +149,7 @@ pub enum PostRecoveryCheck { } /// Expensive erasure coding computations that we want to run on a blocking thread. -pub enum ErasureTask { +enum ErasureTask { /// Reconstructs `AvailableData` from chunks given `n_validators`. Reconstruct( usize, @@ -647,8 +647,8 @@ impl AvailabilityRecoverySubsystem { /// Customise the recovery strategy kind /// Currently only useful for tests. - #[cfg(test)] - fn with_recovery_strategy_kind( + #[cfg(any(test, feature = "subsystem-benchmarks"))] + pub fn with_recovery_strategy_kind( req_receiver: IncomingRequestReceiver, req_protocol_names: &ReqProtocolNames, metrics: Metrics, diff --git a/polkadot/node/subsystem-bench/Cargo.toml b/polkadot/node/subsystem-bench/Cargo.toml index ab3de33fe2c2..e4c6e0464cb5 100644 --- a/polkadot/node/subsystem-bench/Cargo.toml +++ b/polkadot/node/subsystem-bench/Cargo.toml @@ -58,6 +58,7 @@ paste = "1.0.14" orchestra = { version = "0.3.3", default-features = false, features = ["futures_channel"] } pyroscope = "0.5.7" pyroscope_pprofrs = "0.2.7" +strum = { version = "0.25", features = ["derive"] } [features] default = [] diff --git a/polkadot/node/subsystem-bench/src/availability/cli.rs b/polkadot/node/subsystem-bench/src/availability/cli.rs index 65df8c1552aa..8dbb23463a9f 100644 --- a/polkadot/node/subsystem-bench/src/availability/cli.rs +++ b/polkadot/node/subsystem-bench/src/availability/cli.rs @@ -15,6 +15,7 @@ // along with Polkadot. If not, see . use serde::{Deserialize, Serialize}; +use strum::Display; #[derive(clap::ValueEnum, Clone, Copy, Debug, PartialEq)] #[value(rename_all = "kebab-case")] @@ -25,13 +26,24 @@ pub enum NetworkEmulation { Degraded, } +#[derive(clap::ValueEnum, Clone, Copy, Debug, PartialEq, Serialize, Deserialize, Display)] +#[value(rename_all = "kebab-case")] +#[strum(serialize_all = "kebab-case")] +pub enum Strategy { + /// Regular random chunk recovery. This is also the fallback for the next strategies. + Chunks, + /// Recovery from systematic chunks. Much faster than regular chunk recovery becasue it avoid + /// doing the reed-solomon reconstruction. + Systematic, + /// Fetch the full availability datafrom backers first. Saves CPU as we don't need to + /// re-construct from chunks. Typically this is only faster if nodes have enough bandwidth. + FullFromBackers, +} + #[derive(Debug, Clone, Serialize, Deserialize, clap::Parser)] #[clap(rename_all = "kebab-case")] #[allow(missing_docs)] pub struct DataAvailabilityReadOptions { - #[clap(short, long, default_value_t = false)] - /// Turbo boost AD Read by fetching the full availability datafrom backers first. Saves CPU as - /// we don't need to re-construct from chunks. Tipically this is only faster if nodes have - /// enough bandwidth. - pub fetch_from_backers: bool, + #[clap(short, long, default_value_t = Strategy::Systematic)] + pub strategy: Strategy, } diff --git a/polkadot/node/subsystem-bench/src/availability/mod.rs b/polkadot/node/subsystem-bench/src/availability/mod.rs index 61a71771aad6..c83f841e492f 100644 --- a/polkadot/node/subsystem-bench/src/availability/mod.rs +++ b/polkadot/node/subsystem-bench/src/availability/mod.rs @@ -14,9 +14,10 @@ // You should have received a copy of the GNU General Public License // along with Polkadot. If not, see . use itertools::Itertools; +use polkadot_node_subsystem_util::availability_chunks::availability_chunk_indices; use std::{collections::HashMap, iter::Cycle, ops::Sub, sync::Arc, time::Instant}; -use crate::TestEnvironment; +use crate::{core::mock::node_features_with_chunk_mapping_enabled, TestEnvironment}; use polkadot_node_subsystem::{Overseer, OverseerConnector, SpawnGlue}; use polkadot_node_subsystem_test_helpers::derive_erasure_chunks_with_proofs_and_root; use polkadot_overseer::Handle as OverseerHandle; @@ -27,7 +28,7 @@ use colored::Colorize; use futures::{channel::oneshot, stream::FuturesUnordered, StreamExt}; use polkadot_node_metrics::metrics::Metrics; -use polkadot_availability_recovery::AvailabilityRecoverySubsystem; +use polkadot_availability_recovery::{AvailabilityRecoverySubsystem, RecoveryStrategyKind}; use crate::GENESIS_HASH; use parity_scale_codec::Encode; @@ -44,6 +45,8 @@ use crate::core::{ }, }; +use self::cli::Strategy; + use super::core::{configuration::TestConfiguration, mock::dummy_builder, network::*}; const LOG_TARGET: &str = "subsystem-bench::availability"; @@ -53,7 +56,8 @@ use polkadot_node_primitives::{AvailableData, ErasureChunk}; use super::{cli::TestObjective, core::mock::AlwaysSupportsParachains}; use polkadot_node_subsystem_test_helpers::mock::new_block_import_info; use polkadot_primitives::{ - CandidateHash, CandidateReceipt, GroupIndex, Hash, HeadData, PersistedValidationData, + CandidateHash, CandidateReceipt, ChunkIndex, CoreIndex, GroupIndex, Hash, HeadData, + PersistedValidationData, }; use polkadot_primitives_test_helpers::{dummy_candidate_receipt, dummy_hash}; use sc_service::SpawnTaskHandle; @@ -100,13 +104,19 @@ fn prepare_test_inner( let runtime_api = runtime_api::MockRuntimeApi::new(config.clone(), test_authorities.clone()); - let av_store = - av_store::MockAvailabilityStore::new(state.chunks.clone(), state.candidate_hashes.clone()); + let av_store = av_store::MockAvailabilityStore::new( + state.chunks.clone(), + state.chunk_indices.clone(), + state.candidate_hashes.clone(), + state.candidate_hash_to_core_index.clone(), + ); let availability_state = NetworkAvailabilityState { candidate_hashes: state.candidate_hashes.clone(), + candidate_hash_to_core_index: state.candidate_hash_to_core_index.clone(), available_data: state.available_data.clone(), chunks: state.chunks.clone(), + chunk_indices: state.chunk_indices.clone(), }; let req_protocol_names = ReqProtocolNames::new(GENESIS_HASH, None); @@ -114,7 +124,7 @@ fn prepare_test_inner( IncomingRequest::get_config_receiver(&req_protocol_names); let network = - NetworkEmulator::new(&config, &dependencies, &test_authorities, req_protocol_names); + NetworkEmulator::new(&config, &dependencies, &test_authorities, req_protocol_names.clone()); let network_bridge_tx = network_bridge::MockNetworkBridgeTx::new( config.clone(), @@ -122,22 +132,21 @@ fn prepare_test_inner( network.clone(), ); - let use_fast_path = match &state.config().objective { - TestObjective::DataAvailabilityRead(options) => options.fetch_from_backers, + let strategy = match &state.config().objective { + TestObjective::DataAvailabilityRead(options) => options.strategy, _ => panic!("Unexpected objective"), }; - let subsystem = if use_fast_path { - AvailabilityRecoverySubsystem::with_fast_path( - collation_req_receiver, - Metrics::try_register(&dependencies.registry).unwrap(), - ) - } else { - AvailabilityRecoverySubsystem::with_chunks_only( - collation_req_receiver, - Metrics::try_register(&dependencies.registry).unwrap(), - ) - }; + let subsystem = AvailabilityRecoverySubsystem::with_recovery_strategy_kind( + collation_req_receiver, + &req_protocol_names, + Metrics::try_register(&dependencies.registry).unwrap(), + match strategy { + Strategy::Chunks => RecoveryStrategyKind::ChunksAlways, + Strategy::Systematic => RecoveryStrategyKind::SystematicChunks, + Strategy::FullFromBackers => RecoveryStrategyKind::BackersFirstAlways, + }, + ); let (overseer, overseer_handle) = build_overseer( dependencies.task_manager.spawn_handle(), @@ -163,12 +172,16 @@ pub struct TestState { // Map from generated candidate hashes to candidate index in `available_data` // and `chunks`. candidate_hashes: HashMap, + // Map from candidate hash to occupied core index. + candidate_hash_to_core_index: HashMap, // Per candidate index receipts. candidate_receipt_templates: Vec, // Per candidate index `AvailableData` available_data: Vec, - // Per candiadte index chunks + // Per candidate index chunks chunks: Vec>, + // Per-core ValidatorIndex -> ChunkIndex mapping + chunk_indices: Vec>, } impl TestState { @@ -204,6 +217,10 @@ impl TestState { // Store the new candidate in the state self.candidate_hashes.insert(candidate_receipt.hash(), candidate_index); + let core_index = (index % self.config.n_cores) as u32; + self.candidate_hash_to_core_index + .insert(candidate_receipt.hash(), core_index.into()); + gum::debug!(target: LOG_TARGET, candidate_hash = ?candidate_receipt.hash(), "new candidate"); candidate_receipt @@ -217,6 +234,7 @@ impl TestState { let config = config.clone(); let mut chunks = Vec::new(); + let mut chunk_indices = vec![vec![]; config.n_cores]; let mut available_data = Vec::new(); let mut candidate_receipt_templates = Vec::new(); let mut pov_size_to_candidate = HashMap::new(); @@ -229,6 +247,15 @@ impl TestState { relay_parent_storage_root: Default::default(), }; + for core_index in 0..config.n_cores { + chunk_indices[core_index] = availability_chunk_indices( + Some(&node_features_with_chunk_mapping_enabled()), + config.n_validators, + CoreIndex(core_index as u32), + ) + .unwrap(); + } + // For each unique pov we create a candidate receipt. for (index, pov_size) in config.pov_sizes().iter().cloned().unique().enumerate() { gum::info!(target: LOG_TARGET, index, pov_size, "{}", "Generating template candidate".bright_blue()); @@ -264,9 +291,11 @@ impl TestState { available_data, candidate_receipt_templates, chunks, + chunk_indices, pov_size_to_candidate, pov_sizes, candidate_hashes: HashMap::new(), + candidate_hash_to_core_index: HashMap::new(), candidates: Vec::new().into_iter().cycle(), }; @@ -305,7 +334,7 @@ pub async fn benchmark_availability_read(env: &mut TestEnvironment, mut state: T Some(GroupIndex( candidate_num as u32 % (std::cmp::max(5, config.n_cores) / 5) as u32, )), - None, + Some(CoreIndex(candidate_num as u32)), tx, ), ); diff --git a/polkadot/node/subsystem-bench/src/core/mock/av_store.rs b/polkadot/node/subsystem-bench/src/core/mock/av_store.rs index a471230f1b3f..7302f9d06929 100644 --- a/polkadot/node/subsystem-bench/src/core/mock/av_store.rs +++ b/polkadot/node/subsystem-bench/src/core/mock/av_store.rs @@ -17,7 +17,7 @@ //! A generic av store subsystem mockup suitable to be used in benchmarks. use parity_scale_codec::Encode; -use polkadot_primitives::CandidateHash; +use polkadot_primitives::{CandidateHash, ChunkIndex, CoreIndex, ValidatorIndex}; use std::collections::HashMap; @@ -34,6 +34,8 @@ use polkadot_node_subsystem_types::OverseerSignal; pub struct AvailabilityStoreState { candidate_hashes: HashMap, chunks: Vec>, + chunk_indices: Vec>, + candidate_hash_to_core_index: HashMap, } const LOG_TARGET: &str = "subsystem-bench::av-store-mock"; @@ -47,16 +49,25 @@ pub struct MockAvailabilityStore { impl MockAvailabilityStore { pub fn new( chunks: Vec>, + chunk_indices: Vec>, candidate_hashes: HashMap, + candidate_hash_to_core_index: HashMap, ) -> MockAvailabilityStore { - Self { state: AvailabilityStoreState { chunks, candidate_hashes } } + Self { + state: AvailabilityStoreState { + chunks, + candidate_hashes, + chunk_indices, + candidate_hash_to_core_index, + }, + } } async fn respond_to_query_all_request( &self, candidate_hash: CandidateHash, - send_chunk: impl Fn(usize) -> bool, - tx: oneshot::Sender>, + send_chunk: impl Fn(ValidatorIndex) -> bool, + tx: oneshot::Sender>, ) { let candidate_index = self .state @@ -65,15 +76,27 @@ impl MockAvailabilityStore { .expect("candidate was generated previously; qed"); gum::debug!(target: LOG_TARGET, ?candidate_hash, candidate_index, "Candidate mapped to index"); - let v = self - .state - .chunks - .get(*candidate_index) - .unwrap() - .iter() - .filter(|c| send_chunk(c.index.0 as usize)) - .cloned() - .collect(); + let n_validators = self.state.chunks[0].len(); + let candidate_chunks = self.state.chunks.get(*candidate_index).unwrap(); + let core_index = self.state.candidate_hash_to_core_index.get(&candidate_hash).unwrap(); + // We'll likely only send our chunk, so use capacity 1. + let mut v = Vec::with_capacity(1); + + for validator_index in 0..n_validators { + if !send_chunk(ValidatorIndex(validator_index as u32)) { + continue; + } + let chunk_index = self + .state + .chunk_indices + .get(core_index.0 as usize) + .unwrap() + .get(validator_index as usize) + .unwrap(); + + let chunk = candidate_chunks.get(chunk_index.0 as usize).unwrap().clone(); + v.push((ValidatorIndex(validator_index as u32), chunk.clone())); + } let _ = tx.send(v); } @@ -110,8 +133,12 @@ impl MockAvailabilityStore { AvailabilityStoreMessage::QueryAllChunks(candidate_hash, tx) => { // We always have our own chunk. gum::debug!(target: LOG_TARGET, candidate_hash = ?candidate_hash, "Responding to QueryAllChunks"); - self.respond_to_query_all_request(candidate_hash, |index| index == 0, tx) - .await; + self.respond_to_query_all_request( + candidate_hash, + |index| index == 0.into(), + tx, + ) + .await; }, AvailabilityStoreMessage::QueryChunkSize(candidate_hash, tx) => { gum::debug!(target: LOG_TARGET, candidate_hash = ?candidate_hash, "Responding to QueryChunkSize"); @@ -123,8 +150,14 @@ impl MockAvailabilityStore { .expect("candidate was generated previously; qed"); gum::debug!(target: LOG_TARGET, ?candidate_hash, candidate_index, "Candidate mapped to index"); - let chunk_size = - self.state.chunks.get(*candidate_index).unwrap()[0].encoded_size(); + let chunk_size = self + .state + .chunks + .get(*candidate_index) + .unwrap() + .get(0) + .unwrap() + .encoded_size(); let _ = tx.send(Some(chunk_size)); }, _ => { diff --git a/polkadot/node/subsystem-bench/src/core/mock/network_bridge.rs b/polkadot/node/subsystem-bench/src/core/mock/network_bridge.rs index 5d534e37c991..a5cab3daebea 100644 --- a/polkadot/node/subsystem-bench/src/core/mock/network_bridge.rs +++ b/polkadot/node/subsystem-bench/src/core/mock/network_bridge.rs @@ -25,7 +25,7 @@ use futures::FutureExt; use polkadot_node_primitives::{AvailableData, ErasureChunk}; -use polkadot_primitives::CandidateHash; +use polkadot_primitives::{CandidateHash, ChunkIndex, CoreIndex}; use sc_network::{OutboundFailure, RequestFailure}; use polkadot_node_subsystem::{ @@ -33,9 +33,7 @@ use polkadot_node_subsystem::{ }; use polkadot_node_network_protocol::request_response::{ - self as req_res, - v1::{AvailableDataFetchingRequest, ChunkFetchingRequest, ChunkResponse}, - IsRequest, Requests, + self as req_res, v1::AvailableDataFetchingRequest, IsRequest, Requests, }; use polkadot_primitives::AuthorityDiscoveryId; @@ -49,8 +47,10 @@ use crate::core::{ /// by a remote peer on the network pub struct NetworkAvailabilityState { pub candidate_hashes: HashMap, + pub candidate_hash_to_core_index: HashMap, pub available_data: Vec, pub chunks: Vec>, + pub chunk_indices: Vec>, } const LOG_TARGET: &str = "subsystem-bench::network-bridge-tx-mock"; @@ -60,7 +60,7 @@ pub struct MockNetworkBridgeTx { /// The test configurationg config: TestConfiguration, /// The network availability state - availabilty: NetworkAvailabilityState, + availability: NetworkAvailabilityState, /// A network emulator instance network: NetworkEmulator, } @@ -68,10 +68,10 @@ pub struct MockNetworkBridgeTx { impl MockNetworkBridgeTx { pub fn new( config: TestConfiguration, - availabilty: NetworkAvailabilityState, + availability: NetworkAvailabilityState, network: NetworkEmulator, ) -> MockNetworkBridgeTx { - Self { config, availabilty, network } + Self { config, availability, network } } fn not_connected_response( @@ -99,7 +99,7 @@ impl MockNetworkBridgeTx { let ingress_tx = ingress_tx.clone(); match request { - Requests::ChunkFetchingV1(outgoing_request) => { + Requests::ChunkFetching(outgoing_request) => { let authority_discovery_id = match outgoing_request.peer { req_res::Recipient::Authority(authority_discovery_id) => authority_discovery_id, _ => unimplemented!("Peer recipient not supported yet"), @@ -125,20 +125,32 @@ impl MockNetworkBridgeTx { .peer_stats_by_id(&authority_discovery_id) .inc_received(outgoing_request.payload.encoded_size()); - let validator_index: usize = outgoing_request.payload.index.0 as usize; let candidate_hash = outgoing_request.payload.candidate_hash; let candidate_index = self - .availabilty + .availability .candidate_hashes .get(&candidate_hash) .expect("candidate was generated previously; qed"); gum::warn!(target: LOG_TARGET, ?candidate_hash, candidate_index, "Candidate mapped to index"); - let chunk: ChunkResponse = self.availabilty.chunks.get(*candidate_index).unwrap() - [validator_index] - .clone() - .into(); + let validator_index = outgoing_request.payload.index; + + let candidate_chunks = self.availability.chunks.get(*candidate_index).unwrap(); + let core_index = + self.availability.candidate_hash_to_core_index.get(&candidate_hash).unwrap(); + let chunk_index = self + .availability + .chunk_indices + .get(core_index.0 as usize) + .unwrap() + .get(validator_index.0 as usize) + .unwrap(); + + let chunk = candidate_chunks.get(chunk_index.0 as usize).unwrap().clone(); + + assert_eq!(chunk.index, *chunk_index); + let mut size = chunk.encoded_size(); let response = if random_error(self.config.error) { @@ -147,8 +159,10 @@ impl MockNetworkBridgeTx { Err(RequestFailure::Network(OutboundFailure::ConnectionClosed)) } else { Ok(( - req_res::v1::ChunkFetchingResponse::from(Some(chunk)).encode(), - self.network.req_protocol_names().get_name(ChunkFetchingRequest::PROTOCOL), + req_res::v2::ChunkFetchingResponse::from(Some(chunk)).encode(), + self.network + .req_protocol_names() + .get_name(req_res::v2::ChunkFetchingRequest::PROTOCOL), )) }; @@ -179,7 +193,7 @@ impl MockNetworkBridgeTx { Requests::AvailableDataFetchingV1(outgoing_request) => { let candidate_hash = outgoing_request.payload.candidate_hash; let candidate_index = self - .availabilty + .availability .candidate_hashes .get(&candidate_hash) .expect("candidate was generated previously; qed"); @@ -210,7 +224,7 @@ impl MockNetworkBridgeTx { .inc_received(outgoing_request.payload.encoded_size()); let available_data = - self.availabilty.available_data.get(*candidate_index).unwrap().clone(); + self.availability.available_data.get(*candidate_index).unwrap().clone(); let size = available_data.encoded_size(); @@ -325,7 +339,7 @@ impl MockNetworkBridgeTx { // A helper to determine the request payload size. fn request_size(request: &Requests) -> usize { match request { - Requests::ChunkFetchingV1(outgoing_request) => outgoing_request.payload.encoded_size(), + Requests::ChunkFetching(outgoing_request) => outgoing_request.payload.encoded_size(), Requests::AvailableDataFetchingV1(outgoing_request) => outgoing_request.payload.encoded_size(), _ => unimplemented!("received an unexpected request"), diff --git a/polkadot/node/subsystem-bench/src/core/mock/runtime_api.rs b/polkadot/node/subsystem-bench/src/core/mock/runtime_api.rs index d664ebead3cc..90347e269ac1 100644 --- a/polkadot/node/subsystem-bench/src/core/mock/runtime_api.rs +++ b/polkadot/node/subsystem-bench/src/core/mock/runtime_api.rs @@ -16,7 +16,10 @@ //! //! A generic runtime api subsystem mockup suitable to be used in benchmarks. -use polkadot_primitives::{GroupIndex, IndexedVec, SessionInfo, ValidatorIndex}; +use polkadot_primitives::{ + vstaging::{node_features, NodeFeatures}, + ExecutorParams, GroupIndex, IndexedVec, SessionInfo, ValidatorIndex, +}; use polkadot_node_subsystem::{ messages::{RuntimeApiMessage, RuntimeApiRequest}, @@ -31,6 +34,7 @@ const LOG_TARGET: &str = "subsystem-bench::runtime-api-mock"; pub struct RuntimeApiState { authorities: TestAuthorities, + node_features: NodeFeatures, } pub struct MockRuntimeApi { @@ -40,7 +44,10 @@ pub struct MockRuntimeApi { impl MockRuntimeApi { pub fn new(config: TestConfiguration, authorities: TestAuthorities) -> MockRuntimeApi { - Self { state: RuntimeApiState { authorities }, config } + // Enable chunk mapping feature to make systematic av-recovery possible. + let node_features = node_features_with_chunk_mapping_enabled(); + + Self { state: RuntimeApiState { authorities, node_features }, config } } fn session_info(&self) -> SessionInfo { @@ -93,11 +100,23 @@ impl MockRuntimeApi { match msg { RuntimeApiMessage::Request( - _request, + _hash, RuntimeApiRequest::SessionInfo(_session_index, sender), ) => { let _ = sender.send(Ok(Some(self.session_info()))); }, + RuntimeApiMessage::Request( + _hash, + RuntimeApiRequest::SessionExecutorParams(_session_index, sender), + ) => { + let _ = sender.send(Ok(Some(ExecutorParams::new()))); + }, + RuntimeApiMessage::Request( + _hash, + RuntimeApiRequest::NodeFeatures(_session_index, sender), + ) => { + let _ = sender.send(Ok(self.state.node_features.clone())); + }, // Long term TODO: implement more as needed. _ => { unimplemented!("Unexpected runtime-api message") @@ -108,3 +127,10 @@ impl MockRuntimeApi { } } } + +pub fn node_features_with_chunk_mapping_enabled() -> NodeFeatures { + let mut node_features = NodeFeatures::new(); + node_features.resize(node_features::FeatureIndex::AvailabilityChunkMapping as usize + 1, false); + node_features.set(node_features::FeatureIndex::AvailabilityChunkMapping as u8 as usize, true); + node_features +} From bc22b81944695f9f4169434873b7671ff3aecd17 Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 19 Jan 2024 13:27:50 +0200 Subject: [PATCH 086/126] fix clippy --- .../subsystem-bench/src/availability/mod.rs | 19 ++++++++++--------- .../subsystem-bench/src/core/mock/av_store.rs | 2 +- 2 files changed, 11 insertions(+), 10 deletions(-) diff --git a/polkadot/node/subsystem-bench/src/availability/mod.rs b/polkadot/node/subsystem-bench/src/availability/mod.rs index c83f841e492f..378b7a6a85f1 100644 --- a/polkadot/node/subsystem-bench/src/availability/mod.rs +++ b/polkadot/node/subsystem-bench/src/availability/mod.rs @@ -234,7 +234,6 @@ impl TestState { let config = config.clone(); let mut chunks = Vec::new(); - let mut chunk_indices = vec![vec![]; config.n_cores]; let mut available_data = Vec::new(); let mut candidate_receipt_templates = Vec::new(); let mut pov_size_to_candidate = HashMap::new(); @@ -247,14 +246,16 @@ impl TestState { relay_parent_storage_root: Default::default(), }; - for core_index in 0..config.n_cores { - chunk_indices[core_index] = availability_chunk_indices( - Some(&node_features_with_chunk_mapping_enabled()), - config.n_validators, - CoreIndex(core_index as u32), - ) - .unwrap(); - } + let chunk_indices = (0..config.n_cores) + .map(|core_index| { + availability_chunk_indices( + Some(&node_features_with_chunk_mapping_enabled()), + config.n_validators, + CoreIndex(core_index as u32), + ) + .unwrap() + }) + .collect(); // For each unique pov we create a candidate receipt. for (index, pov_size) in config.pov_sizes().iter().cloned().unique().enumerate() { diff --git a/polkadot/node/subsystem-bench/src/core/mock/av_store.rs b/polkadot/node/subsystem-bench/src/core/mock/av_store.rs index 7302f9d06929..d81279ff6d77 100644 --- a/polkadot/node/subsystem-bench/src/core/mock/av_store.rs +++ b/polkadot/node/subsystem-bench/src/core/mock/av_store.rs @@ -91,7 +91,7 @@ impl MockAvailabilityStore { .chunk_indices .get(core_index.0 as usize) .unwrap() - .get(validator_index as usize) + .get(validator_index) .unwrap(); let chunk = candidate_chunks.get(chunk_index.0 as usize).unwrap().clone(); From 3edd33b8ea393d3345d7549b3c1becd69f0c9822 Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 19 Jan 2024 14:56:40 +0200 Subject: [PATCH 087/126] fix some backing tests --- polkadot/node/core/backing/src/tests/mod.rs | 44 +++++++------------ .../src/tests/prospective_parachains.rs | 7 +++ 2 files changed, 24 insertions(+), 27 deletions(-) diff --git a/polkadot/node/core/backing/src/tests/mod.rs b/polkadot/node/core/backing/src/tests/mod.rs index 1957f4e19c54..7512acd52ff2 100644 --- a/polkadot/node/core/backing/src/tests/mod.rs +++ b/polkadot/node/core/backing/src/tests/mod.rs @@ -347,6 +347,15 @@ async fn assert_validation_requests( tx.send(Ok(Some(ExecutorParams::default()))).unwrap(); } ); + + assert_matches!( + virtual_overseer.recv().await, + AllMessages::RuntimeApi( + RuntimeApiMessage::Request(_, RuntimeApiRequest::NodeFeatures(sess_idx, tx)) + ) if sess_idx == 1 => { + tx.send(Ok(NodeFeatures::EMPTY)).unwrap(); + } + ); } async fn assert_validate_from_exhaustive( @@ -1591,7 +1600,7 @@ fn retry_works() { virtual_overseer.send(FromOrchestra::Communication { msg: statement }).await; // Not deterministic which message comes first: - for _ in 0u32..5 { + for _ in 0u32..6 { match virtual_overseer.recv().await { AllMessages::Provisioner(ProvisionerMessage::ProvisionableData( _, @@ -1622,6 +1631,12 @@ fn retry_works() { )) => { tx.send(Ok(Some(ExecutorParams::default()))).unwrap(); }, + AllMessages::RuntimeApi(RuntimeApiMessage::Request( + _, + RuntimeApiRequest::NodeFeatures(1, tx), + )) => { + tx.send(Ok(NodeFeatures::EMPTY)).unwrap(); + }, msg => { assert!(false, "Unexpected message: {:?}", msg); }, @@ -2169,32 +2184,7 @@ fn validator_ignores_statements_from_disabled_validators() { virtual_overseer.send(FromOrchestra::Communication { msg: statement_3 }).await; - assert_matches!( - virtual_overseer.recv().await, - AllMessages::RuntimeApi( - RuntimeApiMessage::Request(_, RuntimeApiRequest::ValidationCodeByHash(hash, tx)) - ) if hash == validation_code.hash() => { - tx.send(Ok(Some(validation_code.clone()))).unwrap(); - } - ); - - assert_matches!( - virtual_overseer.recv().await, - AllMessages::RuntimeApi( - RuntimeApiMessage::Request(_, RuntimeApiRequest::SessionIndexForChild(tx)) - ) => { - tx.send(Ok(1u32.into())).unwrap(); - } - ); - - assert_matches!( - virtual_overseer.recv().await, - AllMessages::RuntimeApi( - RuntimeApiMessage::Request(_, RuntimeApiRequest::SessionExecutorParams(sess_idx, tx)) - ) if sess_idx == 1 => { - tx.send(Ok(Some(ExecutorParams::default()))).unwrap(); - } - ); + assert_validation_requests(&mut virtual_overseer, validation_code.clone()).await; // Sending a `Statement::Seconded` for our assignment will start // validation process. The first thing requested is the PoV. diff --git a/polkadot/node/core/backing/src/tests/prospective_parachains.rs b/polkadot/node/core/backing/src/tests/prospective_parachains.rs index 578f21bef665..a0cc51ba029c 100644 --- a/polkadot/node/core/backing/src/tests/prospective_parachains.rs +++ b/polkadot/node/core/backing/src/tests/prospective_parachains.rs @@ -1388,6 +1388,13 @@ fn concurrent_dependent_candidates() { assert_eq!(sess_idx, 1); tx.send(Ok(Some(ExecutorParams::default()))).unwrap(); }, + AllMessages::RuntimeApi(RuntimeApiMessage::Request( + _, + RuntimeApiRequest::NodeFeatures(sess_idx, tx), + )) => { + assert_eq!(sess_idx, 1); + tx.send(Ok(NodeFeatures::EMPTY)).unwrap(); + }, _ => panic!("unexpected message received from overseer: {:?}", msg), } From 2d2d868942f5296fb0ca05ff88a942a931535021 Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 19 Jan 2024 15:45:00 +0200 Subject: [PATCH 088/126] random small fixes --- Cargo.lock | 1 - cumulus/client/pov-recovery/src/lib.rs | 4 ++-- polkadot/erasure-coding/Cargo.toml | 9 +++++++-- polkadot/node/overseer/src/lib.rs | 2 -- polkadot/node/subsystem-util/Cargo.toml | 9 ++++++--- 5 files changed, 15 insertions(+), 10 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index af866efce464..a84a6235f826 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -13005,7 +13005,6 @@ dependencies = [ "polkadot-primitives-test-helpers", "prioritized-metered-channel", "rand 0.8.5", - "rand_chacha 0.3.1", "sc-client-api", "schnellru", "sp-application-crypto", diff --git a/cumulus/client/pov-recovery/src/lib.rs b/cumulus/client/pov-recovery/src/lib.rs index 69544c516fb1..32aba6c8993a 100644 --- a/cumulus/client/pov-recovery/src/lib.rs +++ b/cumulus/client/pov-recovery/src/lib.rs @@ -264,7 +264,7 @@ where } /// Handle a new pending candidate. - async fn handle_pending_candidate( + fn handle_pending_candidate( &mut self, receipt: CommittedCandidateReceipt, session_index: SessionIndex, @@ -556,7 +556,7 @@ where select! { pending_candidate = pending_candidates.next() => { if let Some((receipt, session_index)) = pending_candidate { - self.handle_pending_candidate(receipt, session_index).await; + self.handle_pending_candidate(receipt, session_index); } else { tracing::debug!(target: LOG_TARGET, "Pending candidates stream ended"); return; diff --git a/polkadot/erasure-coding/Cargo.toml b/polkadot/erasure-coding/Cargo.toml index 7c51e806f779..c2fd907f074e 100644 --- a/polkadot/erasure-coding/Cargo.toml +++ b/polkadot/erasure-coding/Cargo.toml @@ -13,13 +13,18 @@ workspace = true polkadot-primitives = { path = "../primitives" } polkadot-node-primitives = { package = "polkadot-node-primitives", path = "../node/primitives" } novelpoly = { package = "reed-solomon-novelpoly", git = "https://github.com/paritytech/reed-solomon-novelpoly.git" } -parity-scale-codec = { version = "3.6.1", default-features = false, features = ["std", "derive"] } +parity-scale-codec = { version = "3.6.1", default-features = false, features = [ + "derive", + "std", +] } sp-core = { path = "../../substrate/primitives/core" } sp-trie = { path = "../../substrate/primitives/trie" } thiserror = "1.0.48" [dev-dependencies] -criterion = { version = "0.4.0", default-features = false, features = ["cargo_bench_support"] } +criterion = { version = "0.4.0", default-features = false, features = [ + "cargo_bench_support", +] } quickcheck = { version = "1.0.3", default-features = false } [[bench]] diff --git a/polkadot/node/overseer/src/lib.rs b/polkadot/node/overseer/src/lib.rs index 5be4562457bb..f4eddf1f41ce 100644 --- a/polkadot/node/overseer/src/lib.rs +++ b/polkadot/node/overseer/src/lib.rs @@ -510,7 +510,6 @@ pub struct Overseer { NetworkBridgeTxMessage, RuntimeApiMessage, AvailabilityStoreMessage, - ChainApiMessage, ])] availability_recovery: AvailabilityRecovery, @@ -518,7 +517,6 @@ pub struct Overseer { AvailabilityStoreMessage, RuntimeApiMessage, BitfieldDistributionMessage, - ChainApiMessage ])] bitfield_signing: BitfieldSigning, diff --git a/polkadot/node/subsystem-util/Cargo.toml b/polkadot/node/subsystem-util/Cargo.toml index 10494861c9d3..57c13ba63f17 100644 --- a/polkadot/node/subsystem-util/Cargo.toml +++ b/polkadot/node/subsystem-util/Cargo.toml @@ -14,11 +14,12 @@ async-trait = "0.1.74" futures = "0.3.21" futures-channel = "0.3.23" itertools = "0.10" -parity-scale-codec = { version = "3.6.1", default-features = false, features = ["derive"] } +parity-scale-codec = { version = "3.6.1", default-features = false, features = [ + "derive", +] } parking_lot = "0.12.1" pin-project = "1.0.9" rand = "0.8.5" -rand_chacha = { version = "0.3.1", default-features = false } thiserror = "1.0.48" fatality = "0.0.6" gum = { package = "tracing-gum", path = "../gum" } @@ -34,7 +35,9 @@ polkadot-node-network-protocol = { path = "../network/protocol" } polkadot-primitives = { path = "../../primitives" } polkadot-node-primitives = { path = "../primitives" } polkadot-overseer = { path = "../overseer" } -metered = { package = "prioritized-metered-channel", version = "0.5.1", default-features = false, features = ["futures_channel"] } +metered = { package = "prioritized-metered-channel", version = "0.5.1", default-features = false, features = [ + "futures_channel", +] } sp-core = { path = "../../../substrate/primitives/core" } sp-application-crypto = { path = "../../../substrate/primitives/application-crypto" } From 557e410b53e4fbaf66ea2573da510ce2319ad493 Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 19 Jan 2024 16:35:17 +0200 Subject: [PATCH 089/126] try fixing zombienet test --- .../0011-chunk-fetching-network-compatibility.toml | 2 +- .../0011-chunk-fetching-network-compatibility.zndsl | 11 ++++++----- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.toml b/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.toml index 3c1f3f06ca14..d0abfedb4ddc 100644 --- a/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.toml +++ b/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.toml @@ -17,7 +17,7 @@ requests = { memory = "2G", cpu = "1" } [[relaychain.node_groups]] # Fix this to an image that doesn't speak /req_chunk/2 protocol. - image = "{{PARACHAINS_IMAGE_NAME}}:v1.0.0" + image = "{{POLKADOT_IMAGE}}:v1.0.0" name = "old" count = 2 args = ["-lparachain=debug,parachain::availability-recovery=trace,parachain::availability-distribution=trace"] diff --git a/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.zndsl b/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.zndsl index 7620eed4b2f1..91874adfa631 100644 --- a/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.zndsl +++ b/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.zndsl @@ -31,15 +31,16 @@ old: reports polkadot_parachain_approvals_no_shows_total < 3 within 10 seconds # Ensure we used the fallback network request. new: log line contains "Trying the fallback protocol" within 10 seconds -# Ensure that there are no failed recoveries and that we've only made regular chunk recoveries. +# Ensure that there are no failed recoveries. +new: reports polkadot_parachain_availability_recovery_recoveries_finished{result="success"} is at least 5 within 100 seconds +old: reports polkadot_parachain_availability_recovery_recoveries_finished{result="success"} is at least 5 within 100 seconds new: reports polkadot_parachain_availability_recovery_recoveries_finished{result="failure"} is 0 within 10 seconds old: reports polkadot_parachain_availability_recovery_recoveries_finished{result="failure"} is 0 within 10 seconds -new: reports polkadot_parachain_availability_recovery_recoveries_finished{result="success"} is at least 5 within 10 seconds -old: reports polkadot_parachain_availability_recovery_recoveries_finished{result="success"} is at least 5 within 10 seconds + # Ensure availability-distribution worked fine -new: reports polkadot_parachain_fetched_chunks_total{success="succeeded"} is at least 5 within 10 seconds -old: reports polkadot_parachain_fetched_chunks_total{success="succeeded"} is at least 5 within 10 seconds +new: reports polkadot_parachain_fetched_chunks_total{success="succeeded"} is at least 5 within 100 seconds +old: reports polkadot_parachain_fetched_chunks_total{success="succeeded"} is at least 5 within 100 seconds new: reports polkadot_parachain_fetched_chunks_total{success="failed"} is 0 within 10 seconds old: reports polkadot_parachain_fetched_chunks_total{success="failed"} is 0 within 10 seconds From 3df708b58cd4fd6eb16df8391fcec030a5ea341f Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 19 Jan 2024 17:18:56 +0200 Subject: [PATCH 090/126] add v2 protocol to cumulus --- cumulus/client/relay-chain-minimal-node/src/lib.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/cumulus/client/relay-chain-minimal-node/src/lib.rs b/cumulus/client/relay-chain-minimal-node/src/lib.rs index d121d2d33567..403c3c5e0af9 100644 --- a/cumulus/client/relay-chain-minimal-node/src/lib.rs +++ b/cumulus/client/relay-chain-minimal-node/src/lib.rs @@ -251,5 +251,7 @@ fn build_request_response_protocol_receivers( config.add_request_response_protocol(cfg); let cfg = Protocol::ChunkFetchingV1.get_outbound_only_config(request_protocol_names); config.add_request_response_protocol(cfg); + let cfg = Protocol::ChunkFetchingV2.get_outbound_only_config(request_protocol_names); + config.add_request_response_protocol(cfg); (collation_req_receiver_v1, collation_req_receiver_v2, available_data_req_receiver) } From 65c8ecf701461bf0985673179946c6b0ba9c0afa Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 26 Jan 2024 14:49:49 +0200 Subject: [PATCH 091/126] fix clippy --- .../src/core/mock/network_bridge.rs | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/polkadot/node/subsystem-bench/src/core/mock/network_bridge.rs b/polkadot/node/subsystem-bench/src/core/mock/network_bridge.rs index 4e0b6695468d..207c803d0eeb 100644 --- a/polkadot/node/subsystem-bench/src/core/mock/network_bridge.rs +++ b/polkadot/node/subsystem-bench/src/core/mock/network_bridge.rs @@ -35,6 +35,8 @@ use crate::core::network::{ }; const LOG_TARGET: &str = "subsystem-bench::network-bridge"; +const CHUNK_REQ_PROTOCOL_NAME_V2: &str = + "/ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff/req_chunk/2"; /// A mock of the network bridge tx subsystem. pub struct MockNetworkBridgeTx { @@ -159,15 +161,12 @@ impl MockNetworkBridgeRx { }, NetworkMessage::RequestFromPeer(request) => { if let Some(protocol) = self.chunk_request_sender.as_mut() { - if protocol.name == protocol.name { - if let Some(inbound_queue) = protocol.inbound_queue.as_ref() { - inbound_queue - .send(request) - .await - .expect("Forwarding requests to subsystem never fails"); - } - } else { - unimplemented!("Unexpected request type"); + assert_eq!(&*protocol.name, CHUNK_REQ_PROTOCOL_NAME_V2); + if let Some(inbound_queue) = protocol.inbound_queue.as_ref() { + inbound_queue + .send(request) + .await + .expect("Forwarding requests to subsystem never fails"); } } }, From 951119e823632aa1bd54abe2517dbf8461202e4f Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 26 Jan 2024 15:10:39 +0200 Subject: [PATCH 092/126] address some review feedback --- .../availability-recovery/src/task/strategy/chunks.rs | 2 +- .../availability-recovery/src/task/strategy/systematic.rs | 4 ++-- polkadot/node/subsystem-types/src/messages.rs | 6 +++--- .../0011-chunk-fetching-network-compatibility.toml | 2 +- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/task/strategy/chunks.rs b/polkadot/node/network/availability-recovery/src/task/strategy/chunks.rs index 67a7d7876157..6ce73dcd36b9 100644 --- a/polkadot/node/network/availability-recovery/src/task/strategy/chunks.rs +++ b/polkadot/node/network/availability-recovery/src/task/strategy/chunks.rs @@ -154,7 +154,7 @@ impl FetchChunks { }), Err(err) => { recovery_duration.map(|rd| rd.stop_and_discard()); - gum::trace!( + gum::debug!( target: LOG_TARGET, candidate_hash = ?common_params.candidate_hash, erasure_root = ?common_params.erasure_root, diff --git a/polkadot/node/network/availability-recovery/src/task/strategy/systematic.rs b/polkadot/node/network/availability-recovery/src/task/strategy/systematic.rs index 65cac11fd5f0..4da9a0fac862 100644 --- a/polkadot/node/network/availability-recovery/src/task/strategy/systematic.rs +++ b/polkadot/node/network/availability-recovery/src/task/strategy/systematic.rs @@ -49,7 +49,7 @@ pub struct FetchSystematicChunks { threshold: usize, /// Validators that hold the systematic chunks. validators: Vec<(ChunkIndex, ValidatorIndex)>, - /// Backers. to be used as a backup. + /// Backers to be used as a backup. backers: Vec, /// Collection of in-flight requests. requesting_chunks: OngoingRequests, @@ -148,7 +148,7 @@ impl FetchSystematicChunks { reconstruct_duration.map(|rd| rd.stop_and_discard()); recovery_duration.map(|rd| rd.stop_and_discard()); - gum::trace!( + gum::debug!( target: LOG_TARGET, candidate_hash = ?common_params.candidate_hash, erasure_root = ?common_params.erasure_root, diff --git a/polkadot/node/subsystem-types/src/messages.rs b/polkadot/node/subsystem-types/src/messages.rs index 0a28c54c9c8e..6a8345802867 100644 --- a/polkadot/node/subsystem-types/src/messages.rs +++ b/polkadot/node/subsystem-types/src/messages.rs @@ -473,8 +473,8 @@ pub enum AvailabilityRecoveryMessage { CandidateReceipt, SessionIndex, Option, // Optional backing group to request from first. - Option, /* Optional core index that the candidate was occupying. Needed for - * systematic recovery. */ + Option, /* A `CoreIndex` needs to be specified for the recovery process to + * prefer systematic chunk recovery. */ oneshot::Sender>, ), } @@ -546,7 +546,7 @@ pub enum AvailabilityStoreMessage { available_data: AvailableData, /// Erasure root we expect to get after chunking. expected_erasure_root: Hash, - /// Core index that the candidate will begin occupying (that the para is scheduled on). + /// Core index where the candidate was backed. core_index: CoreIndex, /// Node features at the candidate relay parent. Used for computing the validator->chunk /// mapping. diff --git a/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.toml b/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.toml index d0abfedb4ddc..dbcd60f90d54 100644 --- a/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.toml +++ b/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.toml @@ -16,7 +16,7 @@ limits = { memory = "4G", cpu = "2" } requests = { memory = "2G", cpu = "1" } [[relaychain.node_groups]] - # Fix this to an image that doesn't speak /req_chunk/2 protocol. + # An image that doesn't speak /req_chunk/2 protocol. image = "{{POLKADOT_IMAGE}}:v1.0.0" name = "old" count = 2 From c94e7d56752566f4dd87d812021297fecd9e2769 Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 26 Jan 2024 15:18:54 +0200 Subject: [PATCH 093/126] some more comments --- polkadot/node/subsystem-util/src/availability_chunks.rs | 8 ++++---- polkadot/primitives/src/vstaging/mod.rs | 6 +++++- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/polkadot/node/subsystem-util/src/availability_chunks.rs b/polkadot/node/subsystem-util/src/availability_chunks.rs index 337d08a68f5b..98464c93754f 100644 --- a/polkadot/node/subsystem-util/src/availability_chunks.rs +++ b/polkadot/node/subsystem-util/src/availability_chunks.rs @@ -45,11 +45,11 @@ pub fn availability_chunk_index( Ok(validator_index.into()) } -/// Compute the per-core availability chunk indices. Item on position i corresponds to the i-th -/// validator. +/// Compute the per-core availability chunk indices. Returns a Vec which maps ValidatorIndex to +/// ChunkIndex for a given availability core index /// WARNING: THIS FUNCTION IS CRITICAL TO PARACHAIN CONSENSUS. -/// Any modification to the output of the function needs to be coordinated via the runtime. -/// It's best to use minimal/no external dependencies. +/// Any modification to the output of the function needs to be coordinated via the +/// runtime. It's best to use minimal/no external dependencies. pub fn availability_chunk_indices( maybe_node_features: Option<&NodeFeatures>, n_validators: usize, diff --git a/polkadot/primitives/src/vstaging/mod.rs b/polkadot/primitives/src/vstaging/mod.rs index 96681a34bc43..7394abae344a 100644 --- a/polkadot/primitives/src/vstaging/mod.rs +++ b/polkadot/primitives/src/vstaging/mod.rs @@ -66,7 +66,11 @@ pub mod node_features { /// Tells if tranch0 assignments could be sent in a single certificate. /// Reserved for: `` EnableAssignmentsV2 = 0, - /// Index of the availability chunk mapping feature bit. + /// Tells if the chunk mapping feature is enabled. + /// Enables the implementation of + /// [RFC-47](https://github.com/polkadot-fellows/RFCs/blob/main/text/0047-assignment-of-availability-chunks.md). + /// Must not be enabled unless all validators and collators have stopped using `req_chunk` + /// protocol version 1. If it is enabled, validators can start systematic chunk recovery. AvailabilityChunkMapping = 1, /// First unassigned feature bit. /// Every time a new feature flag is assigned it should take this value. From e4cbf1d087d71da851b21fd22fa7e65f6204a419 Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 26 Jan 2024 15:48:41 +0200 Subject: [PATCH 094/126] fill prdoc --- prdoc/pr_1644.prdoc | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/prdoc/pr_1644.prdoc b/prdoc/pr_1644.prdoc index 125ce63f13ac..456ee5216dcc 100644 --- a/prdoc/pr_1644.prdoc +++ b/prdoc/pr_1644.prdoc @@ -3,6 +3,7 @@ title: Add availability-recovery from systematic chunks doc: - audience: Node Dev description: | - ... + Implements https://github.com/polkadot-fellows/RFCs/pull/47 and adds the logic for availability recovery from systematic chunks. + The /req_chunk/1 req-response protocol is now considered deprecated in favour of /req_chunk/2. -crates: [ ] +crates: [] From 2f21035eb01f1b966c47b0edf3509706b64f128d Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 26 Jan 2024 16:29:32 +0200 Subject: [PATCH 095/126] zombienet tests --- .gitlab/pipeline/zombienet/polkadot.yml | 8 ++++ ...-chunk-fetching-network-compatibility.toml | 2 +- ...chunk-fetching-network-compatibility.zndsl | 15 +++---- .../0012-systematic-chunk-recovery.toml | 42 +++++++++++++++++++ .../0012-systematic-chunk-recovery.zndsl | 27 ++++++++++++ 5 files changed, 84 insertions(+), 10 deletions(-) create mode 100644 polkadot/zombienet_tests/functional/0012-systematic-chunk-recovery.toml create mode 100644 polkadot/zombienet_tests/functional/0012-systematic-chunk-recovery.zndsl diff --git a/.gitlab/pipeline/zombienet/polkadot.yml b/.gitlab/pipeline/zombienet/polkadot.yml index 9fed0ab227bf..a60ac9e4d1ed 100644 --- a/.gitlab/pipeline/zombienet/polkadot.yml +++ b/.gitlab/pipeline/zombienet/polkadot.yml @@ -158,6 +158,14 @@ zombienet-polkadot-functional-0011-chunk-fetching-network-compatibility: --local-dir="${LOCAL_DIR}/functional" --test="0011-chunk-fetching-network-compatibility.zndsl" +zombienet-polkadot-functional-0012-systematic-chunk-recovery: + extends: + - .zombienet-polkadot-common + script: + - /home/nonroot/zombie-net/scripts/ci/run-test-local-env-manager.sh + --local-dir="${LOCAL_DIR}/functional" + --test="0012-systematic-chunk-recovery.zndsl" + zombienet-polkadot-smoke-0001-parachains-smoke-test: extends: - .zombienet-polkadot-common diff --git a/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.toml b/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.toml index dbcd60f90d54..dee3821a7260 100644 --- a/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.toml +++ b/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.toml @@ -36,7 +36,7 @@ chain = "glutton-westend-local-{{id}}" [parachains.genesis.runtimeGenesis.patch.glutton] compute = "50000000" - storage = "131000000" + storage = "1310000000" trashDataCount = 5120 [parachains.collator] diff --git a/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.zndsl b/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.zndsl index 91874adfa631..192ee0fc5ede 100644 --- a/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.zndsl +++ b/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.zndsl @@ -13,10 +13,10 @@ old: parachain 2001 is registered within 60 seconds new: parachain 2001 is registered within 60 seconds # Ensure parachains made progress and approval checking works. -new: parachain 2000 block height is at least 10 within 300 seconds -old: parachain 2000 block height is at least 10 within 300 seconds -new: parachain 2001 block height is at least 10 within 300 seconds -old: parachain 2001 block height is at least 10 within 300 seconds +new: parachain 2000 block height is at least 15 within 400 seconds +old: parachain 2000 block height is at least 15 within 400 seconds +new: parachain 2001 block height is at least 15 within 400 seconds +old: parachain 2001 block height is at least 15 within 400 seconds new: reports substrate_block_height{status="finalized"} is at least 30 within 400 seconds old: reports substrate_block_height{status="finalized"} is at least 30 within 400 seconds @@ -32,15 +32,12 @@ old: reports polkadot_parachain_approvals_no_shows_total < 3 within 10 seconds new: log line contains "Trying the fallback protocol" within 10 seconds # Ensure that there are no failed recoveries. -new: reports polkadot_parachain_availability_recovery_recoveries_finished{result="success"} is at least 5 within 100 seconds -old: reports polkadot_parachain_availability_recovery_recoveries_finished{result="success"} is at least 5 within 100 seconds new: reports polkadot_parachain_availability_recovery_recoveries_finished{result="failure"} is 0 within 10 seconds old: reports polkadot_parachain_availability_recovery_recoveries_finished{result="failure"} is 0 within 10 seconds - # Ensure availability-distribution worked fine -new: reports polkadot_parachain_fetched_chunks_total{success="succeeded"} is at least 5 within 100 seconds -old: reports polkadot_parachain_fetched_chunks_total{success="succeeded"} is at least 5 within 100 seconds +new: reports polkadot_parachain_fetched_chunks_total{success="succeeded"} is at least 10 within 100 seconds +old: reports polkadot_parachain_fetched_chunks_total{success="succeeded"} is at least 10 within 100 seconds new: reports polkadot_parachain_fetched_chunks_total{success="failed"} is 0 within 10 seconds old: reports polkadot_parachain_fetched_chunks_total{success="failed"} is 0 within 10 seconds diff --git a/polkadot/zombienet_tests/functional/0012-systematic-chunk-recovery.toml b/polkadot/zombienet_tests/functional/0012-systematic-chunk-recovery.toml new file mode 100644 index 000000000000..8f6ebb6d9a01 --- /dev/null +++ b/polkadot/zombienet_tests/functional/0012-systematic-chunk-recovery.toml @@ -0,0 +1,42 @@ +[settings] +timeout = 1000 +bootnode = true + +[relaychain.genesis.runtimeGenesis.patch.configuration.config] + max_validators_per_core = 5 + needed_approvals = 10 + # TODO: this test won't pass for now, we don't yet have support for modifying a bitvec here + # node_features = "1" + +[relaychain] +default_image = "{{ZOMBIENET_INTEGRATION_TEST_IMAGE}}" +chain = "rococo-local" +default_command = "polkadot" + +[relaychain.default_resources] +limits = { memory = "4G", cpu = "2" } +requests = { memory = "2G", cpu = "1" } + + [[relaychain.node_groups]] + name = "validator" + count = 10 + args = ["-lparachain=debug,parachain::availability-recovery=trace,parachain::availability-distribution=trace"] + +{% for id in range(2000,2002) %} +[[parachains]] +id = {{id}} +addToGenesis = true +cumulus_based = true +chain = "glutton-westend-local-{{id}}" + [parachains.genesis.runtimeGenesis.patch.glutton] + compute = "50000000" + storage = "1310000000" + trashDataCount = 5120 + + [parachains.collator] + name = "collator" + image = "{{CUMULUS_IMAGE}}" + command = "polkadot-parachain" + args = ["-lparachain=debug"] + +{% endfor %} diff --git a/polkadot/zombienet_tests/functional/0012-systematic-chunk-recovery.zndsl b/polkadot/zombienet_tests/functional/0012-systematic-chunk-recovery.zndsl new file mode 100644 index 000000000000..a81e8cab3506 --- /dev/null +++ b/polkadot/zombienet_tests/functional/0012-systematic-chunk-recovery.zndsl @@ -0,0 +1,27 @@ +Description: Systematic chunk recovery is used if the chunk mapping feature is enabled. +Network: ./0012-systematic-chunk-recovery.toml +Creds: config + +# Check authority status. +validator: reports node_roles is 10 + +# Ensure parachains are registered. +validator: parachain 2000 is registered within 60 seconds +validator: parachain 2001 is registered within 60 seconds + +# Ensure parachains made progress and approval checking works. +validator: parachain 2000 block height is at least 15 within 400 seconds +validator: parachain 2001 block height is at least 15 within 400 seconds + +validator: reports substrate_block_height{status="finalized"} is at least 30 within 400 seconds + +validator: reports polkadot_parachain_approval_checking_finality_lag < 3 + +validator: reports polkadot_parachain_approvals_no_shows_total < 3 within 10 seconds + +# Ensure we used systematic recovery +validator: log line contains "Data recovery from systematic chunks complete" within 10 seconds +validator: count of log lines containing "Data recovery from systematic chunks is not possible" is 0 within 10 seconds + +# Ensure that there are no failed recoveries. +validator: reports polkadot_parachain_availability_recovery_recoveries_finished{result="failure"} is 0 within 10 seconds From 33d61629456c5d7f1cfd9031eeebd00db4051e06 Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 29 Jan 2024 12:33:10 +0200 Subject: [PATCH 096/126] try fixing zombienet tests --- ...0011-chunk-fetching-network-compatibility.zndsl | 14 +++++++------- .../0012-systematic-chunk-recovery.zndsl | 8 ++++---- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.zndsl b/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.zndsl index 192ee0fc5ede..1665d5755226 100644 --- a/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.zndsl +++ b/polkadot/zombienet_tests/functional/0011-chunk-fetching-network-compatibility.zndsl @@ -13,10 +13,10 @@ old: parachain 2001 is registered within 60 seconds new: parachain 2001 is registered within 60 seconds # Ensure parachains made progress and approval checking works. -new: parachain 2000 block height is at least 15 within 400 seconds -old: parachain 2000 block height is at least 15 within 400 seconds -new: parachain 2001 block height is at least 15 within 400 seconds -old: parachain 2001 block height is at least 15 within 400 seconds +new: parachain 2000 block height is at least 15 within 600 seconds +old: parachain 2000 block height is at least 15 within 600 seconds +new: parachain 2001 block height is at least 15 within 600 seconds +old: parachain 2001 block height is at least 15 within 600 seconds new: reports substrate_block_height{status="finalized"} is at least 30 within 400 seconds old: reports substrate_block_height{status="finalized"} is at least 30 within 400 seconds @@ -29,15 +29,15 @@ old: reports polkadot_parachain_approvals_no_shows_total < 3 within 10 seconds # Ensure we used the fallback network request. -new: log line contains "Trying the fallback protocol" within 10 seconds +new: log line contains "Trying the fallback protocol" within 100 seconds # Ensure that there are no failed recoveries. new: reports polkadot_parachain_availability_recovery_recoveries_finished{result="failure"} is 0 within 10 seconds old: reports polkadot_parachain_availability_recovery_recoveries_finished{result="failure"} is 0 within 10 seconds # Ensure availability-distribution worked fine -new: reports polkadot_parachain_fetched_chunks_total{success="succeeded"} is at least 10 within 100 seconds -old: reports polkadot_parachain_fetched_chunks_total{success="succeeded"} is at least 10 within 100 seconds +new: reports polkadot_parachain_fetched_chunks_total{success="succeeded"} is at least 10 within 200 seconds +old: reports polkadot_parachain_fetched_chunks_total{success="succeeded"} is at least 10 within 200 seconds new: reports polkadot_parachain_fetched_chunks_total{success="failed"} is 0 within 10 seconds old: reports polkadot_parachain_fetched_chunks_total{success="failed"} is 0 within 10 seconds diff --git a/polkadot/zombienet_tests/functional/0012-systematic-chunk-recovery.zndsl b/polkadot/zombienet_tests/functional/0012-systematic-chunk-recovery.zndsl index a81e8cab3506..245a29650f62 100644 --- a/polkadot/zombienet_tests/functional/0012-systematic-chunk-recovery.zndsl +++ b/polkadot/zombienet_tests/functional/0012-systematic-chunk-recovery.zndsl @@ -10,17 +10,17 @@ validator: parachain 2000 is registered within 60 seconds validator: parachain 2001 is registered within 60 seconds # Ensure parachains made progress and approval checking works. -validator: parachain 2000 block height is at least 15 within 400 seconds -validator: parachain 2001 block height is at least 15 within 400 seconds +validator: parachain 2000 block height is at least 15 within 600 seconds +validator: parachain 2001 block height is at least 15 within 600 seconds validator: reports substrate_block_height{status="finalized"} is at least 30 within 400 seconds validator: reports polkadot_parachain_approval_checking_finality_lag < 3 -validator: reports polkadot_parachain_approvals_no_shows_total < 3 within 10 seconds +validator: reports polkadot_parachain_approvals_no_shows_total < 3 within 100 seconds # Ensure we used systematic recovery -validator: log line contains "Data recovery from systematic chunks complete" within 10 seconds +validator: log line contains "Data recovery from systematic chunks complete" within 100 seconds validator: count of log lines containing "Data recovery from systematic chunks is not possible" is 0 within 10 seconds # Ensure that there are no failed recoveries. From eddae38d59aee0e40826cc038d506849b4ae2c5b Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 30 Jan 2024 10:41:52 +0200 Subject: [PATCH 097/126] update implementer's guide --- .../network/availability-recovery/src/lib.rs | 2 +- .../src/node/approval/approval-voting.md | 4 +- .../availability/availability-recovery.md | 247 ++++++++++-------- .../src/types/overseer-protocol.md | 3 + 4 files changed, 142 insertions(+), 114 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index e937ac200d0e..8a4e3dd4baa8 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -415,7 +415,7 @@ async fn handle_recover( let systematic_threshold = systematic_recovery_threshold(n_validators)?; let mut recovery_strategies: VecDeque< Box::Sender>>, - > = VecDeque::with_capacity(2); + > = VecDeque::with_capacity(3); if let Some(backing_group) = backing_group { if let Some(backing_validators) = session_info.validator_groups.get(backing_group) { diff --git a/polkadot/roadmap/implementers-guide/src/node/approval/approval-voting.md b/polkadot/roadmap/implementers-guide/src/node/approval/approval-voting.md index f01e23b13704..8bbbb5178fea 100644 --- a/polkadot/roadmap/implementers-guide/src/node/approval/approval-voting.md +++ b/polkadot/roadmap/implementers-guide/src/node/approval/approval-voting.md @@ -395,8 +395,8 @@ On receiving an `ApprovedAncestor(Hash, BlockNumber, response_channel)`: * Requires `(SessionIndex, SessionInfo, CandidateReceipt, ValidatorIndex, backing_group, block_hash, candidate_index)` * Extract the public key of the `ValidatorIndex` from the `SessionInfo` for the session. -* Issue an `AvailabilityRecoveryMessage::RecoverAvailableData(candidate, session_index, Some(backing_group), None, - response_sender)` +* Issue an `AvailabilityRecoveryMessage::RecoverAvailableData(candidate, session_index, Some(backing_group), +Some(core_index), response_sender)` * Load the historical validation code of the parachain by dispatching a `RuntimeApiRequest::ValidationCodeByHash(descriptor.validation_code_hash)` against the state of `block_hash`. * Spawn a background task with a clone of `background_tx` diff --git a/polkadot/roadmap/implementers-guide/src/node/availability/availability-recovery.md b/polkadot/roadmap/implementers-guide/src/node/availability/availability-recovery.md index ae9393693668..40357a5ee004 100644 --- a/polkadot/roadmap/implementers-guide/src/node/availability/availability-recovery.md +++ b/polkadot/roadmap/implementers-guide/src/node/availability/availability-recovery.md @@ -1,84 +1,106 @@ # Availability Recovery -This subsystem is the inverse of the [Availability Distribution](availability-distribution.md) subsystem: validators -will serve the availability chunks kept in the availability store to nodes who connect to them. And the subsystem will -also implement the other side: the logic for nodes to connect to validators, request availability pieces, and -reconstruct the `AvailableData`. +This subsystem is responsible for recovering the data made available via the +[Availability Distribution](availability-distribution.md) subsystem, neccessary for candidate validation during the +approval/disputes processes. Additionally, it is also being used by collators to recover PoVs in adversarial scenarios +where the other collators of the para are censoring blocks. -This version of the availability recovery subsystem is based off of direct connections to validators. In order to -recover any given `AvailableData`, we must recover at least `f + 1` pieces from validators of the session. Thus, we will -connect to and query randomly chosen validators until we have received `f + 1` pieces. +According to the polkadot protocol, in order to recover any given `AvailableData`, we generally must recover at least +`f + 1` pieces from validators of the session. Thus, we should connect to and query randomly chosen validators until we +have received `f + 1` pieces. + +In practice, there are various optimisations implemented in this subsystem which avoid querying all chunks from +different validators and/or avoid doing the chunk reconstruction altogether. ## Protocol -`PeerSet`: `Validation` +This version of the availability recovery subsystem is based only on request-response network protocols. Input: -* `NetworkBridgeUpdate(update)` -* `AvailabilityRecoveryMessage::RecoverAvailableData(candidate, session, backing_group, maybe_relay_parent_block_number, response)` +* `AvailabilityRecoveryMessage::RecoverAvailableData(candidate, session, backing_group, core_index, response)` Output: -* `NetworkBridge::SendValidationMessage` -* `NetworkBridge::ReportPeer` -* `AvailabilityStore::QueryChunk` +* `NetworkBridgeMessage::SendRequests` +* `AvailabilityStoreMessage::QueryAllChunks` +* `AvailabilityStoreMessage::QueryAvailableData` +* `AvailabilityStoreMessage::QueryChunkSize` + ## Functionality -We hold a state which tracks the currently ongoing recovery tasks, as well as which request IDs correspond to which -task. A recovery task is a structure encapsulating all recovery tasks with the network necessary to recover the -available data in respect to one candidate. +We hold a state which tracks the currently ongoing recovery tasks. A `RecoveryTask` is a structure encapsulating all +network tasks needed in order to recover the available data in respect to a candidate. + +Each `RecoveryTask` has a collection of ordered recovery strategies to try. ```rust +/// Subsystem state. struct State { - /// Each recovery is implemented as an independent async task, and the handles only supply information about the result. - ongoing_recoveries: FuturesUnordered, - /// A recent block hash for which state should be available. - live_block_hash: Hash, - // An LRU cache of recently recovered data. - availability_lru: LruMap>, + /// Each recovery task is implemented as its own async task, + /// and these handles are for communicating with them. + ongoing_recoveries: FuturesUnordered, + /// A recent block hash for which state should be available. + live_block: (BlockNumber, Hash), + /// An LRU cache of recently recovered data. + availability_lru: LruMap, + /// Cached runtime info. + runtime_info: RuntimeInfo, } -/// This is a future, which concludes either when a response is received from the recovery tasks, -/// or all the `awaiting` channels have closed. -struct RecoveryHandle { - candidate_hash: CandidateHash, - interaction_response: RemoteHandle, - awaiting: Vec>>, -} - -struct Unavailable; -struct Concluded(CandidateHash, Result); - -struct RecoveryTaskParams { - validator_authority_keys: Vec, - validators: Vec, - // The number of pieces needed. - threshold: usize, - candidate_hash: Hash, - erasure_root: Hash, +struct RecoveryParams { + /// Discovery ids of `validators`. + pub validator_authority_keys: Vec, + /// Number of validators. + pub n_validators: usize, + /// The number of regular chunks needed. + pub threshold: usize, + /// The number of systematic chunks needed. + pub systematic_threshold: usize, + /// A hash of the relevant candidate. + pub candidate_hash: CandidateHash, + /// The root of the erasure encoding of the candidate. + pub erasure_root: Hash, + /// Metrics to report. + pub metrics: Metrics, + /// Do not request data from availability-store. Useful for collators. + pub bypass_availability_store: bool, + /// The type of check to perform after available data was recovered. + pub post_recovery_check: PostRecoveryCheck, + /// The blake2-256 hash of the PoV. + pub pov_hash: Hash, + /// Protocol name for ChunkFetchingV1. + pub req_v1_protocol_name: ProtocolName, + /// Protocol name for ChunkFetchingV2. + pub req_v2_protocol_name: ProtocolName, + /// Whether or not chunk mapping is enabled. + pub chunk_mapping_enabled: bool, } -enum RecoveryTask { - RequestFromBackers { - // a random shuffling of the validators from the backing group which indicates the order - // in which we connect to them and request the chunk. - shuffled_backers: Vec, - } - RequestChunksFromValidators { - // a random shuffling of the validators which indicates the order in which we connect to the validators and - // request the chunk from them. - shuffling: Vec, - received_chunks: Map, - requesting_chunks: FuturesUnordered>, - } +pub struct RecoveryTask { + sender: Sender, + params: RecoveryParams, + strategies: VecDeque>>, + state: task::State, } -struct RecoveryTask { - to_subsystems: SubsystemSender, - params: RecoveryTaskParams, - source: Source, +#[async_trait::async_trait] +/// Common trait for runnable recovery strategies. +pub trait RecoveryStrategy: Send { + /// Main entry point of the strategy. + async fn run( + mut self: Box, + state: &mut task::State, + sender: &mut Sender, + common_params: &RecoveryParams, + ) -> Result; + + /// Return the name of the strategy for logging purposes. + fn display_name(&self) -> &'static str; + + /// Return the strategy type for use as a metric label. + fn strategy_type(&self) -> &'static str; } ``` @@ -90,68 +112,71 @@ Ignore `BlockFinalized` signals. On `Conclude`, shut down the subsystem. -#### `AvailabilityRecoveryMessage::RecoverAvailableData(receipt, session, Option, Option response)` +#### `AvailabilityRecoveryMessage::RecoverAvailableData(...)` -1. Check the `availability_lru` for the candidate and return the data if so. -1. Check if there is already an recovery handle for the request. If so, add the response handle to it. +1. Check the `availability_lru` for the candidate and return the data if present. +1. Check if there is already a recovery handle for the request. If so, add the response handle to it. 1. Otherwise, load the session info for the given session under the state of `live_block_hash`, and initiate a recovery - task with *`launch_recovery_task`*. Add a recovery handle to the state and add the response channel to it. + task with `launch_recovery_task`. Add a recovery handle to the state and add the response channel to it. 1. If the session info is not available, return `RecoveryError::Unavailable` on the response channel. ### Recovery logic -#### `launch_recovery_task(session_index, session_info, candidate_receipt, candidate_hash, Option)` +#### `handle_recover(...) -> Result<()>` -1. Compute the threshold from the session info. It should be `f + 1`, where `n = 3f + k`, where `k in {1, 2, 3}`, and - `n` is the number of validators. -1. Set the various fields of `RecoveryParams` based on the validator lists in `session_info` and information about the - candidate. -1. If the `backing_group_index` is `Some`, start in the `RequestFromBackers` phase with a shuffling of the backing group - validator indices and a `None` requesting value. -1. Otherwise, start in the `RequestChunksFromValidators` source with `received_chunks`,`requesting_chunks`, and - `next_shuffling` all empty. -1. Set the `to_subsystems` sender to be equal to a clone of the `SubsystemContext`'s sender. -1. Initialize `received_chunks` to an empty set, as well as `requesting_chunks`. +Instantiate the appropriate `RecoveryStrategy`es, based on the subsystem configuration, params and session info. +Call `launch_recovery_task()`. -Launch the source as a background task running `run(recovery_task)`. +#### `launch_recovery_task(state, ctx, response_sender, recovery_strategies, params) -> Result<()>` -#### `run(recovery_task) -> Result` +Create the `RecoveryTask` and launch it as a background task running `recovery_task.run()`. -```rust -// How many parallel requests to have going at once. -const N_PARALLEL: usize = 50; -``` +#### `recovery_task.run(mut self) -> Result` + +* Loop: + * Pop a strategy from the queue. If none are left, return `RecoveryError::Unavailable`. + * Run the strategy. + * If the strategy returned successfully or returned `RecoveryError::Invalid`, break the loop. + +### Recovery strategies + +#### `FetchFull` + +This strategy tries requesting the full available data from the validators in the backing group to +which the node is already connected. They are tried one by one in a random order. +It is very performant if there's enough network bandwidth and the backing group is not overloaded. +The costly reed-solomon reconstruction is not needed. + +#### `FetchSystematicChunks` + +Very similar to `FetchChunks` below but requests from the validators that hold the systematic chunks, so that we avoid +reed-solomon reconstruction. Only possible if `node_features::FeatureIndex::AvailabilityChunkMapping` is enabled and +the `core_index` is supplied (currently only for recoveries triggered by approval voting). + +More info in +[RFC-47](https://github.com/polkadot-fellows/RFCs/blob/main/text/0047-assignment-of-availability-chunks.md). + +#### `FetchChunks` + +The least performant strategy but also the most comprehensive one. It's the only one that cannot fail under the +byzantine threshold assumption, so it's always added as the last one in the `recovery_strategies` queue. + +Performs parallel chunk requests to validators. When enough chunks were received, do the reconstruction. +In the worst case, all validators will be tried. + +### Default recovery strategy configuration + +#### For validators + +If the estimated available data size is smaller than a configured constant (currently 128Kib), try doing `FetchFull` +first. +Next, if the preconditions described in `FetchSystematicChunks` above are met, try systematic recovery. +As a last resort, do `FetchChunks`. + +#### For collators + +Collators currently only use `FetchChunks`, as they only attempt recoveries in rare scenarios. -* Request `AvailabilityStoreMessage::QueryAvailableData`. If it exists, return that. -* If the task contains `RequestFromBackers` - * Loop: - * If the `requesting_pov` is `Some`, poll for updates on it. If it concludes, set `requesting_pov` to `None`. - * If the `requesting_pov` is `None`, take the next backer off the `shuffled_backers`. - * If the backer is `Some`, issue a `NetworkBridgeMessage::Requests` with a network request for the - `AvailableData` and wait for the response. - * If it concludes with a `None` result, return to beginning. - * If it concludes with available data, attempt a re-encoding. - * If it has the correct erasure-root, break and issue a `Ok(available_data)`. - * If it has an incorrect erasure-root, return to beginning. - * Send the result to each member of `awaiting`. - * If the backer is `None`, set the source to `RequestChunksFromValidators` with a random shuffling of validators - and empty `received_chunks`, and `requesting_chunks` and break the loop. - -* If the task contains `RequestChunksFromValidators`: - * Request `AvailabilityStoreMessage::QueryAllChunks`. For each chunk that exists, add it to `received_chunks` and - remote the validator from `shuffling`. - * Loop: - * If `received_chunks + requesting_chunks + shuffling` lengths are less than the threshold, break and return - `Err(Unavailable)`. - * Poll for new updates from `requesting_chunks`. Check merkle proofs of any received chunks. If the request simply - fails due to network issues, insert into the front of `shuffling` to be retried. - * If `received_chunks` has more than `threshold` entries, attempt to recover the data. - * If that fails, return `Err(RecoveryError::Invalid)` - * If correct: - * If re-encoding produces an incorrect erasure-root, break and issue a `Err(RecoveryError::Invalid)`. - * break and issue `Ok(available_data)` - * Send the result to each member of `awaiting`. - * While there are fewer than `N_PARALLEL` entries in `requesting_chunks`, - * Pop the next item from `shuffling`. If it's empty and `requesting_chunks` is empty, return - `Err(RecoveryError::Unavailable)`. - * Issue a `NetworkBridgeMessage::Requests` and wait for the response in `requesting_chunks`. +Moreover, the recovery task is specially configured to not attempt requesting data from the local availability-store +(because it doesn't exist) and to not reencode the data after a succcessful recovery (because it's an expensive check +that is not needed; checking the pov_hash is enough for collators). diff --git a/polkadot/roadmap/implementers-guide/src/types/overseer-protocol.md b/polkadot/roadmap/implementers-guide/src/types/overseer-protocol.md index 54cdc2edd12d..540e4f13d638 100644 --- a/polkadot/roadmap/implementers-guide/src/types/overseer-protocol.md +++ b/polkadot/roadmap/implementers-guide/src/types/overseer-protocol.md @@ -238,6 +238,9 @@ enum AvailabilityRecoveryMessage { CandidateReceipt, SessionIndex, Option, // Backing validator group to request the data directly from. + Option, /* A `CoreIndex` needs to be specified for the recovery process to + * prefer systematic chunk recovery. This is the core that the candidate + * was occupying while pending availability. */ ResponseChannel>, ), } From c2f0f23dc154532427382e51c3761481bd165d33 Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 30 Jan 2024 10:48:26 +0200 Subject: [PATCH 098/126] markdown format --- .../implementers-guide/src/node/approval/approval-voting.md | 2 +- .../src/node/availability/availability-recovery.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/polkadot/roadmap/implementers-guide/src/node/approval/approval-voting.md b/polkadot/roadmap/implementers-guide/src/node/approval/approval-voting.md index 8bbbb5178fea..9b4082c49e2f 100644 --- a/polkadot/roadmap/implementers-guide/src/node/approval/approval-voting.md +++ b/polkadot/roadmap/implementers-guide/src/node/approval/approval-voting.md @@ -395,7 +395,7 @@ On receiving an `ApprovedAncestor(Hash, BlockNumber, response_channel)`: * Requires `(SessionIndex, SessionInfo, CandidateReceipt, ValidatorIndex, backing_group, block_hash, candidate_index)` * Extract the public key of the `ValidatorIndex` from the `SessionInfo` for the session. -* Issue an `AvailabilityRecoveryMessage::RecoverAvailableData(candidate, session_index, Some(backing_group), +* Issue an `AvailabilityRecoveryMessage::RecoverAvailableData(candidate, session_index, Some(backing_group), Some(core_index), response_sender)` * Load the historical validation code of the parachain by dispatching a `RuntimeApiRequest::ValidationCodeByHash(descriptor.validation_code_hash)` against the state of `block_hash`. diff --git a/polkadot/roadmap/implementers-guide/src/node/availability/availability-recovery.md b/polkadot/roadmap/implementers-guide/src/node/availability/availability-recovery.md index 40357a5ee004..d8b4db9eb766 100644 --- a/polkadot/roadmap/implementers-guide/src/node/availability/availability-recovery.md +++ b/polkadot/roadmap/implementers-guide/src/node/availability/availability-recovery.md @@ -5,7 +5,7 @@ This subsystem is responsible for recovering the data made available via the approval/disputes processes. Additionally, it is also being used by collators to recover PoVs in adversarial scenarios where the other collators of the para are censoring blocks. -According to the polkadot protocol, in order to recover any given `AvailableData`, we generally must recover at least +According to the Polkadot protocol, in order to recover any given `AvailableData`, we generally must recover at least `f + 1` pieces from validators of the session. Thus, we should connect to and query randomly chosen validators until we have received `f + 1` pieces. From 9b49d40c0080fa4a194b54e41ad4b840f458e27e Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 30 Jan 2024 11:14:23 +0200 Subject: [PATCH 099/126] move erasure_task_tx to the common recovery params --- .../network/availability-recovery/src/lib.rs | 13 +++++-------- .../availability-recovery/src/task/mod.rs | 7 +++++-- .../src/task/strategy/chunks.rs | 19 +++++++------------ .../src/task/strategy/full.rs | 7 +++---- .../src/task/strategy/mod.rs | 12 ++++++------ .../src/task/strategy/systematic.rs | 11 ++--------- .../availability/availability-recovery.md | 2 ++ 7 files changed, 30 insertions(+), 41 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/lib.rs b/polkadot/node/network/availability-recovery/src/lib.rs index 8a4e3dd4baa8..7e6c72c59606 100644 --- a/polkadot/node/network/availability-recovery/src/lib.rs +++ b/polkadot/node/network/availability-recovery/src/lib.rs @@ -458,12 +458,10 @@ async fn handle_recover( RecoveryStrategyKind::BackersFirstIfSizeLowerThenSystematicChunks(_), true, ) | - (RecoveryStrategyKind::BackersThenSystematicChunks, _) => recovery_strategies.push_back( - Box::new(FetchFull::new(FetchFullParams { - validators: backing_validators.to_vec(), - erasure_task_tx: erasure_task_tx.clone(), - })), - ), + (RecoveryStrategyKind::BackersThenSystematicChunks, _) => + recovery_strategies.push_back(Box::new(FetchFull::new( + FetchFullParams { validators: backing_validators.to_vec() }, + ))), _ => {}, }; @@ -522,7 +520,6 @@ async fn handle_recover( FetchSystematicChunksParams { validators, backers: backer_group.map(|v| v.to_vec()).unwrap_or_else(|| vec![]), - erasure_task_tx: erasure_task_tx.clone(), }, ))); } @@ -530,7 +527,6 @@ async fn handle_recover( recovery_strategies.push_back(Box::new(FetchChunks::new(FetchChunksParams { n_validators: session_info.validators.len(), - erasure_task_tx, }))); let session_info = session_info.clone(); @@ -556,6 +552,7 @@ async fn handle_recover( req_v1_protocol_name, req_v2_protocol_name, chunk_mapping_enabled, + erasure_task_tx, }, ) .await diff --git a/polkadot/node/network/availability-recovery/src/task/mod.rs b/polkadot/node/network/availability-recovery/src/task/mod.rs index d5d6b25eb51b..a1c4e481c868 100644 --- a/polkadot/node/network/availability-recovery/src/task/mod.rs +++ b/polkadot/node/network/availability-recovery/src/task/mod.rs @@ -26,7 +26,7 @@ pub use self::strategy::{ SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT, }; -use crate::{metrics::Metrics, PostRecoveryCheck, LOG_TARGET}; +use crate::{metrics::Metrics, ErasureTask, PostRecoveryCheck, LOG_TARGET}; use parity_scale_codec::Encode; use polkadot_node_primitives::AvailableData; @@ -34,7 +34,7 @@ use polkadot_node_subsystem::{messages::AvailabilityStoreMessage, overseer, Reco use polkadot_primitives::{AuthorityDiscoveryId, CandidateHash, Hash}; use sc_network::ProtocolName; -use futures::channel::oneshot; +use futures::channel::{mpsc, oneshot}; use std::collections::VecDeque; /// Recovery parameters common to all strategies in a `RecoveryTask`. @@ -78,6 +78,9 @@ pub struct RecoveryParams { /// Whether or not chunk mapping is enabled. pub chunk_mapping_enabled: bool, + + /// Channel to the erasure task handler. + pub erasure_task_tx: mpsc::Sender, } /// A stateful reconstruction of availability data in reference to diff --git a/polkadot/node/network/availability-recovery/src/task/strategy/chunks.rs b/polkadot/node/network/availability-recovery/src/task/strategy/chunks.rs index 6ce73dcd36b9..b6376a5b543e 100644 --- a/polkadot/node/network/availability-recovery/src/task/strategy/chunks.rs +++ b/polkadot/node/network/availability-recovery/src/task/strategy/chunks.rs @@ -37,8 +37,6 @@ use std::collections::VecDeque; /// Parameters specific to the `FetchChunks` strategy. pub struct FetchChunksParams { pub n_validators: usize, - /// Channel to the erasure task handler. - pub erasure_task_tx: futures::channel::mpsc::Sender, } /// `RecoveryStrategy` that requests chunks from validators, in parallel. @@ -51,8 +49,6 @@ pub struct FetchChunks { validators: VecDeque, /// Collection of in-flight requests. requesting_chunks: OngoingRequests, - /// Channel to the erasure task handler. - erasure_task_tx: futures::channel::mpsc::Sender, } impl FetchChunks { @@ -69,7 +65,6 @@ impl FetchChunks { total_received_responses: 0, validators, requesting_chunks: FuturesUndead::new(), - erasure_task_tx: params.erasure_task_tx, } } @@ -118,7 +113,9 @@ impl FetchChunks { // Send request to reconstruct available data from chunks. let (avilable_data_tx, available_data_rx) = oneshot::channel(); - self.erasure_task_tx + + let mut erasure_task_tx = common_params.erasure_task_tx.clone(); + erasure_task_tx .send(ErasureTask::Reconstruct( common_params.n_validators, // Safe to leave an empty vec in place, as we're stopping the recovery process if @@ -137,7 +134,7 @@ impl FetchChunks { match available_data_response { // Attempt post-recovery check. - Ok(data) => do_post_recovery_check(common_params, data, &mut self.erasure_task_tx) + Ok(data) => do_post_recovery_check(common_params, data) .await .map_err(|e| { recovery_duration.map(|rd| rd.stop_and_discard()); @@ -307,12 +304,10 @@ mod tests { #[test] fn test_get_desired_request_count() { - let num_validators = 100; - let threshold = recovery_threshold(num_validators).unwrap(); - let (erasure_task_tx, _erasure_task_rx) = futures::channel::mpsc::channel(16); + let n_validators = 100; + let threshold = recovery_threshold(n_validators).unwrap(); - let mut fetch_chunks_task = - FetchChunks::new(FetchChunksParams { n_validators: num_validators, erasure_task_tx }); + let mut fetch_chunks_task = FetchChunks::new(FetchChunksParams { n_validators }); assert_eq!(fetch_chunks_task.get_desired_request_count(0, threshold), threshold); fetch_chunks_task.error_count = 1; fetch_chunks_task.total_received_responses = 1; diff --git a/polkadot/node/network/availability-recovery/src/task/strategy/full.rs b/polkadot/node/network/availability-recovery/src/task/strategy/full.rs index 5b9b2b5a7440..1d7fbe8ea3c8 100644 --- a/polkadot/node/network/availability-recovery/src/task/strategy/full.rs +++ b/polkadot/node/network/availability-recovery/src/task/strategy/full.rs @@ -34,8 +34,6 @@ use rand::seq::SliceRandom; pub struct FetchFullParams { /// Validators that will be used for fetching the data. pub validators: Vec, - /// Channel to the erasure task handler. - pub erasure_task_tx: futures::channel::mpsc::Sender, } /// `RecoveryStrategy` that sequentially tries to fetch the full `AvailableData` from @@ -101,8 +99,9 @@ impl RecoveryStrategy let maybe_data = match common_params.post_recovery_check { PostRecoveryCheck::Reencode => { let (reencode_tx, reencode_rx) = oneshot::channel(); - self.params - .erasure_task_tx + let mut erasure_task_tx = common_params.erasure_task_tx.clone(); + + erasure_task_tx .send(ErasureTask::Reencode( common_params.n_validators, common_params.erasure_root, diff --git a/polkadot/node/network/availability-recovery/src/task/strategy/mod.rs b/polkadot/node/network/availability-recovery/src/task/strategy/mod.rs index 1b24265844de..ba7f520c97be 100644 --- a/polkadot/node/network/availability-recovery/src/task/strategy/mod.rs +++ b/polkadot/node/network/availability-recovery/src/task/strategy/mod.rs @@ -29,10 +29,7 @@ use crate::{ futures_undead::FuturesUndead, ErasureTask, PostRecoveryCheck, RecoveryParams, LOG_TARGET, }; -use futures::{ - channel::{mpsc, oneshot}, - SinkExt, -}; +use futures::{channel::oneshot, SinkExt}; use parity_scale_codec::Decode; use polkadot_erasure_coding::branch_hash; #[cfg(not(test))] @@ -127,8 +124,8 @@ fn is_chunk_valid(params: &RecoveryParams, chunk: &ErasureChunk) -> bool { async fn do_post_recovery_check( params: &RecoveryParams, data: AvailableData, - erasure_task_tx: &mut mpsc::Sender, ) -> Result { + let mut erasure_task_tx = params.erasure_task_tx.clone(); match params.post_recovery_check { PostRecoveryCheck::Reencode => { // Send request to re-encode the chunks and check merkle root. @@ -624,7 +621,8 @@ mod tests { use crate::{tests::*, Metrics, RecoveryStrategy, RecoveryTask}; use assert_matches::assert_matches; use futures::{ - channel::mpsc::UnboundedReceiver, executor, future, Future, FutureExt, StreamExt, + channel::mpsc::{self, UnboundedReceiver}, + executor, future, Future, FutureExt, StreamExt, }; use parity_scale_codec::Error as DecodingError; use polkadot_erasure_coding::{recovery_threshold, systematic_recovery_threshold}; @@ -652,6 +650,7 @@ mod tests { Sr25519Keyring::One, Sr25519Keyring::Two, ]; + let (erasure_task_tx, _erasure_task_rx) = mpsc::channel(10); Self { validator_authority_keys: validator_authority_id(&validators), @@ -667,6 +666,7 @@ mod tests { req_v1_protocol_name: "/req_chunk/1".into(), req_v2_protocol_name: "/req_chunk/2".into(), chunk_mapping_enabled: true, + erasure_task_tx, } } } diff --git a/polkadot/node/network/availability-recovery/src/task/strategy/systematic.rs b/polkadot/node/network/availability-recovery/src/task/strategy/systematic.rs index 4da9a0fac862..f0a949ce0b2f 100644 --- a/polkadot/node/network/availability-recovery/src/task/strategy/systematic.rs +++ b/polkadot/node/network/availability-recovery/src/task/strategy/systematic.rs @@ -23,7 +23,7 @@ use crate::{ }, RecoveryParams, RecoveryStrategy, State, }, - ErasureTask, LOG_TARGET, + LOG_TARGET, }; use polkadot_node_primitives::AvailableData; @@ -38,8 +38,6 @@ pub struct FetchSystematicChunksParams { pub validators: Vec<(ChunkIndex, ValidatorIndex)>, /// Validators in the backing group, to be used as a backup for requesting systematic chunks. pub backers: Vec, - /// Channel to the erasure task handler. - pub erasure_task_tx: futures::channel::mpsc::Sender, } /// `RecoveryStrategy` that attempts to recover the systematic chunks from the validators that @@ -53,8 +51,6 @@ pub struct FetchSystematicChunks { backers: Vec, /// Collection of in-flight requests. requesting_chunks: OngoingRequests, - /// Channel to the erasure task handler. - erasure_task_tx: futures::channel::mpsc::Sender, } impl FetchSystematicChunks { @@ -65,7 +61,6 @@ impl FetchSystematicChunks { validators: params.validators, backers: params.backers, requesting_chunks: FuturesUndead::new(), - erasure_task_tx: params.erasure_task_tx, } } @@ -128,7 +123,7 @@ impl FetchSystematicChunks { drop(reconstruct_duration); // Attempt post-recovery check. - do_post_recovery_check(common_params, data, &mut self.erasure_task_tx) + do_post_recovery_check(common_params, data) .await .map_err(|e| { recovery_duration.map(|rd| rd.stop_and_discard()); @@ -331,12 +326,10 @@ mod tests { fn test_get_desired_request_count() { let num_validators = 100; let threshold = systematic_recovery_threshold(num_validators).unwrap(); - let (erasure_task_tx, _erasure_task_rx) = futures::channel::mpsc::channel(16); let systematic_chunks_task = FetchSystematicChunks::new(FetchSystematicChunksParams { validators: vec![(1.into(), 1.into()); num_validators], backers: vec![], - erasure_task_tx, }); assert_eq!(systematic_chunks_task.get_desired_request_count(0, threshold), threshold); assert_eq!(systematic_chunks_task.get_desired_request_count(5, threshold), threshold - 5); diff --git a/polkadot/roadmap/implementers-guide/src/node/availability/availability-recovery.md b/polkadot/roadmap/implementers-guide/src/node/availability/availability-recovery.md index d8b4db9eb766..7e82567b0dab 100644 --- a/polkadot/roadmap/implementers-guide/src/node/availability/availability-recovery.md +++ b/polkadot/roadmap/implementers-guide/src/node/availability/availability-recovery.md @@ -76,6 +76,8 @@ struct RecoveryParams { pub req_v2_protocol_name: ProtocolName, /// Whether or not chunk mapping is enabled. pub chunk_mapping_enabled: bool, + /// Channel to the erasure task handler. + pub erasure_task_tx: mpsc::Sender, } pub struct RecoveryTask { From 826208fe8c689a90adcf7271dc358948c5fea42c Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 9 Feb 2024 13:15:25 +0200 Subject: [PATCH 100/126] address some review comments --- polkadot/erasure-coding/src/lib.rs | 12 ++++-------- .../src/requester/fetch_task/mod.rs | 2 +- .../src/task/strategy/systematic.rs | 2 +- .../parachains/src/configuration/migration/v10.rs | 9 ++------- 4 files changed, 8 insertions(+), 17 deletions(-) diff --git a/polkadot/erasure-coding/src/lib.rs b/polkadot/erasure-coding/src/lib.rs index 534680e9fe86..b354c3dac64c 100644 --- a/polkadot/erasure-coding/src/lib.rs +++ b/polkadot/erasure-coding/src/lib.rs @@ -144,7 +144,7 @@ fn code_params(n_validators: usize) -> Result { /// possible. pub fn reconstruct_from_systematic_v1( n_validators: usize, - chunks: Vec<&[u8]>, + chunks: Vec>, ) -> Result { reconstruct_from_systematic(n_validators, chunks) } @@ -155,7 +155,7 @@ pub fn reconstruct_from_systematic_v1( /// recovery is not possible. pub fn reconstruct_from_systematic( n_validators: usize, - chunks: Vec<&[u8]>, + chunks: Vec>, ) -> Result { let code_params = code_params(n_validators)?; let k = code_params.k(); @@ -167,11 +167,7 @@ pub fn reconstruct_from_systematic( } let bytes = code_params.make_encoder().reconstruct_from_systematic( - chunks - .into_iter() - .take(k) - .map(|data| WrappedShard::new(data.to_vec())) - .collect(), + chunks.into_iter().take(k).map(|data| WrappedShard::new(data)).collect(), )?; Decode::decode(&mut &bytes[..]).map_err(|err| Error::Decode(err)) @@ -405,7 +401,7 @@ mod tests { assert_eq!( reconstruct_from_systematic_v1( n_validators as usize, - chunks.iter().take(kpow2).map(|v| &v[..]).collect() + chunks.into_iter().take(kpow2).collect() ) .unwrap(), available_data.0 diff --git a/polkadot/node/network/availability-distribution/src/requester/fetch_task/mod.rs b/polkadot/node/network/availability-distribution/src/requester/fetch_task/mod.rs index c0068fde1702..daec3979895c 100644 --- a/polkadot/node/network/availability-distribution/src/requester/fetch_task/mod.rs +++ b/polkadot/node/network/availability-distribution/src/requester/fetch_task/mod.rs @@ -419,7 +419,7 @@ impl RunningTask { chunk_index = ?self.request.index, candidate_hash = ?self.request.candidate_hash, err = ?e, - "Peer sent us invalid erasure chunk data" + "Peer sent us invalid erasure chunk data (v2)" ); Err(TaskError::PeerError) }, diff --git a/polkadot/node/network/availability-recovery/src/task/strategy/systematic.rs b/polkadot/node/network/availability-recovery/src/task/strategy/systematic.rs index f0a949ce0b2f..677bc2d1375a 100644 --- a/polkadot/node/network/availability-recovery/src/task/strategy/systematic.rs +++ b/polkadot/node/network/availability-recovery/src/task/strategy/systematic.rs @@ -110,7 +110,7 @@ impl FetchSystematicChunks { .expect("validator count should not exceed u32"), ), ) - .map(|(_, chunk)| &chunk.chunk[..]) + .map(|(_, chunk)| chunk.chunk.clone()) .collect::>(); let available_data = polkadot_erasure_coding::reconstruct_from_systematic_v1( diff --git a/polkadot/runtime/parachains/src/configuration/migration/v10.rs b/polkadot/runtime/parachains/src/configuration/migration/v10.rs index 944d9715ef7e..cf228610e5c9 100644 --- a/polkadot/runtime/parachains/src/configuration/migration/v10.rs +++ b/polkadot/runtime/parachains/src/configuration/migration/v10.rs @@ -17,11 +17,7 @@ //! A module that is responsible for migration of storage. use crate::configuration::{Config, Pallet}; -use frame_support::{ - pallet_prelude::*, - traits::{Defensive, OnRuntimeUpgrade}, - weights::Weight, -}; +use frame_support::{pallet_prelude::*, traits::Defensive, weights::Weight}; use frame_system::pallet_prelude::BlockNumberFor; use primitives::{ vstaging::NodeFeatures, AsyncBackingParams, Balance, ExecutorParams, SessionIndex, @@ -30,8 +26,7 @@ use primitives::{ use sp_runtime::Perbill; use sp_std::vec::Vec; -#[cfg(feature = "try-runtime")] -use frame_support::traits::StorageVersion; +use frame_support::traits::OnRuntimeUpgrade; use super::v9::V9HostConfiguration; // All configuration of the runtime with respect to paras. From d25cc1cf3463bd49c2e54174a1560418de975a52 Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 9 Feb 2024 16:19:32 +0200 Subject: [PATCH 101/126] try fixing zombienet tests --- ...-chunk-fetching-network-compatibility.toml | 46 +++++++++++++++++++ ...chunk-fetching-network-compatibility.zndsl | 2 +- .../0013-enable-chunk-mapping-feature.js | 37 +++++++++++++++ .../0013-systematic-chunk-recovery.toml | 8 ++-- .../0013-systematic-chunk-recovery.zndsl | 28 +++++++++-- 5 files changed, 112 insertions(+), 9 deletions(-) create mode 100644 polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.toml create mode 100644 polkadot/zombienet_tests/functional/0013-enable-chunk-mapping-feature.js diff --git a/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.toml b/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.toml new file mode 100644 index 000000000000..3683a0d84fc1 --- /dev/null +++ b/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.toml @@ -0,0 +1,46 @@ +[settings] +timeout = 1000 +bootnode = true + +[relaychain.genesis.runtimeGenesis.patch.configuration.config] + max_validators_per_core = 2 + needed_approvals = 4 + +[relaychain] +default_image = "{{ZOMBIENET_INTEGRATION_TEST_IMAGE}}" +chain = "rococo-local" +default_command = "polkadot" + +[relaychain.default_resources] +limits = { memory = "4G", cpu = "2" } +requests = { memory = "2G", cpu = "1" } + + [[relaychain.node_groups]] + # Use an image that doesn't speak /req_chunk/2 protocol. + image = "{{PARACHAINS_IMAGE_NAME}}:v1.0.0" + name = "old" + count = 2 + args = ["-lparachain=debug,parachain::availability-recovery=trace,parachain::availability-distribution=trace"] + + [[relaychain.node_groups]] + name = "new" + count = 2 + args = ["-lparachain=debug,parachain::availability-recovery=trace,parachain::availability-distribution=trace,sub-libp2p=trace"] + +{% for id in range(2000,2002) %} +[[parachains]] +id = {{id}} +addToGenesis = true +cumulus_based = true +chain = "glutton-westend-local-{{id}}" + [parachains.genesis.runtimeGenesis.patch.glutton] + compute = "50000000" + storage = "1310000000" + trashDataCount = 5120 + + [parachains.collator] + name = "collator" + image = "{{COL_IMAGE}}" + args = ["-lparachain=debug"] + +{% endfor %} diff --git a/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.zndsl b/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.zndsl index 1665d5755226..00e976991c9d 100644 --- a/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.zndsl +++ b/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.zndsl @@ -1,5 +1,5 @@ Description: Validators preserve backwards compatibility with peers speaking an older version of the /req_chunk protocol -Network: ./0011-chunk-fetching-network-compatibility.toml +Network: ./0012-chunk-fetching-network-compatibility.toml Creds: config # Check authority status. diff --git a/polkadot/zombienet_tests/functional/0013-enable-chunk-mapping-feature.js b/polkadot/zombienet_tests/functional/0013-enable-chunk-mapping-feature.js new file mode 100644 index 000000000000..89632145c1c8 --- /dev/null +++ b/polkadot/zombienet_tests/functional/0013-enable-chunk-mapping-feature.js @@ -0,0 +1,37 @@ +async function run(nodeName, networkInfo, _jsArgs) { + const { wsUri, userDefinedTypes } = networkInfo.nodesByName[nodeName]; + const api = await zombie.connect(wsUri, userDefinedTypes); + + await zombie.util.cryptoWaitReady(); + + // account to submit tx + const keyring = new zombie.Keyring({ type: "sr25519" }); + const alice = keyring.addFromUri("//Alice"); + + await new Promise(async (resolve, reject) => { + const unsub = await api.tx.sudo + .sudo(api.tx.configuration.setNodeFeature(1, true)) + .signAndSend(alice, ({ status, isError }) => { + if (status.isInBlock) { + console.log( + `Transaction included at blockhash ${status.asInBlock}`, + ); + } else if (status.isFinalized) { + console.log( + `Transaction finalized at blockHash ${status.asFinalized}`, + ); + unsub(); + return resolve(); + } else if (isError) { + console.log(`Transaction error`); + reject(`Transaction error`); + } + }); + }); + + + + return 0; +} + +module.exports = { run }; diff --git a/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.toml b/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.toml index 8f6ebb6d9a01..f7983531dffb 100644 --- a/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.toml +++ b/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.toml @@ -5,8 +5,6 @@ bootnode = true [relaychain.genesis.runtimeGenesis.patch.configuration.config] max_validators_per_core = 5 needed_approvals = 10 - # TODO: this test won't pass for now, we don't yet have support for modifying a bitvec here - # node_features = "1" [relaychain] default_image = "{{ZOMBIENET_INTEGRATION_TEST_IMAGE}}" @@ -17,9 +15,13 @@ default_command = "polkadot" limits = { memory = "4G", cpu = "2" } requests = { memory = "2G", cpu = "1" } + [[relaychain.nodes]] + name = "alice" + validator = "true" + [[relaychain.node_groups]] name = "validator" - count = 10 + count = 9 args = ["-lparachain=debug,parachain::availability-recovery=trace,parachain::availability-distribution=trace"] {% for id in range(2000,2002) %} diff --git a/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.zndsl b/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.zndsl index 245a29650f62..481f13125eca 100644 --- a/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.zndsl +++ b/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.zndsl @@ -1,9 +1,9 @@ Description: Systematic chunk recovery is used if the chunk mapping feature is enabled. -Network: ./0012-systematic-chunk-recovery.toml +Network: ./0013-systematic-chunk-recovery.toml Creds: config # Check authority status. -validator: reports node_roles is 10 +#validator: reports node_roles is 10 within 120 seconds # Ensure parachains are registered. validator: parachain 2000 is registered within 60 seconds @@ -19,9 +19,27 @@ validator: reports polkadot_parachain_approval_checking_finality_lag < 3 validator: reports polkadot_parachain_approvals_no_shows_total < 3 within 100 seconds -# Ensure we used systematic recovery -validator: log line contains "Data recovery from systematic chunks complete" within 100 seconds -validator: count of log lines containing "Data recovery from systematic chunks is not possible" is 0 within 10 seconds +# Ensure that there are no failed recoveries. +validator: reports polkadot_parachain_availability_recovery_recoveries_finished{result="failure"} is 0 within 10 seconds +validator: reports polkadot_parachain_availability_recovery_recoveries_finished{result="success"} is 10 within 600 seconds + +# Ensure we used regular chunk recovery +validator: log line contains "Data recovery from chunks complete" within 100 seconds +validator: count of log lines containing "Data recovery from chunks is not possible" is 0 within 10 seconds + +# Enable the chunk mapping feature +alice: js-script ./0013-enable-chunk-mapping-feature.js return is 0 within 600 seconds + +validator: reports substrate_block_height{status="finalized"} is at least 60 within 400 seconds # Ensure that there are no failed recoveries. validator: reports polkadot_parachain_availability_recovery_recoveries_finished{result="failure"} is 0 within 10 seconds +validator: reports polkadot_parachain_availability_recovery_recoveries_finished{result="success"} is 20 within 600 seconds + +# Ensure we used systematic chunk recovery +validator: log line contains "Data recovery from systematic chunks complete" within 500 seconds +validator: count of log lines containing "Data recovery from systematic chunks is not possible" is 0 within 10 seconds + +validator: reports polkadot_parachain_approval_checking_finality_lag < 3 + +validator: reports polkadot_parachain_approvals_no_shows_total < 3 within 100 seconds From dbc27b3ac9e31dbda622b615456267bf68d071fc Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 9 Feb 2024 16:39:27 +0200 Subject: [PATCH 102/126] more zombienet --- .../0012-chunk-fetching-network-compatibility.zndsl | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.zndsl b/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.zndsl index 00e976991c9d..e349e7e8f833 100644 --- a/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.zndsl +++ b/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.zndsl @@ -32,12 +32,20 @@ old: reports polkadot_parachain_approvals_no_shows_total < 3 within 10 seconds new: log line contains "Trying the fallback protocol" within 100 seconds # Ensure that there are no failed recoveries. +new: count of log lines containing "Data recovery from chunks is complete" is at least 10 within 300 seconds +old: count of log lines containing "Data recovery from chunks is complete" is at least 10 within 300 seconds +new: count of log lines containing "Data recovery from chunks is not possible" is 0 within 10 seconds +old: count of log lines containing "Data recovery from chunks is not possible" is 0 within 10 seconds new: reports polkadot_parachain_availability_recovery_recoveries_finished{result="failure"} is 0 within 10 seconds old: reports polkadot_parachain_availability_recovery_recoveries_finished{result="failure"} is 0 within 10 seconds +# Ensure systematic recovery was not used. +old: count of log lines containing "Data recovery from systematic chunks is complete" is 0 within 10 seconds +new: count of log lines containing "Data recovery from systematic chunks is complete" is 0 within 10 seconds + # Ensure availability-distribution worked fine -new: reports polkadot_parachain_fetched_chunks_total{success="succeeded"} is at least 10 within 200 seconds -old: reports polkadot_parachain_fetched_chunks_total{success="succeeded"} is at least 10 within 200 seconds +new: reports polkadot_parachain_fetched_chunks_total{success="succeeded"} is at least 10 within 400 seconds +old: reports polkadot_parachain_fetched_chunks_total{success="succeeded"} is at least 10 within 400 seconds new: reports polkadot_parachain_fetched_chunks_total{success="failed"} is 0 within 10 seconds old: reports polkadot_parachain_fetched_chunks_total{success="failed"} is 0 within 10 seconds From 6aa3b164a6e1a61363fb728ced92200c33487151 Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 12 Feb 2024 11:10:19 +0200 Subject: [PATCH 103/126] zombienet --- ...chunk-fetching-network-compatibility.zndsl | 15 +++++----- .../0013-systematic-chunk-recovery.toml | 6 ++-- .../0013-systematic-chunk-recovery.zndsl | 28 +++++++++---------- 3 files changed, 23 insertions(+), 26 deletions(-) diff --git a/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.zndsl b/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.zndsl index e349e7e8f833..7bb24aeb66a5 100644 --- a/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.zndsl +++ b/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.zndsl @@ -27,21 +27,20 @@ old: reports polkadot_parachain_approval_checking_finality_lag < 3 new: reports polkadot_parachain_approvals_no_shows_total < 3 within 10 seconds old: reports polkadot_parachain_approvals_no_shows_total < 3 within 10 seconds - -# Ensure we used the fallback network request. -new: log line contains "Trying the fallback protocol" within 100 seconds - # Ensure that there are no failed recoveries. -new: count of log lines containing "Data recovery from chunks is complete" is at least 10 within 300 seconds -old: count of log lines containing "Data recovery from chunks is complete" is at least 10 within 300 seconds +new: count of log lines containing "Data recovery from chunks complete" is at least 10 within 300 seconds +old: count of log lines containing "Data recovery from chunks complete" is at least 10 within 300 seconds new: count of log lines containing "Data recovery from chunks is not possible" is 0 within 10 seconds old: count of log lines containing "Data recovery from chunks is not possible" is 0 within 10 seconds new: reports polkadot_parachain_availability_recovery_recoveries_finished{result="failure"} is 0 within 10 seconds old: reports polkadot_parachain_availability_recovery_recoveries_finished{result="failure"} is 0 within 10 seconds +# Ensure we used the fallback network request. +new: log line contains "Trying the fallback protocol" within 100 seconds + # Ensure systematic recovery was not used. -old: count of log lines containing "Data recovery from systematic chunks is complete" is 0 within 10 seconds -new: count of log lines containing "Data recovery from systematic chunks is complete" is 0 within 10 seconds +old: count of log lines containing "Data recovery from systematic chunks complete" is 0 within 10 seconds +new: count of log lines containing "Data recovery from systematic chunks complete" is 0 within 10 seconds # Ensure availability-distribution worked fine new: reports polkadot_parachain_fetched_chunks_total{success="succeeded"} is at least 10 within 400 seconds diff --git a/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.toml b/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.toml index f7983531dffb..f069680a7400 100644 --- a/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.toml +++ b/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.toml @@ -3,8 +3,8 @@ timeout = 1000 bootnode = true [relaychain.genesis.runtimeGenesis.patch.configuration.config] - max_validators_per_core = 5 - needed_approvals = 10 + max_validators_per_core = 2 + needed_approvals = 4 [relaychain] default_image = "{{ZOMBIENET_INTEGRATION_TEST_IMAGE}}" @@ -21,7 +21,7 @@ requests = { memory = "2G", cpu = "1" } [[relaychain.node_groups]] name = "validator" - count = 9 + count = 3 args = ["-lparachain=debug,parachain::availability-recovery=trace,parachain::availability-distribution=trace"] {% for id in range(2000,2002) %} diff --git a/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.zndsl b/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.zndsl index 481f13125eca..8991be61d8c0 100644 --- a/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.zndsl +++ b/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.zndsl @@ -3,7 +3,8 @@ Network: ./0013-systematic-chunk-recovery.toml Creds: config # Check authority status. -#validator: reports node_roles is 10 within 120 seconds +alice: reports node_roles is 4 +validator: reports node_roles is 4 # Ensure parachains are registered. validator: parachain 2000 is registered within 60 seconds @@ -19,27 +20,24 @@ validator: reports polkadot_parachain_approval_checking_finality_lag < 3 validator: reports polkadot_parachain_approvals_no_shows_total < 3 within 100 seconds -# Ensure that there are no failed recoveries. -validator: reports polkadot_parachain_availability_recovery_recoveries_finished{result="failure"} is 0 within 10 seconds -validator: reports polkadot_parachain_availability_recovery_recoveries_finished{result="success"} is 10 within 600 seconds - -# Ensure we used regular chunk recovery -validator: log line contains "Data recovery from chunks complete" within 100 seconds +# Ensure we used regular chunk recovery and that there are no failed recoveries. +validator: count of log lines containing "Data recovery from chunks complete" is at least 10 within 300 seconds +validator: count of log lines containing "Data recovery from systematic chunks complete" is 0 within 10 seconds +validator: count of log lines containing "Data recovery from systematic chunks is not possible" is 0 within 10 seconds validator: count of log lines containing "Data recovery from chunks is not possible" is 0 within 10 seconds +validator: reports polkadot_parachain_availability_recovery_recoveries_finished{result="failure"} is 0 within 10 seconds # Enable the chunk mapping feature alice: js-script ./0013-enable-chunk-mapping-feature.js return is 0 within 600 seconds validator: reports substrate_block_height{status="finalized"} is at least 60 within 400 seconds -# Ensure that there are no failed recoveries. -validator: reports polkadot_parachain_availability_recovery_recoveries_finished{result="failure"} is 0 within 10 seconds -validator: reports polkadot_parachain_availability_recovery_recoveries_finished{result="success"} is 20 within 600 seconds - -# Ensure we used systematic chunk recovery -validator: log line contains "Data recovery from systematic chunks complete" within 500 seconds -validator: count of log lines containing "Data recovery from systematic chunks is not possible" is 0 within 10 seconds - validator: reports polkadot_parachain_approval_checking_finality_lag < 3 validator: reports polkadot_parachain_approvals_no_shows_total < 3 within 100 seconds + +# Ensure we used systematic chunk recovery and that there are no failed recoveries. +validator: count of log lines containing "Data recovery from systematic chunks complete" is at least 10 within 300 seconds +validator: count of log lines containing "Data recovery from systematic chunks is not possible" is 0 within 10 seconds +validator: count of log lines containing "Data recovery from chunks is not possible" is 0 within 10 seconds +validator: reports polkadot_parachain_availability_recovery_recoveries_finished{result="failure"} is 0 within 10 seconds From 2316316f62b95891255d731e725a47cb470b8178 Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 12 Feb 2024 11:42:18 +0200 Subject: [PATCH 104/126] fix yaml formatting --- .gitlab/pipeline/zombienet/polkadot.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.gitlab/pipeline/zombienet/polkadot.yml b/.gitlab/pipeline/zombienet/polkadot.yml index 0101e0d733cb..4bfb9c7d8391 100644 --- a/.gitlab/pipeline/zombienet/polkadot.yml +++ b/.gitlab/pipeline/zombienet/polkadot.yml @@ -160,9 +160,9 @@ zombienet-polkadot-functional-0011-async-backing-6-seconds-rate: zombienet-polkadot-functional-0012-chunk-fetching-network-compatibility: extends: - - .zombienet-polkadot-common - script: - - /home/nonroot/zombie-net/scripts/ci/run-test-local-env-manager.sh + - .zombienet-polkadot-common + script: + - /home/nonroot/zombie-net/scripts/ci/run-test-local-env-manager.sh --local-dir="${LOCAL_DIR}/functional" --test="0012-chunk-fetching-network-compatibility.zndsl" From df8a096053350938b94f3afddcc6e95e4284e4e9 Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 12 Feb 2024 12:11:54 +0200 Subject: [PATCH 105/126] fix import --- polkadot/node/network/availability-recovery/src/task/mod.rs | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/task/mod.rs b/polkadot/node/network/availability-recovery/src/task/mod.rs index a1c4e481c868..800a82947d6f 100644 --- a/polkadot/node/network/availability-recovery/src/task/mod.rs +++ b/polkadot/node/network/availability-recovery/src/task/mod.rs @@ -22,10 +22,12 @@ mod strategy; pub use self::strategy::{ FetchChunks, FetchChunksParams, FetchFull, FetchFullParams, FetchSystematicChunks, - FetchSystematicChunksParams, RecoveryStrategy, State, REGULAR_CHUNKS_REQ_RETRY_LIMIT, - SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT, + FetchSystematicChunksParams, RecoveryStrategy, State, }; +#[cfg(test)] +pub use self::strategy::{REGULAR_CHUNKS_REQ_RETRY_LIMIT, SYSTEMATIC_CHUNKS_REQ_RETRY_LIMIT}; + use crate::{metrics::Metrics, ErasureTask, PostRecoveryCheck, LOG_TARGET}; use parity_scale_codec::Encode; From 486cf6483a2b69fe01c9c2bc9b83d470f1c4fdde Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 12 Feb 2024 12:21:37 +0200 Subject: [PATCH 106/126] fix bench --- polkadot/erasure-coding/benches/scaling_with_validators.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/polkadot/erasure-coding/benches/scaling_with_validators.rs b/polkadot/erasure-coding/benches/scaling_with_validators.rs index 0c30f3e0c02b..3d743faa4169 100644 --- a/polkadot/erasure-coding/benches/scaling_with_validators.rs +++ b/polkadot/erasure-coding/benches/scaling_with_validators.rs @@ -83,9 +83,8 @@ fn construct_and_reconstruct_5mb_pov(c: &mut Criterion) { let all_chunks = chunks(n_validators, &pov); let chunks = all_chunks - .iter() + .into_iter() .take(polkadot_erasure_coding::systematic_recovery_threshold(n_validators).unwrap()) - .map(|c| &c[..]) .collect::>(); group.throughput(Throughput::Bytes(pov.len() as u64)); From 70e8840802ed8cee23bbcaf0a717bc1af22b5d05 Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 12 Feb 2024 12:38:41 +0200 Subject: [PATCH 107/126] clippy --- polkadot/node/subsystem-bench/src/core/mock/av_store.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/polkadot/node/subsystem-bench/src/core/mock/av_store.rs b/polkadot/node/subsystem-bench/src/core/mock/av_store.rs index 57ed6f5742ed..619e15a74e20 100644 --- a/polkadot/node/subsystem-bench/src/core/mock/av_store.rs +++ b/polkadot/node/subsystem-bench/src/core/mock/av_store.rs @@ -238,7 +238,7 @@ impl MockAvailabilityStore { .chunks .get(*candidate_index) .unwrap() - .get(0) + .first() .unwrap() .encoded_size(); let _ = tx.send(Some(chunk_size)); From ee93f681fbdbfca814f6069255e25259b94cfba0 Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 12 Feb 2024 14:44:11 +0200 Subject: [PATCH 108/126] fix image name --- .../functional/0012-chunk-fetching-network-compatibility.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.toml b/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.toml index 3683a0d84fc1..f06d66136c53 100644 --- a/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.toml +++ b/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.toml @@ -17,7 +17,7 @@ requests = { memory = "2G", cpu = "1" } [[relaychain.node_groups]] # Use an image that doesn't speak /req_chunk/2 protocol. - image = "{{PARACHAINS_IMAGE_NAME}}:v1.0.0" + image = "{{POLKADOT_IMAGE}}:v1.0.0" name = "old" count = 2 args = ["-lparachain=debug,parachain::availability-recovery=trace,parachain::availability-distribution=trace"] From 7d9e21aa81738cb88c26c5d7d296388e11bf1241 Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 12 Feb 2024 17:34:23 +0200 Subject: [PATCH 109/126] try another image for the zombienet test --- .../functional/0012-chunk-fetching-network-compatibility.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.toml b/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.toml index f06d66136c53..9e3b48ed491a 100644 --- a/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.toml +++ b/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.toml @@ -17,7 +17,7 @@ requests = { memory = "2G", cpu = "1" } [[relaychain.node_groups]] # Use an image that doesn't speak /req_chunk/2 protocol. - image = "{{POLKADOT_IMAGE}}:v1.0.0" + image = "{{POLKADOT_IMAGE}}:master-bde0bbe5" name = "old" count = 2 args = ["-lparachain=debug,parachain::availability-recovery=trace,parachain::availability-distribution=trace"] From 9000f5c9230ea214d1043f644dd4e2f58ab90e37 Mon Sep 17 00:00:00 2001 From: Javier Viola Date: Tue, 13 Feb 2024 10:21:57 -0300 Subject: [PATCH 110/126] Use cumulus image for collator --- .../functional/0012-chunk-fetching-network-compatibility.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.toml b/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.toml index 9e3b48ed491a..81298fa660b2 100644 --- a/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.toml +++ b/polkadot/zombienet_tests/functional/0012-chunk-fetching-network-compatibility.toml @@ -40,7 +40,7 @@ chain = "glutton-westend-local-{{id}}" [parachains.collator] name = "collator" - image = "{{COL_IMAGE}}" + image = "{{CUMULUS_IMAGE}}" args = ["-lparachain=debug"] {% endfor %} From eeab22ae055a11bf011af60570b059d648d4f286 Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 27 Feb 2024 09:28:33 +0200 Subject: [PATCH 111/126] fix yaml format --- .gitlab/pipeline/zombienet/polkadot.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.gitlab/pipeline/zombienet/polkadot.yml b/.gitlab/pipeline/zombienet/polkadot.yml index 544d81734aff..aab699104dd1 100644 --- a/.gitlab/pipeline/zombienet/polkadot.yml +++ b/.gitlab/pipeline/zombienet/polkadot.yml @@ -175,7 +175,7 @@ zombienet-polkadot-functional-0013-systematic-chunk-recovery: --test="0013-systematic-chunk-recovery.zndsl" zombienet-polkadot-functional-0014-chunk-fetching-network-compatibility: -extends: + extends: - .zombienet-polkadot-common script: - /home/nonroot/zombie-net/scripts/ci/run-test-local-env-manager.sh From a57f5fddd0e55d3160324bbd7109e8a1503041a6 Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 11 Mar 2024 17:15:06 +0200 Subject: [PATCH 112/126] fix merge commit --- polkadot/node/core/approval-voting/src/lib.rs | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/polkadot/node/core/approval-voting/src/lib.rs b/polkadot/node/core/approval-voting/src/lib.rs index a0a5cc12a353..599dd8056079 100644 --- a/polkadot/node/core/approval-voting/src/lib.rs +++ b/polkadot/node/core/approval-voting/src/lib.rs @@ -1446,6 +1446,13 @@ async fn distribution_messages_for_activation( None => continue, }; + let core_index = block_entry + .candidates() + .iter() + .find_map(|(core_index, h)| { + (h == candidate_hash).then_some(*core_index) + }); + actions.push(Action::LaunchApproval { claimed_candidate_indices: bitfield, candidate_hash: candidate_entry @@ -1461,6 +1468,7 @@ async fn distribution_messages_for_activation( .clone(), backing_group: approval_entry.backing_group(), distribute_assignment: false, + core_index, }); } }, From 6948ea1291ab92764fade265366987b42b48945f Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 11 Mar 2024 17:51:03 +0200 Subject: [PATCH 113/126] fix bench --- .../tests/availability-recovery-regression-bench.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/polkadot/node/network/availability-recovery/tests/availability-recovery-regression-bench.rs b/polkadot/node/network/availability-recovery/tests/availability-recovery-regression-bench.rs index beb063e7ae0d..b98a6d7e4384 100644 --- a/polkadot/node/network/availability-recovery/tests/availability-recovery-regression-bench.rs +++ b/polkadot/node/network/availability-recovery/tests/availability-recovery-regression-bench.rs @@ -23,7 +23,7 @@ use polkadot_subsystem_bench::{ availability::{ - benchmark_availability_read, prepare_test, DataAvailabilityReadOptions, + benchmark_availability_read, prepare_test, DataAvailabilityReadOptions, Strategy, TestDataAvailability, TestState, }, configuration::{PeerLatency, TestConfiguration}, @@ -38,7 +38,7 @@ fn main() -> Result<(), String> { let mut messages = vec![]; // TODO: Adjust the test configurations to Kusama values - let options = DataAvailabilityReadOptions { fetch_from_backers: true }; + let options = DataAvailabilityReadOptions { strategy: Strategy::Chunks }; let mut config = TestConfiguration::default(); config.latency = Some(PeerLatency { mean_latency_ms: 100, std_dev: 1.0 }); config.n_validators = 300; From da66e8893f0f0db1715ba43fb212e27e94278a7e Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 22 Mar 2024 13:58:42 +0200 Subject: [PATCH 114/126] re-add enable-node-feature script --- .../functional/0013-enable-node-feature.js | 35 +++++++++++++++++++ .../0013-systematic-chunk-recovery.zndsl | 2 +- 2 files changed, 36 insertions(+), 1 deletion(-) create mode 100644 polkadot/zombienet_tests/functional/0013-enable-node-feature.js diff --git a/polkadot/zombienet_tests/functional/0013-enable-node-feature.js b/polkadot/zombienet_tests/functional/0013-enable-node-feature.js new file mode 100644 index 000000000000..5fe2e38dad7d --- /dev/null +++ b/polkadot/zombienet_tests/functional/0013-enable-node-feature.js @@ -0,0 +1,35 @@ +async function run(nodeName, networkInfo, index) { + const { wsUri, userDefinedTypes } = networkInfo.nodesByName[nodeName]; + const api = await zombie.connect(wsUri, userDefinedTypes); + + await zombie.util.cryptoWaitReady(); + + // account to submit tx + const keyring = new zombie.Keyring({ type: "sr25519" }); + const alice = keyring.addFromUri("//Alice"); + + await new Promise(async (resolve, reject) => { + const unsub = await api.tx.sudo + .sudo(api.tx.configuration.setNodeFeature(Number(index), true)) + .signAndSend(alice, ({ status, isError }) => { + if (status.isInBlock) { + console.log( + `Transaction included at blockhash ${status.asInBlock}`, + ); + } else if (status.isFinalized) { + console.log( + `Transaction finalized at blockHash ${status.asFinalized}`, + ); + unsub(); + return resolve(); + } else if (isError) { + console.log(`Transaction error`); + reject(`Transaction error`); + } + }); + }); + + return 0; +} + +module.exports = { run }; diff --git a/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.zndsl b/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.zndsl index 73a5d45e252d..e9e5a429e2a2 100644 --- a/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.zndsl +++ b/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.zndsl @@ -28,7 +28,7 @@ validator: count of log lines containing "Data recovery from chunks is not possi validator: reports polkadot_parachain_availability_recovery_recoveries_finished{result="failure"} is 0 within 10 seconds # Enable the chunk mapping feature -alice: js-script ./0012-enable-node-feature.js with "2" return is 0 within 600 seconds +alice: js-script ./0013-enable-node-feature.js with "2" return is 0 within 600 seconds validator: reports substrate_block_height{status="finalized"} is at least 60 within 400 seconds From d20ced9e318be749593f2b57693975a78b1920eb Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 22 Mar 2024 14:09:42 +0200 Subject: [PATCH 115/126] fix zombienet tests --- .../functional/0013-systematic-chunk-recovery.toml | 4 +++- .../functional/0014-chunk-fetching-network-compatibility.toml | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.toml b/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.toml index f069680a7400..281e63a0466c 100644 --- a/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.toml +++ b/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.toml @@ -2,8 +2,10 @@ timeout = 1000 bootnode = true -[relaychain.genesis.runtimeGenesis.patch.configuration.config] +[relaychain.genesis.runtimeGenesis.patch.configuration.config.scheduler_params] max_validators_per_core = 2 + +[relaychain.genesis.runtimeGenesis.patch.configuration.config] needed_approvals = 4 [relaychain] diff --git a/polkadot/zombienet_tests/functional/0014-chunk-fetching-network-compatibility.toml b/polkadot/zombienet_tests/functional/0014-chunk-fetching-network-compatibility.toml index 81298fa660b2..8924102bb2b5 100644 --- a/polkadot/zombienet_tests/functional/0014-chunk-fetching-network-compatibility.toml +++ b/polkadot/zombienet_tests/functional/0014-chunk-fetching-network-compatibility.toml @@ -2,8 +2,10 @@ timeout = 1000 bootnode = true -[relaychain.genesis.runtimeGenesis.patch.configuration.config] +[relaychain.genesis.runtimeGenesis.patch.configuration.config.scheduler_params] max_validators_per_core = 2 + +[relaychain.genesis.runtimeGenesis.patch.configuration.config] needed_approvals = 4 [relaychain] From 78a7959ddd790f4ce1030bd62d8612a969696ba9 Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 10 May 2024 16:49:48 +0300 Subject: [PATCH 116/126] add semver to prdoc --- prdoc/pr_1644.prdoc | 44 +++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 43 insertions(+), 1 deletion(-) diff --git a/prdoc/pr_1644.prdoc b/prdoc/pr_1644.prdoc index 456ee5216dcc..f507371bef36 100644 --- a/prdoc/pr_1644.prdoc +++ b/prdoc/pr_1644.prdoc @@ -6,4 +6,46 @@ doc: Implements https://github.com/polkadot-fellows/RFCs/pull/47 and adds the logic for availability recovery from systematic chunks. The /req_chunk/1 req-response protocol is now considered deprecated in favour of /req_chunk/2. -crates: [] +crates: + - name: sc-network + bump: minor + - name: polkadot-primitives + bump: minor + - name: cumulus-client-pov-recovery + bump: None + - name: polkadot-overseer + bump: None + - name: polkadot-node-primitives + bump: major + - name: polkadot-erasure-coding + bump: major + - name: polkadot-node-jaeger + bump: major + - name: polkadot-node-subsystem-types + bump: major + - name: polkadot-node-network-protocol + bump: major + - name: polkadot-service + bump: major + - name: polkadot-node-subsystem-util + bump: major + - name: polkadot-availability-distribution + bump: major + - name: polkadot-availability-recovery + bump: major + - name: polkadot-node-core-approval-voting + bump: minor + - name: polkadot-node-core-av-store + bump: major + - name: polkadot-network-bridge + bump: minor + - name: polkadot-node-core-backing + bump: None + - name: polkadot-node-core-bitfield-signing + bump: None + - name: polkadot-node-core-dispute-coordinator + bump: None + - name: cumulus-relay-chain-minimal-node + bump: minor + - name: polkadot + bump: minor From da8ca2267a405cb1fa63ae175c211835d39ab8bf Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 10 May 2024 16:53:41 +0300 Subject: [PATCH 117/126] try fixing prdoc --- prdoc/pr_1644.prdoc | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/prdoc/pr_1644.prdoc b/prdoc/pr_1644.prdoc index f507371bef36..58ee02c5ef76 100644 --- a/prdoc/pr_1644.prdoc +++ b/prdoc/pr_1644.prdoc @@ -12,9 +12,9 @@ crates: - name: polkadot-primitives bump: minor - name: cumulus-client-pov-recovery - bump: None + bump: none - name: polkadot-overseer - bump: None + bump: none - name: polkadot-node-primitives bump: major - name: polkadot-erasure-coding @@ -40,11 +40,11 @@ crates: - name: polkadot-network-bridge bump: minor - name: polkadot-node-core-backing - bump: None + bump: none - name: polkadot-node-core-bitfield-signing - bump: None + bump: none - name: polkadot-node-core-dispute-coordinator - bump: None + bump: none - name: cumulus-relay-chain-minimal-node bump: minor - name: polkadot From 345d89682412f8dd59ab4d27b4bd6bbd349c47e0 Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 21 May 2024 09:55:37 +0300 Subject: [PATCH 118/126] update lockfile --- Cargo.lock | 77 ++++++++++++++++++++++++++++++++++++++++-------------- 1 file changed, 57 insertions(+), 20 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 3ec45f4ac0e2..885d6b5d538b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5299,6 +5299,18 @@ dependencies = [ "futures", ] +[[package]] +name = "expander" +version = "0.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a718c0675c555c5f976fff4ea9e2c150fa06cefa201cadef87cfbf9324075881" +dependencies = [ + "blake3", + "fs-err", + "proc-macro2 1.0.82", + "quote 1.0.35", +] + [[package]] name = "expander" version = "2.0.0" @@ -5360,13 +5372,38 @@ dependencies = [ "bytes", ] +[[package]] +name = "fatality" +version = "0.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2ad875162843b0d046276327afe0136e9ed3a23d5a754210fb6f1f33610d39ab" +dependencies = [ + "fatality-proc-macro 0.0.6", + "thiserror", +] + [[package]] name = "fatality" version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ec6f82451ff7f0568c6181287189126d492b5654e30a788add08027b6363d019" dependencies = [ - "fatality-proc-macro", + "fatality-proc-macro 0.1.1", + "thiserror", +] + +[[package]] +name = "fatality-proc-macro" +version = "0.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f5aa1e3ae159e592ad222dc90c5acbad632b527779ba88486abe92782ab268bd" +dependencies = [ + "expander 0.0.4", + "indexmap 1.9.3", + "proc-macro-crate 1.3.1", + "proc-macro2 1.0.82", + "quote 1.0.35", + "syn 1.0.109", "thiserror", ] @@ -5376,7 +5413,7 @@ version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "eb42427514b063d97ce21d5199f36c0c307d981434a6be32582bc79fe5bd2303" dependencies = [ - "expander", + "expander 2.0.0", "indexmap 2.2.3", "proc-macro-crate 3.1.0", "proc-macro2 1.0.82", @@ -5870,7 +5907,7 @@ dependencies = [ "Inflector", "cfg-expr", "derive-syn-parse 0.2.0", - "expander", + "expander 2.0.0", "frame-support-procedural-tools", "itertools 0.11.0", "macro_magic", @@ -9343,7 +9380,7 @@ version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1344346d5af32c95bbddea91b18a88cc83eac394192d20ef2fc4c40a74332355" dependencies = [ - "expander", + "expander 2.0.0", "indexmap 2.2.3", "itertools 0.11.0", "petgraph", @@ -12633,7 +12670,7 @@ version = "7.0.0" dependencies = [ "assert_matches", "derive_more", - "fatality", + "fatality 0.1.1", "futures", "futures-timer", "parity-scale-codec", @@ -12664,7 +12701,7 @@ version = "7.0.0" dependencies = [ "assert_matches", "async-trait", - "fatality", + "fatality 0.1.1", "futures", "futures-timer", "log", @@ -12727,7 +12764,7 @@ dependencies = [ "assert_matches", "bitvec", "env_logger 0.11.3", - "fatality", + "fatality 0.1.1", "futures", "futures-timer", "log", @@ -12770,7 +12807,7 @@ dependencies = [ "async-channel", "async-trait", "derive_more", - "fatality", + "fatality 0.1.1", "futures", "futures-timer", "indexmap 2.2.3", @@ -12849,7 +12886,7 @@ dependencies = [ "assert_matches", "async-trait", "bytes", - "fatality", + "fatality 0.1.1", "futures", "futures-timer", "parity-scale-codec", @@ -12973,7 +13010,7 @@ version = "7.0.0" dependencies = [ "assert_matches", "bitvec", - "fatality", + "fatality 0.1.1", "futures", "polkadot-erasure-coding", "polkadot-node-primitives", @@ -13082,7 +13119,7 @@ name = "polkadot-node-core-dispute-coordinator" version = "7.0.0" dependencies = [ "assert_matches", - "fatality", + "fatality 0.1.1", "futures", "futures-timer", "kvdb", @@ -13127,7 +13164,7 @@ version = "6.0.0" dependencies = [ "assert_matches", "bitvec", - "fatality", + "fatality 0.1.1", "futures", "parity-scale-codec", "polkadot-node-primitives", @@ -13151,7 +13188,7 @@ name = "polkadot-node-core-provisioner" version = "7.0.0" dependencies = [ "bitvec", - "fatality", + "fatality 0.1.1", "futures", "futures-timer", "polkadot-node-primitives", @@ -13372,7 +13409,7 @@ dependencies = [ "async-trait", "bitvec", "derive_more", - "fatality", + "fatality 0.1.1", "futures", "hex", "parity-scale-codec", @@ -13450,7 +13487,7 @@ dependencies = [ "async-trait", "bitvec", "derive_more", - "fatality", + "fatality 0.0.6", "futures", "orchestra", "polkadot-node-jaeger", @@ -13480,7 +13517,7 @@ dependencies = [ "async-trait", "derive_more", "env_logger 0.11.3", - "fatality", + "fatality 0.1.1", "futures", "futures-channel", "itertools 0.11.0", @@ -14090,7 +14127,7 @@ dependencies = [ "assert_matches", "async-channel", "bitvec", - "fatality", + "fatality 0.1.1", "futures", "futures-timer", "indexmap 2.2.3", @@ -19034,7 +19071,7 @@ dependencies = [ "Inflector", "assert_matches", "blake2 0.10.6", - "expander", + "expander 2.0.0", "proc-macro-crate 3.1.0", "proc-macro2 1.0.82", "quote 1.0.35", @@ -19729,7 +19766,7 @@ name = "sp-runtime-interface-proc-macro" version = "17.0.0" dependencies = [ "Inflector", - "expander", + "expander 2.0.0", "proc-macro-crate 3.1.0", "proc-macro2 1.0.82", "quote 1.0.35", @@ -21685,7 +21722,7 @@ name = "tracing-gum-proc-macro" version = "5.0.0" dependencies = [ "assert_matches", - "expander", + "expander 2.0.0", "proc-macro-crate 3.1.0", "proc-macro2 1.0.82", "quote 1.0.35", From 0ed57f82a3322ae73441d09dd3ae19e5df3b5820 Mon Sep 17 00:00:00 2001 From: alindima Date: Tue, 21 May 2024 13:28:19 +0300 Subject: [PATCH 119/126] unify fatality versions --- Cargo.lock | 77 ++++++------------------ polkadot/node/subsystem-types/Cargo.toml | 2 +- 2 files changed, 21 insertions(+), 58 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 28ea0c9bcb5d..c4c19d2003c8 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5299,18 +5299,6 @@ dependencies = [ "futures", ] -[[package]] -name = "expander" -version = "0.0.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a718c0675c555c5f976fff4ea9e2c150fa06cefa201cadef87cfbf9324075881" -dependencies = [ - "blake3", - "fs-err", - "proc-macro2 1.0.82", - "quote 1.0.35", -] - [[package]] name = "expander" version = "2.0.0" @@ -5372,38 +5360,13 @@ dependencies = [ "bytes", ] -[[package]] -name = "fatality" -version = "0.0.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2ad875162843b0d046276327afe0136e9ed3a23d5a754210fb6f1f33610d39ab" -dependencies = [ - "fatality-proc-macro 0.0.6", - "thiserror", -] - [[package]] name = "fatality" version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ec6f82451ff7f0568c6181287189126d492b5654e30a788add08027b6363d019" dependencies = [ - "fatality-proc-macro 0.1.1", - "thiserror", -] - -[[package]] -name = "fatality-proc-macro" -version = "0.0.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f5aa1e3ae159e592ad222dc90c5acbad632b527779ba88486abe92782ab268bd" -dependencies = [ - "expander 0.0.4", - "indexmap 1.9.3", - "proc-macro-crate 1.3.1", - "proc-macro2 1.0.82", - "quote 1.0.35", - "syn 1.0.109", + "fatality-proc-macro", "thiserror", ] @@ -5413,7 +5376,7 @@ version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "eb42427514b063d97ce21d5199f36c0c307d981434a6be32582bc79fe5bd2303" dependencies = [ - "expander 2.0.0", + "expander", "indexmap 2.2.3", "proc-macro-crate 3.1.0", "proc-macro2 1.0.82", @@ -5907,7 +5870,7 @@ dependencies = [ "Inflector", "cfg-expr", "derive-syn-parse 0.2.0", - "expander 2.0.0", + "expander", "frame-support-procedural-tools", "itertools 0.11.0", "macro_magic", @@ -9380,7 +9343,7 @@ version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1344346d5af32c95bbddea91b18a88cc83eac394192d20ef2fc4c40a74332355" dependencies = [ - "expander 2.0.0", + "expander", "indexmap 2.2.3", "itertools 0.11.0", "petgraph", @@ -12670,7 +12633,7 @@ version = "7.0.0" dependencies = [ "assert_matches", "derive_more", - "fatality 0.1.1", + "fatality", "futures", "futures-timer", "parity-scale-codec", @@ -12701,7 +12664,7 @@ version = "7.0.0" dependencies = [ "assert_matches", "async-trait", - "fatality 0.1.1", + "fatality", "futures", "futures-timer", "log", @@ -12764,7 +12727,7 @@ dependencies = [ "assert_matches", "bitvec", "env_logger 0.11.3", - "fatality 0.1.1", + "fatality", "futures", "futures-timer", "log", @@ -12807,7 +12770,7 @@ dependencies = [ "async-channel", "async-trait", "derive_more", - "fatality 0.1.1", + "fatality", "futures", "futures-timer", "indexmap 2.2.3", @@ -12886,7 +12849,7 @@ dependencies = [ "assert_matches", "async-trait", "bytes", - "fatality 0.1.1", + "fatality", "futures", "futures-timer", "parity-scale-codec", @@ -13010,7 +12973,7 @@ version = "7.0.0" dependencies = [ "assert_matches", "bitvec", - "fatality 0.1.1", + "fatality", "futures", "polkadot-erasure-coding", "polkadot-node-primitives", @@ -13119,7 +13082,7 @@ name = "polkadot-node-core-dispute-coordinator" version = "7.0.0" dependencies = [ "assert_matches", - "fatality 0.1.1", + "fatality", "futures", "futures-timer", "kvdb", @@ -13164,7 +13127,7 @@ version = "6.0.0" dependencies = [ "assert_matches", "bitvec", - "fatality 0.1.1", + "fatality", "futures", "parity-scale-codec", "polkadot-node-primitives", @@ -13188,7 +13151,7 @@ name = "polkadot-node-core-provisioner" version = "7.0.0" dependencies = [ "bitvec", - "fatality 0.1.1", + "fatality", "futures", "futures-timer", "polkadot-node-primitives", @@ -13409,7 +13372,7 @@ dependencies = [ "async-trait", "bitvec", "derive_more", - "fatality 0.1.1", + "fatality", "futures", "hex", "parity-scale-codec", @@ -13487,7 +13450,7 @@ dependencies = [ "async-trait", "bitvec", "derive_more", - "fatality 0.0.6", + "fatality", "futures", "orchestra", "polkadot-node-jaeger", @@ -13517,7 +13480,7 @@ dependencies = [ "async-trait", "derive_more", "env_logger 0.11.3", - "fatality 0.1.1", + "fatality", "futures", "futures-channel", "itertools 0.11.0", @@ -14127,7 +14090,7 @@ dependencies = [ "assert_matches", "async-channel", "bitvec", - "fatality 0.1.1", + "fatality", "futures", "futures-timer", "indexmap 2.2.3", @@ -19072,7 +19035,7 @@ dependencies = [ "Inflector", "assert_matches", "blake2 0.10.6", - "expander 2.0.0", + "expander", "proc-macro-crate 3.1.0", "proc-macro2 1.0.82", "quote 1.0.35", @@ -19767,7 +19730,7 @@ name = "sp-runtime-interface-proc-macro" version = "17.0.0" dependencies = [ "Inflector", - "expander 2.0.0", + "expander", "proc-macro-crate 3.1.0", "proc-macro2 1.0.82", "quote 1.0.35", @@ -21723,7 +21686,7 @@ name = "tracing-gum-proc-macro" version = "5.0.0" dependencies = [ "assert_matches", - "expander 2.0.0", + "expander", "proc-macro-crate 3.1.0", "proc-macro2 1.0.82", "quote 1.0.35", diff --git a/polkadot/node/subsystem-types/Cargo.toml b/polkadot/node/subsystem-types/Cargo.toml index ffc92286f86a..e03fc60a1fd7 100644 --- a/polkadot/node/subsystem-types/Cargo.toml +++ b/polkadot/node/subsystem-types/Cargo.toml @@ -11,7 +11,7 @@ workspace = true [dependencies] derive_more = "0.99.17" -fatality = "0.0.6" +fatality = "0.1.1" futures = "0.3.30" polkadot-primitives = { path = "../../primitives" } polkadot-node-primitives = { path = "../primitives" } From 8acc2c13739ed6704b9f36a347565c5b2c6dee7f Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 24 May 2024 14:32:18 +0300 Subject: [PATCH 120/126] some metrics and logs polishes --- .../availability-recovery/src/error.rs | 2 +- .../availability-recovery/src/metrics.rs | 27 +++++++++++++++++++ .../src/task/strategy/mod.rs | 4 ++- .../examples/availability_read.yaml | 8 +++--- 4 files changed, 35 insertions(+), 6 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/error.rs b/polkadot/node/network/availability-recovery/src/error.rs index 9cf2c63cbfa5..eaec4cbc9d94 100644 --- a/polkadot/node/network/availability-recovery/src/error.rs +++ b/polkadot/node/network/availability-recovery/src/error.rs @@ -86,6 +86,6 @@ pub fn log_error(result: Result<()>) -> std::result::Result<(), FatalError> { impl JfyiError { /// Log a `JfyiError`. pub fn log(self) { - gum::warn!(target: LOG_TARGET, error = ?self); + gum::warn!(target: LOG_TARGET, "{}", self); } } diff --git a/polkadot/node/network/availability-recovery/src/metrics.rs b/polkadot/node/network/availability-recovery/src/metrics.rs index 648b9ff7c98e..4e269df55027 100644 --- a/polkadot/node/network/availability-recovery/src/metrics.rs +++ b/polkadot/node/network/availability-recovery/src/metrics.rs @@ -55,6 +55,9 @@ struct MetricsInner { /// - `success` chunk_requests_finished: CounterVec, + /// A counter for successful chunk requests, split by the network protocol version. + chunk_request_protocols: CounterVec, + /// Number of sent available data requests. full_data_requests_issued: Counter, @@ -202,6 +205,20 @@ impl Metrics { } } + /// A chunk response was received on the v1 protocol. + pub fn on_chunk_response_v1(&self) { + if let Some(metrics) = &self.0 { + metrics.chunk_request_protocols.with_label_values(&["v1"]).inc() + } + } + + /// A chunk response was received on the v2 protocol. + pub fn on_chunk_response_v2(&self) { + if let Some(metrics) = &self.0 { + metrics.chunk_request_protocols.with_label_values(&["v2"]).inc() + } + } + /// A full data request succeeded. pub fn on_full_request_succeeded(&self) { if let Some(metrics) = &self.0 { @@ -314,6 +331,16 @@ impl metrics::Metrics for Metrics { )?, registry, )?, + chunk_request_protocols: prometheus::register( + CounterVec::new( + Opts::new( + "polkadot_parachain_availability_recovery_chunk_request_protocols", + "Total number of successful chunk requests, mapped by the protocol version (v1 or v2).", + ), + &["protocol"], + )?, + registry, + )?, full_data_requests_finished: prometheus::register( CounterVec::new( Opts::new( diff --git a/polkadot/node/network/availability-recovery/src/task/strategy/mod.rs b/polkadot/node/network/availability-recovery/src/task/strategy/mod.rs index ba7f520c97be..97781c05b1f8 100644 --- a/polkadot/node/network/availability-recovery/src/task/strategy/mod.rs +++ b/polkadot/node/network/availability-recovery/src/task/strategy/mod.rs @@ -384,6 +384,7 @@ impl State { let res = match res.await { Ok((bytes, protocol)) => if v2_protocol_name == protocol { + params.metrics.on_chunk_response_v2(); match req_res::v2::ChunkFetchingResponse::decode(&mut &bytes[..]) { Ok(req_res::v2::ChunkFetchingResponse::Chunk(chunk)) => Ok(Some(chunk.into())), @@ -391,6 +392,7 @@ impl State { Err(e) => Err(RequestError::InvalidResponse(e)), } } else if v1_protocol_name == protocol { + params.metrics.on_chunk_response_v1(); // V1 protocol version must not be used when chunk mapping node // feature is enabled, because we can't know the real index of the // returned chunk. @@ -398,7 +400,7 @@ impl State { // `AvailabilityChunkMapping` feature is only enabled after the // v1 version is removed. Still, log this. if chunk_mapping_enabled { - gum::warn!( + gum::info!( target: LOG_TARGET, ?candidate_hash, authority_id = ?authority_id_clone, diff --git a/polkadot/node/subsystem-bench/examples/availability_read.yaml b/polkadot/node/subsystem-bench/examples/availability_read.yaml index 82355b0e2973..263a6988242e 100644 --- a/polkadot/node/subsystem-bench/examples/availability_read.yaml +++ b/polkadot/node/subsystem-bench/examples/availability_read.yaml @@ -1,8 +1,8 @@ TestConfiguration: # Test 1 - objective: !DataAvailabilityRead - fetch_from_backers: true - n_validators: 300 + strategy: FullFromBackers + n_validators: 500 n_cores: 20 min_pov_size: 5120 max_pov_size: 5120 @@ -16,7 +16,7 @@ TestConfiguration: # Test 2 - objective: !DataAvailabilityRead - fetch_from_backers: true + strategy: FullFromBackers n_validators: 500 n_cores: 20 min_pov_size: 5120 @@ -31,7 +31,7 @@ TestConfiguration: # Test 3 - objective: !DataAvailabilityRead - fetch_from_backers: true + strategy: FullFromBackers n_validators: 1000 n_cores: 20 min_pov_size: 5120 From 7ddf494289f5032ff4c8dca0ac11e4d7ab546ed4 Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 24 May 2024 14:32:46 +0300 Subject: [PATCH 121/126] more details to prdoc --- prdoc/pr_1644.prdoc | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/prdoc/pr_1644.prdoc b/prdoc/pr_1644.prdoc index 58ee02c5ef76..5afd4bea56ac 100644 --- a/prdoc/pr_1644.prdoc +++ b/prdoc/pr_1644.prdoc @@ -1,10 +1,18 @@ title: Add availability-recovery from systematic chunks doc: + - audience Node Operator + description: | + Implements https://github.com/polkadot-fellows/RFCs/pull/47. This optimisation is guarded by a configuration bit in + the runtime and will only be enabled once a supermajority of the validators have upgraded to this version. + It's strongly advised to upgrade to this version. - audience: Node Dev description: | Implements https://github.com/polkadot-fellows/RFCs/pull/47 and adds the logic for availability recovery from systematic chunks. - The /req_chunk/1 req-response protocol is now considered deprecated in favour of /req_chunk/2. + The /req_chunk/1 req-response protocol is now considered deprecated in favour of /req_chunk/2. Systematic recovery is guarded + by a configuration bit in the runtime (bit with index 2 of the node_features field from the HostConfiguration) + and must not be enabled until all (or almost all) validators have upgraded to the node version that includes + this PR. crates: - name: sc-network From c7211522adf787a5ba7aff60d0049311708d52e1 Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 24 May 2024 14:35:49 +0300 Subject: [PATCH 122/126] prdoc fixup --- prdoc/pr_1644.prdoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/prdoc/pr_1644.prdoc b/prdoc/pr_1644.prdoc index 5afd4bea56ac..cc43847fa09b 100644 --- a/prdoc/pr_1644.prdoc +++ b/prdoc/pr_1644.prdoc @@ -1,7 +1,7 @@ title: Add availability-recovery from systematic chunks doc: - - audience Node Operator + - audience: Node Operator description: | Implements https://github.com/polkadot-fellows/RFCs/pull/47. This optimisation is guarded by a configuration bit in the runtime and will only be enabled once a supermajority of the validators have upgraded to this version. From 6faa1ee7502c0217efaba84d89e21bb493a3a6db Mon Sep 17 00:00:00 2001 From: alindima Date: Fri, 24 May 2024 17:32:29 +0300 Subject: [PATCH 123/126] fix --- .../src/task/strategy/mod.rs | 128 ++++++++++-------- 1 file changed, 75 insertions(+), 53 deletions(-) diff --git a/polkadot/node/network/availability-recovery/src/task/strategy/mod.rs b/polkadot/node/network/availability-recovery/src/task/strategy/mod.rs index 97781c05b1f8..fb31ff6aa779 100644 --- a/polkadot/node/network/availability-recovery/src/task/strategy/mod.rs +++ b/polkadot/node/network/availability-recovery/src/task/strategy/mod.rs @@ -43,7 +43,7 @@ use polkadot_node_subsystem::{ overseer, RecoveryError, }; use polkadot_primitives::{AuthorityDiscoveryId, BlakeTwo256, ChunkIndex, HashT, ValidatorIndex}; -use sc_network::{IfDisconnected, OutboundFailure, RequestFailure}; +use sc_network::{IfDisconnected, OutboundFailure, ProtocolName, RequestFailure}; use std::{ collections::{BTreeMap, HashMap, VecDeque}, time::Duration, @@ -79,7 +79,7 @@ pub const REGULAR_CHUNKS_REQ_RETRY_LIMIT: u32 = 5; type OngoingRequests = FuturesUndead<( AuthorityDiscoveryId, ValidatorIndex, - Result, RequestError>, + Result<(Option, ProtocolName), RequestError>, )>; const fn is_unavailable( @@ -384,15 +384,14 @@ impl State { let res = match res.await { Ok((bytes, protocol)) => if v2_protocol_name == protocol { - params.metrics.on_chunk_response_v2(); match req_res::v2::ChunkFetchingResponse::decode(&mut &bytes[..]) { Ok(req_res::v2::ChunkFetchingResponse::Chunk(chunk)) => - Ok(Some(chunk.into())), - Ok(req_res::v2::ChunkFetchingResponse::NoSuchChunk) => Ok(None), + Ok((Some(chunk.into()), protocol)), + Ok(req_res::v2::ChunkFetchingResponse::NoSuchChunk) => + Ok((None, protocol)), Err(e) => Err(RequestError::InvalidResponse(e)), } } else if v1_protocol_name == protocol { - params.metrics.on_chunk_response_v1(); // V1 protocol version must not be used when chunk mapping node // feature is enabled, because we can't know the real index of the // returned chunk. @@ -410,9 +409,12 @@ impl State { } match req_res::v1::ChunkFetchingResponse::decode(&mut &bytes[..]) { - Ok(req_res::v1::ChunkFetchingResponse::Chunk(chunk)) => - Ok(Some(chunk.recombine_into_chunk(&raw_request_v1))), - Ok(req_res::v1::ChunkFetchingResponse::NoSuchChunk) => Ok(None), + Ok(req_res::v1::ChunkFetchingResponse::Chunk(chunk)) => Ok(( + Some(chunk.recombine_into_chunk(&raw_request_v1)), + protocol, + )), + Ok(req_res::v1::ChunkFetchingResponse::NoSuchChunk) => + Ok((None, protocol)), Err(e) => Err(RequestError::InvalidResponse(e)), } } else { @@ -480,42 +482,54 @@ impl State { let mut is_error = false; match request_result { - Ok(Some(chunk)) => - if is_chunk_valid(params, &chunk) { - metrics.on_chunk_request_succeeded(strategy_type); - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - ?authority_id, - ?validator_index, - "Received valid chunk", - ); - self.insert_chunk( - chunk.index, - Chunk { chunk: chunk.chunk, validator_index }, - ); - } else { - metrics.on_chunk_request_invalid(strategy_type); - error_count += 1; - // Record that we got an invalid chunk so that subsequent strategies don't - // try requesting this again. - self.record_error_fatal(authority_id.clone(), validator_index); - is_error = true; - }, - Ok(None) => { - metrics.on_chunk_request_no_such_chunk(strategy_type); - gum::trace!( - target: LOG_TARGET, - candidate_hash = ?params.candidate_hash, - ?authority_id, - ?validator_index, - "Validator did not have the chunk", - ); - error_count += 1; - // Record that the validator did not have this chunk so that subsequent - // strategies don't try requesting this again. - self.record_error_fatal(authority_id.clone(), validator_index); - is_error = true; + Ok((maybe_chunk, protocol)) => { + match protocol { + name if name == params.req_v1_protocol_name => + params.metrics.on_chunk_response_v1(), + name if name == params.req_v2_protocol_name => + params.metrics.on_chunk_response_v2(), + _ => {}, + } + + match maybe_chunk { + Some(chunk) => + if is_chunk_valid(params, &chunk) { + metrics.on_chunk_request_succeeded(strategy_type); + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?params.candidate_hash, + ?authority_id, + ?validator_index, + "Received valid chunk", + ); + self.insert_chunk( + chunk.index, + Chunk { chunk: chunk.chunk, validator_index }, + ); + } else { + metrics.on_chunk_request_invalid(strategy_type); + error_count += 1; + // Record that we got an invalid chunk so that subsequent strategies + // don't try requesting this again. + self.record_error_fatal(authority_id.clone(), validator_index); + is_error = true; + }, + None => { + metrics.on_chunk_request_no_such_chunk(strategy_type); + gum::trace!( + target: LOG_TARGET, + candidate_hash = ?params.candidate_hash, + ?authority_id, + ?validator_index, + "Validator did not have the chunk", + ); + error_count += 1; + // Record that the validator did not have this chunk so that subsequent + // strategies don't try requesting this again. + self.record_error_fatal(authority_id.clone(), validator_index); + is_error = true; + }, + } }, Err(err) => { error_count += 1; @@ -1061,7 +1075,7 @@ mod tests { future::ready(( params.validator_authority_keys[0].clone(), 0.into(), - Ok(Some(chunks[0].clone())), + Ok((Some(chunks[0].clone()), "".into())), )) .boxed(), ); @@ -1070,13 +1084,17 @@ mod tests { future::ready(( params.validator_authority_keys[1].clone(), 1.into(), - Ok(Some(chunks[1].clone())), + Ok((Some(chunks[1].clone()), "".into())), )) .boxed(), ); ongoing_reqs.push( - future::ready((params.validator_authority_keys[2].clone(), 2.into(), Ok(None))) - .boxed(), + future::ready(( + params.validator_authority_keys[2].clone(), + 2.into(), + Ok((None, "".into())), + )) + .boxed(), ); ongoing_reqs.push( future::ready(( @@ -1201,7 +1219,7 @@ mod tests { future::ready(( params.validator_authority_keys[0].clone(), 0.into(), - Ok(Some(chunks[0].clone())), + Ok((Some(chunks[0].clone()), "".into())), )) .boxed(), ); @@ -1210,13 +1228,17 @@ mod tests { future::ready(( params.validator_authority_keys[1].clone(), 1.into(), - Ok(Some(chunks[1].clone())), + Ok((Some(chunks[1].clone()), "".into())), )) .boxed(), ); ongoing_reqs.push( - future::ready((params.validator_authority_keys[2].clone(), 2.into(), Ok(None))) - .boxed(), + future::ready(( + params.validator_authority_keys[2].clone(), + 2.into(), + Ok((None, "".into())), + )) + .boxed(), ); ongoing_reqs.push( future::ready(( From 5669eec2a96ff42c71fc4ed91a3fad10babf96da Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 27 May 2024 10:00:07 +0300 Subject: [PATCH 124/126] try to make markdown linter happy --- README.md | 9 +++++---- .../src/node/availability/availability-recovery.md | 2 +- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/README.md b/README.md index e139dc0ee076..da613a66f8e2 100644 --- a/README.md +++ b/README.md @@ -30,7 +30,7 @@ forks](https://img.shields.io/github/forks/paritytech/polkadot-sdk) ## 🚀 Releases -> [!NOTE] +> [!NOTE] > Our release process is still Work-In-Progress and may not yet reflect the aspired outline > here. @@ -62,9 +62,10 @@ Conduct](./docs/contributor/CODE_OF_CONDUCT.md). ### 👾 Ready to Contribute? -Take a look at the issues labeled with [`mentor`](https://github.com/paritytech/polkadot-sdk/labels/C1-mentor) (or alternatively [this](https://mentor.tasty.limo/) page, created by one of -the maintainers) label to get started! We always recognize valuable contributions by proposing an -on-chain tip to the Polkadot network as a token of our appreciation. +Take a look at the issues labeled with [`mentor`](https://github.com/paritytech/polkadot-sdk/labels/C1-mentor) +(or alternatively [this](https://mentor.tasty.limo/) page, created by one of the maintainers) label to get started! +We always recognize valuable contributions by proposing an on-chain tip to the Polkadot network as a token of our +appreciation. ## Polkadot Fellowship diff --git a/polkadot/roadmap/implementers-guide/src/node/availability/availability-recovery.md b/polkadot/roadmap/implementers-guide/src/node/availability/availability-recovery.md index 5d3e2590e046..5b756080becc 100644 --- a/polkadot/roadmap/implementers-guide/src/node/availability/availability-recovery.md +++ b/polkadot/roadmap/implementers-guide/src/node/availability/availability-recovery.md @@ -170,7 +170,7 @@ In the worst case, all validators will be tried. #### For validators -If the estimated available data size is smaller than a configured constant (currently 1Mib for polkadot or 4Mib for +If the estimated available data size is smaller than a configured constant (currently 1Mib for Polkadot or 4Mib for other networks), try doing `FetchFull` first. Next, if the preconditions described in `FetchSystematicChunks` above are met, try systematic recovery. As a last resort, do `FetchChunks`. From 99f09e3d1bbe38f72efc0de3c85debee73d930f2 Mon Sep 17 00:00:00 2001 From: alindima Date: Mon, 27 May 2024 12:20:24 +0300 Subject: [PATCH 125/126] use higher glutton PoV sizes for tests this is needed in order to trigger chunk requests because PoVs smaller than 4Mibs will prefer fetching from backers --- .../functional/0013-systematic-chunk-recovery.toml | 2 +- .../functional/0014-chunk-fetching-network-compatibility.toml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.toml b/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.toml index 281e63a0466c..67925a3d3a7c 100644 --- a/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.toml +++ b/polkadot/zombienet_tests/functional/0013-systematic-chunk-recovery.toml @@ -34,7 +34,7 @@ cumulus_based = true chain = "glutton-westend-local-{{id}}" [parachains.genesis.runtimeGenesis.patch.glutton] compute = "50000000" - storage = "1310000000" + storage = "2500000000" trashDataCount = 5120 [parachains.collator] diff --git a/polkadot/zombienet_tests/functional/0014-chunk-fetching-network-compatibility.toml b/polkadot/zombienet_tests/functional/0014-chunk-fetching-network-compatibility.toml index 8924102bb2b5..881abab64fd0 100644 --- a/polkadot/zombienet_tests/functional/0014-chunk-fetching-network-compatibility.toml +++ b/polkadot/zombienet_tests/functional/0014-chunk-fetching-network-compatibility.toml @@ -37,7 +37,7 @@ cumulus_based = true chain = "glutton-westend-local-{{id}}" [parachains.genesis.runtimeGenesis.patch.glutton] compute = "50000000" - storage = "1310000000" + storage = "2500000000" trashDataCount = 5120 [parachains.collator] From ce08e60f24192148e32847d3d86c771aa2cc69ea Mon Sep 17 00:00:00 2001 From: Javier Viola Date: Mon, 27 May 2024 20:45:10 +0200 Subject: [PATCH 126/126] bump zombienet version --- .gitlab/pipeline/zombienet.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.gitlab/pipeline/zombienet.yml b/.gitlab/pipeline/zombienet.yml index 404b57b07c59..7897e55e291b 100644 --- a/.gitlab/pipeline/zombienet.yml +++ b/.gitlab/pipeline/zombienet.yml @@ -1,7 +1,7 @@ .zombienet-refs: extends: .build-refs variables: - ZOMBIENET_IMAGE: "docker.io/paritytech/zombienet:v1.3.104" + ZOMBIENET_IMAGE: "docker.io/paritytech/zombienet:v1.3.105" PUSHGATEWAY_URL: "http://zombienet-prometheus-pushgateway.managed-monitoring:9091/metrics/job/zombie-metrics" DEBUG: "zombie,zombie::network-node,zombie::kube::client::logs"