mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
feat: use broadcast channel for event listeners (#8193)
Co-authored-by: Emilia Hane <elsaemiliaevahane@gmail.com>
This commit is contained in:
@ -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
|
||||
|
||||
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
@ -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()
|
||||
}
|
||||
}
|
||||
|
||||
@ -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)
|
||||
};
|
||||
|
||||
@ -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>),
|
||||
}
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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,
|
||||
|
||||
Reference in New Issue
Block a user