feat: refactor and integrate local engine into EngineNodeLauncher (#11703)

This commit is contained in:
Arsenii Kulikov
2024-10-14 17:21:41 +04:00
committed by GitHub
parent c05a900542
commit 523bfb9c81
16 changed files with 504 additions and 404 deletions

View File

@ -11,15 +11,19 @@ exclude.workspace = true
[dependencies]
# reth
reth-beacon-consensus.workspace = true
reth-chain-state.workspace = true
reth-chainspec.workspace = true
reth-consensus.workspace = true
reth-engine-primitives.workspace = true
reth-engine-service.workspace = true
reth-engine-tree.workspace = true
reth-evm.workspace = true
reth-ethereum-engine-primitives.workspace = true
reth-node-types.workspace = true
reth-payload-builder.workspace = true
reth-payload-primitives.workspace = true
reth-primitives.workspace = true
reth-payload-validator.workspace = true
reth-provider.workspace = true
reth-prune.workspace = true
reth-rpc-types-compat.workspace = true
reth-transaction-pool.workspace = true
reth-stages-api.workspace = true
@ -36,16 +40,10 @@ futures-util.workspace = true
eyre.workspace = true
tracing.workspace = true
[dev-dependencies]
reth-chainspec.workspace = true
reth-chain-state.workspace = true
reth-config.workspace = true
reth-db = { workspace = true, features = ["test-utils"] }
reth-ethereum-engine-primitives.workspace = true
reth-exex-test-utils.workspace = true
reth-payload-builder = { workspace = true, features = ["test-utils"] }
reth-provider = { workspace = true, features = ["test-utils"] }
reth-tracing.workspace = true
op-alloy-rpc-types-engine = { workspace = true, optional = true }
[lints]
workspace = true
[features]
optimism = ["op-alloy-rpc-types-engine"]

View File

@ -1,4 +1,17 @@
//! A local engine service that can be used to drive a dev chain.
#![doc(
html_logo_url = "https://raw.githubusercontent.com/paradigmxyz/reth/main/assets/reth-docs.png",
html_favicon_url = "https://avatars0.githubusercontent.com/u/97369466?s=256",
issue_tracker_base_url = "https://github.com/paradigmxyz/reth/issues/"
)]
#![cfg_attr(not(test), warn(unused_crate_dependencies))]
#![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))]
pub mod miner;
pub mod payload;
pub mod service;
pub use miner::MiningMode;
pub use payload::LocalPayloadAttributesBuilder;
pub use service::LocalEngineService;

View File

@ -1,16 +1,31 @@
//! Contains the implementation of the mining mode for the local engine.
use alloy_primitives::TxHash;
use alloy_primitives::{TxHash, B256};
use alloy_rpc_types_engine::{CancunPayloadFields, ForkchoiceState};
use eyre::OptionExt;
use futures_util::{stream::Fuse, StreamExt};
use reth_beacon_consensus::BeaconEngineMessage;
use reth_chainspec::EthereumHardforks;
use reth_engine_primitives::EngineTypes;
use reth_payload_builder::PayloadBuilderHandle;
use reth_payload_primitives::{
BuiltPayload, PayloadAttributesBuilder, PayloadBuilder, PayloadTypes,
};
use reth_provider::{BlockReader, ChainSpecProvider};
use reth_rpc_types_compat::engine::payload::block_to_payload;
use reth_transaction_pool::TransactionPool;
use std::{
future::Future,
pin::Pin,
task::{Context, Poll},
time::Duration,
time::{Duration, UNIX_EPOCH},
};
use tokio::{
sync::{mpsc::UnboundedSender, oneshot},
time::Interval,
};
use tokio::time::Interval;
use tokio_stream::wrappers::ReceiverStream;
use tracing::error;
/// A mining mode for the local dev engine.
#[derive(Debug)]
@ -58,3 +73,175 @@ impl Future for MiningMode {
}
}
}
/// Local miner advancing the chain/
#[derive(Debug)]
pub struct LocalMiner<EngineT: EngineTypes, Provider, B> {
/// Provider to read the current tip of the chain.
provider: Provider,
/// The payload attribute builder for the engine
payload_attributes_builder: B,
/// Sender for events to engine.
to_engine: UnboundedSender<BeaconEngineMessage<EngineT>>,
/// The mining mode for the engine
mode: MiningMode,
/// The payload builder for the engine
payload_builder: PayloadBuilderHandle<EngineT>,
/// Timestamp for the next block.
last_timestamp: u64,
/// Stores latest mined blocks.
last_block_hashes: Vec<B256>,
}
impl<EngineT, Provider, B> LocalMiner<EngineT, Provider, B>
where
EngineT: EngineTypes,
Provider: BlockReader + ChainSpecProvider<ChainSpec: EthereumHardforks> + 'static,
B: PayloadAttributesBuilder<<EngineT as PayloadTypes>::PayloadAttributes>,
{
/// Spawns a new [`LocalMiner`] with the given parameters.
pub fn spawn_new(
provider: Provider,
payload_attributes_builder: B,
to_engine: UnboundedSender<BeaconEngineMessage<EngineT>>,
mode: MiningMode,
payload_builder: PayloadBuilderHandle<EngineT>,
) {
let latest_header =
provider.sealed_header(provider.best_block_number().unwrap()).unwrap().unwrap();
let miner = Self {
provider,
payload_attributes_builder,
to_engine,
mode,
payload_builder,
last_timestamp: latest_header.timestamp,
last_block_hashes: vec![latest_header.hash()],
};
// Spawn the miner
tokio::spawn(miner.run());
}
/// Runs the [`LocalMiner`] in a loop, polling the miner and building payloads.
async fn run(mut self) {
let mut fcu_interval = tokio::time::interval(Duration::from_secs(1));
loop {
tokio::select! {
// Wait for the interval or the pool to receive a transaction
_ = &mut self.mode => {
if let Err(e) = self.advance().await {
error!(target: "engine::local", "Error advancing the chain: {:?}", e);
}
}
// send FCU once in a while
_ = fcu_interval.tick() => {
if let Err(e) = self.update_forkchoice_state().await {
error!(target: "engine::local", "Error updating fork choice: {:?}", e);
}
}
}
}
}
/// Returns current forkchoice state.
fn forkchoice_state(&self) -> ForkchoiceState {
ForkchoiceState {
head_block_hash: *self.last_block_hashes.last().expect("at least 1 block exists"),
safe_block_hash: *self
.last_block_hashes
.get(self.last_block_hashes.len().saturating_sub(32))
.expect("at least 1 block exists"),
finalized_block_hash: *self
.last_block_hashes
.get(self.last_block_hashes.len().saturating_sub(64))
.expect("at least 1 block exists"),
}
}
/// Sends a FCU to the engine.
async fn update_forkchoice_state(&self) -> eyre::Result<()> {
let (tx, rx) = oneshot::channel();
self.to_engine.send(BeaconEngineMessage::ForkchoiceUpdated {
state: self.forkchoice_state(),
payload_attrs: None,
tx,
})?;
let res = rx.await??;
if !res.forkchoice_status().is_valid() {
eyre::bail!("Invalid fork choice update")
}
Ok(())
}
/// Generates payload attributes for a new block, passes them to FCU and inserts built payload
/// through newPayload.
async fn advance(&mut self) -> eyre::Result<()> {
let timestamp = std::cmp::max(
self.last_timestamp + 1,
std::time::SystemTime::now()
.duration_since(UNIX_EPOCH)
.expect("cannot be earlier than UNIX_EPOCH")
.as_secs(),
);
let (tx, rx) = oneshot::channel();
self.to_engine.send(BeaconEngineMessage::ForkchoiceUpdated {
state: self.forkchoice_state(),
payload_attrs: Some(self.payload_attributes_builder.build(timestamp)),
tx,
})?;
let res = rx.await??.await?;
if !res.payload_status.is_valid() {
eyre::bail!("Invalid payload status")
}
let payload_id = res.payload_id.ok_or_eyre("No payload id")?;
// wait for some time to let the payload be built
tokio::time::sleep(Duration::from_millis(200)).await;
let Some(Ok(payload)) = self.payload_builder.best_payload(payload_id).await else {
eyre::bail!("No payload")
};
let block = payload.block();
let cancun_fields =
if self.provider.chain_spec().is_cancun_active_at_timestamp(block.timestamp) {
Some(CancunPayloadFields {
parent_beacon_block_root: block.parent_beacon_block_root.unwrap(),
versioned_hashes: block.blob_versioned_hashes().into_iter().copied().collect(),
})
} else {
None
};
let (tx, rx) = oneshot::channel();
self.to_engine.send(BeaconEngineMessage::NewPayload {
payload: block_to_payload(payload.block().clone()),
cancun_fields,
tx,
})?;
let res = rx.await??;
if !res.is_valid() {
eyre::bail!("Invalid payload")
}
self.last_timestamp = timestamp;
self.last_block_hashes.push(block.hash());
// ensure we keep at most 64 blocks
if self.last_block_hashes.len() > 64 {
self.last_block_hashes =
self.last_block_hashes.split_off(self.last_block_hashes.len() - 64);
}
Ok(())
}
}

View File

@ -2,29 +2,62 @@
//! [`LocalEngineService`](super::service::LocalEngineService).
use alloy_primitives::{Address, B256};
use reth_chainspec::EthereumHardforks;
use reth_ethereum_engine_primitives::EthPayloadAttributes;
use reth_payload_primitives::PayloadAttributesBuilder;
use std::{convert::Infallible, time::UNIX_EPOCH};
use std::sync::Arc;
/// The attributes builder for local Ethereum payload.
#[derive(Debug)]
pub struct EthLocalPayloadAttributesBuilder;
#[non_exhaustive]
pub struct LocalPayloadAttributesBuilder<ChainSpec> {
chain_spec: Arc<ChainSpec>,
}
impl PayloadAttributesBuilder for EthLocalPayloadAttributesBuilder {
type PayloadAttributes = EthPayloadAttributes;
type Error = Infallible;
fn build(&self) -> Result<Self::PayloadAttributes, Self::Error> {
let ts = std::time::SystemTime::now()
.duration_since(UNIX_EPOCH)
.expect("cannot be earlier than UNIX_EPOCH");
Ok(EthPayloadAttributes {
timestamp: ts.as_secs(),
prev_randao: B256::random(),
suggested_fee_recipient: Address::random(),
withdrawals: None,
parent_beacon_block_root: None,
})
impl<ChainSpec> LocalPayloadAttributesBuilder<ChainSpec> {
/// Creates a new instance of the builder.
pub const fn new(chain_spec: Arc<ChainSpec>) -> Self {
Self { chain_spec }
}
}
impl<ChainSpec> PayloadAttributesBuilder<EthPayloadAttributes>
for LocalPayloadAttributesBuilder<ChainSpec>
where
ChainSpec: Send + Sync + EthereumHardforks + 'static,
{
fn build(&self, timestamp: u64) -> EthPayloadAttributes {
EthPayloadAttributes {
timestamp,
prev_randao: B256::random(),
suggested_fee_recipient: Address::random(),
withdrawals: if self.chain_spec.is_shanghai_active_at_timestamp(timestamp) {
Some(Default::default())
} else {
None
},
parent_beacon_block_root: if self.chain_spec.is_cancun_active_at_timestamp(timestamp) {
Some(B256::random())
} else {
None
},
}
}
}
#[cfg(feature = "optimism")]
impl<ChainSpec> PayloadAttributesBuilder<op_alloy_rpc_types_engine::OpPayloadAttributes>
for LocalPayloadAttributesBuilder<ChainSpec>
where
ChainSpec: Send + Sync + EthereumHardforks + 'static,
{
fn build(&self, timestamp: u64) -> op_alloy_rpc_types_engine::OpPayloadAttributes {
op_alloy_rpc_types_engine::OpPayloadAttributes {
payload_attributes: self.build(timestamp),
transactions: None,
no_tx_pool: None,
gas_limit: None,
eip_1559_params: None,
}
}
}

View File

@ -6,357 +6,154 @@
//! with a single transaction. The `Interval` mode will initiate block
//! building at a fixed interval.
use crate::miner::MiningMode;
use eyre::eyre;
use reth_beacon_consensus::EngineNodeTypes;
use reth_chain_state::{CanonicalInMemoryState, ExecutedBlock, NewCanonicalChain};
use reth_engine_tree::persistence::PersistenceHandle;
use reth_payload_builder::PayloadBuilderHandle;
use reth_payload_primitives::{
BuiltPayload, PayloadAttributesBuilder, PayloadBuilder, PayloadBuilderAttributes, PayloadTypes,
use core::fmt;
use std::{
fmt::{Debug, Formatter},
pin::Pin,
sync::Arc,
task::{Context, Poll},
};
use reth_provider::ProviderFactory;
use crate::miner::{LocalMiner, MiningMode};
use futures_util::{Stream, StreamExt};
use reth_beacon_consensus::{BeaconConsensusEngineEvent, BeaconEngineMessage, EngineNodeTypes};
use reth_chainspec::EthChainSpec;
use reth_consensus::Consensus;
use reth_engine_service::service::EngineMessageStream;
use reth_engine_tree::{
chain::{ChainEvent, HandlerEvent},
engine::{
EngineApiKind, EngineApiRequest, EngineApiRequestHandler, EngineRequestHandler, FromEngine,
RequestHandlerEvent,
},
persistence::PersistenceHandle,
tree::{EngineApiTreeHandler, InvalidBlockHook, TreeConfig},
};
use reth_evm::execute::BlockExecutorProvider;
use reth_payload_builder::PayloadBuilderHandle;
use reth_payload_primitives::{PayloadAttributesBuilder, PayloadTypes};
use reth_payload_validator::ExecutionPayloadValidator;
use reth_provider::{providers::BlockchainProvider2, ChainSpecProvider, ProviderFactory};
use reth_prune::PrunerWithFactory;
use reth_stages_api::MetricEventsSender;
use tokio::sync::oneshot;
use tracing::debug;
use tokio::sync::mpsc::UnboundedSender;
use tracing::error;
/// Provides a local dev service engine that can be used to drive the
/// chain forward.
#[derive(Debug)]
pub struct LocalEngineService<N, B>
///
/// This service both produces and consumes [`BeaconEngineMessage`]s. This is done to allow
/// modifications of the stream
pub struct LocalEngineService<N>
where
N: EngineNodeTypes,
B: PayloadAttributesBuilder<PayloadAttributes = <N::Engine as PayloadTypes>::PayloadAttributes>,
{
/// The payload builder for the engine
payload_builder: PayloadBuilderHandle<N::Engine>,
/// The payload attribute builder for the engine
payload_attributes_builder: B,
/// Keep track of the Canonical chain state that isn't persisted on disk yet
canonical_in_memory_state: CanonicalInMemoryState,
/// A handle to the persistence layer
persistence_handle: PersistenceHandle,
/// The mining mode for the engine
mode: MiningMode,
/// Processes requests.
///
/// This type is responsible for processing incoming requests.
handler: EngineApiRequestHandler<EngineApiRequest<N::Engine>>,
/// Receiver for incoming requests (from the engine API endpoint) that need to be processed.
incoming_requests: EngineMessageStream<N::Engine>,
}
impl<N, B> LocalEngineService<N, B>
impl<N> LocalEngineService<N>
where
N: EngineNodeTypes,
B: PayloadAttributesBuilder<PayloadAttributes = <N::Engine as PayloadTypes>::PayloadAttributes>,
{
/// Constructor for [`LocalEngineService`].
pub fn new(
payload_builder: PayloadBuilderHandle<N::Engine>,
payload_attributes_builder: B,
#[allow(clippy::too_many_arguments)]
pub fn new<B>(
consensus: Arc<dyn Consensus>,
executor_factory: impl BlockExecutorProvider,
provider: ProviderFactory<N>,
blockchain_db: BlockchainProvider2<N>,
pruner: PrunerWithFactory<ProviderFactory<N>>,
canonical_in_memory_state: CanonicalInMemoryState,
payload_builder: PayloadBuilderHandle<N::Engine>,
tree_config: TreeConfig,
invalid_block_hook: Box<dyn InvalidBlockHook>,
sync_metrics_tx: MetricEventsSender,
to_engine: UnboundedSender<BeaconEngineMessage<N::Engine>>,
from_engine: EngineMessageStream<N::Engine>,
mode: MiningMode,
) -> Self {
payload_attributes_builder: B,
) -> Self
where
B: PayloadAttributesBuilder<<N::Engine as PayloadTypes>::PayloadAttributes>,
{
let chain_spec = provider.chain_spec();
let engine_kind =
if chain_spec.is_optimism() { EngineApiKind::OpStack } else { EngineApiKind::Ethereum };
let persistence_handle =
PersistenceHandle::spawn_service(provider, pruner, sync_metrics_tx);
let payload_validator = ExecutionPayloadValidator::new(chain_spec);
Self {
payload_builder,
payload_attributes_builder,
canonical_in_memory_state,
let canonical_in_memory_state = blockchain_db.canonical_in_memory_state();
let (to_tree_tx, from_tree) = EngineApiTreeHandler::spawn_new(
blockchain_db.clone(),
executor_factory,
consensus,
payload_validator,
persistence_handle,
mode,
}
}
payload_builder.clone(),
canonical_in_memory_state,
tree_config,
invalid_block_hook,
engine_kind,
);
/// Spawn the [`LocalEngineService`] on a tokio green thread. The service will poll the payload
/// builder with two varying modes, [`MiningMode::Instant`] or [`MiningMode::Interval`]
/// which will respectively either execute the block as soon as it finds a
/// transaction in the pool or build the block based on an interval.
pub fn spawn_new(
payload_builder: PayloadBuilderHandle<N::Engine>,
payload_attributes_builder: B,
provider: ProviderFactory<N>,
pruner: PrunerWithFactory<ProviderFactory<N>>,
canonical_in_memory_state: CanonicalInMemoryState,
sync_metrics_tx: MetricEventsSender,
mode: MiningMode,
) {
let engine = Self::new(
payload_builder,
let handler = EngineApiRequestHandler::new(to_tree_tx, from_tree);
LocalMiner::spawn_new(
blockchain_db,
payload_attributes_builder,
provider,
pruner,
canonical_in_memory_state,
sync_metrics_tx,
to_engine,
mode,
payload_builder,
);
// Spawn the engine
tokio::spawn(engine.run());
}
/// Runs the [`LocalEngineService`] in a loop, polling the miner and building
/// payloads.
async fn run(mut self) {
loop {
// Wait for the interval or the pool to receive a transaction
(&mut self.mode).await;
// Start a new payload building job
let executed_block = self.build_and_save_payload().await;
if executed_block.is_err() {
debug!(target: "local_engine", err = ?executed_block.unwrap_err(), "failed payload building");
continue
}
let block = executed_block.expect("not error");
let res = self.update_canonical_in_memory_state(block);
if res.is_err() {
debug!(target: "local_engine", err = ?res.unwrap_err(), "failed canonical state update");
}
}
}
/// Builds a payload by initiating a new payload job via the [`PayloadBuilderHandle`],
/// saving the execution outcome to persistence and returning the executed block.
async fn build_and_save_payload(&self) -> eyre::Result<ExecutedBlock> {
let payload_attributes = self.payload_attributes_builder.build()?;
let parent = self.canonical_in_memory_state.get_canonical_head().hash();
let payload_builder_attributes =
<N::Engine as PayloadTypes>::PayloadBuilderAttributes::try_new(
parent,
payload_attributes,
)
.map_err(|_| eyre::eyre!("failed to fetch payload attributes"))?;
let payload = self
.payload_builder
.send_and_resolve_payload(payload_builder_attributes)
.await?
.await?;
let executed_block =
payload.executed_block().ok_or_else(|| eyre!("missing executed block"))?;
let (tx, rx) = oneshot::channel();
let _ = self.persistence_handle.save_blocks(vec![executed_block.clone()], tx);
// Wait for the persistence_handle to complete
let _ = rx.await?.ok_or_else(|| eyre!("missing new head"))?;
Ok(executed_block)
}
/// Update the canonical in memory state and send notification for a new canon state to
/// all the listeners.
fn update_canonical_in_memory_state(&self, executed_block: ExecutedBlock) -> eyre::Result<()> {
let chain = NewCanonicalChain::Commit { new: vec![executed_block] };
let tip = chain.tip().header.clone();
let notification = chain.to_chain_notification();
// Update the tracked in-memory state with the new chain
self.canonical_in_memory_state.update_chain(chain);
self.canonical_in_memory_state.set_canonical_head(tip);
// Sends an event to all active listeners about the new canonical chain
self.canonical_in_memory_state.notify_canon_state(notification);
Ok(())
Self { handler, incoming_requests: from_engine }
}
}
#[cfg(test)]
mod tests {
use super::*;
use reth_chainspec::MAINNET;
use reth_config::PruneConfig;
use reth_db::test_utils::{create_test_rw_db, create_test_static_files_dir};
use reth_ethereum_engine_primitives::EthEngineTypes;
use reth_exex_test_utils::TestNode;
use reth_node_types::NodeTypesWithDBAdapter;
use reth_payload_builder::test_utils::spawn_test_payload_service;
use reth_provider::{providers::StaticFileProvider, BlockReader, ProviderFactory};
use reth_prune::PrunerBuilder;
use reth_transaction_pool::{
test_utils::{testing_pool, MockTransaction},
TransactionPool,
};
use std::{convert::Infallible, time::Duration};
use tokio::sync::mpsc::unbounded_channel;
impl<N> Stream for LocalEngineService<N>
where
N: EngineNodeTypes,
{
type Item = ChainEvent<BeaconConsensusEngineEvent>;
#[derive(Debug)]
struct TestPayloadAttributesBuilder;
fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
let this = self.get_mut();
impl PayloadAttributesBuilder for TestPayloadAttributesBuilder {
type PayloadAttributes = alloy_rpc_types_engine::PayloadAttributes;
type Error = Infallible;
fn build(&self) -> Result<Self::PayloadAttributes, Self::Error> {
Ok(alloy_rpc_types_engine::PayloadAttributes {
timestamp: 0,
prev_randao: Default::default(),
suggested_fee_recipient: Default::default(),
withdrawals: None,
parent_beacon_block_root: None,
})
if let Poll::Ready(ev) = this.handler.poll(cx) {
return match ev {
RequestHandlerEvent::HandlerEvent(ev) => match ev {
HandlerEvent::BackfillAction(_) => {
error!(target: "engine::local", "received backfill request in local engine");
Poll::Ready(Some(ChainEvent::FatalError))
}
HandlerEvent::Event(ev) => Poll::Ready(Some(ChainEvent::Handler(ev))),
HandlerEvent::FatalError => Poll::Ready(Some(ChainEvent::FatalError)),
},
RequestHandlerEvent::Download(_) => {
error!(target: "engine::local", "received download request in local engine");
Poll::Ready(Some(ChainEvent::FatalError))
}
}
}
}
#[tokio::test]
async fn test_local_engine_service_interval() -> eyre::Result<()> {
reth_tracing::init_test_tracing();
// forward incoming requests to the handler
while let Poll::Ready(Some(req)) = this.incoming_requests.poll_next_unpin(cx) {
this.handler.on_event(FromEngine::Request(req.into()));
}
// Start the provider and the pruner
let (_, static_dir_path) = create_test_static_files_dir();
let provider = ProviderFactory::<NodeTypesWithDBAdapter<TestNode, _>>::new(
create_test_rw_db(),
MAINNET.clone(),
StaticFileProvider::read_write(static_dir_path)?,
);
let pruner = PrunerBuilder::new(PruneConfig::default())
.build_with_provider_factory(provider.clone());
// Create an empty canonical in memory state
let canonical_in_memory_state = CanonicalInMemoryState::empty();
// Start the payload builder service
let payload_handle = spawn_test_payload_service::<EthEngineTypes>();
// Sync metric channel
let (sync_metrics_tx, _) = unbounded_channel();
// Launch the LocalEngineService in interval mode
let period = Duration::from_secs(1);
LocalEngineService::spawn_new(
payload_handle,
TestPayloadAttributesBuilder,
provider.clone(),
pruner,
canonical_in_memory_state,
sync_metrics_tx,
MiningMode::interval(period),
);
// Check that we have no block for now
let block = provider.block_by_number(0)?;
assert!(block.is_none());
// Wait 4 intervals
tokio::time::sleep(2 * period).await;
// Assert a block has been build
let block = provider.block_by_number(0)?;
assert!(block.is_some());
Ok(())
}
#[tokio::test]
async fn test_local_engine_service_instant() -> eyre::Result<()> {
reth_tracing::init_test_tracing();
// Start the provider and the pruner
let (_, static_dir_path) = create_test_static_files_dir();
let provider = ProviderFactory::<NodeTypesWithDBAdapter<TestNode, _>>::new(
create_test_rw_db(),
MAINNET.clone(),
StaticFileProvider::read_write(static_dir_path)?,
);
let pruner = PrunerBuilder::new(PruneConfig::default())
.build_with_provider_factory(provider.clone());
// Create an empty canonical in memory state
let canonical_in_memory_state = CanonicalInMemoryState::empty();
// Start the payload builder service
let payload_handle = spawn_test_payload_service::<EthEngineTypes>();
// Start a transaction pool
let pool = testing_pool();
// Sync metric channel
let (sync_metrics_tx, _) = unbounded_channel();
// Launch the LocalEngineService in instant mode
LocalEngineService::spawn_new(
payload_handle,
TestPayloadAttributesBuilder,
provider.clone(),
pruner,
canonical_in_memory_state,
sync_metrics_tx,
MiningMode::instant(pool.clone()),
);
// Wait for a small period to assert block building is
// triggered by adding a transaction to the pool
let period = Duration::from_millis(500);
tokio::time::sleep(period).await;
let block = provider.block_by_number(0)?;
assert!(block.is_none());
// Add a transaction to the pool
let transaction = MockTransaction::legacy().with_gas_price(10);
pool.add_transaction(Default::default(), transaction).await?;
// Wait for block building
let period = Duration::from_secs(2);
tokio::time::sleep(period).await;
// Assert a block has been build
let block = provider.block_by_number(0)?;
assert!(block.is_some());
Ok(())
}
#[tokio::test]
async fn test_canonical_chain_subscription() -> eyre::Result<()> {
reth_tracing::init_test_tracing();
// Start the provider and the pruner
let (_, static_dir_path) = create_test_static_files_dir();
let provider = ProviderFactory::<NodeTypesWithDBAdapter<TestNode, _>>::new(
create_test_rw_db(),
MAINNET.clone(),
StaticFileProvider::read_write(static_dir_path)?,
);
let pruner = PrunerBuilder::new(PruneConfig::default())
.build_with_provider_factory(provider.clone());
// Create an empty canonical in memory state
let canonical_in_memory_state = CanonicalInMemoryState::empty();
let mut notifications = canonical_in_memory_state.subscribe_canon_state();
// Start the payload builder service
let payload_handle = spawn_test_payload_service::<EthEngineTypes>();
// Start a transaction pool
let pool = testing_pool();
// Sync metric channel
let (sync_metrics_tx, _) = unbounded_channel();
// Launch the LocalEngineService in instant mode
LocalEngineService::spawn_new(
payload_handle,
TestPayloadAttributesBuilder,
provider.clone(),
pruner,
canonical_in_memory_state,
sync_metrics_tx,
MiningMode::instant(pool.clone()),
);
// Add a transaction to the pool
let transaction = MockTransaction::legacy().with_gas_price(10);
pool.add_transaction(Default::default(), transaction).await?;
// Check a notification is received for block 0
let res = notifications.recv().await?;
assert_eq!(res.tip().number, 0);
Ok(())
Poll::Pending
}
}
impl<N: EngineNodeTypes> Debug for LocalEngineService<N> {
fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result {
f.debug_struct("LocalEngineService").finish_non_exhaustive()
}
}