Breaking changes (#5191)

Co-authored-by: Bjerg <onbjerg@users.noreply.github.com>
Co-authored-by: Roman Krasiuk <rokrassyuk@gmail.com>
Co-authored-by: joshieDo <ranriver@protonmail.com>
Co-authored-by: joshieDo <93316087+joshieDo@users.noreply.github.com>
Co-authored-by: Matthias Seitz <matthias.seitz@outlook.de>
Co-authored-by: Oliver Nordbjerg <hi@notbjerg.me>
Co-authored-by: Thomas Coratger <thomas.coratger@gmail.com>
This commit is contained in:
Alexey Shekhirin
2024-02-29 12:37:28 +00:00
committed by GitHub
parent 025fa5f038
commit 6b5b6f7a40
252 changed files with 10154 additions and 6327 deletions

View File

@ -23,7 +23,7 @@ reth-tasks.workspace = true
reth-payload-builder.workspace = true
reth-payload-validator.workspace = true
reth-prune.workspace = true
reth-snapshot.workspace = true
reth-static-file.workspace = true
reth-tokio-util.workspace = true
reth-node-api.workspace = true
@ -59,6 +59,7 @@ reth-node-ethereum.workspace = true
reth-node-optimism.workspace = true
assert_matches.workspace = true
tempfile.workspace = true
[features]
optimism = [

View File

@ -10,7 +10,6 @@ use tracing::debug;
#[derive(Debug)]
pub(crate) struct PolledHook {
#[allow(dead_code)]
pub(crate) name: &'static str,
pub(crate) event: EngineHookEvent,
pub(crate) db_access_level: EngineHookDBAccessLevel,
@ -151,6 +150,8 @@ impl EngineHooksController {
);
return Poll::Ready(Ok(result))
} else {
debug!(target: "consensus::engine::hooks", hook = hook.name(), "Next hook is not ready");
}
Poll::Pending

View File

@ -11,8 +11,8 @@ pub(crate) use controller::{EngineHooksController, PolledHook};
mod prune;
pub use prune::PruneHook;
mod snapshot;
pub use snapshot::SnapshotHook;
mod static_file;
pub use static_file::StaticFileHook;
/// Collection of [engine hooks][`EngineHook`].
#[derive(Default)]

View File

@ -1,156 +0,0 @@
//! Snapshot hook for the engine implementation.
use crate::{
engine::hooks::{EngineContext, EngineHook, EngineHookError, EngineHookEvent},
hooks::EngineHookDBAccessLevel,
};
use futures::FutureExt;
use reth_db::database::Database;
use reth_interfaces::{RethError, RethResult};
use reth_primitives::BlockNumber;
use reth_snapshot::{Snapshotter, SnapshotterError, SnapshotterWithResult};
use reth_tasks::TaskSpawner;
use std::task::{ready, Context, Poll};
use tokio::sync::oneshot;
/// Manages snapshotting under the control of the engine.
///
/// This type controls the [Snapshotter].
#[derive(Debug)]
pub struct SnapshotHook<DB> {
/// The current state of the snapshotter.
state: SnapshotterState<DB>,
/// The type that can spawn the snapshotter task.
task_spawner: Box<dyn TaskSpawner>,
}
impl<DB: Database + 'static> SnapshotHook<DB> {
/// Create a new instance
pub fn new(snapshotter: Snapshotter<DB>, task_spawner: Box<dyn TaskSpawner>) -> Self {
Self { state: SnapshotterState::Idle(Some(snapshotter)), task_spawner }
}
/// Advances the snapshotter state.
///
/// This checks for the result in the channel, or returns pending if the snapshotter is idle.
fn poll_snapshotter(&mut self, cx: &mut Context<'_>) -> Poll<RethResult<EngineHookEvent>> {
let result = match self.state {
SnapshotterState::Idle(_) => return Poll::Pending,
SnapshotterState::Running(ref mut fut) => {
ready!(fut.poll_unpin(cx))
}
};
let event = match result {
Ok((snapshotter, result)) => {
self.state = SnapshotterState::Idle(Some(snapshotter));
match result {
Ok(_) => EngineHookEvent::Finished(Ok(())),
Err(err) => EngineHookEvent::Finished(Err(err.into())),
}
}
Err(_) => {
// failed to receive the snapshotter
EngineHookEvent::Finished(Err(EngineHookError::ChannelClosed))
}
};
Poll::Ready(Ok(event))
}
/// This will try to spawn the snapshotter if it is idle:
/// 1. Check if snapshotting is needed through [Snapshotter::get_snapshot_targets] and then
/// [SnapshotTargets::any](reth_snapshot::SnapshotTargets::any).
/// 2.
/// 1. If snapshotting is needed, pass snapshot request to the [Snapshotter::run] and spawn
/// it in a separate task. Set snapshotter state to [SnapshotterState::Running].
/// 2. If snapshotting is not needed, set snapshotter state back to
/// [SnapshotterState::Idle].
///
/// If snapshotter is already running, do nothing.
fn try_spawn_snapshotter(
&mut self,
finalized_block_number: BlockNumber,
) -> RethResult<Option<EngineHookEvent>> {
Ok(match &mut self.state {
SnapshotterState::Idle(snapshotter) => {
let Some(mut snapshotter) = snapshotter.take() else { return Ok(None) };
let targets = snapshotter.get_snapshot_targets(finalized_block_number)?;
// Check if the snapshotting of any data has been requested.
if targets.any() {
let (tx, rx) = oneshot::channel();
self.task_spawner.spawn_critical_blocking(
"snapshotter task",
Box::pin(async move {
let result = snapshotter.run(targets);
let _ = tx.send((snapshotter, result));
}),
);
self.state = SnapshotterState::Running(rx);
Some(EngineHookEvent::Started)
} else {
self.state = SnapshotterState::Idle(Some(snapshotter));
Some(EngineHookEvent::NotReady)
}
}
SnapshotterState::Running(_) => None,
})
}
}
impl<DB: Database + 'static> EngineHook for SnapshotHook<DB> {
fn name(&self) -> &'static str {
"Snapshot"
}
fn poll(
&mut self,
cx: &mut Context<'_>,
ctx: EngineContext,
) -> Poll<RethResult<EngineHookEvent>> {
let Some(finalized_block_number) = ctx.finalized_block_number else {
return Poll::Ready(Ok(EngineHookEvent::NotReady))
};
// Try to spawn a snapshotter
match self.try_spawn_snapshotter(finalized_block_number)? {
Some(EngineHookEvent::NotReady) => return Poll::Pending,
Some(event) => return Poll::Ready(Ok(event)),
None => (),
}
// Poll snapshotter and check its status
self.poll_snapshotter(cx)
}
fn db_access_level(&self) -> EngineHookDBAccessLevel {
EngineHookDBAccessLevel::ReadOnly
}
}
/// The possible snapshotter states within the sync controller.
///
/// [SnapshotterState::Idle] means that the snapshotter is currently idle.
/// [SnapshotterState::Running] means that the snapshotter is currently running.
#[derive(Debug)]
enum SnapshotterState<DB> {
/// Snapshotter is idle.
Idle(Option<Snapshotter<DB>>),
/// Snapshotter is running and waiting for a response
Running(oneshot::Receiver<SnapshotterWithResult<DB>>),
}
impl From<SnapshotterError> for EngineHookError {
fn from(err: SnapshotterError) -> Self {
match err {
SnapshotterError::InconsistentData(_) => EngineHookError::Internal(Box::new(err)),
SnapshotterError::Interface(err) => err.into(),
SnapshotterError::Database(err) => RethError::Database(err).into(),
SnapshotterError::Provider(err) => RethError::Provider(err).into(),
}
}
}

View File

@ -0,0 +1,163 @@
//! StaticFile hook for the engine implementation.
use crate::{
engine::hooks::{EngineContext, EngineHook, EngineHookError, EngineHookEvent},
hooks::EngineHookDBAccessLevel,
};
use futures::FutureExt;
use reth_db::database::Database;
use reth_interfaces::RethResult;
use reth_primitives::{static_file::HighestStaticFiles, BlockNumber};
use reth_static_file::{StaticFileProducer, StaticFileProducerWithResult};
use reth_tasks::TaskSpawner;
use std::task::{ready, Context, Poll};
use tokio::sync::oneshot;
use tracing::trace;
/// Manages producing static files under the control of the engine.
///
/// This type controls the [StaticFileProducer].
#[derive(Debug)]
pub struct StaticFileHook<DB> {
/// The current state of the static_file_producer.
state: StaticFileProducerState<DB>,
/// The type that can spawn the static_file_producer task.
task_spawner: Box<dyn TaskSpawner>,
}
impl<DB: Database + 'static> StaticFileHook<DB> {
/// Create a new instance
pub fn new(
static_file_producer: StaticFileProducer<DB>,
task_spawner: Box<dyn TaskSpawner>,
) -> Self {
Self { state: StaticFileProducerState::Idle(Some(static_file_producer)), task_spawner }
}
/// Advances the static_file_producer state.
///
/// This checks for the result in the channel, or returns pending if the static_file_producer is
/// idle.
fn poll_static_file_producer(
&mut self,
cx: &mut Context<'_>,
) -> Poll<RethResult<EngineHookEvent>> {
let result = match self.state {
StaticFileProducerState::Idle(_) => return Poll::Pending,
StaticFileProducerState::Running(ref mut fut) => {
ready!(fut.poll_unpin(cx))
}
};
let event = match result {
Ok((static_file_producer, result)) => {
self.state = StaticFileProducerState::Idle(Some(static_file_producer));
match result {
Ok(_) => EngineHookEvent::Finished(Ok(())),
Err(err) => EngineHookEvent::Finished(Err(err.into())),
}
}
Err(_) => {
// failed to receive the static_file_producer
EngineHookEvent::Finished(Err(EngineHookError::ChannelClosed))
}
};
Poll::Ready(Ok(event))
}
/// This will try to spawn the static_file_producer if it is idle:
/// 1. Check if producing static files is needed through
/// [StaticFileProducer::get_static_file_targets] and then
/// [StaticFileTargets::any](reth_static_file::StaticFileTargets::any).
/// 2.
/// 1. If producing static files is needed, pass static file request to the
/// [StaticFileProducer::run] and spawn it in a separate task. Set static file producer
/// state to [StaticFileProducerState::Running].
/// 2. If producing static files is not needed, set static file producer state back to
/// [StaticFileProducerState::Idle].
///
/// If static_file_producer is already running, do nothing.
fn try_spawn_static_file_producer(
&mut self,
finalized_block_number: BlockNumber,
) -> RethResult<Option<EngineHookEvent>> {
Ok(match &mut self.state {
StaticFileProducerState::Idle(static_file_producer) => {
let Some(mut static_file_producer) = static_file_producer.take() else {
trace!(target: "consensus::engine::hooks::static_file", "StaticFileProducer is already running but the state is idle");
return Ok(None);
};
let targets = static_file_producer.get_static_file_targets(HighestStaticFiles {
headers: Some(finalized_block_number),
receipts: Some(finalized_block_number),
transactions: Some(finalized_block_number),
})?;
// Check if the moving data to static files has been requested.
if targets.any() {
let (tx, rx) = oneshot::channel();
self.task_spawner.spawn_critical_blocking(
"static_file_producer task",
Box::pin(async move {
let result = static_file_producer.run(targets);
let _ = tx.send((static_file_producer, result));
}),
);
self.state = StaticFileProducerState::Running(rx);
Some(EngineHookEvent::Started)
} else {
self.state = StaticFileProducerState::Idle(Some(static_file_producer));
Some(EngineHookEvent::NotReady)
}
}
StaticFileProducerState::Running(_) => None,
})
}
}
impl<DB: Database + 'static> EngineHook for StaticFileHook<DB> {
fn name(&self) -> &'static str {
"StaticFile"
}
fn poll(
&mut self,
cx: &mut Context<'_>,
ctx: EngineContext,
) -> Poll<RethResult<EngineHookEvent>> {
let Some(finalized_block_number) = ctx.finalized_block_number else {
trace!(target: "consensus::engine::hooks::static_file", ?ctx, "Finalized block number is not available");
return Poll::Pending;
};
// Try to spawn a static_file_producer
match self.try_spawn_static_file_producer(finalized_block_number)? {
Some(EngineHookEvent::NotReady) => return Poll::Pending,
Some(event) => return Poll::Ready(Ok(event)),
None => (),
}
// Poll static_file_producer and check its status
self.poll_static_file_producer(cx)
}
fn db_access_level(&self) -> EngineHookDBAccessLevel {
EngineHookDBAccessLevel::ReadOnly
}
}
/// The possible static_file_producer states within the sync controller.
///
/// [StaticFileProducerState::Idle] means that the static file producer is currently idle.
/// [StaticFileProducerState::Running] means that the static file producer is currently running.
#[derive(Debug)]
enum StaticFileProducerState<DB> {
/// [StaticFileProducer] is idle.
Idle(Option<StaticFileProducer<DB>>),
/// [StaticFileProducer] is running and waiting for a response
Running(oneshot::Receiver<StaticFileProducerWithResult<DB>>),
}

View File

@ -361,6 +361,9 @@ where
warn!(
target: "consensus::engine",
hook = %hook.name(),
head_block_hash = ?state.head_block_hash,
safe_block_hash = ?state.safe_block_hash,
finalized_block_hash = ?state.finalized_block_hash,
"Hook is in progress, skipping forkchoice update. \
This may affect the performance of your node as a validator."
);
@ -1502,7 +1505,9 @@ where
debug!(target: "consensus::engine", hash=?new_head.hash(), number=new_head.number, "Canonicalized new head");
// we can update the FCU blocks
let _ = self.update_canon_chain(new_head, &target);
if let Err(err) = self.update_canon_chain(new_head, &target) {
debug!(target: "consensus::engine", ?err, ?target, "Failed to update the canonical chain tracker");
}
// we're no longer syncing
self.sync_state_updater.update_sync_state(SyncState::Idle);
@ -1704,9 +1709,18 @@ where
None
}
fn on_hook_result(&self, result: PolledHook) -> Result<(), BeaconConsensusEngineError> {
if result.db_access_level.is_read_write() {
match result.event {
fn on_hook_result(&self, polled_hook: PolledHook) -> Result<(), BeaconConsensusEngineError> {
if let EngineHookEvent::Finished(Err(error)) = &polled_hook.event {
error!(
target: "consensus::engine",
name = %polled_hook.name,
?error,
"Hook finished with error"
)
}
if polled_hook.db_access_level.is_read_write() {
match polled_hook.event {
EngineHookEvent::NotReady => {}
EngineHookEvent::Started => {
// If the hook has read-write access to the database, it means that the engine
@ -1889,9 +1903,7 @@ mod tests {
};
use assert_matches::assert_matches;
use reth_interfaces::test_utils::generators::{self, Rng};
use reth_primitives::{
stage::StageCheckpoint, ChainSpec, ChainSpecBuilder, B256, MAINNET, U256,
};
use reth_primitives::{stage::StageCheckpoint, ChainSpecBuilder, MAINNET};
use reth_provider::{BlockWriter, ProviderFactory};
use reth_rpc_types::engine::{ForkchoiceState, ForkchoiceUpdated, PayloadStatus};
use reth_rpc_types_compat::engine::payload::try_block_to_payload_v1;
@ -2064,12 +2076,10 @@ mod tests {
}
fn insert_blocks<'a, DB: Database>(
db: DB,
chain: Arc<ChainSpec>,
provider_factory: ProviderFactory<DB>,
mut blocks: impl Iterator<Item = &'a SealedBlock>,
) {
let factory = ProviderFactory::new(db, chain);
let provider = factory.provider_rw().unwrap();
let provider = provider_factory.provider_rw().unwrap();
blocks
.try_for_each(|b| {
provider
@ -2085,8 +2095,9 @@ mod tests {
mod fork_choice_updated {
use super::*;
use reth_db::{tables, transaction::DbTxMut};
use reth_db::{tables, test_utils::create_test_static_files_dir, transaction::DbTxMut};
use reth_interfaces::test_utils::generators::random_block;
use reth_primitives::U256;
use reth_rpc_types::engine::ForkchoiceUpdateError;
#[tokio::test]
@ -2139,10 +2150,18 @@ mod tests {
let genesis = random_block(&mut rng, 0, None, None, Some(0));
let block1 = random_block(&mut rng, 1, Some(genesis.hash()), None, Some(0));
insert_blocks(env.db.as_ref(), chain_spec.clone(), [&genesis, &block1].into_iter());
insert_blocks(
ProviderFactory::new(
env.db.as_ref(),
chain_spec.clone(),
create_test_static_files_dir(),
)
.expect("create provider factory with static_files"),
[&genesis, &block1].into_iter(),
);
env.db
.update(|tx| {
tx.put::<tables::SyncStage>(
tx.put::<tables::StageCheckpoints>(
StageId::Finish.to_string(),
StageCheckpoint::new(block1.number),
)
@ -2189,7 +2208,15 @@ mod tests {
let genesis = random_block(&mut rng, 0, None, None, Some(0));
let block1 = random_block(&mut rng, 1, Some(genesis.hash()), None, Some(0));
insert_blocks(env.db.as_ref(), chain_spec.clone(), [&genesis, &block1].into_iter());
insert_blocks(
ProviderFactory::new(
env.db.as_ref(),
chain_spec.clone(),
create_test_static_files_dir(),
)
.expect("create provider factory with static_files"),
[&genesis, &block1].into_iter(),
);
let mut engine_rx = spawn_consensus_engine(consensus_engine);
@ -2205,7 +2232,15 @@ mod tests {
let invalid_rx = env.send_forkchoice_updated(next_forkchoice_state).await;
// Insert next head immediately after sending forkchoice update
insert_blocks(env.db.as_ref(), chain_spec.clone(), [&next_head].into_iter());
insert_blocks(
ProviderFactory::new(
env.db.as_ref(),
chain_spec.clone(),
create_test_static_files_dir(),
)
.expect("create provider factory with static_files"),
[&next_head].into_iter(),
);
let expected_result = ForkchoiceUpdated::from_status(PayloadStatusEnum::Syncing);
assert_matches!(invalid_rx, Ok(result) => assert_eq!(result, expected_result));
@ -2239,7 +2274,15 @@ mod tests {
let genesis = random_block(&mut rng, 0, None, None, Some(0));
let block1 = random_block(&mut rng, 1, Some(genesis.hash()), None, Some(0));
insert_blocks(env.db.as_ref(), chain_spec.clone(), [&genesis, &block1].into_iter());
insert_blocks(
ProviderFactory::new(
env.db.as_ref(),
chain_spec.clone(),
create_test_static_files_dir(),
)
.expect("create provider factory with static_files"),
[&genesis, &block1].into_iter(),
);
let engine = spawn_consensus_engine(consensus_engine);
@ -2287,8 +2330,12 @@ mod tests {
block3.header.set_difficulty(U256::from(1));
insert_blocks(
env.db.as_ref(),
chain_spec.clone(),
ProviderFactory::new(
env.db.as_ref(),
chain_spec.clone(),
create_test_static_files_dir(),
)
.expect("create provider factory with static_files"),
[&genesis, &block1, &block2, &block3].into_iter(),
);
@ -2330,7 +2377,15 @@ mod tests {
let genesis = random_block(&mut rng, 0, None, None, Some(0));
let block1 = random_block(&mut rng, 1, Some(genesis.hash()), None, Some(0));
insert_blocks(env.db.as_ref(), chain_spec.clone(), [&genesis, &block1].into_iter());
insert_blocks(
ProviderFactory::new(
env.db.as_ref(),
chain_spec.clone(),
create_test_static_files_dir(),
)
.expect("create provider factory with static_files"),
[&genesis, &block1].into_iter(),
);
let _engine = spawn_consensus_engine(consensus_engine);
@ -2352,10 +2407,11 @@ mod tests {
mod new_payload {
use super::*;
use reth_db::test_utils::create_test_static_files_dir;
use reth_interfaces::test_utils::generators::random_block;
use reth_primitives::{
genesis::{Genesis, GenesisAllocator},
Hardfork,
Hardfork, U256,
};
use reth_provider::test_utils::blocks::BlockChainTestData;
@ -2426,8 +2482,12 @@ mod tests {
let block1 = random_block(&mut rng, 1, Some(genesis.hash()), None, Some(0));
let block2 = random_block(&mut rng, 2, Some(block1.hash()), None, Some(0));
insert_blocks(
env.db.as_ref(),
chain_spec.clone(),
ProviderFactory::new(
env.db.as_ref(),
chain_spec.clone(),
create_test_static_files_dir(),
)
.expect("create provider factory with static_files"),
[&genesis, &block1, &block2].into_iter(),
);
@ -2492,7 +2552,15 @@ mod tests {
// TODO: add transactions that transfer from the alloc accounts, generating the new
// block tx and state root
insert_blocks(env.db.as_ref(), chain_spec.clone(), [&genesis, &block1].into_iter());
insert_blocks(
ProviderFactory::new(
env.db.as_ref(),
chain_spec.clone(),
create_test_static_files_dir(),
)
.expect("create provider factory with static_files"),
[&genesis, &block1].into_iter(),
);
let mut engine_rx = spawn_consensus_engine(consensus_engine);
@ -2530,7 +2598,15 @@ mod tests {
let genesis = random_block(&mut rng, 0, None, None, Some(0));
insert_blocks(env.db.as_ref(), chain_spec.clone(), [&genesis].into_iter());
insert_blocks(
ProviderFactory::new(
env.db.as_ref(),
chain_spec.clone(),
create_test_static_files_dir(),
)
.expect("create provider factory with static_files"),
[&genesis].into_iter(),
);
let mut engine_rx = spawn_consensus_engine(consensus_engine);
@ -2589,8 +2665,12 @@ mod tests {
.build();
insert_blocks(
env.db.as_ref(),
chain_spec.clone(),
ProviderFactory::new(
env.db.as_ref(),
chain_spec.clone(),
create_test_static_files_dir(),
)
.expect("create provider factory with static_files"),
[&data.genesis, &block1].into_iter(),
);

View File

@ -398,13 +398,14 @@ mod tests {
use reth_interfaces::{p2p::either::EitherDownloader, test_utils::TestFullBlockClient};
use reth_primitives::{
constants::ETHEREUM_BLOCK_GAS_LIMIT, stage::StageCheckpoint, BlockBody, ChainSpecBuilder,
Header, SealedHeader, MAINNET,
Header, PruneModes, SealedHeader, MAINNET,
};
use reth_provider::{
test_utils::{create_test_provider_factory_with_chain_spec, TestExecutorFactory},
BundleStateWithReceipts,
};
use reth_stages::{test_utils::TestStages, ExecOutput, StageError};
use reth_static_file::StaticFileProducer;
use reth_tasks::TokioTaskExecutor;
use std::{collections::VecDeque, future::poll_fn, ops::Range};
use tokio::sync::watch;
@ -465,7 +466,15 @@ mod tests {
pipeline = pipeline.with_max_block(max_block);
}
pipeline.build(create_test_provider_factory_with_chain_spec(chain_spec))
let provider_factory = create_test_provider_factory_with_chain_spec(chain_spec);
let static_file_producer = StaticFileProducer::new(
provider_factory.clone(),
provider_factory.static_file_provider(),
PruneModes::default(),
);
pipeline.build(provider_factory, static_file_producer)
}
}

View File

@ -6,10 +6,7 @@ use crate::{
use reth_blockchain_tree::{
config::BlockchainTreeConfig, externals::TreeExternals, BlockchainTree, ShareableBlockchainTree,
};
use reth_db::{
test_utils::{create_test_rw_db, TempDatabase},
DatabaseEnv as DE,
};
use reth_db::{test_utils::TempDatabase, DatabaseEnv as DE};
type DatabaseEnv = TempDatabase<DE>;
use reth_downloaders::{
bodies::bodies::BodiesDownloaderBuilder,
@ -24,10 +21,11 @@ use reth_interfaces::{
};
use reth_node_ethereum::{EthEngineTypes, EthEvmConfig};
use reth_payload_builder::test_utils::spawn_test_payload_service;
use reth_primitives::{BlockNumber, ChainSpec, B256};
use reth_primitives::{BlockNumber, ChainSpec, PruneModes, B256};
use reth_provider::{
providers::BlockchainProvider, test_utils::TestExecutorFactory, BundleStateWithReceipts,
ExecutorFactory, HeaderSyncMode, ProviderFactory, PrunableBlockExecutor,
providers::BlockchainProvider,
test_utils::{create_test_provider_factory_with_chain_spec, TestExecutorFactory},
BundleStateWithReceipts, ExecutorFactory, HeaderSyncMode, PrunableBlockExecutor,
};
use reth_prune::Pruner;
use reth_revm::EvmProcessorFactory;
@ -35,6 +33,7 @@ use reth_rpc_types::engine::{
CancunPayloadFields, ExecutionPayload, ForkchoiceState, ForkchoiceUpdated, PayloadStatus,
};
use reth_stages::{sets::DefaultStages, test_utils::TestStages, ExecOutput, Pipeline, StageError};
use reth_static_file::StaticFileProducer;
use reth_tasks::TokioTaskExecutor;
use std::{collections::VecDeque, sync::Arc};
use tokio::sync::{oneshot, watch};
@ -348,9 +347,8 @@ where
/// Builds the test consensus engine into a `TestConsensusEngine` and `TestEnv`.
pub fn build(self) -> (TestBeaconConsensusEngine<Client>, TestEnv<Arc<DatabaseEnv>>) {
reth_tracing::init_test_tracing();
let db = create_test_rw_db();
let provider_factory =
ProviderFactory::new(db.clone(), self.base_config.chain_spec.clone());
create_test_provider_factory_with_chain_spec(self.base_config.chain_spec.clone());
let consensus: Arc<dyn Consensus> = match self.base_config.consensus {
TestConsensusConfig::Real => {
@ -380,6 +378,12 @@ where
)),
};
let static_file_producer = StaticFileProducer::new(
provider_factory.clone(),
provider_factory.static_file_provider(),
PruneModes::default(),
);
// Setup pipeline
let (tip_tx, tip_rx) = watch::channel(B256::default());
let mut pipeline = match self.base_config.pipeline_config {
@ -395,14 +399,17 @@ where
.build(client.clone(), consensus.clone(), provider_factory.clone())
.into_task();
Pipeline::builder().add_stages(DefaultStages::new(
ProviderFactory::new(db.clone(), self.base_config.chain_spec.clone()),
HeaderSyncMode::Tip(tip_rx.clone()),
Arc::clone(&consensus),
header_downloader,
body_downloader,
executor_factory.clone(),
))
Pipeline::builder().add_stages(
DefaultStages::new(
provider_factory.clone(),
HeaderSyncMode::Tip(tip_rx.clone()),
Arc::clone(&consensus),
header_downloader,
body_downloader,
executor_factory.clone(),
)
.expect("should build"),
)
}
};
@ -410,7 +417,7 @@ where
pipeline = pipeline.with_max_block(max_block);
}
let pipeline = pipeline.build(provider_factory.clone());
let pipeline = pipeline.build(provider_factory.clone(), static_file_producer);
// Setup blockchain tree
let externals = TreeExternals::new(provider_factory.clone(), consensus, executor_factory);
@ -423,12 +430,11 @@ where
BlockchainProvider::with_latest(provider_factory.clone(), tree, latest);
let pruner = Pruner::new(
provider_factory,
provider_factory.clone(),
vec![],
5,
self.base_config.chain_spec.prune_delete_limit,
config.max_reorg_depth() as usize,
watch::channel(None).1,
);
let mut hooks = EngineHooks::new();
@ -453,7 +459,7 @@ where
engine.sync.set_max_block(max_block)
}
(engine, TestEnv::new(db, tip_rx, handle))
(engine, TestEnv::new(provider_factory.db_ref().clone(), tip_rx, handle))
}
}

View File

@ -439,7 +439,7 @@ mod tests {
gas_price: 0x28f000fff,
gas_limit: 10,
to: TransactionKind::Call(Address::default()),
value: 3_u64.into(),
value: U256::from(3_u64),
input: Bytes::from(vec![1, 2]),
access_list: Default::default(),
});
@ -461,7 +461,7 @@ mod tests {
max_fee_per_blob_gas: 0x7,
gas_limit: 10,
to: TransactionKind::Call(Address::default()),
value: 3_u64.into(),
value: U256::from(3_u64),
input: Bytes::from(vec![1, 2]),
access_list: Default::default(),
blob_versioned_hashes: std::iter::repeat_with(|| rng.gen()).take(num_blobs).collect(),