feat: emit new block events in --auto-mine (#2079)

Co-authored-by: Dan Cline <6798349+Rjected@users.noreply.github.com>
This commit is contained in:
Matthias Seitz
2023-04-05 14:16:03 +02:00
committed by GitHub
parent 633d84ded0
commit 77a41e5edf
10 changed files with 175 additions and 76 deletions

View File

@ -13,6 +13,7 @@ reth-beacon-consensus = { path = "../beacon" }
reth-primitives = { path = "../../primitives" }
reth-interfaces = { path = "../../interfaces" }
reth-provider = { path = "../../storage/provider" }
reth-stages = { path = "../../stages" }
reth-revm = { path = "../../revm" }
reth-transaction-pool = { path = "../../transaction-pool" }

View File

@ -14,11 +14,13 @@
//! These downloaders poll the miner, assemble the block, and return transactions that are ready to
//! be mined.
use reth_beacon_consensus::BeaconEngineMessage;
use reth_interfaces::consensus::{Consensus, ConsensusError};
use reth_primitives::{
BlockBody, BlockHash, BlockHashOrNumber, BlockNumber, ChainSpec, Header, SealedBlock,
SealedHeader, H256, U256,
};
use reth_transaction_pool::TransactionPool;
use std::{collections::HashMap, sync::Arc};
use tokio::sync::{mpsc::UnboundedSender, RwLock, RwLockReadGuard, RwLockWriteGuard};
use tracing::trace;
@ -29,8 +31,7 @@ mod task;
pub use crate::client::AutoSealClient;
pub use mode::{FixedBlockTimeMiner, MiningMode, ReadyTransactionMiner};
use reth_beacon_consensus::BeaconEngineMessage;
use reth_transaction_pool::TransactionPool;
use reth_interfaces::events::NewBlockNotificationSink;
pub use task::MiningTask;
/// A consensus implementation intended for local development and testing purposes.
@ -82,6 +83,7 @@ pub struct AutoSealBuilder<Client, Pool> {
mode: MiningMode,
storage: Storage,
to_engine: UnboundedSender<BeaconEngineMessage>,
new_block_notification_sender: NewBlockNotificationSink,
}
// === impl AutoSealBuilder ===
@ -93,6 +95,7 @@ impl<Client, Pool: TransactionPool> AutoSealBuilder<Client, Pool> {
client: Client,
pool: Pool,
to_engine: UnboundedSender<BeaconEngineMessage>,
new_block_notification_sender: NewBlockNotificationSink,
) -> Self {
let mode = MiningMode::interval(std::time::Duration::from_secs(1));
Self {
@ -102,6 +105,7 @@ impl<Client, Pool: TransactionPool> AutoSealBuilder<Client, Pool> {
pool,
mode,
to_engine,
new_block_notification_sender,
}
}
@ -113,12 +117,21 @@ impl<Client, Pool: TransactionPool> AutoSealBuilder<Client, Pool> {
/// Consumes the type and returns all components
pub fn build(self) -> (AutoSealConsensus, AutoSealClient, MiningTask<Client, Pool>) {
let Self { client, consensus, pool, mode, storage, to_engine } = self;
let Self {
client,
consensus,
pool,
mode,
storage,
to_engine,
new_block_notification_sender,
} = self;
let auto_client = AutoSealClient::new(storage.clone());
let task = MiningTask::new(
Arc::clone(&consensus.chain_spec),
mode,
to_engine,
new_block_notification_sender,
storage,
client,
pool,

View File

@ -1,7 +1,7 @@
use crate::{mode::MiningMode, Storage};
use futures_util::{future::BoxFuture, FutureExt};
use futures_util::{future::BoxFuture, FutureExt, StreamExt};
use reth_beacon_consensus::BeaconEngineMessage;
use reth_interfaces::consensus::ForkchoiceState;
use reth_interfaces::{consensus::ForkchoiceState, events::NewBlockNotificationSink};
use reth_primitives::{
constants::{EMPTY_RECEIPTS, EMPTY_TRANSACTIONS},
proofs, Block, BlockBody, ChainSpec, Header, IntoRecoveredTransaction, ReceiptWithBloom,
@ -12,6 +12,7 @@ use reth_revm::{
database::{State, SubState},
executor::Executor,
};
use reth_stages::{stages::FINISH, PipelineEvent};
use reth_transaction_pool::{TransactionPool, ValidPoolTransaction};
use std::{
collections::VecDeque,
@ -22,7 +23,8 @@ use std::{
time::{SystemTime, UNIX_EPOCH},
};
use tokio::sync::{mpsc::UnboundedSender, oneshot};
use tracing::{trace, warn};
use tokio_stream::wrappers::UnboundedReceiverStream;
use tracing::{debug, trace, warn};
/// A Future that listens for new ready transactions and puts new blocks into storage
pub struct MiningTask<Client, Pool: TransactionPool> {
@ -33,7 +35,7 @@ pub struct MiningTask<Client, Pool: TransactionPool> {
/// The active miner
miner: MiningMode,
/// Single active future that inserts a new block into `storage`
insert_task: Option<BoxFuture<'static, ()>>,
insert_task: Option<BoxFuture<'static, Option<UnboundedReceiverStream<PipelineEvent>>>>,
/// Shared storage to insert new blocks
storage: Storage,
/// Pool where transactions are stored
@ -42,6 +44,10 @@ pub struct MiningTask<Client, Pool: TransactionPool> {
queued: VecDeque<Vec<Arc<ValidPoolTransaction<<Pool as TransactionPool>::Transaction>>>>,
/// TODO: ideally this would just be a sender of hashes
to_engine: UnboundedSender<BeaconEngineMessage>,
/// Used to notify consumers of new blocks
new_block_notification_sender: NewBlockNotificationSink,
/// The pipeline events to listen on
pipe_line_events: Option<UnboundedReceiverStream<PipelineEvent>>,
}
// === impl MiningTask ===
@ -52,6 +58,7 @@ impl<Client, Pool: TransactionPool> MiningTask<Client, Pool> {
chain_spec: Arc<ChainSpec>,
miner: MiningMode,
to_engine: UnboundedSender<BeaconEngineMessage>,
new_block_notification_sender: NewBlockNotificationSink,
storage: Storage,
client: Client,
pool: Pool,
@ -64,9 +71,16 @@ impl<Client, Pool: TransactionPool> MiningTask<Client, Pool> {
storage,
pool,
to_engine,
new_block_notification_sender,
queued: Default::default(),
pipe_line_events: None,
}
}
/// Sets the pipeline events to listen on.
pub fn set_pipeline_events(&mut self, events: UnboundedReceiverStream<PipelineEvent>) {
self.pipe_line_events = Some(events);
}
}
impl<Client, Pool> Future for MiningTask<Client, Pool>
@ -101,6 +115,11 @@ where
let client = this.client.clone();
let chain_spec = Arc::clone(&this.chain_spec);
let pool = this.pool.clone();
let mut events = this.pipe_line_events.take();
let new_block_notification_sender = this.new_block_notification_sender.clone();
// Create the mining future that creates a block, notifies the engine that drives
// the pipeline
this.insert_task = Some(Box::pin(async move {
let mut storage = storage.write().await;
let mut header = Header {
@ -151,7 +170,6 @@ where
let Block { mut header, body, .. } = block;
// clear all transactions from pool
// TODO this should happen automatically via events
pool.remove_transactions(body.iter().map(|tx| tx.hash));
header.receipts_root = if res.receipts().is_empty() {
@ -169,7 +187,7 @@ where
BlockBody { transactions: body, ommers: vec![], withdrawals: None };
header.gas_used = gas_used;
storage.insert_new_block(header, body);
storage.insert_new_block(header.clone(), body);
let new_hash = storage.best_hash;
let state = ForkchoiceState {
@ -177,25 +195,54 @@ where
finalized_block_hash: new_hash,
safe_block_hash: new_hash,
};
drop(storage);
trace!(target: "consensus::auto", ?state, "sending fork choice update");
// send the new update to the engine, this will trigger the pipeline to
// download the block, execute it and store it in the database.
let (tx, _rx) = oneshot::channel();
let _ = to_engine.send(BeaconEngineMessage::ForkchoiceUpdated {
state,
payload_attrs: None,
tx,
});
debug!(target: "consensus::auto", ?state, "sent fork choice update");
// wait for the pipeline to finish
if let Some(events) = events.as_mut() {
debug!(target: "consensus::auto", "waiting for finish stage event...");
// wait for the finish stage to
loop {
if let Some(PipelineEvent::Running { stage_id, .. }) =
events.next().await
{
if stage_id == FINISH {
debug!(target: "consensus::auto", "received finish stage event");
break
}
}
}
}
let header = header.seal_slow();
debug!(target: "consensus::auto", header=?header.hash(), "sending block notification");
// send block notification
let _ = new_block_notification_sender.send(Arc::new(header));
}
Err(err) => {
warn!(target: "consensus::auto", ?err, "failed to execute block")
}
}
events
}));
}
if let Some(mut fut) = this.insert_task.take() {
match fut.poll_unpin(cx) {
Poll::Ready(_) => {}
Poll::Ready(events) => {
this.pipe_line_events = events;
}
Poll::Pending => {
this.insert_task = Some(fut);
break