diff --git a/polkadot/roadmap/implementers-guide/src/runtime/parainherent.md b/polkadot/roadmap/implementers-guide/src/runtime/parainherent.md index 4a771f1df644..9489a286a0cb 100644 --- a/polkadot/roadmap/implementers-guide/src/runtime/parainherent.md +++ b/polkadot/roadmap/implementers-guide/src/runtime/parainherent.md @@ -60,3 +60,34 @@ processing it, so the processed inherent data is simply dropped. This also means that the `enter` function keeps data around for no good reason. This seems acceptable though as the size of a block is rather limited. Nevertheless if we ever wanted to optimize this we can easily implement an inherent collector that has two implementations, where one clones and stores the data and the other just passes it on. + +## Data sanitization +`ParasInherent` with the entry point of `create_inherent` sanitizes the input data, while the `enter` entry point +enforces already sanitized input data. If unsanitized data is provided the module generates an error. + +Disputes are included in the block with a priority for a security reasons. It's important to include as many dispute +votes onchain as possible so that disputes conclude faster and the offenders are punished. However if there are too many +disputes to include in a block the dispute set is trimmed so that it respects max block weight. + +Dispute data is first deduplicated and sorted by block number (older first) and dispute location (local then remote). +Concluded and ancient (disputes initiated before the post conclusion acceptance period) disputes are filtered out. +Votes with invalid signatures or from unknown validators (not found in the active set for the current session) are also +filtered out. + +All dispute statements are included in the order described in the previous paragraph until the available block weight is +exhausted. After the dispute data is included all remaining weight is filled in with candidates and availability +bitfields. Bitfields are included with priority, then candidates containing code updates and finally any backed +candidates. If there is not enough weight for all backed candidates they are trimmed by random selection. Disputes are +processed in three separate functions - `deduplicate_and_sort_dispute_data`, `filter_dispute_data` and +`limit_and_sanitize_disputes`. + +Availability bitfields are also sanitized by dropping malformed ones, containing disputed cores or bad signatures. Refer +to `sanitize_bitfields` function for implementation details. + +Backed candidates sanitization removes malformed ones, candidates which have got concluded invalid disputes against them +or candidates produced by unassigned cores. Furthermore any backing votes from disabled validators for a candidate are +dropped. This is part of the validator disabling strategy. After filtering the statements from disabled validators a +backed candidate may end up with votes count less than `minimum_backing_votes` (a parameter from `HostConfiguiration`). +In this case the whole candidate is dropped otherwise it will be rejected by `process_candidates` from pallet inclusion. +All checks related to backed candidates are implemented in `sanitize_backed_candidates` and +`filter_backed_statements_from_disabled_validators`. diff --git a/polkadot/runtime/common/src/assigned_slots/mod.rs b/polkadot/runtime/common/src/assigned_slots/mod.rs index f5e3aaef6324..efd80a16744d 100644 --- a/polkadot/runtime/common/src/assigned_slots/mod.rs +++ b/polkadot/runtime/common/src/assigned_slots/mod.rs @@ -745,7 +745,9 @@ mod tests { type OnNewHead = (); } - impl parachains_shared::Config for Test {} + impl parachains_shared::Config for Test { + type DisabledValidators = (); + } parameter_types! { pub const LeasePeriod: BlockNumber = 3; diff --git a/polkadot/runtime/common/src/integration_tests.rs b/polkadot/runtime/common/src/integration_tests.rs index b0d277a702d6..825115d82905 100644 --- a/polkadot/runtime/common/src/integration_tests.rs +++ b/polkadot/runtime/common/src/integration_tests.rs @@ -197,7 +197,9 @@ impl configuration::Config for Test { type WeightInfo = configuration::TestWeightInfo; } -impl shared::Config for Test {} +impl shared::Config for Test { + type DisabledValidators = (); +} impl origin::Config for Test {} diff --git a/polkadot/runtime/common/src/paras_registrar/mod.rs b/polkadot/runtime/common/src/paras_registrar/mod.rs index 12376ae6f1ff..b966967f21fd 100644 --- a/polkadot/runtime/common/src/paras_registrar/mod.rs +++ b/polkadot/runtime/common/src/paras_registrar/mod.rs @@ -799,7 +799,9 @@ mod tests { type MaxFreezes = ConstU32<1>; } - impl shared::Config for Test {} + impl shared::Config for Test { + type DisabledValidators = (); + } impl origin::Config for Test {} diff --git a/polkadot/runtime/parachains/src/mock.rs b/polkadot/runtime/parachains/src/mock.rs index 222942922f91..75fb1ee1ad43 100644 --- a/polkadot/runtime/parachains/src/mock.rs +++ b/polkadot/runtime/parachains/src/mock.rs @@ -184,7 +184,22 @@ impl crate::configuration::Config for Test { type WeightInfo = crate::configuration::TestWeightInfo; } -impl crate::shared::Config for Test {} +pub struct MockDisabledValidators {} +impl frame_support::traits::DisabledValidators for MockDisabledValidators { + /// Returns true if the given validator is disabled. + fn is_disabled(index: u32) -> bool { + disabled_validators().iter().any(|v| *v == index) + } + + /// Returns a hardcoded list (`DISABLED_VALIDATORS`) of disabled validators + fn disabled_validators() -> Vec { + disabled_validators() + } +} + +impl crate::shared::Config for Test { + type DisabledValidators = MockDisabledValidators; +} impl origin::Config for Test {} @@ -432,6 +447,8 @@ thread_local! { pub static AVAILABILITY_REWARDS: RefCell> = RefCell::new(HashMap::new()); + + pub static DISABLED_VALIDATORS: RefCell> = RefCell::new(vec![]); } pub fn backing_rewards() -> HashMap { @@ -442,6 +459,10 @@ pub fn availability_rewards() -> HashMap { AVAILABILITY_REWARDS.with(|r| r.borrow().clone()) } +pub fn disabled_validators() -> Vec { + DISABLED_VALIDATORS.with(|r| r.borrow().clone()) +} + parameter_types! { pub static Processed: Vec<(ParaId, UpwardMessage)> = vec![]; } @@ -581,3 +602,7 @@ pub(crate) fn deregister_parachain(id: ParaId) { pub(crate) fn try_deregister_parachain(id: ParaId) -> crate::DispatchResult { frame_support::storage::transactional::with_storage_layer(|| Paras::schedule_para_cleanup(id)) } + +pub(crate) fn set_disabled_validators(disabled: Vec) { + DISABLED_VALIDATORS.with(|d| *d.borrow_mut() = disabled) +} diff --git a/polkadot/runtime/parachains/src/paras_inherent/mod.rs b/polkadot/runtime/parachains/src/paras_inherent/mod.rs index 8e918d35d5ff..7a4cb8ae3106 100644 --- a/polkadot/runtime/parachains/src/paras_inherent/mod.rs +++ b/polkadot/runtime/parachains/src/paras_inherent/mod.rs @@ -30,7 +30,8 @@ use crate::{ metrics::METRICS, paras, scheduler::{self, FreedReason}, - shared, ParaId, + shared::{self, AllowedRelayParentsTracker}, + ParaId, }; use bitvec::prelude::BitVec; use frame_support::{ @@ -42,8 +43,8 @@ use frame_support::{ use frame_system::pallet_prelude::*; use pallet_babe::{self, ParentBlockRandomness}; use primitives::{ - BackedCandidate, CandidateHash, CandidateReceipt, CheckedDisputeStatementSet, - CheckedMultiDisputeStatementSet, CoreIndex, DisputeStatementSet, + effective_minimum_backing_votes, BackedCandidate, CandidateHash, CandidateReceipt, + CheckedDisputeStatementSet, CheckedMultiDisputeStatementSet, CoreIndex, DisputeStatementSet, InherentData as ParachainsInherentData, MultiDisputeStatementSet, ScrapedOnChainVotes, SessionIndex, SignedAvailabilityBitfields, SigningContext, UncheckedSignedAvailabilityBitfield, UncheckedSignedAvailabilityBitfields, ValidatorId, ValidatorIndex, ValidityAttestation, @@ -142,6 +143,8 @@ pub mod pallet { DisputeStatementsUnsortedOrDuplicates, /// A dispute statement was invalid. DisputeInvalid, + /// A candidate was backed by a disabled validator + BackedByDisabled, } /// Whether the paras inherent was included within this block. @@ -378,6 +381,7 @@ impl Pallet { let bitfields_weight = signed_bitfields_weight::(&bitfields); let disputes_weight = multi_dispute_statement_sets_weight::(&disputes); + // Weight before filtering/sanitization let all_weight_before = candidates_weight + bitfields_weight + disputes_weight; METRICS.on_before_filter(all_weight_before.ref_time()); @@ -587,17 +591,19 @@ impl Pallet { METRICS.on_candidates_processed_total(backed_candidates.len() as u64); - let backed_candidates = sanitize_backed_candidates::( - backed_candidates, - |candidate_idx: usize, - backed_candidate: &BackedCandidate<::Hash>| - -> bool { - let para_id = backed_candidate.descriptor().para_id; - let prev_context = >::para_most_recent_context(para_id); - let check_ctx = CandidateCheckContext::::new(prev_context); - - // never include a concluded-invalid candidate - current_concluded_invalid_disputes.contains(&backed_candidate.hash()) || + let SanitizedBackedCandidates { backed_candidates, votes_from_disabled_were_dropped } = + sanitize_backed_candidates::( + backed_candidates, + &allowed_relay_parents, + |candidate_idx: usize, + backed_candidate: &BackedCandidate<::Hash>| + -> bool { + let para_id = backed_candidate.descriptor().para_id; + let prev_context = >::para_most_recent_context(para_id); + let check_ctx = CandidateCheckContext::::new(prev_context); + + // never include a concluded-invalid candidate + current_concluded_invalid_disputes.contains(&backed_candidate.hash()) || // Instead of checking the candidates with code upgrades twice // move the checking up here and skip it in the training wheels fallback. // That way we avoid possible duplicate checks while assuring all @@ -607,12 +613,19 @@ impl Pallet { check_ctx .verify_backed_candidate(&allowed_relay_parents, candidate_idx, backed_candidate) .is_err() - }, - &scheduled, - ); + }, + &scheduled, + ); METRICS.on_candidates_sanitized(backed_candidates.len() as u64); + // In `Enter` context (invoked during execution) there should be no backing votes from + // disabled validators because they should have been filtered out during inherent data + // preparation (`ProvideInherent` context). Abort in such cases. + if context == ProcessInherentDataContext::Enter { + ensure!(!votes_from_disabled_were_dropped, Error::::BackedByDisabled); + } + // Process backed candidates according to scheduled cores. let inclusion::ProcessedCandidates::< as HeaderT>::Hash> { core_indices: occupied, @@ -900,7 +913,19 @@ pub(crate) fn sanitize_bitfields( bitfields } -/// Filter out any candidates that have a concluded invalid dispute. +// Result from `sanitize_backed_candidates` +#[derive(Debug, PartialEq)] +struct SanitizedBackedCandidates { + // Sanitized backed candidates. The `Vec` is sorted according to the occupied core index. + backed_candidates: Vec>, + // Set to true if any votes from disabled validators were dropped from the input. + votes_from_disabled_were_dropped: bool, +} + +/// Filter out: +/// 1. any candidates that have a concluded invalid dispute +/// 2. all backing votes from disabled validators +/// 3. any candidates that end up with less than `effective_minimum_backing_votes` backing votes /// /// `scheduled` follows the same naming scheme as provided in the /// guide: Currently `free` but might become `occupied`. @@ -910,15 +935,17 @@ pub(crate) fn sanitize_bitfields( /// `candidate_has_concluded_invalid_dispute` must return `true` if the candidate /// is disputed, false otherwise. The passed `usize` is the candidate index. /// -/// The returned `Vec` is sorted according to the occupied core index. +/// Returns struct `SanitizedBackedCandidates` where `backed_candidates` are sorted according to the +/// occupied core index. fn sanitize_backed_candidates< T: crate::inclusion::Config, F: FnMut(usize, &BackedCandidate) -> bool, >( mut backed_candidates: Vec>, + allowed_relay_parents: &AllowedRelayParentsTracker>, mut candidate_has_concluded_invalid_dispute_or_is_invalid: F, scheduled: &BTreeMap, -) -> Vec> { +) -> SanitizedBackedCandidates { // Remove any candidates that were concluded invalid. // This does not assume sorting. backed_candidates.indexed_retain(move |candidate_idx, backed_candidate| { @@ -936,6 +963,13 @@ fn sanitize_backed_candidates< scheduled.get(&desc.para_id).is_some() }); + // Filter out backing statements from disabled validators + let dropped_disabled = filter_backed_statements_from_disabled_validators::( + &mut backed_candidates, + &allowed_relay_parents, + scheduled, + ); + // Sort the `Vec` last, once there is a guarantee that these // `BackedCandidates` references the expected relay chain parent, // but more importantly are scheduled for a free core. @@ -946,7 +980,10 @@ fn sanitize_backed_candidates< scheduled[&x.descriptor().para_id].cmp(&scheduled[&y.descriptor().para_id]) }); - backed_candidates + SanitizedBackedCandidates { + backed_candidates, + votes_from_disabled_were_dropped: dropped_disabled, + } } /// Derive entropy from babe provided per block randomness. @@ -1029,3 +1066,105 @@ fn limit_and_sanitize_disputes< (checked, checked_disputes_weight) } } + +// Filters statements from disabled validators in `BackedCandidate`, non-scheduled candidates and +// few more sanity checks. Returns `true` if at least one statement is removed and `false` +// otherwise. +fn filter_backed_statements_from_disabled_validators( + backed_candidates: &mut Vec::Hash>>, + allowed_relay_parents: &AllowedRelayParentsTracker>, + scheduled: &BTreeMap, +) -> bool { + let disabled_validators = + BTreeSet::<_>::from_iter(shared::Pallet::::disabled_validators().into_iter()); + + if disabled_validators.is_empty() { + // No disabled validators - nothing to do + return false + } + + let backed_len_before = backed_candidates.len(); + + // Flag which will be returned. Set to `true` if at least one vote is filtered. + let mut filtered = false; + + let minimum_backing_votes = configuration::Pallet::::config().minimum_backing_votes; + + // Process all backed candidates. `validator_indices` in `BackedCandidates` are indices within + // the validator group assigned to the parachain. To obtain this group we need: + // 1. Core index assigned to the parachain which has produced the candidate + // 2. The relay chain block number of the candidate + backed_candidates.retain_mut(|bc| { + // Get `core_idx` assigned to the `para_id` of the candidate + let core_idx = match scheduled.get(&bc.descriptor().para_id) { + Some(core_idx) => *core_idx, + None => { + log::debug!(target: LOG_TARGET, "Can't get core idx of a backed candidate for para id {:?}. Dropping the candidate.", bc.descriptor().para_id); + return false + } + }; + + // Get relay parent block number of the candidate. We need this to get the group index assigned to this core at this block number + let relay_parent_block_number = match allowed_relay_parents + .acquire_info(bc.descriptor().relay_parent, None) { + Some((_, block_num)) => block_num, + None => { + log::debug!(target: LOG_TARGET, "Relay parent {:?} for candidate is not in the allowed relay parents. Dropping the candidate.", bc.descriptor().relay_parent); + return false + } + }; + + // Get the group index for the core + let group_idx = match >::group_assigned_to_core( + core_idx, + relay_parent_block_number + One::one(), + ) { + Some(group_idx) => group_idx, + None => { + log::debug!(target: LOG_TARGET, "Can't get the group index for core idx {:?}. Dropping the candidate.", core_idx); + return false + }, + }; + + // And finally get the validator group for this group index + let validator_group = match >::group_validators(group_idx) { + Some(validator_group) => validator_group, + None => { + log::debug!(target: LOG_TARGET, "Can't get the validators from group {:?}. Dropping the candidate.", group_idx); + return false + } + }; + + // Bitmask with the disabled indices within the validator group + let disabled_indices = BitVec::::from_iter(validator_group.iter().map(|idx| disabled_validators.contains(idx))); + // The indices of statements from disabled validators in `BackedCandidate`. We have to drop these. + let indices_to_drop = disabled_indices.clone() & &bc.validator_indices; + // Apply the bitmask to drop the disabled validator from `validator_indices` + bc.validator_indices &= !disabled_indices; + // Remove the corresponding votes from `validity_votes` + for idx in indices_to_drop.iter_ones().rev() { + bc.validity_votes.remove(idx); + } + + // If at least one statement was dropped we need to return `true` + if indices_to_drop.count_ones() > 0 { + filtered = true; + } + + // By filtering votes we might render the candidate invalid and cause a failure in + // [`process_candidates`]. To avoid this we have to perform a sanity check here. If there + // are not enough backing votes after filtering we will remove the whole candidate. + if bc.validity_votes.len() < effective_minimum_backing_votes( + validator_group.len(), + minimum_backing_votes + + ) { + return false + } + + true + }); + + // Also return `true` if a whole candidate was dropped from the set + filtered || backed_len_before != backed_candidates.len() +} diff --git a/polkadot/runtime/parachains/src/paras_inherent/tests.rs b/polkadot/runtime/parachains/src/paras_inherent/tests.rs index 4fc60792e346..364c6192bebe 100644 --- a/polkadot/runtime/parachains/src/paras_inherent/tests.rs +++ b/polkadot/runtime/parachains/src/paras_inherent/tests.rs @@ -1206,6 +1206,12 @@ mod sanitizers { } mod candidates { + use crate::{ + mock::set_disabled_validators, + scheduler::{common::Assignment, ParasEntry}, + }; + use sp_std::collections::vec_deque::VecDeque; + use super::*; // Backed candidates and scheduled parachains used for `sanitize_backed_candidates` testing @@ -1214,10 +1220,20 @@ mod sanitizers { scheduled_paras: BTreeMap, } - // Generate test data for the candidates test + // Generate test data for the candidates and assert that the evnironment is set as expected + // (check the comments for details) fn get_test_data() -> TestData { const RELAY_PARENT_NUM: u32 = 3; + // Add the relay parent to `shared` pallet. Otherwise some code (e.g. filtering backing + // votes) won't behave correctly + shared::Pallet::::add_allowed_relay_parent( + default_header().hash(), + Default::default(), + RELAY_PARENT_NUM, + 1, + ); + let header = default_header(); let relay_parent = header.hash(); let session_index = SessionIndex::from(0_u32); @@ -1231,6 +1247,7 @@ mod sanitizers { keyring::Sr25519Keyring::Bob, keyring::Sr25519Keyring::Charlie, keyring::Sr25519Keyring::Dave, + keyring::Sr25519Keyring::Eve, ]; for validator in validators.iter() { Keystore::sr25519_generate_new( @@ -1241,11 +1258,36 @@ mod sanitizers { .unwrap(); } + // Set active validators in `shared` pallet + let validator_ids = + validators.iter().map(|v| v.public().into()).collect::>(); + shared::Pallet::::set_active_validators_ascending(validator_ids); + + // Two scheduled parachains - ParaId(1) on CoreIndex(0) and ParaId(2) on CoreIndex(1) let scheduled = (0_usize..2) .into_iter() .map(|idx| (ParaId::from(1_u32 + idx as u32), CoreIndex::from(idx as u32))) .collect::>(); + // Set the validator groups in `scheduler` + scheduler::Pallet::::set_validator_groups(vec![ + vec![ValidatorIndex(0), ValidatorIndex(1)], + vec![ValidatorIndex(2), ValidatorIndex(3)], + ]); + + // Update scheduler's claimqueue with the parachains + scheduler::Pallet::::set_claimqueue(BTreeMap::from([ + ( + CoreIndex::from(0), + VecDeque::from([Some(ParasEntry::new(Assignment::new(1.into()), 1))]), + ), + ( + CoreIndex::from(1), + VecDeque::from([Some(ParasEntry::new(Assignment::new(2.into()), 1))]), + ), + ])); + + // Callback used for backing candidates let group_validators = |group_index: GroupIndex| { match group_index { group_index if group_index == GroupIndex::from(0) => Some(vec![0, 1]), @@ -1255,6 +1297,7 @@ mod sanitizers { .map(|m| m.into_iter().map(ValidatorIndex).collect::>()) }; + // Two backed candidates from each parachain let backed_candidates = (0_usize..2) .into_iter() .map(|idx0| { @@ -1283,6 +1326,22 @@ mod sanitizers { }) .collect::>(); + // State sanity checks + assert_eq!( + >::scheduled_paras().collect::>(), + vec![(CoreIndex(0), ParaId::from(1)), (CoreIndex(1), ParaId::from(2))] + ); + assert_eq!( + shared::Pallet::::active_validator_indices(), + vec![ + ValidatorIndex(0), + ValidatorIndex(1), + ValidatorIndex(2), + ValidatorIndex(3), + ValidatorIndex(4) + ] + ); + TestData { backed_candidates, scheduled_paras: scheduled } } @@ -1297,10 +1356,14 @@ mod sanitizers { assert_eq!( sanitize_backed_candidates::( backed_candidates.clone(), + &>::allowed_relay_parents(), has_concluded_invalid, &scheduled ), - backed_candidates + SanitizedBackedCandidates { + backed_candidates, + votes_from_disabled_were_dropped: false + } ); {} @@ -1316,12 +1379,18 @@ mod sanitizers { let has_concluded_invalid = |_idx: usize, _backed_candidate: &BackedCandidate| -> bool { false }; - assert!(sanitize_backed_candidates::( + let SanitizedBackedCandidates { + backed_candidates: sanitized_backed_candidates, + votes_from_disabled_were_dropped, + } = sanitize_backed_candidates::( backed_candidates.clone(), + &>::allowed_relay_parents(), has_concluded_invalid, - &scheduled - ) - .is_empty()); + &scheduled, + ); + + assert!(sanitized_backed_candidates.is_empty()); + assert!(!votes_from_disabled_were_dropped); }); } @@ -1343,15 +1412,113 @@ mod sanitizers { }; let has_concluded_invalid = |_idx: usize, candidate: &BackedCandidate| set.contains(&candidate.hash()); + let SanitizedBackedCandidates { + backed_candidates: sanitized_backed_candidates, + votes_from_disabled_were_dropped, + } = sanitize_backed_candidates::( + backed_candidates.clone(), + &>::allowed_relay_parents(), + has_concluded_invalid, + &scheduled, + ); + + assert_eq!(sanitized_backed_candidates.len(), backed_candidates.len() / 2); + assert!(!votes_from_disabled_were_dropped); + }); + } + + #[test] + fn disabled_non_signing_validator_doesnt_get_filtered() { + new_test_ext(MockGenesisConfig::default()).execute_with(|| { + let TestData { mut backed_candidates, scheduled_paras } = get_test_data(); + + // Disable Eve + set_disabled_validators(vec![4]); + + let before = backed_candidates.clone(); + + // Eve is disabled but no backing statement is signed by it so nothing should be + // filtered + assert!(!filter_backed_statements_from_disabled_validators::( + &mut backed_candidates, + &>::allowed_relay_parents(), + &scheduled_paras + )); + assert_eq!(backed_candidates, before); + }); + } + + #[test] + fn drop_statements_from_disabled_without_dropping_candidate() { + new_test_ext(MockGenesisConfig::default()).execute_with(|| { + let TestData { mut backed_candidates, scheduled_paras } = get_test_data(); + + // Disable Alice + set_disabled_validators(vec![0]); + + // Update `minimum_backing_votes` in HostConfig. We want `minimum_backing_votes` set + // to one so that the candidate will have enough backing votes even after dropping + // Alice's one. + let mut hc = configuration::Pallet::::config(); + hc.minimum_backing_votes = 1; + configuration::Pallet::::force_set_active_config(hc); + + // Verify the initial state is as expected + assert_eq!(backed_candidates.get(0).unwrap().validity_votes.len(), 2); assert_eq!( - sanitize_backed_candidates::( - backed_candidates.clone(), - has_concluded_invalid, - &scheduled - ) - .len(), - backed_candidates.len() / 2 + backed_candidates.get(0).unwrap().validator_indices.get(0).unwrap(), + true + ); + assert_eq!( + backed_candidates.get(0).unwrap().validator_indices.get(1).unwrap(), + true + ); + let untouched = backed_candidates.get(1).unwrap().clone(); + + assert!(filter_backed_statements_from_disabled_validators::( + &mut backed_candidates, + &>::allowed_relay_parents(), + &scheduled_paras + )); + + // there should still be two backed candidates + assert_eq!(backed_candidates.len(), 2); + // but the first one should have only one validity vote + assert_eq!(backed_candidates.get(0).unwrap().validity_votes.len(), 1); + // Validator 0 vote should be dropped, validator 1 - retained + assert_eq!( + backed_candidates.get(0).unwrap().validator_indices.get(0).unwrap(), + false ); + assert_eq!( + backed_candidates.get(0).unwrap().validator_indices.get(1).unwrap(), + true + ); + // the second candidate shouldn't be modified + assert_eq!(*backed_candidates.get(1).unwrap(), untouched); + }); + } + + #[test] + fn drop_candidate_if_all_statements_are_from_disabled() { + new_test_ext(MockGenesisConfig::default()).execute_with(|| { + let TestData { mut backed_candidates, scheduled_paras } = get_test_data(); + + // Disable Alice and Bob + set_disabled_validators(vec![0, 1]); + + // Verify the initial state is as expected + assert_eq!(backed_candidates.get(0).unwrap().validity_votes.len(), 2); + let untouched = backed_candidates.get(1).unwrap().clone(); + + assert!(filter_backed_statements_from_disabled_validators::( + &mut backed_candidates, + &>::allowed_relay_parents(), + &scheduled_paras + )); + + assert_eq!(backed_candidates.len(), 1); + assert_eq!(*backed_candidates.get(0).unwrap(), untouched); }); } } diff --git a/polkadot/runtime/parachains/src/runtime_api_impl/vstaging.rs b/polkadot/runtime/parachains/src/runtime_api_impl/vstaging.rs index 200fd57915f9..6aa455a7d0c7 100644 --- a/polkadot/runtime/parachains/src/runtime_api_impl/vstaging.rs +++ b/polkadot/runtime/parachains/src/runtime_api_impl/vstaging.rs @@ -18,29 +18,16 @@ use crate::{configuration, initializer, shared}; use primitives::{vstaging::NodeFeatures, ValidatorIndex}; -use sp_std::{collections::btree_map::BTreeMap, prelude::Vec}; +use sp_std::prelude::Vec; /// Implementation for `DisabledValidators` // CAVEAT: this should only be called on the node side // as it might produce incorrect results on session boundaries pub fn disabled_validators() -> Vec where - T: pallet_session::Config + shared::Config, + T: shared::Config, { - let shuffled_indices = >::active_validator_indices(); - // mapping from raw validator index to `ValidatorIndex` - // this computation is the same within a session, but should be cheap - let reverse_index = shuffled_indices - .iter() - .enumerate() - .map(|(i, v)| (v.0, ValidatorIndex(i as u32))) - .collect::>(); - - // we might have disabled validators who are not parachain validators - >::disabled_validators() - .iter() - .filter_map(|v| reverse_index.get(v).cloned()) - .collect() + >::disabled_validators() } /// Returns the current state of the node features. diff --git a/polkadot/runtime/parachains/src/scheduler.rs b/polkadot/runtime/parachains/src/scheduler.rs index b81b68b5745e..e3534ff9c1d2 100644 --- a/polkadot/runtime/parachains/src/scheduler.rs +++ b/polkadot/runtime/parachains/src/scheduler.rs @@ -743,4 +743,11 @@ impl Pallet { pub(crate) fn set_validator_groups(validator_groups: Vec>) { ValidatorGroups::::set(validator_groups); } + + #[cfg(test)] + pub(crate) fn set_claimqueue( + claimqueue: BTreeMap>>>>, + ) { + ClaimQueue::::set(claimqueue); + } } diff --git a/polkadot/runtime/parachains/src/shared.rs b/polkadot/runtime/parachains/src/shared.rs index ad13c9e48448..bdaffcd505f8 100644 --- a/polkadot/runtime/parachains/src/shared.rs +++ b/polkadot/runtime/parachains/src/shared.rs @@ -19,11 +19,14 @@ //! To avoid cyclic dependencies, it is important that this pallet is not //! dependent on any of the other pallets. -use frame_support::pallet_prelude::*; +use frame_support::{pallet_prelude::*, traits::DisabledValidators}; use frame_system::pallet_prelude::BlockNumberFor; use primitives::{SessionIndex, ValidatorId, ValidatorIndex}; use sp_runtime::traits::AtLeast32BitUnsigned; -use sp_std::{collections::vec_deque::VecDeque, vec::Vec}; +use sp_std::{ + collections::{btree_map::BTreeMap, vec_deque::VecDeque}, + vec::Vec, +}; use rand::{seq::SliceRandom, SeedableRng}; use rand_chacha::ChaCha20Rng; @@ -129,7 +132,9 @@ pub mod pallet { pub struct Pallet(_); #[pallet::config] - pub trait Config: frame_system::Config {} + pub trait Config: frame_system::Config { + type DisabledValidators: frame_support::traits::DisabledValidators; + } /// The current session index. #[pallet::storage] @@ -216,6 +221,25 @@ impl Pallet { Self::session_index().saturating_add(SESSION_DELAY) } + /// Fetches disabled validators list from session pallet. + /// CAVEAT: this might produce incorrect results on session boundaries + pub fn disabled_validators() -> Vec { + let shuffled_indices = Pallet::::active_validator_indices(); + // mapping from raw validator index to `ValidatorIndex` + // this computation is the same within a session, but should be cheap + let reverse_index = shuffled_indices + .iter() + .enumerate() + .map(|(i, v)| (v.0, ValidatorIndex(i as u32))) + .collect::>(); + + // we might have disabled validators who are not parachain validators + T::DisabledValidators::disabled_validators() + .iter() + .filter_map(|v| reverse_index.get(v).cloned()) + .collect() + } + /// Test function for setting the current session index. #[cfg(any(feature = "std", feature = "runtime-benchmarks", test))] pub fn set_session_index(index: SessionIndex) { @@ -239,4 +263,16 @@ impl Pallet { ActiveValidatorIndices::::set(indices); ActiveValidatorKeys::::set(keys); } + + #[cfg(test)] + pub(crate) fn add_allowed_relay_parent( + relay_parent: T::Hash, + state_root: T::Hash, + number: BlockNumberFor, + max_ancestry_len: u32, + ) { + AllowedRelayParents::::mutate(|tracker| { + tracker.update(relay_parent, state_root, number, max_ancestry_len) + }) + } } diff --git a/polkadot/runtime/rococo/src/lib.rs b/polkadot/runtime/rococo/src/lib.rs index 5ac92a737324..2593cd9ca106 100644 --- a/polkadot/runtime/rococo/src/lib.rs +++ b/polkadot/runtime/rococo/src/lib.rs @@ -911,7 +911,9 @@ impl parachains_configuration::Config for Runtime { type WeightInfo = weights::runtime_parachains_configuration::WeightInfo; } -impl parachains_shared::Config for Runtime {} +impl parachains_shared::Config for Runtime { + type DisabledValidators = Session; +} impl parachains_session_info::Config for Runtime { type ValidatorSet = Historical; diff --git a/polkadot/runtime/test-runtime/src/lib.rs b/polkadot/runtime/test-runtime/src/lib.rs index 596e65eca068..0abc780dee86 100644 --- a/polkadot/runtime/test-runtime/src/lib.rs +++ b/polkadot/runtime/test-runtime/src/lib.rs @@ -486,7 +486,9 @@ impl parachains_configuration::Config for Runtime { type WeightInfo = parachains_configuration::TestWeightInfo; } -impl parachains_shared::Config for Runtime {} +impl parachains_shared::Config for Runtime { + type DisabledValidators = Session; +} impl parachains_inclusion::Config for Runtime { type RuntimeEvent = RuntimeEvent; diff --git a/polkadot/runtime/westend/src/lib.rs b/polkadot/runtime/westend/src/lib.rs index f7ff2d5e9e1b..4211c073cbfd 100644 --- a/polkadot/runtime/westend/src/lib.rs +++ b/polkadot/runtime/westend/src/lib.rs @@ -1130,7 +1130,9 @@ impl parachains_configuration::Config for Runtime { type WeightInfo = weights::runtime_parachains_configuration::WeightInfo; } -impl parachains_shared::Config for Runtime {} +impl parachains_shared::Config for Runtime { + type DisabledValidators = Session; +} impl parachains_session_info::Config for Runtime { type ValidatorSet = Historical; diff --git a/polkadot/xcm/xcm-builder/tests/mock/mod.rs b/polkadot/xcm/xcm-builder/tests/mock/mod.rs index 968b294c6a43..5427bffe32e9 100644 --- a/polkadot/xcm/xcm-builder/tests/mock/mod.rs +++ b/polkadot/xcm/xcm-builder/tests/mock/mod.rs @@ -126,7 +126,9 @@ impl pallet_balances::Config for Runtime { type MaxFreezes = ConstU32<0>; } -impl shared::Config for Runtime {} +impl shared::Config for Runtime { + type DisabledValidators = (); +} impl configuration::Config for Runtime { type WeightInfo = configuration::TestWeightInfo; diff --git a/polkadot/xcm/xcm-simulator/example/src/relay_chain.rs b/polkadot/xcm/xcm-simulator/example/src/relay_chain.rs index 20070d192b54..1d1ee385d31e 100644 --- a/polkadot/xcm/xcm-simulator/example/src/relay_chain.rs +++ b/polkadot/xcm/xcm-simulator/example/src/relay_chain.rs @@ -120,7 +120,9 @@ impl pallet_uniques::Config for Runtime { type Helper = (); } -impl shared::Config for Runtime {} +impl shared::Config for Runtime { + type DisabledValidators = (); +} impl configuration::Config for Runtime { type WeightInfo = configuration::TestWeightInfo; diff --git a/polkadot/xcm/xcm-simulator/fuzzer/src/relay_chain.rs b/polkadot/xcm/xcm-simulator/fuzzer/src/relay_chain.rs index 085773f30737..572cee3db536 100644 --- a/polkadot/xcm/xcm-simulator/fuzzer/src/relay_chain.rs +++ b/polkadot/xcm/xcm-simulator/fuzzer/src/relay_chain.rs @@ -98,7 +98,9 @@ impl pallet_balances::Config for Runtime { type MaxFreezes = ConstU32<0>; } -impl shared::Config for Runtime {} +impl shared::Config for Runtime { + type DisabledValidators = (); +} impl configuration::Config for Runtime { type WeightInfo = configuration::TestWeightInfo; diff --git a/substrate/frame/aura/src/mock.rs b/substrate/frame/aura/src/mock.rs index 14b87089ce39..d38a8583819e 100644 --- a/substrate/frame/aura/src/mock.rs +++ b/substrate/frame/aura/src/mock.rs @@ -96,6 +96,10 @@ impl DisabledValidators for MockDisabledValidators { fn is_disabled(index: AuthorityIndex) -> bool { DisabledValidatorTestValue::get().binary_search(&index).is_ok() } + + fn disabled_validators() -> Vec { + DisabledValidatorTestValue::get() + } } impl pallet_aura::Config for Test { diff --git a/substrate/frame/contracts/mock-network/src/relay_chain.rs b/substrate/frame/contracts/mock-network/src/relay_chain.rs index 17e36eada259..cc6b2953a666 100644 --- a/substrate/frame/contracts/mock-network/src/relay_chain.rs +++ b/substrate/frame/contracts/mock-network/src/relay_chain.rs @@ -97,7 +97,9 @@ impl pallet_balances::Config for Runtime { type RuntimeFreezeReason = RuntimeFreezeReason; } -impl shared::Config for Runtime {} +impl shared::Config for Runtime { + type DisabledValidators = (); +} impl configuration::Config for Runtime { type WeightInfo = configuration::TestWeightInfo; diff --git a/substrate/frame/session/src/lib.rs b/substrate/frame/session/src/lib.rs index bf4671a247f0..178d43f596b2 100644 --- a/substrate/frame/session/src/lib.rs +++ b/substrate/frame/session/src/lib.rs @@ -918,6 +918,10 @@ impl frame_support::traits::DisabledValidators for Pallet { fn is_disabled(index: u32) -> bool { >::disabled_validators().binary_search(&index).is_ok() } + + fn disabled_validators() -> Vec { + >::disabled_validators() + } } /// Wraps the author-scraping logic for consensus engines that can recover diff --git a/substrate/frame/support/src/traits/validation.rs b/substrate/frame/support/src/traits/validation.rs index 617cdb2d3f46..4b099b2c766f 100644 --- a/substrate/frame/support/src/traits/validation.rs +++ b/substrate/frame/support/src/traits/validation.rs @@ -251,10 +251,17 @@ pub trait ValidatorRegistration { pub trait DisabledValidators { /// Returns true if the given validator is disabled. fn is_disabled(index: u32) -> bool; + + /// Returns all disabled validators + fn disabled_validators() -> Vec; } impl DisabledValidators for () { fn is_disabled(_index: u32) -> bool { false } + + fn disabled_validators() -> Vec { + vec![] + } }