refactor: split reth-stages into reth-stages-api and reth-stages (#7666)

Co-authored-by: Oliver Nordbjerg <hi@notbjerg.me>
This commit is contained in:
Abner Zheng
2024-04-19 19:35:20 +08:00
committed by GitHub
parent f14bf14d19
commit defe5ff0af
47 changed files with 212 additions and 114 deletions

27
Cargo.lock generated
View File

@ -6147,7 +6147,7 @@ dependencies = [
"reth-primitives",
"reth-provider",
"reth-revm",
"reth-stages",
"reth-stages-api",
"reth-transaction-pool",
"tokio",
"tokio-stream",
@ -6238,7 +6238,7 @@ dependencies = [
"reth-primitives",
"reth-provider",
"reth-revm",
"reth-stages",
"reth-stages-api",
"reth-trie",
"reth-trie-parallel",
"tokio",
@ -7427,6 +7427,7 @@ dependencies = [
"reth-primitives",
"reth-provider",
"reth-revm",
"reth-stages-api",
"reth-static-file",
"reth-tokio-util",
"reth-trie",
@ -7438,6 +7439,28 @@ dependencies = [
"tracing",
]
[[package]]
name = "reth-stages-api"
version = "0.2.0-beta.5"
dependencies = [
"aquamarine",
"assert_matches",
"auto_impl",
"futures-util",
"metrics",
"reth-db",
"reth-interfaces",
"reth-metrics",
"reth-primitives",
"reth-provider",
"reth-static-file",
"reth-tokio-util",
"thiserror",
"tokio",
"tokio-stream",
"tracing",
]
[[package]]
name = "reth-static-file"
version = "0.2.0-beta.5"

View File

@ -53,6 +53,7 @@ members = [
"crates/node-api/",
"crates/node-e2e-tests/",
"crates/stages/",
"crates/stages-api",
"crates/static-file/",
"crates/storage/codecs/",
"crates/storage/codecs/derive/",
@ -95,9 +96,9 @@ resolver = "2"
rust.missing_debug_implementations = "warn"
rust.missing_docs = "warn"
rust.unreachable_pub = "warn"
rustdoc.all = "warn"
rust.unused_must_use = "deny"
rust.rust_2018_idioms = "deny"
rustdoc.all = "warn"
[workspace.lints.clippy]
# These are some of clippy's nursery (i.e., experimental) lints that we like.
@ -254,6 +255,7 @@ reth-rpc-engine-api = { path = "crates/rpc/rpc-engine-api" }
reth-rpc-types = { path = "crates/rpc/rpc-types" }
reth-rpc-types-compat = { path = "crates/rpc/rpc-types-compat" }
reth-stages = { path = "crates/stages" }
reth-stages-api = { path = "crates/stages-api" }
reth-static-file = { path = "crates/static-file" }
reth-tasks = { path = "crates/tasks" }
reth-tokio-util = { path = "crates/tokio-util" }

View File

@ -34,6 +34,7 @@ use reth_provider::{HeaderSyncMode, ProviderFactory, StageCheckpointReader};
use reth_stages::{
prelude::*,
stages::{ExecutionStage, ExecutionStageThresholds, SenderRecoveryStage},
Pipeline, StageSet,
};
use reth_static_file::StaticFileProducer;
use std::{path::PathBuf, sync::Arc};

View File

@ -16,7 +16,7 @@ reth-primitives.workspace = true
reth-interfaces.workspace = true
reth-db.workspace = true
reth-provider.workspace = true
reth-stages.workspace = true
reth-stages-api.workspace = true
reth-trie = { workspace = true, features = ["metrics"] }
reth-trie-parallel = { workspace = true, features = ["parallel"] }

View File

@ -26,7 +26,7 @@ use reth_provider::{
CanonStateNotification, CanonStateNotificationSender, CanonStateNotifications, Chain,
ChainSpecProvider, DisplayBlocksChain, ExecutorFactory, HeaderProvider, ProviderError,
};
use reth_stages::{MetricEvent, MetricEventsSender};
use reth_stages_api::{MetricEvent, MetricEventsSender};
use std::{
collections::{btree_map::Entry, BTreeMap, HashSet},
sync::Arc,

View File

@ -17,7 +17,7 @@ reth-beacon-consensus.workspace = true
reth-primitives.workspace = true
reth-interfaces.workspace = true
reth-provider.workspace = true
reth-stages.workspace = true
reth-stages-api.workspace = true
reth-revm.workspace = true
reth-transaction-pool.workspace = true
reth-evm.workspace = true

View File

@ -6,7 +6,7 @@ use reth_evm::ConfigureEvm;
use reth_interfaces::consensus::ForkchoiceState;
use reth_primitives::{Block, ChainSpec, IntoRecoveredTransaction, SealedBlockWithSenders};
use reth_provider::{CanonChainTracker, CanonStateNotificationSender, Chain, StateProviderFactory};
use reth_stages::PipelineEvent;
use reth_stages_api::PipelineEvent;
use reth_transaction_pool::{TransactionPool, ValidPoolTransaction};
use std::{
collections::VecDeque,

View File

@ -1,6 +1,5 @@
use node_e2e_tests::{node::NodeHelper, wallet::Wallet};
use reth::{
self,
args::RpcServerArgs,
builder::{NodeBuilder, NodeConfig, NodeHandle},
tasks::TaskManager,

View File

@ -0,0 +1,40 @@
[package]
name = "reth-stages-api"
version.workspace = true
edition.workspace = true
rust-version.workspace = true
license.workspace = true
homepage.workspace = true
repository.workspace = true
exclude.workspace = true
[dependencies]
# reth
reth-primitives.workspace = true
reth-provider.workspace = true
reth-db.workspace = true
reth-interfaces.workspace = true
reth-static-file.workspace = true
assert_matches.workspace = true
reth-tokio-util.workspace = true
# metrics
reth-metrics.workspace = true
metrics.workspace = true
# async
tokio = { workspace = true, features = ["sync"] }
tokio-stream.workspace = true
futures-util.workspace = true
# misc
thiserror.workspace = true
tracing.workspace = true
auto_impl = "1"
aquamarine.workspace = true
[features]
test-utils = []
[lints]
workspace = true

View File

@ -1,10 +1,11 @@
use crate::pipeline::PipelineEvent;
use reth_interfaces::{
consensus, db::DatabaseError as DbError, executor, p2p::error::DownloadError,
provider::ProviderError, RethError,
consensus, db::DatabaseError as DbError, executor, p2p::error::DownloadError, RethError,
};
use reth_primitives::{BlockNumber, SealedHeader, StaticFileSegment, TxNumber};
use reth_provider::ProviderError;
use thiserror::Error;
use crate::PipelineEvent;
use tokio::sync::mpsc::error::SendError;
/// Represents the specific error type within a block error.

View File

@ -0,0 +1,14 @@
//! Staged syncing primitives for reth.
mod error;
mod metrics;
mod pipeline;
mod stage;
#[allow(missing_docs)]
#[cfg(any(test, feature = "test-utils"))]
pub mod test_utils;
mod util;
pub use crate::metrics::*;
pub use error::*;
pub use pipeline::*;
pub use stage::*;

View File

@ -45,7 +45,8 @@ pub enum MetricEvent {
#[derive(Debug)]
pub struct MetricsListener {
events_rx: UnboundedReceiver<MetricEvent>,
pub(crate) sync_metrics: SyncMetrics,
/// underline metrics of stages
pub sync_metrics: SyncMetrics,
}
impl MetricsListener {

View File

@ -6,14 +6,14 @@ use reth_primitives::stage::StageId;
use std::collections::HashMap;
#[derive(Debug, Default)]
pub(crate) struct SyncMetrics {
pub(crate) stages: HashMap<StageId, StageMetrics>,
pub(crate) execution_stage: ExecutionStageMetrics,
pub struct SyncMetrics {
pub stages: HashMap<StageId, StageMetrics>,
pub execution_stage: ExecutionStageMetrics,
}
impl SyncMetrics {
/// Returns existing or initializes a new instance of [StageMetrics] for the provided [StageId].
pub(crate) fn get_stage_metrics(&mut self, stage_id: StageId) -> &mut StageMetrics {
pub fn get_stage_metrics(&mut self, stage_id: StageId) -> &mut StageMetrics {
self.stages
.entry(stage_id)
.or_insert_with(|| StageMetrics::new_with_labels(&[("stage", stage_id.to_string())]))
@ -22,19 +22,19 @@ impl SyncMetrics {
#[derive(Metrics)]
#[metrics(scope = "sync")]
pub(crate) struct StageMetrics {
pub struct StageMetrics {
/// The block number of the last commit for a stage.
pub(crate) checkpoint: Gauge,
pub checkpoint: Gauge,
/// The number of processed entities of the last commit for a stage, if applicable.
pub(crate) entities_processed: Gauge,
pub entities_processed: Gauge,
/// The number of total entities of the last commit for a stage, if applicable.
pub(crate) entities_total: Gauge,
pub entities_total: Gauge,
}
/// Execution stage metrics.
#[derive(Metrics)]
#[metrics(scope = "sync.execution")]
pub(crate) struct ExecutionStageMetrics {
pub struct ExecutionStageMetrics {
/// The total amount of gas processed (in millions)
pub(crate) mgas_processed_total: Counter,
pub mgas_processed_total: Counter,
}

View File

@ -1,6 +1,8 @@
use crate::{
error::*, ExecInput, ExecOutput, MetricEvent, MetricEventsSender, Stage, StageExt, UnwindInput,
};
mod ctrl;
mod event;
pub use crate::pipeline::ctrl::ControlFlow;
pub use event::*;
use futures_util::Future;
use reth_db::database::Database;
use reth_interfaces::RethResult;
@ -20,15 +22,22 @@ use tokio::sync::watch;
use tokio_stream::wrappers::UnboundedReceiverStream;
use tracing::*;
mod builder;
mod ctrl;
mod event;
mod progress;
mod set;
// (todo) remove it
#[allow(missing_docs)]
pub mod builder;
pub use crate::pipeline::ctrl::ControlFlow;
// (todo) remove it
#[allow(missing_docs)]
pub mod progress;
// (todo) remove it
#[allow(missing_docs)]
pub mod set;
use crate::{
BlockErrorKind, ExecInput, ExecOutput, MetricEvent, MetricEventsSender, PipelineError, Stage,
StageError, StageExt, UnwindInput,
};
pub use builder::*;
pub use event::*;
use progress::*;
pub use set::*;
@ -372,7 +381,7 @@ where
let exec_input = ExecInput { target, checkpoint: prev_checkpoint };
self.listeners.notify(PipelineEvent::Prepare {
pipeline_stages_progress: event::PipelineStagesProgress {
pipeline_stages_progress: PipelineStagesProgress {
current: stage_index + 1,
total: total_stages,
},
@ -390,7 +399,7 @@ where
}
self.listeners.notify(PipelineEvent::Run {
pipeline_stages_progress: event::PipelineStagesProgress {
pipeline_stages_progress: PipelineStagesProgress {
current: stage_index + 1,
total: total_stages,
},
@ -415,7 +424,7 @@ where
provider_rw.save_stage_checkpoint(stage_id, checkpoint)?;
self.listeners.notify(PipelineEvent::Ran {
pipeline_stages_progress: event::PipelineStagesProgress {
pipeline_stages_progress: PipelineStagesProgress {
current: stage_index + 1,
total: total_stages,
},
@ -896,9 +905,9 @@ mod tests {
///
/// - Stage A syncs to block 10
/// - Stage B triggers an unwind, marking block 5 as bad
/// - Stage B unwinds to it's previous progress, block 0 but since it is still at block 0, it is
/// - Stage B unwinds to its previous progress, block 0 but since it is still at block 0, it is
/// skipped entirely (there is nothing to unwind)
/// - Stage A unwinds to it's previous progress, block 0
/// - Stage A unwinds to its previous progress, block 0
/// - Stage A syncs back up to block 10
/// - Stage B syncs to block 10
/// - The pipeline finishes

View File

@ -1,15 +1,14 @@
use super::ctrl::ControlFlow;
use crate::util::opt;
use crate::{util::opt, ControlFlow};
use reth_primitives::BlockNumber;
#[derive(Debug, Default)]
pub(crate) struct PipelineProgress {
pub struct PipelineProgress {
/// Block number reached by the stage.
pub(crate) block_number: Option<BlockNumber>,
pub block_number: Option<BlockNumber>,
/// The maximum block number achieved by any stage during the execution of the pipeline.
pub(crate) maximum_block_number: Option<BlockNumber>,
pub maximum_block_number: Option<BlockNumber>,
/// The minimum block number achieved by any stage during the execution of the pipeline.
pub(crate) minimum_block_number: Option<BlockNumber>,
pub minimum_block_number: Option<BlockNumber>,
}
impl PipelineProgress {

View File

@ -0,0 +1,2 @@
mod stage;
pub use stage::TestStage;

View File

@ -23,6 +23,7 @@ reth-tokio-util.workspace = true
reth-etl.workspace = true
reth-static-file.workspace = true
reth-config.workspace = true
reth-stages-api = {workspace = true , features = ["test-utils"]}
# async
tokio = { workspace = true, features = ["sync"] }
@ -78,7 +79,7 @@ serde_json.workspace = true
pprof = { workspace = true, features = ["flamegraph", "frame-pointer", "criterion"] }
[features]
test-utils = ["reth-interfaces/test-utils", "reth-db/test-utils", "reth-provider/test-utils"]
test-utils = ["reth-interfaces/test-utils", "reth-db/test-utils", "reth-provider/test-utils", "reth-stages-api/test-utils"]
[[bench]]
name = "criterion"

View File

@ -12,8 +12,8 @@ use reth_primitives::{stage::StageCheckpoint, BlockNumber};
use reth_stages::{
stages::{MerkleStage, SenderRecoveryStage, TransactionLookupStage},
test_utils::TestStageDB,
ExecInput, Stage, StageExt, UnwindInput,
};
use reth_stages_api::{ExecInput, Stage, StageExt, UnwindInput};
use std::{ops::RangeInclusive, sync::Arc};
mod setup;

View File

@ -18,7 +18,6 @@ use reth_primitives::{fs, Account, Address, SealedBlock, B256, U256};
use reth_stages::{
stages::{AccountHashingStage, StorageHashingStage},
test_utils::{StorageKind, TestStageDB},
ExecInput, Stage, UnwindInput,
};
use reth_trie::StateRoot;
use std::{collections::BTreeMap, path::Path, sync::Arc};
@ -27,6 +26,7 @@ mod constants;
mod account_hashing;
pub use account_hashing::*;
use reth_stages_api::{ExecInput, Stage, UnwindInput};
pub(crate) type StageRange = (ExecInput, UnwindInput);

View File

@ -77,12 +77,6 @@
)]
#![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))]
mod error;
mod metrics;
mod pipeline;
mod stage;
mod util;
#[allow(missing_docs)]
#[cfg(any(test, feature = "test-utils"))]
pub mod test_utils;
@ -95,7 +89,5 @@ pub mod stages;
pub mod sets;
pub use crate::metrics::*;
pub use error::*;
pub use pipeline::*;
pub use stage::*;
// re-export the stages API
pub use reth_stages_api::*;

View File

@ -1,8 +1,4 @@
pub use crate::{
error::{PipelineError, StageError},
pipeline::{Pipeline, PipelineBuilder, PipelineEvent, StageSet, StageSetBuilder},
sets::{
DefaultStages, ExecutionStages, HashingStages, HistoryIndexingStages, OfflineStages,
OnlineStages,
},
pub use crate::sets::{
DefaultStages, ExecutionStages, HashingStages, HistoryIndexingStages, OfflineStages,
OnlineStages,
};

View File

@ -1,5 +1,11 @@
use crate::{ExecInput, ExecOutput, Stage, StageError, UnwindInput, UnwindOutput};
use std::{
cmp::Ordering,
task::{ready, Context, Poll},
};
use futures_util::TryStreamExt;
use tracing::*;
use reth_db::{
cursor::{DbCursorRO, DbCursorRW},
database::Database,
@ -19,11 +25,9 @@ use reth_provider::{
providers::{StaticFileProvider, StaticFileWriter},
BlockReader, DatabaseProviderRW, HeaderProvider, ProviderError, StatsReader,
};
use std::{
cmp::Ordering,
task::{ready, Context, Poll},
};
use tracing::*;
use reth_stages_api::{ExecInput, ExecOutput, StageError, UnwindInput, UnwindOutput};
use reth_stages_api::Stage;
// TODO(onbjerg): Metrics and events (gradual status for e.g. CLI)
/// The body stage downloads block bodies.
@ -374,13 +378,16 @@ fn stage_checkpoint<DB: Database>(
#[cfg(test)]
mod tests {
use super::*;
use assert_matches::assert_matches;
use reth_primitives::stage::StageUnitCheckpoint;
use test_utils::*;
use crate::test_utils::{
stage_test_suite_ext, ExecuteStageTestRunner, StageTestRunner, UnwindStageTestRunner,
};
use assert_matches::assert_matches;
use reth_primitives::stage::StageUnitCheckpoint;
use test_utils::*;
use super::*;
stage_test_suite_ext!(BodyTestRunner, body);
@ -588,15 +595,16 @@ mod tests {
}
mod test_utils {
use crate::{
stages::bodies::BodyStage,
test_utils::{
ExecuteStageTestRunner, StageTestRunner, TestRunnerError, TestStageDB,
UnwindStageTestRunner,
},
ExecInput, ExecOutput, UnwindInput,
use std::{
collections::{HashMap, VecDeque},
ops::RangeInclusive,
pin::Pin,
sync::Arc,
task::{Context, Poll},
};
use futures_util::Stream;
use reth_db::{
cursor::DbCursorRO,
models::{StoredBlockBodyIndices, StoredBlockOmmers},
@ -626,12 +634,14 @@ mod tests {
use reth_provider::{
providers::StaticFileWriter, HeaderProvider, ProviderFactory, TransactionsProvider,
};
use std::{
collections::{HashMap, VecDeque},
ops::RangeInclusive,
pin::Pin,
sync::Arc,
task::{Context, Poll},
use reth_stages_api::{ExecInput, ExecOutput, UnwindInput};
use crate::{
stages::bodies::BodyStage,
test_utils::{
ExecuteStageTestRunner, StageTestRunner, TestRunnerError, TestStageDB,
UnwindStageTestRunner,
},
};
/// The block hash of the genesis block.

View File

@ -1,7 +1,4 @@
use crate::{
stages::MERKLE_STAGE_DEFAULT_CLEAN_THRESHOLD, BlockErrorKind, ExecInput, ExecOutput,
MetricEvent, MetricEventsSender, Stage, StageError, UnwindInput, UnwindOutput,
};
use crate::stages::MERKLE_STAGE_DEFAULT_CLEAN_THRESHOLD;
use num_traits::Zero;
use reth_db::{
cursor::{DbCursorRO, DbCursorRW, DbDupCursorRO},
@ -22,6 +19,10 @@ use reth_provider::{
BlockReader, DatabaseProviderRW, ExecutorFactory, HeaderProvider, LatestStateProviderRef,
OriginalValuesKnown, ProviderError, StatsReader, TransactionVariant,
};
use reth_stages_api::{
BlockErrorKind, ExecInput, ExecOutput, MetricEvent, MetricEventsSender, Stage, StageError,
UnwindInput, UnwindOutput,
};
use std::{
cmp::Ordering,
ops::RangeInclusive,

View File

@ -1,7 +1,7 @@
use crate::{ExecInput, ExecOutput, Stage, StageError, UnwindInput, UnwindOutput};
use reth_db::database::Database;
use reth_primitives::stage::{StageCheckpoint, StageId};
use reth_provider::DatabaseProviderRW;
use reth_stages_api::{ExecInput, ExecOutput, Stage, StageError, UnwindInput, UnwindOutput};
/// The finish stage.
///

View File

@ -1,4 +1,3 @@
use crate::{ExecInput, ExecOutput, Stage, StageError, UnwindInput, UnwindOutput};
use itertools::Itertools;
use reth_config::config::EtlConfig;
use reth_db::{
@ -16,6 +15,7 @@ use reth_primitives::{
Account, B256,
};
use reth_provider::{AccountExtReader, DatabaseProviderRW, HashingWriter, StatsReader};
use reth_stages_api::{ExecInput, ExecOutput, Stage, StageError, UnwindInput, UnwindOutput};
use std::{
fmt::Debug,
ops::{Range, RangeInclusive},

View File

@ -1,4 +1,3 @@
use crate::{ExecInput, ExecOutput, Stage, StageError, UnwindInput, UnwindOutput};
use itertools::Itertools;
use reth_config::config::EtlConfig;
use reth_db::{
@ -18,6 +17,7 @@ use reth_primitives::{
BufMut, StorageEntry, B256,
};
use reth_provider::{DatabaseProviderRW, HashingWriter, StatsReader, StorageReader};
use reth_stages_api::{ExecInput, ExecOutput, Stage, StageError, UnwindInput, UnwindOutput};
use std::{
fmt::Debug,
sync::mpsc::{self, Receiver},

View File

@ -1,4 +1,3 @@
use crate::{BlockErrorKind, ExecInput, ExecOutput, Stage, StageError, UnwindInput, UnwindOutput};
use futures_util::StreamExt;
use reth_codecs::Compact;
use reth_config::config::EtlConfig;
@ -26,6 +25,9 @@ use reth_provider::{
BlockHashReader, DatabaseProviderRW, HeaderProvider, HeaderSyncGap, HeaderSyncGapProvider,
HeaderSyncMode,
};
use reth_stages_api::{
BlockErrorKind, ExecInput, ExecOutput, Stage, StageError, UnwindInput, UnwindOutput,
};
use std::{
sync::Arc,
task::{ready, Context, Poll},

View File

@ -1,5 +1,4 @@
use super::{collect_history_indices, load_history_indices};
use crate::{ExecInput, ExecOutput, Stage, StageError, UnwindInput, UnwindOutput};
use reth_config::config::EtlConfig;
use reth_db::{
database::Database, models::ShardedKey, table::Decode, tables, transaction::DbTxMut,
@ -11,6 +10,7 @@ use reth_primitives::{
use reth_provider::{
DatabaseProviderRW, HistoryWriter, PruneCheckpointReader, PruneCheckpointWriter,
};
use reth_stages_api::{ExecInput, ExecOutput, Stage, StageError, UnwindInput, UnwindOutput};
use std::fmt::Debug;
use tracing::info;

View File

@ -1,5 +1,4 @@
use super::{collect_history_indices, load_history_indices};
use crate::{ExecInput, ExecOutput, Stage, StageError, UnwindInput, UnwindOutput};
use reth_config::config::EtlConfig;
use reth_db::{
database::Database,
@ -15,6 +14,7 @@ use reth_primitives::{
use reth_provider::{
DatabaseProviderRW, HistoryWriter, PruneCheckpointReader, PruneCheckpointWriter,
};
use reth_stages_api::{ExecInput, ExecOutput, Stage, StageError, UnwindInput, UnwindOutput};
use std::fmt::Debug;
use tracing::info;

View File

@ -1,4 +1,3 @@
use crate::{BlockErrorKind, ExecInput, ExecOutput, Stage, StageError, UnwindInput, UnwindOutput};
use reth_codecs::Compact;
use reth_db::{
database::Database,
@ -15,6 +14,9 @@ use reth_provider::{
DatabaseProviderRW, HeaderProvider, ProviderError, StageCheckpointReader,
StageCheckpointWriter, StatsReader,
};
use reth_stages_api::{
BlockErrorKind, ExecInput, ExecOutput, Stage, StageError, UnwindInput, UnwindOutput,
};
use reth_trie::{IntermediateStateRootState, StateRoot, StateRootProgress};
use std::fmt::Debug;
use tracing::*;

View File

@ -40,7 +40,7 @@ use utils::*;
#[cfg(test)]
mod tests {
use super::*;
use crate::{stage::Stage, test_utils::TestStageDB, ExecInput};
use crate::test_utils::TestStageDB;
use alloy_rlp::Decodable;
use reth_db::{
cursor::DbCursorRO,
@ -61,6 +61,7 @@ mod tests {
StorageReader,
};
use reth_revm::EvmProcessorFactory;
use reth_stages_api::{ExecInput, Stage};
use std::sync::Arc;
#[tokio::test]

View File

@ -1,4 +1,3 @@
use crate::{BlockErrorKind, ExecInput, ExecOutput, Stage, StageError, UnwindInput, UnwindOutput};
use reth_db::{
cursor::DbCursorRW,
database::Database,
@ -16,6 +15,9 @@ use reth_provider::{
BlockReader, DatabaseProviderRW, HeaderProvider, ProviderError, PruneCheckpointReader,
StatsReader,
};
use reth_stages_api::{
BlockErrorKind, ExecInput, ExecOutput, Stage, StageError, UnwindInput, UnwindOutput,
};
use std::{fmt::Debug, ops::Range, sync::mpsc};
use thiserror::Error;
use tracing::*;

View File

@ -1,4 +1,3 @@
use crate::{ExecInput, ExecOutput, Stage, StageError, UnwindInput, UnwindOutput};
use num_traits::Zero;
use reth_config::config::EtlConfig;
use reth_db::{
@ -18,6 +17,7 @@ use reth_provider::{
BlockReader, DatabaseProviderRW, PruneCheckpointReader, PruneCheckpointWriter, StatsReader,
TransactionsProvider, TransactionsProviderExt,
};
use reth_stages_api::{ExecInput, ExecOutput, Stage, StageError, UnwindInput, UnwindOutput};
use tracing::*;
/// The transaction lookup stage.

View File

@ -1,5 +1,4 @@
//! Utils for `stages`.
use crate::StageError;
use reth_config::config::EtlConfig;
use reth_db::{
cursor::{DbCursorRO, DbCursorRW},
@ -10,6 +9,7 @@ use reth_db::{
};
use reth_etl::Collector;
use reth_primitives::BlockNumber;
use reth_stages_api::StageError;
use std::{collections::HashMap, hash::Hash, ops::RangeBounds};
use tracing::info;

View File

@ -9,7 +9,7 @@ macro_rules! stage_test_suite {
let runner = $runner::default();
// Execute the stage with empty database
let input = crate::stage::ExecInput::default();
let input = reth_stages_api::ExecInput::default();
// Run stage execution
let result = runner.execute(input).await;
@ -34,7 +34,7 @@ macro_rules! stage_test_suite {
// Set up the runner
let mut runner = $runner::default();
let input = crate::stage::ExecInput {
let input = reth_stages_api::ExecInput {
target: Some(target),
checkpoint: Some(reth_primitives::stage::StageCheckpoint::new(current_checkpoint)),
};
@ -67,10 +67,10 @@ macro_rules! stage_test_suite {
async fn [< unwind_no_new_entries_ $name>] () {
// Set up the runner
let mut runner = $runner::default();
let input = crate::stage::UnwindInput::default();
let input = reth_stages_api::UnwindInput::default();
// Seed the database
runner.seed_execution(crate::stage::ExecInput::default()).expect("failed to seed");
runner.seed_execution(reth_stages_api::ExecInput::default()).expect("failed to seed");
runner.before_unwind(input).expect("failed to execute before_unwind hook");
@ -98,7 +98,7 @@ macro_rules! stage_test_suite {
// Set up the runner
let mut runner = $runner::default();
let execute_input = crate::stage::ExecInput {
let execute_input = reth_stages_api::ExecInput {
target: Some(target),
checkpoint: Some(reth_primitives::stage::StageCheckpoint::new(current_checkpoint)),
};
@ -125,7 +125,7 @@ macro_rules! stage_test_suite {
// Run stage unwind
let unwind_input = crate::stage::UnwindInput {
let unwind_input = reth_stages_api::UnwindInput {
unwind_to: current_checkpoint,
checkpoint: reth_primitives::stage::StageCheckpoint::new(target),
bad_block: None,
@ -165,7 +165,7 @@ macro_rules! stage_test_suite_ext {
// Set up the runner
let mut runner = $runner::default();
let input = crate::stage::ExecInput {
let input = reth_stages_api::ExecInput {
target: Some(current_checkpoint),
checkpoint: Some(reth_primitives::stage::StageCheckpoint::new(current_checkpoint)),
};

View File

@ -15,9 +15,6 @@ pub(crate) use runner::{
mod test_db;
pub use test_db::{StorageKind, TestStageDB};
mod stage;
pub use stage::TestStage;
mod set;
pub use set::TestStages;

View File

@ -1,8 +1,10 @@
use super::TestStageDB;
use crate::{ExecInput, ExecOutput, Stage, StageError, StageExt, UnwindInput, UnwindOutput};
use reth_db::{test_utils::TempDatabase, DatabaseEnv};
use reth_interfaces::db::DatabaseError;
use reth_provider::ProviderError;
use reth_stages_api::{
ExecInput, ExecOutput, Stage, StageError, StageExt, UnwindInput, UnwindOutput,
};
use std::sync::Arc;
use tokio::sync::oneshot;

View File

@ -1,6 +1,7 @@
use super::{TestStage, TEST_STAGE_ID};
use crate::{ExecOutput, StageError, StageSet, StageSetBuilder, UnwindOutput};
use super::TEST_STAGE_ID;
use crate::{StageSet, StageSetBuilder};
use reth_db::database::Database;
use reth_stages_api::{test_utils::TestStage, ExecOutput, StageError, UnwindOutput};
use std::collections::VecDeque;
#[derive(Default, Debug)]