Skip to content

Commit

Permalink
Asynchronous backing PR (#2300)
Browse files Browse the repository at this point in the history
* Update substrate & polkadot

* min changes to make async backing compile

* (async backing) parachain-system: track limitations for unincluded blocks (paritytech#2438)

* unincluded segment draft

* read para head from storage proof

* read_para_head -> read_included_para_head

* Provide pub interface

* add errors

* fix unincluded segment update

* BlockTracker -> Ancestor

* add a dmp limit

* Read para head depending on the storage switch

* doc comments

* storage items docs

* add a sanity check on block initialize

* Check watermark

* append to the segment on block finalize

* Move segment update into set_validation_data

* Resolve para head todo

* option watermark

* fix comment

* Drop dmq check

* fix weight

* doc-comments on inherent invariant

* Remove TODO

* add todo

* primitives tests

* pallet tests

* doc comments

* refactor unincluded segment length into a ConsensusHook (paritytech#2501)

* refactor unincluded segment length into a ConsensusHook

* add docs

* refactor bandwidth_out calculation

Co-authored-by: Chris Sosnin <[email protected]>

* test for limits from impl

* fmt

* make tests compile

* update comment

* uncomment test

* fix collator test by adding parent to state proof

* patch HRMP watermark rules for unincluded segment

* get consensus-common tests to pass, using unincluded segment

* fix unincluded segment tests

* get all tests passing

* fmt

* rustdoc CI

* aura-ext: limit the number of authored blocks per slot (paritytech#2551)

* aura_ext consensus hook

* reverse dependency

* include weight into hook

* fix tests

* remove stray println

Co-authored-by: Chris Sosnin <[email protected]>

* fix test warning

* fix doc link

---------

Co-authored-by: Chris Sosnin <[email protected]>
Co-authored-by: Chris Sosnin <[email protected]>

* parachain-system: ignore go ahead signal once upgrade is processed (paritytech#2594)

* handle goahead signal for unincluded segment

* doc comment

* add test

* parachain-system: drop processed messages from inherent data (paritytech#2590)

* implement `drop_processed_messages`

* drop messages based on relay parent number

* adjust tests

* drop changes to mqc

* fix comment

* drop test

* drop more dead code

* clippy

* aura-ext: check slot in consensus hook and remove all `CheckInherents` logic (paritytech#2658)

* aura-ext: check slot in consensus hook

* convert relay chain slot

* Make relay chain slot duration generic

* use fixed velocity hook for pallets with aura

* purge timestamp inherent

* fix warning

* adjust runtime tests

* fix slots in tests

* Make `xcm-emulator` test pass for new consensus hook (paritytech#2722)

* add pallets on_initialize

* tests pass

* add AuraExt on_init

* ".git/.scripts/commands/fmt/fmt.sh"

---------

Co-authored-by: command-bot <>

---------

Co-authored-by: Ignacio Palacios <[email protected]>

* update polkadot git refs

* CollationGenerationConfig closure is now optional (paritytech#2772)

* CollationGenerationConfig closure is now optional

* fix test

* propagate network-protocol-staging feature (paritytech#2899)

* Feature Flagging Consensus Hook Type Parameter (paritytech#2911)

* First pass

* fmt

* Added as default feature in tomls

* Changed to direct dependency feature

* Dealing with clippy error

* Update pallets/parachain-system/src/lib.rs

Co-authored-by: asynchronous rob <[email protected]>

---------

Co-authored-by: asynchronous rob <[email protected]>

* fmt

* bump deps and remove warning

* parachain-system: update RelevantMessagingState according to the unincluded segment (paritytech#2948)

* mostly address 2471 with a bug introduced

* adjust relevant messaging state after computing total

* fmt

* max -> min

* fix test implementation of xcmp source

* add test

* fix test message sending logic

* fix + test

* add more to unincluded segment test

* fmt

---------

Co-authored-by: Chris Sosnin <[email protected]>

* Integrate new Aura / Parachain Consensus Logic in Parachain-Template / Polkadot-Parachain (paritytech#2864)

* add a comment

* refactor client/service utilities

* deprecate start_collator

* update parachain-template

* update test-service in the same way

* update polkadot-parachain crate

* fmt

* wire up new SubmitCollation message

* some runtime utilities for implementing unincluded segment runtime APIs

* allow parachains to configure their level of sybil-resistance when starting the network

* make aura-ext compile

* update to specify sybil resistance levels

* fmt

* specify relay chain slot duration in milliseconds

* update Aura to explicitly produce Send futures

also, make relay_chain_slot_duration a Duration

* add authoring duration to basic collator and document params

* integrate new basic collator into parachain-template

* remove assert_send used for testing

* basic-aura: only author when parent included

* update polkadot-parachain-bin

* fmt

* some fixes

* fixes

* add a RelayNumberMonotonicallyIncreases

* add a utility function for initializing subsystems

* some logging for timestamp adjustment

* fmt

* some fixes for lookahead collator

* add a log

* update `find_potential_parents` to account for sessions

* bound the loop

* restore & deprecate old start_collator and start_full_node functions.

* remove unnecessary await calls

* fix warning

* clippy

* more clippy

* remove unneeded logic

* ci

* update comment

Co-authored-by: Marcin S. <[email protected]>

* (async backing) restore `CheckInherents` for backwards-compatibility (paritytech#2977)

* bring back timestamp

* Restore CheckInherents

* revert to empty CheckInherents

* make CheckInherents optional

* attempt

* properly end system blocks

* add some more comments

* ignore failing system parachain tests

* update refs after main feature branch merge

* comment out the offending tests because CI runs ignored tests

* fix warnings

* fmt

* revert to polkadot master

* cargo update -p polkadot-primitives -p sp-io

---------

Co-authored-by: asynchronous rob <[email protected]>
Co-authored-by: Ignacio Palacios <[email protected]>
Co-authored-by: Bradley Olson <[email protected]>
Co-authored-by: Marcin S. <[email protected]>
Co-authored-by: eskimor <[email protected]>
Co-authored-by: Andronik <[email protected]>
  • Loading branch information
7 people committed Aug 18, 2023
1 parent 4f699c7 commit 6ef1117
Show file tree
Hide file tree
Showing 105 changed files with 4,723 additions and 2,375 deletions.
785 changes: 421 additions & 364 deletions cumulus/Cargo.lock

Large diffs are not rendered by default.

4 changes: 2 additions & 2 deletions cumulus/bridges/bin/runtime-common/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -41,8 +41,8 @@ sp-std = { git = "https://github.com/paritytech/substrate", branch = "master", d
sp-trie = { git = "https://github.com/paritytech/substrate", branch = "master", default-features = false }

# Polkadot dependencies
xcm = { git = "https://github.com/paritytech/polkadot", branch = "master", default-features = false }
xcm-builder = { git = "https://github.com/paritytech/polkadot", branch = "master", default-features = false }
xcm = { git = "https://github.com/paritytech/polkadot", default-features = false , branch = "master" }
xcm-builder = { git = "https://github.com/paritytech/polkadot", default-features = false , branch = "master" }

[dev-dependencies]
bp-test-utils = { path = "../../primitives/test-utils" }
Expand Down
4 changes: 2 additions & 2 deletions cumulus/bridges/modules/xcm-bridge-hub-router/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,8 @@ sp-std = { git = "https://github.com/paritytech/substrate", branch = "master", d

# Polkadot Dependencies

xcm = { git = "https://github.com/paritytech/polkadot", branch = "master", default-features = false }
xcm-builder = { git = "https://github.com/paritytech/polkadot", branch = "master", default-features = false }
xcm = { git = "https://github.com/paritytech/polkadot", default-features = false , branch = "master" }
xcm-builder = { git = "https://github.com/paritytech/polkadot", default-features = false , branch = "master" }

[dev-dependencies]
sp-io = { git = "https://github.com/paritytech/substrate", branch = "master" }
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ sp-api = { git = "https://github.com/paritytech/substrate", branch = "master", d
sp-std = { git = "https://github.com/paritytech/substrate", branch = "master", default-features = false }

# Polkadot Dependencies
polkadot-primitives = { git = "https://github.com/paritytech/polkadot", branch = "master", default-features = false }
polkadot-primitives = { git = "https://github.com/paritytech/polkadot", default-features = false , branch = "master" }

[features]
default = ["std"]
Expand Down
1 change: 1 addition & 0 deletions cumulus/client/collator/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -42,3 +42,4 @@ polkadot-node-subsystem-test-helpers = { git = "https://github.com/paritytech/po
# Cumulus
cumulus-test-client = { path = "../../test/client" }
cumulus-test-runtime = { path = "../../test/runtime" }
cumulus-test-relay-sproof-builder = { path = "../../test/relay-sproof-builder" }
72 changes: 64 additions & 8 deletions cumulus/client/collator/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,8 @@ use sp_core::traits::SpawnNamed;
use sp_runtime::traits::{Block as BlockT, Header as HeaderT};

use cumulus_client_consensus_common::ParachainConsensus;
use polkadot_node_primitives::{CollationResult, MaybeCompressedPoV};
use polkadot_node_primitives::{CollationGenerationConfig, CollationResult, MaybeCompressedPoV};
use polkadot_node_subsystem::messages::{CollationGenerationMessage, CollatorProtocolMessage};
use polkadot_overseer::Handle as OverseerHandle;
use polkadot_primitives::{CollatorPair, Id as ParaId};

Expand Down Expand Up @@ -200,7 +201,7 @@ pub mod relay_chain_driven {
let config = CollationGenerationConfig {
key,
para_id,
collator: Box::new(move |relay_parent, validation_data| {
collator: Some(Box::new(move |relay_parent, validation_data| {
// Cloning the channel on each usage effectively makes the channel
// unbounded. The channel is actually bounded by the block production
// and consensus systems of Polkadot, which limits the amount of possible
Expand All @@ -218,7 +219,7 @@ pub mod relay_chain_driven {

this_rx.await.ok().flatten()
})
}),
})),
};

overseer_handle
Expand All @@ -233,6 +234,31 @@ pub mod relay_chain_driven {
}
}

/// Initialize the collation-related subsystems on the relay-chain side.
///
/// This must be done prior to collation, and does not set up any callback for collation.
/// For callback-driven collators, use the [`relay_chain_driven`] module.
pub async fn initialize_collator_subsystems(
overseer_handle: &mut OverseerHandle,
key: CollatorPair,
para_id: ParaId,
) {
overseer_handle
.send_msg(
CollationGenerationMessage::Initialize(CollationGenerationConfig {
key,
para_id,
collator: None,
}),
"StartCollator",
)
.await;

overseer_handle
.send_msg(CollatorProtocolMessage::CollateOn(para_id), "StartCollator")
.await;
}

/// Parameters for [`start_collator`].
pub struct StartCollatorParams<Block: BlockT, RA, BS, Spawner> {
pub para_id: ParaId,
Expand All @@ -246,7 +272,24 @@ pub struct StartCollatorParams<Block: BlockT, RA, BS, Spawner> {
}

/// Start the collator.
#[deprecated = "Collators should run consensus futures which handle this logic internally"]
pub async fn start_collator<Block, RA, BS, Spawner>(
params: StartCollatorParams<Block, RA, BS, Spawner>,
) where
Block: BlockT,
BS: BlockBackend<Block> + Send + Sync + 'static,
Spawner: SpawnNamed + Clone + Send + Sync + 'static,
RA: ProvideRuntimeApi<Block> + Send + Sync + 'static,
RA::Api: CollectCollationInfo<Block>,
{
// This never needed to be asynchronous, but shouldn't be changed due to backcompat.
#[allow(deprecated)]
start_collator_sync(params);
}

/// Start the collator in a synchronous function.
#[deprecated = "Collators should run consensus futures which handle this logic internally"]
pub fn start_collator_sync<Block, RA, BS, Spawner>(
StartCollatorParams {
para_id,
block_status,
Expand All @@ -269,9 +312,8 @@ pub async fn start_collator<Block, RA, BS, Spawner>(

let collator = Collator::new(collator_service, parachain_consensus);

let mut request_stream = relay_chain_driven::init(key, para_id, overseer_handle).await;

let collation_future = Box::pin(async move {
let mut request_stream = relay_chain_driven::init(key, para_id, overseer_handle).await;
while let Some(request) = request_stream.next().await {
let collation = collator
.clone()
Expand All @@ -298,11 +340,14 @@ mod tests {
Client, ClientBlockImportExt, DefaultTestClientBuilderExt, InitBlockBuilder,
TestClientBuilder, TestClientBuilderExt,
};
use cumulus_test_relay_sproof_builder::RelayStateSproofBuilder;
use cumulus_test_runtime::{Block, Header};
use futures::{channel::mpsc, executor::block_on, StreamExt};
use polkadot_node_primitives::CollationGenerationConfig;
use polkadot_node_subsystem::messages::CollationGenerationMessage;
use polkadot_node_subsystem_test_helpers::ForwardSubsystem;
use polkadot_overseer::{dummy::dummy_overseer_builder, HeadSupportsParachains};
use polkadot_primitives::HeadData;
use sp_consensus::BlockOrigin;
use sp_core::{testing::TaskExecutor, Pair};
use sp_runtime::traits::BlakeTwo256;
Expand Down Expand Up @@ -330,10 +375,14 @@ mod tests {
_: PHash,
validation_data: &PersistedValidationData,
) -> Option<ParachainCandidate<Block>> {
let mut sproof = RelayStateSproofBuilder::default();
sproof.included_para_head = Some(HeadData(parent.encode()));
sproof.para_id = cumulus_test_runtime::PARACHAIN_ID.into();

let builder = self.client.init_block_builder_at(
parent.hash(),
Some(validation_data.clone()),
Default::default(),
sproof,
);

let (block, _, proof) = builder.build().expect("Creates block").into_inner();
Expand Down Expand Up @@ -368,6 +417,7 @@ mod tests {

spawner.spawn("overseer", None, overseer.run().then(|_| async {}).boxed());

#[allow(deprecated)]
let collator_start = start_collator(StartCollatorParams {
runtime_api: client.clone(),
block_status: client.clone(),
Expand All @@ -384,13 +434,19 @@ mod tests {
.0
.expect("message should be send by `start_collator` above.");

let CollationGenerationMessage::Initialize(config) = msg;
let collator_fn = match msg {
CollationGenerationMessage::Initialize(CollationGenerationConfig {
collator: Some(c),
..
}) => c,
_ => panic!("unexpected message or no collator fn"),
};

let validation_data =
PersistedValidationData { parent_head: header.encode().into(), ..Default::default() };
let relay_parent = Default::default();

let collation = block_on((config.collator)(relay_parent, &validation_data))
let collation = block_on(collator_fn(relay_parent, &validation_data))
.expect("Collation is build")
.collation;

Expand Down
1 change: 1 addition & 0 deletions cumulus/client/consensus/aura/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -45,4 +45,5 @@ cumulus-client-collator = { path = "../../collator" }
# Polkadot
polkadot-primitives = { git = "https://github.com/paritytech/polkadot", branch = "master" }
polkadot-node-primitives = { git = "https://github.com/paritytech/polkadot", branch = "master" }
polkadot-node-subsystem = { git = "https://github.com/paritytech/polkadot", branch = "master" }
polkadot-overseer = { git = "https://github.com/paritytech/polkadot", branch = "master" }
46 changes: 32 additions & 14 deletions cumulus/client/consensus/aura/src/collator.rs
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,7 @@ use sp_runtime::{
};
use sp_state_machine::StorageChanges;
use sp_timestamp::Timestamp;
use std::{convert::TryFrom, error::Error, sync::Arc, time::Duration};
use std::{convert::TryFrom, error::Error, time::Duration};

/// Parameters for instantiating a [`Collator`].
pub struct Params<BI, CIDP, RClient, Proposer, CS> {
Expand All @@ -64,7 +64,7 @@ pub struct Params<BI, CIDP, RClient, Proposer, CS> {
/// The block import handle.
pub block_import: BI,
/// An interface to the relay-chain client.
pub relay_client: Arc<RClient>,
pub relay_client: RClient,
/// The keystore handle used for accessing parachain key material.
pub keystore: KeystorePtr,
/// The identifier of the parachain within the relay-chain.
Expand All @@ -81,12 +81,12 @@ pub struct Params<BI, CIDP, RClient, Proposer, CS> {
pub struct Collator<Block, P, BI, CIDP, RClient, Proposer, CS> {
create_inherent_data_providers: CIDP,
block_import: BI,
relay_client: Arc<RClient>,
relay_client: RClient,
keystore: KeystorePtr,
para_id: ParaId,
proposer: Proposer,
collator_service: CS,
_marker: std::marker::PhantomData<(Block, P)>,
_marker: std::marker::PhantomData<(Block, Box<dyn Fn(P) + Send + Sync + 'static>)>,
}

impl<Block, P, BI, CIDP, RClient, Proposer, CS> Collator<Block, P, BI, CIDP, RClient, Proposer, CS>
Expand Down Expand Up @@ -123,7 +123,7 @@ where
validation_data: &PersistedValidationData,
parent_hash: Block::Hash,
timestamp: impl Into<Option<Timestamp>>,
) -> Result<(ParachainInherentData, InherentData), Box<dyn Error>> {
) -> Result<(ParachainInherentData, InherentData), Box<dyn Error + Send + Sync + 'static>> {
let paras_inherent_data = ParachainInherentData::create_at(
relay_parent,
&self.relay_client,
Expand All @@ -143,7 +143,7 @@ where
let mut other_inherent_data = self
.create_inherent_data_providers
.create_inherent_data_providers(parent_hash, ())
.map_err(|e| e as Box<dyn Error>)
.map_err(|e| e as Box<dyn Error + Send + Sync + 'static>)
.await?
.create_inherent_data()
.await
Expand Down Expand Up @@ -172,7 +172,8 @@ where
inherent_data: (ParachainInherentData, InherentData),
proposal_duration: Duration,
max_pov_size: usize,
) -> Result<(Collation, ParachainBlockData<Block>, Block::Hash), Box<dyn Error>> {
) -> Result<(Collation, ParachainBlockData<Block>, Block::Hash), Box<dyn Error + Send + 'static>>
{
let mut digest = additional_pre_digest.into().unwrap_or_default();
digest.push(slot_claim.pre_digest.clone());

Expand All @@ -187,14 +188,15 @@ where
Some(max_pov_size),
)
.await
.map_err(|e| Box::new(e))?;
.map_err(|e| Box::new(e) as Box<dyn Error + Send>)?;

let sealed_importable = seal::<_, P>(
proposal.block,
proposal.storage_changes,
&slot_claim.author_pub,
&self.keystore,
)?;
)
.map_err(|e| e as Box<dyn Error + Send>)?;

let post_hash = sealed_importable.post_hash();
let block = Block::new(
Expand All @@ -206,7 +208,10 @@ where
.clone(),
);

self.block_import.import_block(sealed_importable).await?;
self.block_import
.import_block(sealed_importable)
.map_err(|e| Box::new(e) as Box<dyn Error + Send>)
.await?;

if let Some((collation, block_data)) = self.collator_service.build_collation(
parent_header,
Expand All @@ -231,7 +236,8 @@ where

Ok((collation, block_data, post_hash))
} else {
Err("Unable to produce collation".to_string().into())
Err(Box::<dyn Error + Send + Sync>::from("Unable to produce collation")
as Box<dyn Error + Send>)
}
}

Expand Down Expand Up @@ -285,7 +291,7 @@ pub async fn claim_slot<B, C, P>(
parent_hash: B::Hash,
relay_parent_header: &PHeader,
slot_duration: SlotDuration,
relay_chain_slot_duration: SlotDuration,
relay_chain_slot_duration: Duration,
keystore: &KeystorePtr,
) -> Result<Option<SlotClaim<P::Public>>, Box<dyn Error>>
where
Expand All @@ -304,7 +310,19 @@ where
relay_parent_header,
relay_chain_slot_duration,
) {
Some((_, t)) => (Slot::from_timestamp(t, slot_duration), t),
Some((r_s, t)) => {
let our_slot = Slot::from_timestamp(t, slot_duration);
tracing::debug!(
target: crate::LOG_TARGET,
relay_slot = ?r_s,
para_slot = ?our_slot,
timestamp = ?t,
?slot_duration,
?relay_chain_slot_duration,
"Adjusted relay-chain slot to parachain slot"
);
(our_slot, t)
},
None => return Ok(None),
};

Expand All @@ -326,7 +344,7 @@ pub fn seal<B: BlockT, P>(
storage_changes: StorageChanges<HashingFor<B>>,
author_pub: &P::Public,
keystore: &KeystorePtr,
) -> Result<BlockImportParams<B>, Box<dyn Error>>
) -> Result<BlockImportParams<B>, Box<dyn Error + Send + Sync + 'static>>
where
P: Pair,
P::Signature: Codec + TryFrom<Vec<u8>>,
Expand Down
Loading

0 comments on commit 6ef1117

Please sign in to comment.