mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
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:
27
Cargo.lock
generated
27
Cargo.lock
generated
@ -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"
|
||||
|
||||
@ -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" }
|
||||
|
||||
@ -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};
|
||||
|
||||
@ -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"] }
|
||||
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -1,6 +1,5 @@
|
||||
use node_e2e_tests::{node::NodeHelper, wallet::Wallet};
|
||||
use reth::{
|
||||
self,
|
||||
args::RpcServerArgs,
|
||||
builder::{NodeBuilder, NodeConfig, NodeHandle},
|
||||
tasks::TaskManager,
|
||||
|
||||
40
crates/stages-api/Cargo.toml
Normal file
40
crates/stages-api/Cargo.toml
Normal 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
|
||||
@ -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.
|
||||
14
crates/stages-api/src/lib.rs
Normal file
14
crates/stages-api/src/lib.rs
Normal 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::*;
|
||||
@ -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 {
|
||||
@ -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,
|
||||
}
|
||||
@ -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
|
||||
@ -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 {
|
||||
2
crates/stages-api/src/test_utils/mod.rs
Normal file
2
crates/stages-api/src/test_utils/mod.rs
Normal file
@ -0,0 +1,2 @@
|
||||
mod stage;
|
||||
pub use stage::TestStage;
|
||||
@ -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"
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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);
|
||||
|
||||
|
||||
@ -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::*;
|
||||
|
||||
@ -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,
|
||||
};
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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.
|
||||
///
|
||||
|
||||
@ -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},
|
||||
|
||||
@ -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},
|
||||
|
||||
@ -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},
|
||||
|
||||
@ -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;
|
||||
|
||||
|
||||
@ -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;
|
||||
|
||||
|
||||
@ -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::*;
|
||||
|
||||
@ -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]
|
||||
|
||||
@ -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::*;
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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;
|
||||
|
||||
|
||||
@ -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)),
|
||||
};
|
||||
|
||||
@ -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;
|
||||
|
||||
|
||||
@ -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;
|
||||
|
||||
|
||||
@ -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)]
|
||||
|
||||
Reference in New Issue
Block a user