Unverified Commit 322ccd0d authored by asynchronous rob's avatar asynchronous rob Committed by GitHub
Browse files

Overseer: subsystems communicate directly (#2227)



* overseer: pass messages directly between subsystems

* test that message is held on to

* Update node/overseer/src/lib.rs

Co-authored-by: default avatarPeter Goodspeed-Niklaus <coriolinus@users.noreply.github.com>

* give every subsystem an unbounded sender too

* remove metered_channel::name

1. we don't provide good names
2. these names are never used anywhere

* unused mut

* remove unnecessary &mut

* subsystem unbounded_send

* remove unused MaybeTimer

We have channel size metrics that serve the same purpose better now and the implementation of message timing was pretty ugly.

* remove comment

* split up senders and receivers

* update metrics

* fix tests

* fix test subsystem context

* fix flaky test

* fix docs

* doc

* use select_biased to favor signals

* Update node/subsystem/src/lib.rs

Co-authored-by: Andronik Ordian's avatarAndronik Ordian <write@reusable.software>

Co-authored-by: default avatarPeter Goodspeed-Niklaus <coriolinus@users.noreply.github.com>
Co-authored-by: Andronik Ordian's avatarAndronik Ordian <write@reusable.software>
parent 39124b0a
Pipeline #131095 canceled with stages
in 6 minutes and 38 seconds
......@@ -4236,12 +4236,6 @@ dependencies = [
"parking_lot 0.11.1",
]
[[package]]
name = "oorandom"
version = "11.1.3"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "0ab1bc2a289d34bd04a330323ac98a1b4bc82c9d9fcb1e66b63caa84da26b575"
[[package]]
name = "opaque-debug"
version = "0.2.3"
......@@ -5886,12 +5880,12 @@ dependencies = [
name = "polkadot-overseer"
version = "0.1.0"
dependencies = [
"assert_matches",
"async-trait",
"femme",
"futures 0.3.13",
"futures-timer 3.0.2",
"kv-log-macro",
"oorandom",
"polkadot-node-network-protocol",
"polkadot-node-primitives",
"polkadot-node-subsystem",
......
......@@ -25,10 +25,9 @@ use super::Meter;
/// Create a wrapped `mpsc::channel` pair of `MeteredSender` and `MeteredReceiver`.
pub fn channel<T>(capacity: usize, name: &'static str) -> (MeteredSender<T>, MeteredReceiver<T>) {
pub fn channel<T>(capacity: usize) -> (MeteredSender<T>, MeteredReceiver<T>) {
let (tx, rx) = mpsc::channel(capacity);
let mut shared_meter = Meter::default();
shared_meter.name = name;
let shared_meter = Meter::default();
let tx = MeteredSender { meter: shared_meter.clone(), inner: tx };
let rx = MeteredReceiver { meter: shared_meter, inner: rx };
(tx, rx)
......
......@@ -30,8 +30,6 @@ pub use self::unbounded::*;
/// A peek into the inner state of a meter.
#[derive(Debug, Clone, Default)]
pub struct Meter {
/// Name of the receiver and sender pair.
name: &'static str,
// Number of sends on this channel.
sent: Arc<AtomicUsize>,
// Number of receives on this channel.
......@@ -60,11 +58,6 @@ impl Meter {
}
}
/// Obtain the name of the channel `Sender` and `Receiver` pair.
pub fn name(&self) -> &'static str {
self.name
}
fn note_sent(&self) {
self.sent.fetch_add(1, Ordering::Relaxed);
}
......@@ -92,7 +85,7 @@ mod tests {
#[test]
fn try_send_try_next() {
block_on(async move {
let (mut tx, mut rx) = channel::<Msg>(5, "goofy");
let (mut tx, mut rx) = channel::<Msg>(5);
let msg = Msg::default();
assert_eq!(rx.meter().read(), Readout { sent: 0, received: 0 });
tx.try_send(msg).unwrap();
......@@ -116,7 +109,7 @@ mod tests {
fn with_tasks() {
let (ready, go) = futures::channel::oneshot::channel();
let (mut tx, mut rx) = channel::<Msg>(5, "goofy");
let (mut tx, mut rx) = channel::<Msg>(5);
block_on(async move {
futures::join!(
async move {
......@@ -149,7 +142,7 @@ mod tests {
#[test]
fn stream_and_sink() {
let (mut tx, mut rx) = channel::<Msg>(5, "goofy");
let (mut tx, mut rx) = channel::<Msg>(5);
block_on(async move {
futures::join!(
......@@ -175,8 +168,8 @@ mod tests {
#[test]
fn failed_send_does_not_inc_sent() {
let (mut bounded, _) = channel::<Msg>(5, "pluto");
let (mut unbounded, _) = unbounded::<Msg>("pluto");
let (mut bounded, _) = channel::<Msg>(5);
let (mut unbounded, _) = unbounded::<Msg>();
block_on(async move {
assert!(bounded.send(Msg::default()).await.is_err());
......
......@@ -25,10 +25,9 @@ use super::Meter;
/// Create a wrapped `mpsc::channel` pair of `MeteredSender` and `MeteredReceiver`.
pub fn unbounded<T>(name: &'static str) -> (UnboundedMeteredSender<T>, UnboundedMeteredReceiver<T>) {
pub fn unbounded<T>() -> (UnboundedMeteredSender<T>, UnboundedMeteredReceiver<T>) {
let (tx, rx) = mpsc::unbounded();
let mut shared_meter = Meter::default();
shared_meter.name = name;
let shared_meter = Meter::default();
let tx = UnboundedMeteredSender { meter: shared_meter.clone(), inner: tx };
let rx = UnboundedMeteredReceiver { meter: shared_meter, inner: rx };
(tx, rx)
......@@ -147,7 +146,7 @@ impl<T> UnboundedMeteredSender<T> {
/// Attempt to send message or fail immediately.
pub fn unbounded_send(&mut self, msg: T) -> result::Result<(), mpsc::TrySendError<T>> {
pub fn unbounded_send(&self, msg: T) -> result::Result<(), mpsc::TrySendError<T>> {
self.meter.note_sent();
self.inner.unbounded_send(msg).map_err(|e| {
self.meter.retract_sent();
......
......@@ -734,7 +734,7 @@ mod tests {
TestAuthorityDiscovery,
) {
let (net_tx, net_rx) = polkadot_node_subsystem_test_helpers::single_item_sink();
let (action_tx, action_rx) = metered::unbounded("test_action");
let (action_tx, action_rx) = metered::unbounded();
(
TestNetwork {
......
......@@ -9,7 +9,6 @@ async-trait = "0.1.42"
client = { package = "sc-client-api", git = "https://github.com/paritytech/substrate", branch = "master" }
futures = "0.3.12"
futures-timer = "3.0.2"
oorandom = "11.1.3"
polkadot-node-primitives = { package = "polkadot-node-primitives", path = "../primitives" }
polkadot-node-subsystem-util = { path = "../subsystem-util" }
polkadot-primitives = { path = "../../primitives" }
......@@ -20,6 +19,6 @@ tracing = "0.1.25"
sp-core = { git = "https://github.com/paritytech/substrate", branch = "master" }
polkadot-node-network-protocol = { path = "../network/protocol" }
futures = { version = "0.3.12", features = ["thread-pool"] }
futures-timer = "3.0.2"
femme = "2.1.1"
kv-log-macro = "1.0.7"
assert_matches = "1.4.0"
......@@ -61,20 +61,19 @@
use std::fmt::{self, Debug};
use std::pin::Pin;
use std::sync::Arc;
use std::sync::{atomic::{self, AtomicUsize}, Arc};
use std::task::Poll;
use std::time::Duration;
use std::collections::{hash_map, HashMap};
use futures::channel::{oneshot, mpsc};
use futures::channel::{oneshot};
use futures::{
poll, select,
future::BoxFuture,
stream::{FuturesUnordered, Fuse},
stream::{self, FuturesUnordered, Fuse},
Future, FutureExt, StreamExt,
};
use futures_timer::Delay;
use oorandom::Rand32;
use polkadot_primitives::v1::{Block, BlockNumber, Hash};
use client::{BlockImportNotification, BlockchainEvents, FinalityNotification};
......@@ -89,8 +88,8 @@ use polkadot_subsystem::messages::{
ApprovalVotingMessage, GossipSupportMessage,
};
pub use polkadot_subsystem::{
Subsystem, SubsystemContext, OverseerSignal, FromOverseer, SubsystemError, SubsystemResult,
SpawnedSubsystem, ActiveLeavesUpdate, ActivatedLeaf, DummySubsystem, jaeger,
Subsystem, SubsystemContext, SubsystemSender, OverseerSignal, FromOverseer, SubsystemError,
SubsystemResult, SpawnedSubsystem, ActiveLeavesUpdate, ActivatedLeaf, DummySubsystem, jaeger,
};
use polkadot_node_subsystem_util::{TimeoutExt, metrics::{self, prometheus}, metered, Metronome};
use polkadot_node_primitives::SpawnNamed;
......@@ -101,597 +100,596 @@ const CHANNEL_CAPACITY: usize = 1024;
const STOP_DELAY: u64 = 1;
// Target for logs.
const LOG_TARGET: &'static str = "parachain::overseer";
// Rate at which messages are timed.
const MESSAGE_TIMER_METRIC_CAPTURE_RATE: f64 = 0.005;
trait MapSubsystem<T> {
type Output;
/// A type of messages that are sent from [`Subsystem`] to [`Overseer`].
///
/// It wraps a system-wide [`AllMessages`] type that represents all possible
/// messages in the system.
///
/// [`AllMessages`]: enum.AllMessages.html
/// [`Subsystem`]: trait.Subsystem.html
/// [`Overseer`]: struct.Overseer.html
enum ToOverseer {
/// This is a message sent by a `Subsystem`.
SubsystemMessage(AllMessages),
fn map_subsystem(&self, sub: T) -> Self::Output;
}
/// A message that wraps something the `Subsystem` is desiring to
/// spawn on the overseer and a `oneshot::Sender` to signal the result
/// of the spawn.
SpawnJob {
name: &'static str,
s: BoxFuture<'static, ()>,
},
impl<F, T, U> MapSubsystem<T> for F where F: Fn(T) -> U {
type Output = U;
/// Same as `SpawnJob` but for blocking tasks to be executed on a
/// dedicated thread pool.
SpawnBlockingJob {
name: &'static str,
s: BoxFuture<'static, ()>,
},
fn map_subsystem(&self, sub: T) -> U {
(self)(sub)
}
}
/// An event telling the `Overseer` on the particular block
/// that has been imported or finalized.
/// This struct is passed as an argument to create a new instance of an [`Overseer`].
///
/// This structure exists solely for the purposes of decoupling
/// `Overseer` code from the client code and the necessity to call
/// `HeaderBackend::block_number_from_id()`.
/// As any entity that satisfies the interface may act as a [`Subsystem`] this allows
/// mocking in the test code:
///
/// Each [`Subsystem`] is supposed to implement some interface that is generic over
/// message type that is specific to this [`Subsystem`]. At the moment not all
/// subsystems are implemented and the rest can be mocked with the [`DummySubsystem`].
#[derive(Debug, Clone)]
pub struct BlockInfo {
/// hash of the block.
pub hash: Hash,
/// hash of the parent block.
pub parent_hash: Hash,
/// block's number.
pub number: BlockNumber,
pub struct AllSubsystems<
CV = (), CB = (), CS = (), SD = (), AD = (), AR = (), BS = (), BD = (), P = (),
RA = (), AS = (), NB = (), CA = (), CG = (), CP = (), ApD = (), ApV = (),
GS = (),
> {
/// A candidate validation subsystem.
pub candidate_validation: CV,
/// A candidate backing subsystem.
pub candidate_backing: CB,
/// A candidate selection subsystem.
pub candidate_selection: CS,
/// A statement distribution subsystem.
pub statement_distribution: SD,
/// An availability distribution subsystem.
pub availability_distribution: AD,
/// An availability recovery subsystem.
pub availability_recovery: AR,
/// A bitfield signing subsystem.
pub bitfield_signing: BS,
/// A bitfield distribution subsystem.
pub bitfield_distribution: BD,
/// A provisioner subsystem.
pub provisioner: P,
/// A runtime API subsystem.
pub runtime_api: RA,
/// An availability store subsystem.
pub availability_store: AS,
/// A network bridge subsystem.
pub network_bridge: NB,
/// A Chain API subsystem.
pub chain_api: CA,
/// A Collation Generation subsystem.
pub collation_generation: CG,
/// A Collator Protocol subsystem.
pub collator_protocol: CP,
/// An Approval Distribution subsystem.
pub approval_distribution: ApD,
/// An Approval Voting subsystem.
pub approval_voting: ApV,
/// A Connection Request Issuer subsystem.
pub gossip_support: GS,
}
impl From<BlockImportNotification<Block>> for BlockInfo {
fn from(n: BlockImportNotification<Block>) -> Self {
BlockInfo {
hash: n.hash,
parent_hash: n.header.parent_hash,
number: n.header.number,
impl<CV, CB, CS, SD, AD, AR, BS, BD, P, RA, AS, NB, CA, CG, CP, ApD, ApV, GS>
AllSubsystems<CV, CB, CS, SD, AD, AR, BS, BD, P, RA, AS, NB, CA, CG, CP, ApD, ApV, GS>
{
/// Create a new instance of [`AllSubsystems`].
///
/// Each subsystem is set to [`DummySystem`].
///
///# Note
///
/// Because of a bug in rustc it is required that when calling this function,
/// you provide a "random" type for the first generic parameter:
///
/// ```
/// polkadot_overseer::AllSubsystems::<()>::dummy();
/// ```
pub fn dummy() -> AllSubsystems<
DummySubsystem,
DummySubsystem,
DummySubsystem,
DummySubsystem,
DummySubsystem,
DummySubsystem,
DummySubsystem,
DummySubsystem,
DummySubsystem,
DummySubsystem,
DummySubsystem,
DummySubsystem,
DummySubsystem,
DummySubsystem,
DummySubsystem,
DummySubsystem,
DummySubsystem,
DummySubsystem,
> {
AllSubsystems {
candidate_validation: DummySubsystem,
candidate_backing: DummySubsystem,
candidate_selection: DummySubsystem,
statement_distribution: DummySubsystem,
availability_distribution: DummySubsystem,
availability_recovery: DummySubsystem,
bitfield_signing: DummySubsystem,
bitfield_distribution: DummySubsystem,
provisioner: DummySubsystem,
runtime_api: DummySubsystem,
availability_store: DummySubsystem,
network_bridge: DummySubsystem,
chain_api: DummySubsystem,
collation_generation: DummySubsystem,
collator_protocol: DummySubsystem,
approval_distribution: DummySubsystem,
approval_voting: DummySubsystem,
gossip_support: DummySubsystem,
}
}
}
impl From<FinalityNotification<Block>> for BlockInfo {
fn from(n: FinalityNotification<Block>) -> Self {
BlockInfo {
hash: n.hash,
parent_hash: n.header.parent_hash,
number: n.header.number,
/// Replace the `candidate_validation` instance in `self`.
pub fn replace_candidate_validation<NEW>(
self,
candidate_validation: NEW,
) -> AllSubsystems<NEW, CB, CS, SD, AD, AR, BS, BD, P, RA, AS, NB, CA, CG, CP, ApD, ApV, GS> {
AllSubsystems {
candidate_validation,
candidate_backing: self.candidate_backing,
candidate_selection: self.candidate_selection,
statement_distribution: self.statement_distribution,
availability_distribution: self.availability_distribution,
availability_recovery: self.availability_recovery,
bitfield_signing: self.bitfield_signing,
bitfield_distribution: self.bitfield_distribution,
provisioner: self.provisioner,
runtime_api: self.runtime_api,
availability_store: self.availability_store,
network_bridge: self.network_bridge,
chain_api: self.chain_api,
collation_generation: self.collation_generation,
collator_protocol: self.collator_protocol,
approval_distribution: self.approval_distribution,
approval_voting: self.approval_voting,
gossip_support: self.gossip_support,
}
}
}
/// Some event from the outer world.
enum Event {
BlockImported(BlockInfo),
BlockFinalized(BlockInfo),
MsgToSubsystem(AllMessages),
ExternalRequest(ExternalRequest),
Stop,
}
/// Some request from outer world.
enum ExternalRequest {
WaitForActivation {
hash: Hash,
response_channel: oneshot::Sender<SubsystemResult<()>>,
},
}
/// A handler used to communicate with the [`Overseer`].
///
/// [`Overseer`]: struct.Overseer.html
#[derive(Clone)]
pub struct OverseerHandler {
events_tx: metered::MeteredSender<Event>,
}
impl OverseerHandler {
/// Inform the `Overseer` that that some block was imported.
#[tracing::instrument(level = "trace", skip(self), fields(subsystem = LOG_TARGET))]
pub async fn block_imported(&mut self, block: BlockInfo) {
self.send_and_log_error(Event::BlockImported(block)).await
}
/// Send some message to one of the `Subsystem`s.
#[tracing::instrument(level = "trace", skip(self, msg), fields(subsystem = LOG_TARGET))]
pub async fn send_msg(&mut self, msg: impl Into<AllMessages>) {
self.send_and_log_error(Event::MsgToSubsystem(msg.into())).await
}
/// Inform the `Overseer` that some block was finalized.
#[tracing::instrument(level = "trace", skip(self), fields(subsystem = LOG_TARGET))]
pub async fn block_finalized(&mut self, block: BlockInfo) {
self.send_and_log_error(Event::BlockFinalized(block)).await
/// Replace the `candidate_backing` instance in `self`.
pub fn replace_candidate_backing<NEW>(
self,
candidate_backing: NEW,
) -> AllSubsystems<CV, NEW, CS, SD, AD, AR, BS, BD, P, RA, AS, NB, CA, CG, CP, ApD, ApV, GS> {
AllSubsystems {
candidate_validation: self.candidate_validation,
candidate_backing,
candidate_selection: self.candidate_selection,
statement_distribution: self.statement_distribution,
availability_distribution: self.availability_distribution,
availability_recovery: self.availability_recovery,
bitfield_signing: self.bitfield_signing,
bitfield_distribution: self.bitfield_distribution,
provisioner: self.provisioner,
runtime_api: self.runtime_api,
availability_store: self.availability_store,
network_bridge: self.network_bridge,
chain_api: self.chain_api,
collation_generation: self.collation_generation,
collator_protocol: self.collator_protocol,
approval_distribution: self.approval_distribution,
approval_voting: self.approval_voting,
gossip_support: self.gossip_support,
}
}
/// Wait for a block with the given hash to be in the active-leaves set.
/// This method is used for external code like `Proposer` that doesn't subscribe to Overseer's signals.
///
/// The response channel responds if the hash was activated and is closed if the hash was deactivated.
/// Note that due the fact the overseer doesn't store the whole active-leaves set, only deltas,
/// the response channel may never return if the hash was deactivated before this call.
/// In this case, it's the caller's responsibility to ensure a timeout is set.
#[tracing::instrument(level = "trace", skip(self, response_channel), fields(subsystem = LOG_TARGET))]
pub async fn wait_for_activation(&mut self, hash: Hash, response_channel: oneshot::Sender<SubsystemResult<()>>) {
self.send_and_log_error(Event::ExternalRequest(ExternalRequest::WaitForActivation {
hash,
response_channel
})).await
/// Replace the `candidate_selection` instance in `self`.
pub fn replace_candidate_selection<NEW>(
self,
candidate_selection: NEW,
) -> AllSubsystems<CV, CB, NEW, SD, AD, AR, BS, BD, P, RA, AS, NB, CA, CG, CP, ApD, ApV, GS> {
AllSubsystems {
candidate_validation: self.candidate_validation,
candidate_backing: self.candidate_backing,
candidate_selection,
statement_distribution: self.statement_distribution,
availability_distribution: self.availability_distribution,
availability_recovery: self.availability_recovery,
bitfield_signing: self.bitfield_signing,
bitfield_distribution: self.bitfield_distribution,
provisioner: self.provisioner,
runtime_api: self.runtime_api,
availability_store: self.availability_store,
network_bridge: self.network_bridge,
chain_api: self.chain_api,
collation_generation: self.collation_generation,
collator_protocol: self.collator_protocol,
approval_distribution: self.approval_distribution,
approval_voting: self.approval_voting,
gossip_support: self.gossip_support,
}
}
/// Tell `Overseer` to shutdown.
#[tracing::instrument(level = "trace", skip(self), fields(subsystem = LOG_TARGET))]
pub async fn stop(&mut self) {
self.send_and_log_error(Event::Stop).await
/// Replace the `statement_distribution` instance in `self`.
pub fn replace_statement_distribution<NEW>(
self,
statement_distribution: NEW,
) -> AllSubsystems<CV, CB, CS, NEW, AD, AR, BS, BD, P, RA, AS, NB, CA, CG, CP, ApD, ApV, GS> {
AllSubsystems {
candidate_validation: self.candidate_validation,
candidate_backing: self.candidate_backing,
candidate_selection: self.candidate_selection,
statement_distribution,
availability_distribution: self.availability_distribution,
availability_recovery: self.availability_recovery,
bitfield_signing: self.bitfield_signing,
bitfield_distribution: self.bitfield_distribution,
provisioner: self.provisioner,
runtime_api: self.runtime_api,
availability_store: self.availability_store,
network_bridge: self.network_bridge,
chain_api: self.chain_api,
collation_generation: self.collation_generation,
collator_protocol: self.collator_protocol,
approval_distribution: self.approval_distribution,
approval_voting: self.approval_voting,
gossip_support: self.gossip_support,
}
}
async fn send_and_log_error(&mut self, event: Event) {
if self.events_tx.send(event).await.is_err() {
tracing::info!(target: LOG_TARGET, "Failed to send an event to Overseer");
/// Replace the `availability_distribution` instance in `self`.
pub fn replace_availability_distribution<NEW>(
self,
availability_distribution: NEW,
) -> AllSubsystems<CV, CB, CS, SD, NEW, AR, BS, BD, P, RA, AS, NB, CA, CG, CP, ApD, ApV, GS> {
AllSubsystems {
candidate_validation: self.candidate_validation,
candidate_backing: self.candidate_backing,
candidate_selection: self.candidate_selection,
statement_distribution: self.statement_distribution,
availability_distribution,
availability_recovery: self.availability_recovery,
bitfield_signing: self.bitfield_signing,
bitfield_distribution: self.bitfield_distribution,
provisioner: self.provisioner,
runtime_api: self.runtime_api,
availability_store: self.availability_store,
network_bridge: self.network_bridge,
chain_api: self.chain_api,
collation_generation: self.collation_generation,
collator_protocol: self.collator_protocol,
approval_distribution: self.approval_distribution,
approval_voting: self.approval_voting,
gossip_support: self.gossip_support,
}
}
}
/// Glues together the [`Overseer`] and `BlockchainEvents` by forwarding
/// import and finality notifications into the [`OverseerHandler`].
///
/// [`Overseer`]: struct.Overseer.html
/// [`OverseerHandler`]: struct.OverseerHandler.html
pub async fn forward_events<P: BlockchainEvents<Block>>(
client: Arc<P>,
mut handler: OverseerHandler,
) {
let mut finality = client.finality_notification_stream();
let mut imports = client.import_notification_stream();
loop {
select! {
f = finality.next() => {
match f {
Some(block) => {
handler.block_finalized(block.into()).await;
}
None => break,
}
},
i = imports.next() => {
match i {
Some(block) => {
handler.block_imported(block.into()).await;
}
None => break,
}
},
complete => break,
/// Replace the `availability_recovery` instance in `self`.
pub fn replace_availability_recovery<NEW>(
self,
availability_recovery: NEW,
) -> AllSubsystems<CV, CB, CS, SD, AD, NEW, BS, BD, P, RA, AS, NB, CA, CG, CP, ApD, ApV, GS> {
AllSubsystems {
candidate_validation: self.candidate_validation,
candidate_backing: self.candidate_backing,
candidate_selection: self.candidate_selection,
statement_distribution: self.statement_distribution,
availability_distribution: self.availability_distribution,
availability_recovery,
bitfield_signing: self.bitfield_signing,
bitfield_distribution: self.bitfield_distribution,