refactor: Consensus trait error type (#13655)

This commit is contained in:
Tin Chung
2025-01-07 23:11:42 +07:00
committed by GitHub
parent b2c00418df
commit f7f2012156
33 changed files with 122 additions and 98 deletions

View File

@ -21,7 +21,7 @@ use reth_cli::chainspec::ChainSpecParser;
use reth_cli_commands::common::{AccessRights, CliNodeTypes, Environment, EnvironmentArgs};
use reth_cli_runner::CliContext;
use reth_consensus::{Consensus, FullConsensus};
use reth_errors::RethResult;
use reth_errors::{ConsensusError, RethResult};
use reth_ethereum_payload_builder::EthereumBuilderConfig;
use reth_evm::execute::{BlockExecutorProvider, Executor};
use reth_execution_types::ExecutionOutcome;
@ -128,7 +128,7 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
) -> eyre::Result<()> {
let Environment { provider_factory, .. } = self.env.init::<N>(AccessRights::RW)?;
let consensus: Arc<dyn FullConsensus> =
let consensus: Arc<dyn FullConsensus<Error = ConsensusError>> =
Arc::new(EthBeaconConsensus::new(provider_factory.chain_spec()));
let executor = EthExecutorProvider::ethereum(provider_factory.chain_spec());

View File

@ -18,6 +18,7 @@ use reth_downloaders::{
bodies::bodies::BodiesDownloaderBuilder,
headers::reverse_headers::ReverseHeadersDownloaderBuilder,
};
use reth_errors::ConsensusError;
use reth_exex::ExExManagerHandle;
use reth_network::{BlockDownloaderProvider, NetworkHandle};
use reth_network_api::NetworkInfo;
@ -64,7 +65,7 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
&self,
config: &Config,
client: Client,
consensus: Arc<dyn Consensus>,
consensus: Arc<dyn Consensus<Error = ConsensusError>>,
provider_factory: ProviderFactory<N>,
task_executor: &TaskExecutor,
static_file_producer: StaticFileProducer<ProviderFactory<N>>,
@ -172,7 +173,7 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
let Environment { provider_factory, config, data_dir } =
self.env.init::<N>(AccessRights::RW)?;
let consensus: Arc<dyn Consensus> =
let consensus: Arc<dyn Consensus<Error = ConsensusError>> =
Arc::new(EthBeaconConsensus::new(provider_factory.chain_spec()));
// Configure and build network

View File

@ -10,7 +10,7 @@ use reth_cli_commands::common::{AccessRights, CliNodeTypes, Environment, Environ
use reth_cli_runner::CliContext;
use reth_cli_util::get_secret_key;
use reth_config::Config;
use reth_consensus::Consensus;
use reth_consensus::{Consensus, ConsensusError};
use reth_db::tables;
use reth_db_api::{cursor::DbCursorRO, transaction::DbTx};
use reth_evm::execute::{BatchExecutor, BlockExecutorProvider};
@ -129,7 +129,7 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
info!(target: "reth::cli", target_block_number=self.to, "Finished downloading tip of block range");
// build the full block client
let consensus: Arc<dyn Consensus> =
let consensus: Arc<dyn Consensus<Error = ConsensusError>> =
Arc::new(EthBeaconConsensus::new(provider_factory.chain_spec()));
let block_range_client = FullBlockClient::new(fetch_client, consensus);

View File

@ -12,7 +12,7 @@ use reth_cli_commands::common::{AccessRights, CliNodeTypes, Environment, Environ
use reth_cli_runner::CliContext;
use reth_cli_util::get_secret_key;
use reth_config::Config;
use reth_consensus::FullConsensus;
use reth_consensus::{ConsensusError, FullConsensus};
use reth_db::DatabaseEnv;
use reth_engine_util::engine_store::{EngineMessageStore, StoredEngineApiMessage};
use reth_ethereum_payload_builder::EthereumBuilderConfig;
@ -97,7 +97,7 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
let Environment { provider_factory, config, data_dir } =
self.env.init::<N>(AccessRights::RW)?;
let consensus: Arc<dyn FullConsensus> =
let consensus: Arc<dyn FullConsensus<Error = ConsensusError>> =
Arc::new(EthBeaconConsensus::new(provider_factory.chain_spec()));
let executor = EthExecutorProvider::ethereum(provider_factory.chain_spec());

View File

@ -1,7 +1,7 @@
//! Blockchain tree externals.
use alloy_primitives::{BlockHash, BlockNumber};
use reth_consensus::FullConsensus;
use reth_consensus::{ConsensusError, FullConsensus};
use reth_db::{static_file::BlockHashMask, tables};
use reth_db_api::{cursor::DbCursorRO, transaction::DbTx};
use reth_node_types::NodeTypesWithDB;
@ -28,7 +28,7 @@ pub struct TreeExternals<N: NodeTypesWithDB, E> {
/// The provider factory, used to commit the canonical chain, or unwind it.
pub(crate) provider_factory: ProviderFactory<N>,
/// The consensus engine.
pub(crate) consensus: Arc<dyn FullConsensus>,
pub(crate) consensus: Arc<dyn FullConsensus<Error = ConsensusError>>,
/// The executor factory to execute blocks with.
pub(crate) executor_factory: E,
}
@ -37,7 +37,7 @@ impl<N: ProviderNodeTypes, E> TreeExternals<N, E> {
/// Create new tree externals.
pub fn new(
provider_factory: ProviderFactory<N>,
consensus: Arc<dyn FullConsensus>,
consensus: Arc<dyn FullConsensus<Error = ConsensusError>>,
executor_factory: E,
) -> Self {
Self { provider_factory, consensus, executor_factory }

View File

@ -7,7 +7,7 @@ use reth_beacon_consensus::EthBeaconConsensus;
use reth_chainspec::{EthChainSpec, EthereumHardforks};
use reth_cli::chainspec::ChainSpecParser;
use reth_config::Config;
use reth_consensus::Consensus;
use reth_consensus::{Consensus, ConsensusError};
use reth_db::tables;
use reth_db_api::transaction::DbTx;
use reth_downloaders::{
@ -169,7 +169,7 @@ pub fn build_import_pipeline<N, C, E>(
) -> eyre::Result<(Pipeline<N>, impl Stream<Item = NodeEvent<N::Primitives>>)>
where
N: ProviderNodeTypes + CliNodeTypes,
C: Consensus<HeaderTy<N>, BodyTy<N>> + 'static,
C: Consensus<HeaderTy<N>, BodyTy<N>, Error = ConsensusError> + 'static,
E: BlockExecutorProvider<Primitives = N::Primitives>,
{
if !file_client.has_canonical_blocks() {

View File

@ -13,7 +13,7 @@ use reth_blockchain_tree::{
};
use reth_chainspec::ChainSpec;
use reth_config::config::StageConfig;
use reth_consensus::{test_utils::TestConsensus, FullConsensus};
use reth_consensus::{test_utils::TestConsensus, ConsensusError, FullConsensus};
use reth_db::{test_utils::TempDatabase, DatabaseEnv as DE};
use reth_downloaders::{
bodies::bodies::BodiesDownloaderBuilder,
@ -332,12 +332,13 @@ where
let provider_factory =
create_test_provider_factory_with_chain_spec(self.base_config.chain_spec.clone());
let consensus: Arc<dyn FullConsensus> = match self.base_config.consensus {
TestConsensusConfig::Real => {
Arc::new(EthBeaconConsensus::new(Arc::clone(&self.base_config.chain_spec)))
}
TestConsensusConfig::Test => Arc::new(TestConsensus::default()),
};
let consensus: Arc<dyn FullConsensus<Error = ConsensusError>> =
match self.base_config.consensus {
TestConsensusConfig::Real => {
Arc::new(EthBeaconConsensus::new(Arc::clone(&self.base_config.chain_spec)))
}
TestConsensusConfig::Test => Arc::new(TestConsensus::default()),
};
let payload_builder = spawn_test_payload_service::<EthEngineTypes>();
// use either noop client or a user provided client (for example TestFullBlockClient)

View File

@ -66,12 +66,15 @@ pub trait FullConsensus<N: NodePrimitives = EthPrimitives>:
/// Consensus is a protocol that chooses canonical chain.
#[auto_impl::auto_impl(&, Arc)]
pub trait Consensus<H = Header, B = BlockBody>: AsHeaderValidator<H> {
/// The error type related to consensus.
type Error;
/// Ensures that body field values match the header.
fn validate_body_against_header(
&self,
body: &B,
header: &SealedHeader<H>,
) -> Result<(), ConsensusError>;
) -> Result<(), Self::Error>;
/// Validate a block disregarding world state, i.e. things that can be checked before sender
/// recovery and execution.
@ -82,8 +85,7 @@ pub trait Consensus<H = Header, B = BlockBody>: AsHeaderValidator<H> {
/// **This should not be called for the genesis block**.
///
/// Note: validating blocks does not include other validations of the Consensus
fn validate_block_pre_execution(&self, block: &SealedBlock<H, B>)
-> Result<(), ConsensusError>;
fn validate_block_pre_execution(&self, block: &SealedBlock<H, B>) -> Result<(), Self::Error>;
}
/// HeaderValidator is a protocol that validates headers and their relationships.
@ -170,13 +172,13 @@ impl<T: HeaderValidator<H>, H> AsHeaderValidator<H> for T {
/// Helper trait to cast `Arc<dyn FullConsensus>` to `Arc<dyn Consensus>`
pub trait AsConsensus<H, B>: Consensus<H, B> {
/// Converts the [`Arc`] of self to [`Arc`] of [`HeaderValidator`]
fn as_consensus<'a>(self: Arc<Self>) -> Arc<dyn Consensus<H, B> + 'a>
fn as_consensus<'a>(self: Arc<Self>) -> Arc<dyn Consensus<H, B, Error = Self::Error> + 'a>
where
Self: 'a;
}
impl<T: Consensus<H, B>, H, B> AsConsensus<H, B> for T {
fn as_consensus<'a>(self: Arc<Self>) -> Arc<dyn Consensus<H, B> + 'a>
fn as_consensus<'a>(self: Arc<Self>) -> Arc<dyn Consensus<H, B, Error = Self::Error> + 'a>
where
Self: 'a,
{

View File

@ -30,18 +30,17 @@ impl<H> HeaderValidator<H> for NoopConsensus {
}
impl<H, B> Consensus<H, B> for NoopConsensus {
type Error = ConsensusError;
fn validate_body_against_header(
&self,
_body: &B,
_header: &SealedHeader<H>,
) -> Result<(), ConsensusError> {
) -> Result<(), Self::Error> {
Ok(())
}
fn validate_block_pre_execution(
&self,
_block: &SealedBlock<H, B>,
) -> Result<(), ConsensusError> {
fn validate_block_pre_execution(&self, _block: &SealedBlock<H, B>) -> Result<(), Self::Error> {
Ok(())
}
}
@ -51,7 +50,7 @@ impl<N: NodePrimitives> FullConsensus<N> for NoopConsensus {
&self,
_block: &BlockWithSenders<N::Block>,
_input: PostExecutionInput<'_, N::Receipt>,
) -> Result<(), ConsensusError> {
) -> Result<(), Self::Error> {
Ok(())
}
}

View File

@ -61,11 +61,13 @@ impl<N: NodePrimitives> FullConsensus<N> for TestConsensus {
}
impl<H, B> Consensus<H, B> for TestConsensus {
type Error = ConsensusError;
fn validate_body_against_header(
&self,
_body: &B,
_header: &SealedHeader<H>,
) -> Result<(), ConsensusError> {
) -> Result<(), Self::Error> {
if self.fail_body_against_header() {
Err(ConsensusError::BaseFeeMissing)
} else {
@ -73,10 +75,7 @@ impl<H, B> Consensus<H, B> for TestConsensus {
}
}
fn validate_block_pre_execution(
&self,
_block: &SealedBlock<H, B>,
) -> Result<(), ConsensusError> {
fn validate_block_pre_execution(&self, _block: &SealedBlock<H, B>) -> Result<(), Self::Error> {
if self.fail_validation() {
Err(ConsensusError::BaseFeeMissing)
} else {

View File

@ -18,7 +18,7 @@ use crate::miner::{LocalMiner, MiningMode};
use futures_util::{Stream, StreamExt};
use reth_beacon_consensus::{BeaconConsensusEngineEvent, EngineNodeTypes};
use reth_chainspec::EthChainSpec;
use reth_consensus::FullConsensus;
use reth_consensus::{ConsensusError, FullConsensus};
use reth_engine_primitives::{BeaconEngineMessage, EngineValidator};
use reth_engine_service::service::EngineMessageStream;
use reth_engine_tree::{
@ -64,7 +64,7 @@ where
/// Constructor for [`LocalEngineService`].
#[allow(clippy::too_many_arguments)]
pub fn new<B, V>(
consensus: Arc<dyn FullConsensus<N::Primitives>>,
consensus: Arc<dyn FullConsensus<N::Primitives, Error = ConsensusError>>,
executor_factory: impl BlockExecutorProvider<Primitives = N::Primitives>,
provider: ProviderFactory<N>,
blockchain_db: BlockchainProvider2<N>,

View File

@ -2,7 +2,7 @@ use futures::{Stream, StreamExt};
use pin_project::pin_project;
use reth_beacon_consensus::{BeaconConsensusEngineEvent, EngineNodeTypes};
use reth_chainspec::EthChainSpec;
use reth_consensus::FullConsensus;
use reth_consensus::{ConsensusError, FullConsensus};
use reth_engine_primitives::{BeaconEngineMessage, EngineValidator};
use reth_engine_tree::{
backfill::PipelineSync,
@ -69,7 +69,7 @@ where
/// Constructor for `EngineService`.
#[allow(clippy::too_many_arguments)]
pub fn new<V>(
consensus: Arc<dyn FullConsensus<N::Primitives>>,
consensus: Arc<dyn FullConsensus<N::Primitives, Error = ConsensusError>>,
executor_factory: E,
chain_spec: Arc<N::ChainSpec>,
client: Client,

View File

@ -4,7 +4,7 @@ use crate::{engine::DownloadRequest, metrics::BlockDownloaderMetrics};
use alloy_consensus::BlockHeader;
use alloy_primitives::B256;
use futures::FutureExt;
use reth_consensus::Consensus;
use reth_consensus::{Consensus, ConsensusError};
use reth_network_p2p::{
full_block::{FetchFullBlockFuture, FetchFullBlockRangeFuture, FullBlockClient},
BlockClient,
@ -84,7 +84,7 @@ where
/// Create a new instance
pub fn new(
client: Client,
consensus: Arc<dyn Consensus<Client::Header, Client::Body>>,
consensus: Arc<dyn Consensus<Client::Header, Client::Body, Error = ConsensusError>>,
) -> Self {
Self {
full_block_client: FullBlockClient::new(client, consensus),

View File

@ -496,7 +496,7 @@ where
{
provider: P,
executor_provider: E,
consensus: Arc<dyn FullConsensus<N>>,
consensus: Arc<dyn FullConsensus<N, Error = ConsensusError>>,
payload_validator: V,
/// Keeps track of internals such as executed and buffered blocks.
state: EngineApiTreeState<N>,
@ -585,7 +585,7 @@ where
pub fn new(
provider: P,
executor_provider: E,
consensus: Arc<dyn FullConsensus<N>>,
consensus: Arc<dyn FullConsensus<N, Error = ConsensusError>>,
payload_validator: V,
outgoing: UnboundedSender<EngineApiEvent<N>>,
state: EngineApiTreeState<N>,
@ -643,7 +643,7 @@ where
pub fn spawn_new(
provider: P,
executor_provider: E,
consensus: Arc<dyn FullConsensus<N>>,
consensus: Arc<dyn FullConsensus<N, Error = ConsensusError>>,
payload_validator: V,
persistence: PersistenceHandle<N>,
payload_builder: PayloadBuilderHandle<T>,

View File

@ -116,18 +116,17 @@ where
H: BlockHeader,
B: BlockBody,
{
type Error = ConsensusError;
fn validate_body_against_header(
&self,
body: &B,
header: &SealedHeader<H>,
) -> Result<(), ConsensusError> {
) -> Result<(), Self::Error> {
validate_body_against_header(body, header.header())
}
fn validate_block_pre_execution(
&self,
block: &SealedBlock<H, B>,
) -> Result<(), ConsensusError> {
fn validate_block_pre_execution(&self, block: &SealedBlock<H, B>) -> Result<(), Self::Error> {
validate_block_pre_execution(block, &self.chain_spec)
}
}

View File

@ -3,6 +3,7 @@
use crate::{EthEngineTypes, EthEvmConfig};
use reth_beacon_consensus::EthBeaconConsensus;
use reth_chainspec::ChainSpec;
use reth_consensus::{ConsensusError, FullConsensus};
use reth_ethereum_engine_primitives::{
EthBuiltPayload, EthPayloadAttributes, EthPayloadBuilderAttributes,
};
@ -252,7 +253,7 @@ impl<Node> ConsensusBuilder<Node> for EthereumConsensusBuilder
where
Node: FullNodeTypes<Types: NodeTypes<ChainSpec = ChainSpec, Primitives = EthPrimitives>>,
{
type Consensus = Arc<dyn reth_consensus::FullConsensus>;
type Consensus = Arc<dyn FullConsensus<Error = ConsensusError>>;
async fn build_consensus(self, ctx: &BuilderContext<Node>) -> eyre::Result<Self::Consensus> {
Ok(Arc::new(EthBeaconConsensus::new(ctx.chain_spec())))

View File

@ -5,7 +5,7 @@ use alloy_primitives::BlockNumber;
use futures::Stream;
use futures_util::StreamExt;
use reth_config::BodiesConfig;
use reth_consensus::Consensus;
use reth_consensus::{Consensus, ConsensusError};
use reth_network_p2p::{
bodies::{
client::BodiesClient,
@ -39,7 +39,7 @@ pub struct BodiesDownloader<B: BodiesClient, Provider: HeaderProvider> {
/// The bodies client
client: Arc<B>,
/// The consensus client
consensus: Arc<dyn Consensus<Provider::Header, B::Body>>,
consensus: Arc<dyn Consensus<Provider::Header, B::Body, Error = ConsensusError>>,
/// The database handle
provider: Provider,
/// The maximum number of non-empty blocks per one request
@ -579,7 +579,7 @@ impl BodiesDownloaderBuilder {
pub fn build<B, Provider>(
self,
client: B,
consensus: Arc<dyn Consensus<Provider::Header, B::Body>>,
consensus: Arc<dyn Consensus<Provider::Header, B::Body, Error = ConsensusError>>,
provider: Provider,
) -> BodiesDownloader<B, Provider>
where

View File

@ -4,7 +4,7 @@ use alloy_consensus::BlockHeader;
use alloy_primitives::BlockNumber;
use futures::{stream::FuturesUnordered, Stream};
use futures_util::StreamExt;
use reth_consensus::Consensus;
use reth_consensus::{Consensus, ConsensusError};
use reth_network_p2p::{
bodies::{client::BodiesClient, response::BlockResponse},
error::DownloadResult,
@ -59,7 +59,7 @@ where
pub(crate) fn push_new_request(
&mut self,
client: Arc<B>,
consensus: Arc<dyn Consensus<H, B::Body>>,
consensus: Arc<dyn Consensus<H, B::Body, Error = ConsensusError>>,
request: Vec<SealedHeader<H>>,
) {
// Set last max requested block number

View File

@ -2,7 +2,7 @@ use crate::metrics::{BodyDownloaderMetrics, ResponseMetrics};
use alloy_consensus::BlockHeader;
use alloy_primitives::B256;
use futures::{Future, FutureExt};
use reth_consensus::Consensus;
use reth_consensus::{Consensus, ConsensusError};
use reth_network_p2p::{
bodies::{client::BodiesClient, response::BlockResponse},
error::{DownloadError, DownloadResult},
@ -40,7 +40,7 @@ use std::{
/// and eventually disconnected.
pub(crate) struct BodiesRequestFuture<H, B: BodiesClient> {
client: Arc<B>,
consensus: Arc<dyn Consensus<H, B::Body>>,
consensus: Arc<dyn Consensus<H, B::Body, Error = ConsensusError>>,
metrics: BodyDownloaderMetrics,
/// Metrics for individual responses. This can be used to observe how the size (in bytes) of
/// responses change while bodies are being downloaded.
@ -62,7 +62,7 @@ where
/// Returns an empty future. Use [`BodiesRequestFuture::with_headers`] to set the request.
pub(crate) fn new(
client: Arc<B>,
consensus: Arc<dyn Consensus<H, B::Body>>,
consensus: Arc<dyn Consensus<H, B::Body, Error = ConsensusError>>,
metrics: BodyDownloaderMetrics,
) -> Self {
Self {

View File

@ -43,7 +43,7 @@ impl<H: Send + Sync + Unpin + 'static, B: Send + Sync + Unpin + 'static> TaskDow
/// # Example
///
/// ```
/// use reth_consensus::Consensus;
/// use reth_consensus::{Consensus, ConsensusError};
/// use reth_downloaders::bodies::{bodies::BodiesDownloaderBuilder, task::TaskDownloader};
/// use reth_network_p2p::bodies::client::BodiesClient;
/// use reth_primitives_traits::InMemorySize;
@ -55,7 +55,7 @@ impl<H: Send + Sync + Unpin + 'static, B: Send + Sync + Unpin + 'static> TaskDow
/// Provider: HeaderProvider<Header = alloy_consensus::Header> + Unpin + 'static,
/// >(
/// client: Arc<B>,
/// consensus: Arc<dyn Consensus<Provider::Header, B::Body>>,
/// consensus: Arc<dyn Consensus<Provider::Header, B::Body, Error = ConsensusError>>,
/// provider: Provider,
/// ) {
/// let downloader = BodiesDownloaderBuilder::default().build(client, consensus, provider);

View File

@ -7,7 +7,7 @@ use crate::{
};
use alloy_consensus::BlockHeader;
use alloy_primitives::{Sealable, B256};
use reth_consensus::Consensus;
use reth_consensus::{Consensus, ConsensusError};
use reth_eth_wire_types::HeadersDirection;
use reth_network_peers::WithPeerId;
use reth_primitives::{SealedBlock, SealedHeader};
@ -30,7 +30,7 @@ where
Client: BlockClient,
{
client: Client,
consensus: Arc<dyn Consensus<Client::Header, Client::Body>>,
consensus: Arc<dyn Consensus<Client::Header, Client::Body, Error = ConsensusError>>,
}
impl<Client> FullBlockClient<Client>
@ -40,7 +40,7 @@ where
/// Creates a new instance of `FullBlockClient`.
pub fn new(
client: Client,
consensus: Arc<dyn Consensus<Client::Header, Client::Body>>,
consensus: Arc<dyn Consensus<Client::Header, Client::Body, Error = ConsensusError>>,
) -> Self {
Self { client, consensus }
}
@ -118,7 +118,7 @@ where
Client: BlockClient,
{
client: Client,
consensus: Arc<dyn Consensus<Client::Header, Client::Body>>,
consensus: Arc<dyn Consensus<Client::Header, Client::Body, Error = ConsensusError>>,
hash: B256,
request: FullBlockRequest<Client>,
header: Option<SealedHeader<Client::Header>>,
@ -330,7 +330,7 @@ where
/// The client used to fetch headers and bodies.
client: Client,
/// The consensus instance used to validate the blocks.
consensus: Arc<dyn Consensus<Client::Header, Client::Body>>,
consensus: Arc<dyn Consensus<Client::Header, Client::Body, Error = ConsensusError>>,
/// The block hash to start fetching from (inclusive).
start_hash: B256,
/// How many blocks to fetch: `len([start_hash, ..]) == count`

View File

@ -12,7 +12,7 @@ use crate::{
};
use alloy_consensus::Header;
use futures::{Future, FutureExt, Stream, StreamExt};
use reth_consensus::{test_utils::TestConsensus, Consensus};
use reth_consensus::{test_utils::TestConsensus, Consensus, ConsensusError};
use reth_eth_wire_types::HeadersDirection;
use reth_network_peers::{PeerId, WithPeerId};
use reth_primitives::SealedHeader;
@ -147,7 +147,11 @@ impl Stream for TestDownload {
let empty: SealedHeader = SealedHeader::default();
if let Err(error) =
<dyn Consensus<_>>::validate_header_against_parent(&this.consensus, &empty, &empty)
<dyn Consensus<_, Error = ConsensusError>>::validate_header_against_parent(
&this.consensus,
&empty,
&empty,
)
{
this.done = true;
return Poll::Ready(Some(Err(DownloadError::HeaderValidation {

View File

@ -3,7 +3,7 @@
use crate::ConfigureEvm;
use alloy_rpc_types_engine::JwtSecret;
use reth_beacon_consensus::BeaconConsensusEngineHandle;
use reth_consensus::FullConsensus;
use reth_consensus::{ConsensusError, FullConsensus};
use reth_db_api::{
database_metrics::{DatabaseMetadata, DatabaseMetrics},
Database,
@ -58,7 +58,10 @@ pub trait FullNodeComponents: FullNodeTypes + Clone + 'static {
type Executor: BlockExecutorProvider<Primitives = <Self::Types as NodeTypes>::Primitives>;
/// The consensus type of the node.
type Consensus: FullConsensus<<Self::Types as NodeTypes>::Primitives> + Clone + Unpin + 'static;
type Consensus: FullConsensus<<Self::Types as NodeTypes>::Primitives, Error = ConsensusError>
+ Clone
+ Unpin
+ 'static;
/// Network API.
type Network: FullNetwork;

View File

@ -7,7 +7,7 @@ use crate::{
},
BuilderContext, ConfigureEvm, FullNodeTypes,
};
use reth_consensus::FullConsensus;
use reth_consensus::{ConsensusError, FullConsensus};
use reth_evm::execute::BlockExecutorProvider;
use reth_network::NetworkPrimitives;
use reth_node_api::{BodyTy, HeaderTy, NodeTypes, NodeTypesWithEngine, TxTy};
@ -402,7 +402,10 @@ where
+ 'static,
EVM: ConfigureEvm<Header = HeaderTy<Node::Types>, Transaction = TxTy<Node::Types>>,
Executor: BlockExecutorProvider<Primitives = <Node::Types as NodeTypes>::Primitives>,
Cons: FullConsensus<<Node::Types as NodeTypes>::Primitives> + Clone + Unpin + 'static,
Cons: FullConsensus<<Node::Types as NodeTypes>::Primitives, Error = ConsensusError>
+ Clone
+ Unpin
+ 'static,
{
type Components = Components<Node, N, Pool, EVM, Executor, Cons>;

View File

@ -1,4 +1,5 @@
//! Consensus component for the node builder.
use reth_consensus::{ConsensusError, FullConsensus};
use reth_node_api::NodeTypes;
use crate::{BuilderContext, FullNodeTypes};
@ -7,7 +8,7 @@ use std::future::Future;
/// A type that knows how to build the consensus implementation.
pub trait ConsensusBuilder<Node: FullNodeTypes>: Send {
/// The consensus implementation to build.
type Consensus: reth_consensus::FullConsensus<<Node::Types as NodeTypes>::Primitives>
type Consensus: FullConsensus<<Node::Types as NodeTypes>::Primitives, Error = ConsensusError>
+ Clone
+ Unpin
+ 'static;
@ -22,7 +23,7 @@ pub trait ConsensusBuilder<Node: FullNodeTypes>: Send {
impl<Node, F, Fut, Consensus> ConsensusBuilder<Node> for F
where
Node: FullNodeTypes,
Consensus: reth_consensus::FullConsensus<<Node::Types as NodeTypes>::Primitives>
Consensus: FullConsensus<<Node::Types as NodeTypes>::Primitives, Error = ConsensusError>
+ Clone
+ Unpin
+ 'static,

View File

@ -23,7 +23,7 @@ pub use pool::*;
use reth_network_p2p::BlockClient;
use crate::{ConfigureEvm, FullNodeTypes};
use reth_consensus::FullConsensus;
use reth_consensus::{ConsensusError, FullConsensus};
use reth_evm::execute::BlockExecutorProvider;
use reth_network::{NetworkHandle, NetworkPrimitives};
use reth_network_api::FullNetwork;
@ -47,7 +47,10 @@ pub trait NodeComponents<T: FullNodeTypes>: Clone + Unpin + Send + Sync + 'stati
type Executor: BlockExecutorProvider<Primitives = <T::Types as NodeTypes>::Primitives>;
/// The consensus type of the node.
type Consensus: FullConsensus<<T::Types as NodeTypes>::Primitives> + Clone + Unpin + 'static;
type Consensus: FullConsensus<<T::Types as NodeTypes>::Primitives, Error = ConsensusError>
+ Clone
+ Unpin
+ 'static;
/// Network API.
type Network: FullNetwork<
@ -106,7 +109,10 @@ where
+ 'static,
EVM: ConfigureEvm<Header = HeaderTy<Node::Types>, Transaction = TxTy<Node::Types>>,
Executor: BlockExecutorProvider<Primitives = <Node::Types as NodeTypes>::Primitives>,
Cons: FullConsensus<<Node::Types as NodeTypes>::Primitives> + Clone + Unpin + 'static,
Cons: FullConsensus<<Node::Types as NodeTypes>::Primitives, Error = ConsensusError>
+ Clone
+ Unpin
+ 'static,
{
type Pool = Pool;
type Evm = EVM;

View File

@ -4,7 +4,7 @@ use std::sync::Arc;
use alloy_primitives::{BlockNumber, B256};
use reth_config::{config::StageConfig, PruneConfig};
use reth_consensus::Consensus;
use reth_consensus::{Consensus, ConsensusError};
use reth_downloaders::{
bodies::bodies::BodiesDownloaderBuilder,
headers::reverse_headers::ReverseHeadersDownloaderBuilder,
@ -27,7 +27,7 @@ use tokio::sync::watch;
pub fn build_networked_pipeline<N, Client, Executor>(
config: &StageConfig,
client: Client,
consensus: Arc<dyn Consensus<Client::Header, Client::Body>>,
consensus: Arc<dyn Consensus<Client::Header, Client::Body, Error = ConsensusError>>,
provider_factory: ProviderFactory<N>,
task_executor: &TaskExecutor,
metrics_tx: reth_stages::MetricEventsSender,
@ -75,7 +75,7 @@ pub fn build_pipeline<N, H, B, Executor>(
stage_config: &StageConfig,
header_downloader: H,
body_downloader: B,
consensus: Arc<dyn Consensus<H::Header, B::Body>>,
consensus: Arc<dyn Consensus<H::Header, B::Body, Error = ConsensusError>>,
max_block: Option<u64>,
metrics_tx: reth_stages::MetricEventsSender,
prune_config: Option<PruneConfig>,

View File

@ -5,7 +5,7 @@ use alloy_consensus::BlockHeader;
use alloy_eips::BlockHashOrNumber;
use alloy_rpc_types_engine::{JwtError, JwtSecret};
use eyre::Result;
use reth_consensus::Consensus;
use reth_consensus::{Consensus, ConsensusError};
use reth_network_p2p::{
bodies::client::BodiesClient, headers::client::HeadersClient, priority::Priority,
};
@ -72,7 +72,7 @@ where
pub async fn get_single_body<H, Client>(
client: Client,
header: SealedHeader<H>,
consensus: impl Consensus<H, Client::Body>,
consensus: impl Consensus<H, Client::Body, Error = ConsensusError>,
) -> Result<SealedBlock<H, Client::Body>>
where
Client: BodiesClient,

View File

@ -61,6 +61,8 @@ impl FullConsensus<OpPrimitives> for OpBeaconConsensus {
}
impl Consensus<Header, OpBlockBody> for OpBeaconConsensus {
type Error = ConsensusError;
fn validate_body_against_header(
&self,
body: &OpBlockBody,

View File

@ -16,6 +16,7 @@
//! Configure only an http server with a selection of [`RethRpcModule`]s
//!
//! ```
//! use reth_consensus::{ConsensusError, FullConsensus};
//! use reth_engine_primitives::PayloadValidator;
//! use reth_evm::{execute::BlockExecutorProvider, ConfigureEvm};
//! use reth_network_api::{NetworkInfo, Peers};
@ -67,7 +68,7 @@
//! CanonStateSubscriptions<Primitives = reth_primitives::EthPrimitives> + Clone + 'static,
//! EvmConfig: ConfigureEvm<Header = Header, Transaction = TransactionSigned>,
//! BlockExecutor: BlockExecutorProvider<Primitives = Events::Primitives>,
//! Consensus: reth_consensus::FullConsensus + Clone + 'static,
//! Consensus: FullConsensus<Error = ConsensusError> + Clone + 'static,
//! Validator: PayloadValidator<Block = reth_primitives::Block>,
//! {
//! // configure the rpc module per transport
@ -99,6 +100,7 @@
//!
//!
//! ```
//! use reth_consensus::{ConsensusError, FullConsensus};
//! use reth_engine_primitives::{EngineTypes, PayloadValidator};
//! use reth_evm::{execute::BlockExecutorProvider, ConfigureEvm};
//! use reth_network_api::{NetworkInfo, Peers};
@ -159,7 +161,7 @@
//! EngineT: EngineTypes,
//! EvmConfig: ConfigureEvm<Header = Header, Transaction = TransactionSigned>,
//! BlockExecutor: BlockExecutorProvider<Primitives = Events::Primitives>,
//! Consensus: reth_consensus::FullConsensus + Clone + 'static,
//! Consensus: FullConsensus<Error = ConsensusError> + Clone + 'static,
//! Validator: PayloadValidator<Block = reth_primitives::Block>,
//! {
//! // configure the rpc module per transport
@ -226,7 +228,7 @@ use jsonrpsee::{
Methods, RpcModule,
};
use reth_chainspec::EthereumHardforks;
use reth_consensus::FullConsensus;
use reth_consensus::{ConsensusError, FullConsensus};
use reth_engine_primitives::{EngineTypes, PayloadValidator};
use reth_evm::{execute::BlockExecutorProvider, ConfigureEvm};
use reth_network_api::{noop::NoopNetwork, NetworkInfo, Peers};
@ -298,7 +300,7 @@ pub async fn launch<Provider, Pool, Network, Tasks, Events, EvmConfig, EthApi, B
evm_config: EvmConfig,
eth: DynEthApiBuilder<Provider, Pool, EvmConfig, Network, Tasks, Events, EthApi>,
block_executor: BlockExecutor,
consensus: Arc<dyn FullConsensus<BlockExecutor::Primitives>>,
consensus: Arc<dyn FullConsensus<BlockExecutor::Primitives, Error = ConsensusError>>,
payload_validator: Arc<dyn PayloadValidator<Block = Provider::Block>>,
) -> Result<RpcServerHandle, RpcError>
where
@ -684,7 +686,7 @@ where
Transaction = <BlockExecutor::Primitives as NodePrimitives>::SignedTx,
>,
BlockExecutor: BlockExecutorProvider,
Consensus: reth_consensus::FullConsensus<BlockExecutor::Primitives> + Clone + 'static,
Consensus: FullConsensus<BlockExecutor::Primitives, Error = ConsensusError> + Clone + 'static,
{
/// Configures all [`RpcModule`]s specific to the given [`TransportRpcModuleConfig`] which can
/// be used to start the transport server(s).
@ -1347,7 +1349,8 @@ where
/// Instantiates `ValidationApi`
pub fn validation_api(&self) -> ValidationApi<Provider, BlockExecutor>
where
Consensus: reth_consensus::FullConsensus<BlockExecutor::Primitives> + Clone + 'static,
Consensus:
FullConsensus<BlockExecutor::Primitives, Error = ConsensusError> + Clone + 'static,
Provider: BlockReader<Block = <BlockExecutor::Primitives as NodePrimitives>::Block>,
{
ValidationApi::new(
@ -1379,7 +1382,7 @@ where
>,
>,
BlockExecutor: BlockExecutorProvider,
Consensus: reth_consensus::FullConsensus<BlockExecutor::Primitives> + Clone + 'static,
Consensus: FullConsensus<BlockExecutor::Primitives, Error = ConsensusError> + Clone + 'static,
{
/// Configures the auth module that includes the
/// * `engine_` namespace

View File

@ -43,7 +43,7 @@ where
/// Create a new instance of the [`ValidationApi`]
pub fn new(
provider: Provider,
consensus: Arc<dyn FullConsensus<E::Primitives>>,
consensus: Arc<dyn FullConsensus<E::Primitives, Error = ConsensusError>>,
executor_provider: E,
config: ValidationApiConfig,
task_spawner: Box<dyn TaskSpawner>,
@ -461,7 +461,7 @@ pub struct ValidationApiInner<Provider, E: BlockExecutorProvider> {
/// The provider that can interact with the chain.
provider: Provider,
/// Consensus implementation.
consensus: Arc<dyn FullConsensus<E::Primitives>>,
consensus: Arc<dyn FullConsensus<E::Primitives, Error = ConsensusError>>,
/// Execution payload validator.
payload_validator: Arc<dyn PayloadValidator<Block = <E::Primitives as NodePrimitives>::Block>>,
/// Block executor factory.

View File

@ -30,11 +30,11 @@
//! # use reth_provider::test_utils::{create_test_provider_factory, MockNodeTypesWithDB};
//! # use reth_static_file::StaticFileProducer;
//! # use reth_config::config::StageConfig;
//! # use reth_consensus::Consensus;
//! # use reth_consensus::{Consensus, ConsensusError};
//! # use reth_consensus::test_utils::TestConsensus;
//! #
//! # let chain_spec = MAINNET.clone();
//! # let consensus: Arc<dyn Consensus> = Arc::new(TestConsensus::default());
//! # let consensus: Arc<dyn Consensus<Error = ConsensusError>> = Arc::new(TestConsensus::default());
//! # let headers_downloader = ReverseHeadersDownloaderBuilder::default().build(
//! # Arc::new(TestHeadersClient::default()),
//! # consensus.clone().as_header_validator()

View File

@ -44,7 +44,7 @@ use crate::{
};
use alloy_primitives::B256;
use reth_config::config::StageConfig;
use reth_consensus::Consensus;
use reth_consensus::{Consensus, ConsensusError};
use reth_evm::execute::BlockExecutorProvider;
use reth_network_p2p::{bodies::downloader::BodyDownloader, headers::downloader::HeaderDownloader};
use reth_provider::HeaderSyncGapProvider;
@ -102,7 +102,7 @@ where
pub fn new(
provider: Provider,
tip: watch::Receiver<B256>,
consensus: Arc<dyn Consensus<H::Header, B::Body>>,
consensus: Arc<dyn Consensus<H::Header, B::Body, Error = ConsensusError>>,
header_downloader: H,
body_downloader: B,
executor_factory: E,
@ -185,7 +185,7 @@ where
/// The tip for the headers stage.
tip: watch::Receiver<B256>,
/// The consensus engine used to validate incoming data.
consensus: Arc<dyn Consensus<H::Header, B::Body>>,
consensus: Arc<dyn Consensus<H::Header, B::Body, Error = ConsensusError>>,
/// The block header downloader
header_downloader: H,
/// The block body downloader
@ -203,7 +203,7 @@ where
pub fn new(
provider: Provider,
tip: watch::Receiver<B256>,
consensus: Arc<dyn Consensus<H::Header, B::Body>>,
consensus: Arc<dyn Consensus<H::Header, B::Body, Error = ConsensusError>>,
header_downloader: H,
body_downloader: B,
stages_config: StageConfig,
@ -236,7 +236,7 @@ where
provider: P,
tip: watch::Receiver<B256>,
header_downloader: H,
consensus: Arc<dyn Consensus<H::Header, B::Body>>,
consensus: Arc<dyn Consensus<H::Header, B::Body, Error = ConsensusError>>,
stages_config: StageConfig,
) -> StageSetBuilder<Provider>
where