feat: use broadcast channel for event listeners (#8193)

Co-authored-by: Emilia Hane <elsaemiliaevahane@gmail.com>
This commit is contained in:
Federico Gimenez
2024-05-22 19:36:51 +02:00
committed by GitHub
parent f45ca74772
commit d0386b8166
35 changed files with 293 additions and 197 deletions

View File

@ -25,6 +25,7 @@ reth-engine-primitives.workspace = true
reth-consensus.workspace = true
reth-rpc-types.workspace = true
reth-network-types.workspace = true
reth-tokio-util.workspace = true
# async
futures-util.workspace = true

View File

@ -9,6 +9,7 @@ use reth_primitives::{
use reth_provider::{CanonChainTracker, CanonStateNotificationSender, Chain, StateProviderFactory};
use reth_rpc_types::engine::ForkchoiceState;
use reth_stages_api::PipelineEvent;
use reth_tokio_util::EventStream;
use reth_transaction_pool::{TransactionPool, ValidPoolTransaction};
use std::{
collections::VecDeque,
@ -18,7 +19,6 @@ use std::{
task::{Context, Poll},
};
use tokio::sync::{mpsc::UnboundedSender, oneshot};
use tokio_stream::wrappers::UnboundedReceiverStream;
use tracing::{debug, error, warn};
/// A Future that listens for new ready transactions and puts new blocks into storage
@ -30,7 +30,7 @@ pub struct MiningTask<Client, Pool: TransactionPool, Executor, Engine: EngineTyp
/// The active miner
miner: MiningMode,
/// Single active future that inserts a new block into `storage`
insert_task: Option<BoxFuture<'static, Option<UnboundedReceiverStream<PipelineEvent>>>>,
insert_task: Option<BoxFuture<'static, Option<EventStream<PipelineEvent>>>>,
/// Shared storage to insert new blocks
storage: Storage,
/// Pool where transactions are stored
@ -42,7 +42,7 @@ pub struct MiningTask<Client, Pool: TransactionPool, Executor, Engine: EngineTyp
/// Used to notify consumers of new blocks
canon_state_notification: CanonStateNotificationSender,
/// The pipeline events to listen on
pipe_line_events: Option<UnboundedReceiverStream<PipelineEvent>>,
pipe_line_events: Option<EventStream<PipelineEvent>>,
/// The type used for block execution
block_executor: Executor,
}
@ -80,7 +80,7 @@ impl<Executor, Client, Pool: TransactionPool, Engine: EngineTypes>
}
/// Sets the pipeline events to listen on.
pub fn set_pipeline_events(&mut self, events: UnboundedReceiverStream<PipelineEvent>) {
pub fn set_pipeline_events(&mut self, events: EventStream<PipelineEvent>) {
self.pipe_line_events = Some(events);
}
}

View File

@ -10,28 +10,20 @@ use reth_interfaces::RethResult;
use reth_rpc_types::engine::{
CancunPayloadFields, ExecutionPayload, ForkchoiceState, ForkchoiceUpdated, PayloadStatus,
};
use tokio::sync::{mpsc, mpsc::UnboundedSender, oneshot};
use tokio_stream::wrappers::UnboundedReceiverStream;
use reth_tokio_util::{EventSender, EventStream};
use tokio::sync::{mpsc::UnboundedSender, oneshot};
/// A _shareable_ beacon consensus frontend type. Used to interact with the spawned beacon consensus
/// engine task.
///
/// See also `BeaconConsensusEngine`
#[derive(Debug)]
#[derive(Debug, Clone)]
pub struct BeaconConsensusEngineHandle<Engine>
where
Engine: EngineTypes,
{
pub(crate) to_engine: UnboundedSender<BeaconEngineMessage<Engine>>,
}
impl<Engine> Clone for BeaconConsensusEngineHandle<Engine>
where
Engine: EngineTypes,
{
fn clone(&self) -> Self {
Self { to_engine: self.to_engine.clone() }
}
event_sender: EventSender<BeaconConsensusEngineEvent>,
}
// === impl BeaconConsensusEngineHandle ===
@ -41,8 +33,11 @@ where
Engine: EngineTypes,
{
/// Creates a new beacon consensus engine handle.
pub fn new(to_engine: UnboundedSender<BeaconEngineMessage<Engine>>) -> Self {
Self { to_engine }
pub fn new(
to_engine: UnboundedSender<BeaconEngineMessage<Engine>>,
event_sender: EventSender<BeaconConsensusEngineEvent>,
) -> Self {
Self { to_engine, event_sender }
}
/// Sends a new payload message to the beacon consensus engine and waits for a response.
@ -97,9 +92,7 @@ where
}
/// Creates a new [`BeaconConsensusEngineEvent`] listener stream.
pub fn event_listener(&self) -> UnboundedReceiverStream<BeaconConsensusEngineEvent> {
let (tx, rx) = mpsc::unbounded_channel();
let _ = self.to_engine.send(BeaconEngineMessage::EventListener(tx));
UnboundedReceiverStream::new(rx)
pub fn event_listener(&self) -> EventStream<BeaconConsensusEngineEvent> {
self.event_sender.new_listener()
}
}

View File

@ -91,8 +91,7 @@ impl<DB: Database + 'static> StaticFileHook<DB> {
return Ok(None)
};
let Some(mut locked_static_file_producer) = static_file_producer.try_lock_arc()
else {
let Some(locked_static_file_producer) = static_file_producer.try_lock_arc() else {
trace!(target: "consensus::engine::hooks::static_file", "StaticFileProducer lock is already taken");
return Ok(None)
};

View File

@ -1,7 +1,4 @@
use crate::{
engine::{error::BeaconOnNewPayloadError, forkchoice::ForkchoiceStatus},
BeaconConsensusEngineEvent,
};
use crate::engine::{error::BeaconOnNewPayloadError, forkchoice::ForkchoiceStatus};
use futures::{future::Either, FutureExt};
use reth_engine_primitives::EngineTypes;
use reth_interfaces::RethResult;
@ -15,7 +12,7 @@ use std::{
pin::Pin,
task::{ready, Context, Poll},
};
use tokio::sync::{mpsc::UnboundedSender, oneshot};
use tokio::sync::oneshot;
/// Represents the outcome of forkchoice update.
///
@ -162,6 +159,4 @@ pub enum BeaconEngineMessage<Engine: EngineTypes> {
},
/// Message with exchanged transition configuration.
TransitionConfigurationExchanged,
/// Add a new listener for [`BeaconEngineMessage`].
EventListener(UnboundedSender<BeaconConsensusEngineEvent>),
}

View File

@ -29,7 +29,7 @@ use reth_rpc_types::engine::{
};
use reth_stages_api::{ControlFlow, Pipeline};
use reth_tasks::TaskSpawner;
use reth_tokio_util::EventListeners;
use reth_tokio_util::EventSender;
use std::{
pin::Pin,
sync::Arc,
@ -202,8 +202,8 @@ where
/// be used to download and execute the missing blocks.
pipeline_run_threshold: u64,
hooks: EngineHooksController,
/// Listeners for engine events.
listeners: EventListeners<BeaconConsensusEngineEvent>,
/// Sender for engine events.
event_sender: EventSender<BeaconConsensusEngineEvent>,
/// Consensus engine metrics.
metrics: EngineMetrics,
}
@ -282,8 +282,8 @@ where
engine_message_stream: BoxStream<'static, BeaconEngineMessage<EngineT>>,
hooks: EngineHooks,
) -> RethResult<(Self, BeaconConsensusEngineHandle<EngineT>)> {
let handle = BeaconConsensusEngineHandle { to_engine };
let listeners = EventListeners::default();
let event_sender = EventSender::default();
let handle = BeaconConsensusEngineHandle::new(to_engine, event_sender.clone());
let sync = EngineSyncController::new(
pipeline,
client,
@ -291,7 +291,7 @@ where
run_pipeline_continuously,
max_block,
blockchain.chain_spec(),
listeners.clone(),
event_sender.clone(),
);
let mut this = Self {
sync,
@ -306,7 +306,7 @@ where
blockchain_tree_action: None,
pipeline_run_threshold,
hooks: EngineHooksController::new(hooks),
listeners,
event_sender,
metrics: EngineMetrics::default(),
};
@ -406,7 +406,7 @@ where
if should_update_head {
let head = outcome.header();
let _ = self.update_head(head.clone());
self.listeners.notify(BeaconConsensusEngineEvent::CanonicalChainCommitted(
self.event_sender.notify(BeaconConsensusEngineEvent::CanonicalChainCommitted(
Box::new(head.clone()),
elapsed,
));
@ -543,7 +543,7 @@ where
}
// notify listeners about new processed FCU
self.listeners.notify(BeaconConsensusEngineEvent::ForkchoiceUpdated(state, status));
self.event_sender.notify(BeaconConsensusEngineEvent::ForkchoiceUpdated(state, status));
}
/// Check if the pipeline is consistent (all stages have the checkpoint block numbers no less
@ -597,13 +597,6 @@ where
self.handle.clone()
}
/// Pushes an [UnboundedSender] to the engine's listeners. Also pushes an [UnboundedSender] to
/// the sync controller's listeners.
pub(crate) fn push_listener(&mut self, listener: UnboundedSender<BeaconConsensusEngineEvent>) {
self.listeners.push_listener(listener.clone());
self.sync.push_listener(listener);
}
/// Returns true if the distance from the local tip to the block is greater than the configured
/// threshold.
///
@ -1255,7 +1248,7 @@ where
} else {
BeaconConsensusEngineEvent::ForkBlockAdded(block)
};
self.listeners.notify(event);
self.event_sender.notify(event);
PayloadStatusEnum::Valid
}
InsertPayloadOk::AlreadySeen(BlockStatus::Valid(_)) => {
@ -1429,7 +1422,7 @@ where
match make_canonical_result {
Ok(outcome) => {
if let CanonicalOutcome::Committed { head } = &outcome {
self.listeners.notify(BeaconConsensusEngineEvent::CanonicalChainCommitted(
self.event_sender.notify(BeaconConsensusEngineEvent::CanonicalChainCommitted(
Box::new(head.clone()),
elapsed,
));
@ -1878,7 +1871,6 @@ where
BeaconEngineMessage::TransitionConfigurationExchanged => {
this.blockchain.on_transition_configuration_exchanged();
}
BeaconEngineMessage::EventListener(tx) => this.push_listener(tx),
}
continue
}

View File

@ -14,14 +14,14 @@ use reth_interfaces::p2p::{
use reth_primitives::{stage::PipelineTarget, BlockNumber, ChainSpec, SealedBlock, B256};
use reth_stages_api::{ControlFlow, Pipeline, PipelineError, PipelineWithResult};
use reth_tasks::TaskSpawner;
use reth_tokio_util::EventListeners;
use reth_tokio_util::EventSender;
use std::{
cmp::{Ordering, Reverse},
collections::{binary_heap::PeekMut, BinaryHeap},
sync::Arc,
task::{ready, Context, Poll},
};
use tokio::sync::{mpsc::UnboundedSender, oneshot};
use tokio::sync::oneshot;
use tracing::trace;
/// Manages syncing under the control of the engine.
@ -49,8 +49,8 @@ where
inflight_full_block_requests: Vec<FetchFullBlockFuture<Client>>,
/// In-flight full block _range_ requests in progress.
inflight_block_range_requests: Vec<FetchFullBlockRangeFuture<Client>>,
/// Listeners for engine events.
listeners: EventListeners<BeaconConsensusEngineEvent>,
/// Sender for engine events.
event_sender: EventSender<BeaconConsensusEngineEvent>,
/// Buffered blocks from downloads - this is a min-heap of blocks, using the block number for
/// ordering. This means the blocks will be popped from the heap with ascending block numbers.
range_buffered_blocks: BinaryHeap<Reverse<OrderedSealedBlock>>,
@ -76,7 +76,7 @@ where
run_pipeline_continuously: bool,
max_block: Option<BlockNumber>,
chain_spec: Arc<ChainSpec>,
listeners: EventListeners<BeaconConsensusEngineEvent>,
event_sender: EventSender<BeaconConsensusEngineEvent>,
) -> Self {
Self {
full_block_client: FullBlockClient::new(
@ -90,7 +90,7 @@ where
inflight_block_range_requests: Vec::new(),
range_buffered_blocks: BinaryHeap::new(),
run_pipeline_continuously,
listeners,
event_sender,
max_block,
metrics: EngineSyncMetrics::default(),
}
@ -127,11 +127,6 @@ where
self.run_pipeline_continuously
}
/// Pushes an [UnboundedSender] to the sync controller's listeners.
pub(crate) fn push_listener(&mut self, listener: UnboundedSender<BeaconConsensusEngineEvent>) {
self.listeners.push_listener(listener);
}
/// Returns `true` if a pipeline target is queued and will be triggered on the next `poll`.
#[allow(dead_code)]
pub(crate) fn is_pipeline_sync_pending(&self) -> bool {
@ -169,7 +164,7 @@ where
);
// notify listeners that we're downloading a block range
self.listeners.notify(BeaconConsensusEngineEvent::LiveSyncProgress(
self.event_sender.notify(BeaconConsensusEngineEvent::LiveSyncProgress(
ConsensusEngineLiveSyncProgress::DownloadingBlocks {
remaining_blocks: count,
target: hash,
@ -198,7 +193,7 @@ where
);
// notify listeners that we're downloading a block
self.listeners.notify(BeaconConsensusEngineEvent::LiveSyncProgress(
self.event_sender.notify(BeaconConsensusEngineEvent::LiveSyncProgress(
ConsensusEngineLiveSyncProgress::DownloadingBlocks {
remaining_blocks: 1,
target: hash,