diff --git a/polkadot/node/core/approval-voting/src/lib.rs b/polkadot/node/core/approval-voting/src/lib.rs index 7cea22d1a6a7a..27361df373104 100644 --- a/polkadot/node/core/approval-voting/src/lib.rs +++ b/polkadot/node/core/approval-voting/src/lib.rs @@ -21,6 +21,7 @@ //! of others. It uses this information to determine when candidates and blocks have //! been sufficiently approved to finalize. +use futures_timer::Delay; use polkadot_node_primitives::{ approval::{ v1::{BlockApprovalMeta, DelayTranche}, @@ -122,6 +123,9 @@ const APPROVAL_CHECKING_TIMEOUT: Duration = Duration::from_secs(120); const WAIT_FOR_SIGS_TIMEOUT: Duration = Duration::from_millis(500); const APPROVAL_CACHE_SIZE: u32 = 1024; +/// The maximum number of times we retry to approve a block if is still needed. +const MAX_APPROVAL_RETRIES: u32 = 16; + const APPROVAL_DELAY: Tick = 2; pub(crate) const LOG_TARGET: &str = "parachain::approval-voting"; @@ -165,6 +169,10 @@ pub struct ApprovalVotingSubsystem { metrics: Metrics, clock: Arc, spawner: Arc, + /// The maximum time we retry to approve a block if it is still needed and PoV fetch failed. + max_approval_retries: u32, + /// The backoff before we retry the approval. + retry_backoff: Duration, } #[derive(Clone)] @@ -493,6 +501,8 @@ impl ApprovalVotingSubsystem { metrics, Arc::new(SystemClock {}), spawner, + MAX_APPROVAL_RETRIES, + APPROVAL_CHECKING_TIMEOUT / 2, ) } @@ -505,6 +515,8 @@ impl ApprovalVotingSubsystem { metrics: Metrics, clock: Arc, spawner: Arc, + max_approval_retries: u32, + retry_backoff: Duration, ) -> Self { ApprovalVotingSubsystem { keystore, @@ -515,6 +527,8 @@ impl ApprovalVotingSubsystem { metrics, clock, spawner, + max_approval_retries, + retry_backoff, } } @@ -706,18 +720,53 @@ enum ApprovalOutcome { TimedOut, } +#[derive(Clone)] +struct RetryApprovalInfo { + candidate: CandidateReceipt, + backing_group: GroupIndex, + executor_params: ExecutorParams, + core_index: Option, + session_index: SessionIndex, + attempts_remaining: u32, + backoff: Duration, +} + struct ApprovalState { validator_index: ValidatorIndex, candidate_hash: CandidateHash, approval_outcome: ApprovalOutcome, + retry_info: Option, } impl ApprovalState { fn approved(validator_index: ValidatorIndex, candidate_hash: CandidateHash) -> Self { - Self { validator_index, candidate_hash, approval_outcome: ApprovalOutcome::Approved } + Self { + validator_index, + candidate_hash, + approval_outcome: ApprovalOutcome::Approved, + retry_info: None, + } } fn failed(validator_index: ValidatorIndex, candidate_hash: CandidateHash) -> Self { - Self { validator_index, candidate_hash, approval_outcome: ApprovalOutcome::Failed } + Self { + validator_index, + candidate_hash, + approval_outcome: ApprovalOutcome::Failed, + retry_info: None, + } + } + + fn failed_with_retry( + validator_index: ValidatorIndex, + candidate_hash: CandidateHash, + retry_info: Option, + ) -> Self { + Self { + validator_index, + candidate_hash, + approval_outcome: ApprovalOutcome::Failed, + retry_info, + } } } @@ -757,6 +806,7 @@ impl CurrentlyCheckingSet { candidate_hash, validator_index, approval_outcome: ApprovalOutcome::TimedOut, + retry_info: None, }, Some(approval_state) => approval_state, } @@ -1271,18 +1321,19 @@ where validator_index, candidate_hash, approval_outcome, + retry_info, } ) = approval_state; if matches!(approval_outcome, ApprovalOutcome::Approved) { let mut approvals: Vec = relay_block_hashes - .into_iter() + .iter() .map(|block_hash| Action::IssueApproval( candidate_hash, ApprovalVoteRequest { validator_index, - block_hash, + block_hash: *block_hash, }, ) ) @@ -1290,6 +1341,43 @@ where actions.append(&mut approvals); } + if let Some(retry_info) = retry_info { + for block_hash in relay_block_hashes { + if overlayed_db.load_block_entry(&block_hash).map(|block_info| block_info.is_some()).unwrap_or(false) { + let sender = to_other_subsystems.clone(); + let spawn_handle = subsystem.spawner.clone(); + let metrics = subsystem.metrics.clone(); + let retry_info = retry_info.clone(); + let executor_params = retry_info.executor_params.clone(); + let candidate = retry_info.candidate.clone(); + + currently_checking_set + .insert_relay_block_hash( + candidate_hash, + validator_index, + block_hash, + async move { + launch_approval( + sender, + spawn_handle, + metrics, + retry_info.session_index, + candidate, + validator_index, + block_hash, + retry_info.backing_group, + executor_params, + retry_info.core_index, + retry_info, + ) + .await + }, + ) + .await?; + } + } + } + actions }, (block_hash, validator_index) = delayed_approvals_timers.select_next_some() => { @@ -1340,6 +1428,8 @@ where &mut approvals_cache, &mut subsystem.mode, actions, + subsystem.max_approval_retries, + subsystem.retry_backoff, ) .await? { @@ -1389,6 +1479,8 @@ pub async fn start_approval_worker< metrics, clock, spawner, + MAX_APPROVAL_RETRIES, + APPROVAL_CHECKING_TIMEOUT / 2, ); let backend = DbBackend::new(db.clone(), approval_voting.db_config); let spawner = approval_voting.spawner.clone(); @@ -1456,6 +1548,8 @@ async fn handle_actions< approvals_cache: &mut LruMap, mode: &mut Mode, actions: Vec, + max_approval_retries: u32, + retry_backoff: Duration, ) -> SubsystemResult { let mut conclude = false; let mut actions_iter = actions.into_iter(); @@ -1542,6 +1636,16 @@ async fn handle_actions< let sender = sender.clone(); let spawn_handle = spawn_handle.clone(); + let retry = RetryApprovalInfo { + candidate: candidate.clone(), + backing_group, + executor_params: executor_params.clone(), + core_index, + session_index: session, + attempts_remaining: max_approval_retries, + backoff: retry_backoff, + }; + currently_checking_set .insert_relay_block_hash( candidate_hash, @@ -1559,6 +1663,7 @@ async fn handle_actions< backing_group, executor_params, core_index, + retry, ) .await }, @@ -3329,6 +3434,7 @@ async fn launch_approval< backing_group: GroupIndex, executor_params: ExecutorParams, core_index: Option, + retry: RetryApprovalInfo, ) -> SubsystemResult> { let (a_tx, a_rx) = oneshot::channel(); let (code_tx, code_rx) = oneshot::channel(); @@ -3360,6 +3466,7 @@ async fn launch_approval< let candidate_hash = candidate.hash(); let para_id = candidate.descriptor.para_id(); + let mut next_retry = None; gum::trace!(target: LOG_TARGET, ?candidate_hash, ?para_id, "Recovering data."); let timer = metrics.time_recover_and_approve(); @@ -3388,7 +3495,6 @@ async fn launch_approval< let background = async move { // Force the move of the timer into the background task. let _timer = timer; - let available_data = match a_rx.await { Err(_) => return ApprovalState::failed(validator_index, candidate_hash), Ok(Ok(a)) => a, @@ -3399,10 +3505,27 @@ async fn launch_approval< target: LOG_TARGET, ?para_id, ?candidate_hash, + attempts_remaining = retry.attempts_remaining, "Data unavailable for candidate {:?}", (candidate_hash, candidate.descriptor.para_id()), ); - // do nothing. we'll just be a no-show and that'll cause others to rise up. + // Availability could fail if we did not discover much of the network, so + // let's back off and order the subsystem to retry at a later point if the + // approval is still needed, because no-show wasn't covered yet. + if retry.attempts_remaining > 0 { + Delay::new(retry.backoff).await; + next_retry = Some(RetryApprovalInfo { + candidate, + backing_group, + executor_params, + core_index, + session_index, + attempts_remaining: retry.attempts_remaining - 1, + backoff: retry.backoff, + }); + } else { + next_retry = None; + } metrics_guard.take().on_approval_unavailable(); }, &RecoveryError::ChannelClosed => { @@ -3433,7 +3556,7 @@ async fn launch_approval< metrics_guard.take().on_approval_invalid(); }, } - return ApprovalState::failed(validator_index, candidate_hash) + return ApprovalState::failed_with_retry(validator_index, candidate_hash, next_retry) }, }; diff --git a/polkadot/node/core/approval-voting/src/tests.rs b/polkadot/node/core/approval-voting/src/tests.rs index be569a1de3ecb..b72993fe1a94a 100644 --- a/polkadot/node/core/approval-voting/src/tests.rs +++ b/polkadot/node/core/approval-voting/src/tests.rs @@ -78,6 +78,9 @@ const SLOT_DURATION_MILLIS: u64 = 5000; const TIMEOUT: Duration = Duration::from_millis(2000); +const NUM_APPROVAL_RETRIES: u32 = 3; +const RETRY_BACKOFF: Duration = Duration::from_millis(300); + #[derive(Clone)] struct TestSyncOracle { flag: Arc, @@ -573,6 +576,8 @@ fn test_harness>( Metrics::default(), clock.clone(), Arc::new(SpawnGlue(pool)), + NUM_APPROVAL_RETRIES, + RETRY_BACKOFF, ), assignment_criteria, backend, @@ -3202,6 +3207,20 @@ async fn recover_available_data(virtual_overseer: &mut VirtualOverseer) { ); } +async fn recover_available_data_failure(virtual_overseer: &mut VirtualOverseer) { + let available_data = RecoveryError::Unavailable; + + assert_matches!( + virtual_overseer.recv().await, + AllMessages::AvailabilityRecovery( + AvailabilityRecoveryMessage::RecoverAvailableData(_, _, _, _, tx) + ) => { + tx.send(Err(available_data)).unwrap(); + }, + "overseer did not receive recover available data message", + ); +} + struct TriggersAssignmentConfig { our_assigned_tranche: DelayTranche, assign_validator_tranche: F1, @@ -4791,6 +4810,133 @@ fn subsystem_relaunches_approval_work_on_restart() { }); } +/// Test that we retry the approval of candidate on availability failure, up to max retries. +#[test] +fn subsystem_relaunches_approval_work_on_availability_failure() { + let assignment_criteria = Box::new(MockAssignmentCriteria( + || { + let mut assignments = HashMap::new(); + + let _ = assignments.insert( + CoreIndex(0), + approval_db::v2::OurAssignment { + cert: garbage_assignment_cert_v2(AssignmentCertKindV2::RelayVRFModuloCompact { + core_bitfield: vec![CoreIndex(0), CoreIndex(2)].try_into().unwrap(), + }), + tranche: 0, + validator_index: ValidatorIndex(0), + triggered: false, + } + .into(), + ); + + let _ = assignments.insert( + CoreIndex(1), + approval_db::v2::OurAssignment { + cert: garbage_assignment_cert_v2(AssignmentCertKindV2::RelayVRFDelay { + core_index: CoreIndex(1), + }), + tranche: 0, + validator_index: ValidatorIndex(0), + triggered: false, + } + .into(), + ); + assignments + }, + |_| Ok(0), + )); + let config = HarnessConfigBuilder::default().assignment_criteria(assignment_criteria).build(); + let store = config.backend(); + + test_harness(config, |test_harness| async move { + let TestHarness { mut virtual_overseer, clock, sync_oracle_handle } = test_harness; + + setup_overseer_with_blocks_with_two_assignments_triggered( + &mut virtual_overseer, + store, + &clock, + sync_oracle_handle, + ) + .await; + + // We have two candidates for one we are going to fail the availability for up to + // max_retries and for the other we are going to succeed on the last retry, so we should + // see the approval being distributed. + assert_matches!( + overseer_recv(&mut virtual_overseer).await, + AllMessages::ApprovalDistribution(ApprovalDistributionMessage::DistributeAssignment( + _, + _, + )) => { + } + ); + + recover_available_data_failure(&mut virtual_overseer).await; + fetch_validation_code(&mut virtual_overseer).await; + + assert_matches!( + overseer_recv(&mut virtual_overseer).await, + AllMessages::ApprovalDistribution(ApprovalDistributionMessage::DistributeAssignment( + _, + _ + )) => { + } + ); + + recover_available_data_failure(&mut virtual_overseer).await; + fetch_validation_code(&mut virtual_overseer).await; + + recover_available_data_failure(&mut virtual_overseer).await; + fetch_validation_code(&mut virtual_overseer).await; + + recover_available_data_failure(&mut virtual_overseer).await; + fetch_validation_code(&mut virtual_overseer).await; + + recover_available_data_failure(&mut virtual_overseer).await; + fetch_validation_code(&mut virtual_overseer).await; + + recover_available_data_failure(&mut virtual_overseer).await; + fetch_validation_code(&mut virtual_overseer).await; + + recover_available_data_failure(&mut virtual_overseer).await; + fetch_validation_code(&mut virtual_overseer).await; + + recover_available_data(&mut virtual_overseer).await; + fetch_validation_code(&mut virtual_overseer).await; + + assert_matches!( + overseer_recv(&mut virtual_overseer).await, + AllMessages::CandidateValidation(CandidateValidationMessage::ValidateFromExhaustive { + exec_kind, + response_sender, + .. + }) if exec_kind == PvfExecKind::Approval => { + response_sender.send(Ok(ValidationResult::Valid(Default::default(), Default::default()))) + .unwrap(); + } + ); + assert_matches!( + overseer_recv(&mut virtual_overseer).await, + AllMessages::RuntimeApi(RuntimeApiMessage::Request(_, RuntimeApiRequest::ApprovalVotingParams(_, sender))) => { + let _ = sender.send(Ok(ApprovalVotingParams { + max_approval_coalesce_count: 1, + })); + } + ); + + assert_matches!( + overseer_recv(&mut virtual_overseer).await, + AllMessages::ApprovalDistribution(ApprovalDistributionMessage::DistributeApproval(_)) + ); + + // Assert that there are no more messages being sent by the subsystem + assert!(overseer_recv(&mut virtual_overseer).timeout(TIMEOUT / 2).await.is_none()); + + virtual_overseer + }); +} + // Test that cached approvals, which are candidates that we approved but we didn't issue // the signature yet because we want to coalesce it with more candidate are sent after restart. #[test] diff --git a/polkadot/node/core/candidate-validation/src/lib.rs b/polkadot/node/core/candidate-validation/src/lib.rs index 25614349486ea..2a4643031bf87 100644 --- a/polkadot/node/core/candidate-validation/src/lib.rs +++ b/polkadot/node/core/candidate-validation/src/lib.rs @@ -912,15 +912,10 @@ async fn validate_candidate_exhaustive( // invalid. Ok(ValidationResult::Invalid(InvalidCandidate::CommitmentsHashMismatch)) } else { - let core_index = candidate_receipt.descriptor.core_index(); - - match (core_index, exec_kind) { + match exec_kind { // Core selectors are optional for V2 descriptors, but we still check the // descriptor core index. - ( - Some(_core_index), - PvfExecKind::Backing(_) | PvfExecKind::BackingSystemParas(_), - ) => { + PvfExecKind::Backing(_) | PvfExecKind::BackingSystemParas(_) => { let Some(claim_queue) = maybe_claim_queue else { let error = "cannot fetch the claim queue from the runtime"; gum::warn!( @@ -937,9 +932,9 @@ async fn validate_candidate_exhaustive( { gum::warn!( target: LOG_TARGET, - ?err, candidate_hash = ?candidate_receipt.hash(), - "Candidate core index is invalid", + "Candidate core index is invalid: {}", + err ); return Ok(ValidationResult::Invalid( InvalidCandidate::InvalidCoreIndex, @@ -947,7 +942,7 @@ async fn validate_candidate_exhaustive( } }, // No checks for approvals and disputes - (_, _) => {}, + _ => {}, } Ok(ValidationResult::Valid( diff --git a/polkadot/node/core/candidate-validation/src/tests.rs b/polkadot/node/core/candidate-validation/src/tests.rs index 98e34a1cb4c13..795d7c93f8a70 100644 --- a/polkadot/node/core/candidate-validation/src/tests.rs +++ b/polkadot/node/core/candidate-validation/src/tests.rs @@ -30,8 +30,8 @@ use polkadot_node_subsystem_util::reexports::SubsystemContext; use polkadot_overseer::ActivatedLeaf; use polkadot_primitives::{ vstaging::{ - CandidateDescriptorV2, ClaimQueueOffset, CoreSelector, MutateDescriptorV2, UMPSignal, - UMP_SEPARATOR, + CandidateDescriptorV2, CandidateDescriptorVersion, ClaimQueueOffset, CoreSelector, + MutateDescriptorV2, UMPSignal, UMP_SEPARATOR, }, CandidateDescriptor, CoreIndex, GroupIndex, HeadData, Id as ParaId, OccupiedCoreAssumption, SessionInfo, UpwardMessage, ValidatorId, @@ -851,7 +851,7 @@ fn invalid_session_or_core_index() { )) .unwrap(); - // Validation doesn't fail for approvals, core/session index is not checked. + // Validation doesn't fail for disputes, core/session index is not checked. assert_matches!(v, ValidationResult::Valid(outputs, used_validation_data) => { assert_eq!(outputs.head_data, HeadData(vec![1, 1, 1])); assert_eq!(outputs.upward_messages, commitments.upward_messages); @@ -911,6 +911,69 @@ fn invalid_session_or_core_index() { assert_eq!(outputs.hrmp_watermark, 0); assert_eq!(used_validation_data, validation_data); }); + + // Test that a v1 candidate that outputs the core selector UMP signal is invalid. + let descriptor_v1 = make_valid_candidate_descriptor( + ParaId::from(1_u32), + dummy_hash(), + dummy_hash(), + pov.hash(), + validation_code.hash(), + validation_result.head_data.hash(), + dummy_hash(), + sp_keyring::Sr25519Keyring::Ferdie, + ); + let descriptor: CandidateDescriptorV2 = descriptor_v1.into(); + + perform_basic_checks(&descriptor, validation_data.max_pov_size, &pov, &validation_code.hash()) + .unwrap(); + assert_eq!(descriptor.version(), CandidateDescriptorVersion::V1); + let candidate_receipt = CandidateReceipt { descriptor, commitments_hash: commitments.hash() }; + + for exec_kind in + [PvfExecKind::Backing(dummy_hash()), PvfExecKind::BackingSystemParas(dummy_hash())] + { + let result = executor::block_on(validate_candidate_exhaustive( + Some(1), + MockValidateCandidateBackend::with_hardcoded_result(Ok(validation_result.clone())), + validation_data.clone(), + validation_code.clone(), + candidate_receipt.clone(), + Arc::new(pov.clone()), + ExecutorParams::default(), + exec_kind, + &Default::default(), + Some(Default::default()), + )) + .unwrap(); + assert_matches!(result, ValidationResult::Invalid(InvalidCandidate::InvalidCoreIndex)); + } + + // Validation doesn't fail for approvals and disputes, core/session index is not checked. + for exec_kind in [PvfExecKind::Approval, PvfExecKind::Dispute] { + let v = executor::block_on(validate_candidate_exhaustive( + Some(1), + MockValidateCandidateBackend::with_hardcoded_result(Ok(validation_result.clone())), + validation_data.clone(), + validation_code.clone(), + candidate_receipt.clone(), + Arc::new(pov.clone()), + ExecutorParams::default(), + exec_kind, + &Default::default(), + Default::default(), + )) + .unwrap(); + + assert_matches!(v, ValidationResult::Valid(outputs, used_validation_data) => { + assert_eq!(outputs.head_data, HeadData(vec![1, 1, 1])); + assert_eq!(outputs.upward_messages, commitments.upward_messages); + assert_eq!(outputs.horizontal_messages, Vec::new()); + assert_eq!(outputs.new_validation_code, Some(vec![2, 2, 2].into())); + assert_eq!(outputs.hrmp_watermark, 0); + assert_eq!(used_validation_data, validation_data); + }); + } } #[test] @@ -1407,7 +1470,7 @@ fn compressed_code_works() { ExecutorParams::default(), PvfExecKind::Backing(dummy_hash()), &Default::default(), - Default::default(), + Some(Default::default()), )); assert_matches!(v, Ok(ValidationResult::Valid(_, _))); diff --git a/polkadot/node/service/src/lib.rs b/polkadot/node/service/src/lib.rs index 227bc52539946..820cce8d083a6 100644 --- a/polkadot/node/service/src/lib.rs +++ b/polkadot/node/service/src/lib.rs @@ -944,14 +944,9 @@ pub fn new_full< secure_validator_mode, prep_worker_path, exec_worker_path, - pvf_execute_workers_max_num: execute_workers_max_num.unwrap_or_else( - || match config.chain_spec.identify_chain() { - // The intention is to use this logic for gradual increasing from 2 to 4 - // of this configuration chain by chain until it reaches production chain. - Chain::Polkadot | Chain::Kusama => 2, - Chain::Rococo | Chain::Westend | Chain::Unknown => 4, - }, - ), + // Default execution workers is 4 because we have 8 cores on the reference hardware, + // and this accounts for 50% of that cpu capacity. + pvf_execute_workers_max_num: execute_workers_max_num.unwrap_or(4), pvf_prepare_workers_soft_max_num: prepare_workers_soft_max_num.unwrap_or(1), pvf_prepare_workers_hard_max_num: prepare_workers_hard_max_num.unwrap_or(2), }) diff --git a/polkadot/node/subsystem-bench/src/lib/approval/mod.rs b/polkadot/node/subsystem-bench/src/lib/approval/mod.rs index 1b20960a3f8a6..5f1689cb226b3 100644 --- a/polkadot/node/subsystem-bench/src/lib/approval/mod.rs +++ b/polkadot/node/subsystem-bench/src/lib/approval/mod.rs @@ -891,6 +891,8 @@ fn build_overseer( state.approval_voting_parallel_metrics.approval_voting_metrics(), Arc::new(system_clock.clone()), Arc::new(SpawnGlue(spawn_task_handle.clone())), + 1, + Duration::from_secs(1), ); let approval_distribution = ApprovalDistribution::new_with_clock( diff --git a/polkadot/primitives/src/vstaging/mod.rs b/polkadot/primitives/src/vstaging/mod.rs index 271f78efe0901..c52f3539c3e53 100644 --- a/polkadot/primitives/src/vstaging/mod.rs +++ b/polkadot/primitives/src/vstaging/mod.rs @@ -505,6 +505,10 @@ pub enum CommittedCandidateReceiptError { /// Currenly only one such message is allowed. #[cfg_attr(feature = "std", error("Too many UMP signals"))] TooManyUMPSignals, + /// If the parachain runtime started sending core selectors, v1 descriptors are no longer + /// allowed. + #[cfg_attr(feature = "std", error("Version 1 receipt does not support core selectors"))] + CoreSelectorWithV1Decriptor, } macro_rules! impl_getter { @@ -603,15 +607,25 @@ impl CommittedCandidateReceiptV2 { &self, cores_per_para: &TransposedClaimQueue, ) -> Result<(), CommittedCandidateReceiptError> { + let maybe_core_selector = self.commitments.core_selector()?; + match self.descriptor.version() { - // Don't check v1 descriptors. - CandidateDescriptorVersion::V1 => return Ok(()), + CandidateDescriptorVersion::V1 => { + // If the parachain runtime started sending core selectors, v1 descriptors are no + // longer allowed. + if maybe_core_selector.is_some() { + return Err(CommittedCandidateReceiptError::CoreSelectorWithV1Decriptor) + } else { + // Nothing else to check for v1 descriptors. + return Ok(()) + } + }, CandidateDescriptorVersion::V2 => {}, CandidateDescriptorVersion::Unknown => return Err(CommittedCandidateReceiptError::UnknownVersion(self.descriptor.version)), } - let (maybe_core_index_selector, cq_offset) = self.commitments.core_selector()?.map_or_else( + let (maybe_core_index_selector, cq_offset) = maybe_core_selector.map_or_else( || (None, ClaimQueueOffset(DEFAULT_CLAIM_QUEUE_OFFSET)), |(sel, off)| (Some(sel), off), ); @@ -1207,8 +1221,7 @@ mod tests { assert_eq!(new_ccr.hash(), v2_ccr.hash()); } - // Only check descriptor `core_index` field of v2 descriptors. If it is v1, that field - // will be garbage. + // V1 descriptors are forbidden once the parachain runtime started sending UMP signals. #[test] fn test_v1_descriptors_with_ump_signal() { let mut ccr = dummy_old_committed_candidate_receipt(); @@ -1234,9 +1247,12 @@ mod tests { cq.insert(CoreIndex(0), vec![v1_ccr.descriptor.para_id()].into()); cq.insert(CoreIndex(1), vec![v1_ccr.descriptor.para_id()].into()); - assert!(v1_ccr.check_core_index(&transpose_claim_queue(cq)).is_ok()); - assert_eq!(v1_ccr.descriptor.core_index(), None); + + assert_eq!( + v1_ccr.check_core_index(&transpose_claim_queue(cq)), + Err(CommittedCandidateReceiptError::CoreSelectorWithV1Decriptor) + ); } #[test] diff --git a/polkadot/runtime/rococo/src/lib.rs b/polkadot/runtime/rococo/src/lib.rs index cab4394eb5a8d..e5d703700fee5 100644 --- a/polkadot/runtime/rococo/src/lib.rs +++ b/polkadot/runtime/rococo/src/lib.rs @@ -2276,7 +2276,7 @@ sp_api::impl_runtime_apis! { } fn current_set_id() -> fg_primitives::SetId { - Grandpa::current_set_id() + pallet_grandpa::CurrentSetId::::get() } fn submit_report_equivocation_unsigned_extrinsic( diff --git a/polkadot/runtime/test-runtime/src/lib.rs b/polkadot/runtime/test-runtime/src/lib.rs index 82564d5c278ca..4f9ba8d8508cd 100644 --- a/polkadot/runtime/test-runtime/src/lib.rs +++ b/polkadot/runtime/test-runtime/src/lib.rs @@ -1186,7 +1186,7 @@ sp_api::impl_runtime_apis! { } fn current_set_id() -> fg_primitives::SetId { - Grandpa::current_set_id() + pallet_grandpa::CurrentSetId::::get() } fn submit_report_equivocation_unsigned_extrinsic( diff --git a/polkadot/runtime/westend/src/lib.rs b/polkadot/runtime/westend/src/lib.rs index 166f3fc42eefe..9d77a5e5eea1f 100644 --- a/polkadot/runtime/westend/src/lib.rs +++ b/polkadot/runtime/westend/src/lib.rs @@ -2300,7 +2300,7 @@ sp_api::impl_runtime_apis! { } fn current_set_id() -> fg_primitives::SetId { - Grandpa::current_set_id() + pallet_grandpa::CurrentSetId::::get() } fn submit_report_equivocation_unsigned_extrinsic( diff --git a/prdoc/pr_4529.prdoc b/prdoc/pr_4529.prdoc new file mode 100644 index 0000000000000..32beea17ad6b6 --- /dev/null +++ b/prdoc/pr_4529.prdoc @@ -0,0 +1,22 @@ +# Schema: Polkadot SDK PRDoc Schema (prdoc) v1.0.0 +# See doc at https://raw.githubusercontent.com/paritytech/polkadot-sdk/master/prdoc/schema_user.json + +title: Removed `pallet::getter` usage from pallet-grandpa + +doc: + - audience: Runtime Dev + description: | + This PR removed the `pallet::getter`s from `pallet-grandpa`. + The syntax `StorageItem::::get()` should be used instead + +crates: + - name: pallet-grandpa + bump: minor + - name: kitchensink-runtime + bump: none + - name: westend-runtime + bump: none + - name: polkadot-test-runtime + bump: none + - name: rococo-runtime + bump: none diff --git a/prdoc/pr_6647.prdoc b/prdoc/pr_6647.prdoc new file mode 100644 index 0000000000000..47af9924ef1c0 --- /dev/null +++ b/prdoc/pr_6647.prdoc @@ -0,0 +1,8 @@ +title: '`fatxpool`: proper handling of priorities when mempool is full' +doc: +- audience: Node Dev + description: |- + Higher-priority transactions can now replace lower-priority transactions even when the internal _tx_mem_pool_ is full. +crates: +- name: sc-transaction-pool + bump: minor diff --git a/prdoc/pr_6807.prdoc b/prdoc/pr_6807.prdoc new file mode 100644 index 0000000000000..b9564dfb2fe26 --- /dev/null +++ b/prdoc/pr_6807.prdoc @@ -0,0 +1,19 @@ +# Schema: Polkadot SDK PRDoc Schema (prdoc) v1.0.0 +# See doc at https://raw.githubusercontent.com/paritytech/polkadot-sdk/master/prdoc/schema_user.json + +title: Retry approval on availability failure if the check is still needed + +doc: + - audience: Node Dev + description: | + Recovering the POV can fail in situation where the node just restart and the DHT topology + wasn't fully discovered yet, so the current node can't connect to most of its Peers. + This is bad because for gossiping the assignment you need to be connected to just a few + peers, so because we can't approve the candidate other nodes will see this as a no show. + Fix it by retrying to approve a candidate for a fixed number of atttempts if the block is + still needed. + + +crates: + - name: polkadot-node-core-approval-voting + bump: minor diff --git a/prdoc/pr_7102.prdoc b/prdoc/pr_7102.prdoc new file mode 100644 index 0000000000000..b1923aafc3db4 --- /dev/null +++ b/prdoc/pr_7102.prdoc @@ -0,0 +1,8 @@ +title: '`fatxpool`: rotator cache size now depends on pool''s limits' +doc: +- audience: Node Dev + description: |- + This PR modifies the hard-coded size of extrinsics cache within `PoolRotator` to be inline with pool limits. It only applies to fork-aware transaction pool. For the legacy (single-state) transaction pool the logic remains untouched. +crates: +- name: sc-transaction-pool + bump: minor diff --git a/prdoc/pr_7116.prdoc b/prdoc/pr_7116.prdoc new file mode 100644 index 0000000000000..95a5254778a4d --- /dev/null +++ b/prdoc/pr_7116.prdoc @@ -0,0 +1,8 @@ +title: Increase the number of pvf execution workers from 2 to 4 +doc: +- audience: Node Dev + description: |- + Increase the number of pvf execution workers from 2 to 4. +crates: +- name: polkadot-service + bump: patch diff --git a/prdoc/pr_7127.prdoc b/prdoc/pr_7127.prdoc new file mode 100644 index 0000000000000..761ddd04dbe15 --- /dev/null +++ b/prdoc/pr_7127.prdoc @@ -0,0 +1,9 @@ +title: 'Forbid v1 descriptors with UMP signals' +doc: +- audience: [Runtime Dev, Node Dev] + description: Adds a check that parachain candidates do not send out UMP signals with v1 descriptors. +crates: +- name: polkadot-node-core-candidate-validation + bump: minor +- name: polkadot-primitives + bump: major diff --git a/prdoc/pr_7133.prdoc b/prdoc/pr_7133.prdoc new file mode 100644 index 0000000000000..ca0d2bb0bd483 --- /dev/null +++ b/prdoc/pr_7133.prdoc @@ -0,0 +1,15 @@ +# Schema: Polkadot SDK PRDoc Schema (prdoc) v1.0.0 +# See doc at https://raw.githubusercontent.com/paritytech/polkadot-sdk/master/prdoc/schema_user.json + +title: Sufix litep2p to the identify agent version for visibility + +doc: + - audience: [Node Dev, Node Operator] + description: | + This PR adds the `(litep2p)` suffix to the agent version (user agent) of the identify protocol. + The change is needed to gain visibility into network backends and determine exactly the number of validators that are running litep2p. + Using tools like subp2p-explorer, we can determine if the validators are running litep2p nodes. + +crates: +- name: sc-network + bump: patch diff --git a/substrate/bin/node/runtime/src/lib.rs b/substrate/bin/node/runtime/src/lib.rs index 6735b09710fcb..b23845fd701ba 100644 --- a/substrate/bin/node/runtime/src/lib.rs +++ b/substrate/bin/node/runtime/src/lib.rs @@ -2979,7 +2979,7 @@ impl_runtime_apis! { } fn current_set_id() -> sp_consensus_grandpa::SetId { - Grandpa::current_set_id() + pallet_grandpa::CurrentSetId::::get() } fn submit_report_equivocation_unsigned_extrinsic( diff --git a/substrate/client/network/src/litep2p/discovery.rs b/substrate/client/network/src/litep2p/discovery.rs index 2bea2e5a80dce..b55df374f60ec 100644 --- a/substrate/client/network/src/litep2p/discovery.rs +++ b/substrate/client/network/src/litep2p/discovery.rs @@ -254,7 +254,7 @@ impl Discovery { _peerstore_handle: Arc, ) -> (Self, PingConfig, IdentifyConfig, KademliaConfig, Option) { let (ping_config, ping_event_stream) = PingConfig::default(); - let user_agent = format!("{} ({})", config.client_version, config.node_name); + let user_agent = format!("{} ({}) (litep2p)", config.client_version, config.node_name); let (identify_config, identify_event_stream) = IdentifyConfig::new("/substrate/1.0".to_string(), Some(user_agent)); diff --git a/substrate/client/transaction-pool/benches/basics.rs b/substrate/client/transaction-pool/benches/basics.rs index 5e40b0fb72d6b..5ba9dd40c1568 100644 --- a/substrate/client/transaction-pool/benches/basics.rs +++ b/substrate/client/transaction-pool/benches/basics.rs @@ -197,14 +197,22 @@ fn benchmark_main(c: &mut Criterion) { c.bench_function("sequential 50 tx", |b| { b.iter(|| { let api = Arc::from(TestApi::new_dependant()); - bench_configured(Pool::new(Default::default(), true.into(), api.clone()), 50, api); + bench_configured( + Pool::new_with_staticly_sized_rotator(Default::default(), true.into(), api.clone()), + 50, + api, + ); }); }); c.bench_function("random 100 tx", |b| { b.iter(|| { let api = Arc::from(TestApi::default()); - bench_configured(Pool::new(Default::default(), true.into(), api.clone()), 100, api); + bench_configured( + Pool::new_with_staticly_sized_rotator(Default::default(), true.into(), api.clone()), + 100, + api, + ); }); }); } diff --git a/substrate/client/transaction-pool/src/common/tests.rs b/substrate/client/transaction-pool/src/common/tests.rs index b00cf5fbfede9..7f2cbe24d8ef6 100644 --- a/substrate/client/transaction-pool/src/common/tests.rs +++ b/substrate/client/transaction-pool/src/common/tests.rs @@ -222,5 +222,5 @@ pub(crate) fn uxt(transfer: Transfer) -> Extrinsic { pub(crate) fn pool() -> (Pool, Arc) { let api = Arc::new(TestApi::default()); - (Pool::new(Default::default(), true.into(), api.clone()), api) + (Pool::new_with_staticly_sized_rotator(Default::default(), true.into(), api.clone()), api) } diff --git a/substrate/client/transaction-pool/src/fork_aware_txpool/dropped_watcher.rs b/substrate/client/transaction-pool/src/fork_aware_txpool/dropped_watcher.rs index 7679e3b169d2e..bf61558b00b0d 100644 --- a/substrate/client/transaction-pool/src/fork_aware_txpool/dropped_watcher.rs +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/dropped_watcher.rs @@ -53,11 +53,13 @@ pub struct DroppedTransaction { } impl DroppedTransaction { - fn new_usurped(tx_hash: Hash, by: Hash) -> Self { + /// Creates a new instance with reason set to `DroppedReason::Usurped(by)`. + pub fn new_usurped(tx_hash: Hash, by: Hash) -> Self { Self { reason: DroppedReason::Usurped(by), tx_hash } } - fn new_enforced_by_limts(tx_hash: Hash) -> Self { + /// Creates a new instance with reason set to `DroppedReason::LimitsEnforced`. + pub fn new_enforced_by_limts(tx_hash: Hash) -> Self { Self { reason: DroppedReason::LimitsEnforced, tx_hash } } } @@ -256,11 +258,13 @@ where self.future_transaction_views.entry(tx_hash).or_default().insert(block_hash); }, TransactionStatus::Ready | TransactionStatus::InBlock(..) => { - // note: if future transaction was once seens as the ready we may want to treat it - // as ready transactions. Unreferenced future transactions are more likely to be - // removed when the last referencing view is removed then ready transactions. - // Transcaction seen as ready is likely quite close to be included in some - // future fork. + // note: if future transaction was once seen as the ready we may want to treat it + // as ready transaction. The rationale behind this is as follows: we want to remove + // unreferenced future transactions when the last referencing view is removed (to + // avoid clogging mempool). For ready transactions we prefer to keep them in mempool + // even if no view is currently referencing them. Future transcaction once seen as + // ready is likely quite close to be included in some future fork (it is close to be + // ready, so we make exception and treat such transaction as ready). if let Some(mut views) = self.future_transaction_views.remove(&tx_hash) { views.insert(block_hash); self.ready_transaction_views.insert(tx_hash, views); @@ -329,14 +333,14 @@ where let stream_map = futures::stream::unfold(ctx, |mut ctx| async move { loop { if let Some(dropped) = ctx.get_pending_dropped_transaction() { - debug!("dropped_watcher: sending out (pending): {dropped:?}"); + trace!("dropped_watcher: sending out (pending): {dropped:?}"); return Some((dropped, ctx)); } tokio::select! { biased; Some(event) = next_event(&mut ctx.stream_map) => { if let Some(dropped) = ctx.handle_event(event.0, event.1) { - debug!("dropped_watcher: sending out: {dropped:?}"); + trace!("dropped_watcher: sending out: {dropped:?}"); return Some((dropped, ctx)); } }, diff --git a/substrate/client/transaction-pool/src/fork_aware_txpool/fork_aware_txpool.rs b/substrate/client/transaction-pool/src/fork_aware_txpool/fork_aware_txpool.rs index 4ec87f1fefa40..7660457182520 100644 --- a/substrate/client/transaction-pool/src/fork_aware_txpool/fork_aware_txpool.rs +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/fork_aware_txpool.rs @@ -31,7 +31,10 @@ use crate::{ api::FullChainApi, common::log_xt::log_xt_trace, enactment_state::{EnactmentAction, EnactmentState}, - fork_aware_txpool::{dropped_watcher::DroppedReason, revalidation_worker}, + fork_aware_txpool::{ + dropped_watcher::{DroppedReason, DroppedTransaction}, + revalidation_worker, + }, graph::{ self, base_pool::{TimedTransactionSource, Transaction}, @@ -49,14 +52,16 @@ use futures::{ use parking_lot::Mutex; use prometheus_endpoint::Registry as PrometheusRegistry; use sc_transaction_pool_api::{ - ChainEvent, ImportNotificationStream, MaintainedTransactionPool, PoolStatus, TransactionFor, - TransactionPool, TransactionSource, TransactionStatusStreamFor, TxHash, + error::Error as TxPoolApiError, ChainEvent, ImportNotificationStream, + MaintainedTransactionPool, PoolStatus, TransactionFor, TransactionPool, TransactionPriority, + TransactionSource, TransactionStatusStreamFor, TxHash, }; use sp_blockchain::{HashAndNumber, TreeRoute}; use sp_core::traits::SpawnEssentialNamed; use sp_runtime::{ generic::BlockId, traits::{Block as BlockT, NumberFor}, + transaction_validity::{TransactionValidityError, ValidTransaction}, }; use std::{ collections::{HashMap, HashSet}, @@ -287,7 +292,7 @@ where DroppedReason::LimitsEnforced => {}, }; - mempool.remove_dropped_transaction(&dropped_tx_hash).await; + mempool.remove_transaction(&dropped_tx_hash); view_store.listener.transaction_dropped(dropped); import_notification_sink.clean_notified_items(&[dropped_tx_hash]); } @@ -318,7 +323,7 @@ where pool_api.clone(), listener.clone(), metrics.clone(), - TXMEMPOOL_TRANSACTION_LIMIT_MULTIPLIER * (options.ready.count + options.future.count), + TXMEMPOOL_TRANSACTION_LIMIT_MULTIPLIER * options.total_count(), options.ready.total_bytes + options.future.total_bytes, )); @@ -598,7 +603,7 @@ where /// out: /// [ Ok(xth0), Ok(xth1), Err ] /// ``` -fn reduce_multiview_result(input: HashMap>>) -> Vec> { +fn reduce_multiview_result(input: HashMap>>) -> Vec> { let mut values = input.values(); let Some(first) = values.next() else { return Default::default(); @@ -650,9 +655,28 @@ where let mempool_results = self.mempool.extend_unwatched(source, &xts); if view_store.is_empty() { - return Ok(mempool_results.into_iter().map(|r| r.map(|r| r.hash)).collect::>()) + return Ok(mempool_results + .into_iter() + .map(|r| r.map(|r| r.hash).map_err(Into::into)) + .collect::>()) } + // Submit all the transactions to the mempool + let retries = mempool_results + .into_iter() + .zip(xts.clone()) + .map(|(result, xt)| async move { + match result { + Err(TxPoolApiError::ImmediatelyDropped) => + self.attempt_transaction_replacement(source, false, xt).await, + _ => result, + } + }) + .collect::>(); + + let mempool_results = futures::future::join_all(retries).await; + + // Collect transactions that were successfully submitted to the mempool... let to_be_submitted = mempool_results .iter() .zip(xts) @@ -664,22 +688,47 @@ where self.metrics .report(|metrics| metrics.submitted_transactions.inc_by(to_be_submitted.len() as _)); + // ... and submit them to the view_store. Please note that transactions rejected by mempool + // are not sent here. let mempool = self.mempool.clone(); let results_map = view_store.submit(to_be_submitted.into_iter()).await; let mut submission_results = reduce_multiview_result(results_map).into_iter(); + // Note for composing final result: + // + // For each failed insertion into the mempool, the mempool result should be placed into + // the returned vector. + // + // For each successful insertion into the mempool, the corresponding + // view_store submission result needs to be examined: + // - If there is an error during view_store submission, the transaction is removed from + // the mempool, and the final result recorded in the vector for this transaction is the + // view_store submission error. + // + // - If the view_store submission is successful, the transaction priority is updated in the + // mempool. + // + // Finally, it collects the hashes of updated transactions or submission errors (either + // from the mempool or view_store) into a returned vector. Ok(mempool_results .into_iter() .map(|result| { - result.and_then(|insertion| { - submission_results - .next() - .expect("The number of Ok results in mempool is exactly the same as the size of to-views-submission result. qed.") - .inspect_err(|_| - mempool.remove(insertion.hash) - ) + result + .map_err(Into::into) + .and_then(|insertion| { + submission_results + .next() + .expect("The number of Ok results in mempool is exactly the same as the size of view_store submission result. qed.") + .inspect_err(|_|{ + mempool.remove_transaction(&insertion.hash); + }) }) + }) + .map(|r| r.map(|r| { + mempool.update_transaction_priority(&r); + r.hash() + })) .collect::>()) } @@ -712,10 +761,13 @@ where ) -> Result>>, Self::Error> { log::trace!(target: LOG_TARGET, "[{:?}] fatp::submit_and_watch views:{}", self.tx_hash(&xt), self.active_views_count()); let xt = Arc::from(xt); - let InsertionInfo { hash: xt_hash, source: timed_source } = + + let InsertionInfo { hash: xt_hash, source: timed_source, .. } = match self.mempool.push_watched(source, xt.clone()) { Ok(result) => result, - Err(e) => return Err(e), + Err(TxPoolApiError::ImmediatelyDropped) => + self.attempt_transaction_replacement(source, true, xt.clone()).await?, + Err(e) => return Err(e.into()), }; self.metrics.report(|metrics| metrics.submitted_transactions.inc()); @@ -723,7 +775,13 @@ where self.view_store .submit_and_watch(at, timed_source, xt) .await - .inspect_err(|_| self.mempool.remove(xt_hash)) + .inspect_err(|_| { + self.mempool.remove_transaction(&xt_hash); + }) + .map(|mut outcome| { + self.mempool.update_transaction_priority(&outcome); + outcome.expect_watcher() + }) } /// Intended to remove transactions identified by the given hashes, and any dependent @@ -828,22 +886,16 @@ where } } -impl sc_transaction_pool_api::LocalTransactionPool - for ForkAwareTxPool, Block> +impl sc_transaction_pool_api::LocalTransactionPool + for ForkAwareTxPool where Block: BlockT, + ChainApi: 'static + graph::ChainApi, ::Hash: Unpin, - Client: sp_api::ProvideRuntimeApi - + sc_client_api::BlockBackend - + sc_client_api::blockchain::HeaderBackend - + sp_runtime::traits::BlockIdTo - + sp_blockchain::HeaderMetadata, - Client: Send + Sync + 'static, - Client::Api: sp_transaction_pool::runtime_api::TaggedTransactionQueue, { type Block = Block; - type Hash = ExtrinsicHash>; - type Error = as graph::ChainApi>::Error; + type Hash = ExtrinsicHash; + type Error = ChainApi::Error; fn submit_local( &self, @@ -852,12 +904,29 @@ where ) -> Result { log::debug!(target: LOG_TARGET, "fatp::submit_local views:{}", self.active_views_count()); let xt = Arc::from(xt); - let InsertionInfo { hash: xt_hash, .. } = self - .mempool - .extend_unwatched(TransactionSource::Local, &[xt.clone()]) - .remove(0)?; - self.view_store.submit_local(xt).or_else(|_| Ok(xt_hash)) + let result = + self.mempool.extend_unwatched(TransactionSource::Local, &[xt.clone()]).remove(0); + + let insertion = match result { + Err(TxPoolApiError::ImmediatelyDropped) => self.attempt_transaction_replacement_sync( + TransactionSource::Local, + false, + xt.clone(), + ), + _ => result, + }?; + + self.view_store + .submit_local(xt) + .inspect_err(|_| { + self.mempool.remove_transaction(&insertion.hash); + }) + .map(|outcome| { + self.mempool.update_transaction_priority(&outcome); + outcome.hash() + }) + .or_else(|_| Ok(insertion.hash)) } } @@ -1109,7 +1178,11 @@ where .await .into_iter() .zip(hashes) - .map(|(result, tx_hash)| result.or_else(|_| Err(tx_hash))) + .map(|(result, tx_hash)| { + result + .map(|outcome| self.mempool.update_transaction_priority(&outcome.into())) + .or_else(|_| Err(tx_hash)) + }) .collect::>(); let submitted_count = watched_results.len(); @@ -1131,7 +1204,7 @@ where for result in watched_results { if let Err(tx_hash) = result { self.view_store.listener.invalidate_transactions(&[tx_hash]); - self.mempool.remove(tx_hash); + self.mempool.remove_transaction(&tx_hash); } } } @@ -1263,6 +1336,101 @@ where fn tx_hash(&self, xt: &TransactionFor) -> TxHash { self.api.hash_and_length(xt).0 } + + /// Attempts to find and replace a lower-priority transaction in the transaction pool with a new + /// one. + /// + /// This asynchronous function verifies the new transaction against the most recent view. If a + /// transaction with a lower priority exists in the transaction pool, it is replaced with the + /// new transaction. + /// + /// If no lower-priority transaction is found, the function returns an error indicating the + /// transaction was dropped immediately. + async fn attempt_transaction_replacement( + &self, + source: TransactionSource, + watched: bool, + xt: ExtrinsicFor, + ) -> Result>, TxPoolApiError> { + let at = self + .view_store + .most_recent_view + .read() + .ok_or(TxPoolApiError::ImmediatelyDropped)?; + + let (best_view, _) = self + .view_store + .get_view_at(at, false) + .ok_or(TxPoolApiError::ImmediatelyDropped)?; + + let (xt_hash, validated_tx) = best_view + .pool + .verify_one( + best_view.at.hash, + best_view.at.number, + TimedTransactionSource::from_transaction_source(source, false), + xt.clone(), + crate::graph::CheckBannedBeforeVerify::Yes, + ) + .await; + + let Some(priority) = validated_tx.priority() else { + return Err(TxPoolApiError::ImmediatelyDropped) + }; + + self.attempt_transaction_replacement_inner(xt, xt_hash, priority, source, watched) + } + + /// Sync version of [`Self::attempt_transaction_replacement`]. + fn attempt_transaction_replacement_sync( + &self, + source: TransactionSource, + watched: bool, + xt: ExtrinsicFor, + ) -> Result>, TxPoolApiError> { + let at = self + .view_store + .most_recent_view + .read() + .ok_or(TxPoolApiError::ImmediatelyDropped)?; + + let ValidTransaction { priority, .. } = self + .api + .validate_transaction_blocking(at, TransactionSource::Local, Arc::from(xt.clone())) + .map_err(|_| TxPoolApiError::ImmediatelyDropped)? + .map_err(|e| match e { + TransactionValidityError::Invalid(i) => TxPoolApiError::InvalidTransaction(i), + TransactionValidityError::Unknown(u) => TxPoolApiError::UnknownTransaction(u), + })?; + let xt_hash = self.hash_of(&xt); + self.attempt_transaction_replacement_inner(xt, xt_hash, priority, source, watched) + } + + fn attempt_transaction_replacement_inner( + &self, + xt: ExtrinsicFor, + tx_hash: ExtrinsicHash, + priority: TransactionPriority, + source: TransactionSource, + watched: bool, + ) -> Result>, TxPoolApiError> { + let insertion_info = + self.mempool.try_insert_with_replacement(xt, priority, source, watched)?; + + for worst_hash in &insertion_info.removed { + log::trace!(target: LOG_TARGET, "removed: {worst_hash:?} replaced by {tx_hash:?}"); + self.view_store + .listener + .transaction_dropped(DroppedTransaction::new_enforced_by_limts(*worst_hash)); + + self.view_store + .remove_transaction_subtree(*worst_hash, |listener, removed_tx_hash| { + listener.limits_enforced(&removed_tx_hash); + }); + } + + return Ok(insertion_info) + } } #[async_trait] @@ -1410,7 +1578,7 @@ mod reduce_multiview_result_tests { fn empty() { sp_tracing::try_init_simple(); let input = HashMap::default(); - let r = reduce_multiview_result::(input); + let r = reduce_multiview_result::(input); assert!(r.is_empty()); } diff --git a/substrate/client/transaction-pool/src/fork_aware_txpool/tx_mem_pool.rs b/substrate/client/transaction-pool/src/fork_aware_txpool/tx_mem_pool.rs index 989ae4425dc48..c8a4d0c72dd36 100644 --- a/substrate/client/transaction-pool/src/fork_aware_txpool/tx_mem_pool.rs +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/tx_mem_pool.rs @@ -26,7 +26,10 @@ //! it), while on other forks tx can be valid. Depending on which view is chosen to be cloned, //! such transaction could not be present in the newly created view. -use super::{metrics::MetricsLink as PrometheusMetrics, multi_view_listener::MultiViewListener}; +use super::{ + metrics::MetricsLink as PrometheusMetrics, multi_view_listener::MultiViewListener, + view_store::ViewStoreSubmitOutcome, +}; use crate::{ common::log_xt::log_xt_trace, graph, @@ -35,15 +38,20 @@ use crate::{ }; use futures::FutureExt; use itertools::Itertools; -use sc_transaction_pool_api::TransactionSource; +use parking_lot::RwLock; +use sc_transaction_pool_api::{TransactionPriority, TransactionSource}; use sp_blockchain::HashAndNumber; use sp_runtime::{ traits::Block as BlockT, transaction_validity::{InvalidTransaction, TransactionValidityError}, }; use std::{ + cmp::Ordering, collections::HashMap, - sync::{atomic, atomic::AtomicU64, Arc}, + sync::{ + atomic::{self, AtomicU64}, + Arc, + }, time::Instant, }; @@ -77,6 +85,9 @@ where source: TimedTransactionSource, /// When the transaction was revalidated, used to periodically revalidate the mem pool buffer. validated_at: AtomicU64, + /// Priority of transaction at some block. It is assumed it will not be changed often. None if + /// not known. + priority: RwLock>, //todo: we need to add future / ready status at finalized block. //If future transactions are stuck in tx_mem_pool (due to limits being hit), we need a means // to replace them somehow with newly coming transactions. @@ -101,23 +112,50 @@ where /// Creates a new instance of wrapper for unwatched transaction. fn new_unwatched(source: TransactionSource, tx: ExtrinsicFor, bytes: usize) -> Self { - Self { - watched: false, - tx, - source: TimedTransactionSource::from_transaction_source(source, true), - validated_at: AtomicU64::new(0), - bytes, - } + Self::new(false, source, tx, bytes) } /// Creates a new instance of wrapper for watched transaction. fn new_watched(source: TransactionSource, tx: ExtrinsicFor, bytes: usize) -> Self { + Self::new(true, source, tx, bytes) + } + + /// Creates a new instance of wrapper for a transaction with no priority. + fn new( + watched: bool, + source: TransactionSource, + tx: ExtrinsicFor, + bytes: usize, + ) -> Self { + Self::new_with_optional_priority(watched, source, tx, bytes, None) + } + + /// Creates a new instance of wrapper for a transaction with given priority. + fn new_with_priority( + watched: bool, + source: TransactionSource, + tx: ExtrinsicFor, + bytes: usize, + priority: TransactionPriority, + ) -> Self { + Self::new_with_optional_priority(watched, source, tx, bytes, Some(priority)) + } + + /// Creates a new instance of wrapper for a transaction with optional priority. + fn new_with_optional_priority( + watched: bool, + source: TransactionSource, + tx: ExtrinsicFor, + bytes: usize, + priority: Option, + ) -> Self { Self { - watched: true, + watched, tx, source: TimedTransactionSource::from_transaction_source(source, true), validated_at: AtomicU64::new(0), bytes, + priority: priority.into(), } } @@ -132,6 +170,11 @@ where pub(crate) fn source(&self) -> TimedTransactionSource { self.source.clone() } + + /// Returns the priority of the transaction. + pub(crate) fn priority(&self) -> Option { + *self.priority.read() + } } impl Size for Arc> @@ -191,11 +234,15 @@ where pub(super) struct InsertionInfo { pub(super) hash: Hash, pub(super) source: TimedTransactionSource, + pub(super) removed: Vec, } impl InsertionInfo { fn new(hash: Hash, source: TimedTransactionSource) -> Self { - Self { hash, source } + Self::new_with_removed(hash, source, Default::default()) + } + fn new_with_removed(hash: Hash, source: TimedTransactionSource, removed: Vec) -> Self { + Self { hash, source, removed } } } @@ -279,27 +326,109 @@ where &self, hash: ExtrinsicHash, tx: TxInMemPool, - ) -> Result>, ChainApi::Error> { - let bytes = self.transactions.bytes(); + ) -> Result>, sc_transaction_pool_api::error::Error> { let mut transactions = self.transactions.write(); + + let bytes = self.transactions.bytes(); + let result = match ( - !self.is_limit_exceeded(transactions.len() + 1, bytes + tx.bytes), + self.is_limit_exceeded(transactions.len() + 1, bytes + tx.bytes), transactions.contains_key(&hash), ) { - (true, false) => { + (false, false) => { let source = tx.source(); transactions.insert(hash, Arc::from(tx)); Ok(InsertionInfo::new(hash, source)) }, (_, true) => - Err(sc_transaction_pool_api::error::Error::AlreadyImported(Box::new(hash)).into()), - (false, _) => Err(sc_transaction_pool_api::error::Error::ImmediatelyDropped.into()), + Err(sc_transaction_pool_api::error::Error::AlreadyImported(Box::new(hash))), + (true, _) => Err(sc_transaction_pool_api::error::Error::ImmediatelyDropped), }; log::trace!(target: LOG_TARGET, "[{:?}] mempool::try_insert: {:?}", hash, result.as_ref().map(|r| r.hash)); result } + /// Attempts to insert a new transaction in the memory pool and drop some worse existing + /// transactions. + /// + /// A "worse" transaction means transaction with lower priority, or older transaction with the + /// same prio. + /// + /// This operation will not overflow the limit of the mempool. It means that cumulative + /// size of removed transactions will be equal (or greated) then size of newly inserted + /// transaction. + /// + /// Returns a `Result` containing `InsertionInfo` if the new transaction is successfully + /// inserted; otherwise, returns an appropriate error indicating the failure. + pub(super) fn try_insert_with_replacement( + &self, + new_tx: ExtrinsicFor, + priority: TransactionPriority, + source: TransactionSource, + watched: bool, + ) -> Result>, sc_transaction_pool_api::error::Error> { + let (hash, length) = self.api.hash_and_length(&new_tx); + let new_tx = TxInMemPool::new_with_priority(watched, source, new_tx, length, priority); + if new_tx.bytes > self.max_transactions_total_bytes { + return Err(sc_transaction_pool_api::error::Error::ImmediatelyDropped); + } + + let mut transactions = self.transactions.write(); + + if transactions.contains_key(&hash) { + return Err(sc_transaction_pool_api::error::Error::AlreadyImported(Box::new(hash))); + } + + let mut sorted = transactions + .iter() + .filter_map(|(h, v)| v.priority().map(|_| (*h, v.clone()))) + .collect::>(); + + // When pushing higher prio transaction, we need to find a number of lower prio txs, such + // that the sum of their bytes is ge then size of new tx. Otherwise we could overflow size + // limits. Naive way to do it - rev-sort by priority and eat the tail. + + // reverse (oldest, lowest prio last) + sorted.sort_by(|(_, a), (_, b)| match b.priority().cmp(&a.priority()) { + Ordering::Equal => match (a.source.timestamp, b.source.timestamp) { + (Some(a), Some(b)) => b.cmp(&a), + _ => Ordering::Equal, + }, + ordering => ordering, + }); + + let mut total_size_removed = 0usize; + let mut to_be_removed = vec![]; + let free_bytes = self.max_transactions_total_bytes - self.transactions.bytes(); + + loop { + let Some((worst_hash, worst_tx)) = sorted.pop() else { + return Err(sc_transaction_pool_api::error::Error::ImmediatelyDropped); + }; + + if worst_tx.priority() >= new_tx.priority() { + return Err(sc_transaction_pool_api::error::Error::ImmediatelyDropped); + } + + total_size_removed += worst_tx.bytes; + to_be_removed.push(worst_hash); + + if free_bytes + total_size_removed >= new_tx.bytes { + break; + } + } + + let source = new_tx.source(); + transactions.insert(hash, Arc::from(new_tx)); + for worst_hash in &to_be_removed { + transactions.remove(worst_hash); + } + debug_assert!(!self.is_limit_exceeded(transactions.len(), self.transactions.bytes())); + + Ok(InsertionInfo::new_with_removed(hash, source, to_be_removed)) + } + /// Adds a new unwatched transactions to the internal buffer not exceeding the limit. /// /// Returns the vector of results for each transaction, the order corresponds to the input @@ -308,7 +437,8 @@ where &self, source: TransactionSource, xts: &[ExtrinsicFor], - ) -> Vec>, ChainApi::Error>> { + ) -> Vec>, sc_transaction_pool_api::error::Error>> + { let result = xts .iter() .map(|xt| { @@ -325,20 +455,11 @@ where &self, source: TransactionSource, xt: ExtrinsicFor, - ) -> Result>, ChainApi::Error> { + ) -> Result>, sc_transaction_pool_api::error::Error> { let (hash, length) = self.api.hash_and_length(&xt); self.try_insert(hash, TxInMemPool::new_watched(source, xt.clone(), length)) } - /// Removes transaction from the memory pool which are specified by the given list of hashes. - pub(super) async fn remove_dropped_transaction( - &self, - dropped: &ExtrinsicHash, - ) -> Option>> { - log::debug!(target: LOG_TARGET, "[{:?}] mempool::remove_dropped_transaction", dropped); - self.transactions.write().remove(dropped) - } - /// Clones and returns a `HashMap` of references to all unwatched transactions in the memory /// pool. pub(super) fn clone_unwatched( @@ -362,9 +483,13 @@ where .collect::>() } - /// Removes a transaction from the memory pool based on a given hash. - pub(super) fn remove(&self, hash: ExtrinsicHash) { - let _ = self.transactions.write().remove(&hash); + /// Removes a transaction with given hash from the memory pool. + pub(super) fn remove_transaction( + &self, + hash: &ExtrinsicHash, + ) -> Option>> { + log::debug!(target: LOG_TARGET, "[{hash:?}] mempool::remove_transaction"); + self.transactions.write().remove(hash) } /// Revalidates a batch of transactions against the provided finalized block. @@ -462,6 +587,17 @@ where }); self.listener.invalidate_transactions(&invalid_hashes); } + + /// Updates the priority of transaction stored in mempool using provided view_store submission + /// outcome. + pub(super) fn update_transaction_priority(&self, outcome: &ViewStoreSubmitOutcome) { + outcome.priority().map(|priority| { + self.transactions + .write() + .get_mut(&outcome.hash()) + .map(|p| *p.priority.write() = Some(priority)) + }); + } } #[cfg(test)] @@ -583,6 +719,9 @@ mod tx_mem_pool_tests { assert_eq!(mempool.unwatched_and_watched_count(), (10, 5)); } + /// size of large extrinsic + const LARGE_XT_SIZE: usize = 1129; + fn large_uxt(x: usize) -> Extrinsic { ExtrinsicBuilder::new_include_data(vec![x as u8; 1024]).build() } @@ -592,8 +731,7 @@ mod tx_mem_pool_tests { sp_tracing::try_init_simple(); let max = 10; let api = Arc::from(TestApi::default()); - //size of large extrinsic is: 1129 - let mempool = TxMemPool::new_test(api.clone(), usize::MAX, max * 1129); + let mempool = TxMemPool::new_test(api.clone(), usize::MAX, max * LARGE_XT_SIZE); let xts = (0..max).map(|x| Arc::from(large_uxt(x))).collect::>(); @@ -617,4 +755,200 @@ mod tx_mem_pool_tests { sc_transaction_pool_api::error::Error::ImmediatelyDropped )); } + + #[test] + fn replacing_txs_works_for_same_tx_size() { + sp_tracing::try_init_simple(); + let max = 10; + let api = Arc::from(TestApi::default()); + let mempool = TxMemPool::new_test(api.clone(), usize::MAX, max * LARGE_XT_SIZE); + + let xts = (0..max).map(|x| Arc::from(large_uxt(x))).collect::>(); + + let low_prio = 0u64; + let hi_prio = u64::MAX; + + let total_xts_bytes = xts.iter().fold(0, |r, x| r + api.hash_and_length(&x).1); + let (submit_outcomes, hashes): (Vec<_>, Vec<_>) = xts + .iter() + .map(|t| { + let h = api.hash_and_length(t).0; + (ViewStoreSubmitOutcome::new(h, Some(low_prio)), h) + }) + .unzip(); + + let results = mempool.extend_unwatched(TransactionSource::External, &xts); + assert!(results.iter().all(Result::is_ok)); + assert_eq!(mempool.bytes(), total_xts_bytes); + + submit_outcomes + .into_iter() + .for_each(|o| mempool.update_transaction_priority(&o)); + + let xt = Arc::from(large_uxt(98)); + let hash = api.hash_and_length(&xt).0; + let result = mempool + .try_insert_with_replacement(xt, hi_prio, TransactionSource::External, false) + .unwrap(); + + assert_eq!(result.hash, hash); + assert_eq!(result.removed, hashes[0..1]); + } + + #[test] + fn replacing_txs_removes_proper_size_of_txs() { + sp_tracing::try_init_simple(); + let max = 10; + let api = Arc::from(TestApi::default()); + let mempool = TxMemPool::new_test(api.clone(), usize::MAX, max * LARGE_XT_SIZE); + + let xts = (0..max).map(|x| Arc::from(large_uxt(x))).collect::>(); + + let low_prio = 0u64; + let hi_prio = u64::MAX; + + let total_xts_bytes = xts.iter().fold(0, |r, x| r + api.hash_and_length(&x).1); + let (submit_outcomes, hashes): (Vec<_>, Vec<_>) = xts + .iter() + .map(|t| { + let h = api.hash_and_length(t).0; + (ViewStoreSubmitOutcome::new(h, Some(low_prio)), h) + }) + .unzip(); + + let results = mempool.extend_unwatched(TransactionSource::External, &xts); + assert!(results.iter().all(Result::is_ok)); + assert_eq!(mempool.bytes(), total_xts_bytes); + assert_eq!(total_xts_bytes, max * LARGE_XT_SIZE); + + submit_outcomes + .into_iter() + .for_each(|o| mempool.update_transaction_priority(&o)); + + //this one should drop 2 xts (size: 1130): + let xt = Arc::from(ExtrinsicBuilder::new_include_data(vec![98 as u8; 1025]).build()); + let (hash, length) = api.hash_and_length(&xt); + assert_eq!(length, 1130); + let result = mempool + .try_insert_with_replacement(xt, hi_prio, TransactionSource::External, false) + .unwrap(); + + assert_eq!(result.hash, hash); + assert_eq!(result.removed, hashes[0..2]); + } + + #[test] + fn replacing_txs_removes_proper_size_and_prios() { + sp_tracing::try_init_simple(); + const COUNT: usize = 10; + let api = Arc::from(TestApi::default()); + let mempool = TxMemPool::new_test(api.clone(), usize::MAX, COUNT * LARGE_XT_SIZE); + + let xts = (0..COUNT).map(|x| Arc::from(large_uxt(x))).collect::>(); + + let hi_prio = u64::MAX; + + let total_xts_bytes = xts.iter().fold(0, |r, x| r + api.hash_and_length(&x).1); + let (submit_outcomes, hashes): (Vec<_>, Vec<_>) = xts + .iter() + .enumerate() + .map(|(prio, t)| { + let h = api.hash_and_length(t).0; + (ViewStoreSubmitOutcome::new(h, Some((COUNT - prio).try_into().unwrap())), h) + }) + .unzip(); + + let results = mempool.extend_unwatched(TransactionSource::External, &xts); + assert!(results.iter().all(Result::is_ok)); + assert_eq!(mempool.bytes(), total_xts_bytes); + + submit_outcomes + .into_iter() + .for_each(|o| mempool.update_transaction_priority(&o)); + + //this one should drop 3 xts (each of size 1129) + let xt = Arc::from(ExtrinsicBuilder::new_include_data(vec![98 as u8; 2154]).build()); + let (hash, length) = api.hash_and_length(&xt); + // overhead is 105, thus length: 105 + 2154 + assert_eq!(length, 2 * LARGE_XT_SIZE + 1); + let result = mempool + .try_insert_with_replacement(xt, hi_prio, TransactionSource::External, false) + .unwrap(); + + assert_eq!(result.hash, hash); + assert!(result.removed.iter().eq(hashes[COUNT - 3..COUNT].iter().rev())); + } + + #[test] + fn replacing_txs_skips_lower_prio_tx() { + sp_tracing::try_init_simple(); + const COUNT: usize = 10; + let api = Arc::from(TestApi::default()); + let mempool = TxMemPool::new_test(api.clone(), usize::MAX, COUNT * LARGE_XT_SIZE); + + let xts = (0..COUNT).map(|x| Arc::from(large_uxt(x))).collect::>(); + + let hi_prio = 100u64; + let low_prio = 10u64; + + let total_xts_bytes = xts.iter().fold(0, |r, x| r + api.hash_and_length(&x).1); + let submit_outcomes = xts + .iter() + .map(|t| { + let h = api.hash_and_length(t).0; + ViewStoreSubmitOutcome::new(h, Some(hi_prio)) + }) + .collect::>(); + + let results = mempool.extend_unwatched(TransactionSource::External, &xts); + assert!(results.iter().all(Result::is_ok)); + assert_eq!(mempool.bytes(), total_xts_bytes); + + submit_outcomes + .into_iter() + .for_each(|o| mempool.update_transaction_priority(&o)); + + let xt = Arc::from(large_uxt(98)); + let result = + mempool.try_insert_with_replacement(xt, low_prio, TransactionSource::External, false); + + // lower prio tx is rejected immediately + assert!(matches!( + result.unwrap_err(), + sc_transaction_pool_api::error::Error::ImmediatelyDropped + )); + } + + #[test] + fn replacing_txs_is_skipped_if_prios_are_not_set() { + sp_tracing::try_init_simple(); + const COUNT: usize = 10; + let api = Arc::from(TestApi::default()); + let mempool = TxMemPool::new_test(api.clone(), usize::MAX, COUNT * LARGE_XT_SIZE); + + let xts = (0..COUNT).map(|x| Arc::from(large_uxt(x))).collect::>(); + + let hi_prio = u64::MAX; + + let total_xts_bytes = xts.iter().fold(0, |r, x| r + api.hash_and_length(&x).1); + + let results = mempool.extend_unwatched(TransactionSource::External, &xts); + assert!(results.iter().all(Result::is_ok)); + assert_eq!(mempool.bytes(), total_xts_bytes); + + //this one could drop 3 xts (each of size 1129) + let xt = Arc::from(ExtrinsicBuilder::new_include_data(vec![98 as u8; 2154]).build()); + let length = api.hash_and_length(&xt).1; + // overhead is 105, thus length: 105 + 2154 + assert_eq!(length, 2 * LARGE_XT_SIZE + 1); + + let result = + mempool.try_insert_with_replacement(xt, hi_prio, TransactionSource::External, false); + + // we did not update priorities (update_transaction_priority was not called): + assert!(matches!( + result.unwrap_err(), + sc_transaction_pool_api::error::Error::ImmediatelyDropped + )); + } } diff --git a/substrate/client/transaction-pool/src/fork_aware_txpool/view.rs b/substrate/client/transaction-pool/src/fork_aware_txpool/view.rs index 3cbb8fa4871d0..a35d68120a3ab 100644 --- a/substrate/client/transaction-pool/src/fork_aware_txpool/view.rs +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/view.rs @@ -28,7 +28,7 @@ use crate::{ common::log_xt::log_xt_trace, graph::{ self, base_pool::TimedTransactionSource, watcher::Watcher, ExtrinsicFor, ExtrinsicHash, - IsValidator, ValidatedTransaction, ValidatedTransactionFor, + IsValidator, ValidatedPoolSubmitOutcome, ValidatedTransaction, ValidatedTransactionFor, }, LOG_TARGET, }; @@ -158,7 +158,7 @@ where pub(super) async fn submit_many( &self, xts: impl IntoIterator)>, - ) -> Vec, ChainApi::Error>> { + ) -> Vec, ChainApi::Error>> { if log::log_enabled!(target: LOG_TARGET, log::Level::Trace) { let xts = xts.into_iter().collect::>(); log_xt_trace!(target: LOG_TARGET, xts.iter().map(|(_,xt)| self.pool.validated_pool().api().hash_and_length(xt).0), "[{:?}] view::submit_many at:{}", self.at.hash); @@ -173,7 +173,7 @@ where &self, source: TimedTransactionSource, xt: ExtrinsicFor, - ) -> Result, ExtrinsicHash>, ChainApi::Error> { + ) -> Result, ChainApi::Error> { log::trace!(target: LOG_TARGET, "[{:?}] view::submit_and_watch at:{}", self.pool.validated_pool().api().hash_and_length(&xt).0, self.at.hash); self.pool.submit_and_watch(&self.at, source, xt).await } @@ -182,7 +182,7 @@ where pub(super) fn submit_local( &self, xt: ExtrinsicFor, - ) -> Result, ChainApi::Error> { + ) -> Result, ChainApi::Error> { let (hash, length) = self.pool.validated_pool().api().hash_and_length(&xt); log::trace!(target: LOG_TARGET, "[{:?}] view::submit_local at:{}", hash, self.at.hash); @@ -460,4 +460,18 @@ where const IGNORE_BANNED: bool = false; self.pool.validated_pool().check_is_known(tx_hash, IGNORE_BANNED).is_err() } + + /// Removes the whole transaction subtree from the inner pool. + /// + /// Refer to [`crate::graph::ValidatedPool::remove_subtree`] for more details. + pub fn remove_subtree( + &self, + tx_hash: ExtrinsicHash, + listener_action: F, + ) -> Vec> + where + F: Fn(&mut crate::graph::Listener, ExtrinsicHash), + { + self.pool.validated_pool().remove_subtree(tx_hash, listener_action) + } } diff --git a/substrate/client/transaction-pool/src/fork_aware_txpool/view_store.rs b/substrate/client/transaction-pool/src/fork_aware_txpool/view_store.rs index a06c051f0a7eb..43ed5bbf8869f 100644 --- a/substrate/client/transaction-pool/src/fork_aware_txpool/view_store.rs +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/view_store.rs @@ -27,7 +27,7 @@ use crate::{ graph::{ self, base_pool::{TimedTransactionSource, Transaction}, - ExtrinsicFor, ExtrinsicHash, TransactionFor, + BaseSubmitOutcome, ExtrinsicFor, ExtrinsicHash, TransactionFor, ValidatedPoolSubmitOutcome, }, ReadyIteratorFor, LOG_TARGET, }; @@ -38,20 +38,18 @@ use sc_transaction_pool_api::{error::Error as PoolError, PoolStatus}; use sp_blockchain::TreeRoute; use sp_runtime::{generic::BlockId, traits::Block as BlockT}; use std::{ - collections::{hash_map::Entry, HashMap}, + collections::{hash_map::Entry, HashMap, HashSet}, sync::Arc, time::Instant, }; -/// Helper struct to keep the context for transaction replacements. +/// Helper struct to maintain the context for pending transaction submission, executed for +/// newly inserted views. #[derive(Clone)] -struct PendingTxReplacement +struct PendingTxSubmission where ChainApi: graph::ChainApi, { - /// Indicates if the new transaction was already submitted to all the views in the view_store. - /// If true, it can be removed after inserting any new view. - processed: bool, /// New transaction replacing the old one. xt: ExtrinsicFor, /// Source of the transaction. @@ -60,13 +58,84 @@ where watched: bool, } -impl PendingTxReplacement +/// Helper type representing the callback allowing to trigger per-transaction events on +/// `ValidatedPool`'s listener. +type RemovalListener = + Arc, ExtrinsicHash) + Send + Sync>; + +/// Helper struct to maintain the context for pending transaction removal, executed for +/// newly inserted views. +struct PendingTxRemoval +where + ChainApi: graph::ChainApi, +{ + /// Hash of the transaction that will be removed, + xt_hash: ExtrinsicHash, + /// Action that shall be executed on underlying `ValidatedPool`'s listener. + listener_action: RemovalListener, +} + +/// This enum represents an action that should be executed on the newly built +/// view before this view is inserted into the view store. +enum PreInsertAction +where + ChainApi: graph::ChainApi, +{ + /// Represents the action of submitting a new transaction. Intended to use to handle usurped + /// transactions. + SubmitTx(PendingTxSubmission), + + /// Represents the action of removing a subtree of transactions. + RemoveSubtree(PendingTxRemoval), +} + +/// Represents a task awaiting execution, to be performed immediately prior to the view insertion +/// into the view store. +struct PendingPreInsertTask +where + ChainApi: graph::ChainApi, +{ + /// The action to be applied when inserting a new view. + action: PreInsertAction, + /// Indicates if the action was already applied to all the views in the view_store. + /// If true, it can be removed after inserting any new view. + processed: bool, +} + +impl PendingPreInsertTask where ChainApi: graph::ChainApi, { - /// Creates new unprocessed instance of pending transaction replacement. - fn new(xt: ExtrinsicFor, source: TimedTransactionSource, watched: bool) -> Self { - Self { processed: false, xt, source, watched } + /// Creates new unprocessed instance of pending transaction submission. + fn new_submission_action( + xt: ExtrinsicFor, + source: TimedTransactionSource, + watched: bool, + ) -> Self { + Self { + processed: false, + action: PreInsertAction::SubmitTx(PendingTxSubmission { xt, source, watched }), + } + } + + /// Creates new unprocessed instance of pending transaction removal. + fn new_removal_action( + xt_hash: ExtrinsicHash, + listener: RemovalListener, + ) -> Self { + Self { + processed: false, + action: PreInsertAction::RemoveSubtree(PendingTxRemoval { + xt_hash, + listener_action: listener, + }), + } + } + + /// Marks a task as done for every view present in view store. Basically means that can be + /// removed on new view insertion. + fn mark_processed(&mut self) { + self.processed = true; } } @@ -100,9 +169,20 @@ where /// notifcication threads. It is meant to assure that replaced transaction is also removed from /// newly built views in maintain process. /// - /// The map's key is hash of replaced extrinsic. - pending_txs_replacements: - RwLock, PendingTxReplacement>>, + /// The map's key is hash of actionable extrinsic (to avoid duplicated entries). + pending_txs_tasks: RwLock, PendingPreInsertTask>>, +} + +/// Type alias to outcome of submission to `ViewStore`. +pub(super) type ViewStoreSubmitOutcome = + BaseSubmitOutcome>; + +impl From> + for ViewStoreSubmitOutcome +{ + fn from(value: ValidatedPoolSubmitOutcome) -> Self { + Self::new(value.hash(), value.priority()) + } } impl ViewStore @@ -124,7 +204,7 @@ where listener, most_recent_view: RwLock::from(None), dropped_stream_controller, - pending_txs_replacements: Default::default(), + pending_txs_tasks: Default::default(), } } @@ -132,7 +212,7 @@ where pub(super) async fn submit( &self, xts: impl IntoIterator)> + Clone, - ) -> HashMap, ChainApi::Error>>> { + ) -> HashMap, ChainApi::Error>>> { let submit_futures = { let active_views = self.active_views.read(); active_views @@ -140,7 +220,16 @@ where .map(|(_, view)| { let view = view.clone(); let xts = xts.clone(); - async move { (view.at.hash, view.submit_many(xts).await) } + async move { + ( + view.at.hash, + view.submit_many(xts) + .await + .into_iter() + .map(|r| r.map(Into::into)) + .collect::>(), + ) + } }) .collect::>() }; @@ -153,7 +242,7 @@ where pub(super) fn submit_local( &self, xt: ExtrinsicFor, - ) -> Result, ChainApi::Error> { + ) -> Result, ChainApi::Error> { let active_views = self .active_views .read() @@ -168,12 +257,14 @@ where .map(|view| view.submit_local(xt.clone())) .find_or_first(Result::is_ok); - if let Some(Err(err)) = result { - log::trace!(target: LOG_TARGET, "[{:?}] submit_local: err: {}", tx_hash, err); - return Err(err) - }; - - Ok(tx_hash) + match result { + Some(Err(err)) => { + log::trace!(target: LOG_TARGET, "[{:?}] submit_local: err: {}", tx_hash, err); + Err(err) + }, + None => Ok(ViewStoreSubmitOutcome::new(tx_hash, None)), + Some(Ok(r)) => Ok(r.into()), + } } /// Import a single extrinsic and starts to watch its progress in the pool. @@ -188,7 +279,7 @@ where _at: Block::Hash, source: TimedTransactionSource, xt: ExtrinsicFor, - ) -> Result, ChainApi::Error> { + ) -> Result, ChainApi::Error> { let tx_hash = self.api.hash_and_length(&xt).0; let Some(external_watcher) = self.listener.create_external_watcher_for_tx(tx_hash) else { return Err(PoolError::AlreadyImported(Box::new(tx_hash)).into()) @@ -203,13 +294,13 @@ where let source = source.clone(); async move { match view.submit_and_watch(source, xt).await { - Ok(watcher) => { + Ok(mut result) => { self.listener.add_view_watcher_for_tx( tx_hash, view.at.hash, - watcher.into_stream().boxed(), + result.expect_watcher().into_stream().boxed(), ); - Ok(()) + Ok(result) }, Err(e) => Err(e), } @@ -217,17 +308,20 @@ where }) .collect::>() }; - let maybe_error = futures::future::join_all(submit_and_watch_futures) + let result = futures::future::join_all(submit_and_watch_futures) .await .into_iter() .find_or_first(Result::is_ok); - if let Some(Err(err)) = maybe_error { - log::trace!(target: LOG_TARGET, "[{:?}] submit_and_watch: err: {}", tx_hash, err); - return Err(err); - }; - - Ok(external_watcher) + match result { + Some(Err(err)) => { + log::trace!(target: LOG_TARGET, "[{:?}] submit_and_watch: err: {}", tx_hash, err); + return Err(err); + }, + Some(Ok(result)) => + Ok(ViewStoreSubmitOutcome::from(result).with_watcher(external_watcher)), + None => Ok(ViewStoreSubmitOutcome::new(tx_hash, None).with_watcher(external_watcher)), + } } /// Returns the pool status for every active view. @@ -575,8 +669,12 @@ where replaced: ExtrinsicHash, watched: bool, ) { - if let Entry::Vacant(entry) = self.pending_txs_replacements.write().entry(replaced) { - entry.insert(PendingTxReplacement::new(xt.clone(), source.clone(), watched)); + if let Entry::Vacant(entry) = self.pending_txs_tasks.write().entry(replaced) { + entry.insert(PendingPreInsertTask::new_submission_action( + xt.clone(), + source.clone(), + watched, + )); } else { return }; @@ -586,8 +684,8 @@ where self.replace_transaction_in_views(source, xt, xt_hash, replaced, watched).await; - if let Some(replacement) = self.pending_txs_replacements.write().get_mut(&replaced) { - replacement.processed = true; + if let Some(replacement) = self.pending_txs_tasks.write().get_mut(&replaced) { + replacement.mark_processed(); } } @@ -596,18 +694,25 @@ where /// After application, all already processed replacements are removed. async fn apply_pending_tx_replacements(&self, view: Arc>) { let mut futures = vec![]; - for replacement in self.pending_txs_replacements.read().values() { - let xt_hash = self.api.hash_and_length(&replacement.xt).0; - futures.push(self.replace_transaction_in_view( - view.clone(), - replacement.source.clone(), - replacement.xt.clone(), - xt_hash, - replacement.watched, - )); + for replacement in self.pending_txs_tasks.read().values() { + match replacement.action { + PreInsertAction::SubmitTx(ref submission) => { + let xt_hash = self.api.hash_and_length(&submission.xt).0; + futures.push(self.replace_transaction_in_view( + view.clone(), + submission.source.clone(), + submission.xt.clone(), + xt_hash, + submission.watched, + )); + }, + PreInsertAction::RemoveSubtree(ref removal) => { + view.remove_subtree(removal.xt_hash, &*removal.listener_action); + }, + } } let _results = futures::future::join_all(futures).await; - self.pending_txs_replacements.write().retain(|_, r| r.processed); + self.pending_txs_tasks.write().retain(|_, r| r.processed); } /// Submits `xt` to the given view. @@ -623,11 +728,11 @@ where ) { if watched { match view.submit_and_watch(source, xt).await { - Ok(watcher) => { + Ok(mut result) => { self.listener.add_view_watcher_for_tx( xt_hash, view.at.hash, - watcher.into_stream().boxed(), + result.expect_watcher().into_stream().boxed(), ); }, Err(e) => { @@ -690,4 +795,58 @@ where }; let _results = futures::future::join_all(submit_futures).await; } + + /// Removes a transaction subtree from every view in the view_store, starting from the given + /// transaction hash. + /// + /// This function traverses the dependency graph of transactions and removes the specified + /// transaction along with all its descendant transactions from every view. + /// + /// A `listener_action` callback function is invoked for every transaction that is removed, + /// providing a reference to the pool's listener and the hash of the removed transaction. This + /// allows to trigger the required events. Note that listener may be called multiple times for + /// the same hash. + /// + /// Function will also schedule view pre-insertion actions to ensure that transactions will be + /// removed from newly created view. + /// + /// Returns a vector containing the hashes of all removed transactions, including the root + /// transaction specified by `tx_hash`. Vector contains only unique hashes. + pub(super) fn remove_transaction_subtree( + &self, + xt_hash: ExtrinsicHash, + listener_action: F, + ) -> Vec> + where + F: Fn(&mut crate::graph::Listener, ExtrinsicHash) + + Clone + + Send + + Sync + + 'static, + { + if let Entry::Vacant(entry) = self.pending_txs_tasks.write().entry(xt_hash) { + entry.insert(PendingPreInsertTask::new_removal_action( + xt_hash, + Arc::from(listener_action.clone()), + )); + }; + + let mut seen = HashSet::new(); + + let removed = self + .active_views + .read() + .iter() + .chain(self.inactive_views.read().iter()) + .filter(|(_, view)| view.is_imported(&xt_hash)) + .flat_map(|(_, view)| view.remove_subtree(xt_hash, &listener_action)) + .filter(|xt_hash| seen.insert(*xt_hash)) + .collect(); + + if let Some(removal_action) = self.pending_txs_tasks.write().get_mut(&xt_hash) { + removal_action.mark_processed(); + } + + removed + } } diff --git a/substrate/client/transaction-pool/src/graph/base_pool.rs b/substrate/client/transaction-pool/src/graph/base_pool.rs index 04eaa998f42e6..3b4afc88b7897 100644 --- a/substrate/client/transaction-pool/src/graph/base_pool.rs +++ b/substrate/client/transaction-pool/src/graph/base_pool.rs @@ -453,27 +453,29 @@ impl BasePool, _>(|worst, current| { - let transaction = ¤t.transaction; - worst - .map(|worst| { - // Here we don't use `TransactionRef`'s ordering implementation because - // while it prefers priority like need here, it also prefers older - // transactions for inclusion purposes and limit enforcement needs to prefer - // newer transactions instead and drop the older ones. - match worst.transaction.priority.cmp(&transaction.transaction.priority) { - Ordering::Less => worst, - Ordering::Equal => - if worst.insertion_id > transaction.insertion_id { - transaction.clone() - } else { - worst - }, - Ordering::Greater => transaction.clone(), - } - }) - .or_else(|| Some(transaction.clone())) - }); + let worst = + self.ready.fold::>, _>(None, |worst, current| { + let transaction = ¤t.transaction; + worst + .map(|worst| { + // Here we don't use `TransactionRef`'s ordering implementation because + // while it prefers priority like need here, it also prefers older + // transactions for inclusion purposes and limit enforcement needs to + // prefer newer transactions instead and drop the older ones. + match worst.transaction.priority.cmp(&transaction.transaction.priority) + { + Ordering::Less => worst, + Ordering::Equal => + if worst.insertion_id > transaction.insertion_id { + transaction.clone() + } else { + worst + }, + Ordering::Greater => transaction.clone(), + } + }) + .or_else(|| Some(transaction.clone())) + }); if let Some(worst) = worst { removed.append(&mut self.remove_subtree(&[worst.transaction.hash.clone()])) diff --git a/substrate/client/transaction-pool/src/graph/listener.rs b/substrate/client/transaction-pool/src/graph/listener.rs index 41daf5491f709..7b09ee4c64095 100644 --- a/substrate/client/transaction-pool/src/graph/listener.rs +++ b/substrate/client/transaction-pool/src/graph/listener.rs @@ -126,8 +126,8 @@ impl Listener usize { + self.ready.count + self.future.count + } +} + /// Should we check that the transaction is banned /// in the pool, before we verify it? #[derive(Copy, Clone)] -enum CheckBannedBeforeVerify { +pub(crate) enum CheckBannedBeforeVerify { Yes, No, } @@ -172,6 +179,21 @@ pub struct Pool { } impl Pool { + /// Create a new transaction pool with statically sized rotator. + pub fn new_with_staticly_sized_rotator( + options: Options, + is_validator: IsValidator, + api: Arc, + ) -> Self { + Self { + validated_pool: Arc::new(ValidatedPool::new_with_staticly_sized_rotator( + options, + is_validator, + api, + )), + } + } + /// Create a new transaction pool. pub fn new(options: Options, is_validator: IsValidator, api: Arc) -> Self { Self { validated_pool: Arc::new(ValidatedPool::new(options, is_validator, api)) } @@ -182,7 +204,7 @@ impl Pool { &self, at: &HashAndNumber, xts: impl IntoIterator)>, - ) -> Vec, B::Error>> { + ) -> Vec, B::Error>> { let validated_transactions = self.verify(at, xts, CheckBannedBeforeVerify::Yes).await; self.validated_pool.submit(validated_transactions.into_values()) } @@ -194,7 +216,7 @@ impl Pool { &self, at: &HashAndNumber, xts: impl IntoIterator)>, - ) -> Vec, B::Error>> { + ) -> Vec, B::Error>> { let validated_transactions = self.verify(at, xts, CheckBannedBeforeVerify::No).await; self.validated_pool.submit(validated_transactions.into_values()) } @@ -205,7 +227,7 @@ impl Pool { at: &HashAndNumber, source: base::TimedTransactionSource, xt: ExtrinsicFor, - ) -> Result, B::Error> { + ) -> Result, B::Error> { let res = self.submit_at(at, std::iter::once((source, xt))).await.pop(); res.expect("One extrinsic passed; one result returned; qed") } @@ -216,7 +238,7 @@ impl Pool { at: &HashAndNumber, source: base::TimedTransactionSource, xt: ExtrinsicFor, - ) -> Result, ExtrinsicHash>, B::Error> { + ) -> Result, B::Error> { let (_, tx) = self .verify_one(at.hash, at.number, source, xt, CheckBannedBeforeVerify::Yes) .await; @@ -284,6 +306,7 @@ impl Pool { let mut validated_counter: usize = 0; let mut future_tags = Vec::new(); + let now = Instant::now(); for (extrinsic, in_pool_tags) in all { match in_pool_tags { // reuse the tags for extrinsics that were found in the pool @@ -319,7 +342,7 @@ impl Pool { } } - log::trace!(target: LOG_TARGET,"prune: validated_counter:{validated_counter}"); + log::debug!(target: LOG_TARGET,"prune: validated_counter:{validated_counter}, took:{:?}", now.elapsed()); self.prune_tags(at, future_tags, in_pool_hashes).await } @@ -351,6 +374,7 @@ impl Pool { tags: impl IntoIterator, known_imported_hashes: impl IntoIterator> + Clone, ) { + let now = Instant::now(); log::trace!(target: LOG_TARGET, "Pruning at {:?}", at); // Prune all transactions that provide given tags let prune_status = self.validated_pool.prune_tags(tags); @@ -369,9 +393,8 @@ impl Pool { let reverified_transactions = self.verify(at, pruned_transactions, CheckBannedBeforeVerify::Yes).await; - let pruned_hashes = reverified_transactions.keys().map(Clone::clone).collect(); - - log::trace!(target: LOG_TARGET, "Pruning at {:?}. Resubmitting transactions: {}", &at, reverified_transactions.len()); + let pruned_hashes = reverified_transactions.keys().map(Clone::clone).collect::>(); + log::debug!(target: LOG_TARGET, "Pruning at {:?}. Resubmitting transactions: {}, reverification took: {:?}", &at, reverified_transactions.len(), now.elapsed()); log_xt_trace!(data: tuple, target: LOG_TARGET, &reverified_transactions, "[{:?}] Resubmitting transaction: {:?}"); // And finally - submit reverified transactions back to the pool @@ -409,7 +432,7 @@ impl Pool { } /// Returns future that validates single transaction at given block. - async fn verify_one( + pub(crate) async fn verify_one( &self, block_hash: ::Hash, block_number: NumberFor, @@ -516,6 +539,7 @@ mod tests { .into(), ), ) + .map(|outcome| outcome.hash()) .unwrap(); // then @@ -544,7 +568,10 @@ mod tests { // when let txs = txs.into_iter().map(|x| (SOURCE, Arc::from(x))).collect::>(); - let hashes = block_on(pool.submit_at(&api.expect_hash_and_number(0), txs)); + let hashes = block_on(pool.submit_at(&api.expect_hash_and_number(0), txs)) + .into_iter() + .map(|r| r.map(|o| o.hash())) + .collect::>(); log::debug!("--> {hashes:#?}"); // then @@ -568,7 +595,8 @@ mod tests { // when pool.validated_pool.ban(&Instant::now(), vec![pool.hash_of(&uxt)]); - let res = block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt.into())); + let res = block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt.into())) + .map(|o| o.hash()); assert_eq!(pool.validated_pool().status().ready, 0); assert_eq!(pool.validated_pool().status().future, 0); @@ -580,7 +608,7 @@ mod tests { fn should_reject_unactionable_transactions() { // given let api = Arc::new(TestApi::default()); - let pool = Pool::new( + let pool = Pool::new_with_staticly_sized_rotator( Default::default(), // the node does not author blocks false.into(), @@ -591,7 +619,8 @@ mod tests { let uxt = ExtrinsicBuilder::new_include_data(vec![42]).build(); // when - let res = block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt.into())); + let res = block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt.into())) + .map(|o| o.hash()); // then assert_matches!(res.unwrap_err(), error::Error::Unactionable); @@ -619,7 +648,8 @@ mod tests { .into(), ), ) - .unwrap(); + .unwrap() + .hash(); let hash1 = block_on( pool.submit_one( &han_of_block0, @@ -633,7 +663,8 @@ mod tests { .into(), ), ) - .unwrap(); + .unwrap() + .hash(); // future doesn't count let _hash = block_on( pool.submit_one( @@ -648,7 +679,8 @@ mod tests { .into(), ), ) - .unwrap(); + .unwrap() + .hash(); assert_eq!(pool.validated_pool().status().ready, 2); assert_eq!(pool.validated_pool().status().future, 1); @@ -681,7 +713,8 @@ mod tests { .into(), ), ) - .unwrap(); + .unwrap() + .hash(); let hash2 = block_on( pool.submit_one( &han_of_block0, @@ -695,7 +728,8 @@ mod tests { .into(), ), ) - .unwrap(); + .unwrap() + .hash(); let hash3 = block_on( pool.submit_one( &han_of_block0, @@ -709,7 +743,8 @@ mod tests { .into(), ), ) - .unwrap(); + .unwrap() + .hash(); // when pool.validated_pool.clear_stale(&api.expect_hash_and_number(5)); @@ -741,7 +776,8 @@ mod tests { .into(), ), ) - .unwrap(); + .unwrap() + .hash(); // when block_on(pool.prune_tags(&api.expect_hash_and_number(1), vec![vec![0]], vec![hash1])); @@ -767,10 +803,11 @@ mod tests { let options = Options { ready: limit.clone(), future: limit.clone(), ..Default::default() }; let api = Arc::new(TestApi::default()); - let pool = Pool::new(options, true.into(), api.clone()); + let pool = Pool::new_with_staticly_sized_rotator(options, true.into(), api.clone()); - let hash1 = - block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, xt.into())).unwrap(); + let hash1 = block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, xt.into())) + .unwrap() + .hash(); assert_eq!(pool.validated_pool().status().future, 1); // when @@ -787,7 +824,8 @@ mod tests { .into(), ), ) - .unwrap(); + .unwrap() + .hash(); // then assert_eq!(pool.validated_pool().status().future, 1); @@ -803,7 +841,7 @@ mod tests { let options = Options { ready: limit.clone(), future: limit.clone(), ..Default::default() }; let api = Arc::new(TestApi::default()); - let pool = Pool::new(options, true.into(), api.clone()); + let pool = Pool::new_with_staticly_sized_rotator(options, true.into(), api.clone()); // when block_on( @@ -819,6 +857,7 @@ mod tests { .into(), ), ) + .map(|o| o.hash()) .unwrap_err(); // then @@ -845,6 +884,7 @@ mod tests { .into(), ), ) + .map(|o| o.hash()) .unwrap_err(); // then @@ -873,7 +913,8 @@ mod tests { .into(), ), ) - .unwrap(); + .unwrap() + .expect_watcher(); assert_eq!(pool.validated_pool().status().ready, 1); assert_eq!(pool.validated_pool().status().future, 0); @@ -910,7 +951,8 @@ mod tests { .into(), ), ) - .unwrap(); + .unwrap() + .expect_watcher(); assert_eq!(pool.validated_pool().status().ready, 1); assert_eq!(pool.validated_pool().status().future, 0); @@ -949,7 +991,8 @@ mod tests { .into(), ), ) - .unwrap(); + .unwrap() + .expect_watcher(); assert_eq!(pool.validated_pool().status().ready, 0); assert_eq!(pool.validated_pool().status().future, 1); @@ -988,7 +1031,8 @@ mod tests { }); let watcher = block_on(pool.submit_and_watch(&api.expect_hash_and_number(0), SOURCE, uxt.into())) - .unwrap(); + .unwrap() + .expect_watcher(); assert_eq!(pool.validated_pool().status().ready, 1); // when @@ -1013,7 +1057,8 @@ mod tests { }); let watcher = block_on(pool.submit_and_watch(&api.expect_hash_and_number(0), SOURCE, uxt.into())) - .unwrap(); + .unwrap() + .expect_watcher(); assert_eq!(pool.validated_pool().status().ready, 1); // when @@ -1036,7 +1081,7 @@ mod tests { Options { ready: limit.clone(), future: limit.clone(), ..Default::default() }; let api = Arc::new(TestApi::default()); - let pool = Pool::new(options, true.into(), api.clone()); + let pool = Pool::new_with_staticly_sized_rotator(options, true.into(), api.clone()); let xt = uxt(Transfer { from: Alice.into(), @@ -1046,7 +1091,8 @@ mod tests { }); let watcher = block_on(pool.submit_and_watch(&api.expect_hash_and_number(0), SOURCE, xt.into())) - .unwrap(); + .unwrap() + .expect_watcher(); assert_eq!(pool.validated_pool().status().ready, 1); // when @@ -1074,7 +1120,7 @@ mod tests { Options { ready: limit.clone(), future: limit.clone(), ..Default::default() }; let api = Arc::new(TestApi::default()); - let pool = Pool::new(options, true.into(), api.clone()); + let pool = Pool::new_with_staticly_sized_rotator(options, true.into(), api.clone()); // after validation `IncludeData` will have priority set to 9001 // (validate_transaction mock) @@ -1106,14 +1152,16 @@ mod tests { Options { ready: limit.clone(), future: limit.clone(), ..Default::default() }; let api = Arc::new(TestApi::default()); - let pool = Pool::new(options, true.into(), api.clone()); + let pool = Pool::new_with_staticly_sized_rotator(options, true.into(), api.clone()); let han_of_block0 = api.expect_hash_and_number(0); // after validation `IncludeData` will have priority set to 9001 // (validate_transaction mock) let xt = ExtrinsicBuilder::new_include_data(Vec::new()).build(); - block_on(pool.submit_and_watch(&han_of_block0, SOURCE, xt.into())).unwrap(); + block_on(pool.submit_and_watch(&han_of_block0, SOURCE, xt.into())) + .unwrap() + .expect_watcher(); assert_eq!(pool.validated_pool().status().ready, 1); // after validation `Transfer` will have priority set to 4 (validate_transaction @@ -1124,8 +1172,9 @@ mod tests { amount: 5, nonce: 0, }); - let watcher = - block_on(pool.submit_and_watch(&han_of_block0, SOURCE, xt.into())).unwrap(); + let watcher = block_on(pool.submit_and_watch(&han_of_block0, SOURCE, xt.into())) + .unwrap() + .expect_watcher(); assert_eq!(pool.validated_pool().status().ready, 2); // when @@ -1151,7 +1200,11 @@ mod tests { let mut api = TestApi::default(); api.delay = Arc::new(Mutex::new(rx.into())); let api = Arc::new(api); - let pool = Arc::new(Pool::new(Default::default(), true.into(), api.clone())); + let pool = Arc::new(Pool::new_with_staticly_sized_rotator( + Default::default(), + true.into(), + api.clone(), + )); let han_of_block0 = api.expect_hash_and_number(0); diff --git a/substrate/client/transaction-pool/src/graph/ready.rs b/substrate/client/transaction-pool/src/graph/ready.rs index 9061d0e255811..b8aef99e638dc 100644 --- a/substrate/client/transaction-pool/src/graph/ready.rs +++ b/substrate/client/transaction-pool/src/graph/ready.rs @@ -232,12 +232,10 @@ impl ReadyTransactions { Ok(replaced) } - /// Fold a list of ready transactions to compute a single value. - pub fn fold, &ReadyTx) -> Option>( - &mut self, - f: F, - ) -> Option { - self.ready.read().values().fold(None, f) + /// Fold a list of ready transactions to compute a single value using initial value of + /// accumulator. + pub fn fold) -> R>(&self, init: R, f: F) -> R { + self.ready.read().values().fold(init, f) } /// Returns true if given transaction is part of the queue. diff --git a/substrate/client/transaction-pool/src/graph/rotator.rs b/substrate/client/transaction-pool/src/graph/rotator.rs index 9a2e269b5eede..80d8f24144c8a 100644 --- a/substrate/client/transaction-pool/src/graph/rotator.rs +++ b/substrate/client/transaction-pool/src/graph/rotator.rs @@ -31,7 +31,10 @@ use std::{ use super::base_pool::Transaction; /// Expected size of the banned extrinsics cache. -const EXPECTED_SIZE: usize = 2048; +const DEFAULT_EXPECTED_SIZE: usize = 2048; + +/// The default duration, in seconds, for which an extrinsic is banned. +const DEFAULT_BAN_TIME_SECS: u64 = 30 * 60; /// Pool rotator is responsible to only keep fresh extrinsics in the pool. /// @@ -42,18 +45,39 @@ pub struct PoolRotator { ban_time: Duration, /// Currently banned extrinsics. banned_until: RwLock>, + /// Expected size of the banned extrinsics cache. + expected_size: usize, +} + +impl Clone for PoolRotator { + fn clone(&self) -> Self { + Self { + ban_time: self.ban_time, + banned_until: RwLock::new(self.banned_until.read().clone()), + expected_size: self.expected_size, + } + } } impl Default for PoolRotator { fn default() -> Self { - Self { ban_time: Duration::from_secs(60 * 30), banned_until: Default::default() } + Self { + ban_time: Duration::from_secs(DEFAULT_BAN_TIME_SECS), + banned_until: Default::default(), + expected_size: DEFAULT_EXPECTED_SIZE, + } } } impl PoolRotator { /// New rotator instance with specified ban time. pub fn new(ban_time: Duration) -> Self { - Self { ban_time, banned_until: Default::default() } + Self { ban_time, ..Self::default() } + } + + /// New rotator instance with specified ban time and expected cache size. + pub fn new_with_expected_size(ban_time: Duration, expected_size: usize) -> Self { + Self { expected_size, ..Self::new(ban_time) } } /// Returns `true` if extrinsic hash is currently banned. @@ -69,8 +93,8 @@ impl PoolRotator { banned.insert(hash, *now + self.ban_time); } - if banned.len() > 2 * EXPECTED_SIZE { - while banned.len() > EXPECTED_SIZE { + if banned.len() > 2 * self.expected_size { + while banned.len() > self.expected_size { if let Some(key) = banned.keys().next().cloned() { banned.remove(&key); } @@ -201,16 +225,16 @@ mod tests { let past_block = 0; // when - for i in 0..2 * EXPECTED_SIZE { + for i in 0..2 * DEFAULT_EXPECTED_SIZE { let tx = tx_with(i as u64, past_block); assert!(rotator.ban_if_stale(&now, past_block, &tx)); } - assert_eq!(rotator.banned_until.read().len(), 2 * EXPECTED_SIZE); + assert_eq!(rotator.banned_until.read().len(), 2 * DEFAULT_EXPECTED_SIZE); // then - let tx = tx_with(2 * EXPECTED_SIZE as u64, past_block); + let tx = tx_with(2 * DEFAULT_EXPECTED_SIZE as u64, past_block); // trigger a garbage collection assert!(rotator.ban_if_stale(&now, past_block, &tx)); - assert_eq!(rotator.banned_until.read().len(), EXPECTED_SIZE); + assert_eq!(rotator.banned_until.read().len(), DEFAULT_EXPECTED_SIZE); } } diff --git a/substrate/client/transaction-pool/src/graph/tracked_map.rs b/substrate/client/transaction-pool/src/graph/tracked_map.rs index 6c3bbbf34b553..fe15c6eca3080 100644 --- a/substrate/client/transaction-pool/src/graph/tracked_map.rs +++ b/substrate/client/transaction-pool/src/graph/tracked_map.rs @@ -173,6 +173,11 @@ where pub fn len(&mut self) -> usize { self.inner_guard.len() } + + /// Returns an iterator over all key-value pairs. + pub fn iter(&self) -> Iter<'_, K, V> { + self.inner_guard.iter() + } } #[cfg(test)] diff --git a/substrate/client/transaction-pool/src/graph/validated_pool.rs b/substrate/client/transaction-pool/src/graph/validated_pool.rs index 14df63d9673e3..bc2b07896dba0 100644 --- a/substrate/client/transaction-pool/src/graph/validated_pool.rs +++ b/substrate/client/transaction-pool/src/graph/validated_pool.rs @@ -18,25 +18,22 @@ use std::{ collections::{HashMap, HashSet}, - hash, sync::Arc, }; use crate::{common::log_xt::log_xt_trace, LOG_TARGET}; use futures::channel::mpsc::{channel, Sender}; use parking_lot::{Mutex, RwLock}; -use sc_transaction_pool_api::{error, PoolStatus, ReadyTransactions}; -use serde::Serialize; +use sc_transaction_pool_api::{error, PoolStatus, ReadyTransactions, TransactionPriority}; use sp_blockchain::HashAndNumber; use sp_runtime::{ - traits::{self, SaturatedConversion}, + traits::SaturatedConversion, transaction_validity::{TransactionTag as Tag, ValidTransaction}, }; use std::time::Instant; use super::{ base_pool::{self as base, PruneStatus}, - listener::Listener, pool::{ BlockHash, ChainApi, EventStream, ExtrinsicFor, ExtrinsicHash, Options, TransactionFor, }, @@ -79,12 +76,23 @@ impl ValidatedTransaction { valid_till: at.saturated_into::().saturating_add(validity.longevity), }) } + + /// Returns priority for valid transaction, None if transaction is not valid. + pub fn priority(&self) -> Option { + match self { + ValidatedTransaction::Valid(base::Transaction { priority, .. }) => Some(*priority), + _ => None, + } + } } -/// A type of validated transaction stored in the pool. +/// A type of validated transaction stored in the validated pool. pub type ValidatedTransactionFor = ValidatedTransaction, ExtrinsicFor, ::Error>; +/// A type alias representing ValidatedPool listener for given ChainApi type. +pub type Listener = super::listener::Listener, B>; + /// A closure that returns true if the local node is a validator that can author blocks. #[derive(Clone)] pub struct IsValidator(Arc bool + Send + Sync>>); @@ -101,12 +109,56 @@ impl From bool + Send + Sync>> for IsValidator { } } +/// Represents the result of `submit` or `submit_and_watch` operations. +pub struct BaseSubmitOutcome { + /// The hash of the submitted transaction. + hash: ExtrinsicHash, + /// A transaction watcher. This is `Some` for `submit_and_watch` and `None` for `submit`. + watcher: Option, + + /// The priority of the transaction. Defaults to None if unknown. + priority: Option, +} + +/// Type alias to outcome of submission to `ValidatedPool`. +pub type ValidatedPoolSubmitOutcome = + BaseSubmitOutcome, ExtrinsicHash>>; + +impl BaseSubmitOutcome { + /// Creates a new instance with given hash and priority. + pub fn new(hash: ExtrinsicHash, priority: Option) -> Self { + Self { hash, priority, watcher: None } + } + + /// Sets the transaction watcher. + pub fn with_watcher(mut self, watcher: W) -> Self { + self.watcher = Some(watcher); + self + } + + /// Provides priority of submitted transaction. + pub fn priority(&self) -> Option { + self.priority + } + + /// Provides hash of submitted transaction. + pub fn hash(&self) -> ExtrinsicHash { + self.hash + } + + /// Provides a watcher. Should only be called on outcomes of `submit_and_watch`. Otherwise will + /// panic (that would mean logical error in program). + pub fn expect_watcher(&mut self) -> W { + self.watcher.take().expect("watcher was set in submit_and_watch. qed") + } +} + /// Pool that deals with validated transactions. pub struct ValidatedPool { api: Arc, is_validator: IsValidator, options: Options, - listener: RwLock, B>>, + listener: RwLock>, pub(crate) pool: RwLock, ExtrinsicFor>>, import_notification_sinks: Mutex>>>, rotator: PoolRotator>, @@ -121,16 +173,41 @@ impl Clone for ValidatedPool { listener: Default::default(), pool: RwLock::from(self.pool.read().clone()), import_notification_sinks: Default::default(), - rotator: PoolRotator::default(), + rotator: self.rotator.clone(), } } } impl ValidatedPool { + /// Create a new transaction pool with statically sized rotator. + pub fn new_with_staticly_sized_rotator( + options: Options, + is_validator: IsValidator, + api: Arc, + ) -> Self { + let ban_time = options.ban_time; + Self::new_with_rotator(options, is_validator, api, PoolRotator::new(ban_time)) + } + /// Create a new transaction pool. pub fn new(options: Options, is_validator: IsValidator, api: Arc) -> Self { - let base_pool = base::BasePool::new(options.reject_future_transactions); let ban_time = options.ban_time; + let total_count = options.total_count(); + Self::new_with_rotator( + options, + is_validator, + api, + PoolRotator::new_with_expected_size(ban_time, total_count), + ) + } + + fn new_with_rotator( + options: Options, + is_validator: IsValidator, + api: Arc, + rotator: PoolRotator>, + ) -> Self { + let base_pool = base::BasePool::new(options.reject_future_transactions); Self { is_validator, options, @@ -138,7 +215,7 @@ impl ValidatedPool { api, pool: RwLock::new(base_pool), import_notification_sinks: Default::default(), - rotator: PoolRotator::new(ban_time), + rotator, } } @@ -175,7 +252,7 @@ impl ValidatedPool { pub fn submit( &self, txs: impl IntoIterator>, - ) -> Vec, B::Error>> { + ) -> Vec, B::Error>> { let results = txs .into_iter() .map(|validated_tx| self.submit_one(validated_tx)) @@ -191,7 +268,7 @@ impl ValidatedPool { results .into_iter() .map(|res| match res { - Ok(ref hash) if removed.contains(hash) => + Ok(outcome) if removed.contains(&outcome.hash) => Err(error::Error::ImmediatelyDropped.into()), other => other, }) @@ -199,9 +276,13 @@ impl ValidatedPool { } /// Submit single pre-validated transaction to the pool. - fn submit_one(&self, tx: ValidatedTransactionFor) -> Result, B::Error> { + fn submit_one( + &self, + tx: ValidatedTransactionFor, + ) -> Result, B::Error> { match tx { ValidatedTransaction::Valid(tx) => { + let priority = tx.priority; log::trace!(target: LOG_TARGET, "[{:?}] ValidatedPool::submit_one", tx.hash); if !tx.propagate && !(self.is_validator.0)() { return Err(error::Error::Unactionable.into()) @@ -229,7 +310,7 @@ impl ValidatedPool { let mut listener = self.listener.write(); fire_events(&mut *listener, &imported); - Ok(*imported.hash()) + Ok(ValidatedPoolSubmitOutcome::new(*imported.hash(), Some(priority))) }, ValidatedTransaction::Invalid(hash, err) => { log::trace!(target: LOG_TARGET, "[{:?}] ValidatedPool::submit_one invalid: {:?}", hash, err); @@ -280,7 +361,7 @@ impl ValidatedPool { // run notifications let mut listener = self.listener.write(); for h in &removed { - listener.limit_enforced(h); + listener.limits_enforced(h); } removed @@ -293,7 +374,7 @@ impl ValidatedPool { pub fn submit_and_watch( &self, tx: ValidatedTransactionFor, - ) -> Result, ExtrinsicHash>, B::Error> { + ) -> Result, B::Error> { match tx { ValidatedTransaction::Valid(tx) => { let hash = self.api.hash_and_length(&tx.data).0; @@ -301,7 +382,7 @@ impl ValidatedPool { self.submit(std::iter::once(ValidatedTransaction::Valid(tx))) .pop() .expect("One extrinsic passed; one result returned; qed") - .map(|_| watcher) + .map(|outcome| outcome.with_watcher(watcher)) }, ValidatedTransaction::Invalid(hash, err) => { self.rotator.ban(&Instant::now(), std::iter::once(hash)); @@ -686,11 +767,42 @@ impl ValidatedPool { listener.future(&f.hash); }); } + + /// Removes a transaction subtree from the pool, starting from the given transaction hash. + /// + /// This function traverses the dependency graph of transactions and removes the specified + /// transaction along with all its descendant transactions from the pool. + /// + /// A `listener_action` callback function is invoked for every transaction that is removed, + /// providing a reference to the pool's listener and the hash of the removed transaction. This + /// allows to trigger the required events. + /// + /// Returns a vector containing the hashes of all removed transactions, including the root + /// transaction specified by `tx_hash`. + pub fn remove_subtree( + &self, + tx_hash: ExtrinsicHash, + listener_action: F, + ) -> Vec> + where + F: Fn(&mut Listener, ExtrinsicHash), + { + self.pool + .write() + .remove_subtree(&[tx_hash]) + .into_iter() + .map(|tx| { + let removed_tx_hash = tx.hash; + let mut listener = self.listener.write(); + listener_action(&mut *listener, removed_tx_hash); + removed_tx_hash + }) + .collect::>() + } } -fn fire_events(listener: &mut Listener, imported: &base::Imported) +fn fire_events(listener: &mut Listener, imported: &base::Imported, Ex>) where - H: hash::Hash + Eq + traits::Member + Serialize, B: ChainApi, { match *imported { diff --git a/substrate/client/transaction-pool/src/single_state_txpool/revalidation.rs b/substrate/client/transaction-pool/src/single_state_txpool/revalidation.rs index f22fa2ddabdee..2a691ae35eaf7 100644 --- a/substrate/client/transaction-pool/src/single_state_txpool/revalidation.rs +++ b/substrate/client/transaction-pool/src/single_state_txpool/revalidation.rs @@ -384,7 +384,11 @@ mod tests { #[test] fn revalidation_queue_works() { let api = Arc::new(TestApi::default()); - let pool = Arc::new(Pool::new(Default::default(), true.into(), api.clone())); + let pool = Arc::new(Pool::new_with_staticly_sized_rotator( + Default::default(), + true.into(), + api.clone(), + )); let queue = Arc::new(RevalidationQueue::new(api.clone(), pool.clone())); let uxt = uxt(Transfer { @@ -401,7 +405,8 @@ mod tests { TimedTransactionSource::new_external(false), uxt.clone().into(), )) - .expect("Should be valid"); + .expect("Should be valid") + .hash(); block_on(queue.revalidate_later(han_of_block0.hash, vec![uxt_hash])); @@ -414,7 +419,11 @@ mod tests { #[test] fn revalidation_queue_skips_revalidation_for_unknown_block_hash() { let api = Arc::new(TestApi::default()); - let pool = Arc::new(Pool::new(Default::default(), true.into(), api.clone())); + let pool = Arc::new(Pool::new_with_staticly_sized_rotator( + Default::default(), + true.into(), + api.clone(), + )); let queue = Arc::new(RevalidationQueue::new(api.clone(), pool.clone())); let uxt0 = uxt(Transfer { @@ -440,7 +449,7 @@ mod tests { vec![(source.clone(), uxt0.into()), (source, uxt1.into())], )) .into_iter() - .map(|r| r.expect("Should be valid")) + .map(|r| r.expect("Should be valid").hash()) .collect::>(); assert_eq!(api.validation_requests().len(), 2); diff --git a/substrate/client/transaction-pool/src/single_state_txpool/single_state_txpool.rs b/substrate/client/transaction-pool/src/single_state_txpool/single_state_txpool.rs index e7504012ca67b..3598f9dbc2af1 100644 --- a/substrate/client/transaction-pool/src/single_state_txpool/single_state_txpool.rs +++ b/substrate/client/transaction-pool/src/single_state_txpool/single_state_txpool.rs @@ -141,7 +141,11 @@ where finalized_hash: Block::Hash, options: graph::Options, ) -> (Self, Pin + Send>>) { - let pool = Arc::new(graph::Pool::new(options, true.into(), pool_api.clone())); + let pool = Arc::new(graph::Pool::new_with_staticly_sized_rotator( + options, + true.into(), + pool_api.clone(), + )); let (revalidation_queue, background_task) = revalidation::RevalidationQueue::new_background( pool_api.clone(), pool.clone(), @@ -177,7 +181,11 @@ where best_block_hash: Block::Hash, finalized_hash: Block::Hash, ) -> Self { - let pool = Arc::new(graph::Pool::new(options, is_validator, pool_api.clone())); + let pool = Arc::new(graph::Pool::new_with_staticly_sized_rotator( + options, + is_validator, + pool_api.clone(), + )); let (revalidation_queue, background_task) = match revalidation_type { RevalidationType::Light => (revalidation::RevalidationQueue::new(pool_api.clone(), pool.clone()), None), @@ -266,7 +274,12 @@ where let number = self.api.resolve_block_number(at); let at = HashAndNumber { hash: at, number: number? }; - Ok(pool.submit_at(&at, xts).await) + Ok(pool + .submit_at(&at, xts) + .await + .into_iter() + .map(|result| result.map(|outcome| outcome.hash())) + .collect()) } async fn submit_one( @@ -284,6 +297,7 @@ where let at = HashAndNumber { hash: at, number: number? }; pool.submit_one(&at, TimedTransactionSource::from_transaction_source(source, false), xt) .await + .map(|outcome| outcome.hash()) } async fn submit_and_watch( @@ -300,15 +314,13 @@ where let number = self.api.resolve_block_number(at); let at = HashAndNumber { hash: at, number: number? }; - let watcher = pool - .submit_and_watch( - &at, - TimedTransactionSource::from_transaction_source(source, false), - xt, - ) - .await?; - - Ok(watcher.into_stream().boxed()) + pool.submit_and_watch( + &at, + TimedTransactionSource::from_transaction_source(source, false), + xt, + ) + .await + .map(|mut outcome| outcome.expect_watcher().into_stream().boxed()) } fn remove_invalid(&self, hashes: &[TxHash]) -> Vec> { @@ -476,7 +488,11 @@ where validity, ); - self.pool.validated_pool().submit(vec![validated]).remove(0) + self.pool + .validated_pool() + .submit(vec![validated]) + .remove(0) + .map(|outcome| outcome.hash()) } } diff --git a/substrate/client/transaction-pool/tests/fatp.rs b/substrate/client/transaction-pool/tests/fatp.rs index 8bf08122995c1..dd82c52a6047b 100644 --- a/substrate/client/transaction-pool/tests/fatp.rs +++ b/substrate/client/transaction-pool/tests/fatp.rs @@ -2199,7 +2199,7 @@ fn import_sink_works3() { pool.submit_one(genesis, SOURCE, xt1.clone()), ]; - let x = block_on(futures::future::join_all(submissions)); + block_on(futures::future::join_all(submissions)); let header01a = api.push_block(1, vec![], true); let header01b = api.push_block(1, vec![], true); @@ -2213,8 +2213,6 @@ fn import_sink_works3() { assert_pool_status!(header01a.hash(), &pool, 1, 1); assert_pool_status!(header01b.hash(), &pool, 1, 1); - log::debug!("xxx {x:#?}"); - let import_events = futures::executor::block_on_stream(import_stream).take(1).collect::>(); diff --git a/substrate/client/transaction-pool/tests/fatp_common/mod.rs b/substrate/client/transaction-pool/tests/fatp_common/mod.rs index aaffebc0db0ac..530c25caf88e7 100644 --- a/substrate/client/transaction-pool/tests/fatp_common/mod.rs +++ b/substrate/client/transaction-pool/tests/fatp_common/mod.rs @@ -192,12 +192,9 @@ macro_rules! assert_ready_iterator { let output: Vec<_> = ready_iterator.collect(); log::debug!(target:LOG_TARGET, "expected: {:#?}", expected); log::debug!(target:LOG_TARGET, "output: {:#?}", output); + let output = output.into_iter().map(|t|t.hash).collect::>(); assert_eq!(expected.len(), output.len()); - assert!( - output.iter().zip(expected.iter()).all(|(o,e)| { - o.hash == *e - }) - ); + assert_eq!(output,expected); }}; } @@ -215,6 +212,18 @@ macro_rules! assert_future_iterator { }}; } +#[macro_export] +macro_rules! assert_watcher_stream { + ($stream:ident, [$( $event:expr ),*]) => {{ + let expected = vec![ $($event),*]; + log::debug!(target:LOG_TARGET, "expected: {:#?} {}, block now:", expected, expected.len()); + let output = futures::executor::block_on_stream($stream).take(expected.len()).collect::>(); + log::debug!(target:LOG_TARGET, "output: {:#?}", output); + assert_eq!(expected.len(), output.len()); + assert_eq!(output, expected); + }}; +} + pub const SOURCE: TransactionSource = TransactionSource::External; #[cfg(test)] diff --git a/substrate/client/transaction-pool/tests/fatp_prios.rs b/substrate/client/transaction-pool/tests/fatp_prios.rs index 4ed9b45038614..af5e7e8c5a6a8 100644 --- a/substrate/client/transaction-pool/tests/fatp_prios.rs +++ b/substrate/client/transaction-pool/tests/fatp_prios.rs @@ -20,13 +20,15 @@ pub mod fatp_common; -use fatp_common::{new_best_block_event, TestPoolBuilder, LOG_TARGET, SOURCE}; +use fatp_common::{invalid_hash, new_best_block_event, TestPoolBuilder, LOG_TARGET, SOURCE}; use futures::{executor::block_on, FutureExt}; use sc_transaction_pool::ChainApi; -use sc_transaction_pool_api::{MaintainedTransactionPool, TransactionPool, TransactionStatus}; +use sc_transaction_pool_api::{ + error::Error as TxPoolError, LocalTransactionPool, MaintainedTransactionPool, TransactionPool, + TransactionStatus, +}; use substrate_test_runtime_client::Sr25519Keyring::*; use substrate_test_runtime_transaction_pool::uxt; - #[test] fn fatp_prio_ready_higher_evicts_lower() { sp_tracing::try_init_simple(); @@ -247,3 +249,312 @@ fn fatp_prio_watcher_future_lower_prio_gets_dropped_from_all_views() { assert_ready_iterator!(header01.hash(), pool, [xt2, xt1]); assert_ready_iterator!(header02.hash(), pool, [xt2, xt1]); } + +#[test] +fn fatp_prios_watcher_full_mempool_higher_prio_is_accepted() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(4).with_ready_count(2).build(); + api.set_nonce(api.genesis_hash(), Bob.into(), 300); + api.set_nonce(api.genesis_hash(), Charlie.into(), 400); + api.set_nonce(api.genesis_hash(), Dave.into(), 500); + api.set_nonce(api.genesis_hash(), Eve.into(), 600); + api.set_nonce(api.genesis_hash(), Ferdie.into(), 700); + + let header01 = api.push_block(1, vec![], true); + let event = new_best_block_event(&pool, None, header01.hash()); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Bob, 300); + let xt2 = uxt(Charlie, 400); + + let xt3 = uxt(Dave, 500); + + let xt4 = uxt(Eve, 600); + let xt5 = uxt(Ferdie, 700); + + api.set_priority(&xt0, 1); + api.set_priority(&xt1, 2); + api.set_priority(&xt2, 3); + api.set_priority(&xt3, 4); + + api.set_priority(&xt4, 5); + api.set_priority(&xt5, 6); + + let xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + let xt1_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + + assert_pool_status!(header01.hash(), &pool, 2, 0); + assert_eq!(pool.mempool_len().1, 2); + + let header02 = api.push_block_with_parent(header01.hash(), vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header01.hash()), header02.hash()))); + + let _xt2_watcher = + block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone())).unwrap(); + let _xt3_watcher = + block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt3.clone())).unwrap(); + + assert_pool_status!(header02.hash(), &pool, 2, 0); + assert_eq!(pool.mempool_len().1, 4); + + let header03 = api.push_block_with_parent(header02.hash(), vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header02.hash()), header03.hash()))); + + let _xt4_watcher = + block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt4.clone())).unwrap(); + let _xt5_watcher = + block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt5.clone())).unwrap(); + + assert_pool_status!(header03.hash(), &pool, 2, 0); + assert_eq!(pool.mempool_len().1, 4); + + assert_watcher_stream!(xt0_watcher, [TransactionStatus::Ready, TransactionStatus::Dropped]); + assert_watcher_stream!(xt1_watcher, [TransactionStatus::Ready, TransactionStatus::Dropped]); + + assert_ready_iterator!(header01.hash(), pool, []); + assert_ready_iterator!(header02.hash(), pool, [xt3, xt2]); + assert_ready_iterator!(header03.hash(), pool, [xt5, xt4]); +} + +#[test] +fn fatp_prios_watcher_full_mempool_higher_prio_is_accepted_with_subtree() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(4).with_ready_count(4).build(); + api.set_nonce(api.genesis_hash(), Bob.into(), 300); + api.set_nonce(api.genesis_hash(), Charlie.into(), 400); + + let header01 = api.push_block(1, vec![], true); + let event = new_best_block_event(&pool, None, header01.hash()); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Alice, 201); + let xt2 = uxt(Alice, 202); + let xt3 = uxt(Bob, 300); + let xt4 = uxt(Charlie, 400); + + api.set_priority(&xt0, 1); + api.set_priority(&xt1, 3); + api.set_priority(&xt2, 3); + api.set_priority(&xt3, 2); + api.set_priority(&xt4, 2); + + let xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + let xt1_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + let xt2_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone())).unwrap(); + let xt3_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt3.clone())).unwrap(); + + assert_ready_iterator!(header01.hash(), pool, [xt3, xt0, xt1, xt2]); + assert_pool_status!(header01.hash(), &pool, 4, 0); + assert_eq!(pool.mempool_len().1, 4); + + let xt4_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt4.clone())).unwrap(); + assert_pool_status!(header01.hash(), &pool, 2, 0); + assert_ready_iterator!(header01.hash(), pool, [xt3, xt4]); + + assert_watcher_stream!(xt0_watcher, [TransactionStatus::Ready, TransactionStatus::Dropped]); + assert_watcher_stream!(xt1_watcher, [TransactionStatus::Ready, TransactionStatus::Dropped]); + assert_watcher_stream!(xt2_watcher, [TransactionStatus::Ready, TransactionStatus::Dropped]); + assert_watcher_stream!(xt3_watcher, [TransactionStatus::Ready]); + assert_watcher_stream!(xt4_watcher, [TransactionStatus::Ready]); +} + +#[test] +fn fatp_prios_watcher_full_mempool_higher_prio_is_accepted_with_subtree2() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(4).with_ready_count(4).build(); + api.set_nonce(api.genesis_hash(), Bob.into(), 300); + api.set_nonce(api.genesis_hash(), Charlie.into(), 400); + + let header01 = api.push_block(1, vec![], true); + let event = new_best_block_event(&pool, None, header01.hash()); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Alice, 201); + let xt2 = uxt(Alice, 202); + let xt3 = uxt(Bob, 300); + let xt4 = uxt(Charlie, 400); + + api.set_priority(&xt0, 1); + api.set_priority(&xt1, 3); + api.set_priority(&xt2, 3); + api.set_priority(&xt3, 2); + api.set_priority(&xt4, 2); + + let xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + let xt1_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + let xt2_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone())).unwrap(); + let xt3_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt3.clone())).unwrap(); + + assert_ready_iterator!(header01.hash(), pool, [xt3, xt0, xt1, xt2]); + assert_pool_status!(header01.hash(), &pool, 4, 0); + assert_eq!(pool.mempool_len().1, 4); + + let header02 = api.push_block_with_parent(header01.hash(), vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header01.hash()), header02.hash()))); + + let xt4_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt4.clone())).unwrap(); + assert_ready_iterator!(header01.hash(), pool, [xt3]); + assert_pool_status!(header02.hash(), &pool, 2, 0); + assert_ready_iterator!(header02.hash(), pool, [xt3, xt4]); + + assert_watcher_stream!(xt0_watcher, [TransactionStatus::Ready, TransactionStatus::Dropped]); + assert_watcher_stream!(xt1_watcher, [TransactionStatus::Ready, TransactionStatus::Dropped]); + assert_watcher_stream!(xt2_watcher, [TransactionStatus::Ready, TransactionStatus::Dropped]); + assert_watcher_stream!(xt3_watcher, [TransactionStatus::Ready]); + assert_watcher_stream!(xt4_watcher, [TransactionStatus::Ready]); +} + +#[test] +fn fatp_prios_watcher_full_mempool_lower_prio_gets_rejected() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(2).with_ready_count(2).build(); + api.set_nonce(api.genesis_hash(), Bob.into(), 300); + api.set_nonce(api.genesis_hash(), Charlie.into(), 400); + api.set_nonce(api.genesis_hash(), Dave.into(), 500); + + let header01 = api.push_block(1, vec![], true); + let event = new_best_block_event(&pool, None, header01.hash()); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Bob, 300); + let xt2 = uxt(Charlie, 400); + let xt3 = uxt(Dave, 500); + + api.set_priority(&xt0, 2); + api.set_priority(&xt1, 2); + api.set_priority(&xt2, 2); + api.set_priority(&xt3, 1); + + let _xt0_watcher = + block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + let _xt1_watcher = + block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + + assert_pool_status!(header01.hash(), &pool, 2, 0); + assert_eq!(pool.mempool_len().1, 2); + + let header02 = api.push_block_with_parent(header01.hash(), vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header01.hash()), header02.hash()))); + + assert_pool_status!(header02.hash(), &pool, 2, 0); + assert_eq!(pool.mempool_len().1, 2); + + assert_ready_iterator!(header01.hash(), pool, [xt0, xt1]); + assert_ready_iterator!(header02.hash(), pool, [xt0, xt1]); + + let result2 = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone())).map(|_| ()); + assert!(matches!(result2.as_ref().unwrap_err().0, TxPoolError::ImmediatelyDropped)); + let result3 = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt3.clone())).map(|_| ()); + assert!(matches!(result3.as_ref().unwrap_err().0, TxPoolError::ImmediatelyDropped)); +} + +#[test] +fn fatp_prios_watcher_full_mempool_does_not_keep_dropped_transaction() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(4).with_ready_count(2).build(); + api.set_nonce(api.genesis_hash(), Bob.into(), 300); + api.set_nonce(api.genesis_hash(), Charlie.into(), 400); + api.set_nonce(api.genesis_hash(), Dave.into(), 500); + + let header01 = api.push_block(1, vec![], true); + let event = new_best_block_event(&pool, None, header01.hash()); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Bob, 300); + let xt2 = uxt(Charlie, 400); + let xt3 = uxt(Dave, 500); + + api.set_priority(&xt0, 2); + api.set_priority(&xt1, 2); + api.set_priority(&xt2, 2); + api.set_priority(&xt3, 2); + + let xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + let xt1_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + let xt2_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone())).unwrap(); + let xt3_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt3.clone())).unwrap(); + + assert_pool_status!(header01.hash(), &pool, 2, 0); + assert_ready_iterator!(header01.hash(), pool, [xt2, xt3]); + + assert_watcher_stream!(xt0_watcher, [TransactionStatus::Ready, TransactionStatus::Dropped]); + assert_watcher_stream!(xt1_watcher, [TransactionStatus::Ready, TransactionStatus::Dropped]); + assert_watcher_stream!(xt2_watcher, [TransactionStatus::Ready]); + assert_watcher_stream!(xt3_watcher, [TransactionStatus::Ready]); +} + +#[test] +fn fatp_prios_submit_local_full_mempool_higher_prio_is_accepted() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(4).with_ready_count(2).build(); + api.set_nonce(api.genesis_hash(), Bob.into(), 300); + api.set_nonce(api.genesis_hash(), Charlie.into(), 400); + api.set_nonce(api.genesis_hash(), Dave.into(), 500); + api.set_nonce(api.genesis_hash(), Eve.into(), 600); + api.set_nonce(api.genesis_hash(), Ferdie.into(), 700); + + let header01 = api.push_block(1, vec![], true); + let event = new_best_block_event(&pool, None, header01.hash()); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Bob, 300); + let xt2 = uxt(Charlie, 400); + + let xt3 = uxt(Dave, 500); + + let xt4 = uxt(Eve, 600); + let xt5 = uxt(Ferdie, 700); + + api.set_priority(&xt0, 1); + api.set_priority(&xt1, 2); + api.set_priority(&xt2, 3); + api.set_priority(&xt3, 4); + + api.set_priority(&xt4, 5); + api.set_priority(&xt5, 6); + pool.submit_local(invalid_hash(), xt0.clone()).unwrap(); + pool.submit_local(invalid_hash(), xt1.clone()).unwrap(); + + assert_pool_status!(header01.hash(), &pool, 2, 0); + assert_eq!(pool.mempool_len().0, 2); + + let header02 = api.push_block_with_parent(header01.hash(), vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header01.hash()), header02.hash()))); + + pool.submit_local(invalid_hash(), xt2.clone()).unwrap(); + pool.submit_local(invalid_hash(), xt3.clone()).unwrap(); + + assert_pool_status!(header02.hash(), &pool, 2, 0); + assert_eq!(pool.mempool_len().0, 4); + + let header03 = api.push_block_with_parent(header02.hash(), vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header02.hash()), header03.hash()))); + + pool.submit_local(invalid_hash(), xt4.clone()).unwrap(); + pool.submit_local(invalid_hash(), xt5.clone()).unwrap(); + + assert_pool_status!(header03.hash(), &pool, 2, 0); + assert_eq!(pool.mempool_len().0, 4); + + assert_ready_iterator!(header01.hash(), pool, []); + assert_ready_iterator!(header02.hash(), pool, [xt3, xt2]); + assert_ready_iterator!(header03.hash(), pool, [xt5, xt4]); +} diff --git a/substrate/client/transaction-pool/tests/pool.rs b/substrate/client/transaction-pool/tests/pool.rs index 20997606c607c..c70f454833145 100644 --- a/substrate/client/transaction-pool/tests/pool.rs +++ b/substrate/client/transaction-pool/tests/pool.rs @@ -49,7 +49,7 @@ const LOG_TARGET: &str = "txpool"; fn pool() -> (Pool, Arc) { let api = Arc::new(TestApi::with_alice_nonce(209)); - (Pool::new(Default::default(), true.into(), api.clone()), api) + (Pool::new_with_staticly_sized_rotator(Default::default(), true.into(), api.clone()), api) } fn maintained_pool() -> (BasicPool, Arc, futures::executor::ThreadPool) { @@ -158,6 +158,7 @@ fn prune_tags_should_work() { let (pool, api) = pool(); let hash209 = block_on(pool.submit_one(&api.expect_hash_and_number(0), TSOURCE, uxt(Alice, 209).into())) + .map(|o| o.hash()) .unwrap(); block_on(pool.submit_one(&api.expect_hash_and_number(0), TSOURCE, uxt(Alice, 210).into())) .unwrap(); @@ -184,10 +185,13 @@ fn prune_tags_should_work() { fn should_ban_invalid_transactions() { let (pool, api) = pool(); let uxt = Arc::from(uxt(Alice, 209)); - let hash = - block_on(pool.submit_one(&api.expect_hash_and_number(0), TSOURCE, uxt.clone())).unwrap(); + let hash = block_on(pool.submit_one(&api.expect_hash_and_number(0), TSOURCE, uxt.clone())) + .unwrap() + .hash(); pool.validated_pool().remove_invalid(&[hash]); - block_on(pool.submit_one(&api.expect_hash_and_number(0), TSOURCE, uxt.clone())).unwrap_err(); + block_on(pool.submit_one(&api.expect_hash_and_number(0), TSOURCE, uxt.clone())) + .map(|_| ()) + .unwrap_err(); // when let pending: Vec<_> = pool @@ -198,7 +202,9 @@ fn should_ban_invalid_transactions() { assert_eq!(pending, Vec::::new()); // then - block_on(pool.submit_one(&api.expect_hash_and_number(0), TSOURCE, uxt.clone())).unwrap_err(); + block_on(pool.submit_one(&api.expect_hash_and_number(0), TSOURCE, uxt.clone())) + .map(|_| ()) + .unwrap_err(); } #[test] @@ -224,7 +230,7 @@ fn should_correctly_prune_transactions_providing_more_than_one_tag() { api.set_valid_modifier(Box::new(|v: &mut ValidTransaction| { v.provides.push(vec![155]); })); - let pool = Pool::new(Default::default(), true.into(), api.clone()); + let pool = Pool::new_with_staticly_sized_rotator(Default::default(), true.into(), api.clone()); let xt0 = Arc::from(uxt(Alice, 209)); block_on(pool.submit_one(&api.expect_hash_and_number(0), TSOURCE, xt0.clone())) .expect("1. Imported"); diff --git a/substrate/frame/grandpa/src/benchmarking.rs b/substrate/frame/grandpa/src/benchmarking.rs index 0a10e58827761..56048efa22cae 100644 --- a/substrate/frame/grandpa/src/benchmarking.rs +++ b/substrate/frame/grandpa/src/benchmarking.rs @@ -17,7 +17,7 @@ //! Benchmarks for the GRANDPA pallet. -use super::{Pallet as Grandpa, *}; +use super::*; use frame_benchmarking::v2::*; use frame_system::RawOrigin; use sp_core::H256; @@ -69,7 +69,7 @@ mod benchmarks { #[extrinsic_call] _(RawOrigin::Root, delay, best_finalized_block_number); - assert!(Grandpa::::stalled().is_some()); + assert!(Stalled::::get().is_some()); } impl_benchmark_test_suite!( diff --git a/substrate/frame/grandpa/src/equivocation.rs b/substrate/frame/grandpa/src/equivocation.rs index 2366c957e9ab1..4ebdbc1eecd30 100644 --- a/substrate/frame/grandpa/src/equivocation.rs +++ b/substrate/frame/grandpa/src/equivocation.rs @@ -177,7 +177,7 @@ where evidence: (EquivocationProof>, T::KeyOwnerProof), ) -> Result<(), DispatchError> { let (equivocation_proof, key_owner_proof) = evidence; - let reporter = reporter.or_else(|| >::author()); + let reporter = reporter.or_else(|| pallet_authorship::Pallet::::author()); let offender = equivocation_proof.offender().clone(); // We check the equivocation within the context of its set id (and diff --git a/substrate/frame/grandpa/src/lib.rs b/substrate/frame/grandpa/src/lib.rs index 4f69aeaef5236..9017eec2ca8f8 100644 --- a/substrate/frame/grandpa/src/lib.rs +++ b/substrate/frame/grandpa/src/lib.rs @@ -127,7 +127,7 @@ pub mod pallet { impl Hooks> for Pallet { fn on_finalize(block_number: BlockNumberFor) { // check for scheduled pending authority set changes - if let Some(pending_change) = >::get() { + if let Some(pending_change) = PendingChange::::get() { // emit signal if we're at the block that scheduled the change if block_number == pending_change.scheduled_at { let next_authorities = pending_change.next_authorities.to_vec(); @@ -150,12 +150,12 @@ pub mod pallet { Self::deposit_event(Event::NewAuthorities { authority_set: pending_change.next_authorities.into_inner(), }); - >::kill(); + PendingChange::::kill(); } } // check for scheduled pending state changes - match >::get() { + match State::::get() { StoredState::PendingPause { scheduled_at, delay } => { // signal change to pause if block_number == scheduled_at { @@ -164,7 +164,7 @@ pub mod pallet { // enact change to paused state if block_number == scheduled_at + delay { - >::put(StoredState::Paused); + State::::put(StoredState::Paused); Self::deposit_event(Event::Paused); } }, @@ -176,7 +176,7 @@ pub mod pallet { // enact change to live state if block_number == scheduled_at + delay { - >::put(StoredState::Live); + State::::put(StoredState::Live); Self::deposit_event(Event::Resumed); } }, @@ -297,37 +297,32 @@ pub mod pallet { } #[pallet::type_value] - pub(super) fn DefaultForState() -> StoredState> { + pub fn DefaultForState() -> StoredState> { StoredState::Live } /// State of the current authority set. #[pallet::storage] - #[pallet::getter(fn state)] - pub(super) type State = + pub type State = StorageValue<_, StoredState>, ValueQuery, DefaultForState>; /// Pending change: (signaled at, scheduled change). #[pallet::storage] - #[pallet::getter(fn pending_change)] - pub(super) type PendingChange = + pub type PendingChange = StorageValue<_, StoredPendingChange, T::MaxAuthorities>>; /// next block number where we can force a change. #[pallet::storage] - #[pallet::getter(fn next_forced)] - pub(super) type NextForced = StorageValue<_, BlockNumberFor>; + pub type NextForced = StorageValue<_, BlockNumberFor>; /// `true` if we are currently stalled. #[pallet::storage] - #[pallet::getter(fn stalled)] - pub(super) type Stalled = StorageValue<_, (BlockNumberFor, BlockNumberFor)>; + pub type Stalled = StorageValue<_, (BlockNumberFor, BlockNumberFor)>; /// The number of changes (both in terms of keys and underlying economic responsibilities) /// in the "set" of Grandpa validators from genesis. #[pallet::storage] - #[pallet::getter(fn current_set_id)] - pub(super) type CurrentSetId = StorageValue<_, SetId, ValueQuery>; + pub type CurrentSetId = StorageValue<_, SetId, ValueQuery>; /// A mapping from grandpa set ID to the index of the *most recent* session for which its /// members were responsible. @@ -340,12 +335,11 @@ pub mod pallet { /// /// TWOX-NOTE: `SetId` is not under user control. #[pallet::storage] - #[pallet::getter(fn session_for_set)] - pub(super) type SetIdSession = StorageMap<_, Twox64Concat, SetId, SessionIndex>; + pub type SetIdSession = StorageMap<_, Twox64Concat, SetId, SessionIndex>; /// The current list of authorities. #[pallet::storage] - pub(crate) type Authorities = + pub type Authorities = StorageValue<_, BoundedAuthorityList, ValueQuery>; #[derive(frame_support::DefaultNoBound)] @@ -432,6 +426,44 @@ pub enum StoredState { } impl Pallet { + /// State of the current authority set. + pub fn state() -> StoredState> { + State::::get() + } + + /// Pending change: (signaled at, scheduled change). + pub fn pending_change() -> Option, T::MaxAuthorities>> { + PendingChange::::get() + } + + /// next block number where we can force a change. + pub fn next_forced() -> Option> { + NextForced::::get() + } + + /// `true` if we are currently stalled. + pub fn stalled() -> Option<(BlockNumberFor, BlockNumberFor)> { + Stalled::::get() + } + + /// The number of changes (both in terms of keys and underlying economic responsibilities) + /// in the "set" of Grandpa validators from genesis. + pub fn current_set_id() -> SetId { + CurrentSetId::::get() + } + + /// A mapping from grandpa set ID to the index of the *most recent* session for which its + /// members were responsible. + /// + /// This is only used for validating equivocation proofs. An equivocation proof must + /// contains a key-ownership proof for a given session, therefore we need a way to tie + /// together sessions and GRANDPA set ids, i.e. we need to validate that a validator + /// was the owner of a given key on a given session, and what the active set ID was + /// during that session. + pub fn session_for_set(set_id: SetId) -> Option { + SetIdSession::::get(set_id) + } + /// Get the current set of authorities, along with their respective weights. pub fn grandpa_authorities() -> AuthorityList { Authorities::::get().into_inner() @@ -440,9 +472,9 @@ impl Pallet { /// Schedule GRANDPA to pause starting in the given number of blocks. /// Cannot be done when already paused. pub fn schedule_pause(in_blocks: BlockNumberFor) -> DispatchResult { - if let StoredState::Live = >::get() { - let scheduled_at = >::block_number(); - >::put(StoredState::PendingPause { delay: in_blocks, scheduled_at }); + if let StoredState::Live = State::::get() { + let scheduled_at = frame_system::Pallet::::block_number(); + State::::put(StoredState::PendingPause { delay: in_blocks, scheduled_at }); Ok(()) } else { @@ -452,9 +484,9 @@ impl Pallet { /// Schedule a resume of GRANDPA after pausing. pub fn schedule_resume(in_blocks: BlockNumberFor) -> DispatchResult { - if let StoredState::Paused = >::get() { - let scheduled_at = >::block_number(); - >::put(StoredState::PendingResume { delay: in_blocks, scheduled_at }); + if let StoredState::Paused = State::::get() { + let scheduled_at = frame_system::Pallet::::block_number(); + State::::put(StoredState::PendingResume { delay: in_blocks, scheduled_at }); Ok(()) } else { @@ -481,17 +513,17 @@ impl Pallet { in_blocks: BlockNumberFor, forced: Option>, ) -> DispatchResult { - if !>::exists() { - let scheduled_at = >::block_number(); + if !PendingChange::::exists() { + let scheduled_at = frame_system::Pallet::::block_number(); if forced.is_some() { - if Self::next_forced().map_or(false, |next| next > scheduled_at) { + if NextForced::::get().map_or(false, |next| next > scheduled_at) { return Err(Error::::TooSoon.into()) } // only allow the next forced change when twice the window has passed since // this one. - >::put(scheduled_at + in_blocks * 2u32.into()); + NextForced::::put(scheduled_at + in_blocks * 2u32.into()); } let next_authorities = WeakBoundedVec::<_, T::MaxAuthorities>::force_from( @@ -502,7 +534,7 @@ impl Pallet { ), ); - >::put(StoredPendingChange { + PendingChange::::put(StoredPendingChange { delay: in_blocks, scheduled_at, next_authorities, @@ -518,7 +550,7 @@ impl Pallet { /// Deposit one of this module's logs. fn deposit_log(log: ConsensusLog>) { let log = DigestItem::Consensus(GRANDPA_ENGINE_ID, log.encode()); - >::deposit_log(log); + frame_system::Pallet::::deposit_log(log); } // Perform module initialization, abstracted so that it can be called either through genesis @@ -554,7 +586,7 @@ impl Pallet { // when we record old authority sets we could try to figure out _who_ // failed. until then, we can't meaningfully guard against // `next == last` the way that normal session changes do. - >::put((further_wait, median)); + Stalled::::put((further_wait, median)); } } @@ -583,10 +615,10 @@ where // Always issue a change if `session` says that the validators have changed. // Even if their session keys are the same as before, the underlying economic // identities have changed. - let current_set_id = if changed || >::exists() { + let current_set_id = if changed || Stalled::::exists() { let next_authorities = validators.map(|(_, k)| (k, 1)).collect::>(); - let res = if let Some((further_wait, median)) = >::take() { + let res = if let Some((further_wait, median)) = Stalled::::take() { Self::schedule_change(next_authorities, further_wait, Some(median)) } else { Self::schedule_change(next_authorities, Zero::zero(), None) @@ -608,17 +640,17 @@ where // either the session module signalled that the validators have changed // or the set was stalled. but since we didn't successfully schedule // an authority set change we do not increment the set id. - Self::current_set_id() + CurrentSetId::::get() } } else { // nothing's changed, neither economic conditions nor session keys. update the pointer // of the current set. - Self::current_set_id() + CurrentSetId::::get() }; // update the mapping to note that the current set corresponds to the // latest equivalent session (i.e. now). - let session_index = >::current_index(); + let session_index = pallet_session::Pallet::::current_index(); SetIdSession::::insert(current_set_id, &session_index); } diff --git a/substrate/frame/grandpa/src/tests.rs b/substrate/frame/grandpa/src/tests.rs index 383f77f00de71..f4720966b1797 100644 --- a/substrate/frame/grandpa/src/tests.rs +++ b/substrate/frame/grandpa/src/tests.rs @@ -110,7 +110,7 @@ fn cannot_schedule_change_when_one_pending() { new_test_ext(vec![(1, 1), (2, 1), (3, 1)]).execute_with(|| { initialize_block(1, Default::default()); Grandpa::schedule_change(to_authorities(vec![(4, 1), (5, 1), (6, 1)]), 1, None).unwrap(); - assert!(>::exists()); + assert!(PendingChange::::exists()); assert_noop!( Grandpa::schedule_change(to_authorities(vec![(5, 1)]), 1, None), Error::::ChangePending @@ -120,7 +120,7 @@ fn cannot_schedule_change_when_one_pending() { let header = System::finalize(); initialize_block(2, header.hash()); - assert!(>::exists()); + assert!(PendingChange::::exists()); assert_noop!( Grandpa::schedule_change(to_authorities(vec![(5, 1)]), 1, None), Error::::ChangePending @@ -130,7 +130,7 @@ fn cannot_schedule_change_when_one_pending() { let header = System::finalize(); initialize_block(3, header.hash()); - assert!(!>::exists()); + assert!(!PendingChange::::exists()); assert_ok!(Grandpa::schedule_change(to_authorities(vec![(5, 1)]), 1, None)); Grandpa::on_finalize(3); @@ -144,7 +144,7 @@ fn dispatch_forced_change() { initialize_block(1, Default::default()); Grandpa::schedule_change(to_authorities(vec![(4, 1), (5, 1), (6, 1)]), 5, Some(0)).unwrap(); - assert!(>::exists()); + assert!(PendingChange::::exists()); assert_noop!( Grandpa::schedule_change(to_authorities(vec![(5, 1)]), 1, Some(0)), Error::::ChangePending @@ -155,8 +155,8 @@ fn dispatch_forced_change() { for i in 2..7 { initialize_block(i, header.hash()); - assert!(>::get().unwrap().forced.is_some()); - assert_eq!(Grandpa::next_forced(), Some(11)); + assert!(PendingChange::::get().unwrap().forced.is_some()); + assert_eq!(NextForced::::get(), Some(11)); assert_noop!( Grandpa::schedule_change(to_authorities(vec![(5, 1)]), 1, None), Error::::ChangePending @@ -174,7 +174,7 @@ fn dispatch_forced_change() { // add a normal change. { initialize_block(7, header.hash()); - assert!(!>::exists()); + assert!(!PendingChange::::exists()); assert_eq!( Grandpa::grandpa_authorities(), to_authorities(vec![(4, 1), (5, 1), (6, 1)]) @@ -187,7 +187,7 @@ fn dispatch_forced_change() { // run the normal change. { initialize_block(8, header.hash()); - assert!(>::exists()); + assert!(PendingChange::::exists()); assert_eq!( Grandpa::grandpa_authorities(), to_authorities(vec![(4, 1), (5, 1), (6, 1)]) @@ -204,9 +204,9 @@ fn dispatch_forced_change() { // time. for i in 9..11 { initialize_block(i, header.hash()); - assert!(!>::exists()); + assert!(!PendingChange::::exists()); assert_eq!(Grandpa::grandpa_authorities(), to_authorities(vec![(5, 1)])); - assert_eq!(Grandpa::next_forced(), Some(11)); + assert_eq!(NextForced::::get(), Some(11)); assert_noop!( Grandpa::schedule_change(to_authorities(vec![(5, 1), (6, 1)]), 5, Some(0)), Error::::TooSoon @@ -217,13 +217,13 @@ fn dispatch_forced_change() { { initialize_block(11, header.hash()); - assert!(!>::exists()); + assert!(!PendingChange::::exists()); assert_ok!(Grandpa::schedule_change( to_authorities(vec![(5, 1), (6, 1), (7, 1)]), 5, Some(0) )); - assert_eq!(Grandpa::next_forced(), Some(21)); + assert_eq!(NextForced::::get(), Some(21)); Grandpa::on_finalize(11); header = System::finalize(); } @@ -239,7 +239,10 @@ fn schedule_pause_only_when_live() { Grandpa::schedule_pause(1).unwrap(); // we've switched to the pending pause state - assert_eq!(Grandpa::state(), StoredState::PendingPause { scheduled_at: 1u64, delay: 1 }); + assert_eq!( + State::::get(), + StoredState::PendingPause { scheduled_at: 1u64, delay: 1 } + ); Grandpa::on_finalize(1); let _ = System::finalize(); @@ -253,7 +256,7 @@ fn schedule_pause_only_when_live() { let _ = System::finalize(); // after finalizing block 2 the set should have switched to paused state - assert_eq!(Grandpa::state(), StoredState::Paused); + assert_eq!(State::::get(), StoredState::Paused); }); } @@ -265,14 +268,14 @@ fn schedule_resume_only_when_paused() { // the set is currently live, resuming it is an error assert_noop!(Grandpa::schedule_resume(1), Error::::ResumeFailed); - assert_eq!(Grandpa::state(), StoredState::Live); + assert_eq!(State::::get(), StoredState::Live); // we schedule a pause to be applied instantly Grandpa::schedule_pause(0).unwrap(); Grandpa::on_finalize(1); let _ = System::finalize(); - assert_eq!(Grandpa::state(), StoredState::Paused); + assert_eq!(State::::get(), StoredState::Paused); // we schedule the set to go back live in 2 blocks initialize_block(2, Default::default()); @@ -289,7 +292,7 @@ fn schedule_resume_only_when_paused() { let _ = System::finalize(); // it should be live at block 4 - assert_eq!(Grandpa::state(), StoredState::Live); + assert_eq!(State::::get(), StoredState::Live); }); } @@ -342,7 +345,7 @@ fn report_equivocation_current_set_works() { let equivocation_key = &authorities[equivocation_authority_index].0; let equivocation_keyring = extract_keyring(equivocation_key); - let set_id = Grandpa::current_set_id(); + let set_id = CurrentSetId::::get(); // generate an equivocation proof, with two votes in the same round for // different block hashes signed by the same key @@ -424,7 +427,7 @@ fn report_equivocation_old_set_works() { let equivocation_keyring = extract_keyring(equivocation_key); - let set_id = Grandpa::current_set_id(); + let set_id = CurrentSetId::::get(); // generate an equivocation proof for the old set, let equivocation_proof = generate_equivocation_proof( @@ -487,7 +490,7 @@ fn report_equivocation_invalid_set_id() { let key_owner_proof = Historical::prove((sp_consensus_grandpa::KEY_TYPE, &equivocation_key)).unwrap(); - let set_id = Grandpa::current_set_id(); + let set_id = CurrentSetId::::get(); // generate an equivocation for a future set let equivocation_proof = generate_equivocation_proof( @@ -527,7 +530,7 @@ fn report_equivocation_invalid_session() { start_era(2); - let set_id = Grandpa::current_set_id(); + let set_id = CurrentSetId::::get(); // generate an equivocation proof at set id = 2 let equivocation_proof = generate_equivocation_proof( @@ -568,7 +571,7 @@ fn report_equivocation_invalid_key_owner_proof() { let equivocation_key = &authorities[equivocation_authority_index].0; let equivocation_keyring = extract_keyring(equivocation_key); - let set_id = Grandpa::current_set_id(); + let set_id = CurrentSetId::::get(); // generate an equivocation proof for the authority at index 0 let equivocation_proof = generate_equivocation_proof( @@ -611,7 +614,7 @@ fn report_equivocation_invalid_equivocation_proof() { let key_owner_proof = Historical::prove((sp_consensus_grandpa::KEY_TYPE, &equivocation_key)).unwrap(); - let set_id = Grandpa::current_set_id(); + let set_id = CurrentSetId::::get(); let assert_invalid_equivocation_proof = |equivocation_proof| { assert_err!( @@ -675,7 +678,7 @@ fn report_equivocation_validate_unsigned_prevents_duplicates() { let equivocation_authority_index = 0; let equivocation_key = &authorities[equivocation_authority_index].0; let equivocation_keyring = extract_keyring(equivocation_key); - let set_id = Grandpa::current_set_id(); + let set_id = CurrentSetId::::get(); let equivocation_proof = generate_equivocation_proof( set_id, @@ -748,12 +751,12 @@ fn report_equivocation_validate_unsigned_prevents_duplicates() { #[test] fn on_new_session_doesnt_start_new_set_if_schedule_change_failed() { new_test_ext(vec![(1, 1), (2, 1), (3, 1)]).execute_with(|| { - assert_eq!(Grandpa::current_set_id(), 0); + assert_eq!(CurrentSetId::::get(), 0); // starting a new era should lead to a change in the session // validators and trigger a new set start_era(1); - assert_eq!(Grandpa::current_set_id(), 1); + assert_eq!(CurrentSetId::::get(), 1); // we schedule a change delayed by 2 blocks, this should make it so that // when we try to rotate the session at the beginning of the era we will @@ -761,22 +764,22 @@ fn on_new_session_doesnt_start_new_set_if_schedule_change_failed() { // not increment the set id. Grandpa::schedule_change(to_authorities(vec![(1, 1)]), 2, None).unwrap(); start_era(2); - assert_eq!(Grandpa::current_set_id(), 1); + assert_eq!(CurrentSetId::::get(), 1); // everything should go back to normal after. start_era(3); - assert_eq!(Grandpa::current_set_id(), 2); + assert_eq!(CurrentSetId::::get(), 2); // session rotation might also fail to schedule a change if it's for a // forced change (i.e. grandpa is stalled) and it is too soon. - >::put(1000); - >::put((30, 1)); + NextForced::::put(1000); + Stalled::::put((30, 1)); // NOTE: we cannot go through normal era rotation since having `Stalled` // defined will also trigger a new set (regardless of whether the // session validators changed) Grandpa::on_new_session(true, std::iter::empty(), std::iter::empty()); - assert_eq!(Grandpa::current_set_id(), 2); + assert_eq!(CurrentSetId::::get(), 2); }); } @@ -790,19 +793,19 @@ fn cleans_up_old_set_id_session_mappings() { // we should have a session id mapping for all the set ids from // `max_set_id_session_entries` eras we have observed for i in 1..=max_set_id_session_entries { - assert!(Grandpa::session_for_set(i as u64).is_some()); + assert!(SetIdSession::::get(i as u64).is_some()); } start_era(max_set_id_session_entries * 2); // we should keep tracking the new mappings for new eras for i in max_set_id_session_entries + 1..=max_set_id_session_entries * 2 { - assert!(Grandpa::session_for_set(i as u64).is_some()); + assert!(SetIdSession::::get(i as u64).is_some()); } // but the old ones should have been pruned by now for i in 1..=max_set_id_session_entries { - assert!(Grandpa::session_for_set(i as u64).is_none()); + assert!(SetIdSession::::get(i as u64).is_none()); } }); } @@ -812,24 +815,24 @@ fn always_schedules_a_change_on_new_session_when_stalled() { new_test_ext(vec![(1, 1), (2, 1), (3, 1)]).execute_with(|| { start_era(1); - assert!(Grandpa::pending_change().is_none()); - assert_eq!(Grandpa::current_set_id(), 1); + assert!(PendingChange::::get().is_none()); + assert_eq!(CurrentSetId::::get(), 1); // if the session handler reports no change then we should not schedule // any pending change Grandpa::on_new_session(false, std::iter::empty(), std::iter::empty()); - assert!(Grandpa::pending_change().is_none()); - assert_eq!(Grandpa::current_set_id(), 1); + assert!(PendingChange::::get().is_none()); + assert_eq!(CurrentSetId::::get(), 1); // if grandpa is stalled then we should **always** schedule a forced // change on a new session - >::put((10, 1)); + Stalled::::put((10, 1)); Grandpa::on_new_session(false, std::iter::empty(), std::iter::empty()); - assert!(Grandpa::pending_change().is_some()); - assert!(Grandpa::pending_change().unwrap().forced.is_some()); - assert_eq!(Grandpa::current_set_id(), 2); + assert!(PendingChange::::get().is_some()); + assert!(PendingChange::::get().unwrap().forced.is_some()); + assert_eq!(CurrentSetId::::get(), 2); }); } @@ -861,7 +864,7 @@ fn valid_equivocation_reports_dont_pay_fees() { let equivocation_key = &Grandpa::grandpa_authorities()[0].0; let equivocation_keyring = extract_keyring(equivocation_key); - let set_id = Grandpa::current_set_id(); + let set_id = CurrentSetId::::get(); // generate an equivocation proof. let equivocation_proof = generate_equivocation_proof( diff --git a/substrate/frame/revive/rpc/src/client.rs b/substrate/frame/revive/rpc/src/client.rs index 5684d1906f46b..ef5bd8f19977a 100644 --- a/substrate/frame/revive/rpc/src/client.rs +++ b/substrate/frame/revive/rpc/src/client.rs @@ -111,6 +111,7 @@ pub enum ClientError { /// The block hash was not found. #[error("hash not found")] BlockNotFound, + #[error("No Ethereum extrinsic found")] EthExtrinsicNotFound, /// The transaction fee could not be found @@ -478,7 +479,7 @@ impl Client { } /// Get receipts count per block. - pub async fn receipts_count_per_block(&self, block_hash: &SubstrateBlockHash) -> Option { + pub async fn receipts_count_per_block(&self, block_hash: &SubstrateBlockHash) -> Option { self.receipt_provider.receipts_count_per_block(block_hash).await } diff --git a/substrate/test-utils/runtime/transaction-pool/src/lib.rs b/substrate/test-utils/runtime/transaction-pool/src/lib.rs index 93e5855eefc6c..f88694fb1071e 100644 --- a/substrate/test-utils/runtime/transaction-pool/src/lib.rs +++ b/substrate/test-utils/runtime/transaction-pool/src/lib.rs @@ -352,9 +352,18 @@ impl ChainApi for TestApi { fn validate_transaction( &self, at: ::Hash, - _source: TransactionSource, + source: TransactionSource, uxt: Arc<::Extrinsic>, ) -> Self::ValidationFuture { + ready(self.validate_transaction_blocking(at, source, uxt)) + } + + fn validate_transaction_blocking( + &self, + at: ::Hash, + _source: TransactionSource, + uxt: Arc<::Extrinsic>, + ) -> Result { let uxt = (*uxt).clone(); self.validation_requests.write().push(uxt.clone()); let block_number; @@ -374,16 +383,12 @@ impl ChainApi for TestApi { // the transaction. (This is not required for this test function, but in real // environment it would fail because of this). if !found_best { - return ready(Ok(Err(TransactionValidityError::Invalid( - InvalidTransaction::Custom(1), - )))) + return Ok(Err(TransactionValidityError::Invalid(InvalidTransaction::Custom(1)))) } }, Ok(None) => - return ready(Ok(Err(TransactionValidityError::Invalid( - InvalidTransaction::Custom(2), - )))), - Err(e) => return ready(Err(e)), + return Ok(Err(TransactionValidityError::Invalid(InvalidTransaction::Custom(2)))), + Err(e) => return Err(e), } let (requires, provides) = if let Ok(transfer) = TransferData::try_from(&uxt) { @@ -423,7 +428,7 @@ impl ChainApi for TestApi { if self.enable_stale_check && transfer.nonce < chain_nonce { log::info!("test_api::validate_transaction: invalid_transaction(stale)...."); - return ready(Ok(Err(TransactionValidityError::Invalid(InvalidTransaction::Stale)))) + return Ok(Err(TransactionValidityError::Invalid(InvalidTransaction::Stale))) } (requires, provides) @@ -433,7 +438,7 @@ impl ChainApi for TestApi { if self.chain.read().invalid_hashes.contains(&self.hash_and_length(&uxt).0) { log::info!("test_api::validate_transaction: invalid_transaction...."); - return ready(Ok(Err(TransactionValidityError::Invalid(InvalidTransaction::Custom(0))))) + return Ok(Err(TransactionValidityError::Invalid(InvalidTransaction::Custom(0)))) } let priority = self.chain.read().priorities.get(&self.hash_and_length(&uxt).0).cloned(); @@ -447,16 +452,7 @@ impl ChainApi for TestApi { (self.valid_modifier.read())(&mut validity); - ready(Ok(Ok(validity))) - } - - fn validate_transaction_blocking( - &self, - _at: ::Hash, - _source: TransactionSource, - _uxt: Arc<::Extrinsic>, - ) -> Result { - unimplemented!(); + Ok(Ok(validity)) } fn block_id_to_number(