chore: Move consensus trait to standalone crate (#7757)

Co-authored-by: Matthias Seitz <matthias.seitz@outlook.de>
This commit is contained in:
ThreeHrSleep
2024-04-23 17:20:19 +05:30
committed by GitHub
parent 3224837523
commit 6d2e20cd85
66 changed files with 185 additions and 146 deletions

View File

@ -19,6 +19,7 @@ reth-provider.workspace = true
reth-stages-api.workspace = true
reth-trie = { workspace = true, features = ["metrics"] }
reth-trie-parallel = { workspace = true, features = ["parallel"] }
reth-consensus.workspace = true
# common
parking_lot.workspace = true

View File

@ -5,13 +5,13 @@ use crate::{
state::{BlockChainId, TreeState},
AppendableChain, BlockIndices, BlockchainTreeConfig, BundleStateData, TreeExternals,
};
use reth_consensus::{Consensus, ConsensusError};
use reth_db::database::Database;
use reth_interfaces::{
blockchain_tree::{
error::{BlockchainTreeError, CanonicalError, InsertBlockError, InsertBlockErrorKind},
BlockAttachment, BlockStatus, BlockValidationKind, CanonicalOutcome, InsertPayloadOk,
},
consensus::{Consensus, ConsensusError},
executor::{BlockExecutionError, BlockValidationError},
provider::RootMismatch,
RethResult,

View File

@ -5,13 +5,13 @@
use super::externals::TreeExternals;
use crate::BundleStateDataRef;
use reth_consensus::{Consensus, ConsensusError};
use reth_db::database::Database;
use reth_interfaces::{
blockchain_tree::{
error::{BlockchainTreeError, InsertBlockErrorKind},
BlockAttachment, BlockValidationKind,
},
consensus::{Consensus, ConsensusError},
RethResult,
};
use reth_primitives::{

View File

@ -1,9 +1,10 @@
//! Blockchain tree externals.
use reth_consensus::Consensus;
use reth_db::{
cursor::DbCursorRO, database::Database, static_file::HeaderMask, tables, transaction::DbTx,
};
use reth_interfaces::{consensus::Consensus, RethResult};
use reth_interfaces::RethResult;
use reth_primitives::{BlockHash, BlockNumber, StaticFileSegment};
use reth_provider::{ProviderFactory, StatsReader};
use std::{collections::BTreeMap, sync::Arc};

View File

@ -22,6 +22,8 @@ reth-revm.workspace = true
reth-transaction-pool.workspace = true
reth-evm.workspace = true
reth-engine-primitives.workspace = true
reth-consensus.workspace = true
reth-rpc-types.workspace = true
# async
futures-util.workspace = true

View File

@ -16,12 +16,10 @@
#![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))]
use reth_beacon_consensus::BeaconEngineMessage;
use reth_consensus::{Consensus, ConsensusError};
use reth_engine_primitives::EngineTypes;
use reth_evm::ConfigureEvm;
use reth_interfaces::{
consensus::{Consensus, ConsensusError},
executor::{BlockExecutionError, BlockValidationError},
};
use reth_interfaces::executor::{BlockExecutionError, BlockValidationError};
use reth_primitives::{
constants::{EMPTY_RECEIPTS, EMPTY_TRANSACTIONS, ETHEREUM_BLOCK_GAS_LIMIT},
eip4844::calculate_excess_blob_gas,

View File

@ -3,9 +3,9 @@ use futures_util::{future::BoxFuture, FutureExt};
use reth_beacon_consensus::{BeaconEngineMessage, ForkchoiceStatus};
use reth_engine_primitives::EngineTypes;
use reth_evm::ConfigureEvm;
use reth_interfaces::consensus::ForkchoiceState;
use reth_primitives::{Block, ChainSpec, IntoRecoveredTransaction, SealedBlockWithSenders};
use reth_provider::{CanonChainTracker, CanonStateNotificationSender, Chain, StateProviderFactory};
use reth_rpc_types::engine::ForkchoiceState;
use reth_stages_api::PipelineEvent;
use reth_transaction_pool::{TransactionPool, ValidPoolTransaction};
use std::{

View File

@ -14,7 +14,7 @@ workspace = true
# reth
reth-consensus-common.workspace = true
reth-primitives.workspace = true
reth-interfaces.workspace = true
reth-consensus.workspace = true
[features]
optimism = ["reth-primitives/optimism"]

View File

@ -8,8 +8,8 @@
#![cfg_attr(not(test), warn(unused_crate_dependencies))]
#![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))]
use reth_consensus::{Consensus, ConsensusError};
use reth_consensus_common::validation;
use reth_interfaces::consensus::{Consensus, ConsensusError};
use reth_primitives::{
constants::MAXIMUM_EXTRA_DATA_SIZE, Chain, ChainSpec, Hardfork, Header, SealedBlock,
SealedHeader, EMPTY_OMMER_ROOT_HASH, U256,

View File

@ -57,6 +57,7 @@ reth-downloaders.workspace = true
reth-evm-ethereum.workspace = true
reth-ethereum-engine-primitives.workspace = true
reth-config.workspace = true
reth-consensus.workspace = true
assert_matches.workspace = true

View File

@ -1,6 +1,6 @@
use crate::engine::forkchoice::ForkchoiceStatus;
use reth_interfaces::consensus::ForkchoiceState;
use reth_primitives::{SealedBlock, SealedHeader, B256};
use reth_rpc_types::engine::ForkchoiceState;
use std::{sync::Arc, time::Duration};
/// Events emitted by [crate::BeaconConsensusEngine].

View File

@ -4,11 +4,11 @@ use crate::{
};
use futures::{future::Either, FutureExt};
use reth_engine_primitives::EngineTypes;
use reth_interfaces::{consensus::ForkchoiceState, RethResult};
use reth_interfaces::RethResult;
use reth_payload_builder::error::PayloadBuilderError;
use reth_rpc_types::engine::{
CancunPayloadFields, ExecutionPayload, ForkChoiceUpdateResult, ForkchoiceUpdateError,
ForkchoiceUpdated, PayloadId, PayloadStatus, PayloadStatusEnum,
CancunPayloadFields, ExecutionPayload, ForkChoiceUpdateResult, ForkchoiceState,
ForkchoiceUpdateError, ForkchoiceUpdated, PayloadId, PayloadStatus, PayloadStatusEnum,
};
use std::{
future::Future,

View File

@ -14,7 +14,6 @@ use reth_interfaces::{
error::{BlockchainTreeError, CanonicalError, InsertBlockError, InsertBlockErrorKind},
BlockStatus, BlockchainTreeEngine, CanonicalOutcome, InsertPayloadOk,
},
consensus::ForkchoiceState,
executor::BlockValidationError,
p2p::{bodies::client::BodiesClient, headers::client::HeadersClient},
provider::ProviderResult,
@ -31,7 +30,8 @@ use reth_provider::{
StageCheckpointReader,
};
use reth_rpc_types::engine::{
CancunPayloadFields, ExecutionPayload, PayloadStatus, PayloadStatusEnum, PayloadValidationError,
CancunPayloadFields, ExecutionPayload, ForkchoiceState, PayloadStatus, PayloadStatusEnum,
PayloadValidationError,
};
use reth_stages_api::{ControlFlow, Pipeline};
use reth_tasks::TaskSpawner;

View File

@ -7,16 +7,15 @@ use reth_blockchain_tree::{
config::BlockchainTreeConfig, externals::TreeExternals, BlockchainTree, ShareableBlockchainTree,
};
use reth_config::config::EtlConfig;
use reth_consensus::Consensus;
use reth_db::{test_utils::TempDatabase, DatabaseEnv as DE};
use reth_ethereum_engine_primitives::EthEngineTypes;
use reth_evm_ethereum::EthEvmConfig;
type DatabaseEnv = TempDatabase<DE>;
use reth_downloaders::{
bodies::bodies::BodiesDownloaderBuilder,
headers::reverse_headers::ReverseHeadersDownloaderBuilder,
};
use reth_ethereum_engine_primitives::EthEngineTypes;
use reth_evm_ethereum::EthEvmConfig;
use reth_interfaces::{
consensus::Consensus,
executor::BlockExecutionError,
p2p::{bodies::client::BodiesClient, either::EitherDownloader, headers::client::HeadersClient},
sync::NoopSyncStateUpdater,
@ -40,6 +39,8 @@ use reth_tasks::TokioTaskExecutor;
use std::{collections::VecDeque, sync::Arc};
use tokio::sync::{oneshot, watch};
type DatabaseEnv = TempDatabase<DE>;
type TestBeaconConsensusEngine<Client> = BeaconConsensusEngine<
Arc<DatabaseEnv>,
BlockchainProvider<

View File

@ -15,6 +15,7 @@ workspace = true
reth-primitives.workspace = true
reth-interfaces.workspace = true
reth-provider.workspace = true
reth-consensus.workspace=true
[dev-dependencies]
reth-interfaces = { workspace = true, features = ["test-utils"] }

View File

@ -1,6 +1,7 @@
//! Collection of methods for block validation.
use reth_interfaces::{consensus::ConsensusError, RethResult};
use reth_consensus::ConsensusError;
use reth_interfaces::RethResult;
use reth_primitives::{
constants::eip4844::{DATA_GAS_PER_BLOB, MAX_DATA_GAS_PER_BLOCK},
BlockNumber, ChainSpec, GotExpected, Hardfork, Header, InvalidTransactionError, SealedBlock,

View File

@ -0,0 +1,18 @@
[package]
name = "reth-consensus"
version.workspace = true
edition.workspace = true
rust-version.workspace = true
license.workspace = true
homepage.workspace = true
repository.workspace = true
[lints]
workspace = true
[dependencies]
reth-primitives.workspace = true
# misc
auto_impl.workspace = true
thiserror.workspace = true

View File

@ -1,12 +1,19 @@
//! Consensus protocol functions
#![doc(
html_logo_url = "https://raw.githubusercontent.com/paradigmxyz/reth/main/assets/reth-docs.png",
html_favicon_url = "https://avatars0.githubusercontent.com/u/97369466?s=256",
issue_tracker_base_url = "https://github.com/paradigmxyz/reth/issues/"
)]
#![cfg_attr(not(test), warn(unused_crate_dependencies))]
#![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))]
use reth_primitives::{
BlockHash, BlockNumber, GotExpected, GotExpectedBoxed, Header, HeaderValidationError,
InvalidTransactionError, SealedBlock, SealedHeader, B256, U256,
};
use std::fmt::Debug;
/// Re-export fork choice state
pub use reth_rpc_types::engine::ForkchoiceState;
/// Consensus is a protocol that chooses canonical chain.
#[auto_impl::auto_impl(&, Arc)]
pub trait Consensus: Debug + Send + Sync {

View File

@ -12,9 +12,9 @@ workspace = true
[dependencies]
reth-primitives.workspace = true
reth-rpc-types.workspace = true
reth-network-api.workspace = true
reth-eth-wire-types.workspace = true
reth-consensus.workspace = true
# async
futures.workspace = true

View File

@ -1,11 +1,11 @@
//! Error handling for the blockchain tree
use crate::{
consensus::ConsensusError,
executor::{BlockExecutionError, BlockValidationError},
provider::ProviderError,
RethError,
};
use reth_consensus::ConsensusError;
use reth_primitives::{BlockHash, BlockNumber, SealedBlock};
/// Various error cases that can occur when a block violates tree assumptions.

View File

@ -1,10 +1,10 @@
use crate::{
blockchain_tree::error::{BlockchainTreeError, CanonicalError},
consensus::ConsensusError,
db::DatabaseError,
executor::BlockExecutionError,
provider::ProviderError,
};
use reth_consensus::ConsensusError;
use reth_network_api::NetworkError;
use reth_primitives::fs::FsPathError;

View File

@ -12,9 +12,6 @@
#![cfg_attr(not(test), warn(unused_crate_dependencies))]
#![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))]
/// Consensus traits.
pub mod consensus;
/// Database error
pub mod db;

View File

@ -1,5 +1,6 @@
use super::headers::client::HeadersRequest;
use crate::{consensus::ConsensusError, db::DatabaseError, provider::ProviderError};
use crate::{db::DatabaseError, provider::ProviderError};
use reth_consensus::ConsensusError;
use reth_network_api::ReputationChangeKind;
use reth_primitives::{
BlockHashOrNumber, BlockNumber, GotExpected, GotExpectedBoxed, Header, WithPeerId, B256,

View File

@ -1,13 +1,11 @@
use super::headers::client::HeadersRequest;
use crate::{
consensus::{Consensus, ConsensusError},
p2p::{
bodies::client::{BodiesClient, SingleBodyRequest},
error::PeerRequestResult,
headers::client::{HeadersClient, SingleHeaderRequest},
},
use crate::p2p::{
bodies::client::{BodiesClient, SingleBodyRequest},
error::PeerRequestResult,
headers::client::{HeadersClient, SingleHeaderRequest},
};
use futures::Stream;
use reth_consensus::{Consensus, ConsensusError};
use reth_primitives::{
BlockBody, GotExpected, Header, HeadersDirection, SealedBlock, SealedHeader, WithPeerId, B256,
};

View File

@ -1,11 +1,8 @@
use super::error::HeadersDownloaderResult;
use crate::{
consensus::Consensus,
p2p::error::{DownloadError, DownloadResult},
};
use crate::p2p::error::{DownloadError, DownloadResult};
use futures::Stream;
use reth_consensus::Consensus;
use reth_primitives::{BlockHashOrNumber, SealedHeader, B256};
/// A downloader capable of fetching and yielding block headers.
///
/// A downloader represents a distinct strategy for submitting requests to download block headers,

View File

@ -1,4 +1,4 @@
use crate::consensus::ConsensusError;
use reth_consensus::ConsensusError;
use reth_primitives::SealedHeader;
use thiserror::Error;

View File

@ -6,7 +6,7 @@ pub mod client;
/// A downloader that receives and verifies block headers, is generic
/// over the Consensus and the HeadersClient being used.
///
/// [`Consensus`]: crate::consensus::Consensus
/// [`Consensus`]: reth_consensus::Consensus
/// [`HeadersClient`]: client::HeadersClient
pub mod downloader;

View File

@ -14,7 +14,7 @@ pub mod full_block;
/// of a Linear and a Parallel downloader generic over the [`Consensus`] and
/// [`HeadersClient`].
///
/// [`Consensus`]: crate::consensus::Consensus
/// [`Consensus`]: reth_consensus::Consensus
/// [`HeadersClient`]: crate::p2p::headers::client::HeadersClient
pub mod headers;

View File

@ -1,19 +1,17 @@
//! Testing support for headers related interfaces.
use crate::{
consensus::{self, Consensus, ConsensusError},
p2p::{
download::DownloadClient,
error::{DownloadError, DownloadResult, PeerRequestResult, RequestError},
headers::{
client::{HeadersClient, HeadersRequest},
downloader::{HeaderDownloader, SyncTarget},
error::HeadersDownloaderResult,
},
priority::Priority,
use crate::p2p::{
download::DownloadClient,
error::{DownloadError, DownloadResult, PeerRequestResult, RequestError},
headers::{
client::{HeadersClient, HeadersRequest},
downloader::{HeaderDownloader, SyncTarget},
error::HeadersDownloaderResult,
},
priority::Priority,
};
use futures::{Future, FutureExt, Stream, StreamExt};
use reth_consensus::{Consensus, ConsensusError};
use reth_primitives::{
Header, HeadersDirection, PeerId, SealedBlock, SealedHeader, WithPeerId, U256,
};
@ -274,7 +272,7 @@ impl TestConsensus {
impl Consensus for TestConsensus {
fn validate_header(&self, _header: &SealedHeader) -> Result<(), ConsensusError> {
if self.fail_validation() {
Err(consensus::ConsensusError::BaseFeeMissing)
Err(ConsensusError::BaseFeeMissing)
} else {
Ok(())
}
@ -286,7 +284,7 @@ impl Consensus for TestConsensus {
_parent: &SealedHeader,
) -> Result<(), ConsensusError> {
if self.fail_validation() {
Err(consensus::ConsensusError::BaseFeeMissing)
Err(ConsensusError::BaseFeeMissing)
} else {
Ok(())
}
@ -298,15 +296,15 @@ impl Consensus for TestConsensus {
_total_difficulty: U256,
) -> Result<(), ConsensusError> {
if self.fail_validation() {
Err(consensus::ConsensusError::BaseFeeMissing)
Err(ConsensusError::BaseFeeMissing)
} else {
Ok(())
}
}
fn validate_block(&self, _block: &SealedBlock) -> Result<(), consensus::ConsensusError> {
fn validate_block(&self, _block: &SealedBlock) -> Result<(), ConsensusError> {
if self.fail_validation() {
Err(consensus::ConsensusError::BaseFeeMissing)
Err(ConsensusError::BaseFeeMissing)
} else {
Ok(())
}

View File

@ -18,6 +18,7 @@ reth-primitives.workspace = true
reth-tasks.workspace = true
reth-provider.workspace = true
reth-config.workspace = true
reth-consensus.workspace = true
# async
futures.workspace = true

View File

@ -3,16 +3,14 @@ use crate::{bodies::task::TaskDownloader, metrics::BodyDownloaderMetrics};
use futures::Stream;
use futures_util::StreamExt;
use reth_config::BodiesConfig;
use reth_interfaces::{
consensus::Consensus,
p2p::{
bodies::{
client::BodiesClient,
downloader::{BodyDownloader, BodyDownloaderResult},
response::BlockResponse,
},
error::{DownloadError, DownloadResult},
use reth_consensus::Consensus;
use reth_interfaces::p2p::{
bodies::{
client::BodiesClient,
downloader::{BodyDownloader, BodyDownloaderResult},
response::BlockResponse,
},
error::{DownloadError, DownloadResult},
};
use reth_primitives::{BlockNumber, SealedHeader};
use reth_provider::HeaderProvider;

View File

@ -2,12 +2,10 @@ use super::request::BodiesRequestFuture;
use crate::metrics::BodyDownloaderMetrics;
use futures::{stream::FuturesUnordered, Stream};
use futures_util::StreamExt;
use reth_interfaces::{
consensus::Consensus,
p2p::{
bodies::{client::BodiesClient, response::BlockResponse},
error::DownloadResult,
},
use reth_consensus::Consensus;
use reth_interfaces::p2p::{
bodies::{client::BodiesClient, response::BlockResponse},
error::DownloadResult,
};
use reth_primitives::{BlockNumber, SealedHeader};
use std::{

View File

@ -1,12 +1,10 @@
use crate::metrics::{BodyDownloaderMetrics, ResponseMetrics};
use futures::{Future, FutureExt};
use reth_interfaces::{
consensus::{Consensus as ConsensusTrait, Consensus},
p2p::{
bodies::{client::BodiesClient, response::BlockResponse},
error::{DownloadError, DownloadResult},
priority::Priority,
},
use reth_consensus::Consensus;
use reth_interfaces::p2p::{
bodies::{client::BodiesClient, response::BlockResponse},
error::{DownloadError, DownloadResult},
priority::Priority,
};
use reth_primitives::{
BlockBody, GotExpected, PeerId, SealedBlock, SealedHeader, WithPeerId, B256,

View File

@ -42,8 +42,9 @@ impl TaskDownloader {
/// # Example
///
/// ```
/// use reth_consensus::Consensus;
/// use reth_downloaders::bodies::{bodies::BodiesDownloaderBuilder, task::TaskDownloader};
/// use reth_interfaces::{consensus::Consensus, p2p::bodies::client::BodiesClient};
/// use reth_interfaces::p2p::bodies::client::BodiesClient;
/// use reth_provider::HeaderProvider;
/// use std::sync::Arc;
///

View File

@ -6,17 +6,15 @@ use futures::{stream::Stream, FutureExt};
use futures_util::{stream::FuturesUnordered, StreamExt};
use rayon::prelude::*;
use reth_config::config::HeadersConfig;
use reth_interfaces::{
consensus::Consensus,
p2p::{
error::{DownloadError, DownloadResult, PeerRequestResult},
headers::{
client::{HeadersClient, HeadersRequest},
downloader::{validate_header_download, HeaderDownloader, SyncTarget},
error::{HeadersDownloaderError, HeadersDownloaderResult},
},
priority::Priority,
use reth_consensus::Consensus;
use reth_interfaces::p2p::{
error::{DownloadError, DownloadResult, PeerRequestResult},
headers::{
client::{HeadersClient, HeadersRequest},
downloader::{validate_header_download, HeaderDownloader, SyncTarget},
error::{HeadersDownloaderError, HeadersDownloaderResult},
},
priority::Priority,
};
use reth_primitives::{
BlockHashOrNumber, BlockNumber, GotExpected, Header, HeadersDirection, PeerId, SealedHeader,

View File

@ -44,7 +44,7 @@ impl TaskDownloader {
/// # use std::sync::Arc;
/// # use reth_downloaders::headers::reverse_headers::ReverseHeadersDownloader;
/// # use reth_downloaders::headers::task::TaskDownloader;
/// # use reth_interfaces::consensus::Consensus;
/// # use reth_consensus::Consensus;
/// # use reth_interfaces::p2p::headers::client::HeadersClient;
/// # fn t<H: HeadersClient + 'static>(consensus:Arc<dyn Consensus>, client: Arc<H>) {
/// let downloader = ReverseHeadersDownloader::<H>::builder().build(

View File

@ -27,6 +27,7 @@ reth-transaction-pool.workspace = true
reth-provider.workspace = true
reth-rpc-types.workspace = true
reth-tokio-util.workspace = true
reth-consensus.workspace = true
# ethereum
enr = { workspace = true, features = ["serde", "rust-secp256k1"] }

View File

@ -50,7 +50,7 @@ pub enum BlockValidation {
pub enum BlockImportError {
/// Consensus error
#[error(transparent)]
Consensus(#[from] reth_interfaces::consensus::ConsensusError),
Consensus(#[from] reth_consensus::ConsensusError),
}
/// An implementation of `BlockImport` used in Proof-of-Stake consensus that does nothing.

View File

@ -37,6 +37,7 @@ reth-stages.workspace = true
reth-config.workspace = true
reth-downloaders.workspace = true
reth-node-events.workspace = true
reth-consensus.workspace = true
## async
futures.workspace = true

View File

@ -22,6 +22,7 @@ use reth_blockchain_tree::{
BlockchainTree, BlockchainTreeConfig, ShareableBlockchainTree, TreeExternals,
};
use reth_config::config::EtlConfig;
use reth_consensus::Consensus;
use reth_db::{
database::Database,
database_metrics::{DatabaseMetadata, DatabaseMetrics},
@ -29,7 +30,7 @@ use reth_db::{
DatabaseEnv,
};
use reth_exex::{ExExContext, ExExHandle, ExExManager, ExExManagerHandle};
use reth_interfaces::{consensus::Consensus, p2p::either::EitherDownloader};
use reth_interfaces::p2p::either::EitherDownloader;
use reth_network::{NetworkBuilder, NetworkConfig, NetworkEvents, NetworkHandle};
use reth_node_api::{
FullNodeComponents, FullNodeComponentsAdapter, FullNodeTypes, FullNodeTypesAdapter, NodeTypes,

View File

@ -2,18 +2,16 @@
use crate::ConfigureEvm;
use reth_config::{config::StageConfig, PruneConfig};
use reth_consensus::Consensus;
use reth_db::database::Database;
use reth_downloaders::{
bodies::bodies::BodiesDownloaderBuilder,
headers::reverse_headers::ReverseHeadersDownloaderBuilder,
};
use reth_exex::ExExManagerHandle;
use reth_interfaces::{
consensus::Consensus,
p2p::{
bodies::{client::BodiesClient, downloader::BodyDownloader},
headers::{client::HeadersClient, downloader::HeaderDownloader},
},
use reth_interfaces::p2p::{
bodies::{client::BodiesClient, downloader::BodyDownloader},
headers::{client::HeadersClient, downloader::HeaderDownloader},
};
use reth_node_core::{
node_config::NodeConfig,

View File

@ -18,9 +18,9 @@ reth-network-api.workspace = true
reth-stages.workspace = true
reth-prune.workspace = true
reth-static-file.workspace = true
reth-interfaces.workspace = true
reth-db.workspace = true
reth-primitives.workspace = true
reth-rpc-types.workspace = true
# async
tokio.workspace = true

View File

@ -6,7 +6,6 @@ use reth_beacon_consensus::{
BeaconConsensusEngineEvent, ConsensusEngineLiveSyncProgress, ForkchoiceStatus,
};
use reth_db::{database::Database, database_metrics::DatabaseMetadata};
use reth_interfaces::consensus::ForkchoiceState;
use reth_network::{NetworkEvent, NetworkHandle};
use reth_network_api::PeersInfo;
use reth_primitives::{
@ -15,6 +14,7 @@ use reth_primitives::{
BlockNumber, B256,
};
use reth_prune::PrunerEvent;
use reth_rpc_types::engine::ForkchoiceState;
use reth_stages::{ExecOutput, PipelineEvent};
use reth_static_file::StaticFileProducerEvent;
use std::{

View File

@ -14,7 +14,6 @@ workspace = true
[dependencies]
# reth
reth-primitives.workspace = true
reth-interfaces.workspace = true
reth-provider.workspace = true
reth-rpc-types.workspace = true
reth-rpc-api.workspace = true

View File

@ -6,15 +6,14 @@ use reth_engine_primitives::{
validate_payload_timestamp, EngineApiMessageVersion, EngineTypes, PayloadAttributes,
PayloadBuilderAttributes, PayloadOrAttributes,
};
use reth_interfaces::consensus::ForkchoiceState;
use reth_payload_builder::PayloadStore;
use reth_primitives::{BlockHash, BlockHashOrNumber, BlockNumber, ChainSpec, Hardfork, B256, U64};
use reth_provider::{BlockReader, EvmEnvProvider, HeaderProvider, StateProviderFactory};
use reth_rpc_api::EngineApiServer;
use reth_rpc_types::engine::{
CancunPayloadFields, ExecutionPayload, ExecutionPayloadBodiesV1, ExecutionPayloadInputV2,
ExecutionPayloadV1, ExecutionPayloadV3, ForkchoiceUpdated, PayloadId, PayloadStatus,
TransitionConfiguration, CAPABILITIES,
ExecutionPayloadV1, ExecutionPayloadV3, ForkchoiceState, ForkchoiceUpdated, PayloadId,
PayloadStatus, TransitionConfiguration, CAPABILITIES,
};
use reth_rpc_types_compat::engine::payload::{
convert_payload_input_v2_to_payload, convert_to_payload_body_v1,

View File

@ -18,6 +18,7 @@ reth-db.workspace = true
reth-interfaces.workspace = true
reth-static-file.workspace = true
reth-tokio-util.workspace = true
reth-consensus.workspace = true
# metrics
reth-metrics.workspace = true

View File

@ -1,5 +1,6 @@
use reth_consensus::ConsensusError;
use reth_interfaces::{
consensus, db::DatabaseError as DbError, executor, p2p::error::DownloadError, RethError,
db::DatabaseError as DbError, executor, p2p::error::DownloadError, RethError,
};
use reth_primitives::{BlockNumber, SealedHeader, StaticFileSegment, TxNumber};
use reth_provider::ProviderError;
@ -13,7 +14,7 @@ use tokio::sync::mpsc::error::SendError;
pub enum BlockErrorKind {
/// The block encountered a validation error.
#[error("validation error: {0}")]
Validation(#[from] consensus::ConsensusError),
Validation(#[from] ConsensusError),
/// The block encountered an execution error.
#[error("execution error: {0}")]
Execution(#[from] executor::BlockExecutionError),
@ -49,7 +50,7 @@ pub enum StageError {
header: Box<SealedHeader>,
/// The error that occurred when attempting to attach the header.
#[source]
error: Box<consensus::ConsensusError>,
error: Box<ConsensusError>,
},
/// The headers stage is missing sync gap.
#[error("missing sync gap")]

View File

@ -552,8 +552,8 @@ mod tests {
use super::*;
use crate::{test_utils::TestStage, UnwindOutput};
use assert_matches::assert_matches;
use reth_consensus::ConsensusError;
use reth_interfaces::{
consensus,
provider::ProviderError,
test_utils::{generators, generators::random_header},
};
@ -922,9 +922,7 @@ mod tests {
5,
Default::default(),
)),
error: BlockErrorKind::Validation(
consensus::ConsensusError::BaseFeeMissing,
),
error: BlockErrorKind::Validation(ConsensusError::BaseFeeMissing),
}))
.add_unwind(Ok(UnwindOutput { checkpoint: StageCheckpoint::new(0) }))
.add_exec(Ok(ExecOutput { checkpoint: StageCheckpoint::new(10), done: true })),

View File

@ -23,6 +23,7 @@ reth-trie = { workspace = true, features = ["metrics"] }
reth-etl.workspace = true
reth-config.workspace = true
reth-stages-api = { workspace = true, features = ["test-utils"] }
reth-consensus.workspace = true
# async
tokio = { workspace = true, features = ["sync"] }

View File

@ -15,7 +15,6 @@
//! # use std::sync::Arc;
//! # use reth_downloaders::bodies::bodies::BodiesDownloaderBuilder;
//! # use reth_downloaders::headers::reverse_headers::ReverseHeadersDownloaderBuilder;
//! # use reth_interfaces::consensus::Consensus;
//! # use reth_interfaces::test_utils::{TestBodiesClient, TestConsensus, TestHeadersClient};
//! # use reth_revm::EvmProcessorFactory;
//! # use reth_primitives::{PeerId, MAINNET, B256, PruneModes};
@ -28,6 +27,7 @@
//! # use reth_provider::test_utils::create_test_provider_factory;
//! # use reth_static_file::StaticFileProducer;
//! # use reth_config::config::EtlConfig;
//! # use reth_consensus::Consensus;
//! #
//! # let chain_spec = MAINNET.clone();
//! # let consensus: Arc<dyn Consensus> = Arc::new(TestConsensus::default());

View File

@ -58,10 +58,10 @@ use crate::{
StageSet, StageSetBuilder,
};
use reth_config::config::EtlConfig;
use reth_consensus::Consensus;
use reth_db::database::Database;
use reth_interfaces::{
consensus::Consensus,
p2p::{bodies::downloader::BodyDownloader, headers::downloader::HeaderDownloader},
use reth_interfaces::p2p::{
bodies::downloader::BodyDownloader, headers::downloader::HeaderDownloader,
};
use reth_provider::{ExecutorFactory, HeaderSyncGapProvider, HeaderSyncMode};
use std::sync::Arc;

View File

@ -1,6 +1,7 @@
use futures_util::StreamExt;
use reth_codecs::Compact;
use reth_config::config::EtlConfig;
use reth_consensus::Consensus;
use reth_db::{
cursor::{DbCursorRO, DbCursorRW},
database::Database,
@ -10,7 +11,6 @@ use reth_db::{
};
use reth_etl::Collector;
use reth_interfaces::{
consensus::Consensus,
p2p::headers::{downloader::HeaderDownloader, error::HeadersDownloaderError},
provider::ProviderError,
};

View File

@ -1,10 +1,10 @@
use reth_codecs::Compact;
use reth_consensus::ConsensusError;
use reth_db::{
database::Database,
tables,
transaction::{DbTx, DbTxMut},
};
use reth_interfaces::consensus;
use reth_primitives::{
stage::{EntitiesCheckpoint, MerkleCheckpoint, StageCheckpoint, StageId},
trie::StoredSubNode,
@ -327,7 +327,7 @@ fn validate_state_root(
} else {
warn!(target: "sync::stages::merkle", ?target_block, ?got, ?expected, "Failed to verify block state root");
Err(StageError::Block {
error: BlockErrorKind::Validation(consensus::ConsensusError::BodyStateRootDiff(
error: BlockErrorKind::Validation(ConsensusError::BodyStateRootDiff(
GotExpected { got, expected: expected.state_root }.into(),
)),
block: Box::new(expected),

View File

@ -1,3 +1,4 @@
use reth_consensus::ConsensusError;
use reth_db::{
cursor::DbCursorRW,
database::Database,
@ -6,7 +7,6 @@ use reth_db::{
transaction::{DbTx, DbTxMut},
RawValue,
};
use reth_interfaces::consensus;
use reth_primitives::{
stage::{EntitiesCheckpoint, StageCheckpoint, StageId},
Address, PruneSegment, StaticFileSegment, TransactionSignedNoHash, TxNumber,
@ -209,7 +209,7 @@ fn recover_range<DB: Database>(
Err(StageError::Block {
block: Box::new(sealed_header),
error: BlockErrorKind::Validation(
consensus::ConsensusError::TransactionSignerRecoveryError,
ConsensusError::TransactionSignerRecoveryError,
),
})
}

View File

@ -15,6 +15,7 @@ workspace = true
# reth
reth-primitives.workspace = true
reth-interfaces.workspace = true
reth-rpc-types.workspace = true
reth-db.workspace = true
reth-trie = { workspace = true, features = ["metrics"] }
reth-nippy-jar.workspace = true

View File

@ -17,7 +17,6 @@ use reth_interfaces::{
BlockValidationKind, BlockchainTreeEngine, BlockchainTreeViewer, CanonicalOutcome,
InsertPayloadOk,
},
consensus::ForkchoiceState,
provider::ProviderResult,
RethResult,
};
@ -61,6 +60,7 @@ use chain_info::ChainInfoTracker;
mod consistent_view;
pub use consistent_view::{ConsistentDbView, ConsistentViewError};
use reth_rpc_types::engine::ForkchoiceState;
/// The main type for interacting with the blockchain.
///

View File

@ -1,5 +1,5 @@
use reth_interfaces::consensus::ForkchoiceState;
use reth_primitives::SealedHeader;
use reth_rpc_types::engine::ForkchoiceState;
use std::time::Instant;
/// A type that can track updates related to fork choice updates.