mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
feat: integrate NodeTypesWithDB (#10698)
Co-authored-by: joshieDo <93316087+joshieDo@users.noreply.github.com>
This commit is contained in:
@ -29,6 +29,7 @@ reth-trie-db = { workspace = true, features = ["metrics"] }
|
||||
reth-trie-parallel = { workspace = true, features = ["parallel"] }
|
||||
reth-network.workspace = true
|
||||
reth-consensus.workspace = true
|
||||
reth-node-types.workspace = true
|
||||
|
||||
# common
|
||||
parking_lot.workspace = true
|
||||
|
||||
@ -10,18 +10,19 @@ use reth_blockchain_tree_api::{
|
||||
BlockAttachment, BlockStatus, BlockValidationKind, CanonicalOutcome, InsertPayloadOk,
|
||||
};
|
||||
use reth_consensus::{Consensus, ConsensusError};
|
||||
use reth_db_api::database::Database;
|
||||
use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_execution_errors::{BlockExecutionError, BlockValidationError};
|
||||
use reth_execution_types::{Chain, ExecutionOutcome};
|
||||
use reth_node_types::NodeTypesWithDB;
|
||||
use reth_primitives::{
|
||||
BlockHash, BlockNumHash, BlockNumber, EthereumHardfork, ForkBlock, GotExpected, Receipt,
|
||||
SealedBlock, SealedBlockWithSenders, SealedHeader, StaticFileSegment, B256, U256,
|
||||
};
|
||||
use reth_provider::{
|
||||
BlockExecutionWriter, BlockNumReader, BlockWriter, CanonStateNotification,
|
||||
CanonStateNotificationSender, CanonStateNotifications, ChainSpecProvider, ChainSplit,
|
||||
ChainSplitTarget, DisplayBlocksChain, HeaderProvider, ProviderError, StaticFileProviderFactory,
|
||||
providers::ProviderNodeTypes, BlockExecutionWriter, BlockNumReader, BlockWriter,
|
||||
CanonStateNotification, CanonStateNotificationSender, CanonStateNotifications,
|
||||
ChainSpecProvider, ChainSplit, ChainSplitTarget, DisplayBlocksChain, HeaderProvider,
|
||||
ProviderError, StaticFileProviderFactory,
|
||||
};
|
||||
use reth_prune_types::PruneModes;
|
||||
use reth_stages_api::{MetricEvent, MetricEventsSender};
|
||||
@ -58,13 +59,13 @@ use tracing::{debug, error, info, instrument, trace, warn};
|
||||
/// * [`BlockchainTree::make_canonical`]: Check if we have the hash of a block that is the current
|
||||
/// canonical head and commit it to db.
|
||||
#[derive(Debug)]
|
||||
pub struct BlockchainTree<DB, E> {
|
||||
pub struct BlockchainTree<N: NodeTypesWithDB, E> {
|
||||
/// The state of the tree
|
||||
///
|
||||
/// Tracks all the chains, the block indices, and the block buffer.
|
||||
state: TreeState,
|
||||
/// External components (the database, consensus engine etc.)
|
||||
externals: TreeExternals<DB, E>,
|
||||
externals: TreeExternals<N, E>,
|
||||
/// Tree configuration
|
||||
config: BlockchainTreeConfig,
|
||||
/// Broadcast channel for canon state changes notifications.
|
||||
@ -75,7 +76,7 @@ pub struct BlockchainTree<DB, E> {
|
||||
metrics: TreeMetrics,
|
||||
}
|
||||
|
||||
impl<DB, E> BlockchainTree<DB, E> {
|
||||
impl<N: NodeTypesWithDB, E> BlockchainTree<N, E> {
|
||||
/// Subscribe to new blocks events.
|
||||
///
|
||||
/// Note: Only canonical blocks are emitted by the tree.
|
||||
@ -89,9 +90,9 @@ impl<DB, E> BlockchainTree<DB, E> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB, E> BlockchainTree<DB, E>
|
||||
impl<N, E> BlockchainTree<N, E>
|
||||
where
|
||||
DB: Database + Clone,
|
||||
N: ProviderNodeTypes,
|
||||
E: BlockExecutorProvider,
|
||||
{
|
||||
/// Builds the blockchain tree for the node.
|
||||
@ -115,7 +116,7 @@ where
|
||||
/// storage space efficiently. It's important to validate this configuration to ensure it does
|
||||
/// not lead to unintended data loss.
|
||||
pub fn new(
|
||||
mut externals: TreeExternals<DB, E>,
|
||||
mut externals: TreeExternals<N, E>,
|
||||
config: BlockchainTreeConfig,
|
||||
prune_modes: PruneModes,
|
||||
) -> ProviderResult<Self> {
|
||||
@ -1382,7 +1383,7 @@ mod tests {
|
||||
use linked_hash_set::LinkedHashSet;
|
||||
use reth_chainspec::{ChainSpecBuilder, MAINNET};
|
||||
use reth_consensus::test_utils::TestConsensus;
|
||||
use reth_db::{tables, test_utils::TempDatabase, DatabaseEnv};
|
||||
use reth_db::tables;
|
||||
use reth_db_api::transaction::DbTxMut;
|
||||
use reth_evm::test_utils::MockExecutorProvider;
|
||||
use reth_evm_ethereum::execute::EthExecutorProvider;
|
||||
@ -1395,7 +1396,10 @@ mod tests {
|
||||
TransactionSignedEcRecovered, TxEip1559, Withdrawals, B256,
|
||||
};
|
||||
use reth_provider::{
|
||||
test_utils::{blocks::BlockchainTestData, create_test_provider_factory_with_chain_spec},
|
||||
test_utils::{
|
||||
blocks::BlockchainTestData, create_test_provider_factory_with_chain_spec,
|
||||
MockNodeTypesWithDB,
|
||||
},
|
||||
ProviderFactory,
|
||||
};
|
||||
use reth_stages_api::StageCheckpoint;
|
||||
@ -1404,7 +1408,7 @@ mod tests {
|
||||
|
||||
fn setup_externals(
|
||||
exec_res: Vec<ExecutionOutcome>,
|
||||
) -> TreeExternals<Arc<TempDatabase<DatabaseEnv>>, MockExecutorProvider> {
|
||||
) -> TreeExternals<MockNodeTypesWithDB, MockExecutorProvider> {
|
||||
let chain_spec = Arc::new(
|
||||
ChainSpecBuilder::default()
|
||||
.chain(MAINNET.chain)
|
||||
@ -1420,7 +1424,7 @@ mod tests {
|
||||
TreeExternals::new(provider_factory, consensus, executor_factory)
|
||||
}
|
||||
|
||||
fn setup_genesis<DB: Database>(factory: &ProviderFactory<DB>, mut genesis: SealedBlock) {
|
||||
fn setup_genesis<N: ProviderNodeTypes>(factory: &ProviderFactory<N>, mut genesis: SealedBlock) {
|
||||
// insert genesis to db.
|
||||
|
||||
genesis.header.set_block_number(10);
|
||||
@ -1497,7 +1501,7 @@ mod tests {
|
||||
self
|
||||
}
|
||||
|
||||
fn assert<DB: Database, E: BlockExecutorProvider>(self, tree: &BlockchainTree<DB, E>) {
|
||||
fn assert<N: NodeTypesWithDB, E: BlockExecutorProvider>(self, tree: &BlockchainTree<N, E>) {
|
||||
if let Some(chain_num) = self.chain_num {
|
||||
assert_eq!(tree.state.chains.len(), chain_num);
|
||||
}
|
||||
|
||||
@ -10,7 +10,6 @@ use reth_blockchain_tree_api::{
|
||||
BlockAttachment, BlockValidationKind,
|
||||
};
|
||||
use reth_consensus::{Consensus, ConsensusError, PostExecutionInput};
|
||||
use reth_db_api::database::Database;
|
||||
use reth_evm::execute::{BlockExecutorProvider, Executor};
|
||||
use reth_execution_errors::BlockExecutionError;
|
||||
use reth_execution_types::{Chain, ExecutionOutcome};
|
||||
@ -18,7 +17,7 @@ use reth_primitives::{
|
||||
BlockHash, BlockNumber, ForkBlock, GotExpected, SealedBlockWithSenders, SealedHeader, U256,
|
||||
};
|
||||
use reth_provider::{
|
||||
providers::{BundleStateProvider, ConsistentDbView},
|
||||
providers::{BundleStateProvider, ConsistentDbView, ProviderNodeTypes},
|
||||
FullExecutionDataProvider, ProviderError, StateRootProvider,
|
||||
};
|
||||
use reth_revm::database::StateProviderDatabase;
|
||||
@ -66,17 +65,17 @@ impl AppendableChain {
|
||||
///
|
||||
/// if [`BlockValidationKind::Exhaustive`] is specified, the method will verify the state root
|
||||
/// of the block.
|
||||
pub fn new_canonical_fork<DB, E>(
|
||||
pub fn new_canonical_fork<N, E>(
|
||||
block: SealedBlockWithSenders,
|
||||
parent_header: &SealedHeader,
|
||||
canonical_block_hashes: &BTreeMap<BlockNumber, BlockHash>,
|
||||
canonical_fork: ForkBlock,
|
||||
externals: &TreeExternals<DB, E>,
|
||||
externals: &TreeExternals<N, E>,
|
||||
block_attachment: BlockAttachment,
|
||||
block_validation_kind: BlockValidationKind,
|
||||
) -> Result<Self, InsertBlockErrorKind>
|
||||
where
|
||||
DB: Database + Clone,
|
||||
N: ProviderNodeTypes,
|
||||
E: BlockExecutorProvider,
|
||||
{
|
||||
let execution_outcome = ExecutionOutcome::default();
|
||||
@ -104,17 +103,17 @@ impl AppendableChain {
|
||||
/// Create a new chain that forks off of an existing sidechain.
|
||||
///
|
||||
/// This differs from [`AppendableChain::new_canonical_fork`] in that this starts a new fork.
|
||||
pub(crate) fn new_chain_fork<DB, E>(
|
||||
pub(crate) fn new_chain_fork<N, E>(
|
||||
&self,
|
||||
block: SealedBlockWithSenders,
|
||||
side_chain_block_hashes: BTreeMap<BlockNumber, BlockHash>,
|
||||
canonical_block_hashes: &BTreeMap<BlockNumber, BlockHash>,
|
||||
canonical_fork: ForkBlock,
|
||||
externals: &TreeExternals<DB, E>,
|
||||
externals: &TreeExternals<N, E>,
|
||||
block_validation_kind: BlockValidationKind,
|
||||
) -> Result<Self, InsertBlockErrorKind>
|
||||
where
|
||||
DB: Database + Clone,
|
||||
N: ProviderNodeTypes,
|
||||
E: BlockExecutorProvider,
|
||||
{
|
||||
let parent_number =
|
||||
@ -167,17 +166,17 @@ impl AppendableChain {
|
||||
/// - [`BlockAttachment`] represents if the block extends the canonical chain, and thus we can
|
||||
/// cache the trie state updates.
|
||||
/// - [`BlockValidationKind`] determines if the state root __should__ be validated.
|
||||
fn validate_and_execute<EDP, DB, E>(
|
||||
fn validate_and_execute<EDP, N, E>(
|
||||
block: SealedBlockWithSenders,
|
||||
parent_block: &SealedHeader,
|
||||
bundle_state_data_provider: EDP,
|
||||
externals: &TreeExternals<DB, E>,
|
||||
externals: &TreeExternals<N, E>,
|
||||
block_attachment: BlockAttachment,
|
||||
block_validation_kind: BlockValidationKind,
|
||||
) -> Result<(ExecutionOutcome, Option<TrieUpdates>), BlockExecutionError>
|
||||
where
|
||||
EDP: FullExecutionDataProvider,
|
||||
DB: Database + Clone,
|
||||
N: ProviderNodeTypes,
|
||||
E: BlockExecutorProvider,
|
||||
{
|
||||
// some checks are done before blocks comes here.
|
||||
@ -271,18 +270,18 @@ impl AppendableChain {
|
||||
/// __not__ the canonical head.
|
||||
#[track_caller]
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
pub(crate) fn append_block<DB, E>(
|
||||
pub(crate) fn append_block<N, E>(
|
||||
&mut self,
|
||||
block: SealedBlockWithSenders,
|
||||
side_chain_block_hashes: BTreeMap<BlockNumber, BlockHash>,
|
||||
canonical_block_hashes: &BTreeMap<BlockNumber, BlockHash>,
|
||||
externals: &TreeExternals<DB, E>,
|
||||
externals: &TreeExternals<N, E>,
|
||||
canonical_fork: ForkBlock,
|
||||
block_attachment: BlockAttachment,
|
||||
block_validation_kind: BlockValidationKind,
|
||||
) -> Result<(), InsertBlockErrorKind>
|
||||
where
|
||||
DB: Database + Clone,
|
||||
N: ProviderNodeTypes,
|
||||
E: BlockExecutorProvider,
|
||||
{
|
||||
let parent_block = self.chain.tip();
|
||||
|
||||
@ -2,11 +2,12 @@
|
||||
|
||||
use reth_consensus::Consensus;
|
||||
use reth_db::{static_file::HeaderMask, tables};
|
||||
use reth_db_api::{cursor::DbCursorRO, database::Database, transaction::DbTx};
|
||||
use reth_db_api::{cursor::DbCursorRO, transaction::DbTx};
|
||||
use reth_node_types::NodeTypesWithDB;
|
||||
use reth_primitives::{BlockHash, BlockNumber, StaticFileSegment};
|
||||
use reth_provider::{
|
||||
FinalizedBlockReader, FinalizedBlockWriter, ProviderFactory, StaticFileProviderFactory,
|
||||
StatsReader,
|
||||
providers::ProviderNodeTypes, FinalizedBlockReader, FinalizedBlockWriter, ProviderFactory,
|
||||
StaticFileProviderFactory, StatsReader,
|
||||
};
|
||||
use reth_storage_errors::provider::ProviderResult;
|
||||
use std::{collections::BTreeMap, sync::Arc};
|
||||
@ -21,19 +22,19 @@ use std::{collections::BTreeMap, sync::Arc};
|
||||
/// - The executor factory to execute blocks with
|
||||
/// - The chain spec
|
||||
#[derive(Debug)]
|
||||
pub struct TreeExternals<DB, E> {
|
||||
pub struct TreeExternals<N: NodeTypesWithDB, E> {
|
||||
/// The provider factory, used to commit the canonical chain, or unwind it.
|
||||
pub(crate) provider_factory: ProviderFactory<DB>,
|
||||
pub(crate) provider_factory: ProviderFactory<N>,
|
||||
/// The consensus engine.
|
||||
pub(crate) consensus: Arc<dyn Consensus>,
|
||||
/// The executor factory to execute blocks with.
|
||||
pub(crate) executor_factory: E,
|
||||
}
|
||||
|
||||
impl<DB, E> TreeExternals<DB, E> {
|
||||
impl<N: ProviderNodeTypes, E> TreeExternals<N, E> {
|
||||
/// Create new tree externals.
|
||||
pub fn new(
|
||||
provider_factory: ProviderFactory<DB>,
|
||||
provider_factory: ProviderFactory<N>,
|
||||
consensus: Arc<dyn Consensus>,
|
||||
executor_factory: E,
|
||||
) -> Self {
|
||||
@ -41,7 +42,7 @@ impl<DB, E> TreeExternals<DB, E> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database, E> TreeExternals<DB, E> {
|
||||
impl<N: ProviderNodeTypes, E> TreeExternals<N, E> {
|
||||
/// Fetches the latest canonical block hashes by walking backwards from the head.
|
||||
///
|
||||
/// Returns the hashes sorted by increasing block numbers
|
||||
|
||||
@ -7,15 +7,15 @@ use reth_blockchain_tree_api::{
|
||||
BlockValidationKind, BlockchainTreeEngine, BlockchainTreeViewer, CanonicalOutcome,
|
||||
InsertPayloadOk,
|
||||
};
|
||||
use reth_db_api::database::Database;
|
||||
use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_node_types::NodeTypesWithDB;
|
||||
use reth_primitives::{
|
||||
BlockHash, BlockNumHash, BlockNumber, Receipt, SealedBlock, SealedBlockWithSenders,
|
||||
SealedHeader,
|
||||
};
|
||||
use reth_provider::{
|
||||
BlockchainTreePendingStateProvider, CanonStateSubscriptions, FullExecutionDataProvider,
|
||||
ProviderError,
|
||||
providers::ProviderNodeTypes, BlockchainTreePendingStateProvider, CanonStateSubscriptions,
|
||||
FullExecutionDataProvider, ProviderError,
|
||||
};
|
||||
use reth_storage_errors::provider::ProviderResult;
|
||||
use std::{collections::BTreeMap, sync::Arc};
|
||||
@ -23,21 +23,21 @@ use tracing::trace;
|
||||
|
||||
/// Shareable blockchain tree that is behind a `RwLock`
|
||||
#[derive(Clone, Debug)]
|
||||
pub struct ShareableBlockchainTree<DB, E> {
|
||||
pub struct ShareableBlockchainTree<N: NodeTypesWithDB, E> {
|
||||
/// `BlockchainTree`
|
||||
pub tree: Arc<RwLock<BlockchainTree<DB, E>>>,
|
||||
pub tree: Arc<RwLock<BlockchainTree<N, E>>>,
|
||||
}
|
||||
|
||||
impl<DB, E> ShareableBlockchainTree<DB, E> {
|
||||
impl<N: NodeTypesWithDB, E> ShareableBlockchainTree<N, E> {
|
||||
/// Create a new shareable database.
|
||||
pub fn new(tree: BlockchainTree<DB, E>) -> Self {
|
||||
pub fn new(tree: BlockchainTree<N, E>) -> Self {
|
||||
Self { tree: Arc::new(RwLock::new(tree)) }
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB, E> BlockchainTreeEngine for ShareableBlockchainTree<DB, E>
|
||||
impl<N, E> BlockchainTreeEngine for ShareableBlockchainTree<N, E>
|
||||
where
|
||||
DB: Database + Clone,
|
||||
N: ProviderNodeTypes,
|
||||
E: BlockExecutorProvider,
|
||||
{
|
||||
fn buffer_block(&self, block: SealedBlockWithSenders) -> Result<(), InsertBlockError> {
|
||||
@ -106,9 +106,9 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB, E> BlockchainTreeViewer for ShareableBlockchainTree<DB, E>
|
||||
impl<N, E> BlockchainTreeViewer for ShareableBlockchainTree<N, E>
|
||||
where
|
||||
DB: Database + Clone,
|
||||
N: ProviderNodeTypes,
|
||||
E: BlockExecutorProvider,
|
||||
{
|
||||
fn header_by_hash(&self, hash: BlockHash) -> Option<SealedHeader> {
|
||||
@ -169,9 +169,9 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB, E> BlockchainTreePendingStateProvider for ShareableBlockchainTree<DB, E>
|
||||
impl<N, E> BlockchainTreePendingStateProvider for ShareableBlockchainTree<N, E>
|
||||
where
|
||||
DB: Database + Clone,
|
||||
N: ProviderNodeTypes,
|
||||
E: BlockExecutorProvider,
|
||||
{
|
||||
fn find_pending_state_provider(
|
||||
@ -184,9 +184,9 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB, E> CanonStateSubscriptions for ShareableBlockchainTree<DB, E>
|
||||
impl<N, E> CanonStateSubscriptions for ShareableBlockchainTree<N, E>
|
||||
where
|
||||
DB: Send + Sync,
|
||||
N: ProviderNodeTypes,
|
||||
E: Send + Sync,
|
||||
{
|
||||
fn subscribe_to_canonical_state(&self) -> reth_provider::CanonStateNotifications {
|
||||
|
||||
@ -1,8 +1,10 @@
|
||||
use core::fmt::Debug;
|
||||
|
||||
use crate::ChainSpec;
|
||||
use alloy_chains::Chain;
|
||||
|
||||
/// Trait representing type configuring a chain spec.
|
||||
pub trait EthChainSpec: Send + Sync + Unpin + 'static {
|
||||
pub trait EthChainSpec: Send + Sync + Unpin + Debug + 'static {
|
||||
// todo: make chain spec type generic over hardfork
|
||||
//type Hardfork: Clone + Copy + 'static;
|
||||
|
||||
|
||||
@ -9,6 +9,7 @@ use reth_db::{init_db, open_db_read_only, DatabaseEnv};
|
||||
use reth_db_common::init::init_genesis;
|
||||
use reth_downloaders::{bodies::noop::NoopBodiesDownloader, headers::noop::NoopHeaderDownloader};
|
||||
use reth_evm::noop::NoopBlockExecutorProvider;
|
||||
use reth_node_builder::{NodeTypesWithDBAdapter, NodeTypesWithEngine};
|
||||
use reth_node_core::{
|
||||
args::{DatabaseArgs, DatadirArgs},
|
||||
dirs::{ChainPath, DataDirPath},
|
||||
@ -52,7 +53,10 @@ pub struct EnvironmentArgs<C: ChainSpecParser> {
|
||||
impl<C: ChainSpecParser<ChainSpec = ChainSpec>> EnvironmentArgs<C> {
|
||||
/// Initializes environment according to [`AccessRights`] and returns an instance of
|
||||
/// [`Environment`].
|
||||
pub fn init(&self, access: AccessRights) -> eyre::Result<Environment> {
|
||||
pub fn init<N: NodeTypesWithEngine<ChainSpec = C::ChainSpec>>(
|
||||
&self,
|
||||
access: AccessRights,
|
||||
) -> eyre::Result<Environment<N>> {
|
||||
let data_dir = self.datadir.clone().resolve_datadir(self.chain.chain);
|
||||
let db_path = data_dir.db();
|
||||
let sf_path = data_dir.static_files();
|
||||
@ -101,17 +105,21 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> EnvironmentArgs<C> {
|
||||
/// If it's a read-write environment and an issue is found, it will attempt to heal (including a
|
||||
/// pipeline unwind). Otherwise, it will print out an warning, advising the user to restart the
|
||||
/// node to heal.
|
||||
fn create_provider_factory(
|
||||
fn create_provider_factory<N: NodeTypesWithEngine<ChainSpec = C::ChainSpec>>(
|
||||
&self,
|
||||
config: &Config,
|
||||
db: Arc<DatabaseEnv>,
|
||||
static_file_provider: StaticFileProvider,
|
||||
) -> eyre::Result<ProviderFactory<Arc<DatabaseEnv>>> {
|
||||
) -> eyre::Result<ProviderFactory<NodeTypesWithDBAdapter<N, Arc<DatabaseEnv>>>> {
|
||||
let has_receipt_pruning = config.prune.as_ref().map_or(false, |a| a.has_receipts_pruning());
|
||||
let prune_modes =
|
||||
config.prune.as_ref().map(|prune| prune.segments.clone()).unwrap_or_default();
|
||||
let factory = ProviderFactory::new(db, self.chain.clone(), static_file_provider)
|
||||
.with_prune_modes(prune_modes.clone());
|
||||
let factory = ProviderFactory::<NodeTypesWithDBAdapter<N, Arc<DatabaseEnv>>>::new(
|
||||
db,
|
||||
self.chain.clone(),
|
||||
static_file_provider,
|
||||
)
|
||||
.with_prune_modes(prune_modes.clone());
|
||||
|
||||
// Check for consistency between database and static files.
|
||||
if let Some(unwind_target) = factory
|
||||
@ -132,7 +140,7 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> EnvironmentArgs<C> {
|
||||
let (_tip_tx, tip_rx) = watch::channel(B256::ZERO);
|
||||
|
||||
// Builds and executes an unwind-only pipeline
|
||||
let mut pipeline = Pipeline::builder()
|
||||
let mut pipeline = Pipeline::<NodeTypesWithDBAdapter<N, Arc<DatabaseEnv>>>::builder()
|
||||
.add_stages(DefaultStages::new(
|
||||
factory.clone(),
|
||||
tip_rx,
|
||||
@ -156,11 +164,11 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> EnvironmentArgs<C> {
|
||||
|
||||
/// Environment built from [`EnvironmentArgs`].
|
||||
#[derive(Debug)]
|
||||
pub struct Environment {
|
||||
pub struct Environment<N: NodeTypesWithEngine> {
|
||||
/// Configuration for reth node
|
||||
pub config: Config,
|
||||
/// Provider factory.
|
||||
pub provider_factory: ProviderFactory<Arc<DatabaseEnv>>,
|
||||
pub provider_factory: ProviderFactory<NodeTypesWithDBAdapter<N, Arc<DatabaseEnv>>>,
|
||||
/// Datadir path.
|
||||
pub data_dir: ChainPath<DataDirPath>,
|
||||
}
|
||||
|
||||
@ -1,9 +1,11 @@
|
||||
use crate::db::get::{maybe_json_value_parser, table_key};
|
||||
use ahash::RandomState;
|
||||
use clap::Parser;
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_db::{DatabaseEnv, RawKey, RawTable, RawValue, TableViewer, Tables};
|
||||
use reth_db_api::{cursor::DbCursorRO, database::Database, table::Table, transaction::DbTx};
|
||||
use reth_db_api::{cursor::DbCursorRO, table::Table, transaction::DbTx};
|
||||
use reth_db_common::DbTool;
|
||||
use reth_node_builder::{NodeTypesWithDB, NodeTypesWithDBAdapter, NodeTypesWithEngine};
|
||||
use std::{
|
||||
hash::{BuildHasher, Hasher},
|
||||
sync::Arc,
|
||||
@ -33,7 +35,10 @@ pub struct Command {
|
||||
|
||||
impl Command {
|
||||
/// Execute `db checksum` command
|
||||
pub fn execute(self, tool: &DbTool<Arc<DatabaseEnv>>) -> eyre::Result<()> {
|
||||
pub fn execute<N: NodeTypesWithEngine<ChainSpec = ChainSpec>>(
|
||||
self,
|
||||
tool: &DbTool<NodeTypesWithDBAdapter<N, Arc<DatabaseEnv>>>,
|
||||
) -> eyre::Result<()> {
|
||||
warn!("This command should be run without the node running!");
|
||||
self.table.view(&ChecksumViewer {
|
||||
tool,
|
||||
@ -45,20 +50,22 @@ impl Command {
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) struct ChecksumViewer<'a, DB: Database> {
|
||||
tool: &'a DbTool<DB>,
|
||||
pub(crate) struct ChecksumViewer<'a, N: NodeTypesWithDB> {
|
||||
tool: &'a DbTool<N>,
|
||||
start_key: Option<String>,
|
||||
end_key: Option<String>,
|
||||
limit: Option<usize>,
|
||||
}
|
||||
|
||||
impl<DB: Database> ChecksumViewer<'_, DB> {
|
||||
pub(crate) const fn new(tool: &'_ DbTool<DB>) -> ChecksumViewer<'_, DB> {
|
||||
impl<N: NodeTypesWithDB> ChecksumViewer<'_, N> {
|
||||
pub(crate) const fn new(tool: &'_ DbTool<N>) -> ChecksumViewer<'_, N> {
|
||||
ChecksumViewer { tool, start_key: None, end_key: None, limit: None }
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> TableViewer<(u64, Duration)> for ChecksumViewer<'_, DB> {
|
||||
impl<N: NodeTypesWithDB<ChainSpec = ChainSpec>> TableViewer<(u64, Duration)>
|
||||
for ChecksumViewer<'_, N>
|
||||
{
|
||||
type Error = eyre::Report;
|
||||
|
||||
fn view<T: Table>(&self) -> Result<(u64, Duration), Self::Error> {
|
||||
|
||||
@ -5,6 +5,7 @@ use reth_db_api::{
|
||||
table::Table,
|
||||
transaction::{DbTx, DbTxMut},
|
||||
};
|
||||
use reth_node_builder::NodeTypesWithDB;
|
||||
use reth_provider::{ProviderFactory, StaticFileProviderFactory};
|
||||
use reth_static_file_types::{find_fixed_range, StaticFileSegment};
|
||||
|
||||
@ -17,7 +18,10 @@ pub struct Command {
|
||||
|
||||
impl Command {
|
||||
/// Execute `db clear` command
|
||||
pub fn execute<DB: Database>(self, provider_factory: ProviderFactory<DB>) -> eyre::Result<()> {
|
||||
pub fn execute<N: NodeTypesWithDB>(
|
||||
self,
|
||||
provider_factory: ProviderFactory<N>,
|
||||
) -> eyre::Result<()> {
|
||||
match self.subcommand {
|
||||
Subcommands::Mdbx { table } => {
|
||||
table.view(&ClearViewer { db: provider_factory.db_ref() })?
|
||||
|
||||
@ -2,6 +2,7 @@ use clap::Parser;
|
||||
use reth_db::{open_db_read_only, tables_to_generic, DatabaseEnv, Tables};
|
||||
use reth_db_api::{cursor::DbCursorRO, database::Database, table::Table, transaction::DbTx};
|
||||
use reth_db_common::DbTool;
|
||||
use reth_node_builder::{NodeTypesWithDBAdapter, NodeTypesWithEngine};
|
||||
use reth_node_core::{
|
||||
args::DatabaseArgs,
|
||||
dirs::{DataDirPath, PlatformPath},
|
||||
@ -51,7 +52,10 @@ impl Command {
|
||||
///
|
||||
/// The discrepancies and extra elements, along with a brief summary of the diff results are
|
||||
/// then written to a file in the output directory.
|
||||
pub fn execute(self, tool: &DbTool<Arc<DatabaseEnv>>) -> eyre::Result<()> {
|
||||
pub fn execute<T: NodeTypesWithEngine>(
|
||||
self,
|
||||
tool: &DbTool<NodeTypesWithDBAdapter<T, Arc<DatabaseEnv>>>,
|
||||
) -> eyre::Result<()> {
|
||||
warn!("Make sure the node is not running when running `reth db diff`!");
|
||||
// open second db
|
||||
let second_db_path: PathBuf = self.secondary_datadir.join("db").into();
|
||||
|
||||
@ -1,13 +1,12 @@
|
||||
use clap::Parser;
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_db::{
|
||||
static_file::{ColumnSelectorOne, ColumnSelectorTwo, HeaderMask, ReceiptMask, TransactionMask},
|
||||
tables, RawKey, RawTable, Receipts, TableViewer, Transactions,
|
||||
};
|
||||
use reth_db_api::{
|
||||
database::Database,
|
||||
table::{Decompress, DupSort, Table},
|
||||
};
|
||||
use reth_db_api::table::{Decompress, DupSort, Table};
|
||||
use reth_db_common::DbTool;
|
||||
use reth_node_builder::NodeTypesWithDB;
|
||||
use reth_primitives::{BlockHash, Header};
|
||||
use reth_provider::StaticFileProviderFactory;
|
||||
use reth_static_file_types::StaticFileSegment;
|
||||
@ -54,7 +53,10 @@ enum Subcommand {
|
||||
|
||||
impl Command {
|
||||
/// Execute `db get` command
|
||||
pub fn execute<DB: Database>(self, tool: &DbTool<DB>) -> eyre::Result<()> {
|
||||
pub fn execute<N: NodeTypesWithDB<ChainSpec = ChainSpec>>(
|
||||
self,
|
||||
tool: &DbTool<N>,
|
||||
) -> eyre::Result<()> {
|
||||
match self.subcommand {
|
||||
Subcommand::Mdbx { table, key, subkey, raw } => {
|
||||
table.view(&GetValueViewer { tool, key, subkey, raw })?
|
||||
@ -138,14 +140,14 @@ fn table_subkey<T: DupSort>(subkey: &Option<String>) -> Result<T::SubKey, eyre::
|
||||
.map_err(|e| eyre::eyre!(e))
|
||||
}
|
||||
|
||||
struct GetValueViewer<'a, DB: Database> {
|
||||
tool: &'a DbTool<DB>,
|
||||
struct GetValueViewer<'a, N: NodeTypesWithDB> {
|
||||
tool: &'a DbTool<N>,
|
||||
key: String,
|
||||
subkey: Option<String>,
|
||||
raw: bool,
|
||||
}
|
||||
|
||||
impl<DB: Database> TableViewer<()> for GetValueViewer<'_, DB> {
|
||||
impl<N: NodeTypesWithDB<ChainSpec = ChainSpec>> TableViewer<()> for GetValueViewer<'_, N> {
|
||||
type Error = eyre::Report;
|
||||
|
||||
fn view<T: Table>(&self) -> Result<(), Self::Error> {
|
||||
|
||||
@ -1,9 +1,11 @@
|
||||
use super::tui::DbListTUI;
|
||||
use clap::Parser;
|
||||
use eyre::WrapErr;
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_db::{DatabaseEnv, RawValue, TableViewer, Tables};
|
||||
use reth_db_api::{database::Database, table::Table};
|
||||
use reth_db_common::{DbTool, ListFilter};
|
||||
use reth_node_builder::{NodeTypesWithDBAdapter, NodeTypesWithEngine};
|
||||
use reth_primitives::hex;
|
||||
use std::{cell::RefCell, sync::Arc};
|
||||
use tracing::error;
|
||||
@ -51,7 +53,10 @@ pub struct Command {
|
||||
|
||||
impl Command {
|
||||
/// Execute `db list` command
|
||||
pub fn execute(self, tool: &DbTool<Arc<DatabaseEnv>>) -> eyre::Result<()> {
|
||||
pub fn execute<N: NodeTypesWithEngine<ChainSpec = ChainSpec>>(
|
||||
self,
|
||||
tool: &DbTool<NodeTypesWithDBAdapter<N, Arc<DatabaseEnv>>>,
|
||||
) -> eyre::Result<()> {
|
||||
self.table.view(&ListTableViewer { tool, args: &self })
|
||||
}
|
||||
|
||||
@ -81,12 +86,12 @@ impl Command {
|
||||
}
|
||||
}
|
||||
|
||||
struct ListTableViewer<'a> {
|
||||
tool: &'a DbTool<Arc<DatabaseEnv>>,
|
||||
struct ListTableViewer<'a, N: NodeTypesWithEngine> {
|
||||
tool: &'a DbTool<NodeTypesWithDBAdapter<N, Arc<DatabaseEnv>>>,
|
||||
args: &'a Command,
|
||||
}
|
||||
|
||||
impl TableViewer<()> for ListTableViewer<'_> {
|
||||
impl<N: NodeTypesWithEngine> TableViewer<()> for ListTableViewer<'_, N> {
|
||||
type Error = eyre::Report;
|
||||
|
||||
fn view<T: Table>(&self) -> Result<(), Self::Error> {
|
||||
|
||||
@ -4,6 +4,7 @@ use reth_chainspec::ChainSpec;
|
||||
use reth_cli::chainspec::ChainSpecParser;
|
||||
use reth_db::version::{get_db_version, DatabaseVersionError, DB_VERSION};
|
||||
use reth_db_common::DbTool;
|
||||
use reth_node_builder::NodeTypesWithEngine;
|
||||
use std::io::{self, Write};
|
||||
|
||||
mod checksum;
|
||||
@ -54,8 +55,8 @@ pub enum Subcommands {
|
||||
|
||||
/// `db_ro_exec` opens a database in read-only mode, and then execute with the provided command
|
||||
macro_rules! db_ro_exec {
|
||||
($env:expr, $tool:ident, $command:block) => {
|
||||
let Environment { provider_factory, .. } = $env.init(AccessRights::RO)?;
|
||||
($env:expr, $tool:ident, $N:ident, $command:block) => {
|
||||
let Environment { provider_factory, .. } = $env.init::<$N>(AccessRights::RO)?;
|
||||
|
||||
let $tool = DbTool::new(provider_factory.clone())?;
|
||||
$command;
|
||||
@ -64,7 +65,9 @@ macro_rules! db_ro_exec {
|
||||
|
||||
impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
|
||||
/// Execute `db` command
|
||||
pub async fn execute(self) -> eyre::Result<()> {
|
||||
pub async fn execute<N: NodeTypesWithEngine<ChainSpec = C::ChainSpec>>(
|
||||
self,
|
||||
) -> eyre::Result<()> {
|
||||
let data_dir = self.env.datadir.clone().resolve_datadir(self.env.chain.chain);
|
||||
let db_path = data_dir.db();
|
||||
let static_files_path = data_dir.static_files();
|
||||
@ -82,27 +85,27 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
|
||||
match self.command {
|
||||
// TODO: We'll need to add this on the DB trait.
|
||||
Subcommands::Stats(command) => {
|
||||
db_ro_exec!(self.env, tool, {
|
||||
db_ro_exec!(self.env, tool, N, {
|
||||
command.execute(data_dir, &tool)?;
|
||||
});
|
||||
}
|
||||
Subcommands::List(command) => {
|
||||
db_ro_exec!(self.env, tool, {
|
||||
db_ro_exec!(self.env, tool, N, {
|
||||
command.execute(&tool)?;
|
||||
});
|
||||
}
|
||||
Subcommands::Checksum(command) => {
|
||||
db_ro_exec!(self.env, tool, {
|
||||
db_ro_exec!(self.env, tool, N, {
|
||||
command.execute(&tool)?;
|
||||
});
|
||||
}
|
||||
Subcommands::Diff(command) => {
|
||||
db_ro_exec!(self.env, tool, {
|
||||
db_ro_exec!(self.env, tool, N, {
|
||||
command.execute(&tool)?;
|
||||
});
|
||||
}
|
||||
Subcommands::Get(command) => {
|
||||
db_ro_exec!(self.env, tool, {
|
||||
db_ro_exec!(self.env, tool, N, {
|
||||
command.execute(&tool)?;
|
||||
});
|
||||
}
|
||||
@ -122,12 +125,12 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
|
||||
}
|
||||
}
|
||||
|
||||
let Environment { provider_factory, .. } = self.env.init(AccessRights::RW)?;
|
||||
let Environment { provider_factory, .. } = self.env.init::<N>(AccessRights::RW)?;
|
||||
let tool = DbTool::new(provider_factory)?;
|
||||
tool.drop(db_path, static_files_path)?;
|
||||
}
|
||||
Subcommands::Clear(command) => {
|
||||
let Environment { provider_factory, .. } = self.env.init(AccessRights::RW)?;
|
||||
let Environment { provider_factory, .. } = self.env.init::<N>(AccessRights::RW)?;
|
||||
command.execute(provider_factory)?;
|
||||
}
|
||||
Subcommands::Version => {
|
||||
|
||||
@ -4,10 +4,12 @@ use comfy_table::{Cell, Row, Table as ComfyTable};
|
||||
use eyre::WrapErr;
|
||||
use human_bytes::human_bytes;
|
||||
use itertools::Itertools;
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_db::{mdbx, static_file::iter_static_files, DatabaseEnv, TableViewer, Tables};
|
||||
use reth_db_api::database::Database;
|
||||
use reth_db_common::DbTool;
|
||||
use reth_fs_util as fs;
|
||||
use reth_node_builder::{NodeTypesWithDB, NodeTypesWithDBAdapter, NodeTypesWithEngine};
|
||||
use reth_node_core::dirs::{ChainPath, DataDirPath};
|
||||
use reth_provider::providers::StaticFileProvider;
|
||||
use reth_static_file_types::{find_fixed_range, SegmentRangeInclusive};
|
||||
@ -36,10 +38,10 @@ pub struct Command {
|
||||
|
||||
impl Command {
|
||||
/// Execute `db stats` command
|
||||
pub fn execute(
|
||||
pub fn execute<N: NodeTypesWithEngine<ChainSpec = ChainSpec>>(
|
||||
self,
|
||||
data_dir: ChainPath<DataDirPath>,
|
||||
tool: &DbTool<Arc<DatabaseEnv>>,
|
||||
tool: &DbTool<NodeTypesWithDBAdapter<N, Arc<DatabaseEnv>>>,
|
||||
) -> eyre::Result<()> {
|
||||
if self.checksum {
|
||||
let checksum_report = self.checksum_report(tool)?;
|
||||
@ -58,7 +60,10 @@ impl Command {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn db_stats_table(&self, tool: &DbTool<Arc<DatabaseEnv>>) -> eyre::Result<ComfyTable> {
|
||||
fn db_stats_table<N: NodeTypesWithDB<DB = Arc<DatabaseEnv>>>(
|
||||
&self,
|
||||
tool: &DbTool<N>,
|
||||
) -> eyre::Result<ComfyTable> {
|
||||
let mut table = ComfyTable::new();
|
||||
table.load_preset(comfy_table::presets::ASCII_MARKDOWN);
|
||||
table.set_header([
|
||||
@ -306,7 +311,10 @@ impl Command {
|
||||
Ok(table)
|
||||
}
|
||||
|
||||
fn checksum_report(&self, tool: &DbTool<Arc<DatabaseEnv>>) -> eyre::Result<ComfyTable> {
|
||||
fn checksum_report<N: NodeTypesWithDB<ChainSpec = ChainSpec>>(
|
||||
&self,
|
||||
tool: &DbTool<N>,
|
||||
) -> eyre::Result<ComfyTable> {
|
||||
let mut table = ComfyTable::new();
|
||||
table.load_preset(comfy_table::presets::ASCII_MARKDOWN);
|
||||
table.set_header(vec![Cell::new("Table"), Cell::new("Checksum"), Cell::new("Elapsed")]);
|
||||
|
||||
@ -8,7 +8,7 @@ use reth_cli::chainspec::ChainSpecParser;
|
||||
use reth_config::Config;
|
||||
use reth_consensus::Consensus;
|
||||
use reth_db::tables;
|
||||
use reth_db_api::{database::Database, transaction::DbTx};
|
||||
use reth_db_api::transaction::DbTx;
|
||||
use reth_downloaders::{
|
||||
bodies::bodies::BodiesDownloaderBuilder,
|
||||
file_client::{ChunkedFileReader, FileClient, DEFAULT_BYTE_LEN_CHUNK_CHAIN_FILE},
|
||||
@ -19,6 +19,7 @@ use reth_network_p2p::{
|
||||
bodies::downloader::BodyDownloader,
|
||||
headers::downloader::{HeaderDownloader, SyncTarget},
|
||||
};
|
||||
use reth_node_builder::{NodeTypesWithDB, NodeTypesWithEngine};
|
||||
use reth_node_core::version::SHORT_VERSION;
|
||||
use reth_node_events::node::NodeEvent;
|
||||
use reth_primitives::B256;
|
||||
@ -57,10 +58,11 @@ pub struct ImportCommand<C: ChainSpecParser> {
|
||||
|
||||
impl<C: ChainSpecParser<ChainSpec = ChainSpec>> ImportCommand<C> {
|
||||
/// Execute `import` command
|
||||
pub async fn execute<E, F>(self, executor: F) -> eyre::Result<()>
|
||||
pub async fn execute<N, E, F>(self, executor: F) -> eyre::Result<()>
|
||||
where
|
||||
N: NodeTypesWithEngine<ChainSpec = C::ChainSpec>,
|
||||
E: BlockExecutorProvider,
|
||||
F: FnOnce(Arc<ChainSpec>) -> E,
|
||||
F: FnOnce(Arc<N::ChainSpec>) -> E,
|
||||
{
|
||||
info!(target: "reth::cli", "reth {} starting", SHORT_VERSION);
|
||||
|
||||
@ -73,7 +75,7 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> ImportCommand<C> {
|
||||
"Chunking chain import"
|
||||
);
|
||||
|
||||
let Environment { provider_factory, config, .. } = self.env.init(AccessRights::RW)?;
|
||||
let Environment { provider_factory, config, .. } = self.env.init::<N>(AccessRights::RW)?;
|
||||
|
||||
let executor = executor(provider_factory.chain_spec());
|
||||
let consensus = Arc::new(EthBeaconConsensus::new(self.env.chain.clone()));
|
||||
@ -156,17 +158,17 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> ImportCommand<C> {
|
||||
///
|
||||
/// If configured to execute, all stages will run. Otherwise, only stages that don't require state
|
||||
/// will run.
|
||||
pub fn build_import_pipeline<DB, C, E>(
|
||||
pub fn build_import_pipeline<N, C, E>(
|
||||
config: &Config,
|
||||
provider_factory: ProviderFactory<DB>,
|
||||
provider_factory: ProviderFactory<N>,
|
||||
consensus: &Arc<C>,
|
||||
file_client: Arc<FileClient>,
|
||||
static_file_producer: StaticFileProducer<DB>,
|
||||
static_file_producer: StaticFileProducer<N>,
|
||||
disable_exec: bool,
|
||||
executor: E,
|
||||
) -> eyre::Result<(Pipeline<DB>, impl Stream<Item = NodeEvent>)>
|
||||
) -> eyre::Result<(Pipeline<N>, impl Stream<Item = NodeEvent>)>
|
||||
where
|
||||
DB: Database + Clone + Unpin + 'static,
|
||||
N: NodeTypesWithDB<ChainSpec = ChainSpec>,
|
||||
C: Consensus + 'static,
|
||||
E: BlockExecutorProvider,
|
||||
{
|
||||
@ -201,7 +203,7 @@ where
|
||||
|
||||
let max_block = file_client.max_block().unwrap_or(0);
|
||||
|
||||
let pipeline = Pipeline::builder()
|
||||
let pipeline = Pipeline::<N>::builder()
|
||||
.with_tip_sender(tip_tx)
|
||||
// we want to sync all blocks the file client provides or 0 if empty
|
||||
.with_max_block(max_block)
|
||||
|
||||
@ -4,6 +4,7 @@ use crate::common::{AccessRights, Environment, EnvironmentArgs};
|
||||
use clap::Parser;
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_cli::chainspec::ChainSpecParser;
|
||||
use reth_node_builder::NodeTypesWithEngine;
|
||||
use reth_provider::BlockHashReader;
|
||||
use tracing::info;
|
||||
|
||||
@ -16,10 +17,12 @@ pub struct InitCommand<C: ChainSpecParser> {
|
||||
|
||||
impl<C: ChainSpecParser<ChainSpec = ChainSpec>> InitCommand<C> {
|
||||
/// Execute the `init` command
|
||||
pub async fn execute(self) -> eyre::Result<()> {
|
||||
pub async fn execute<N: NodeTypesWithEngine<ChainSpec = C::ChainSpec>>(
|
||||
self,
|
||||
) -> eyre::Result<()> {
|
||||
info!(target: "reth::cli", "reth init starting");
|
||||
|
||||
let Environment { provider_factory, .. } = self.env.init(AccessRights::RW)?;
|
||||
let Environment { provider_factory, .. } = self.env.init::<N>(AccessRights::RW)?;
|
||||
|
||||
let hash = provider_factory
|
||||
.block_hash(0)?
|
||||
|
||||
@ -5,8 +5,8 @@ use clap::Parser;
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_cli::chainspec::ChainSpecParser;
|
||||
use reth_config::config::EtlConfig;
|
||||
use reth_db_api::database::Database;
|
||||
use reth_db_common::init::init_from_state_dump;
|
||||
use reth_node_builder::{NodeTypesWithDB, NodeTypesWithEngine};
|
||||
use reth_primitives::B256;
|
||||
use reth_provider::ProviderFactory;
|
||||
|
||||
@ -42,10 +42,12 @@ pub struct InitStateCommand<C: ChainSpecParser> {
|
||||
|
||||
impl<C: ChainSpecParser<ChainSpec = ChainSpec>> InitStateCommand<C> {
|
||||
/// Execute the `init` command
|
||||
pub async fn execute(self) -> eyre::Result<()> {
|
||||
pub async fn execute<N: NodeTypesWithEngine<ChainSpec = C::ChainSpec>>(
|
||||
self,
|
||||
) -> eyre::Result<()> {
|
||||
info!(target: "reth::cli", "Reth init-state starting");
|
||||
|
||||
let Environment { config, provider_factory, .. } = self.env.init(AccessRights::RW)?;
|
||||
let Environment { config, provider_factory, .. } = self.env.init::<N>(AccessRights::RW)?;
|
||||
|
||||
info!(target: "reth::cli", "Initiating state dump");
|
||||
|
||||
@ -57,9 +59,9 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> InitStateCommand<C> {
|
||||
}
|
||||
|
||||
/// Initialize chain with state at specific block, from a file with state dump.
|
||||
pub fn init_at_state<DB: Database>(
|
||||
pub fn init_at_state<N: NodeTypesWithDB<ChainSpec = ChainSpec>>(
|
||||
state_dump_path: PathBuf,
|
||||
factory: ProviderFactory<DB>,
|
||||
factory: ProviderFactory<N>,
|
||||
etl_config: EtlConfig,
|
||||
) -> eyre::Result<B256> {
|
||||
info!(target: "reth::cli",
|
||||
|
||||
@ -3,6 +3,7 @@ use crate::common::{AccessRights, Environment, EnvironmentArgs};
|
||||
use clap::Parser;
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_cli::chainspec::ChainSpecParser;
|
||||
use reth_node_builder::NodeTypesWithEngine;
|
||||
use reth_prune::PrunerBuilder;
|
||||
use reth_static_file::StaticFileProducer;
|
||||
use tracing::info;
|
||||
@ -16,8 +17,10 @@ pub struct PruneCommand<C: ChainSpecParser> {
|
||||
|
||||
impl<C: ChainSpecParser<ChainSpec = ChainSpec>> PruneCommand<C> {
|
||||
/// Execute the `prune` command
|
||||
pub async fn execute(self) -> eyre::Result<()> {
|
||||
let Environment { config, provider_factory, .. } = self.env.init(AccessRights::RW)?;
|
||||
pub async fn execute<N: NodeTypesWithEngine<ChainSpec = C::ChainSpec>>(
|
||||
self,
|
||||
) -> eyre::Result<()> {
|
||||
let Environment { config, provider_factory, .. } = self.env.init::<N>(AccessRights::RW)?;
|
||||
let prune_config = config.prune.unwrap_or_default();
|
||||
|
||||
// Copy data from database to static files
|
||||
|
||||
@ -4,6 +4,7 @@ use clap::{Parser, Subcommand};
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_cli::chainspec::ChainSpecParser;
|
||||
use reth_cli_runner::CliContext;
|
||||
use reth_node_builder::NodeTypesWithEngine;
|
||||
|
||||
mod storage_tries;
|
||||
|
||||
@ -23,9 +24,12 @@ pub enum Subcommands<C: ChainSpecParser> {
|
||||
|
||||
impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
|
||||
/// Execute `recover` command
|
||||
pub async fn execute(self, ctx: CliContext) -> eyre::Result<()> {
|
||||
pub async fn execute<N: NodeTypesWithEngine<ChainSpec = C::ChainSpec>>(
|
||||
self,
|
||||
ctx: CliContext,
|
||||
) -> eyre::Result<()> {
|
||||
match self.command {
|
||||
Subcommands::StorageTries(command) => command.execute(ctx).await,
|
||||
Subcommands::StorageTries(command) => command.execute::<N>(ctx).await,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -8,6 +8,7 @@ use reth_db_api::{
|
||||
cursor::{DbCursorRO, DbDupCursorRW},
|
||||
transaction::DbTx,
|
||||
};
|
||||
use reth_node_builder::NodeTypesWithEngine;
|
||||
use reth_provider::{BlockNumReader, HeaderProvider, ProviderError};
|
||||
use reth_trie::StateRoot;
|
||||
use reth_trie_db::DatabaseStateRoot;
|
||||
@ -22,8 +23,11 @@ pub struct Command<C: ChainSpecParser> {
|
||||
|
||||
impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
|
||||
/// Execute `storage-tries` recovery command
|
||||
pub async fn execute(self, _ctx: CliContext) -> eyre::Result<()> {
|
||||
let Environment { provider_factory, .. } = self.env.init(AccessRights::RW)?;
|
||||
pub async fn execute<N: NodeTypesWithEngine<ChainSpec = C::ChainSpec>>(
|
||||
self,
|
||||
_ctx: CliContext,
|
||||
) -> eyre::Result<()> {
|
||||
let Environment { provider_factory, .. } = self.env.init::<N>(AccessRights::RW)?;
|
||||
|
||||
let mut provider = provider_factory.provider_rw()?;
|
||||
let best_block = provider.best_block_number()?;
|
||||
|
||||
@ -10,6 +10,7 @@ use reth_db_common::{
|
||||
init::{insert_genesis_header, insert_genesis_history, insert_genesis_state},
|
||||
DbTool,
|
||||
};
|
||||
use reth_node_builder::NodeTypesWithEngine;
|
||||
use reth_node_core::args::StageEnum;
|
||||
use reth_provider::{writer::UnifiedStorageWriter, StaticFileProviderFactory};
|
||||
use reth_stages::StageId;
|
||||
@ -26,8 +27,10 @@ pub struct Command<C: ChainSpecParser> {
|
||||
|
||||
impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
|
||||
/// Execute `db` command
|
||||
pub async fn execute(self) -> eyre::Result<()> {
|
||||
let Environment { provider_factory, .. } = self.env.init(AccessRights::RW)?;
|
||||
pub async fn execute<N: NodeTypesWithEngine<ChainSpec = C::ChainSpec>>(
|
||||
self,
|
||||
) -> eyre::Result<()> {
|
||||
let Environment { provider_factory, .. } = self.env.init::<N>(AccessRights::RW)?;
|
||||
|
||||
let static_file_provider = provider_factory.static_file_provider();
|
||||
|
||||
|
||||
@ -1,17 +1,21 @@
|
||||
use std::sync::Arc;
|
||||
|
||||
use super::setup;
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_db::{tables, DatabaseEnv};
|
||||
use reth_db_api::{
|
||||
cursor::DbCursorRO, database::Database, table::TableImporter, transaction::DbTx,
|
||||
};
|
||||
use reth_db_common::DbTool;
|
||||
use reth_evm::{execute::BlockExecutorProvider, noop::NoopBlockExecutorProvider};
|
||||
use reth_node_builder::{NodeTypesWithDB, NodeTypesWithDBAdapter};
|
||||
use reth_node_core::dirs::{ChainPath, DataDirPath};
|
||||
use reth_provider::{providers::StaticFileProvider, ProviderFactory};
|
||||
use reth_stages::{stages::ExecutionStage, Stage, StageCheckpoint, UnwindInput};
|
||||
use tracing::info;
|
||||
|
||||
pub(crate) async fn dump_execution_stage<DB, E>(
|
||||
db_tool: &DbTool<DB>,
|
||||
pub(crate) async fn dump_execution_stage<N, E>(
|
||||
db_tool: &DbTool<N>,
|
||||
from: u64,
|
||||
to: u64,
|
||||
output_datadir: ChainPath<DataDirPath>,
|
||||
@ -19,7 +23,7 @@ pub(crate) async fn dump_execution_stage<DB, E>(
|
||||
executor: E,
|
||||
) -> eyre::Result<()>
|
||||
where
|
||||
DB: Database,
|
||||
N: NodeTypesWithDB<ChainSpec = ChainSpec>,
|
||||
E: BlockExecutorProvider,
|
||||
{
|
||||
let (output_db, tip_block_number) = setup(from, to, &output_datadir.db(), db_tool)?;
|
||||
@ -30,8 +34,8 @@ where
|
||||
|
||||
if should_run {
|
||||
dry_run(
|
||||
ProviderFactory::new(
|
||||
output_db,
|
||||
ProviderFactory::<NodeTypesWithDBAdapter<N, Arc<DatabaseEnv>>>::new(
|
||||
Arc::new(output_db),
|
||||
db_tool.chain(),
|
||||
StaticFileProvider::read_write(output_datadir.static_files())?,
|
||||
),
|
||||
@ -45,9 +49,9 @@ where
|
||||
}
|
||||
|
||||
/// Imports all the tables that can be copied over a range.
|
||||
fn import_tables_with_range<DB: Database>(
|
||||
fn import_tables_with_range<N: NodeTypesWithDB>(
|
||||
output_db: &DatabaseEnv,
|
||||
db_tool: &DbTool<DB>,
|
||||
db_tool: &DbTool<N>,
|
||||
from: u64,
|
||||
to: u64,
|
||||
) -> eyre::Result<()> {
|
||||
@ -125,8 +129,8 @@ fn import_tables_with_range<DB: Database>(
|
||||
/// Dry-run an unwind to FROM block, so we can get the `PlainStorageState` and
|
||||
/// `PlainAccountState` safely. There might be some state dependency from an address
|
||||
/// which hasn't been changed in the given range.
|
||||
fn unwind_and_copy<DB: Database>(
|
||||
db_tool: &DbTool<DB>,
|
||||
fn unwind_and_copy<N: NodeTypesWithDB<ChainSpec = ChainSpec>>(
|
||||
db_tool: &DbTool<N>,
|
||||
from: u64,
|
||||
tip_block_number: u64,
|
||||
output_db: &DatabaseEnv,
|
||||
@ -155,14 +159,14 @@ fn unwind_and_copy<DB: Database>(
|
||||
}
|
||||
|
||||
/// Try to re-execute the stage without committing
|
||||
fn dry_run<DB, E>(
|
||||
output_provider_factory: ProviderFactory<DB>,
|
||||
fn dry_run<N, E>(
|
||||
output_provider_factory: ProviderFactory<N>,
|
||||
to: u64,
|
||||
from: u64,
|
||||
executor: E,
|
||||
) -> eyre::Result<()>
|
||||
where
|
||||
DB: Database,
|
||||
N: NodeTypesWithDB<ChainSpec = ChainSpec>,
|
||||
E: BlockExecutorProvider,
|
||||
{
|
||||
info!(target: "reth::cli", "Executing stage. [dry-run]");
|
||||
|
||||
@ -1,16 +1,20 @@
|
||||
use std::sync::Arc;
|
||||
|
||||
use super::setup;
|
||||
use eyre::Result;
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_db::{tables, DatabaseEnv};
|
||||
use reth_db_api::{database::Database, table::TableImporter};
|
||||
use reth_db_common::DbTool;
|
||||
use reth_node_builder::{NodeTypesWithDB, NodeTypesWithDBAdapter};
|
||||
use reth_node_core::dirs::{ChainPath, DataDirPath};
|
||||
use reth_primitives::BlockNumber;
|
||||
use reth_provider::{providers::StaticFileProvider, ProviderFactory};
|
||||
use reth_stages::{stages::AccountHashingStage, Stage, StageCheckpoint, UnwindInput};
|
||||
use tracing::info;
|
||||
|
||||
pub(crate) async fn dump_hashing_account_stage<DB: Database>(
|
||||
db_tool: &DbTool<DB>,
|
||||
pub(crate) async fn dump_hashing_account_stage<N: NodeTypesWithDB<ChainSpec = ChainSpec>>(
|
||||
db_tool: &DbTool<N>,
|
||||
from: BlockNumber,
|
||||
to: BlockNumber,
|
||||
output_datadir: ChainPath<DataDirPath>,
|
||||
@ -31,8 +35,8 @@ pub(crate) async fn dump_hashing_account_stage<DB: Database>(
|
||||
|
||||
if should_run {
|
||||
dry_run(
|
||||
ProviderFactory::new(
|
||||
output_db,
|
||||
ProviderFactory::<NodeTypesWithDBAdapter<N, Arc<DatabaseEnv>>>::new(
|
||||
Arc::new(output_db),
|
||||
db_tool.chain(),
|
||||
StaticFileProvider::read_write(output_datadir.static_files())?,
|
||||
),
|
||||
@ -45,8 +49,8 @@ pub(crate) async fn dump_hashing_account_stage<DB: Database>(
|
||||
}
|
||||
|
||||
/// Dry-run an unwind to FROM block and copy the necessary table data to the new database.
|
||||
fn unwind_and_copy<DB: Database>(
|
||||
db_tool: &DbTool<DB>,
|
||||
fn unwind_and_copy<N: NodeTypesWithDB<ChainSpec = ChainSpec>>(
|
||||
db_tool: &DbTool<N>,
|
||||
from: u64,
|
||||
tip_block_number: u64,
|
||||
output_db: &DatabaseEnv,
|
||||
@ -70,8 +74,8 @@ fn unwind_and_copy<DB: Database>(
|
||||
}
|
||||
|
||||
/// Try to re-execute the stage straight away
|
||||
fn dry_run<DB: Database>(
|
||||
output_provider_factory: ProviderFactory<DB>,
|
||||
fn dry_run<N: NodeTypesWithDB<ChainSpec = ChainSpec>>(
|
||||
output_provider_factory: ProviderFactory<N>,
|
||||
to: u64,
|
||||
from: u64,
|
||||
) -> eyre::Result<()> {
|
||||
|
||||
@ -1,15 +1,19 @@
|
||||
use std::sync::Arc;
|
||||
|
||||
use super::setup;
|
||||
use eyre::Result;
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_db::{tables, DatabaseEnv};
|
||||
use reth_db_api::{database::Database, table::TableImporter};
|
||||
use reth_db_common::DbTool;
|
||||
use reth_node_builder::{NodeTypesWithDB, NodeTypesWithDBAdapter};
|
||||
use reth_node_core::dirs::{ChainPath, DataDirPath};
|
||||
use reth_provider::{providers::StaticFileProvider, ProviderFactory};
|
||||
use reth_stages::{stages::StorageHashingStage, Stage, StageCheckpoint, UnwindInput};
|
||||
use tracing::info;
|
||||
|
||||
pub(crate) async fn dump_hashing_storage_stage<DB: Database>(
|
||||
db_tool: &DbTool<DB>,
|
||||
pub(crate) async fn dump_hashing_storage_stage<N: NodeTypesWithDB<ChainSpec = ChainSpec>>(
|
||||
db_tool: &DbTool<N>,
|
||||
from: u64,
|
||||
to: u64,
|
||||
output_datadir: ChainPath<DataDirPath>,
|
||||
@ -21,8 +25,8 @@ pub(crate) async fn dump_hashing_storage_stage<DB: Database>(
|
||||
|
||||
if should_run {
|
||||
dry_run(
|
||||
ProviderFactory::new(
|
||||
output_db,
|
||||
ProviderFactory::<NodeTypesWithDBAdapter<N, Arc<DatabaseEnv>>>::new(
|
||||
Arc::new(output_db),
|
||||
db_tool.chain(),
|
||||
StaticFileProvider::read_write(output_datadir.static_files())?,
|
||||
),
|
||||
@ -35,8 +39,8 @@ pub(crate) async fn dump_hashing_storage_stage<DB: Database>(
|
||||
}
|
||||
|
||||
/// Dry-run an unwind to FROM block and copy the necessary table data to the new database.
|
||||
fn unwind_and_copy<DB: Database>(
|
||||
db_tool: &DbTool<DB>,
|
||||
fn unwind_and_copy<N: NodeTypesWithDB<ChainSpec = ChainSpec>>(
|
||||
db_tool: &DbTool<N>,
|
||||
from: u64,
|
||||
tip_block_number: u64,
|
||||
output_db: &DatabaseEnv,
|
||||
@ -65,8 +69,8 @@ fn unwind_and_copy<DB: Database>(
|
||||
}
|
||||
|
||||
/// Try to re-execute the stage straight away
|
||||
fn dry_run<DB: Database>(
|
||||
output_provider_factory: ProviderFactory<DB>,
|
||||
fn dry_run<N: NodeTypesWithDB<ChainSpec = ChainSpec>>(
|
||||
output_provider_factory: ProviderFactory<N>,
|
||||
to: u64,
|
||||
from: u64,
|
||||
) -> eyre::Result<()> {
|
||||
|
||||
@ -1,11 +1,15 @@
|
||||
use std::sync::Arc;
|
||||
|
||||
use super::setup;
|
||||
use eyre::Result;
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_config::config::EtlConfig;
|
||||
use reth_db::{tables, DatabaseEnv};
|
||||
use reth_db_api::{database::Database, table::TableImporter};
|
||||
use reth_db_common::DbTool;
|
||||
use reth_evm::noop::NoopBlockExecutorProvider;
|
||||
use reth_exex::ExExManagerHandle;
|
||||
use reth_node_builder::{NodeTypesWithDB, NodeTypesWithDBAdapter};
|
||||
use reth_node_core::dirs::{ChainPath, DataDirPath};
|
||||
use reth_primitives::BlockNumber;
|
||||
use reth_provider::{providers::StaticFileProvider, ProviderFactory};
|
||||
@ -19,8 +23,8 @@ use reth_stages::{
|
||||
};
|
||||
use tracing::info;
|
||||
|
||||
pub(crate) async fn dump_merkle_stage<DB: Database>(
|
||||
db_tool: &DbTool<DB>,
|
||||
pub(crate) async fn dump_merkle_stage<N: NodeTypesWithDB<ChainSpec = ChainSpec>>(
|
||||
db_tool: &DbTool<N>,
|
||||
from: BlockNumber,
|
||||
to: BlockNumber,
|
||||
output_datadir: ChainPath<DataDirPath>,
|
||||
@ -48,8 +52,8 @@ pub(crate) async fn dump_merkle_stage<DB: Database>(
|
||||
|
||||
if should_run {
|
||||
dry_run(
|
||||
ProviderFactory::new(
|
||||
output_db,
|
||||
ProviderFactory::<NodeTypesWithDBAdapter<N, Arc<DatabaseEnv>>>::new(
|
||||
Arc::new(output_db),
|
||||
db_tool.chain(),
|
||||
StaticFileProvider::read_write(output_datadir.static_files())?,
|
||||
),
|
||||
@ -62,8 +66,8 @@ pub(crate) async fn dump_merkle_stage<DB: Database>(
|
||||
}
|
||||
|
||||
/// Dry-run an unwind to FROM block and copy the necessary table data to the new database.
|
||||
fn unwind_and_copy<DB: Database>(
|
||||
db_tool: &DbTool<DB>,
|
||||
fn unwind_and_copy<N: NodeTypesWithDB<ChainSpec = ChainSpec>>(
|
||||
db_tool: &DbTool<N>,
|
||||
range: (u64, u64),
|
||||
tip_block_number: u64,
|
||||
output_db: &DatabaseEnv,
|
||||
@ -140,8 +144,8 @@ fn unwind_and_copy<DB: Database>(
|
||||
}
|
||||
|
||||
/// Try to re-execute the stage straight away
|
||||
fn dry_run<DB: Database>(
|
||||
output_provider_factory: ProviderFactory<DB>,
|
||||
fn dry_run<N: NodeTypesWithDB<ChainSpec = ChainSpec>>(
|
||||
output_provider_factory: ProviderFactory<N>,
|
||||
to: u64,
|
||||
from: u64,
|
||||
) -> eyre::Result<()> {
|
||||
|
||||
@ -10,6 +10,7 @@ use reth_db_api::{
|
||||
};
|
||||
use reth_db_common::DbTool;
|
||||
use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_node_builder::{NodeTypesWithDB, NodeTypesWithEngine};
|
||||
use reth_node_core::{
|
||||
args::DatadirArgs,
|
||||
dirs::{DataDirPath, PlatformPath},
|
||||
@ -87,12 +88,13 @@ macro_rules! handle_stage {
|
||||
|
||||
impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
|
||||
/// Execute `dump-stage` command
|
||||
pub async fn execute<E, F>(self, executor: F) -> eyre::Result<()>
|
||||
pub async fn execute<N, E, F>(self, executor: F) -> eyre::Result<()>
|
||||
where
|
||||
N: NodeTypesWithEngine<ChainSpec = C::ChainSpec>,
|
||||
E: BlockExecutorProvider,
|
||||
F: FnOnce(Arc<ChainSpec>) -> E,
|
||||
{
|
||||
let Environment { provider_factory, .. } = self.env.init(AccessRights::RO)?;
|
||||
let Environment { provider_factory, .. } = self.env.init::<N>(AccessRights::RO)?;
|
||||
let tool = DbTool::new(provider_factory)?;
|
||||
|
||||
match &self.command {
|
||||
@ -111,11 +113,11 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
|
||||
|
||||
/// Sets up the database and initial state on [`tables::BlockBodyIndices`]. Also returns the tip
|
||||
/// block number.
|
||||
pub(crate) fn setup<DB: Database>(
|
||||
pub(crate) fn setup<N: NodeTypesWithDB>(
|
||||
from: u64,
|
||||
to: u64,
|
||||
output_db: &PathBuf,
|
||||
db_tool: &DbTool<DB>,
|
||||
db_tool: &DbTool<N>,
|
||||
) -> eyre::Result<(DatabaseEnv, u64)> {
|
||||
assert!(from < to, "FROM block should be bigger than TO block.");
|
||||
|
||||
|
||||
@ -7,6 +7,7 @@ use reth_chainspec::ChainSpec;
|
||||
use reth_cli::chainspec::ChainSpecParser;
|
||||
use reth_cli_runner::CliContext;
|
||||
use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_node_builder::NodeTypesWithEngine;
|
||||
|
||||
pub mod drop;
|
||||
pub mod dump;
|
||||
@ -40,16 +41,17 @@ pub enum Subcommands<C: ChainSpecParser> {
|
||||
|
||||
impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
|
||||
/// Execute `stage` command
|
||||
pub async fn execute<E, F>(self, ctx: CliContext, executor: F) -> eyre::Result<()>
|
||||
pub async fn execute<N, E, F>(self, ctx: CliContext, executor: F) -> eyre::Result<()>
|
||||
where
|
||||
N: NodeTypesWithEngine<ChainSpec = C::ChainSpec>,
|
||||
E: BlockExecutorProvider,
|
||||
F: FnOnce(Arc<ChainSpec>) -> E,
|
||||
{
|
||||
match self.command {
|
||||
Subcommands::Run(command) => command.execute(ctx, executor).await,
|
||||
Subcommands::Drop(command) => command.execute().await,
|
||||
Subcommands::Dump(command) => command.execute(executor).await,
|
||||
Subcommands::Unwind(command) => command.execute().await,
|
||||
Subcommands::Run(command) => command.execute::<N, _, _>(ctx, executor).await,
|
||||
Subcommands::Drop(command) => command.execute::<N>().await,
|
||||
Subcommands::Dump(command) => command.execute::<N, _, _>(executor).await,
|
||||
Subcommands::Unwind(command) => command.execute::<N>().await,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -18,6 +18,7 @@ use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_exex::ExExManagerHandle;
|
||||
use reth_network::BlockDownloaderProvider;
|
||||
use reth_network_p2p::HeadersClient;
|
||||
use reth_node_builder::NodeTypesWithEngine;
|
||||
use reth_node_core::{
|
||||
args::{NetworkArgs, StageEnum},
|
||||
primitives::BlockHashOrNumber,
|
||||
@ -102,8 +103,9 @@ pub struct Command<C: ChainSpecParser> {
|
||||
|
||||
impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
|
||||
/// Execute `stage` command
|
||||
pub async fn execute<E, F>(self, ctx: CliContext, executor: F) -> eyre::Result<()>
|
||||
pub async fn execute<N, E, F>(self, ctx: CliContext, executor: F) -> eyre::Result<()>
|
||||
where
|
||||
N: NodeTypesWithEngine<ChainSpec = C::ChainSpec>,
|
||||
E: BlockExecutorProvider,
|
||||
F: FnOnce(Arc<ChainSpec>) -> E,
|
||||
{
|
||||
@ -111,7 +113,8 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
|
||||
// Does not do anything on windows.
|
||||
let _ = fdlimit::raise_fd_limit();
|
||||
|
||||
let Environment { provider_factory, config, data_dir } = self.env.init(AccessRights::RW)?;
|
||||
let Environment { provider_factory, config, data_dir } =
|
||||
self.env.init::<N>(AccessRights::RW)?;
|
||||
|
||||
let mut provider_rw = provider_factory.provider_rw()?;
|
||||
|
||||
|
||||
@ -7,10 +7,11 @@ use reth_chainspec::ChainSpec;
|
||||
use reth_cli::chainspec::ChainSpecParser;
|
||||
use reth_config::Config;
|
||||
use reth_consensus::Consensus;
|
||||
use reth_db_api::database::Database;
|
||||
use reth_db::DatabaseEnv;
|
||||
use reth_downloaders::{bodies::noop::NoopBodiesDownloader, headers::noop::NoopHeaderDownloader};
|
||||
use reth_evm::noop::NoopBlockExecutorProvider;
|
||||
use reth_exex::ExExManagerHandle;
|
||||
use reth_node_builder::{NodeTypesWithDB, NodeTypesWithEngine};
|
||||
use reth_node_core::args::NetworkArgs;
|
||||
use reth_primitives::{BlockHashOrNumber, BlockNumber, B256};
|
||||
use reth_provider::{
|
||||
@ -48,8 +49,10 @@ pub struct Command<C: ChainSpecParser> {
|
||||
|
||||
impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
|
||||
/// Execute `db stage unwind` command
|
||||
pub async fn execute(self) -> eyre::Result<()> {
|
||||
let Environment { provider_factory, config, .. } = self.env.init(AccessRights::RW)?;
|
||||
pub async fn execute<N: NodeTypesWithEngine<ChainSpec = C::ChainSpec>>(
|
||||
self,
|
||||
) -> eyre::Result<()> {
|
||||
let Environment { provider_factory, config, .. } = self.env.init::<N>(AccessRights::RW)?;
|
||||
|
||||
let range = self.command.unwind_range(provider_factory.clone())?;
|
||||
if *range.start() == 0 {
|
||||
@ -112,11 +115,11 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn build_pipeline<DB: Database + 'static>(
|
||||
fn build_pipeline<N: NodeTypesWithDB<ChainSpec = C::ChainSpec>>(
|
||||
self,
|
||||
config: Config,
|
||||
provider_factory: ProviderFactory<Arc<DB>>,
|
||||
) -> Result<Pipeline<Arc<DB>>, eyre::Error> {
|
||||
provider_factory: ProviderFactory<N>,
|
||||
) -> Result<Pipeline<N>, eyre::Error> {
|
||||
let consensus: Arc<dyn Consensus> =
|
||||
Arc::new(EthBeaconConsensus::new(provider_factory.chain_spec()));
|
||||
let stage_conf = &config.stages;
|
||||
@ -128,13 +131,13 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
|
||||
let executor = NoopBlockExecutorProvider::default();
|
||||
|
||||
let builder = if self.offline {
|
||||
Pipeline::builder().add_stages(
|
||||
Pipeline::<N>::builder().add_stages(
|
||||
OfflineStages::new(executor, config.stages, PruneModes::default())
|
||||
.builder()
|
||||
.disable(reth_stages::StageId::SenderRecovery),
|
||||
)
|
||||
} else {
|
||||
Pipeline::builder().with_tip_sender(tip_tx).add_stages(
|
||||
Pipeline::<N>::builder().with_tip_sender(tip_tx).add_stages(
|
||||
DefaultStages::new(
|
||||
provider_factory.clone(),
|
||||
tip_rx,
|
||||
@ -185,9 +188,9 @@ impl Subcommands {
|
||||
/// Returns the block range to unwind.
|
||||
///
|
||||
/// This returns an inclusive range: [target..=latest]
|
||||
fn unwind_range<DB: Database>(
|
||||
fn unwind_range<N: NodeTypesWithDB<ChainSpec = ChainSpec, DB = Arc<DatabaseEnv>>>(
|
||||
&self,
|
||||
factory: ProviderFactory<DB>,
|
||||
factory: ProviderFactory<N>,
|
||||
) -> eyre::Result<RangeInclusive<u64>> {
|
||||
let provider = factory.provider()?;
|
||||
let last = provider.last_block_number()?;
|
||||
|
||||
@ -12,13 +12,11 @@ workspace = true
|
||||
|
||||
[dependencies]
|
||||
# reth
|
||||
reth-chainspec.workspace = true
|
||||
reth-ethereum-consensus.workspace = true
|
||||
reth-blockchain-tree-api.workspace = true
|
||||
reth-primitives.workspace = true
|
||||
reth-stages-api.workspace = true
|
||||
reth-errors.workspace = true
|
||||
reth-db-api.workspace = true
|
||||
reth-provider.workspace = true
|
||||
reth-rpc-types.workspace = true
|
||||
reth-tasks.workspace = true
|
||||
@ -30,7 +28,7 @@ reth-static-file.workspace = true
|
||||
reth-tokio-util.workspace = true
|
||||
reth-engine-primitives.workspace = true
|
||||
reth-network-p2p.workspace = true
|
||||
|
||||
reth-node-types.workspace = true
|
||||
|
||||
# async
|
||||
tokio = { workspace = true, features = ["sync"] }
|
||||
@ -55,6 +53,7 @@ reth-consensus = { workspace = true, features = ["test-utils"] }
|
||||
reth-stages = { workspace = true, features = ["test-utils"] }
|
||||
reth-blockchain-tree = { workspace = true, features = ["test-utils"] }
|
||||
reth-db = { workspace = true, features = ["test-utils"] }
|
||||
reth-db-api.workspace = true
|
||||
reth-provider = { workspace = true, features = ["test-utils"] }
|
||||
reth-evm = { workspace = true, features = ["test-utils"] }
|
||||
reth-network-p2p = { workspace = true, features = ["test-utils"] }
|
||||
@ -67,6 +66,7 @@ reth-config.workspace = true
|
||||
reth-testing-utils.workspace = true
|
||||
reth-exex-types.workspace = true
|
||||
reth-prune-types.workspace = true
|
||||
reth-chainspec.workspace = true
|
||||
alloy-genesis.workspace = true
|
||||
|
||||
assert_matches.workspace = true
|
||||
|
||||
@ -6,14 +6,14 @@ use crate::{
|
||||
};
|
||||
use futures::FutureExt;
|
||||
use metrics::Counter;
|
||||
use reth_db_api::database::Database;
|
||||
use reth_errors::{RethError, RethResult};
|
||||
use reth_node_types::NodeTypesWithDB;
|
||||
use reth_primitives::BlockNumber;
|
||||
use reth_provider::ProviderFactory;
|
||||
use reth_provider::{providers::ProviderNodeTypes, ProviderFactory};
|
||||
use reth_prune::{Pruner, PrunerError, PrunerWithResult};
|
||||
use reth_tasks::TaskSpawner;
|
||||
use std::{
|
||||
fmt,
|
||||
fmt::{self, Debug},
|
||||
task::{ready, Context, Poll},
|
||||
};
|
||||
use tokio::sync::oneshot;
|
||||
@ -21,15 +21,15 @@ use tokio::sync::oneshot;
|
||||
/// Manages pruning under the control of the engine.
|
||||
///
|
||||
/// This type controls the [Pruner].
|
||||
pub struct PruneHook<DB> {
|
||||
pub struct PruneHook<N: NodeTypesWithDB> {
|
||||
/// The current state of the pruner.
|
||||
pruner_state: PrunerState<DB>,
|
||||
pruner_state: PrunerState<N>,
|
||||
/// The type that can spawn the pruner task.
|
||||
pruner_task_spawner: Box<dyn TaskSpawner>,
|
||||
metrics: Metrics,
|
||||
}
|
||||
|
||||
impl<DB: fmt::Debug> fmt::Debug for PruneHook<DB> {
|
||||
impl<N: NodeTypesWithDB> fmt::Debug for PruneHook<N> {
|
||||
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
|
||||
f.debug_struct("PruneHook")
|
||||
.field("pruner_state", &self.pruner_state)
|
||||
@ -38,10 +38,10 @@ impl<DB: fmt::Debug> fmt::Debug for PruneHook<DB> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database + 'static> PruneHook<DB> {
|
||||
impl<N: ProviderNodeTypes> PruneHook<N> {
|
||||
/// Create a new instance
|
||||
pub fn new(
|
||||
pruner: Pruner<DB, ProviderFactory<DB>>,
|
||||
pruner: Pruner<N::DB, ProviderFactory<N>>,
|
||||
pruner_task_spawner: Box<dyn TaskSpawner>,
|
||||
) -> Self {
|
||||
Self {
|
||||
@ -117,7 +117,7 @@ impl<DB: Database + 'static> PruneHook<DB> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database + 'static> EngineHook for PruneHook<DB> {
|
||||
impl<N: ProviderNodeTypes> EngineHook for PruneHook<N> {
|
||||
fn name(&self) -> &'static str {
|
||||
"Prune"
|
||||
}
|
||||
@ -152,12 +152,23 @@ impl<DB: Database + 'static> EngineHook for PruneHook<DB> {
|
||||
/// running, it acquires the write lock over the database. This means that we cannot forward to the
|
||||
/// blockchain tree any messages that would result in database writes, since it would result in a
|
||||
/// deadlock.
|
||||
#[derive(Debug)]
|
||||
enum PrunerState<DB> {
|
||||
enum PrunerState<N: NodeTypesWithDB> {
|
||||
/// Pruner is idle.
|
||||
Idle(Option<Pruner<DB, ProviderFactory<DB>>>),
|
||||
Idle(Option<Pruner<N::DB, ProviderFactory<N>>>),
|
||||
/// Pruner is running and waiting for a response
|
||||
Running(oneshot::Receiver<PrunerWithResult<DB, ProviderFactory<DB>>>),
|
||||
Running(oneshot::Receiver<PrunerWithResult<N::DB, ProviderFactory<N>>>),
|
||||
}
|
||||
|
||||
impl<N> fmt::Debug for PrunerState<N>
|
||||
where
|
||||
N: NodeTypesWithDB<DB: Debug, ChainSpec: Debug>,
|
||||
{
|
||||
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
|
||||
match self {
|
||||
Self::Idle(f0) => f.debug_tuple("Idle").field(&f0).finish(),
|
||||
Self::Running(f0) => f.debug_tuple("Running").field(&f0).finish(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(reth_metrics::Metrics)]
|
||||
|
||||
@ -5,9 +5,10 @@ use crate::{
|
||||
hooks::EngineHookDBAccessLevel,
|
||||
};
|
||||
use futures::FutureExt;
|
||||
use reth_db_api::database::Database;
|
||||
use reth_errors::RethResult;
|
||||
use reth_node_types::NodeTypesWithDB;
|
||||
use reth_primitives::{static_file::HighestStaticFiles, BlockNumber};
|
||||
use reth_provider::providers::ProviderNodeTypes;
|
||||
use reth_static_file::{StaticFileProducer, StaticFileProducerWithResult};
|
||||
use reth_tasks::TaskSpawner;
|
||||
use std::task::{ready, Context, Poll};
|
||||
@ -18,17 +19,17 @@ use tracing::trace;
|
||||
///
|
||||
/// This type controls the [`StaticFileProducer`].
|
||||
#[derive(Debug)]
|
||||
pub struct StaticFileHook<DB> {
|
||||
pub struct StaticFileHook<N: NodeTypesWithDB> {
|
||||
/// The current state of the `static_file_producer`.
|
||||
state: StaticFileProducerState<DB>,
|
||||
state: StaticFileProducerState<N>,
|
||||
/// The type that can spawn the `static_file_producer` task.
|
||||
task_spawner: Box<dyn TaskSpawner>,
|
||||
}
|
||||
|
||||
impl<DB: Database + 'static> StaticFileHook<DB> {
|
||||
impl<N: ProviderNodeTypes> StaticFileHook<N> {
|
||||
/// Create a new instance
|
||||
pub fn new(
|
||||
static_file_producer: StaticFileProducer<DB>,
|
||||
static_file_producer: StaticFileProducer<N>,
|
||||
task_spawner: Box<dyn TaskSpawner>,
|
||||
) -> Self {
|
||||
Self { state: StaticFileProducerState::Idle(Some(static_file_producer)), task_spawner }
|
||||
@ -126,7 +127,7 @@ impl<DB: Database + 'static> StaticFileHook<DB> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database + 'static> EngineHook for StaticFileHook<DB> {
|
||||
impl<N: ProviderNodeTypes> EngineHook for StaticFileHook<N> {
|
||||
fn name(&self) -> &'static str {
|
||||
"StaticFile"
|
||||
}
|
||||
@ -162,9 +163,9 @@ impl<DB: Database + 'static> EngineHook for StaticFileHook<DB> {
|
||||
/// [`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> {
|
||||
enum StaticFileProducerState<N: NodeTypesWithDB> {
|
||||
/// [`StaticFileProducer`] is idle.
|
||||
Idle(Option<StaticFileProducer<DB>>),
|
||||
Idle(Option<StaticFileProducer<N>>),
|
||||
/// [`StaticFileProducer`] is running and waiting for a response
|
||||
Running(oneshot::Receiver<StaticFileProducerWithResult<DB>>),
|
||||
Running(oneshot::Receiver<StaticFileProducerWithResult<N>>),
|
||||
}
|
||||
|
||||
@ -4,14 +4,13 @@ use reth_blockchain_tree_api::{
|
||||
error::{BlockchainTreeError, CanonicalError, InsertBlockError, InsertBlockErrorKind},
|
||||
BlockStatus, BlockValidationKind, BlockchainTreeEngine, CanonicalOutcome, InsertPayloadOk,
|
||||
};
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_db_api::database::Database;
|
||||
use reth_engine_primitives::EngineTypes;
|
||||
use reth_engine_primitives::{EngineTypes, PayloadTypes};
|
||||
use reth_errors::{BlockValidationError, ProviderResult, RethError, RethResult};
|
||||
use reth_network_p2p::{
|
||||
sync::{NetworkSyncUpdater, SyncState},
|
||||
BlockClient,
|
||||
};
|
||||
use reth_node_types::NodeTypesWithDB;
|
||||
use reth_payload_builder::PayloadBuilderHandle;
|
||||
use reth_payload_primitives::{PayloadAttributes, PayloadBuilderAttributes};
|
||||
use reth_payload_validator::ExecutionPayloadValidator;
|
||||
@ -20,8 +19,8 @@ use reth_primitives::{
|
||||
B256,
|
||||
};
|
||||
use reth_provider::{
|
||||
BlockIdReader, BlockReader, BlockSource, CanonChainTracker, ChainSpecProvider, ProviderError,
|
||||
StageCheckpointReader,
|
||||
providers::ProviderNodeTypes, BlockIdReader, BlockReader, BlockSource, CanonChainTracker,
|
||||
ChainSpecProvider, ProviderError, StageCheckpointReader,
|
||||
};
|
||||
use reth_rpc_types::engine::{
|
||||
CancunPayloadFields, ExecutionPayload, ForkchoiceState, PayloadStatus, PayloadStatusEnum,
|
||||
@ -168,40 +167,40 @@ type PendingForkchoiceUpdate<PayloadAttributes> =
|
||||
/// If the future is polled more than once. Leads to undefined state.
|
||||
#[must_use = "Future does nothing unless polled"]
|
||||
#[allow(missing_debug_implementations)]
|
||||
pub struct BeaconConsensusEngine<DB, BT, Client, EngineT>
|
||||
pub struct BeaconConsensusEngine<N, BT, Client>
|
||||
where
|
||||
DB: Database,
|
||||
N: NodeTypesWithDB,
|
||||
Client: BlockClient,
|
||||
BT: BlockchainTreeEngine
|
||||
+ BlockReader
|
||||
+ BlockIdReader
|
||||
+ CanonChainTracker
|
||||
+ StageCheckpointReader,
|
||||
EngineT: EngineTypes,
|
||||
{
|
||||
/// Controls syncing triggered by engine updates.
|
||||
sync: EngineSyncController<DB, Client>,
|
||||
sync: EngineSyncController<N, Client>,
|
||||
/// The type we can use to query both the database and the blockchain tree.
|
||||
blockchain: BT,
|
||||
/// Used for emitting updates about whether the engine is syncing or not.
|
||||
sync_state_updater: Box<dyn NetworkSyncUpdater>,
|
||||
/// The Engine API message receiver.
|
||||
engine_message_stream: BoxStream<'static, BeaconEngineMessage<EngineT>>,
|
||||
engine_message_stream: BoxStream<'static, BeaconEngineMessage<N::Engine>>,
|
||||
/// A clone of the handle
|
||||
handle: BeaconConsensusEngineHandle<EngineT>,
|
||||
handle: BeaconConsensusEngineHandle<N::Engine>,
|
||||
/// Tracks the received forkchoice state updates received by the CL.
|
||||
forkchoice_state_tracker: ForkchoiceStateTracker,
|
||||
/// The payload store.
|
||||
payload_builder: PayloadBuilderHandle<EngineT>,
|
||||
payload_builder: PayloadBuilderHandle<N::Engine>,
|
||||
/// Validator for execution payloads
|
||||
payload_validator: ExecutionPayloadValidator,
|
||||
/// Current blockchain tree action.
|
||||
blockchain_tree_action: Option<BlockchainTreeAction<EngineT>>,
|
||||
blockchain_tree_action: Option<BlockchainTreeAction<N::Engine>>,
|
||||
/// Pending forkchoice update.
|
||||
/// It is recorded if we cannot process the forkchoice update because
|
||||
/// a hook with database read-write access is active.
|
||||
/// This is a temporary solution to always process missed FCUs.
|
||||
pending_forkchoice_update: Option<PendingForkchoiceUpdate<EngineT::PayloadAttributes>>,
|
||||
pending_forkchoice_update:
|
||||
Option<PendingForkchoiceUpdate<<N::Engine as PayloadTypes>::PayloadAttributes>>,
|
||||
/// Tracks the header of invalid payloads that were rejected by the engine because they're
|
||||
/// invalid.
|
||||
invalid_headers: InvalidHeaderCache,
|
||||
@ -224,33 +223,32 @@ where
|
||||
metrics: EngineMetrics,
|
||||
}
|
||||
|
||||
impl<DB, BT, Client, EngineT> BeaconConsensusEngine<DB, BT, Client, EngineT>
|
||||
impl<N, BT, Client> BeaconConsensusEngine<N, BT, Client>
|
||||
where
|
||||
DB: Database + Unpin + 'static,
|
||||
N: ProviderNodeTypes,
|
||||
BT: BlockchainTreeEngine
|
||||
+ BlockReader
|
||||
+ BlockIdReader
|
||||
+ CanonChainTracker
|
||||
+ StageCheckpointReader
|
||||
+ ChainSpecProvider<ChainSpec = ChainSpec>
|
||||
+ ChainSpecProvider<ChainSpec = N::ChainSpec>
|
||||
+ 'static,
|
||||
Client: BlockClient + 'static,
|
||||
EngineT: EngineTypes + Unpin,
|
||||
{
|
||||
/// Create a new instance of the [`BeaconConsensusEngine`].
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
pub fn new(
|
||||
client: Client,
|
||||
pipeline: Pipeline<DB>,
|
||||
pipeline: Pipeline<N>,
|
||||
blockchain: BT,
|
||||
task_spawner: Box<dyn TaskSpawner>,
|
||||
sync_state_updater: Box<dyn NetworkSyncUpdater>,
|
||||
max_block: Option<BlockNumber>,
|
||||
payload_builder: PayloadBuilderHandle<EngineT>,
|
||||
payload_builder: PayloadBuilderHandle<N::Engine>,
|
||||
target: Option<B256>,
|
||||
pipeline_run_threshold: u64,
|
||||
hooks: EngineHooks,
|
||||
) -> RethResult<(Self, BeaconConsensusEngineHandle<EngineT>)> {
|
||||
) -> RethResult<(Self, BeaconConsensusEngineHandle<N::Engine>)> {
|
||||
let (to_engine, rx) = mpsc::unbounded_channel();
|
||||
Self::with_channel(
|
||||
client,
|
||||
@ -284,18 +282,18 @@ where
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
pub fn with_channel(
|
||||
client: Client,
|
||||
pipeline: Pipeline<DB>,
|
||||
pipeline: Pipeline<N>,
|
||||
blockchain: BT,
|
||||
task_spawner: Box<dyn TaskSpawner>,
|
||||
sync_state_updater: Box<dyn NetworkSyncUpdater>,
|
||||
max_block: Option<BlockNumber>,
|
||||
payload_builder: PayloadBuilderHandle<EngineT>,
|
||||
payload_builder: PayloadBuilderHandle<N::Engine>,
|
||||
target: Option<B256>,
|
||||
pipeline_run_threshold: u64,
|
||||
to_engine: UnboundedSender<BeaconEngineMessage<EngineT>>,
|
||||
engine_message_stream: BoxStream<'static, BeaconEngineMessage<EngineT>>,
|
||||
to_engine: UnboundedSender<BeaconEngineMessage<N::Engine>>,
|
||||
engine_message_stream: BoxStream<'static, BeaconEngineMessage<N::Engine>>,
|
||||
hooks: EngineHooks,
|
||||
) -> RethResult<(Self, BeaconConsensusEngineHandle<EngineT>)> {
|
||||
) -> RethResult<(Self, BeaconConsensusEngineHandle<N::Engine>)> {
|
||||
let event_sender = EventSender::default();
|
||||
let handle = BeaconConsensusEngineHandle::new(to_engine, event_sender.clone());
|
||||
let sync = EngineSyncController::new(
|
||||
@ -349,7 +347,7 @@ where
|
||||
}
|
||||
|
||||
/// Set the next blockchain tree action.
|
||||
fn set_blockchain_tree_action(&mut self, action: BlockchainTreeAction<EngineT>) {
|
||||
fn set_blockchain_tree_action(&mut self, action: BlockchainTreeAction<N::Engine>) {
|
||||
let previous_action = self.blockchain_tree_action.replace(action);
|
||||
debug_assert!(previous_action.is_none(), "Pre-existing action found");
|
||||
}
|
||||
@ -391,7 +389,7 @@ where
|
||||
fn on_forkchoice_updated_make_canonical_result(
|
||||
&mut self,
|
||||
state: ForkchoiceState,
|
||||
mut attrs: Option<EngineT::PayloadAttributes>,
|
||||
mut attrs: Option<<N::Engine as PayloadTypes>::PayloadAttributes>,
|
||||
make_canonical_result: Result<CanonicalOutcome, CanonicalError>,
|
||||
elapsed: Duration,
|
||||
) -> Result<OnForkChoiceUpdated, CanonicalError> {
|
||||
@ -455,7 +453,7 @@ where
|
||||
&self,
|
||||
head: &BlockNumHash,
|
||||
header: &SealedHeader,
|
||||
attrs: &mut Option<EngineT::PayloadAttributes>,
|
||||
attrs: &mut Option<<N::Engine as PayloadTypes>::PayloadAttributes>,
|
||||
) -> bool {
|
||||
// On Optimism, the proposers are allowed to reorg their own chain at will.
|
||||
#[cfg(feature = "optimism")]
|
||||
@ -499,7 +497,7 @@ where
|
||||
fn on_forkchoice_updated(
|
||||
&mut self,
|
||||
state: ForkchoiceState,
|
||||
attrs: Option<EngineT::PayloadAttributes>,
|
||||
attrs: Option<<N::Engine as PayloadTypes>::PayloadAttributes>,
|
||||
tx: oneshot::Sender<RethResult<OnForkChoiceUpdated>>,
|
||||
) {
|
||||
self.metrics.forkchoice_updated_messages.increment(1);
|
||||
@ -621,7 +619,7 @@ where
|
||||
///
|
||||
/// The [`BeaconConsensusEngineHandle`] can be used to interact with this
|
||||
/// [`BeaconConsensusEngine`]
|
||||
pub fn handle(&self) -> BeaconConsensusEngineHandle<EngineT> {
|
||||
pub fn handle(&self) -> BeaconConsensusEngineHandle<N::Engine> {
|
||||
self.handle.clone()
|
||||
}
|
||||
|
||||
@ -1157,7 +1155,7 @@ where
|
||||
/// return an error if the payload attributes are invalid.
|
||||
fn process_payload_attributes(
|
||||
&self,
|
||||
attrs: EngineT::PayloadAttributes,
|
||||
attrs: <N::Engine as PayloadTypes>::PayloadAttributes,
|
||||
head: Header,
|
||||
state: ForkchoiceState,
|
||||
) -> OnForkChoiceUpdated {
|
||||
@ -1174,7 +1172,7 @@ where
|
||||
// forkchoiceState.headBlockHash and identified via buildProcessId value if
|
||||
// payloadAttributes is not null and the forkchoice state has been updated successfully.
|
||||
// The build process is specified in the Payload building section.
|
||||
match <EngineT::PayloadBuilderAttributes as PayloadBuilderAttributes>::try_new(
|
||||
match <<N:: Engine as PayloadTypes>::PayloadBuilderAttributes as PayloadBuilderAttributes>::try_new(
|
||||
state.head_block_hash,
|
||||
attrs,
|
||||
) {
|
||||
@ -1596,7 +1594,7 @@ where
|
||||
/// so the state change should be handled accordingly.
|
||||
fn on_blockchain_tree_action(
|
||||
&mut self,
|
||||
action: BlockchainTreeAction<EngineT>,
|
||||
action: BlockchainTreeAction<N::Engine>,
|
||||
) -> RethResult<EngineEventOutcome> {
|
||||
match action {
|
||||
BlockchainTreeAction::MakeForkchoiceHeadCanonical { state, attrs, tx } => {
|
||||
@ -1789,19 +1787,18 @@ where
|
||||
/// local forkchoice state, it will launch the pipeline to sync to the head hash.
|
||||
/// While the pipeline is syncing, the consensus engine will keep processing messages from the
|
||||
/// receiver and forwarding them to the blockchain tree.
|
||||
impl<DB, BT, Client, EngineT> Future for BeaconConsensusEngine<DB, BT, Client, EngineT>
|
||||
impl<N, BT, Client> Future for BeaconConsensusEngine<N, BT, Client>
|
||||
where
|
||||
DB: Database + Unpin + 'static,
|
||||
N: ProviderNodeTypes,
|
||||
Client: BlockClient + 'static,
|
||||
BT: BlockchainTreeEngine
|
||||
+ BlockReader
|
||||
+ BlockIdReader
|
||||
+ CanonChainTracker
|
||||
+ StageCheckpointReader
|
||||
+ ChainSpecProvider<ChainSpec = ChainSpec>
|
||||
+ ChainSpecProvider<ChainSpec = N::ChainSpec>
|
||||
+ Unpin
|
||||
+ 'static,
|
||||
EngineT: EngineTypes + Unpin,
|
||||
{
|
||||
type Output = Result<(), BeaconConsensusEngineError>;
|
||||
|
||||
@ -2156,8 +2153,8 @@ mod tests {
|
||||
assert_matches!(rx.await, Ok(Ok(())));
|
||||
}
|
||||
|
||||
fn insert_blocks<'a, DB: Database>(
|
||||
provider_factory: ProviderFactory<DB>,
|
||||
fn insert_blocks<'a, N: ProviderNodeTypes>(
|
||||
provider_factory: ProviderFactory<N>,
|
||||
mut blocks: impl Iterator<Item = &'a SealedBlock>,
|
||||
) {
|
||||
let provider = provider_factory.provider_rw().unwrap();
|
||||
@ -2176,10 +2173,10 @@ mod tests {
|
||||
mod fork_choice_updated {
|
||||
use super::*;
|
||||
use generators::BlockParams;
|
||||
use reth_db::{tables, test_utils::create_test_static_files_dir};
|
||||
use reth_db::{tables, test_utils::create_test_static_files_dir, Database};
|
||||
use reth_db_api::transaction::DbTxMut;
|
||||
use reth_primitives::U256;
|
||||
use reth_provider::providers::StaticFileProvider;
|
||||
use reth_provider::{providers::StaticFileProvider, test_utils::MockNodeTypesWithDB};
|
||||
use reth_rpc_types::engine::ForkchoiceUpdateError;
|
||||
use reth_testing_utils::generators::random_block;
|
||||
|
||||
@ -2248,8 +2245,8 @@ mod tests {
|
||||
let (_static_dir, static_dir_path) = create_test_static_files_dir();
|
||||
|
||||
insert_blocks(
|
||||
ProviderFactory::new(
|
||||
env.db.as_ref(),
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
env.db.clone(),
|
||||
chain_spec.clone(),
|
||||
StaticFileProvider::read_write(static_dir_path).unwrap(),
|
||||
),
|
||||
@ -2316,8 +2313,8 @@ mod tests {
|
||||
let (_static_dir, static_dir_path) = create_test_static_files_dir();
|
||||
|
||||
insert_blocks(
|
||||
ProviderFactory::new(
|
||||
env.db.as_ref(),
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
env.db.clone(),
|
||||
chain_spec.clone(),
|
||||
StaticFileProvider::read_write(static_dir_path).unwrap(),
|
||||
),
|
||||
@ -2347,8 +2344,8 @@ mod tests {
|
||||
|
||||
// Insert next head immediately after sending forkchoice update
|
||||
insert_blocks(
|
||||
ProviderFactory::new(
|
||||
env.db.as_ref(),
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
env.db.clone(),
|
||||
chain_spec.clone(),
|
||||
StaticFileProvider::read_write(static_dir_path).unwrap(),
|
||||
),
|
||||
@ -2403,8 +2400,8 @@ mod tests {
|
||||
let (_static_dir, static_dir_path) = create_test_static_files_dir();
|
||||
|
||||
insert_blocks(
|
||||
ProviderFactory::new(
|
||||
env.db.as_ref(),
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
env.db.clone(),
|
||||
chain_spec.clone(),
|
||||
StaticFileProvider::read_write(static_dir_path).unwrap(),
|
||||
),
|
||||
@ -2486,8 +2483,8 @@ mod tests {
|
||||
|
||||
let (_static_dir, static_dir_path) = create_test_static_files_dir();
|
||||
insert_blocks(
|
||||
ProviderFactory::new(
|
||||
env.db.as_ref(),
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
env.db.clone(),
|
||||
chain_spec.clone(),
|
||||
StaticFileProvider::read_write(static_dir_path).unwrap(),
|
||||
),
|
||||
@ -2547,8 +2544,8 @@ mod tests {
|
||||
let (_temp_dir, temp_dir_path) = create_test_static_files_dir();
|
||||
|
||||
insert_blocks(
|
||||
ProviderFactory::new(
|
||||
env.db.as_ref(),
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
env.db.clone(),
|
||||
chain_spec.clone(),
|
||||
StaticFileProvider::read_write(temp_dir_path).unwrap(),
|
||||
),
|
||||
@ -2580,7 +2577,8 @@ mod tests {
|
||||
use reth_db::test_utils::create_test_static_files_dir;
|
||||
use reth_primitives::{EthereumHardfork, U256};
|
||||
use reth_provider::{
|
||||
providers::StaticFileProvider, test_utils::blocks::BlockchainTestData,
|
||||
providers::StaticFileProvider,
|
||||
test_utils::{blocks::BlockchainTestData, MockNodeTypesWithDB},
|
||||
};
|
||||
use reth_testing_utils::{generators::random_block, GenesisAllocator};
|
||||
#[tokio::test]
|
||||
@ -2680,8 +2678,8 @@ mod tests {
|
||||
|
||||
let (_static_dir, static_dir_path) = create_test_static_files_dir();
|
||||
insert_blocks(
|
||||
ProviderFactory::new(
|
||||
env.db.as_ref(),
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
env.db.clone(),
|
||||
chain_spec.clone(),
|
||||
StaticFileProvider::read_write(static_dir_path).unwrap(),
|
||||
),
|
||||
@ -2760,8 +2758,8 @@ mod tests {
|
||||
let (_static_dir, static_dir_path) = create_test_static_files_dir();
|
||||
|
||||
insert_blocks(
|
||||
ProviderFactory::new(
|
||||
env.db.as_ref(),
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
env.db.clone(),
|
||||
chain_spec.clone(),
|
||||
StaticFileProvider::read_write(static_dir_path).unwrap(),
|
||||
),
|
||||
@ -2810,8 +2808,8 @@ mod tests {
|
||||
let (_static_dir, static_dir_path) = create_test_static_files_dir();
|
||||
|
||||
insert_blocks(
|
||||
ProviderFactory::new(
|
||||
env.db.as_ref(),
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
env.db.clone(),
|
||||
chain_spec.clone(),
|
||||
StaticFileProvider::read_write(static_dir_path).unwrap(),
|
||||
),
|
||||
@ -2881,8 +2879,8 @@ mod tests {
|
||||
let (_static_dir, static_dir_path) = create_test_static_files_dir();
|
||||
|
||||
insert_blocks(
|
||||
ProviderFactory::new(
|
||||
env.db.as_ref(),
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
env.db.clone(),
|
||||
chain_spec.clone(),
|
||||
StaticFileProvider::read_write(static_dir_path).unwrap(),
|
||||
),
|
||||
|
||||
@ -5,13 +5,13 @@ use crate::{
|
||||
ConsensusEngineLiveSyncProgress, EthBeaconConsensus,
|
||||
};
|
||||
use futures::FutureExt;
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_db_api::database::Database;
|
||||
use reth_network_p2p::{
|
||||
full_block::{FetchFullBlockFuture, FetchFullBlockRangeFuture, FullBlockClient},
|
||||
BlockClient,
|
||||
};
|
||||
use reth_node_types::NodeTypesWithDB;
|
||||
use reth_primitives::{BlockNumber, SealedBlock, B256};
|
||||
use reth_provider::providers::ProviderNodeTypes;
|
||||
use reth_stages_api::{ControlFlow, Pipeline, PipelineError, PipelineTarget, PipelineWithResult};
|
||||
use reth_tasks::TaskSpawner;
|
||||
use reth_tokio_util::EventSender;
|
||||
@ -31,9 +31,9 @@ use tracing::trace;
|
||||
/// Caution: If the pipeline is running, this type will not emit blocks downloaded from the network
|
||||
/// [`EngineSyncEvent::FetchedFullBlock`] until the pipeline is idle to prevent commits to the
|
||||
/// database while the pipeline is still active.
|
||||
pub(crate) struct EngineSyncController<DB, Client>
|
||||
pub(crate) struct EngineSyncController<N, Client>
|
||||
where
|
||||
DB: Database,
|
||||
N: NodeTypesWithDB,
|
||||
Client: BlockClient,
|
||||
{
|
||||
/// A downloader that can download full blocks from the network.
|
||||
@ -42,7 +42,7 @@ where
|
||||
pipeline_task_spawner: Box<dyn TaskSpawner>,
|
||||
/// The current state of the pipeline.
|
||||
/// The pipeline is used for large ranges.
|
||||
pipeline_state: PipelineState<DB>,
|
||||
pipeline_state: PipelineState<N>,
|
||||
/// Pending target block for the pipeline to sync
|
||||
pending_pipeline_target: Option<PipelineTarget>,
|
||||
/// In-flight full block requests in progress.
|
||||
@ -61,18 +61,18 @@ where
|
||||
metrics: EngineSyncMetrics,
|
||||
}
|
||||
|
||||
impl<DB, Client> EngineSyncController<DB, Client>
|
||||
impl<N, Client> EngineSyncController<N, Client>
|
||||
where
|
||||
DB: Database + 'static,
|
||||
N: ProviderNodeTypes,
|
||||
Client: BlockClient + 'static,
|
||||
{
|
||||
/// Create a new instance
|
||||
pub(crate) fn new(
|
||||
pipeline: Pipeline<DB>,
|
||||
pipeline: Pipeline<N>,
|
||||
client: Client,
|
||||
pipeline_task_spawner: Box<dyn TaskSpawner>,
|
||||
max_block: Option<BlockNumber>,
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
chain_spec: Arc<N::ChainSpec>,
|
||||
event_sender: EventSender<BeaconConsensusEngineEvent>,
|
||||
) -> Self {
|
||||
Self {
|
||||
@ -393,14 +393,14 @@ pub(crate) enum EngineSyncEvent {
|
||||
/// running, it acquires the write lock over the database. This means that we cannot forward to the
|
||||
/// blockchain tree any messages that would result in database writes, since it would result in a
|
||||
/// deadlock.
|
||||
enum PipelineState<DB: Database> {
|
||||
enum PipelineState<N: NodeTypesWithDB> {
|
||||
/// Pipeline is idle.
|
||||
Idle(Option<Pipeline<DB>>),
|
||||
Idle(Option<Pipeline<N>>),
|
||||
/// Pipeline is running and waiting for a response
|
||||
Running(oneshot::Receiver<PipelineWithResult<DB>>),
|
||||
Running(oneshot::Receiver<PipelineWithResult<N>>),
|
||||
}
|
||||
|
||||
impl<DB: Database> PipelineState<DB> {
|
||||
impl<N: NodeTypesWithDB> PipelineState<N> {
|
||||
/// Returns `true` if the state matches idle.
|
||||
const fn is_idle(&self) -> bool {
|
||||
matches!(self, Self::Idle(_))
|
||||
@ -412,12 +412,12 @@ mod tests {
|
||||
use super::*;
|
||||
use assert_matches::assert_matches;
|
||||
use futures::poll;
|
||||
use reth_chainspec::{ChainSpecBuilder, MAINNET};
|
||||
use reth_db::{mdbx::DatabaseEnv, test_utils::TempDatabase};
|
||||
use reth_chainspec::{ChainSpec, ChainSpecBuilder, MAINNET};
|
||||
use reth_network_p2p::{either::Either, test_utils::TestFullBlockClient};
|
||||
use reth_primitives::{BlockBody, Header, SealedHeader};
|
||||
use reth_provider::{
|
||||
test_utils::create_test_provider_factory_with_chain_spec, ExecutionOutcome,
|
||||
test_utils::{create_test_provider_factory_with_chain_spec, MockNodeTypesWithDB},
|
||||
ExecutionOutcome,
|
||||
};
|
||||
use reth_prune_types::PruneModes;
|
||||
use reth_stages::{test_utils::TestStages, ExecOutput, StageError};
|
||||
@ -467,12 +467,12 @@ mod tests {
|
||||
}
|
||||
|
||||
/// Builds the pipeline.
|
||||
fn build(self, chain_spec: Arc<ChainSpec>) -> Pipeline<Arc<TempDatabase<DatabaseEnv>>> {
|
||||
fn build(self, chain_spec: Arc<ChainSpec>) -> Pipeline<MockNodeTypesWithDB> {
|
||||
reth_tracing::init_test_tracing();
|
||||
|
||||
// Setup pipeline
|
||||
let (tip_tx, _tip_rx) = watch::channel(B256::default());
|
||||
let mut pipeline = Pipeline::builder()
|
||||
let mut pipeline = Pipeline::<MockNodeTypesWithDB>::builder()
|
||||
.add_stages(TestStages::new(self.pipeline_exec_outputs, Default::default()))
|
||||
.with_tip_sender(tip_tx);
|
||||
|
||||
@ -514,13 +514,13 @@ mod tests {
|
||||
}
|
||||
|
||||
/// Builds the sync controller.
|
||||
fn build<DB>(
|
||||
fn build<N>(
|
||||
self,
|
||||
pipeline: Pipeline<DB>,
|
||||
pipeline: Pipeline<N>,
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
) -> EngineSyncController<DB, Either<Client, TestFullBlockClient>>
|
||||
) -> EngineSyncController<N, Either<Client, TestFullBlockClient>>
|
||||
where
|
||||
DB: Database + 'static,
|
||||
N: ProviderNodeTypes,
|
||||
Client: BlockClient + 'static,
|
||||
{
|
||||
let client = self
|
||||
|
||||
@ -22,7 +22,8 @@ use reth_network_p2p::{sync::NoopSyncStateUpdater, test_utils::NoopFullBlockClie
|
||||
use reth_payload_builder::test_utils::spawn_test_payload_service;
|
||||
use reth_primitives::{BlockNumber, B256};
|
||||
use reth_provider::{
|
||||
providers::BlockchainProvider, test_utils::create_test_provider_factory_with_chain_spec,
|
||||
providers::BlockchainProvider,
|
||||
test_utils::{create_test_provider_factory_with_chain_spec, MockNodeTypesWithDB},
|
||||
ExecutionOutcome, ProviderFactory,
|
||||
};
|
||||
use reth_prune::Pruner;
|
||||
@ -39,10 +40,9 @@ use tokio::sync::{oneshot, watch};
|
||||
type DatabaseEnv = TempDatabase<DE>;
|
||||
|
||||
type TestBeaconConsensusEngine<Client> = BeaconConsensusEngine<
|
||||
Arc<DatabaseEnv>,
|
||||
BlockchainProvider<Arc<DatabaseEnv>>,
|
||||
MockNodeTypesWithDB,
|
||||
BlockchainProvider<MockNodeTypesWithDB>,
|
||||
Arc<Either<Client, NoopFullBlockClient>>,
|
||||
EthEngineTypes,
|
||||
>;
|
||||
|
||||
#[derive(Debug)]
|
||||
@ -355,7 +355,7 @@ where
|
||||
// Setup pipeline
|
||||
let (tip_tx, tip_rx) = watch::channel(B256::default());
|
||||
let mut pipeline = match self.base_config.pipeline_config {
|
||||
TestPipelineConfig::Test(outputs) => Pipeline::builder()
|
||||
TestPipelineConfig::Test(outputs) => Pipeline::<MockNodeTypesWithDB>::builder()
|
||||
.add_stages(TestStages::new(outputs, Default::default()))
|
||||
.with_tip_sender(tip_tx),
|
||||
TestPipelineConfig::Real => {
|
||||
@ -367,7 +367,7 @@ where
|
||||
.build(client.clone(), consensus.clone(), provider_factory.clone())
|
||||
.into_task();
|
||||
|
||||
Pipeline::builder().add_stages(DefaultStages::new(
|
||||
Pipeline::<MockNodeTypesWithDB>::builder().add_stages(DefaultStages::new(
|
||||
provider_factory.clone(),
|
||||
tip_rx.clone(),
|
||||
Arc::clone(&consensus),
|
||||
|
||||
@ -117,8 +117,10 @@ where
|
||||
// Type aliases
|
||||
|
||||
type TmpDB = Arc<TempDatabase<DatabaseEnv>>;
|
||||
type TmpNodeAdapter<N> =
|
||||
FullNodeTypesAdapter<NodeTypesWithDBAdapter<N, TmpDB>, BlockchainProvider<TmpDB>>;
|
||||
type TmpNodeAdapter<N> = FullNodeTypesAdapter<
|
||||
NodeTypesWithDBAdapter<N, TmpDB>,
|
||||
BlockchainProvider<NodeTypesWithDBAdapter<N, TmpDB>>,
|
||||
>;
|
||||
|
||||
type Adapter<N> = NodeAdapter<
|
||||
RethFullAdapter<TmpDB, N>,
|
||||
|
||||
@ -13,10 +13,7 @@ workspace = true
|
||||
[dependencies]
|
||||
# reth
|
||||
reth-beacon-consensus.workspace = true
|
||||
reth-chainspec.workspace = true
|
||||
reth-consensus.workspace = true
|
||||
reth-db-api.workspace = true
|
||||
reth-engine-primitives.workspace = true
|
||||
reth-engine-tree.workspace = true
|
||||
reth-evm.workspace = true
|
||||
reth-network-p2p.workspace = true
|
||||
@ -26,6 +23,7 @@ reth-provider.workspace = true
|
||||
reth-prune.workspace = true
|
||||
reth-stages-api.workspace = true
|
||||
reth-tasks.workspace = true
|
||||
reth-node-types.workspace = true
|
||||
|
||||
# async
|
||||
futures.workspace = true
|
||||
@ -43,6 +41,7 @@ reth-evm-ethereum.workspace = true
|
||||
reth-exex-types.workspace = true
|
||||
reth-primitives.workspace = true
|
||||
reth-prune-types.workspace = true
|
||||
reth-chainspec.workspace = true
|
||||
|
||||
tokio = { workspace = true, features = ["sync"] }
|
||||
tokio-stream.workspace = true
|
||||
|
||||
@ -1,10 +1,7 @@
|
||||
use futures::{Stream, StreamExt};
|
||||
use pin_project::pin_project;
|
||||
use reth_beacon_consensus::{BeaconConsensusEngineEvent, BeaconEngineMessage};
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_consensus::Consensus;
|
||||
use reth_db_api::database::Database;
|
||||
use reth_engine_primitives::EngineTypes;
|
||||
use reth_engine_tree::{
|
||||
backfill::PipelineSync,
|
||||
download::BasicBlockDownloader,
|
||||
@ -18,9 +15,13 @@ pub use reth_engine_tree::{
|
||||
};
|
||||
use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_network_p2p::BlockClient;
|
||||
use reth_node_types::NodeTypesWithEngine;
|
||||
use reth_payload_builder::PayloadBuilderHandle;
|
||||
use reth_payload_validator::ExecutionPayloadValidator;
|
||||
use reth_provider::{providers::BlockchainProvider2, ProviderFactory};
|
||||
use reth_provider::{
|
||||
providers::{BlockchainProvider2, ProviderNodeTypes},
|
||||
ProviderFactory,
|
||||
};
|
||||
use reth_prune::Pruner;
|
||||
use reth_stages_api::Pipeline;
|
||||
use reth_tasks::TaskSpawner;
|
||||
@ -35,50 +36,48 @@ use std::{
|
||||
type EngineMessageStream<T> = Pin<Box<dyn Stream<Item = BeaconEngineMessage<T>> + Send + Sync>>;
|
||||
|
||||
/// Alias for chain orchestrator.
|
||||
type EngineServiceType<DB, Client, T> = ChainOrchestrator<
|
||||
type EngineServiceType<N, Client> = ChainOrchestrator<
|
||||
EngineHandler<
|
||||
EngineApiRequestHandler<EngineApiRequest<T>>,
|
||||
EngineMessageStream<T>,
|
||||
EngineApiRequestHandler<EngineApiRequest<<N as NodeTypesWithEngine>::Engine>>,
|
||||
EngineMessageStream<<N as NodeTypesWithEngine>::Engine>,
|
||||
BasicBlockDownloader<Client>,
|
||||
>,
|
||||
PipelineSync<DB>,
|
||||
PipelineSync<N>,
|
||||
>;
|
||||
|
||||
/// The type that drives the chain forward and communicates progress.
|
||||
#[pin_project]
|
||||
#[allow(missing_debug_implementations)]
|
||||
pub struct EngineService<DB, Client, E, T>
|
||||
pub struct EngineService<N, Client, E>
|
||||
where
|
||||
DB: Database + 'static,
|
||||
N: ProviderNodeTypes,
|
||||
Client: BlockClient + 'static,
|
||||
E: BlockExecutorProvider + 'static,
|
||||
T: EngineTypes,
|
||||
{
|
||||
orchestrator: EngineServiceType<DB, Client, T>,
|
||||
orchestrator: EngineServiceType<N, Client>,
|
||||
_marker: PhantomData<E>,
|
||||
}
|
||||
|
||||
impl<DB, Client, E, T> EngineService<DB, Client, E, T>
|
||||
impl<N, Client, E> EngineService<N, Client, E>
|
||||
where
|
||||
DB: Database + 'static,
|
||||
N: ProviderNodeTypes,
|
||||
Client: BlockClient + 'static,
|
||||
E: BlockExecutorProvider + 'static,
|
||||
T: EngineTypes + 'static,
|
||||
{
|
||||
/// Constructor for `EngineService`.
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
pub fn new(
|
||||
consensus: Arc<dyn Consensus>,
|
||||
executor_factory: E,
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
chain_spec: Arc<N::ChainSpec>,
|
||||
client: Client,
|
||||
incoming_requests: EngineMessageStream<T>,
|
||||
pipeline: Pipeline<DB>,
|
||||
incoming_requests: EngineMessageStream<N::Engine>,
|
||||
pipeline: Pipeline<N>,
|
||||
pipeline_task_spawner: Box<dyn TaskSpawner>,
|
||||
provider: ProviderFactory<DB>,
|
||||
blockchain_db: BlockchainProvider2<DB>,
|
||||
pruner: Pruner<DB, ProviderFactory<DB>>,
|
||||
payload_builder: PayloadBuilderHandle<T>,
|
||||
provider: ProviderFactory<N>,
|
||||
blockchain_db: BlockchainProvider2<N>,
|
||||
pruner: Pruner<N::DB, ProviderFactory<N>>,
|
||||
payload_builder: PayloadBuilderHandle<N::Engine>,
|
||||
tree_config: TreeConfig,
|
||||
invalid_block_hook: Box<dyn InvalidBlockHook>,
|
||||
) -> Self {
|
||||
@ -113,17 +112,16 @@ where
|
||||
}
|
||||
|
||||
/// Returns a mutable reference to the orchestrator.
|
||||
pub fn orchestrator_mut(&mut self) -> &mut EngineServiceType<DB, Client, T> {
|
||||
pub fn orchestrator_mut(&mut self) -> &mut EngineServiceType<N, Client> {
|
||||
&mut self.orchestrator
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB, Client, E, T> Stream for EngineService<DB, Client, E, T>
|
||||
impl<N, Client, E> Stream for EngineService<N, Client, E>
|
||||
where
|
||||
DB: Database + 'static,
|
||||
N: ProviderNodeTypes,
|
||||
Client: BlockClient + 'static,
|
||||
E: BlockExecutorProvider + 'static,
|
||||
T: EngineTypes + 'static,
|
||||
{
|
||||
type Item = ChainEvent<BeaconConsensusEngineEvent>;
|
||||
|
||||
|
||||
@ -17,8 +17,6 @@ reth-blockchain-tree.workspace = true
|
||||
reth-blockchain-tree-api.workspace = true
|
||||
reth-chain-state.workspace = true
|
||||
reth-consensus.workspace = true
|
||||
reth-db.workspace = true
|
||||
reth-db-api.workspace = true
|
||||
reth-engine-primitives.workspace = true
|
||||
reth-errors.workspace = true
|
||||
reth-evm.workspace = true
|
||||
@ -34,6 +32,7 @@ reth-rpc-types.workspace = true
|
||||
reth-stages-api.workspace = true
|
||||
reth-tasks.workspace = true
|
||||
reth-trie.workspace = true
|
||||
reth-node-types.workspace = true
|
||||
|
||||
# common
|
||||
futures.workspace = true
|
||||
@ -48,16 +47,15 @@ reth-metrics = { workspace = true, features = ["common"] }
|
||||
tracing.workspace = true
|
||||
|
||||
# optional deps for test-utils
|
||||
reth-chainspec = { workspace = true, optional = true }
|
||||
reth-prune-types = { workspace = true, optional = true }
|
||||
reth-stages = { workspace = true, optional = true }
|
||||
reth-static-file = { workspace = true, optional = true }
|
||||
reth-tracing = { workspace = true, optional = true }
|
||||
reth-chainspec = { workspace = true, optional = true }
|
||||
|
||||
[dev-dependencies]
|
||||
# reth
|
||||
reth-db = { workspace = true, features = ["test-utils"] }
|
||||
reth-chainspec.workspace = true
|
||||
reth-chain-state = { workspace = true, features = ["test-utils"] }
|
||||
reth-ethereum-engine-primitives.workspace = true
|
||||
reth-evm = { workspace = true, features = ["test-utils"] }
|
||||
@ -69,6 +67,7 @@ reth-rpc-types-compat.workspace = true
|
||||
reth-stages = { workspace = true, features = ["test-utils"] }
|
||||
reth-static-file.workspace = true
|
||||
reth-tracing.workspace = true
|
||||
reth-chainspec.workspace = true
|
||||
|
||||
alloy-rlp.workspace = true
|
||||
|
||||
@ -79,10 +78,10 @@ rand.workspace = true
|
||||
test-utils = [
|
||||
"reth-db/test-utils",
|
||||
"reth-chain-state/test-utils",
|
||||
"reth-chainspec",
|
||||
"reth-network-p2p/test-utils",
|
||||
"reth-prune-types",
|
||||
"reth-stages/test-utils",
|
||||
"reth-static-file",
|
||||
"reth-tracing",
|
||||
"reth-chainspec"
|
||||
]
|
||||
|
||||
@ -8,7 +8,8 @@
|
||||
//! These modes are mutually exclusive and the node can only be in one mode at a time.
|
||||
|
||||
use futures::FutureExt;
|
||||
use reth_db_api::database::Database;
|
||||
use reth_node_types::NodeTypesWithDB;
|
||||
use reth_provider::providers::ProviderNodeTypes;
|
||||
use reth_stages_api::{ControlFlow, Pipeline, PipelineError, PipelineTarget, PipelineWithResult};
|
||||
use reth_tasks::TaskSpawner;
|
||||
use std::task::{ready, Context, Poll};
|
||||
@ -78,25 +79,19 @@ pub enum BackfillEvent {
|
||||
|
||||
/// Pipeline sync.
|
||||
#[derive(Debug)]
|
||||
pub struct PipelineSync<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
pub struct PipelineSync<N: NodeTypesWithDB> {
|
||||
/// The type that can spawn the pipeline task.
|
||||
pipeline_task_spawner: Box<dyn TaskSpawner>,
|
||||
/// The current state of the pipeline.
|
||||
/// The pipeline is used for large ranges.
|
||||
pipeline_state: PipelineState<DB>,
|
||||
pipeline_state: PipelineState<N>,
|
||||
/// Pending target block for the pipeline to sync
|
||||
pending_pipeline_target: Option<PipelineTarget>,
|
||||
}
|
||||
|
||||
impl<DB> PipelineSync<DB>
|
||||
where
|
||||
DB: Database + 'static,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> PipelineSync<N> {
|
||||
/// Create a new instance.
|
||||
pub fn new(pipeline: Pipeline<DB>, pipeline_task_spawner: Box<dyn TaskSpawner>) -> Self {
|
||||
pub fn new(pipeline: Pipeline<N>, pipeline_task_spawner: Box<dyn TaskSpawner>) -> Self {
|
||||
Self {
|
||||
pipeline_task_spawner,
|
||||
pipeline_state: PipelineState::Idle(Some(pipeline)),
|
||||
@ -183,10 +178,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> BackfillSync for PipelineSync<DB>
|
||||
where
|
||||
DB: Database + 'static,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> BackfillSync for PipelineSync<N> {
|
||||
fn on_action(&mut self, event: BackfillAction) {
|
||||
match event {
|
||||
BackfillAction::Start(target) => self.set_pipeline_sync_target(target),
|
||||
@ -221,14 +213,14 @@ where
|
||||
/// blockchain tree any messages that would result in database writes, since it would result in a
|
||||
/// deadlock.
|
||||
#[derive(Debug)]
|
||||
enum PipelineState<DB: Database> {
|
||||
enum PipelineState<N: NodeTypesWithDB> {
|
||||
/// Pipeline is idle.
|
||||
Idle(Option<Pipeline<DB>>),
|
||||
Idle(Option<Pipeline<N>>),
|
||||
/// Pipeline is running and waiting for a response
|
||||
Running(oneshot::Receiver<PipelineWithResult<DB>>),
|
||||
Running(oneshot::Receiver<PipelineWithResult<N>>),
|
||||
}
|
||||
|
||||
impl<DB: Database> PipelineState<DB> {
|
||||
impl<N: NodeTypesWithDB> PipelineState<N> {
|
||||
/// Returns `true` if the state matches idle.
|
||||
const fn is_idle(&self) -> bool {
|
||||
matches!(self, Self::Idle(_))
|
||||
@ -242,16 +234,16 @@ mod tests {
|
||||
use assert_matches::assert_matches;
|
||||
use futures::poll;
|
||||
use reth_chainspec::{ChainSpecBuilder, MAINNET};
|
||||
use reth_db::{mdbx::DatabaseEnv, test_utils::TempDatabase};
|
||||
use reth_network_p2p::test_utils::TestFullBlockClient;
|
||||
use reth_primitives::{BlockNumber, Header, B256};
|
||||
use reth_provider::test_utils::MockNodeTypesWithDB;
|
||||
use reth_stages::ExecOutput;
|
||||
use reth_stages_api::StageCheckpoint;
|
||||
use reth_tasks::TokioTaskExecutor;
|
||||
use std::{collections::VecDeque, future::poll_fn, sync::Arc};
|
||||
|
||||
struct TestHarness {
|
||||
pipeline_sync: PipelineSync<Arc<TempDatabase<DatabaseEnv>>>,
|
||||
pipeline_sync: PipelineSync<MockNodeTypesWithDB>,
|
||||
tip: B256,
|
||||
}
|
||||
|
||||
|
||||
@ -2,11 +2,12 @@
|
||||
|
||||
use crate::metrics::PersistenceMetrics;
|
||||
use reth_chain_state::ExecutedBlock;
|
||||
use reth_db::Database;
|
||||
use reth_errors::ProviderError;
|
||||
use reth_node_types::NodeTypesWithDB;
|
||||
use reth_primitives::B256;
|
||||
use reth_provider::{
|
||||
writer::UnifiedStorageWriter, BlockHashReader, ProviderFactory, StaticFileProviderFactory,
|
||||
providers::ProviderNodeTypes, writer::UnifiedStorageWriter, BlockHashReader, ProviderFactory,
|
||||
StaticFileProviderFactory,
|
||||
};
|
||||
use reth_prune::{Pruner, PrunerError, PrunerOutput};
|
||||
use std::{
|
||||
@ -25,23 +26,23 @@ use tracing::{debug, error};
|
||||
/// This should be spawned in its own thread with [`std::thread::spawn`], since this performs
|
||||
/// blocking I/O operations in an endless loop.
|
||||
#[derive(Debug)]
|
||||
pub struct PersistenceService<DB> {
|
||||
pub struct PersistenceService<N: NodeTypesWithDB> {
|
||||
/// The provider factory to use
|
||||
provider: ProviderFactory<DB>,
|
||||
provider: ProviderFactory<N>,
|
||||
/// Incoming requests
|
||||
incoming: Receiver<PersistenceAction>,
|
||||
/// The pruner
|
||||
pruner: Pruner<DB, ProviderFactory<DB>>,
|
||||
pruner: Pruner<N::DB, ProviderFactory<N>>,
|
||||
/// metrics
|
||||
metrics: PersistenceMetrics,
|
||||
}
|
||||
|
||||
impl<DB: Database> PersistenceService<DB> {
|
||||
impl<N: ProviderNodeTypes> PersistenceService<N> {
|
||||
/// Create a new persistence service
|
||||
pub fn new(
|
||||
provider: ProviderFactory<DB>,
|
||||
provider: ProviderFactory<N>,
|
||||
incoming: Receiver<PersistenceAction>,
|
||||
pruner: Pruner<DB, ProviderFactory<DB>>,
|
||||
pruner: Pruner<N::DB, ProviderFactory<N>>,
|
||||
) -> Self {
|
||||
Self { provider, incoming, pruner, metrics: PersistenceMetrics::default() }
|
||||
}
|
||||
@ -58,10 +59,7 @@ impl<DB: Database> PersistenceService<DB> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> PersistenceService<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> PersistenceService<N> {
|
||||
/// This is the main loop, that will listen to database events and perform the requested
|
||||
/// database actions
|
||||
pub fn run(mut self) -> Result<(), PersistenceError> {
|
||||
@ -167,9 +165,9 @@ impl PersistenceHandle {
|
||||
}
|
||||
|
||||
/// Create a new [`PersistenceHandle`], and spawn the persistence service.
|
||||
pub fn spawn_service<DB: Database + 'static>(
|
||||
provider_factory: ProviderFactory<DB>,
|
||||
pruner: Pruner<DB, ProviderFactory<DB>>,
|
||||
pub fn spawn_service<N: ProviderNodeTypes>(
|
||||
provider_factory: ProviderFactory<N>,
|
||||
pruner: Pruner<N::DB, ProviderFactory<N>>,
|
||||
) -> Self {
|
||||
// create the initial channels
|
||||
let (db_service_tx, db_service_rx) = std::sync::mpsc::channel();
|
||||
|
||||
@ -1,8 +1,10 @@
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_db::{mdbx::DatabaseEnv, test_utils::TempDatabase};
|
||||
use reth_network_p2p::test_utils::TestFullBlockClient;
|
||||
use reth_primitives::{BlockBody, SealedHeader, B256};
|
||||
use reth_provider::{test_utils::create_test_provider_factory_with_chain_spec, ExecutionOutcome};
|
||||
use reth_provider::{
|
||||
test_utils::{create_test_provider_factory_with_chain_spec, MockNodeTypesWithDB},
|
||||
ExecutionOutcome,
|
||||
};
|
||||
use reth_prune_types::PruneModes;
|
||||
use reth_stages::{test_utils::TestStages, ExecOutput, StageError};
|
||||
use reth_stages_api::Pipeline;
|
||||
@ -40,12 +42,12 @@ impl TestPipelineBuilder {
|
||||
}
|
||||
|
||||
/// Builds the pipeline.
|
||||
pub fn build(self, chain_spec: Arc<ChainSpec>) -> Pipeline<Arc<TempDatabase<DatabaseEnv>>> {
|
||||
pub fn build(self, chain_spec: Arc<ChainSpec>) -> Pipeline<MockNodeTypesWithDB> {
|
||||
reth_tracing::init_test_tracing();
|
||||
|
||||
// Setup pipeline
|
||||
let (tip_tx, _tip_rx) = watch::channel(B256::default());
|
||||
let pipeline = Pipeline::builder()
|
||||
let pipeline = Pipeline::<MockNodeTypesWithDB>::builder()
|
||||
.add_stages(TestStages::new(self.pipeline_exec_outputs, Default::default()))
|
||||
.with_tip_sender(tip_tx);
|
||||
|
||||
|
||||
@ -6,6 +6,7 @@ use reth_db::{
|
||||
test_utils::{create_test_rw_db, TempDatabase},
|
||||
DatabaseEnv,
|
||||
};
|
||||
use reth_node_api::NodeTypesWithDBAdapter;
|
||||
use reth_node_builder::{EngineNodeLauncher, FullNodeComponents, NodeBuilder, NodeConfig};
|
||||
use reth_node_ethereum::node::{EthereumAddOns, EthereumNode};
|
||||
use reth_provider::providers::BlockchainProvider2;
|
||||
@ -46,15 +47,18 @@ async fn test_eth_launcher() {
|
||||
let tasks = TaskManager::current();
|
||||
let config = NodeConfig::test();
|
||||
let db = create_test_rw_db();
|
||||
let _builder = NodeBuilder::new(config)
|
||||
.with_database(db)
|
||||
.with_types_and_provider::<EthereumNode, BlockchainProvider2<Arc<TempDatabase<DatabaseEnv>>>>()
|
||||
.with_components(EthereumNode::components())
|
||||
.with_add_ons::<EthereumAddOns>()
|
||||
.launch_with_fn(|builder| {
|
||||
let launcher = EngineNodeLauncher::new(tasks.executor(), builder.config.datadir());
|
||||
builder.launch_with(launcher)
|
||||
});
|
||||
let _builder =
|
||||
NodeBuilder::new(config)
|
||||
.with_database(db)
|
||||
.with_types_and_provider::<EthereumNode, BlockchainProvider2<
|
||||
NodeTypesWithDBAdapter<EthereumNode, Arc<TempDatabase<DatabaseEnv>>>,
|
||||
>>()
|
||||
.with_components(EthereumNode::components())
|
||||
.with_add_ons::<EthereumAddOns>()
|
||||
.launch_with_fn(|builder| {
|
||||
let launcher = EngineNodeLauncher::new(tasks.executor(), builder.config.datadir());
|
||||
builder.launch_with(launcher)
|
||||
});
|
||||
}
|
||||
|
||||
#[test]
|
||||
|
||||
@ -10,7 +10,10 @@ use reth_primitives::{
|
||||
b256, constants::ETH_TO_WEI, Address, Block, BlockWithSenders, Genesis, GenesisAccount, Header,
|
||||
Receipt, Requests, SealedBlockWithSenders, Transaction, TxEip2930, TxKind, U256,
|
||||
};
|
||||
use reth_provider::{BlockWriter as _, ExecutionOutcome, LatestStateProviderRef, ProviderFactory};
|
||||
use reth_provider::{
|
||||
providers::ProviderNodeTypes, BlockWriter as _, ExecutionOutcome, LatestStateProviderRef,
|
||||
ProviderFactory,
|
||||
};
|
||||
use reth_revm::database::StateProviderDatabase;
|
||||
use reth_testing_utils::generators::sign_tx_with_key_pair;
|
||||
use secp256k1::Keypair;
|
||||
@ -46,13 +49,13 @@ pub(crate) fn chain_spec(address: Address) -> Arc<ChainSpec> {
|
||||
)
|
||||
}
|
||||
|
||||
pub(crate) fn execute_block_and_commit_to_database<DB>(
|
||||
provider_factory: &ProviderFactory<DB>,
|
||||
pub(crate) fn execute_block_and_commit_to_database<N>(
|
||||
provider_factory: &ProviderFactory<N>,
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
block: &BlockWithSenders,
|
||||
) -> eyre::Result<BlockExecutionOutput<Receipt>>
|
||||
where
|
||||
DB: reth_db_api::database::Database,
|
||||
N: ProviderNodeTypes,
|
||||
{
|
||||
let provider = provider_factory.provider()?;
|
||||
|
||||
@ -149,13 +152,13 @@ fn blocks(
|
||||
Ok((block1, block2))
|
||||
}
|
||||
|
||||
pub(crate) fn blocks_and_execution_outputs<DB>(
|
||||
provider_factory: ProviderFactory<DB>,
|
||||
pub(crate) fn blocks_and_execution_outputs<N>(
|
||||
provider_factory: ProviderFactory<N>,
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
key_pair: Keypair,
|
||||
) -> eyre::Result<Vec<(SealedBlockWithSenders, BlockExecutionOutput<Receipt>)>>
|
||||
where
|
||||
DB: reth_db_api::database::Database,
|
||||
N: ProviderNodeTypes,
|
||||
{
|
||||
let (block1, block2) = blocks(chain_spec.clone(), key_pair)?;
|
||||
|
||||
@ -170,13 +173,13 @@ where
|
||||
Ok(vec![(block1, block_output1), (block2, block_output2)])
|
||||
}
|
||||
|
||||
pub(crate) fn blocks_and_execution_outcome<DB>(
|
||||
provider_factory: ProviderFactory<DB>,
|
||||
pub(crate) fn blocks_and_execution_outcome<N>(
|
||||
provider_factory: ProviderFactory<N>,
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
key_pair: Keypair,
|
||||
) -> eyre::Result<(Vec<SealedBlockWithSenders>, ExecutionOutcome)>
|
||||
where
|
||||
DB: reth_db_api::database::Database,
|
||||
N: ProviderNodeTypes,
|
||||
{
|
||||
let (block1, block2) = blocks(chain_spec.clone(), key_pair)?;
|
||||
|
||||
|
||||
@ -12,7 +12,10 @@ use futures_util::FutureExt;
|
||||
use reth_blockchain_tree::noop::NoopBlockchainTree;
|
||||
use reth_chainspec::{ChainSpec, MAINNET};
|
||||
use reth_consensus::test_utils::TestConsensus;
|
||||
use reth_db::{test_utils::TempDatabase, DatabaseEnv};
|
||||
use reth_db::{
|
||||
test_utils::{create_test_rw_db, create_test_static_files_dir, TempDatabase},
|
||||
DatabaseEnv,
|
||||
};
|
||||
use reth_db_common::init::init_genesis;
|
||||
use reth_evm::test_utils::MockExecutorProvider;
|
||||
use reth_execution_types::Chain;
|
||||
@ -36,7 +39,7 @@ use reth_node_ethereum::{
|
||||
use reth_payload_builder::noop::NoopPayloadBuilderService;
|
||||
use reth_primitives::{Head, SealedBlockWithSenders};
|
||||
use reth_provider::{
|
||||
providers::BlockchainProvider, test_utils::create_test_provider_factory_with_chain_spec,
|
||||
providers::{BlockchainProvider, StaticFileProvider},
|
||||
BlockReader, ProviderFactory,
|
||||
};
|
||||
use reth_tasks::TaskManager;
|
||||
@ -151,7 +154,10 @@ pub type TmpDB = Arc<TempDatabase<DatabaseEnv>>;
|
||||
pub type Adapter = NodeAdapter<
|
||||
RethFullAdapter<TmpDB, TestNode>,
|
||||
<<TestNode as Node<
|
||||
FullNodeTypesAdapter<NodeTypesWithDBAdapter<TestNode, TmpDB>, BlockchainProvider<TmpDB>>,
|
||||
FullNodeTypesAdapter<
|
||||
NodeTypesWithDBAdapter<TestNode, TmpDB>,
|
||||
BlockchainProvider<NodeTypesWithDBAdapter<TestNode, TmpDB>>,
|
||||
>,
|
||||
>>::ComponentsBuilder as NodeComponentsBuilder<RethFullAdapter<TmpDB, TestNode>>>::Components,
|
||||
>;
|
||||
/// An [`ExExContext`] using the [`Adapter`] type.
|
||||
@ -163,7 +169,7 @@ pub struct TestExExHandle {
|
||||
/// Genesis block that was inserted into the storage
|
||||
pub genesis: SealedBlockWithSenders,
|
||||
/// Provider Factory for accessing the emphemeral storage of the host node
|
||||
pub provider_factory: ProviderFactory<TmpDB>,
|
||||
pub provider_factory: ProviderFactory<NodeTypesWithDBAdapter<TestNode, TmpDB>>,
|
||||
/// Channel for receiving events from the Execution Extension
|
||||
pub events_rx: UnboundedReceiver<ExExEvent>,
|
||||
/// Channel for sending notifications to the Execution Extension
|
||||
@ -236,7 +242,14 @@ pub async fn test_exex_context_with_chain_spec(
|
||||
let executor = MockExecutorProvider::default();
|
||||
let consensus = Arc::new(TestConsensus::default());
|
||||
|
||||
let provider_factory = create_test_provider_factory_with_chain_spec(chain_spec);
|
||||
let (static_dir, _) = create_test_static_files_dir();
|
||||
let db = create_test_rw_db();
|
||||
let provider_factory = ProviderFactory::new(
|
||||
db,
|
||||
chain_spec,
|
||||
StaticFileProvider::read_write(static_dir.into_path()).expect("static file provider"),
|
||||
);
|
||||
|
||||
let genesis_hash = init_genesis(provider_factory.clone())?;
|
||||
let provider =
|
||||
BlockchainProvider::new(provider_factory.clone(), Arc::new(NoopBlockchainTree::default()))?;
|
||||
|
||||
@ -606,7 +606,9 @@ mod tests {
|
||||
use reth_consensus::test_utils::TestConsensus;
|
||||
use reth_db::test_utils::{create_test_rw_db, create_test_static_files_dir};
|
||||
use reth_primitives::{BlockBody, B256};
|
||||
use reth_provider::{providers::StaticFileProvider, ProviderFactory};
|
||||
use reth_provider::{
|
||||
providers::StaticFileProvider, test_utils::MockNodeTypesWithDB, ProviderFactory,
|
||||
};
|
||||
use reth_testing_utils::generators::{self, random_block_range, BlockRangeParams};
|
||||
use std::collections::HashMap;
|
||||
|
||||
@ -628,7 +630,7 @@ mod tests {
|
||||
let mut downloader = BodiesDownloaderBuilder::default().build(
|
||||
client.clone(),
|
||||
Arc::new(TestConsensus::default()),
|
||||
ProviderFactory::new(
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
db,
|
||||
MAINNET.clone(),
|
||||
StaticFileProvider::read_write(static_dir_path).unwrap(),
|
||||
@ -682,7 +684,7 @@ mod tests {
|
||||
BodiesDownloaderBuilder::default().with_request_limit(request_limit).build(
|
||||
client.clone(),
|
||||
Arc::new(TestConsensus::default()),
|
||||
ProviderFactory::new(
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
db,
|
||||
MAINNET.clone(),
|
||||
StaticFileProvider::read_write(static_dir_path).unwrap(),
|
||||
@ -716,7 +718,7 @@ mod tests {
|
||||
.build(
|
||||
client.clone(),
|
||||
Arc::new(TestConsensus::default()),
|
||||
ProviderFactory::new(
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
db,
|
||||
MAINNET.clone(),
|
||||
StaticFileProvider::read_write(static_dir_path).unwrap(),
|
||||
@ -752,7 +754,7 @@ mod tests {
|
||||
let mut downloader = BodiesDownloaderBuilder::default().with_stream_batch_size(100).build(
|
||||
client.clone(),
|
||||
Arc::new(TestConsensus::default()),
|
||||
ProviderFactory::new(
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
db,
|
||||
MAINNET.clone(),
|
||||
StaticFileProvider::read_write(static_dir_path).unwrap(),
|
||||
@ -798,7 +800,7 @@ mod tests {
|
||||
.build(
|
||||
client.clone(),
|
||||
Arc::new(TestConsensus::default()),
|
||||
ProviderFactory::new(
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
db,
|
||||
MAINNET.clone(),
|
||||
StaticFileProvider::read_write(static_dir_path).unwrap(),
|
||||
@ -835,7 +837,7 @@ mod tests {
|
||||
.build(
|
||||
client.clone(),
|
||||
Arc::new(TestConsensus::default()),
|
||||
ProviderFactory::new(
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
db,
|
||||
MAINNET.clone(),
|
||||
StaticFileProvider::read_write(static_dir_path).unwrap(),
|
||||
|
||||
@ -14,8 +14,6 @@ workspace = true
|
||||
# reth
|
||||
reth-evm.workspace = true
|
||||
reth-provider.workspace = true
|
||||
reth-chainspec.workspace = true
|
||||
reth-db-api.workspace = true
|
||||
reth-engine-primitives.workspace = true
|
||||
reth-transaction-pool.workspace = true
|
||||
reth-payload-builder.workspace = true
|
||||
|
||||
@ -2,91 +2,16 @@
|
||||
|
||||
use std::marker::PhantomData;
|
||||
|
||||
use reth_chainspec::EthChainSpec;
|
||||
use reth_db_api::{
|
||||
database::Database,
|
||||
database_metrics::{DatabaseMetadata, DatabaseMetrics},
|
||||
};
|
||||
use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_network_api::FullNetwork;
|
||||
use reth_node_types::{NodePrimitives, NodeTypes, NodeTypesWithDB, NodeTypesWithEngine};
|
||||
use reth_node_types::{NodeTypesWithDB, NodeTypesWithEngine};
|
||||
use reth_payload_builder::PayloadBuilderHandle;
|
||||
use reth_provider::FullProvider;
|
||||
use reth_rpc_eth_api::EthApiTypes;
|
||||
use reth_tasks::TaskExecutor;
|
||||
use reth_transaction_pool::TransactionPool;
|
||||
|
||||
use crate::{ConfigureEvm, EngineTypes};
|
||||
|
||||
/// A [`NodeTypes`] type builder.
|
||||
#[derive(Default, Debug)]
|
||||
pub struct AnyNodeTypes<P = (), C = ()>(PhantomData<P>, PhantomData<C>);
|
||||
|
||||
impl<P, C> AnyNodeTypes<P, C> {
|
||||
/// Sets the `Primitives` associated type.
|
||||
pub const fn primitives<T>(self) -> AnyNodeTypes<T, C> {
|
||||
AnyNodeTypes::<T, C>(PhantomData::<T>, PhantomData::<C>)
|
||||
}
|
||||
|
||||
/// Sets the `ChainSpec` associated type.
|
||||
pub const fn chain_spec<T>(self) -> AnyNodeTypes<P, T> {
|
||||
AnyNodeTypes::<P, T>(PhantomData::<P>, PhantomData::<T>)
|
||||
}
|
||||
}
|
||||
|
||||
impl<P, C> NodeTypes for AnyNodeTypes<P, C>
|
||||
where
|
||||
P: NodePrimitives + Send + Sync + Unpin + 'static,
|
||||
C: EthChainSpec,
|
||||
{
|
||||
type Primitives = P;
|
||||
type ChainSpec = C;
|
||||
}
|
||||
|
||||
/// A [`NodeTypesWithEngine`] type builder.
|
||||
#[derive(Default, Debug)]
|
||||
pub struct AnyNodeTypesWithEngine<P = (), E = (), C = ()> {
|
||||
/// Embedding the basic node types.
|
||||
base: AnyNodeTypes<P, C>,
|
||||
/// Phantom data for the engine.
|
||||
_engine: PhantomData<E>,
|
||||
}
|
||||
|
||||
impl<P, E, C> AnyNodeTypesWithEngine<P, E, C> {
|
||||
/// Sets the `Primitives` associated type.
|
||||
pub const fn primitives<T>(self) -> AnyNodeTypesWithEngine<T, E, C> {
|
||||
AnyNodeTypesWithEngine { base: self.base.primitives::<T>(), _engine: PhantomData }
|
||||
}
|
||||
|
||||
/// Sets the `Engine` associated type.
|
||||
pub const fn engine<T>(self) -> AnyNodeTypesWithEngine<P, T, C> {
|
||||
AnyNodeTypesWithEngine { base: self.base, _engine: PhantomData::<T> }
|
||||
}
|
||||
|
||||
/// Sets the `ChainSpec` associated type.
|
||||
pub const fn chain_spec<T>(self) -> AnyNodeTypesWithEngine<P, E, T> {
|
||||
AnyNodeTypesWithEngine { base: self.base.chain_spec::<T>(), _engine: PhantomData }
|
||||
}
|
||||
}
|
||||
|
||||
impl<P, E, C> NodeTypes for AnyNodeTypesWithEngine<P, E, C>
|
||||
where
|
||||
P: NodePrimitives + Send + Sync + Unpin + 'static,
|
||||
E: EngineTypes + Send + Sync + Unpin,
|
||||
C: EthChainSpec,
|
||||
{
|
||||
type Primitives = P;
|
||||
type ChainSpec = C;
|
||||
}
|
||||
|
||||
impl<P, E, C> NodeTypesWithEngine for AnyNodeTypesWithEngine<P, E, C>
|
||||
where
|
||||
P: NodePrimitives + Send + Sync + Unpin + 'static,
|
||||
E: EngineTypes + Send + Sync + Unpin,
|
||||
C: EthChainSpec,
|
||||
{
|
||||
type Engine = E;
|
||||
}
|
||||
use crate::ConfigureEvm;
|
||||
|
||||
/// A helper trait that is downstream of the [`NodeTypesWithEngine`] trait and adds stateful
|
||||
/// components to the node.
|
||||
@ -99,57 +24,6 @@ pub trait FullNodeTypes: Send + Sync + Unpin + 'static {
|
||||
type Provider: FullProvider<Self::Types>;
|
||||
}
|
||||
|
||||
/// An adapter type combining [`NodeTypes`] and db into [`NodeTypesWithDB`].
|
||||
#[derive(Debug)]
|
||||
pub struct NodeTypesWithDBAdapter<Types, DB> {
|
||||
types: PhantomData<Types>,
|
||||
db: PhantomData<DB>,
|
||||
}
|
||||
|
||||
impl<Types, DB> NodeTypesWithDBAdapter<Types, DB> {
|
||||
/// Create a new adapter with the configured types.
|
||||
pub fn new() -> Self {
|
||||
Self { types: Default::default(), db: Default::default() }
|
||||
}
|
||||
}
|
||||
|
||||
impl<Types, DB> Default for NodeTypesWithDBAdapter<Types, DB> {
|
||||
fn default() -> Self {
|
||||
Self::new()
|
||||
}
|
||||
}
|
||||
|
||||
impl<Types, DB> Clone for NodeTypesWithDBAdapter<Types, DB> {
|
||||
fn clone(&self) -> Self {
|
||||
Self { types: self.types, db: self.db }
|
||||
}
|
||||
}
|
||||
|
||||
impl<Types, DB> NodeTypes for NodeTypesWithDBAdapter<Types, DB>
|
||||
where
|
||||
Types: NodeTypes,
|
||||
DB: Send + Sync + Unpin + 'static,
|
||||
{
|
||||
type Primitives = Types::Primitives;
|
||||
type ChainSpec = Types::ChainSpec;
|
||||
}
|
||||
|
||||
impl<Types, DB> NodeTypesWithEngine for NodeTypesWithDBAdapter<Types, DB>
|
||||
where
|
||||
Types: NodeTypesWithEngine,
|
||||
DB: Send + Sync + Unpin + 'static,
|
||||
{
|
||||
type Engine = Types::Engine;
|
||||
}
|
||||
|
||||
impl<Types, DB> NodeTypesWithDB for NodeTypesWithDBAdapter<Types, DB>
|
||||
where
|
||||
Types: NodeTypesWithEngine,
|
||||
DB: Database + DatabaseMetrics + DatabaseMetadata + Clone + Unpin + 'static,
|
||||
{
|
||||
type DB = DB;
|
||||
}
|
||||
|
||||
/// An adapter type that adds the builtin provider type to the user configured node types.
|
||||
#[derive(Debug)]
|
||||
pub struct FullNodeTypesAdapter<Types, Provider> {
|
||||
|
||||
@ -49,8 +49,10 @@ use crate::{
|
||||
|
||||
/// The adapter type for a reth node with the builtin provider type
|
||||
// Note: we need to hardcode this because custom components might depend on it in associated types.
|
||||
pub type RethFullAdapter<DB, Types> =
|
||||
FullNodeTypesAdapter<NodeTypesWithDBAdapter<Types, DB>, BlockchainProvider<DB>>;
|
||||
pub type RethFullAdapter<DB, Types> = FullNodeTypesAdapter<
|
||||
NodeTypesWithDBAdapter<Types, DB>,
|
||||
BlockchainProvider<NodeTypesWithDBAdapter<Types, DB>>,
|
||||
>;
|
||||
|
||||
#[allow(clippy::doc_markdown)]
|
||||
#[cfg_attr(doc, aquamarine::aquamarine)]
|
||||
|
||||
@ -1,6 +1,6 @@
|
||||
//! Helper types that can be used by launchers.
|
||||
|
||||
use std::{marker::PhantomData, sync::Arc, thread::available_parallelism};
|
||||
use std::{sync::Arc, thread::available_parallelism};
|
||||
|
||||
use eyre::Context;
|
||||
use rayon::ThreadPoolBuilder;
|
||||
@ -12,13 +12,13 @@ use reth_blockchain_tree::{
|
||||
use reth_chainspec::{Chain, ChainSpec};
|
||||
use reth_config::{config::EtlConfig, PruneConfig};
|
||||
use reth_consensus::Consensus;
|
||||
use reth_db_api::{database::Database, database_metrics::DatabaseMetrics};
|
||||
use reth_db_api::database::Database;
|
||||
use reth_db_common::init::{init_genesis, InitDatabaseError};
|
||||
use reth_downloaders::{bodies::noop::NoopBodiesDownloader, headers::noop::NoopHeaderDownloader};
|
||||
use reth_engine_tree::tree::{InvalidBlockHook, InvalidBlockHooks, NoopInvalidBlockHook};
|
||||
use reth_evm::noop::NoopBlockExecutorProvider;
|
||||
use reth_network_p2p::headers::client::HeadersClient;
|
||||
use reth_node_api::FullNodeTypes;
|
||||
use reth_node_api::{FullNodeTypes, NodeTypes, NodeTypesWithDB};
|
||||
use reth_node_core::{
|
||||
dirs::{ChainPath, DataDirPath},
|
||||
node_config::NodeConfig,
|
||||
@ -65,13 +65,13 @@ pub trait WithTree {
|
||||
fn set_tree(self, tree: Arc<dyn TreeViewer>) -> Self;
|
||||
}
|
||||
|
||||
impl<DB: Database> WithTree for BlockchainProvider<DB> {
|
||||
impl<N: NodeTypesWithDB> WithTree for BlockchainProvider<N> {
|
||||
fn set_tree(self, tree: Arc<dyn TreeViewer>) -> Self {
|
||||
self.with_tree(tree)
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> WithTree for BlockchainProvider2<DB> {
|
||||
impl<N: NodeTypesWithDB> WithTree for BlockchainProvider2<N> {
|
||||
fn set_tree(self, _tree: Arc<dyn TreeViewer>) -> Self {
|
||||
self
|
||||
}
|
||||
@ -386,7 +386,9 @@ where
|
||||
/// Returns the [`ProviderFactory`] for the attached storage after executing a consistent check
|
||||
/// between the database and static files. **It may execute a pipeline unwind if it fails this
|
||||
/// check.**
|
||||
pub async fn create_provider_factory(&self) -> eyre::Result<ProviderFactory<DB>> {
|
||||
pub async fn create_provider_factory<N: NodeTypesWithDB<DB = DB, ChainSpec = ChainSpec>>(
|
||||
&self,
|
||||
) -> eyre::Result<ProviderFactory<N>> {
|
||||
let factory = ProviderFactory::new(
|
||||
self.right().clone(),
|
||||
self.chain_spec(),
|
||||
@ -413,7 +415,7 @@ where
|
||||
let (_tip_tx, tip_rx) = watch::channel(B256::ZERO);
|
||||
|
||||
// Builds an unwind-only pipeline
|
||||
let pipeline = Pipeline::builder()
|
||||
let pipeline = Pipeline::<N>::builder()
|
||||
.add_stages(DefaultStages::new(
|
||||
factory.clone(),
|
||||
tip_rx,
|
||||
@ -447,9 +449,9 @@ where
|
||||
}
|
||||
|
||||
/// Creates a new [`ProviderFactory`] and attaches it to the launch context.
|
||||
pub async fn with_provider_factory(
|
||||
pub async fn with_provider_factory<N: NodeTypesWithDB<DB = DB, ChainSpec = ChainSpec>>(
|
||||
self,
|
||||
) -> eyre::Result<LaunchContextWith<Attached<WithConfigs, ProviderFactory<DB>>>> {
|
||||
) -> eyre::Result<LaunchContextWith<Attached<WithConfigs, ProviderFactory<N>>>> {
|
||||
let factory = self.create_provider_factory().await?;
|
||||
let ctx = LaunchContextWith {
|
||||
inner: self.inner,
|
||||
@ -460,17 +462,17 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> LaunchContextWith<Attached<WithConfigs, ProviderFactory<DB>>>
|
||||
impl<T> LaunchContextWith<Attached<WithConfigs, ProviderFactory<T>>>
|
||||
where
|
||||
DB: Database + DatabaseMetrics + Send + Sync + Clone + 'static,
|
||||
T: NodeTypesWithDB<ChainSpec = ChainSpec>,
|
||||
{
|
||||
/// Returns access to the underlying database.
|
||||
pub fn database(&self) -> &DB {
|
||||
pub const fn database(&self) -> &T::DB {
|
||||
self.right().db_ref()
|
||||
}
|
||||
|
||||
/// Returns the configured `ProviderFactory`.
|
||||
pub const fn provider_factory(&self) -> &ProviderFactory<DB> {
|
||||
pub const fn provider_factory(&self) -> &ProviderFactory<T> {
|
||||
self.right()
|
||||
}
|
||||
|
||||
@ -530,7 +532,7 @@ where
|
||||
/// prometheus.
|
||||
pub fn with_metrics_task(
|
||||
self,
|
||||
) -> LaunchContextWith<Attached<WithConfigs, WithMeteredProvider<DB>>> {
|
||||
) -> LaunchContextWith<Attached<WithConfigs, WithMeteredProvider<T>>> {
|
||||
let (metrics_sender, metrics_receiver) = unbounded_channel();
|
||||
|
||||
let with_metrics =
|
||||
@ -547,12 +549,12 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> LaunchContextWith<Attached<WithConfigs, WithMeteredProvider<DB>>>
|
||||
impl<N> LaunchContextWith<Attached<WithConfigs, WithMeteredProvider<N>>>
|
||||
where
|
||||
DB: Database + DatabaseMetrics + Send + Sync + Clone + 'static,
|
||||
N: NodeTypesWithDB,
|
||||
{
|
||||
/// Returns the configured `ProviderFactory`.
|
||||
const fn provider_factory(&self) -> &ProviderFactory<DB> {
|
||||
const fn provider_factory(&self) -> &ProviderFactory<N> {
|
||||
&self.right().provider_factory
|
||||
}
|
||||
|
||||
@ -567,10 +569,10 @@ where
|
||||
create_blockchain_provider: F,
|
||||
tree_config: BlockchainTreeConfig,
|
||||
canon_state_notification_sender: CanonStateNotificationSender,
|
||||
) -> eyre::Result<LaunchContextWith<Attached<WithConfigs, WithMeteredProviders<DB, T>>>>
|
||||
) -> eyre::Result<LaunchContextWith<Attached<WithConfigs, WithMeteredProviders<T>>>>
|
||||
where
|
||||
T: FullNodeTypes,
|
||||
F: FnOnce(ProviderFactory<DB>) -> eyre::Result<T::Provider>,
|
||||
T: FullNodeTypes<Types = N>,
|
||||
F: FnOnce(ProviderFactory<N>) -> eyre::Result<T::Provider>,
|
||||
{
|
||||
let blockchain_db = create_blockchain_provider(self.provider_factory().clone())?;
|
||||
|
||||
@ -582,8 +584,6 @@ where
|
||||
blockchain_db,
|
||||
tree_config,
|
||||
canon_state_notification_sender,
|
||||
// we store here a reference to T.
|
||||
phantom_data: PhantomData,
|
||||
};
|
||||
|
||||
let ctx = LaunchContextWith {
|
||||
@ -595,18 +595,17 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB, T> LaunchContextWith<Attached<WithConfigs, WithMeteredProviders<DB, T>>>
|
||||
impl<T> LaunchContextWith<Attached<WithConfigs, WithMeteredProviders<T>>>
|
||||
where
|
||||
DB: Database + DatabaseMetrics + Send + Sync + Clone + 'static,
|
||||
T: FullNodeTypes<Provider: WithTree>,
|
||||
T: FullNodeTypes<Types: NodeTypesWithDB<ChainSpec = ChainSpec>, Provider: WithTree>,
|
||||
{
|
||||
/// Returns access to the underlying database.
|
||||
pub fn database(&self) -> &DB {
|
||||
pub const fn database(&self) -> &<T::Types as NodeTypesWithDB>::DB {
|
||||
self.provider_factory().db_ref()
|
||||
}
|
||||
|
||||
/// Returns the configured `ProviderFactory`.
|
||||
pub const fn provider_factory(&self) -> &ProviderFactory<DB> {
|
||||
pub const fn provider_factory(&self) -> &ProviderFactory<T::Types> {
|
||||
&self.right().db_provider_container.provider_factory
|
||||
}
|
||||
|
||||
@ -646,7 +645,7 @@ where
|
||||
on_component_initialized: Box<
|
||||
dyn OnComponentInitializedHook<NodeAdapter<T, CB::Components>>,
|
||||
>,
|
||||
) -> eyre::Result<LaunchContextWith<Attached<WithConfigs, WithComponents<DB, T, CB>>>>
|
||||
) -> eyre::Result<LaunchContextWith<Attached<WithConfigs, WithComponents<T, CB>>>>
|
||||
where
|
||||
CB: NodeComponentsBuilder<T>,
|
||||
{
|
||||
@ -714,14 +713,13 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB, T, CB> LaunchContextWith<Attached<WithConfigs, WithComponents<DB, T, CB>>>
|
||||
impl<T, CB> LaunchContextWith<Attached<WithConfigs, WithComponents<T, CB>>>
|
||||
where
|
||||
DB: Database + DatabaseMetrics + Send + Sync + Clone + 'static,
|
||||
T: FullNodeTypes<Provider: WithTree>,
|
||||
T: FullNodeTypes<Provider: WithTree, Types: NodeTypes<ChainSpec = ChainSpec>>,
|
||||
CB: NodeComponentsBuilder<T>,
|
||||
{
|
||||
/// Returns the configured `ProviderFactory`.
|
||||
pub const fn provider_factory(&self) -> &ProviderFactory<DB> {
|
||||
pub const fn provider_factory(&self) -> &ProviderFactory<T::Types> {
|
||||
&self.right().db_provider_container.provider_factory
|
||||
}
|
||||
|
||||
@ -740,7 +738,7 @@ where
|
||||
}
|
||||
|
||||
/// Creates a new [`StaticFileProducer`] with the attached database.
|
||||
pub fn static_file_producer(&self) -> StaticFileProducer<DB> {
|
||||
pub fn static_file_producer(&self) -> StaticFileProducer<T::Types> {
|
||||
StaticFileProducer::new(self.provider_factory().clone(), self.prune_modes())
|
||||
}
|
||||
|
||||
@ -929,37 +927,32 @@ pub struct WithConfigs {
|
||||
/// Helper container type to bundle the [`ProviderFactory`] and the metrics
|
||||
/// sender.
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct WithMeteredProvider<DB> {
|
||||
provider_factory: ProviderFactory<DB>,
|
||||
pub struct WithMeteredProvider<N: NodeTypesWithDB> {
|
||||
provider_factory: ProviderFactory<N>,
|
||||
metrics_sender: UnboundedSender<MetricEvent>,
|
||||
}
|
||||
|
||||
/// Helper container to bundle the [`ProviderFactory`], [`BlockchainProvider`]
|
||||
/// and a metrics sender.
|
||||
#[allow(missing_debug_implementations)]
|
||||
pub struct WithMeteredProviders<DB, T>
|
||||
pub struct WithMeteredProviders<T>
|
||||
where
|
||||
DB: Database,
|
||||
T: FullNodeTypes,
|
||||
{
|
||||
db_provider_container: WithMeteredProvider<DB>,
|
||||
db_provider_container: WithMeteredProvider<T::Types>,
|
||||
blockchain_db: T::Provider,
|
||||
canon_state_notification_sender: CanonStateNotificationSender,
|
||||
tree_config: BlockchainTreeConfig,
|
||||
// this field is used to store a reference to the FullNodeTypes so that we
|
||||
// can build the components in `with_components` method.
|
||||
phantom_data: PhantomData<T>,
|
||||
}
|
||||
|
||||
/// Helper container to bundle the metered providers container and [`NodeAdapter`].
|
||||
#[allow(missing_debug_implementations)]
|
||||
pub struct WithComponents<DB, T, CB>
|
||||
pub struct WithComponents<T, CB>
|
||||
where
|
||||
DB: Database,
|
||||
T: FullNodeTypes,
|
||||
CB: NodeComponentsBuilder<T>,
|
||||
{
|
||||
db_provider_container: WithMeteredProvider<DB>,
|
||||
db_provider_container: WithMeteredProvider<T::Types>,
|
||||
tree_config: BlockchainTreeConfig,
|
||||
blockchain_db: T::Provider,
|
||||
node_adapter: NodeAdapter<T, CB::Components>,
|
||||
|
||||
@ -59,7 +59,7 @@ impl EngineNodeLauncher {
|
||||
impl<Types, T, CB, AO> LaunchNode<NodeBuilderWithComponents<T, CB, AO>> for EngineNodeLauncher
|
||||
where
|
||||
Types: NodeTypesWithDB<ChainSpec = ChainSpec>,
|
||||
T: FullNodeTypes<Types = Types, Provider = BlockchainProvider2<Types::DB>>,
|
||||
T: FullNodeTypes<Types = Types, Provider = BlockchainProvider2<Types>>,
|
||||
CB: NodeComponentsBuilder<T>,
|
||||
AO: NodeAddOns<
|
||||
NodeAdapter<T, CB::Components>,
|
||||
|
||||
@ -103,7 +103,7 @@ impl DefaultNodeLauncher {
|
||||
impl<Types, T, CB, AO> LaunchNode<NodeBuilderWithComponents<T, CB, AO>> for DefaultNodeLauncher
|
||||
where
|
||||
Types: NodeTypesWithDB<ChainSpec = ChainSpec>,
|
||||
T: FullNodeTypes<Provider = BlockchainProvider<Types::DB>, Types = Types>,
|
||||
T: FullNodeTypes<Provider = BlockchainProvider<Types>, Types = Types>,
|
||||
CB: NodeComponentsBuilder<T>,
|
||||
AO: NodeAddOns<
|
||||
NodeAdapter<T, CB::Components>,
|
||||
|
||||
@ -4,7 +4,6 @@ use std::sync::Arc;
|
||||
|
||||
use reth_config::{config::StageConfig, PruneConfig};
|
||||
use reth_consensus::Consensus;
|
||||
use reth_db_api::database::Database;
|
||||
use reth_downloaders::{
|
||||
bodies::bodies::BodiesDownloaderBuilder,
|
||||
headers::reverse_headers::ReverseHeadersDownloaderBuilder,
|
||||
@ -15,7 +14,7 @@ use reth_network_p2p::{
|
||||
bodies::downloader::BodyDownloader, headers::downloader::HeaderDownloader, BlockClient,
|
||||
};
|
||||
use reth_node_core::primitives::{BlockNumber, B256};
|
||||
use reth_provider::ProviderFactory;
|
||||
use reth_provider::{providers::ProviderNodeTypes, ProviderFactory};
|
||||
use reth_stages::{prelude::DefaultStages, stages::ExecutionStage, Pipeline, StageSet};
|
||||
use reth_static_file::StaticFileProducer;
|
||||
use reth_tasks::TaskExecutor;
|
||||
@ -24,21 +23,21 @@ use tokio::sync::watch;
|
||||
|
||||
/// Constructs a [Pipeline] that's wired to the network
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
pub fn build_networked_pipeline<DB, Client, Executor>(
|
||||
pub fn build_networked_pipeline<N, Client, Executor>(
|
||||
config: &StageConfig,
|
||||
client: Client,
|
||||
consensus: Arc<dyn Consensus>,
|
||||
provider_factory: ProviderFactory<DB>,
|
||||
provider_factory: ProviderFactory<N>,
|
||||
task_executor: &TaskExecutor,
|
||||
metrics_tx: reth_stages::MetricEventsSender,
|
||||
prune_config: Option<PruneConfig>,
|
||||
max_block: Option<BlockNumber>,
|
||||
static_file_producer: StaticFileProducer<DB>,
|
||||
static_file_producer: StaticFileProducer<N>,
|
||||
executor: Executor,
|
||||
exex_manager_handle: ExExManagerHandle,
|
||||
) -> eyre::Result<Pipeline<DB>>
|
||||
) -> eyre::Result<Pipeline<N>>
|
||||
where
|
||||
DB: Database + Unpin + Clone + 'static,
|
||||
N: ProviderNodeTypes,
|
||||
Client: BlockClient + 'static,
|
||||
Executor: BlockExecutorProvider,
|
||||
{
|
||||
@ -70,8 +69,8 @@ where
|
||||
|
||||
/// Builds the [Pipeline] with the given [`ProviderFactory`] and downloaders.
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
pub fn build_pipeline<DB, H, B, Executor>(
|
||||
provider_factory: ProviderFactory<DB>,
|
||||
pub fn build_pipeline<N, H, B, Executor>(
|
||||
provider_factory: ProviderFactory<N>,
|
||||
stage_config: &StageConfig,
|
||||
header_downloader: H,
|
||||
body_downloader: B,
|
||||
@ -79,17 +78,17 @@ pub fn build_pipeline<DB, H, B, Executor>(
|
||||
max_block: Option<u64>,
|
||||
metrics_tx: reth_stages::MetricEventsSender,
|
||||
prune_config: Option<PruneConfig>,
|
||||
static_file_producer: StaticFileProducer<DB>,
|
||||
static_file_producer: StaticFileProducer<N>,
|
||||
executor: Executor,
|
||||
exex_manager_handle: ExExManagerHandle,
|
||||
) -> eyre::Result<Pipeline<DB>>
|
||||
) -> eyre::Result<Pipeline<N>>
|
||||
where
|
||||
DB: Database + Clone + 'static,
|
||||
N: ProviderNodeTypes,
|
||||
H: HeaderDownloader + 'static,
|
||||
B: BodyDownloader + 'static,
|
||||
Executor: BlockExecutorProvider,
|
||||
{
|
||||
let mut builder = Pipeline::builder();
|
||||
let mut builder = Pipeline::<N>::builder();
|
||||
|
||||
if let Some(max_block) = max_block {
|
||||
debug!(target: "reth::cli", max_block, "Configuring builder to use max block");
|
||||
|
||||
@ -40,6 +40,7 @@ reth-consensus-common.workspace = true
|
||||
reth-prune-types.workspace = true
|
||||
reth-stages-types.workspace = true
|
||||
reth-optimism-chainspec = { workspace = true, optional = true }
|
||||
reth-node-types.workspace = true
|
||||
|
||||
# ethereum
|
||||
alloy-genesis.workspace = true
|
||||
|
||||
@ -11,11 +11,11 @@ use crate::{
|
||||
use eyre::eyre;
|
||||
use reth_chainspec::{ChainSpec, MAINNET};
|
||||
use reth_config::config::PruneConfig;
|
||||
use reth_db_api::database::Database;
|
||||
use reth_network_p2p::headers::client::HeadersClient;
|
||||
use serde::{de::DeserializeOwned, Serialize};
|
||||
use std::{fs, path::Path};
|
||||
|
||||
use reth_node_types::NodeTypesWithDB;
|
||||
use reth_primitives::{
|
||||
revm_primitives::EnvKzgSettings, BlockHashOrNumber, BlockNumber, Head, SealedHeader, B256,
|
||||
};
|
||||
@ -269,7 +269,10 @@ impl NodeConfig {
|
||||
/// Fetches the head block from the database.
|
||||
///
|
||||
/// If the database is empty, returns the genesis block.
|
||||
pub fn lookup_head<DB: Database>(&self, factory: ProviderFactory<DB>) -> ProviderResult<Head> {
|
||||
pub fn lookup_head<N: NodeTypesWithDB<ChainSpec = ChainSpec>>(
|
||||
&self,
|
||||
factory: ProviderFactory<N>,
|
||||
) -> ProviderResult<Head> {
|
||||
let provider = factory.provider()?;
|
||||
|
||||
let head = provider.get_stage_checkpoint(StageId::Finish)?.unwrap_or_default().block_number;
|
||||
|
||||
@ -34,10 +34,10 @@ tikv-jemalloc-ctl = { version = "0.5.0", optional = true }
|
||||
procfs = "0.16.0"
|
||||
|
||||
[dev-dependencies]
|
||||
reth-db = { workspace = true, features = ["test-utils"] }
|
||||
reqwest.workspace = true
|
||||
reth-chainspec.workspace = true
|
||||
socket2 = { version = "0.4", default-features = false }
|
||||
reth-provider = { workspace = true, features = ["test-utils"] }
|
||||
|
||||
[lints]
|
||||
workspace = true
|
||||
|
||||
@ -204,27 +204,11 @@ const fn describe_io_stats() {}
|
||||
mod tests {
|
||||
use super::*;
|
||||
use reqwest::Client;
|
||||
use reth_chainspec::MAINNET;
|
||||
use reth_db::{
|
||||
test_utils::{create_test_rw_db, create_test_static_files_dir, TempDatabase},
|
||||
DatabaseEnv,
|
||||
};
|
||||
use reth_provider::{
|
||||
providers::StaticFileProvider, ProviderFactory, StaticFileProviderFactory,
|
||||
};
|
||||
use reth_provider::{test_utils::create_test_provider_factory, StaticFileProviderFactory};
|
||||
use reth_tasks::TaskManager;
|
||||
use socket2::{Domain, Socket, Type};
|
||||
use std::net::{SocketAddr, TcpListener};
|
||||
|
||||
fn create_test_db() -> ProviderFactory<Arc<TempDatabase<DatabaseEnv>>> {
|
||||
let (_, static_dir_path) = create_test_static_files_dir();
|
||||
ProviderFactory::new(
|
||||
create_test_rw_db(),
|
||||
MAINNET.clone(),
|
||||
StaticFileProvider::read_write(static_dir_path).unwrap(),
|
||||
)
|
||||
}
|
||||
|
||||
fn get_random_available_addr() -> SocketAddr {
|
||||
let addr = &"127.0.0.1:0".parse::<SocketAddr>().unwrap().into();
|
||||
let socket = Socket::new(Domain::IPV4, Type::STREAM, None).unwrap();
|
||||
@ -249,7 +233,7 @@ mod tests {
|
||||
let tasks = TaskManager::current();
|
||||
let executor = tasks.executor();
|
||||
|
||||
let factory = create_test_db();
|
||||
let factory = create_test_provider_factory();
|
||||
let hooks = Hooks::new(factory.db_ref().clone(), factory.static_file_provider());
|
||||
|
||||
let listen_addr = get_random_available_addr();
|
||||
|
||||
@ -8,6 +8,8 @@
|
||||
#![cfg_attr(not(test), warn(unused_crate_dependencies))]
|
||||
#![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))]
|
||||
|
||||
use std::marker::PhantomData;
|
||||
|
||||
use reth_chainspec::EthChainSpec;
|
||||
use reth_db_api::{
|
||||
database_metrics::{DatabaseMetadata, DatabaseMetrics},
|
||||
@ -48,3 +50,124 @@ pub trait NodeTypesWithDB: NodeTypesWithEngine {
|
||||
/// Underlying database type used by the node to store and retrieve data.
|
||||
type DB: Database + DatabaseMetrics + DatabaseMetadata + Clone + Unpin + 'static;
|
||||
}
|
||||
|
||||
/// An adapter type combining [`NodeTypes`] and db into [`NodeTypesWithDB`].
|
||||
#[derive(Debug)]
|
||||
pub struct NodeTypesWithDBAdapter<Types, DB> {
|
||||
types: PhantomData<Types>,
|
||||
db: PhantomData<DB>,
|
||||
}
|
||||
|
||||
impl<Types, DB> NodeTypesWithDBAdapter<Types, DB> {
|
||||
/// Create a new adapter with the configured types.
|
||||
pub fn new() -> Self {
|
||||
Self { types: Default::default(), db: Default::default() }
|
||||
}
|
||||
}
|
||||
|
||||
impl<Types, DB> Default for NodeTypesWithDBAdapter<Types, DB> {
|
||||
fn default() -> Self {
|
||||
Self::new()
|
||||
}
|
||||
}
|
||||
|
||||
impl<Types, DB> Clone for NodeTypesWithDBAdapter<Types, DB> {
|
||||
fn clone(&self) -> Self {
|
||||
Self { types: self.types, db: self.db }
|
||||
}
|
||||
}
|
||||
|
||||
impl<Types, DB> NodeTypes for NodeTypesWithDBAdapter<Types, DB>
|
||||
where
|
||||
Types: NodeTypes,
|
||||
DB: Send + Sync + Unpin + 'static,
|
||||
{
|
||||
type Primitives = Types::Primitives;
|
||||
type ChainSpec = Types::ChainSpec;
|
||||
}
|
||||
|
||||
impl<Types, DB> NodeTypesWithEngine for NodeTypesWithDBAdapter<Types, DB>
|
||||
where
|
||||
Types: NodeTypesWithEngine,
|
||||
DB: Send + Sync + Unpin + 'static,
|
||||
{
|
||||
type Engine = Types::Engine;
|
||||
}
|
||||
|
||||
impl<Types, DB> NodeTypesWithDB for NodeTypesWithDBAdapter<Types, DB>
|
||||
where
|
||||
Types: NodeTypesWithEngine,
|
||||
DB: Database + DatabaseMetrics + DatabaseMetadata + Clone + Unpin + 'static,
|
||||
{
|
||||
type DB = DB;
|
||||
}
|
||||
|
||||
/// A [`NodeTypes`] type builder.
|
||||
#[derive(Default, Debug)]
|
||||
pub struct AnyNodeTypes<P = (), C = ()>(PhantomData<P>, PhantomData<C>);
|
||||
|
||||
impl<P, C> AnyNodeTypes<P, C> {
|
||||
/// Sets the `Primitives` associated type.
|
||||
pub const fn primitives<T>(self) -> AnyNodeTypes<T, C> {
|
||||
AnyNodeTypes::<T, C>(PhantomData::<T>, PhantomData::<C>)
|
||||
}
|
||||
|
||||
/// Sets the `ChainSpec` associated type.
|
||||
pub const fn chain_spec<T>(self) -> AnyNodeTypes<P, T> {
|
||||
AnyNodeTypes::<P, T>(PhantomData::<P>, PhantomData::<T>)
|
||||
}
|
||||
}
|
||||
|
||||
impl<P, C> NodeTypes for AnyNodeTypes<P, C>
|
||||
where
|
||||
P: NodePrimitives + Send + Sync + Unpin + 'static,
|
||||
C: EthChainSpec,
|
||||
{
|
||||
type Primitives = P;
|
||||
type ChainSpec = C;
|
||||
}
|
||||
|
||||
/// A [`NodeTypesWithEngine`] type builder.
|
||||
#[derive(Default, Debug)]
|
||||
pub struct AnyNodeTypesWithEngine<P = (), E = (), C = ()> {
|
||||
/// Embedding the basic node types.
|
||||
base: AnyNodeTypes<P, C>,
|
||||
/// Phantom data for the engine.
|
||||
_engine: PhantomData<E>,
|
||||
}
|
||||
|
||||
impl<P, E, C> AnyNodeTypesWithEngine<P, E, C> {
|
||||
/// Sets the `Primitives` associated type.
|
||||
pub const fn primitives<T>(self) -> AnyNodeTypesWithEngine<T, E, C> {
|
||||
AnyNodeTypesWithEngine { base: self.base.primitives::<T>(), _engine: PhantomData }
|
||||
}
|
||||
|
||||
/// Sets the `Engine` associated type.
|
||||
pub const fn engine<T>(self) -> AnyNodeTypesWithEngine<P, T, C> {
|
||||
AnyNodeTypesWithEngine { base: self.base, _engine: PhantomData::<T> }
|
||||
}
|
||||
|
||||
/// Sets the `ChainSpec` associated type.
|
||||
pub const fn chain_spec<T>(self) -> AnyNodeTypesWithEngine<P, E, T> {
|
||||
AnyNodeTypesWithEngine { base: self.base.chain_spec::<T>(), _engine: PhantomData }
|
||||
}
|
||||
}
|
||||
|
||||
impl<P, E, C> NodeTypes for AnyNodeTypesWithEngine<P, E, C>
|
||||
where
|
||||
P: NodePrimitives + Send + Sync + Unpin + 'static,
|
||||
E: EngineTypes + Send + Sync + Unpin,
|
||||
C: EthChainSpec,
|
||||
{
|
||||
type Primitives = P;
|
||||
type ChainSpec = C;
|
||||
}
|
||||
|
||||
impl<P, E, C> NodeTypesWithEngine for AnyNodeTypesWithEngine<P, E, C>
|
||||
where
|
||||
P: NodePrimitives + Send + Sync + Unpin + 'static,
|
||||
E: EngineTypes + Send + Sync + Unpin,
|
||||
C: EthChainSpec,
|
||||
{
|
||||
type Engine = E;
|
||||
}
|
||||
|
||||
@ -23,6 +23,7 @@ reth-stages.workspace = true
|
||||
reth-static-file.workspace = true
|
||||
reth-execution-types.workspace = true
|
||||
reth-node-core.workspace = true
|
||||
reth-node-optimism.workspace = true
|
||||
reth-primitives.workspace = true
|
||||
|
||||
## optimisim
|
||||
@ -72,4 +73,5 @@ reth-db-common.workspace = true
|
||||
"reth-primitives/optimism",
|
||||
"reth-evm-optimism/optimism",
|
||||
"reth-provider/optimism",
|
||||
"reth-node-optimism/optimism",
|
||||
]
|
||||
|
||||
@ -1,8 +1,8 @@
|
||||
use alloy_primitives::B256;
|
||||
use futures_util::{Stream, StreamExt};
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_config::Config;
|
||||
use reth_consensus::Consensus;
|
||||
use reth_db_api::database::Database;
|
||||
use reth_downloaders::{
|
||||
bodies::bodies::BodiesDownloaderBuilder, file_client::FileClient,
|
||||
headers::reverse_headers::ReverseHeadersDownloaderBuilder,
|
||||
@ -13,6 +13,7 @@ use reth_network_p2p::{
|
||||
bodies::downloader::BodyDownloader,
|
||||
headers::downloader::{HeaderDownloader, SyncTarget},
|
||||
};
|
||||
use reth_node_builder::NodeTypesWithDB;
|
||||
use reth_node_events::node::NodeEvent;
|
||||
use reth_provider::{BlockNumReader, ChainSpecProvider, HeaderProvider, ProviderFactory};
|
||||
use reth_prune::PruneModes;
|
||||
@ -26,16 +27,16 @@ use tokio::sync::watch;
|
||||
///
|
||||
/// If configured to execute, all stages will run. Otherwise, only stages that don't require state
|
||||
/// will run.
|
||||
pub(crate) async fn build_import_pipeline<DB, C>(
|
||||
pub(crate) async fn build_import_pipeline<N, C>(
|
||||
config: &Config,
|
||||
provider_factory: ProviderFactory<DB>,
|
||||
provider_factory: ProviderFactory<N>,
|
||||
consensus: &Arc<C>,
|
||||
file_client: Arc<FileClient>,
|
||||
static_file_producer: StaticFileProducer<DB>,
|
||||
static_file_producer: StaticFileProducer<N>,
|
||||
disable_exec: bool,
|
||||
) -> eyre::Result<(Pipeline<DB>, impl Stream<Item = NodeEvent>)>
|
||||
) -> eyre::Result<(Pipeline<N>, impl Stream<Item = NodeEvent>)>
|
||||
where
|
||||
DB: Database + Clone + Unpin + 'static,
|
||||
N: NodeTypesWithDB<ChainSpec = ChainSpec>,
|
||||
C: Consensus + 'static,
|
||||
{
|
||||
if !file_client.has_canonical_blocks() {
|
||||
@ -70,7 +71,7 @@ where
|
||||
|
||||
let max_block = file_client.max_block().unwrap_or(0);
|
||||
|
||||
let pipeline = Pipeline::builder()
|
||||
let pipeline = Pipeline::<N>::builder()
|
||||
.with_tip_sender(tip_tx)
|
||||
// we want to sync all blocks the file client provides or 0 if empty
|
||||
.with_max_block(max_block)
|
||||
|
||||
@ -10,6 +10,7 @@ use reth_db_api::transaction::DbTx;
|
||||
use reth_downloaders::file_client::{
|
||||
ChunkedFileReader, FileClient, DEFAULT_BYTE_LEN_CHUNK_CHAIN_FILE,
|
||||
};
|
||||
use reth_node_builder::NodeTypesWithEngine;
|
||||
use reth_node_core::version::SHORT_VERSION;
|
||||
use reth_optimism_primitives::bedrock_import::is_dup_tx;
|
||||
use reth_provider::StageCheckpointReader;
|
||||
@ -41,7 +42,9 @@ pub struct ImportOpCommand<C: ChainSpecParser> {
|
||||
|
||||
impl<C: ChainSpecParser<ChainSpec = ChainSpec>> ImportOpCommand<C> {
|
||||
/// Execute `import` command
|
||||
pub async fn execute(self) -> eyre::Result<()> {
|
||||
pub async fn execute<N: NodeTypesWithEngine<ChainSpec = C::ChainSpec>>(
|
||||
self,
|
||||
) -> eyre::Result<()> {
|
||||
info!(target: "reth::cli", "reth {} starting", SHORT_VERSION);
|
||||
|
||||
info!(target: "reth::cli",
|
||||
@ -53,7 +56,7 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> ImportOpCommand<C> {
|
||||
"Chunking chain import"
|
||||
);
|
||||
|
||||
let Environment { provider_factory, config, .. } = self.env.init(AccessRights::RW)?;
|
||||
let Environment { provider_factory, config, .. } = self.env.init::<N>(AccessRights::RW)?;
|
||||
|
||||
// we use noop here because we expect the inputs to be valid
|
||||
let consensus = Arc::new(NoopConsensus::default());
|
||||
|
||||
@ -8,12 +8,12 @@ use reth_chainspec::ChainSpec;
|
||||
use reth_cli::chainspec::ChainSpecParser;
|
||||
use reth_cli_commands::common::{AccessRights, Environment, EnvironmentArgs};
|
||||
use reth_db::tables;
|
||||
use reth_db_api::database::Database;
|
||||
use reth_downloaders::{
|
||||
file_client::{ChunkedFileReader, DEFAULT_BYTE_LEN_CHUNK_CHAIN_FILE},
|
||||
receipt_file_client::ReceiptFileClient,
|
||||
};
|
||||
use reth_execution_types::ExecutionOutcome;
|
||||
use reth_node_builder::{NodeTypesWithDB, NodeTypesWithEngine};
|
||||
use reth_node_core::version::SHORT_VERSION;
|
||||
use reth_optimism_primitives::bedrock_import::is_dup_tx;
|
||||
use reth_primitives::Receipts;
|
||||
@ -47,7 +47,9 @@ pub struct ImportReceiptsOpCommand<C: ChainSpecParser> {
|
||||
|
||||
impl<C: ChainSpecParser<ChainSpec = ChainSpec>> ImportReceiptsOpCommand<C> {
|
||||
/// Execute `import` command
|
||||
pub async fn execute(self) -> eyre::Result<()> {
|
||||
pub async fn execute<N: NodeTypesWithEngine<ChainSpec = C::ChainSpec>>(
|
||||
self,
|
||||
) -> eyre::Result<()> {
|
||||
info!(target: "reth::cli", "reth {} starting", SHORT_VERSION);
|
||||
|
||||
debug!(target: "reth::cli",
|
||||
@ -55,7 +57,7 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> ImportReceiptsOpCommand<C> {
|
||||
"Chunking receipts import"
|
||||
);
|
||||
|
||||
let Environment { provider_factory, .. } = self.env.init(AccessRights::RW)?;
|
||||
let Environment { provider_factory, .. } = self.env.init::<N>(AccessRights::RW)?;
|
||||
|
||||
import_receipts_from_file(
|
||||
provider_factory,
|
||||
@ -78,14 +80,14 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> ImportReceiptsOpCommand<C> {
|
||||
}
|
||||
|
||||
/// Imports receipts to static files from file in chunks. See [`import_receipts_from_reader`].
|
||||
pub async fn import_receipts_from_file<DB, P, F>(
|
||||
provider_factory: ProviderFactory<DB>,
|
||||
pub async fn import_receipts_from_file<N, P, F>(
|
||||
provider_factory: ProviderFactory<N>,
|
||||
path: P,
|
||||
chunk_len: Option<u64>,
|
||||
filter: F,
|
||||
) -> eyre::Result<()>
|
||||
where
|
||||
DB: Database,
|
||||
N: NodeTypesWithDB<ChainSpec = ChainSpec>,
|
||||
P: AsRef<Path>,
|
||||
F: FnMut(u64, &mut Receipts) -> usize,
|
||||
{
|
||||
@ -170,13 +172,13 @@ where
|
||||
/// Caution! Filter callback must replace completely filtered out receipts for a block, with empty
|
||||
/// vectors, rather than `vec!(None)`. This is since the code for writing to static files, expects
|
||||
/// indices in the [`Receipts`] list, to map to sequential block numbers.
|
||||
pub async fn import_receipts_from_reader<DB, F>(
|
||||
provider_factory: &ProviderFactory<DB>,
|
||||
pub async fn import_receipts_from_reader<N, F>(
|
||||
provider_factory: &ProviderFactory<N>,
|
||||
mut reader: ChunkedFileReader,
|
||||
mut filter: F,
|
||||
) -> eyre::Result<ImportReceiptsResult>
|
||||
where
|
||||
DB: Database,
|
||||
N: NodeTypesWithDB<ChainSpec = ChainSpec>,
|
||||
F: FnMut(u64, &mut Receipts) -> usize,
|
||||
{
|
||||
let mut total_decoded_receipts = 0;
|
||||
|
||||
@ -46,6 +46,7 @@ use reth_node_core::{
|
||||
args::LogArgs,
|
||||
version::{LONG_VERSION, SHORT_VERSION},
|
||||
};
|
||||
use reth_node_optimism::OptimismNode;
|
||||
use reth_tracing::FileWorkerGuard;
|
||||
use tracing::info;
|
||||
|
||||
@ -130,20 +131,31 @@ impl<Ext: clap::Args + fmt::Debug> Cli<Ext> {
|
||||
Commands::Node(command) => {
|
||||
runner.run_command_until_exit(|ctx| command.execute(ctx, launcher))
|
||||
}
|
||||
Commands::Init(command) => runner.run_blocking_until_ctrl_c(command.execute()),
|
||||
Commands::InitState(command) => runner.run_blocking_until_ctrl_c(command.execute()),
|
||||
Commands::ImportOp(command) => runner.run_blocking_until_ctrl_c(command.execute()),
|
||||
Commands::Init(command) => {
|
||||
runner.run_blocking_until_ctrl_c(command.execute::<OptimismNode>())
|
||||
}
|
||||
Commands::InitState(command) => {
|
||||
runner.run_blocking_until_ctrl_c(command.execute::<OptimismNode>())
|
||||
}
|
||||
Commands::ImportOp(command) => {
|
||||
runner.run_blocking_until_ctrl_c(command.execute::<OptimismNode>())
|
||||
}
|
||||
Commands::ImportReceiptsOp(command) => {
|
||||
runner.run_blocking_until_ctrl_c(command.execute())
|
||||
runner.run_blocking_until_ctrl_c(command.execute::<OptimismNode>())
|
||||
}
|
||||
Commands::DumpGenesis(command) => runner.run_blocking_until_ctrl_c(command.execute()),
|
||||
Commands::Db(command) => runner.run_blocking_until_ctrl_c(command.execute()),
|
||||
Commands::Stage(command) => runner
|
||||
.run_command_until_exit(|ctx| command.execute(ctx, OpExecutorProvider::optimism)),
|
||||
Commands::Db(command) => {
|
||||
runner.run_blocking_until_ctrl_c(command.execute::<OptimismNode>())
|
||||
}
|
||||
Commands::Stage(command) => runner.run_command_until_exit(|ctx| {
|
||||
command.execute::<OptimismNode, _, _>(ctx, OpExecutorProvider::optimism)
|
||||
}),
|
||||
Commands::P2P(command) => runner.run_until_ctrl_c(command.execute()),
|
||||
Commands::Config(command) => runner.run_until_ctrl_c(command.execute()),
|
||||
Commands::Recover(command) => runner.run_command_until_exit(|ctx| command.execute(ctx)),
|
||||
Commands::Prune(command) => runner.run_until_ctrl_c(command.execute()),
|
||||
Commands::Recover(command) => {
|
||||
runner.run_command_until_exit(|ctx| command.execute::<OptimismNode>(ctx))
|
||||
}
|
||||
Commands::Prune(command) => runner.run_until_ctrl_c(command.execute::<OptimismNode>()),
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -23,6 +23,7 @@ reth-tokio-util.workspace = true
|
||||
reth-config.workspace = true
|
||||
reth-prune-types.workspace = true
|
||||
reth-static-file-types.workspace = true
|
||||
reth-node-types.workspace = true
|
||||
|
||||
# metrics
|
||||
reth-metrics.workspace = true
|
||||
|
||||
@ -3,6 +3,7 @@ use reth_chainspec::MAINNET;
|
||||
use reth_config::PruneConfig;
|
||||
use reth_db_api::database::Database;
|
||||
use reth_exex_types::FinishedExExHeight;
|
||||
use reth_node_types::NodeTypesWithDB;
|
||||
use reth_provider::{providers::StaticFileProvider, ProviderFactory, StaticFileProviderFactory};
|
||||
use reth_prune_types::PruneModes;
|
||||
use std::time::Duration;
|
||||
@ -71,16 +72,14 @@ impl PrunerBuilder {
|
||||
}
|
||||
|
||||
/// Builds a [Pruner] from the current configuration with the given provider factory.
|
||||
pub fn build_with_provider_factory<DB: Database>(
|
||||
pub fn build_with_provider_factory<N: NodeTypesWithDB>(
|
||||
self,
|
||||
provider_factory: ProviderFactory<DB>,
|
||||
) -> Pruner<DB, ProviderFactory<DB>> {
|
||||
let segments = SegmentSet::<DB>::from_components(
|
||||
provider_factory.static_file_provider(),
|
||||
self.segments,
|
||||
);
|
||||
provider_factory: ProviderFactory<N>,
|
||||
) -> Pruner<N::DB, ProviderFactory<N>> {
|
||||
let segments =
|
||||
SegmentSet::from_components(provider_factory.static_file_provider(), self.segments);
|
||||
|
||||
Pruner::<_, ProviderFactory<DB>>::new(
|
||||
Pruner::<_, ProviderFactory<N>>::new(
|
||||
provider_factory,
|
||||
segments.into_vec(),
|
||||
self.block_interval,
|
||||
|
||||
@ -7,7 +7,10 @@ use crate::{
|
||||
use alloy_primitives::BlockNumber;
|
||||
use reth_db_api::database::Database;
|
||||
use reth_exex_types::FinishedExExHeight;
|
||||
use reth_provider::{DatabaseProviderRW, ProviderFactory, PruneCheckpointReader};
|
||||
use reth_node_types::NodeTypesWithDB;
|
||||
use reth_provider::{
|
||||
providers::ProviderNodeTypes, DatabaseProviderRW, ProviderFactory, PruneCheckpointReader,
|
||||
};
|
||||
use reth_prune_types::{PruneLimiter, PruneProgress, PruneSegment, PrunerOutput};
|
||||
use reth_tokio_util::{EventSender, EventStream};
|
||||
use std::time::{Duration, Instant};
|
||||
@ -69,11 +72,11 @@ impl<DB> Pruner<DB, ()> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> Pruner<DB, ProviderFactory<DB>> {
|
||||
impl<N: NodeTypesWithDB> Pruner<N::DB, ProviderFactory<N>> {
|
||||
/// Crates a new pruner with the given provider factory.
|
||||
pub fn new(
|
||||
provider_factory: ProviderFactory<DB>,
|
||||
segments: Vec<Box<dyn Segment<DB>>>,
|
||||
provider_factory: ProviderFactory<N>,
|
||||
segments: Vec<Box<dyn Segment<N::DB>>>,
|
||||
min_block_interval: usize,
|
||||
delete_limit: usize,
|
||||
timeout: Option<Duration>,
|
||||
@ -312,7 +315,7 @@ impl<DB: Database> Pruner<DB, ()> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> Pruner<DB, ProviderFactory<DB>> {
|
||||
impl<N: ProviderNodeTypes> Pruner<N::DB, ProviderFactory<N>> {
|
||||
/// Run the pruner. This will only prune data up to the highest finished ExEx height, if there
|
||||
/// are no ExExes.
|
||||
///
|
||||
|
||||
@ -23,6 +23,7 @@ reth-prune.workspace = true
|
||||
reth-errors.workspace = true
|
||||
reth-stages-types.workspace = true
|
||||
reth-static-file-types.workspace = true
|
||||
reth-node-types.workspace = true
|
||||
|
||||
alloy-primitives.workspace = true
|
||||
|
||||
|
||||
@ -1,6 +1,7 @@
|
||||
use crate::{pipeline::BoxedStage, MetricEventsSender, Pipeline, Stage, StageId, StageSet};
|
||||
use alloy_primitives::{BlockNumber, B256};
|
||||
use reth_db_api::database::Database;
|
||||
use reth_node_types::NodeTypesWithDB;
|
||||
use reth_provider::ProviderFactory;
|
||||
use reth_static_file::StaticFileProducer;
|
||||
use tokio::sync::watch;
|
||||
@ -68,11 +69,11 @@ where
|
||||
}
|
||||
|
||||
/// Builds the final [`Pipeline`] using the given database.
|
||||
pub fn build(
|
||||
pub fn build<N: NodeTypesWithDB<DB = DB>>(
|
||||
self,
|
||||
provider_factory: ProviderFactory<DB>,
|
||||
static_file_producer: StaticFileProducer<DB>,
|
||||
) -> Pipeline<DB> {
|
||||
provider_factory: ProviderFactory<N>,
|
||||
static_file_producer: StaticFileProducer<N>,
|
||||
) -> Pipeline<N> {
|
||||
let Self { stages, max_block, tip_tx, metrics_tx } = self;
|
||||
Pipeline {
|
||||
provider_factory,
|
||||
|
||||
@ -5,11 +5,12 @@ use crate::{PipelineTarget, StageCheckpoint, StageId};
|
||||
use alloy_primitives::{BlockNumber, B256};
|
||||
pub use event::*;
|
||||
use futures_util::Future;
|
||||
use reth_db_api::database::Database;
|
||||
use reth_node_types::NodeTypesWithDB;
|
||||
use reth_primitives_traits::constants::BEACON_CONSENSUS_REORG_UNWIND_DEPTH;
|
||||
use reth_provider::{
|
||||
writer::UnifiedStorageWriter, FinalizedBlockReader, FinalizedBlockWriter, ProviderFactory,
|
||||
StageCheckpointReader, StageCheckpointWriter, StaticFileProviderFactory,
|
||||
providers::ProviderNodeTypes, writer::UnifiedStorageWriter, FinalizedBlockReader,
|
||||
FinalizedBlockWriter, ProviderFactory, StageCheckpointReader, StageCheckpointWriter,
|
||||
StaticFileProviderFactory,
|
||||
};
|
||||
use reth_prune::PrunerBuilder;
|
||||
use reth_static_file::StaticFileProducer;
|
||||
@ -36,10 +37,10 @@ pub(crate) type BoxedStage<DB> = Box<dyn Stage<DB>>;
|
||||
|
||||
/// The future that returns the owned pipeline and the result of the pipeline run. See
|
||||
/// [`Pipeline::run_as_fut`].
|
||||
pub type PipelineFut<DB> = Pin<Box<dyn Future<Output = PipelineWithResult<DB>> + Send>>;
|
||||
pub type PipelineFut<N> = Pin<Box<dyn Future<Output = PipelineWithResult<N>> + Send>>;
|
||||
|
||||
/// The pipeline type itself with the result of [`Pipeline::run_as_fut`]
|
||||
pub type PipelineWithResult<DB> = (Pipeline<DB>, Result<ControlFlow, PipelineError>);
|
||||
pub type PipelineWithResult<N> = (Pipeline<N>, Result<ControlFlow, PipelineError>);
|
||||
|
||||
#[cfg_attr(doc, aquamarine::aquamarine)]
|
||||
/// A staged sync pipeline.
|
||||
@ -63,14 +64,14 @@ pub type PipelineWithResult<DB> = (Pipeline<DB>, Result<ControlFlow, PipelineErr
|
||||
/// # Defaults
|
||||
///
|
||||
/// The [`DefaultStages`](crate::sets::DefaultStages) are used to fully sync reth.
|
||||
pub struct Pipeline<DB: Database> {
|
||||
pub struct Pipeline<N: NodeTypesWithDB> {
|
||||
/// Provider factory.
|
||||
provider_factory: ProviderFactory<DB>,
|
||||
provider_factory: ProviderFactory<N>,
|
||||
/// All configured stages in the order they will be executed.
|
||||
stages: Vec<BoxedStage<DB>>,
|
||||
stages: Vec<BoxedStage<N::DB>>,
|
||||
/// The maximum block number to sync to.
|
||||
max_block: Option<BlockNumber>,
|
||||
static_file_producer: StaticFileProducer<DB>,
|
||||
static_file_producer: StaticFileProducer<N>,
|
||||
/// Sender for events the pipeline emits.
|
||||
event_sender: EventSender<PipelineEvent>,
|
||||
/// Keeps track of the progress of the pipeline.
|
||||
@ -80,12 +81,9 @@ pub struct Pipeline<DB: Database> {
|
||||
metrics_tx: Option<MetricEventsSender>,
|
||||
}
|
||||
|
||||
impl<DB> Pipeline<DB>
|
||||
where
|
||||
DB: Database + 'static,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> Pipeline<N> {
|
||||
/// Construct a pipeline using a [`PipelineBuilder`].
|
||||
pub fn builder() -> PipelineBuilder<DB> {
|
||||
pub fn builder() -> PipelineBuilder<N::DB> {
|
||||
PipelineBuilder::default()
|
||||
}
|
||||
|
||||
@ -107,7 +105,9 @@ where
|
||||
pub fn events(&self) -> EventStream<PipelineEvent> {
|
||||
self.event_sender.new_listener()
|
||||
}
|
||||
}
|
||||
|
||||
impl<N: ProviderNodeTypes> Pipeline<N> {
|
||||
/// Registers progress metrics for each registered stage
|
||||
pub fn register_metrics(&mut self) -> Result<(), PipelineError> {
|
||||
let Some(metrics_tx) = &mut self.metrics_tx else { return Ok(()) };
|
||||
@ -127,7 +127,7 @@ where
|
||||
/// Consume the pipeline and run it until it reaches the provided tip, if set. Return the
|
||||
/// pipeline and its result as a future.
|
||||
#[track_caller]
|
||||
pub fn run_as_fut(mut self, target: Option<PipelineTarget>) -> PipelineFut<DB> {
|
||||
pub fn run_as_fut(mut self, target: Option<PipelineTarget>) -> PipelineFut<N> {
|
||||
// TODO: fix this in a follow up PR. ideally, consensus engine would be responsible for
|
||||
// updating metrics.
|
||||
let _ = self.register_metrics(); // ignore error
|
||||
@ -487,8 +487,8 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
fn on_stage_error<DB: Database>(
|
||||
factory: &ProviderFactory<DB>,
|
||||
fn on_stage_error<N: ProviderNodeTypes>(
|
||||
factory: &ProviderFactory<N>,
|
||||
stage_id: StageId,
|
||||
prev_checkpoint: Option<StageCheckpoint>,
|
||||
err: StageError,
|
||||
@ -574,7 +574,7 @@ fn on_stage_error<DB: Database>(
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> std::fmt::Debug for Pipeline<DB> {
|
||||
impl<N: NodeTypesWithDB> std::fmt::Debug for Pipeline<N> {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
f.debug_struct("Pipeline")
|
||||
.field("stages", &self.stages.iter().map(|stage| stage.id()).collect::<Vec<StageId>>())
|
||||
@ -591,7 +591,7 @@ mod tests {
|
||||
use assert_matches::assert_matches;
|
||||
use reth_consensus::ConsensusError;
|
||||
use reth_errors::ProviderError;
|
||||
use reth_provider::test_utils::create_test_provider_factory;
|
||||
use reth_provider::test_utils::{create_test_provider_factory, MockNodeTypesWithDB};
|
||||
use reth_prune::PruneModes;
|
||||
use reth_testing_utils::{generators, generators::random_header};
|
||||
use tokio_stream::StreamExt;
|
||||
@ -628,7 +628,7 @@ mod tests {
|
||||
async fn run_pipeline() {
|
||||
let provider_factory = create_test_provider_factory();
|
||||
|
||||
let mut pipeline = Pipeline::builder()
|
||||
let mut pipeline = Pipeline::<MockNodeTypesWithDB>::builder()
|
||||
.add_stage(
|
||||
TestStage::new(StageId::Other("A"))
|
||||
.add_exec(Ok(ExecOutput { checkpoint: StageCheckpoint::new(20), done: true })),
|
||||
@ -696,7 +696,7 @@ mod tests {
|
||||
async fn unwind_pipeline() {
|
||||
let provider_factory = create_test_provider_factory();
|
||||
|
||||
let mut pipeline = Pipeline::builder()
|
||||
let mut pipeline = Pipeline::<MockNodeTypesWithDB>::builder()
|
||||
.add_stage(
|
||||
TestStage::new(StageId::Other("A"))
|
||||
.add_exec(Ok(ExecOutput { checkpoint: StageCheckpoint::new(100), done: true }))
|
||||
@ -830,7 +830,7 @@ mod tests {
|
||||
async fn unwind_pipeline_with_intermediate_progress() {
|
||||
let provider_factory = create_test_provider_factory();
|
||||
|
||||
let mut pipeline = Pipeline::builder()
|
||||
let mut pipeline = Pipeline::<MockNodeTypesWithDB>::builder()
|
||||
.add_stage(
|
||||
TestStage::new(StageId::Other("A"))
|
||||
.add_exec(Ok(ExecOutput { checkpoint: StageCheckpoint::new(100), done: true }))
|
||||
@ -930,7 +930,7 @@ mod tests {
|
||||
async fn run_pipeline_with_unwind() {
|
||||
let provider_factory = create_test_provider_factory();
|
||||
|
||||
let mut pipeline = Pipeline::builder()
|
||||
let mut pipeline = Pipeline::<MockNodeTypesWithDB>::builder()
|
||||
.add_stage(
|
||||
TestStage::new(StageId::Other("A"))
|
||||
.add_exec(Ok(ExecOutput { checkpoint: StageCheckpoint::new(10), done: true }))
|
||||
@ -1051,7 +1051,7 @@ mod tests {
|
||||
async fn pipeline_error_handling() {
|
||||
// Non-fatal
|
||||
let provider_factory = create_test_provider_factory();
|
||||
let mut pipeline = Pipeline::builder()
|
||||
let mut pipeline = Pipeline::<MockNodeTypesWithDB>::builder()
|
||||
.add_stage(
|
||||
TestStage::new(StageId::Other("NonFatal"))
|
||||
.add_exec(Err(StageError::Recoverable(Box::new(std::fmt::Error))))
|
||||
@ -1067,7 +1067,7 @@ mod tests {
|
||||
|
||||
// Fatal
|
||||
let provider_factory = create_test_provider_factory();
|
||||
let mut pipeline = Pipeline::builder()
|
||||
let mut pipeline = Pipeline::<MockNodeTypesWithDB>::builder()
|
||||
.add_stage(TestStage::new(StageId::Other("Fatal")).add_exec(Err(
|
||||
StageError::DatabaseIntegrity(ProviderError::BlockBodyIndicesNotFound(5)),
|
||||
)))
|
||||
|
||||
@ -27,7 +27,7 @@
|
||||
//! # use reth_evm_ethereum::EthEvmConfig;
|
||||
//! # use reth_provider::ProviderFactory;
|
||||
//! # use reth_provider::StaticFileProviderFactory;
|
||||
//! # use reth_provider::test_utils::create_test_provider_factory;
|
||||
//! # use reth_provider::test_utils::{create_test_provider_factory, MockNodeTypesWithDB};
|
||||
//! # use reth_static_file::StaticFileProducer;
|
||||
//! # use reth_config::config::StageConfig;
|
||||
//! # use reth_consensus::Consensus;
|
||||
@ -53,7 +53,7 @@
|
||||
//! # );
|
||||
//! // Create a pipeline that can fully sync
|
||||
//! # let pipeline =
|
||||
//! Pipeline::builder()
|
||||
//! Pipeline::<MockNodeTypesWithDB>::builder()
|
||||
//! .with_tip_sender(tip_tx)
|
||||
//! .add_stages(DefaultStages::new(
|
||||
//! provider_factory.clone(),
|
||||
|
||||
@ -16,7 +16,7 @@
|
||||
//! # use reth_prune_types::PruneModes;
|
||||
//! # use reth_evm_ethereum::EthEvmConfig;
|
||||
//! # use reth_provider::StaticFileProviderFactory;
|
||||
//! # use reth_provider::test_utils::create_test_provider_factory;
|
||||
//! # use reth_provider::test_utils::{create_test_provider_factory, MockNodeTypesWithDB};
|
||||
//! # use reth_static_file::StaticFileProducer;
|
||||
//! # use reth_config::config::StageConfig;
|
||||
//! # use reth_evm::execute::BlockExecutorProvider;
|
||||
@ -27,7 +27,7 @@
|
||||
//! let static_file_producer =
|
||||
//! StaticFileProducer::new(provider_factory.clone(), PruneModes::default());
|
||||
//! // Build a pipeline with all offline stages.
|
||||
//! let pipeline = Pipeline::builder()
|
||||
//! let pipeline = Pipeline::<MockNodeTypesWithDB>::builder()
|
||||
//! .add_stages(OfflineStages::new(exec, StageConfig::default(), PruneModes::default()))
|
||||
//! .build(provider_factory, static_file_producer);
|
||||
//!
|
||||
|
||||
@ -618,7 +618,7 @@ mod tests {
|
||||
},
|
||||
};
|
||||
use futures_util::Stream;
|
||||
use reth_db::{static_file::HeaderMask, tables, test_utils::TempDatabase, DatabaseEnv};
|
||||
use reth_db::{static_file::HeaderMask, tables};
|
||||
use reth_db_api::{
|
||||
cursor::DbCursorRO,
|
||||
models::{StoredBlockBodyIndices, StoredBlockOmmers},
|
||||
@ -636,8 +636,8 @@ mod tests {
|
||||
StaticFileSegment, TxNumber, B256,
|
||||
};
|
||||
use reth_provider::{
|
||||
providers::StaticFileWriter, HeaderProvider, ProviderFactory,
|
||||
StaticFileProviderFactory, TransactionsProvider,
|
||||
providers::StaticFileWriter, test_utils::MockNodeTypesWithDB, HeaderProvider,
|
||||
ProviderFactory, StaticFileProviderFactory, TransactionsProvider,
|
||||
};
|
||||
use reth_stages_api::{ExecInput, ExecOutput, UnwindInput};
|
||||
use reth_testing_utils::generators::{
|
||||
@ -647,7 +647,6 @@ mod tests {
|
||||
collections::{HashMap, VecDeque},
|
||||
ops::RangeInclusive,
|
||||
pin::Pin,
|
||||
sync::Arc,
|
||||
task::{Context, Poll},
|
||||
};
|
||||
|
||||
@ -892,7 +891,7 @@ mod tests {
|
||||
/// A [`BodyDownloader`] that is backed by an internal [`HashMap`] for testing.
|
||||
#[derive(Debug)]
|
||||
pub(crate) struct TestBodyDownloader {
|
||||
provider_factory: ProviderFactory<Arc<TempDatabase<DatabaseEnv>>>,
|
||||
provider_factory: ProviderFactory<MockNodeTypesWithDB>,
|
||||
responses: HashMap<B256, BlockBody>,
|
||||
headers: VecDeque<SealedHeader>,
|
||||
batch_size: u64,
|
||||
@ -900,7 +899,7 @@ mod tests {
|
||||
|
||||
impl TestBodyDownloader {
|
||||
pub(crate) fn new(
|
||||
provider_factory: ProviderFactory<Arc<TempDatabase<DatabaseEnv>>>,
|
||||
provider_factory: ProviderFactory<MockNodeTypesWithDB>,
|
||||
responses: HashMap<B256, BlockBody>,
|
||||
batch_size: u64,
|
||||
) -> Self {
|
||||
|
||||
@ -389,12 +389,11 @@ mod tests {
|
||||
use super::*;
|
||||
use crate::test_utils::{TestRunnerError, TestStageDB};
|
||||
use reth_consensus::test_utils::TestConsensus;
|
||||
use reth_db::{test_utils::TempDatabase, DatabaseEnv};
|
||||
use reth_downloaders::headers::reverse_headers::{
|
||||
ReverseHeadersDownloader, ReverseHeadersDownloaderBuilder,
|
||||
};
|
||||
use reth_network_p2p::test_utils::{TestHeaderDownloader, TestHeadersClient};
|
||||
use reth_provider::BlockNumReader;
|
||||
use reth_provider::{test_utils::MockNodeTypesWithDB, BlockNumReader};
|
||||
use tokio::sync::watch;
|
||||
|
||||
pub(crate) struct HeadersTestRunner<D: HeaderDownloader> {
|
||||
@ -426,7 +425,7 @@ mod tests {
|
||||
}
|
||||
|
||||
impl<D: HeaderDownloader + 'static> StageTestRunner for HeadersTestRunner<D> {
|
||||
type S = HeaderStage<ProviderFactory<Arc<TempDatabase<DatabaseEnv>>>, D>;
|
||||
type S = HeaderStage<ProviderFactory<MockNodeTypesWithDB>, D>;
|
||||
|
||||
fn db(&self) -> &TestStageDB {
|
||||
&self.db
|
||||
|
||||
@ -46,9 +46,7 @@ mod tests {
|
||||
use reth_chainspec::ChainSpecBuilder;
|
||||
use reth_db::{
|
||||
mdbx::{cursor::Cursor, RW},
|
||||
tables,
|
||||
test_utils::TempDatabase,
|
||||
AccountsHistory, DatabaseEnv,
|
||||
tables, AccountsHistory,
|
||||
};
|
||||
use reth_db_api::{
|
||||
cursor::{DbCursorRO, DbCursorRW},
|
||||
@ -63,6 +61,7 @@ mod tests {
|
||||
};
|
||||
use reth_provider::{
|
||||
providers::{StaticFileProvider, StaticFileWriter},
|
||||
test_utils::MockNodeTypesWithDB,
|
||||
AccountExtReader, BlockReader, DatabaseProviderFactory, ProviderFactory, ProviderResult,
|
||||
ReceiptProvider, StageCheckpointWriter, StaticFileProviderFactory, StorageReader,
|
||||
};
|
||||
@ -140,7 +139,7 @@ mod tests {
|
||||
.unwrap();
|
||||
provider_rw.commit().unwrap();
|
||||
|
||||
let check_pruning = |factory: ProviderFactory<Arc<TempDatabase<DatabaseEnv>>>,
|
||||
let check_pruning = |factory: ProviderFactory<MockNodeTypesWithDB>,
|
||||
prune_modes: PruneModes,
|
||||
expect_num_receipts: usize,
|
||||
expect_num_acc_changesets: usize,
|
||||
|
||||
@ -1,9 +1,7 @@
|
||||
use reth_chainspec::MAINNET;
|
||||
use reth_db::{
|
||||
tables,
|
||||
test_utils::{
|
||||
create_test_rw_db, create_test_rw_db_with_path, create_test_static_files_dir, TempDatabase,
|
||||
},
|
||||
test_utils::{create_test_rw_db, create_test_rw_db_with_path, create_test_static_files_dir},
|
||||
DatabaseEnv,
|
||||
};
|
||||
use reth_db_api::{
|
||||
@ -21,17 +19,18 @@ use reth_primitives::{
|
||||
};
|
||||
use reth_provider::{
|
||||
providers::{StaticFileProvider, StaticFileProviderRWRefMut, StaticFileWriter},
|
||||
test_utils::MockNodeTypesWithDB,
|
||||
HistoryWriter, ProviderError, ProviderFactory, StaticFileProviderFactory,
|
||||
};
|
||||
use reth_storage_errors::provider::ProviderResult;
|
||||
use reth_testing_utils::generators::ChangeSet;
|
||||
use std::{collections::BTreeMap, path::Path, sync::Arc};
|
||||
use std::{collections::BTreeMap, path::Path};
|
||||
use tempfile::TempDir;
|
||||
|
||||
/// Test database that is used for testing stage implementations.
|
||||
#[derive(Debug)]
|
||||
pub struct TestStageDB {
|
||||
pub factory: ProviderFactory<Arc<TempDatabase<DatabaseEnv>>>,
|
||||
pub factory: ProviderFactory<MockNodeTypesWithDB>,
|
||||
pub temp_static_files_dir: TempDir,
|
||||
}
|
||||
|
||||
|
||||
@ -13,6 +13,7 @@ workspace = true
|
||||
|
||||
[dependencies]
|
||||
# reth
|
||||
reth-chainspec.workspace = true
|
||||
reth-db.workspace = true
|
||||
reth-db-api.workspace = true
|
||||
reth-provider.workspace = true
|
||||
@ -22,6 +23,7 @@ reth-tokio-util.workspace = true
|
||||
reth-prune-types.workspace = true
|
||||
reth-static-file-types.workspace = true
|
||||
reth-stages-types.workspace = true
|
||||
reth-node-types.workspace = true
|
||||
|
||||
alloy-primitives.workspace = true
|
||||
|
||||
|
||||
@ -4,7 +4,8 @@ use crate::{segments, segments::Segment, StaticFileProducerEvent};
|
||||
use alloy_primitives::BlockNumber;
|
||||
use parking_lot::Mutex;
|
||||
use rayon::prelude::*;
|
||||
use reth_db_api::database::Database;
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_node_types::NodeTypesWithDB;
|
||||
use reth_provider::{
|
||||
providers::StaticFileWriter, ProviderFactory, StageCheckpointReader as _,
|
||||
StaticFileProviderFactory,
|
||||
@ -25,22 +26,28 @@ use tracing::{debug, trace};
|
||||
pub type StaticFileProducerResult = ProviderResult<StaticFileTargets>;
|
||||
|
||||
/// The [`StaticFileProducer`] instance itself with the result of [`StaticFileProducerInner::run`]
|
||||
pub type StaticFileProducerWithResult<DB> = (StaticFileProducer<DB>, StaticFileProducerResult);
|
||||
pub type StaticFileProducerWithResult<N> = (StaticFileProducer<N>, StaticFileProducerResult);
|
||||
|
||||
/// Static File producer. It's a wrapper around [`StaticFileProducer`] that allows to share it
|
||||
/// between threads.
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct StaticFileProducer<DB>(Arc<Mutex<StaticFileProducerInner<DB>>>);
|
||||
#[derive(Debug)]
|
||||
pub struct StaticFileProducer<N: NodeTypesWithDB>(Arc<Mutex<StaticFileProducerInner<N>>>);
|
||||
|
||||
impl<DB: Database> StaticFileProducer<DB> {
|
||||
impl<N: NodeTypesWithDB> StaticFileProducer<N> {
|
||||
/// Creates a new [`StaticFileProducer`].
|
||||
pub fn new(provider_factory: ProviderFactory<DB>, prune_modes: PruneModes) -> Self {
|
||||
pub fn new(provider_factory: ProviderFactory<N>, prune_modes: PruneModes) -> Self {
|
||||
Self(Arc::new(Mutex::new(StaticFileProducerInner::new(provider_factory, prune_modes))))
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> Deref for StaticFileProducer<DB> {
|
||||
type Target = Arc<Mutex<StaticFileProducerInner<DB>>>;
|
||||
impl<N: NodeTypesWithDB> Clone for StaticFileProducer<N> {
|
||||
fn clone(&self) -> Self {
|
||||
Self(self.0.clone())
|
||||
}
|
||||
}
|
||||
|
||||
impl<N: NodeTypesWithDB> Deref for StaticFileProducer<N> {
|
||||
type Target = Arc<Mutex<StaticFileProducerInner<N>>>;
|
||||
|
||||
fn deref(&self) -> &Self::Target {
|
||||
&self.0
|
||||
@ -50,9 +57,9 @@ impl<DB> Deref for StaticFileProducer<DB> {
|
||||
/// Static File producer routine. See [`StaticFileProducerInner::run`] for more detailed
|
||||
/// description.
|
||||
#[derive(Debug)]
|
||||
pub struct StaticFileProducerInner<DB> {
|
||||
pub struct StaticFileProducerInner<N: NodeTypesWithDB> {
|
||||
/// Provider factory
|
||||
provider_factory: ProviderFactory<DB>,
|
||||
provider_factory: ProviderFactory<N>,
|
||||
/// Pruning configuration for every part of the data that can be pruned. Set by user, and
|
||||
/// needed in [`StaticFileProducerInner`] to prevent attempting to move prunable data to static
|
||||
/// files. See [`StaticFileProducerInner::get_static_file_targets`].
|
||||
@ -94,11 +101,13 @@ impl StaticFileTargets {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> StaticFileProducerInner<DB> {
|
||||
fn new(provider_factory: ProviderFactory<DB>, prune_modes: PruneModes) -> Self {
|
||||
impl<N: NodeTypesWithDB> StaticFileProducerInner<N> {
|
||||
fn new(provider_factory: ProviderFactory<N>, prune_modes: PruneModes) -> Self {
|
||||
Self { provider_factory, prune_modes, event_sender: Default::default() }
|
||||
}
|
||||
}
|
||||
|
||||
impl<N: NodeTypesWithDB<ChainSpec = ChainSpec>> StaticFileProducerInner<N> {
|
||||
/// Listen for events on the `static_file_producer`.
|
||||
pub fn events(&self) -> EventStream<StaticFileProducerEvent> {
|
||||
self.event_sender.new_listener()
|
||||
@ -128,7 +137,7 @@ impl<DB: Database> StaticFileProducerInner<DB> {
|
||||
debug!(target: "static_file", ?targets, "StaticFileProducer started");
|
||||
let start = Instant::now();
|
||||
|
||||
let mut segments = Vec::<(Box<dyn Segment<DB>>, RangeInclusive<BlockNumber>)>::new();
|
||||
let mut segments = Vec::<(Box<dyn Segment<N::DB>>, RangeInclusive<BlockNumber>)>::new();
|
||||
|
||||
if let Some(block_range) = targets.transactions.clone() {
|
||||
segments.push((Box::new(segments::Transactions), block_range));
|
||||
@ -257,10 +266,10 @@ mod tests {
|
||||
};
|
||||
use alloy_primitives::{B256, U256};
|
||||
use assert_matches::assert_matches;
|
||||
use reth_db::{test_utils::TempDatabase, DatabaseEnv};
|
||||
use reth_db_api::{database::Database, transaction::DbTx};
|
||||
use reth_provider::{
|
||||
providers::StaticFileWriter, ProviderError, ProviderFactory, StaticFileProviderFactory,
|
||||
providers::StaticFileWriter, test_utils::MockNodeTypesWithDB, ProviderError,
|
||||
ProviderFactory, StaticFileProviderFactory,
|
||||
};
|
||||
use reth_prune_types::PruneModes;
|
||||
use reth_stages::test_utils::{StorageKind, TestStageDB};
|
||||
@ -268,13 +277,10 @@ mod tests {
|
||||
use reth_testing_utils::generators::{
|
||||
self, random_block_range, random_receipt, BlockRangeParams,
|
||||
};
|
||||
use std::{
|
||||
sync::{mpsc::channel, Arc},
|
||||
time::Duration,
|
||||
};
|
||||
use std::{sync::mpsc::channel, time::Duration};
|
||||
use tempfile::TempDir;
|
||||
|
||||
fn setup() -> (ProviderFactory<Arc<TempDatabase<DatabaseEnv>>>, TempDir) {
|
||||
fn setup() -> (ProviderFactory<MockNodeTypesWithDB>, TempDir) {
|
||||
let mut rng = generators::rng();
|
||||
let db = TestStageDB::default();
|
||||
|
||||
|
||||
@ -8,7 +8,7 @@ use std::{fmt::Debug, sync::Arc};
|
||||
/// Main Database trait that can open read-only and read-write transactions.
|
||||
///
|
||||
/// Sealed trait which cannot be implemented by 3rd parties, exposed only for consumption.
|
||||
pub trait Database: Send + Sync {
|
||||
pub trait Database: Send + Sync + Debug {
|
||||
/// Read-Only database transaction
|
||||
type TX: DbTx + Send + Sync + Debug + 'static;
|
||||
/// Read-Write database transaction
|
||||
|
||||
@ -21,6 +21,7 @@ reth-etl.workspace = true
|
||||
reth-codecs.workspace = true
|
||||
reth-stages-types.workspace = true
|
||||
reth-fs-util.workspace = true
|
||||
reth-node-types.workspace = true
|
||||
|
||||
# eth
|
||||
alloy-genesis.workspace = true
|
||||
@ -39,6 +40,7 @@ tracing.workspace = true
|
||||
|
||||
[dev-dependencies]
|
||||
reth-primitives-traits.workspace = true
|
||||
reth-provider = { workspace = true, features = ["test-utils"] }
|
||||
|
||||
[lints]
|
||||
workspace = true
|
||||
|
||||
@ -12,28 +12,21 @@ use reth_db_api::{
|
||||
DatabaseError,
|
||||
};
|
||||
use reth_fs_util as fs;
|
||||
use reth_node_types::NodeTypesWithDB;
|
||||
use reth_provider::{ChainSpecProvider, ProviderFactory};
|
||||
use std::{path::Path, rc::Rc, sync::Arc};
|
||||
use tracing::info;
|
||||
|
||||
/// Wrapper over DB that implements many useful DB queries.
|
||||
#[derive(Debug)]
|
||||
pub struct DbTool<DB: Database> {
|
||||
pub struct DbTool<N: NodeTypesWithDB> {
|
||||
/// The provider factory that the db tool will use.
|
||||
pub provider_factory: ProviderFactory<DB>,
|
||||
pub provider_factory: ProviderFactory<N>,
|
||||
}
|
||||
|
||||
impl<DB: Database> DbTool<DB> {
|
||||
/// Takes a DB where the tables have already been created.
|
||||
pub fn new(provider_factory: ProviderFactory<DB>) -> eyre::Result<Self> {
|
||||
// Disable timeout because we are entering a TUI which might read for a long time. We
|
||||
// disable on the [`DbTool`] level since it's only used in the CLI.
|
||||
provider_factory.provider()?.disable_long_read_transaction_safety();
|
||||
Ok(Self { provider_factory })
|
||||
}
|
||||
|
||||
impl<N: NodeTypesWithDB> DbTool<N> {
|
||||
/// Get an [`Arc`] to the [`ChainSpec`].
|
||||
pub fn chain(&self) -> Arc<ChainSpec> {
|
||||
pub fn chain(&self) -> Arc<N::ChainSpec> {
|
||||
self.provider_factory.chain_spec()
|
||||
}
|
||||
|
||||
@ -115,6 +108,16 @@ impl<DB: Database> DbTool<DB> {
|
||||
|
||||
Ok((data.map_err(|e: DatabaseError| eyre::eyre!(e))?, hits))
|
||||
}
|
||||
}
|
||||
|
||||
impl<N: NodeTypesWithDB<ChainSpec = ChainSpec>> DbTool<N> {
|
||||
/// Takes a DB where the tables have already been created.
|
||||
pub fn new(provider_factory: ProviderFactory<N>) -> eyre::Result<Self> {
|
||||
// Disable timeout because we are entering a TUI which might read for a long time. We
|
||||
// disable on the [`DbTool`] level since it's only used in the CLI.
|
||||
provider_factory.provider()?.disable_long_read_transaction_safety();
|
||||
Ok(Self { provider_factory })
|
||||
}
|
||||
|
||||
/// Grabs the content of the table for the given key
|
||||
pub fn get<T: Table>(&self, key: T::Key) -> Result<Option<T::Value>> {
|
||||
|
||||
@ -7,6 +7,7 @@ use reth_config::config::EtlConfig;
|
||||
use reth_db::tables;
|
||||
use reth_db_api::{database::Database, transaction::DbTxMut, DatabaseError};
|
||||
use reth_etl::Collector;
|
||||
use reth_node_types::NodeTypesWithDB;
|
||||
use reth_primitives::{
|
||||
Account, Address, Bytecode, Receipts, StaticFileSegment, StorageEntry, B256, U256,
|
||||
};
|
||||
@ -81,7 +82,9 @@ impl From<DatabaseError> for InitDatabaseError {
|
||||
}
|
||||
|
||||
/// Write the genesis block if it has not already been written
|
||||
pub fn init_genesis<DB: Database>(factory: ProviderFactory<DB>) -> Result<B256, InitDatabaseError> {
|
||||
pub fn init_genesis<N: NodeTypesWithDB<ChainSpec = ChainSpec>>(
|
||||
factory: ProviderFactory<N>,
|
||||
) -> Result<B256, InitDatabaseError> {
|
||||
let chain = factory.chain_spec();
|
||||
|
||||
let genesis = chain.genesis();
|
||||
@ -317,9 +320,9 @@ pub fn insert_genesis_header<DB: Database>(
|
||||
/// It's similar to [`init_genesis`] but supports importing state too big to fit in memory, and can
|
||||
/// be set to the highest block present. One practical usecase is to import OP mainnet state at
|
||||
/// bedrock transition block.
|
||||
pub fn init_from_state_dump<DB: Database>(
|
||||
pub fn init_from_state_dump<N: NodeTypesWithDB<ChainSpec = ChainSpec>>(
|
||||
mut reader: impl BufRead,
|
||||
factory: ProviderFactory<DB>,
|
||||
factory: ProviderFactory<N>,
|
||||
etl_config: EtlConfig,
|
||||
) -> eyre::Result<B256> {
|
||||
let block = factory.last_block_number()?;
|
||||
@ -545,7 +548,9 @@ mod tests {
|
||||
};
|
||||
use reth_primitives::{HOLESKY_GENESIS_HASH, MAINNET_GENESIS_HASH, SEPOLIA_GENESIS_HASH};
|
||||
use reth_primitives_traits::IntegerList;
|
||||
use reth_provider::test_utils::create_test_provider_factory_with_chain_spec;
|
||||
use reth_provider::test_utils::{
|
||||
create_test_provider_factory_with_chain_spec, MockNodeTypesWithDB,
|
||||
};
|
||||
|
||||
fn collect_table_entries<DB, T>(
|
||||
tx: &<DB as Database>::TX,
|
||||
@ -591,7 +596,7 @@ mod tests {
|
||||
init_genesis(factory.clone()).unwrap();
|
||||
|
||||
// Try to init db with a different genesis block
|
||||
let genesis_hash = init_genesis(ProviderFactory::new(
|
||||
let genesis_hash = init_genesis(ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
factory.into_db(),
|
||||
MAINNET.clone(),
|
||||
static_file_provider,
|
||||
|
||||
@ -57,6 +57,7 @@ strum.workspace = true
|
||||
|
||||
# test-utils
|
||||
once_cell = { workspace = true, optional = true }
|
||||
reth-ethereum-engine-primitives = { workspace = true, optional = true }
|
||||
|
||||
# parallel utils
|
||||
rayon.workspace = true
|
||||
@ -83,4 +84,5 @@ test-utils = [
|
||||
"reth-trie/test-utils",
|
||||
"reth-chain-state/test-utils",
|
||||
"once_cell",
|
||||
"reth-ethereum-engine-primitives",
|
||||
]
|
||||
|
||||
@ -12,13 +12,11 @@ use reth_chain_state::{
|
||||
BlockState, CanonicalInMemoryState, ForkChoiceNotifications, ForkChoiceSubscriptions,
|
||||
MemoryOverlayStateProvider,
|
||||
};
|
||||
use reth_chainspec::{ChainInfo, ChainSpec};
|
||||
use reth_db_api::{
|
||||
database::Database,
|
||||
models::{AccountBeforeTx, StoredBlockBodyIndices},
|
||||
};
|
||||
use reth_chainspec::ChainInfo;
|
||||
use reth_db_api::models::{AccountBeforeTx, StoredBlockBodyIndices};
|
||||
use reth_evm::ConfigureEvmEnv;
|
||||
use reth_execution_types::ExecutionOutcome;
|
||||
use reth_node_types::NodeTypesWithDB;
|
||||
use reth_primitives::{
|
||||
Account, Address, Block, BlockHash, BlockHashOrNumber, BlockId, BlockNumHash, BlockNumber,
|
||||
BlockNumberOrTag, BlockWithSenders, EthereumHardforks, Header, Receipt, SealedBlock,
|
||||
@ -36,21 +34,23 @@ use std::{
|
||||
};
|
||||
use tracing::trace;
|
||||
|
||||
use super::ProviderNodeTypes;
|
||||
|
||||
/// The main type for interacting with the blockchain.
|
||||
///
|
||||
/// This type serves as the main entry point for interacting with the blockchain and provides data
|
||||
/// from database storage and from the blockchain tree (pending state etc.) It is a simple wrapper
|
||||
/// type that holds an instance of the database and the blockchain tree.
|
||||
#[derive(Debug)]
|
||||
pub struct BlockchainProvider2<DB, Spec = ChainSpec> {
|
||||
pub struct BlockchainProvider2<N: NodeTypesWithDB> {
|
||||
/// Provider type used to access the database.
|
||||
database: ProviderFactory<DB, Spec>,
|
||||
database: ProviderFactory<N>,
|
||||
/// Tracks the chain info wrt forkchoice updates and in memory canonical
|
||||
/// state.
|
||||
pub(super) canonical_in_memory_state: CanonicalInMemoryState,
|
||||
}
|
||||
|
||||
impl<DB> Clone for BlockchainProvider2<DB> {
|
||||
impl<N: NodeTypesWithDB> Clone for BlockchainProvider2<N> {
|
||||
fn clone(&self) -> Self {
|
||||
Self {
|
||||
database: self.database.clone(),
|
||||
@ -59,13 +59,10 @@ impl<DB> Clone for BlockchainProvider2<DB> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> BlockchainProvider2<N> {
|
||||
/// Create a new provider using only the database, fetching the latest header from
|
||||
/// the database to initialize the provider.
|
||||
pub fn new(database: ProviderFactory<DB>) -> ProviderResult<Self> {
|
||||
pub fn new(database: ProviderFactory<N>) -> ProviderResult<Self> {
|
||||
let provider = database.provider()?;
|
||||
let best: ChainInfo = provider.chain_info()?;
|
||||
match provider.header_by_number(best.best_number)? {
|
||||
@ -82,10 +79,7 @@ where
|
||||
///
|
||||
/// This returns a `ProviderResult` since it tries the retrieve the last finalized header from
|
||||
/// `database`.
|
||||
pub fn with_latest(
|
||||
database: ProviderFactory<DB>,
|
||||
latest: SealedHeader,
|
||||
) -> ProviderResult<Self> {
|
||||
pub fn with_latest(database: ProviderFactory<N>, latest: SealedHeader) -> ProviderResult<Self> {
|
||||
let provider = database.provider()?;
|
||||
let finalized_header = provider
|
||||
.last_finalized_block_number()?
|
||||
@ -191,7 +185,7 @@ where
|
||||
/// 2. The in-block transaction index.
|
||||
fn block_state_by_tx_id(
|
||||
&self,
|
||||
provider: &DatabaseProviderRO<DB>,
|
||||
provider: &DatabaseProviderRO<N::DB>,
|
||||
id: TxNumber,
|
||||
) -> ProviderResult<Option<(Option<Arc<BlockState>>, usize)>> {
|
||||
// Get the last block number stored in the database
|
||||
@ -247,10 +241,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> BlockchainProvider2<N> {
|
||||
/// Ensures that the given block number is canonical (synced)
|
||||
///
|
||||
/// This is a helper for guarding the `HistoricalStateProvider` against block numbers that are
|
||||
@ -270,25 +261,19 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> DatabaseProviderFactory<DB> for BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
fn database_provider_ro(&self) -> ProviderResult<DatabaseProviderRO<DB>> {
|
||||
impl<N: ProviderNodeTypes> DatabaseProviderFactory<N::DB> for BlockchainProvider2<N> {
|
||||
fn database_provider_ro(&self) -> ProviderResult<DatabaseProviderRO<N::DB>> {
|
||||
self.database.provider()
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> StaticFileProviderFactory for BlockchainProvider2<DB> {
|
||||
impl<N: ProviderNodeTypes> StaticFileProviderFactory for BlockchainProvider2<N> {
|
||||
fn static_file_provider(&self) -> StaticFileProvider {
|
||||
self.database.static_file_provider()
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> HeaderProvider for BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> HeaderProvider for BlockchainProvider2<N> {
|
||||
fn header(&self, block_hash: &BlockHash) -> ProviderResult<Option<Header>> {
|
||||
if let Some(block_state) = self.canonical_in_memory_state.state_by_hash(*block_hash) {
|
||||
return Ok(Some(block_state.block().block().header.header().clone()));
|
||||
@ -388,10 +373,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> BlockHashReader for BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> BlockHashReader for BlockchainProvider2<N> {
|
||||
fn block_hash(&self, number: u64) -> ProviderResult<Option<B256>> {
|
||||
if let Some(block_state) = self.canonical_in_memory_state.state_by_number(number) {
|
||||
return Ok(Some(block_state.hash()));
|
||||
@ -418,10 +400,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> BlockNumReader for BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> BlockNumReader for BlockchainProvider2<N> {
|
||||
fn chain_info(&self) -> ProviderResult<ChainInfo> {
|
||||
Ok(self.canonical_in_memory_state.chain_info())
|
||||
}
|
||||
@ -443,10 +422,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> BlockIdReader for BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> BlockIdReader for BlockchainProvider2<N> {
|
||||
fn pending_block_num_hash(&self) -> ProviderResult<Option<BlockNumHash>> {
|
||||
Ok(self.canonical_in_memory_state.pending_block_num_hash())
|
||||
}
|
||||
@ -460,10 +436,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> BlockReader for BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> BlockReader for BlockchainProvider2<N> {
|
||||
fn find_block_by_hash(&self, hash: B256, source: BlockSource) -> ProviderResult<Option<Block>> {
|
||||
match source {
|
||||
BlockSource::Any | BlockSource::Canonical => {
|
||||
@ -662,10 +635,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> TransactionsProvider for BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> TransactionsProvider for BlockchainProvider2<N> {
|
||||
fn transaction_id(&self, tx_hash: TxHash) -> ProviderResult<Option<TxNumber>> {
|
||||
// First, check the database
|
||||
if let Some(id) = self.database.transaction_id(tx_hash)? {
|
||||
@ -855,10 +825,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> ReceiptProvider for BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> ReceiptProvider for BlockchainProvider2<N> {
|
||||
fn receipt(&self, id: TxNumber) -> ProviderResult<Option<Receipt>> {
|
||||
let provider = self.database.provider()?;
|
||||
let Some((block_state, tx_index)) = self.block_state_by_tx_id(&provider, id)? else {
|
||||
@ -920,10 +887,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> ReceiptProviderIdExt for BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> ReceiptProviderIdExt for BlockchainProvider2<N> {
|
||||
fn receipts_by_block_id(&self, block: BlockId) -> ProviderResult<Option<Vec<Receipt>>> {
|
||||
match block {
|
||||
BlockId::Hash(rpc_block_hash) => {
|
||||
@ -954,10 +918,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> WithdrawalsProvider for BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> WithdrawalsProvider for BlockchainProvider2<N> {
|
||||
fn withdrawals_by_block(
|
||||
&self,
|
||||
id: BlockHashOrNumber,
|
||||
@ -989,10 +950,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> RequestsProvider for BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> RequestsProvider for BlockchainProvider2<N> {
|
||||
fn requests_by_block(
|
||||
&self,
|
||||
id: BlockHashOrNumber,
|
||||
@ -1010,10 +968,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> StageCheckpointReader for BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> StageCheckpointReader for BlockchainProvider2<N> {
|
||||
fn get_stage_checkpoint(&self, id: StageId) -> ProviderResult<Option<StageCheckpoint>> {
|
||||
self.database.provider()?.get_stage_checkpoint(id)
|
||||
}
|
||||
@ -1027,10 +982,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> EvmEnvProvider for BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> EvmEnvProvider for BlockchainProvider2<N> {
|
||||
fn fill_env_at<EvmConfig>(
|
||||
&self,
|
||||
cfg: &mut CfgEnvWithHandlerCfg,
|
||||
@ -1100,10 +1052,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> PruneCheckpointReader for BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> PruneCheckpointReader for BlockchainProvider2<N> {
|
||||
fn get_prune_checkpoint(
|
||||
&self,
|
||||
segment: PruneSegment,
|
||||
@ -1116,21 +1065,15 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> ChainSpecProvider for BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Send + Sync,
|
||||
{
|
||||
type ChainSpec = ChainSpec;
|
||||
impl<N: NodeTypesWithDB> ChainSpecProvider for BlockchainProvider2<N> {
|
||||
type ChainSpec = N::ChainSpec;
|
||||
|
||||
fn chain_spec(&self) -> Arc<ChainSpec> {
|
||||
fn chain_spec(&self) -> Arc<N::ChainSpec> {
|
||||
self.database.chain_spec()
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> StateProviderFactory for BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> StateProviderFactory for BlockchainProvider2<N> {
|
||||
/// Storage provider for latest block
|
||||
fn latest(&self) -> ProviderResult<StateProviderBox> {
|
||||
trace!(target: "providers::blockchain", "Getting latest block state provider");
|
||||
@ -1247,9 +1190,8 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> CanonChainTracker for BlockchainProvider2<DB>
|
||||
impl<N: NodeTypesWithDB> CanonChainTracker for BlockchainProvider2<N>
|
||||
where
|
||||
DB: Send + Sync,
|
||||
Self: BlockReader,
|
||||
{
|
||||
fn on_forkchoice_update_received(&self, _update: &ForkchoiceState) {
|
||||
@ -1282,7 +1224,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> BlockReaderIdExt for BlockchainProvider2<DB>
|
||||
impl<N: NodeTypesWithDB> BlockReaderIdExt for BlockchainProvider2<N>
|
||||
where
|
||||
Self: BlockReader + BlockIdReader + ReceiptProviderIdExt,
|
||||
{
|
||||
@ -1370,19 +1312,13 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> CanonStateSubscriptions for BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Send + Sync,
|
||||
{
|
||||
impl<N: NodeTypesWithDB> CanonStateSubscriptions for BlockchainProvider2<N> {
|
||||
fn subscribe_to_canonical_state(&self) -> CanonStateNotifications {
|
||||
self.canonical_in_memory_state.subscribe_canon_state()
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> ForkChoiceSubscriptions for BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Send + Sync,
|
||||
{
|
||||
impl<N: NodeTypesWithDB> ForkChoiceSubscriptions for BlockchainProvider2<N> {
|
||||
fn subscribe_safe_block(&self) -> ForkChoiceNotifications {
|
||||
let receiver = self.canonical_in_memory_state.subscribe_safe_block();
|
||||
ForkChoiceNotifications(receiver)
|
||||
@ -1394,10 +1330,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> ChangeSetReader for BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> ChangeSetReader for BlockchainProvider2<N> {
|
||||
fn account_block_changeset(
|
||||
&self,
|
||||
block_number: BlockNumber,
|
||||
@ -1422,10 +1355,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> AccountReader for BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Database + Sync + Send,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> AccountReader for BlockchainProvider2<N> {
|
||||
/// Get basic account information.
|
||||
fn basic_account(&self, address: Address) -> ProviderResult<Option<Account>> {
|
||||
// use latest state provider
|
||||
@ -1434,10 +1364,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> StateReader for BlockchainProvider2<DB>
|
||||
where
|
||||
DB: Database + Sync + Send,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> StateReader for BlockchainProvider2<N> {
|
||||
fn get_state(&self, block: BlockNumber) -> ProviderResult<Option<ExecutionOutcome>> {
|
||||
if let Some(state) = self.canonical_in_memory_state.state_by_number(block) {
|
||||
let state = state.block().execution_outcome().clone();
|
||||
@ -1458,7 +1385,10 @@ mod tests {
|
||||
|
||||
use crate::{
|
||||
providers::BlockchainProvider2,
|
||||
test_utils::{create_test_provider_factory, create_test_provider_factory_with_chain_spec},
|
||||
test_utils::{
|
||||
create_test_provider_factory, create_test_provider_factory_with_chain_spec,
|
||||
MockNodeTypesWithDB,
|
||||
},
|
||||
writer::UnifiedStorageWriter,
|
||||
BlockWriter, CanonChainTracker, StaticFileProviderFactory, StaticFileWriter,
|
||||
};
|
||||
@ -1471,11 +1401,7 @@ mod tests {
|
||||
use reth_chainspec::{
|
||||
ChainSpec, ChainSpecBuilder, ChainSpecProvider, EthereumHardfork, MAINNET,
|
||||
};
|
||||
use reth_db::{
|
||||
models::{AccountBeforeTx, StoredBlockBodyIndices},
|
||||
test_utils::TempDatabase,
|
||||
DatabaseEnv,
|
||||
};
|
||||
use reth_db::models::{AccountBeforeTx, StoredBlockBodyIndices};
|
||||
use reth_execution_types::{Chain, ExecutionOutcome};
|
||||
use reth_primitives::{
|
||||
BlockHashOrNumber, BlockNumHash, BlockNumberOrTag, BlockWithSenders, Receipt, SealedBlock,
|
||||
@ -1539,7 +1465,7 @@ mod tests {
|
||||
in_memory_blocks: usize,
|
||||
block_range_params: BlockRangeParams,
|
||||
) -> eyre::Result<(
|
||||
BlockchainProvider2<Arc<TempDatabase<DatabaseEnv>>>,
|
||||
BlockchainProvider2<MockNodeTypesWithDB>,
|
||||
Vec<SealedBlock>,
|
||||
Vec<SealedBlock>,
|
||||
Vec<Vec<Receipt>>,
|
||||
@ -1630,7 +1556,7 @@ mod tests {
|
||||
in_memory_blocks: usize,
|
||||
block_range_params: BlockRangeParams,
|
||||
) -> eyre::Result<(
|
||||
BlockchainProvider2<Arc<TempDatabase<DatabaseEnv>>>,
|
||||
BlockchainProvider2<MockNodeTypesWithDB>,
|
||||
Vec<SealedBlock>,
|
||||
Vec<SealedBlock>,
|
||||
Vec<Vec<Receipt>>,
|
||||
|
||||
@ -7,11 +7,13 @@ use crate::{
|
||||
PruneCheckpointReader, RequestsProvider, StageCheckpointReader, StateProviderBox,
|
||||
StaticFileProviderFactory, TransactionVariant, TransactionsProvider, WithdrawalsProvider,
|
||||
};
|
||||
use reth_chainspec::{ChainInfo, ChainSpec, EthChainSpec};
|
||||
use core::fmt;
|
||||
use reth_chainspec::ChainInfo;
|
||||
use reth_db::{init_db, mdbx::DatabaseArguments, DatabaseEnv};
|
||||
use reth_db_api::{database::Database, models::StoredBlockBodyIndices};
|
||||
use reth_errors::{RethError, RethResult};
|
||||
use reth_evm::ConfigureEvmEnv;
|
||||
use reth_node_types::NodeTypesWithDB;
|
||||
use reth_primitives::{
|
||||
Address, Block, BlockHash, BlockHashOrNumber, BlockNumber, BlockWithSenders, Header, Receipt,
|
||||
SealedBlock, SealedBlockWithSenders, SealedHeader, StaticFileSegment, TransactionMeta,
|
||||
@ -33,31 +35,47 @@ use tracing::trace;
|
||||
mod provider;
|
||||
pub use provider::{DatabaseProvider, DatabaseProviderRO, DatabaseProviderRW};
|
||||
|
||||
use super::ProviderNodeTypes;
|
||||
|
||||
mod metrics;
|
||||
|
||||
/// A common provider that fetches data from a database or static file.
|
||||
///
|
||||
/// This provider implements most provider or provider factory traits.
|
||||
#[derive(Debug)]
|
||||
pub struct ProviderFactory<DB, Spec = ChainSpec> {
|
||||
pub struct ProviderFactory<N: NodeTypesWithDB> {
|
||||
/// Database
|
||||
db: Arc<DB>,
|
||||
db: N::DB,
|
||||
/// Chain spec
|
||||
chain_spec: Arc<Spec>,
|
||||
chain_spec: Arc<N::ChainSpec>,
|
||||
/// Static File Provider
|
||||
static_file_provider: StaticFileProvider,
|
||||
/// Optional pruning configuration
|
||||
prune_modes: PruneModes,
|
||||
}
|
||||
|
||||
impl<DB> ProviderFactory<DB> {
|
||||
impl<N> fmt::Debug for ProviderFactory<N>
|
||||
where
|
||||
N: NodeTypesWithDB<DB: fmt::Debug, ChainSpec: fmt::Debug>,
|
||||
{
|
||||
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
|
||||
let Self { db, chain_spec, static_file_provider, prune_modes } = self;
|
||||
f.debug_struct("ProviderFactory")
|
||||
.field("db", &db)
|
||||
.field("chain_spec", &chain_spec)
|
||||
.field("static_file_provider", &static_file_provider)
|
||||
.field("prune_modes", &prune_modes)
|
||||
.finish()
|
||||
}
|
||||
}
|
||||
|
||||
impl<N: NodeTypesWithDB> ProviderFactory<N> {
|
||||
/// Create new database provider factory.
|
||||
pub fn new(
|
||||
db: DB,
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
db: N::DB,
|
||||
chain_spec: Arc<N::ChainSpec>,
|
||||
static_file_provider: StaticFileProvider,
|
||||
) -> Self {
|
||||
Self { db: Arc::new(db), chain_spec, static_file_provider, prune_modes: PruneModes::none() }
|
||||
Self { db, chain_spec, static_file_provider, prune_modes: PruneModes::none() }
|
||||
}
|
||||
|
||||
/// Enables metrics on the static file provider.
|
||||
@ -73,23 +91,23 @@ impl<DB> ProviderFactory<DB> {
|
||||
}
|
||||
|
||||
/// Returns reference to the underlying database.
|
||||
pub fn db_ref(&self) -> &DB {
|
||||
pub const fn db_ref(&self) -> &N::DB {
|
||||
&self.db
|
||||
}
|
||||
|
||||
#[cfg(any(test, feature = "test-utils"))]
|
||||
/// Consumes Self and returns DB
|
||||
pub fn into_db(self) -> Arc<DB> {
|
||||
pub fn into_db(self) -> N::DB {
|
||||
self.db
|
||||
}
|
||||
}
|
||||
|
||||
impl ProviderFactory<DatabaseEnv> {
|
||||
impl<N: NodeTypesWithDB<DB = Arc<DatabaseEnv>>> ProviderFactory<N> {
|
||||
/// Create new database provider by passing a path. [`ProviderFactory`] will own the database
|
||||
/// instance.
|
||||
pub fn new_with_database_path<P: AsRef<Path>>(
|
||||
path: P,
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
chain_spec: Arc<N::ChainSpec>,
|
||||
args: DatabaseArguments,
|
||||
static_file_provider: StaticFileProvider,
|
||||
) -> RethResult<Self> {
|
||||
@ -102,7 +120,7 @@ impl ProviderFactory<DatabaseEnv> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> ProviderFactory<DB> {
|
||||
impl<N: ProviderNodeTypes> ProviderFactory<N> {
|
||||
/// Returns a provider with a created `DbTx` inside, which allows fetching data from the
|
||||
/// database using different types of providers. Example: [`HeaderProvider`]
|
||||
/// [`BlockHashReader`]. This may fail if the inner read database transaction fails to open.
|
||||
@ -110,7 +128,7 @@ impl<DB: Database> ProviderFactory<DB> {
|
||||
/// This sets the [`PruneModes`] to [`None`], because they should only be relevant for writing
|
||||
/// data.
|
||||
#[track_caller]
|
||||
pub fn provider(&self) -> ProviderResult<DatabaseProviderRO<DB>> {
|
||||
pub fn provider(&self) -> ProviderResult<DatabaseProviderRO<N::DB>> {
|
||||
Ok(DatabaseProvider::new(
|
||||
self.db.tx()?,
|
||||
self.chain_spec.clone(),
|
||||
@ -124,7 +142,7 @@ impl<DB: Database> ProviderFactory<DB> {
|
||||
/// [`BlockHashReader`]. This may fail if the inner read/write database transaction fails to
|
||||
/// open.
|
||||
#[track_caller]
|
||||
pub fn provider_rw(&self) -> ProviderResult<DatabaseProviderRW<DB>> {
|
||||
pub fn provider_rw(&self) -> ProviderResult<DatabaseProviderRW<N::DB>> {
|
||||
Ok(DatabaseProviderRW(DatabaseProvider::new_rw(
|
||||
self.db.tx_mut()?,
|
||||
self.chain_spec.clone(),
|
||||
@ -164,20 +182,20 @@ impl<DB: Database> ProviderFactory<DB> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> DatabaseProviderFactory<DB> for ProviderFactory<DB> {
|
||||
fn database_provider_ro(&self) -> ProviderResult<DatabaseProviderRO<DB>> {
|
||||
impl<N: ProviderNodeTypes> DatabaseProviderFactory<N::DB> for ProviderFactory<N> {
|
||||
fn database_provider_ro(&self) -> ProviderResult<DatabaseProviderRO<N::DB>> {
|
||||
self.provider()
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> StaticFileProviderFactory for ProviderFactory<DB> {
|
||||
impl<N: NodeTypesWithDB> StaticFileProviderFactory for ProviderFactory<N> {
|
||||
/// Returns static file provider
|
||||
fn static_file_provider(&self) -> StaticFileProvider {
|
||||
self.static_file_provider.clone()
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> HeaderSyncGapProvider for ProviderFactory<DB> {
|
||||
impl<N: ProviderNodeTypes> HeaderSyncGapProvider for ProviderFactory<N> {
|
||||
fn sync_gap(
|
||||
&self,
|
||||
tip: watch::Receiver<B256>,
|
||||
@ -187,7 +205,7 @@ impl<DB: Database> HeaderSyncGapProvider for ProviderFactory<DB> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> HeaderProvider for ProviderFactory<DB> {
|
||||
impl<N: ProviderNodeTypes> HeaderProvider for ProviderFactory<N> {
|
||||
fn header(&self, block_hash: &BlockHash) -> ProviderResult<Option<Header>> {
|
||||
self.provider()?.header(block_hash)
|
||||
}
|
||||
@ -261,7 +279,7 @@ impl<DB: Database> HeaderProvider for ProviderFactory<DB> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> BlockHashReader for ProviderFactory<DB> {
|
||||
impl<N: ProviderNodeTypes> BlockHashReader for ProviderFactory<N> {
|
||||
fn block_hash(&self, number: u64) -> ProviderResult<Option<B256>> {
|
||||
self.static_file_provider.get_with_static_file_or_database(
|
||||
StaticFileSegment::Headers,
|
||||
@ -286,7 +304,7 @@ impl<DB: Database> BlockHashReader for ProviderFactory<DB> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> BlockNumReader for ProviderFactory<DB> {
|
||||
impl<N: ProviderNodeTypes> BlockNumReader for ProviderFactory<N> {
|
||||
fn chain_info(&self) -> ProviderResult<ChainInfo> {
|
||||
self.provider()?.chain_info()
|
||||
}
|
||||
@ -304,7 +322,7 @@ impl<DB: Database> BlockNumReader for ProviderFactory<DB> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> BlockReader for ProviderFactory<DB> {
|
||||
impl<N: ProviderNodeTypes> BlockReader for ProviderFactory<N> {
|
||||
fn find_block_by_hash(&self, hash: B256, source: BlockSource) -> ProviderResult<Option<Block>> {
|
||||
self.provider()?.find_block_by_hash(hash, source)
|
||||
}
|
||||
@ -371,7 +389,7 @@ impl<DB: Database> BlockReader for ProviderFactory<DB> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> TransactionsProvider for ProviderFactory<DB> {
|
||||
impl<N: ProviderNodeTypes> TransactionsProvider for ProviderFactory<N> {
|
||||
fn transaction_id(&self, tx_hash: TxHash) -> ProviderResult<Option<TxNumber>> {
|
||||
self.provider()?.transaction_id(tx_hash)
|
||||
}
|
||||
@ -445,7 +463,7 @@ impl<DB: Database> TransactionsProvider for ProviderFactory<DB> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> ReceiptProvider for ProviderFactory<DB> {
|
||||
impl<N: ProviderNodeTypes> ReceiptProvider for ProviderFactory<N> {
|
||||
fn receipt(&self, id: TxNumber) -> ProviderResult<Option<Receipt>> {
|
||||
self.static_file_provider.get_with_static_file_or_database(
|
||||
StaticFileSegment::Receipts,
|
||||
@ -477,7 +495,7 @@ impl<DB: Database> ReceiptProvider for ProviderFactory<DB> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> WithdrawalsProvider for ProviderFactory<DB> {
|
||||
impl<N: ProviderNodeTypes> WithdrawalsProvider for ProviderFactory<N> {
|
||||
fn withdrawals_by_block(
|
||||
&self,
|
||||
id: BlockHashOrNumber,
|
||||
@ -491,10 +509,7 @@ impl<DB: Database> WithdrawalsProvider for ProviderFactory<DB> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> RequestsProvider for ProviderFactory<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> RequestsProvider for ProviderFactory<N> {
|
||||
fn requests_by_block(
|
||||
&self,
|
||||
id: BlockHashOrNumber,
|
||||
@ -504,7 +519,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> StageCheckpointReader for ProviderFactory<DB> {
|
||||
impl<N: ProviderNodeTypes> StageCheckpointReader for ProviderFactory<N> {
|
||||
fn get_stage_checkpoint(&self, id: StageId) -> ProviderResult<Option<StageCheckpoint>> {
|
||||
self.provider()?.get_stage_checkpoint(id)
|
||||
}
|
||||
@ -517,7 +532,7 @@ impl<DB: Database> StageCheckpointReader for ProviderFactory<DB> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> EvmEnvProvider for ProviderFactory<DB> {
|
||||
impl<N: ProviderNodeTypes> EvmEnvProvider for ProviderFactory<N> {
|
||||
fn fill_env_at<EvmConfig>(
|
||||
&self,
|
||||
cfg: &mut CfgEnvWithHandlerCfg,
|
||||
@ -569,18 +584,15 @@ impl<DB: Database> EvmEnvProvider for ProviderFactory<DB> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB, ChainSpec> ChainSpecProvider for ProviderFactory<DB, ChainSpec>
|
||||
where
|
||||
DB: Send + Sync,
|
||||
ChainSpec: EthChainSpec,
|
||||
{
|
||||
type ChainSpec = ChainSpec;
|
||||
fn chain_spec(&self) -> Arc<ChainSpec> {
|
||||
impl<N: NodeTypesWithDB> ChainSpecProvider for ProviderFactory<N> {
|
||||
type ChainSpec = N::ChainSpec;
|
||||
|
||||
fn chain_spec(&self) -> Arc<N::ChainSpec> {
|
||||
self.chain_spec.clone()
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> PruneCheckpointReader for ProviderFactory<DB> {
|
||||
impl<N: ProviderNodeTypes> PruneCheckpointReader for ProviderFactory<N> {
|
||||
fn get_prune_checkpoint(
|
||||
&self,
|
||||
segment: PruneSegment,
|
||||
@ -593,10 +605,10 @@ impl<DB: Database> PruneCheckpointReader for ProviderFactory<DB> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB, Spec> Clone for ProviderFactory<DB, Spec> {
|
||||
impl<N: NodeTypesWithDB> Clone for ProviderFactory<N> {
|
||||
fn clone(&self) -> Self {
|
||||
Self {
|
||||
db: Arc::clone(&self.db),
|
||||
db: self.db.clone(),
|
||||
chain_spec: self.chain_spec.clone(),
|
||||
static_file_provider: self.static_file_provider.clone(),
|
||||
prune_modes: self.prune_modes.clone(),
|
||||
@ -609,7 +621,7 @@ mod tests {
|
||||
use super::*;
|
||||
use crate::{
|
||||
providers::{StaticFileProvider, StaticFileWriter},
|
||||
test_utils::{blocks::TEST_BLOCK, create_test_provider_factory},
|
||||
test_utils::{blocks::TEST_BLOCK, create_test_provider_factory, MockNodeTypesWithDB},
|
||||
BlockHashReader, BlockNumReader, BlockWriter, HeaderSyncGapProvider, TransactionsProvider,
|
||||
};
|
||||
use assert_matches::assert_matches;
|
||||
@ -657,7 +669,7 @@ mod tests {
|
||||
fn provider_factory_with_database_path() {
|
||||
let chain_spec = ChainSpecBuilder::mainnet().build();
|
||||
let (_static_dir, static_dir_path) = create_test_static_files_dir();
|
||||
let factory = ProviderFactory::new_with_database_path(
|
||||
let factory = ProviderFactory::<MockNodeTypesWithDB<DatabaseEnv>>::new_with_database_path(
|
||||
tempfile::TempDir::new().expect(ERROR_TEMPDIR).into_path(),
|
||||
Arc::new(chain_spec),
|
||||
DatabaseArguments::new(Default::default()),
|
||||
|
||||
@ -13,12 +13,10 @@ use reth_blockchain_tree_api::{
|
||||
InsertPayloadOk,
|
||||
};
|
||||
use reth_chain_state::{ChainInfoTracker, ForkChoiceNotifications, ForkChoiceSubscriptions};
|
||||
use reth_chainspec::{ChainInfo, ChainSpec, EthChainSpec};
|
||||
use reth_db_api::{
|
||||
database::Database,
|
||||
models::{AccountBeforeTx, StoredBlockBodyIndices},
|
||||
};
|
||||
use reth_chainspec::{ChainInfo, ChainSpec};
|
||||
use reth_db_api::models::{AccountBeforeTx, StoredBlockBodyIndices};
|
||||
use reth_evm::ConfigureEvmEnv;
|
||||
use reth_node_types::NodeTypesWithDB;
|
||||
use reth_primitives::{
|
||||
Account, Address, Block, BlockHash, BlockHashOrNumber, BlockId, BlockNumHash, BlockNumber,
|
||||
BlockNumberOrTag, BlockWithSenders, Header, Receipt, SealedBlock, SealedBlockWithSenders,
|
||||
@ -62,22 +60,27 @@ pub use consistent_view::{ConsistentDbView, ConsistentViewError};
|
||||
mod blockchain_provider;
|
||||
pub use blockchain_provider::BlockchainProvider2;
|
||||
|
||||
/// Helper trait keeping common requirements of providers for [`NodeTypesWithDB`].
|
||||
pub trait ProviderNodeTypes: NodeTypesWithDB<ChainSpec = ChainSpec> {}
|
||||
|
||||
impl<T> ProviderNodeTypes for T where T: NodeTypesWithDB<ChainSpec = ChainSpec> {}
|
||||
|
||||
/// The main type for interacting with the blockchain.
|
||||
///
|
||||
/// This type serves as the main entry point for interacting with the blockchain and provides data
|
||||
/// from database storage and from the blockchain tree (pending state etc.) It is a simple wrapper
|
||||
/// type that holds an instance of the database and the blockchain tree.
|
||||
#[allow(missing_debug_implementations)]
|
||||
pub struct BlockchainProvider<DB, Spec = ChainSpec> {
|
||||
pub struct BlockchainProvider<N: NodeTypesWithDB> {
|
||||
/// Provider type used to access the database.
|
||||
database: ProviderFactory<DB, Spec>,
|
||||
database: ProviderFactory<N>,
|
||||
/// The blockchain tree instance.
|
||||
tree: Arc<dyn TreeViewer>,
|
||||
/// Tracks the chain info wrt forkchoice updates
|
||||
chain_info: ChainInfoTracker,
|
||||
}
|
||||
|
||||
impl<DB, Spec> Clone for BlockchainProvider<DB, Spec> {
|
||||
impl<N: ProviderNodeTypes> Clone for BlockchainProvider<N> {
|
||||
fn clone(&self) -> Self {
|
||||
Self {
|
||||
database: self.database.clone(),
|
||||
@ -87,7 +90,7 @@ impl<DB, Spec> Clone for BlockchainProvider<DB, Spec> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> BlockchainProvider<DB> {
|
||||
impl<N: NodeTypesWithDB> BlockchainProvider<N> {
|
||||
/// Sets the treeviewer for the provider.
|
||||
#[doc(hidden)]
|
||||
pub fn with_tree(mut self, tree: Arc<dyn TreeViewer>) -> Self {
|
||||
@ -96,15 +99,12 @@ impl<DB> BlockchainProvider<DB> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> BlockchainProvider<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> BlockchainProvider<N> {
|
||||
/// Create new provider instance that wraps the database and the blockchain tree, using the
|
||||
/// provided latest header to initialize the chain info tracker, alongside the finalized header
|
||||
/// if it exists.
|
||||
pub fn with_blocks(
|
||||
database: ProviderFactory<DB>,
|
||||
database: ProviderFactory<N>,
|
||||
tree: Arc<dyn TreeViewer>,
|
||||
latest: SealedHeader,
|
||||
finalized: Option<SealedHeader>,
|
||||
@ -114,7 +114,7 @@ where
|
||||
|
||||
/// Create a new provider using only the database and the tree, fetching the latest header from
|
||||
/// the database to initialize the provider.
|
||||
pub fn new(database: ProviderFactory<DB>, tree: Arc<dyn TreeViewer>) -> ProviderResult<Self> {
|
||||
pub fn new(database: ProviderFactory<N>, tree: Arc<dyn TreeViewer>) -> ProviderResult<Self> {
|
||||
let provider = database.provider()?;
|
||||
let best: ChainInfo = provider.chain_info()?;
|
||||
let latest_header = provider
|
||||
@ -149,10 +149,10 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> BlockchainProvider<DB>
|
||||
impl<N> BlockchainProvider<N>
|
||||
where
|
||||
Self: StateProviderFactory,
|
||||
DB: Database,
|
||||
N: NodeTypesWithDB,
|
||||
{
|
||||
/// Return a [`StateProviderBox`] that contains bundle state data provider.
|
||||
/// Used to inspect or execute transaction on the pending state.
|
||||
@ -169,25 +169,19 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> DatabaseProviderFactory<DB> for BlockchainProvider<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
fn database_provider_ro(&self) -> ProviderResult<DatabaseProviderRO<DB>> {
|
||||
impl<N: ProviderNodeTypes> DatabaseProviderFactory<N::DB> for BlockchainProvider<N> {
|
||||
fn database_provider_ro(&self) -> ProviderResult<DatabaseProviderRO<N::DB>> {
|
||||
self.database.provider()
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> StaticFileProviderFactory for BlockchainProvider<DB> {
|
||||
impl<N: ProviderNodeTypes> StaticFileProviderFactory for BlockchainProvider<N> {
|
||||
fn static_file_provider(&self) -> StaticFileProvider {
|
||||
self.database.static_file_provider()
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> HeaderProvider for BlockchainProvider<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> HeaderProvider for BlockchainProvider<N> {
|
||||
fn header(&self, block_hash: &BlockHash) -> ProviderResult<Option<Header>> {
|
||||
self.database.header(block_hash)
|
||||
}
|
||||
@ -228,10 +222,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> BlockHashReader for BlockchainProvider<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> BlockHashReader for BlockchainProvider<N> {
|
||||
fn block_hash(&self, number: u64) -> ProviderResult<Option<B256>> {
|
||||
self.database.block_hash(number)
|
||||
}
|
||||
@ -245,10 +236,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> BlockNumReader for BlockchainProvider<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> BlockNumReader for BlockchainProvider<N> {
|
||||
fn chain_info(&self) -> ProviderResult<ChainInfo> {
|
||||
Ok(self.chain_info.chain_info())
|
||||
}
|
||||
@ -266,10 +254,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> BlockIdReader for BlockchainProvider<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> BlockIdReader for BlockchainProvider<N> {
|
||||
fn pending_block_num_hash(&self) -> ProviderResult<Option<BlockNumHash>> {
|
||||
Ok(self.tree.pending_block_num_hash())
|
||||
}
|
||||
@ -283,10 +268,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> BlockReader for BlockchainProvider<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> BlockReader for BlockchainProvider<N> {
|
||||
fn find_block_by_hash(&self, hash: B256, source: BlockSource) -> ProviderResult<Option<Block>> {
|
||||
let block = match source {
|
||||
BlockSource::Any => {
|
||||
@ -377,10 +359,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> TransactionsProvider for BlockchainProvider<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> TransactionsProvider for BlockchainProvider<N> {
|
||||
fn transaction_id(&self, tx_hash: TxHash) -> ProviderResult<Option<TxNumber>> {
|
||||
self.database.transaction_id(tx_hash)
|
||||
}
|
||||
@ -444,10 +423,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> ReceiptProvider for BlockchainProvider<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> ReceiptProvider for BlockchainProvider<N> {
|
||||
fn receipt(&self, id: TxNumber) -> ProviderResult<Option<Receipt>> {
|
||||
self.database.receipt(id)
|
||||
}
|
||||
@ -468,10 +444,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> ReceiptProviderIdExt for BlockchainProvider<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> ReceiptProviderIdExt for BlockchainProvider<N> {
|
||||
fn receipts_by_block_id(&self, block: BlockId) -> ProviderResult<Option<Vec<Receipt>>> {
|
||||
match block {
|
||||
BlockId::Hash(rpc_block_hash) => {
|
||||
@ -495,10 +468,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> WithdrawalsProvider for BlockchainProvider<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> WithdrawalsProvider for BlockchainProvider<N> {
|
||||
fn withdrawals_by_block(
|
||||
&self,
|
||||
id: BlockHashOrNumber,
|
||||
@ -512,10 +482,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> RequestsProvider for BlockchainProvider<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> RequestsProvider for BlockchainProvider<N> {
|
||||
fn requests_by_block(
|
||||
&self,
|
||||
id: BlockHashOrNumber,
|
||||
@ -525,10 +492,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> StageCheckpointReader for BlockchainProvider<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> StageCheckpointReader for BlockchainProvider<N> {
|
||||
fn get_stage_checkpoint(&self, id: StageId) -> ProviderResult<Option<StageCheckpoint>> {
|
||||
self.database.provider()?.get_stage_checkpoint(id)
|
||||
}
|
||||
@ -542,10 +506,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> EvmEnvProvider for BlockchainProvider<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> EvmEnvProvider for BlockchainProvider<N> {
|
||||
fn fill_env_at<EvmConfig>(
|
||||
&self,
|
||||
cfg: &mut CfgEnvWithHandlerCfg,
|
||||
@ -597,10 +558,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> PruneCheckpointReader for BlockchainProvider<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> PruneCheckpointReader for BlockchainProvider<N> {
|
||||
fn get_prune_checkpoint(
|
||||
&self,
|
||||
segment: PruneSegment,
|
||||
@ -613,22 +571,15 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB, ChainSpec> ChainSpecProvider for BlockchainProvider<DB, ChainSpec>
|
||||
where
|
||||
DB: Send + Sync,
|
||||
ChainSpec: EthChainSpec,
|
||||
{
|
||||
type ChainSpec = ChainSpec;
|
||||
impl<N: ProviderNodeTypes> ChainSpecProvider for BlockchainProvider<N> {
|
||||
type ChainSpec = N::ChainSpec;
|
||||
|
||||
fn chain_spec(&self) -> Arc<Self::ChainSpec> {
|
||||
self.database.chain_spec()
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> StateProviderFactory for BlockchainProvider<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> StateProviderFactory for BlockchainProvider<N> {
|
||||
/// Storage provider for latest block
|
||||
fn latest(&self) -> ProviderResult<StateProviderBox> {
|
||||
trace!(target: "providers::blockchain", "Getting latest block state provider");
|
||||
@ -721,10 +672,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> BlockchainTreeEngine for BlockchainProvider<DB>
|
||||
where
|
||||
DB: Send + Sync,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> BlockchainTreeEngine for BlockchainProvider<N> {
|
||||
fn buffer_block(&self, block: SealedBlockWithSenders) -> Result<(), InsertBlockError> {
|
||||
self.tree.buffer_block(block)
|
||||
}
|
||||
@ -763,10 +711,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> BlockchainTreeViewer for BlockchainProvider<DB>
|
||||
where
|
||||
DB: Send + Sync,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> BlockchainTreeViewer for BlockchainProvider<N> {
|
||||
fn header_by_hash(&self, hash: BlockHash) -> Option<SealedHeader> {
|
||||
self.tree.header_by_hash(hash)
|
||||
}
|
||||
@ -808,9 +753,8 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> CanonChainTracker for BlockchainProvider<DB>
|
||||
impl<N: ProviderNodeTypes> CanonChainTracker for BlockchainProvider<N>
|
||||
where
|
||||
DB: Send + Sync,
|
||||
Self: BlockReader,
|
||||
{
|
||||
fn on_forkchoice_update_received(&self, _update: &ForkchoiceState) {
|
||||
@ -843,7 +787,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> BlockReaderIdExt for BlockchainProvider<DB>
|
||||
impl<N: ProviderNodeTypes> BlockReaderIdExt for BlockchainProvider<N>
|
||||
where
|
||||
Self: BlockReader + BlockIdReader + ReceiptProviderIdExt,
|
||||
{
|
||||
@ -859,7 +803,7 @@ where
|
||||
// check the database, canonical blocks are only stored in the database
|
||||
self.find_block_by_hash(hash.block_hash, BlockSource::Canonical)
|
||||
} else {
|
||||
self.block_by_hash(hash.block_hash)
|
||||
BlockReader::block_by_hash(self, hash.block_hash)
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -922,10 +866,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> BlockchainTreePendingStateProvider for BlockchainProvider<DB>
|
||||
where
|
||||
DB: Send + Sync,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> BlockchainTreePendingStateProvider for BlockchainProvider<N> {
|
||||
fn find_pending_state_provider(
|
||||
&self,
|
||||
block_hash: BlockHash,
|
||||
@ -934,19 +875,13 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> CanonStateSubscriptions for BlockchainProvider<DB>
|
||||
where
|
||||
DB: Send + Sync,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> CanonStateSubscriptions for BlockchainProvider<N> {
|
||||
fn subscribe_to_canonical_state(&self) -> CanonStateNotifications {
|
||||
self.tree.subscribe_to_canonical_state()
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> ForkChoiceSubscriptions for BlockchainProvider<DB>
|
||||
where
|
||||
DB: Send + Sync,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> ForkChoiceSubscriptions for BlockchainProvider<N> {
|
||||
fn subscribe_safe_block(&self) -> ForkChoiceNotifications {
|
||||
let receiver = self.chain_info.subscribe_safe_block();
|
||||
ForkChoiceNotifications(receiver)
|
||||
@ -958,10 +893,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> ChangeSetReader for BlockchainProvider<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> ChangeSetReader for BlockchainProvider<N> {
|
||||
fn account_block_changeset(
|
||||
&self,
|
||||
block_number: BlockNumber,
|
||||
@ -970,10 +902,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> AccountReader for BlockchainProvider<DB>
|
||||
where
|
||||
DB: Database + Sync + Send,
|
||||
{
|
||||
impl<N: ProviderNodeTypes> AccountReader for BlockchainProvider<N> {
|
||||
/// Get basic account information.
|
||||
fn basic_account(&self, address: Address) -> ProviderResult<Option<Account>> {
|
||||
self.database.provider()?.basic_account(address)
|
||||
|
||||
@ -2,9 +2,10 @@ use crate::{providers::StaticFileProvider, HashingWriter, ProviderFactory, TrieW
|
||||
use reth_chainspec::{ChainSpec, MAINNET};
|
||||
use reth_db::{
|
||||
test_utils::{create_test_rw_db, create_test_static_files_dir, TempDatabase},
|
||||
Database, DatabaseEnv,
|
||||
DatabaseEnv,
|
||||
};
|
||||
use reth_errors::ProviderResult;
|
||||
use reth_node_types::{NodeTypesWithDB, NodeTypesWithDBAdapter};
|
||||
use reth_primitives::{Account, StorageEntry, B256};
|
||||
use reth_trie::StateRoot;
|
||||
use reth_trie_db::DatabaseStateRoot;
|
||||
@ -18,15 +19,26 @@ pub use mock::{ExtendedAccount, MockEthProvider};
|
||||
pub use noop::NoopProvider;
|
||||
pub use reth_chain_state::test_utils::TestCanonStateSubscriptions;
|
||||
|
||||
/// Mock [`reth_node_types::NodeTypes`] for testing.
|
||||
pub type MockNodeTypes = reth_node_types::AnyNodeTypesWithEngine<
|
||||
(),
|
||||
reth_ethereum_engine_primitives::EthEngineTypes,
|
||||
reth_chainspec::ChainSpec,
|
||||
>;
|
||||
|
||||
/// Mock [`reth_node_types::NodeTypesWithDB`] for testing.
|
||||
pub type MockNodeTypesWithDB<DB = TempDatabase<DatabaseEnv>> =
|
||||
NodeTypesWithDBAdapter<MockNodeTypes, Arc<DB>>;
|
||||
|
||||
/// Creates test provider factory with mainnet chain spec.
|
||||
pub fn create_test_provider_factory() -> ProviderFactory<Arc<TempDatabase<DatabaseEnv>>> {
|
||||
pub fn create_test_provider_factory() -> ProviderFactory<MockNodeTypesWithDB> {
|
||||
create_test_provider_factory_with_chain_spec(MAINNET.clone())
|
||||
}
|
||||
|
||||
/// Creates test provider factory with provided chain spec.
|
||||
pub fn create_test_provider_factory_with_chain_spec(
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
) -> ProviderFactory<Arc<TempDatabase<DatabaseEnv>>> {
|
||||
) -> ProviderFactory<MockNodeTypesWithDB> {
|
||||
let (static_dir, _) = create_test_static_files_dir();
|
||||
let db = create_test_rw_db();
|
||||
ProviderFactory::new(
|
||||
@ -37,9 +49,9 @@ pub fn create_test_provider_factory_with_chain_spec(
|
||||
}
|
||||
|
||||
/// Inserts the genesis alloc from the provided chain spec into the trie.
|
||||
pub fn insert_genesis<DB: Database>(
|
||||
provider_factory: &ProviderFactory<DB>,
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
pub fn insert_genesis<N: NodeTypesWithDB<ChainSpec = ChainSpec>>(
|
||||
provider_factory: &ProviderFactory<N>,
|
||||
chain_spec: Arc<N::ChainSpec>,
|
||||
) -> ProviderResult<B256> {
|
||||
let provider = provider_factory.provider_rw()?;
|
||||
|
||||
|
||||
@ -55,6 +55,7 @@ reth-provider = { workspace = true, features = ["test-utils"] }
|
||||
reth-storage-errors.workspace = true
|
||||
reth-trie-common = { workspace = true, features = ["test-utils", "arbitrary"] }
|
||||
reth-trie = { workspace = true, features = ["test-utils"] }
|
||||
reth-node-types.workspace = true
|
||||
|
||||
# trie
|
||||
triehash = "0.8"
|
||||
|
||||
@ -1,15 +1,9 @@
|
||||
use reth_chainspec::{Chain, ChainSpec, HOLESKY, MAINNET};
|
||||
use reth_db_api::database::Database;
|
||||
use reth_primitives::{
|
||||
constants::EMPTY_ROOT_HASH, keccak256, Account, Address, Bytes, StorageEntry, B256, U256,
|
||||
};
|
||||
use reth_provider::{
|
||||
test_utils::create_test_provider_factory, HashingWriter, ProviderFactory, TrieWriter,
|
||||
};
|
||||
use reth_storage_errors::provider::ProviderResult;
|
||||
use reth_trie::{proof::Proof, Nibbles, StateRoot};
|
||||
use reth_primitives::{constants::EMPTY_ROOT_HASH, keccak256, Account, Address, Bytes, B256, U256};
|
||||
use reth_provider::test_utils::{create_test_provider_factory, insert_genesis};
|
||||
use reth_trie::{proof::Proof, Nibbles};
|
||||
use reth_trie_common::{AccountProof, StorageProof};
|
||||
use reth_trie_db::{DatabaseProof, DatabaseStateRoot};
|
||||
use reth_trie_db::DatabaseProof;
|
||||
use std::{
|
||||
str::FromStr,
|
||||
sync::{Arc, LazyLock},
|
||||
@ -40,39 +34,6 @@ fn convert_to_proof<'a>(path: impl IntoIterator<Item = &'a str>) -> Vec<Bytes> {
|
||||
path.into_iter().map(Bytes::from_str).collect::<Result<Vec<_>, _>>().unwrap()
|
||||
}
|
||||
|
||||
fn insert_genesis<DB: Database>(
|
||||
provider_factory: &ProviderFactory<DB>,
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
) -> ProviderResult<B256> {
|
||||
let provider = provider_factory.provider_rw()?;
|
||||
|
||||
// Hash accounts and insert them into hashing table.
|
||||
let genesis = chain_spec.genesis();
|
||||
let alloc_accounts =
|
||||
genesis.alloc.iter().map(|(addr, account)| (*addr, Some(Account::from(account))));
|
||||
provider.insert_account_for_hashing(alloc_accounts).unwrap();
|
||||
|
||||
let alloc_storage = genesis.alloc.clone().into_iter().filter_map(|(addr, account)| {
|
||||
// Only return `Some` if there is storage.
|
||||
account.storage.map(|storage| {
|
||||
(
|
||||
addr,
|
||||
storage.into_iter().map(|(key, value)| StorageEntry { key, value: value.into() }),
|
||||
)
|
||||
})
|
||||
});
|
||||
provider.insert_storage_for_hashing(alloc_storage)?;
|
||||
|
||||
let (root, updates) = StateRoot::from_tx(provider.tx_ref())
|
||||
.root_with_updates()
|
||||
.map_err(Into::<reth_db::DatabaseError>::into)?;
|
||||
provider.write_trie_updates(&updates).unwrap();
|
||||
|
||||
provider.commit()?;
|
||||
|
||||
Ok(root)
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn testspec_proofs() {
|
||||
// Create test database and insert genesis accounts.
|
||||
|
||||
Reference in New Issue
Block a user