engine-primitives: make engine-tree independent of beacon-consensus crate (#12560)

This commit is contained in:
Thomas Coratger
2024-11-15 09:56:46 +01:00
committed by GitHub
parent 93ec6d48fe
commit 44964ac171
27 changed files with 101 additions and 82 deletions

8
Cargo.lock generated
View File

@ -7181,11 +7181,16 @@ name = "reth-engine-primitives"
version = "1.1.1" version = "1.1.1"
dependencies = [ dependencies = [
"alloy-primitives", "alloy-primitives",
"alloy-rpc-types-engine",
"futures",
"reth-errors",
"reth-execution-types", "reth-execution-types",
"reth-payload-primitives", "reth-payload-primitives",
"reth-primitives", "reth-primitives",
"reth-trie", "reth-trie",
"serde", "serde",
"thiserror 1.0.69",
"tokio",
] ]
[[package]] [[package]]
@ -7197,6 +7202,7 @@ dependencies = [
"reth-beacon-consensus", "reth-beacon-consensus",
"reth-chainspec", "reth-chainspec",
"reth-consensus", "reth-consensus",
"reth-engine-primitives",
"reth-engine-tree", "reth-engine-tree",
"reth-ethereum-engine-primitives", "reth-ethereum-engine-primitives",
"reth-evm", "reth-evm",
@ -7278,7 +7284,6 @@ dependencies = [
"futures", "futures",
"itertools 0.13.0", "itertools 0.13.0",
"pin-project", "pin-project",
"reth-beacon-consensus",
"reth-engine-primitives", "reth-engine-primitives",
"reth-errors", "reth-errors",
"reth-ethereum-forks", "reth-ethereum-forks",
@ -8109,6 +8114,7 @@ dependencies = [
"humantime", "humantime",
"pin-project", "pin-project",
"reth-beacon-consensus", "reth-beacon-consensus",
"reth-engine-primitives",
"reth-network-api", "reth-network-api",
"reth-primitives-traits", "reth-primitives-traits",
"reth-prune", "reth-prune",

View File

@ -77,24 +77,3 @@ impl From<DatabaseError> for BeaconForkChoiceUpdateError {
Self::internal(e) Self::internal(e)
} }
} }
/// Represents all error cases when handling a new payload.
///
/// This represents all possible error cases that must be returned as JSON RCP errors back to the
/// beacon node.
#[derive(Debug, thiserror::Error)]
pub enum BeaconOnNewPayloadError {
/// Thrown when the engine task is unavailable/stopped.
#[error("beacon consensus engine task stopped")]
EngineUnavailable,
/// An internal error occurred, not necessarily related to the payload.
#[error(transparent)]
Internal(Box<dyn core::error::Error + Send + Sync>),
}
impl BeaconOnNewPayloadError {
/// Create a new internal error.
pub fn internal<E: core::error::Error + Send + Sync + 'static>(e: E) -> Self {
Self::Internal(Box::new(e))
}
}

View File

@ -1,6 +1,6 @@
use crate::engine::forkchoice::ForkchoiceStatus;
use alloy_primitives::B256; use alloy_primitives::B256;
use alloy_rpc_types_engine::ForkchoiceState; use alloy_rpc_types_engine::ForkchoiceState;
use reth_engine_primitives::ForkchoiceStatus;
use reth_primitives::{SealedBlock, SealedHeader}; use reth_primitives::{SealedBlock, SealedHeader};
use std::{ use std::{
fmt::{Display, Formatter, Result}, fmt::{Display, Formatter, Result},

View File

@ -1,14 +1,14 @@
//! `BeaconConsensusEngine` external API //! `BeaconConsensusEngine` external API
use crate::{ use crate::{BeaconConsensusEngineEvent, BeaconForkChoiceUpdateError};
engine::message::OnForkChoiceUpdated, BeaconConsensusEngineEvent, BeaconEngineMessage,
BeaconForkChoiceUpdateError, BeaconOnNewPayloadError,
};
use alloy_rpc_types_engine::{ use alloy_rpc_types_engine::{
ExecutionPayload, ExecutionPayloadSidecar, ForkchoiceState, ForkchoiceUpdated, PayloadStatus, ExecutionPayload, ExecutionPayloadSidecar, ForkchoiceState, ForkchoiceUpdated, PayloadStatus,
}; };
use futures::TryFutureExt; use futures::TryFutureExt;
use reth_engine_primitives::{EngineApiMessageVersion, EngineTypes}; use reth_engine_primitives::{
BeaconEngineMessage, BeaconOnNewPayloadError, EngineApiMessageVersion, EngineTypes,
OnForkChoiceUpdated,
};
use reth_errors::RethResult; use reth_errors::RethResult;
use reth_tokio_util::{EventSender, EventStream}; use reth_tokio_util::{EventSender, EventStream};
use tokio::sync::{mpsc::UnboundedSender, oneshot}; use tokio::sync::{mpsc::UnboundedSender, oneshot};

View File

@ -11,7 +11,11 @@ use reth_blockchain_tree_api::{
error::{BlockchainTreeError, CanonicalError, InsertBlockError, InsertBlockErrorKind}, error::{BlockchainTreeError, CanonicalError, InsertBlockError, InsertBlockErrorKind},
BlockStatus, BlockValidationKind, BlockchainTreeEngine, CanonicalOutcome, InsertPayloadOk, BlockStatus, BlockValidationKind, BlockchainTreeEngine, CanonicalOutcome, InsertPayloadOk,
}; };
use reth_engine_primitives::{EngineApiMessageVersion, EngineTypes, PayloadTypes}; use reth_engine_primitives::{
BeaconEngineMessage, BeaconOnNewPayloadError, EngineApiMessageVersion, EngineTypes,
ForkchoiceStateHash, ForkchoiceStateTracker, ForkchoiceStatus, OnForkChoiceUpdated,
PayloadTypes,
};
use reth_errors::{BlockValidationError, ProviderResult, RethError, RethResult}; use reth_errors::{BlockValidationError, ProviderResult, RethError, RethResult};
use reth_network_p2p::{ use reth_network_p2p::{
sync::{NetworkSyncUpdater, SyncState}, sync::{NetworkSyncUpdater, SyncState},
@ -42,14 +46,8 @@ use tokio::sync::{
use tokio_stream::wrappers::UnboundedReceiverStream; use tokio_stream::wrappers::UnboundedReceiverStream;
use tracing::*; use tracing::*;
mod message;
pub use message::{BeaconEngineMessage, OnForkChoiceUpdated};
mod error; mod error;
pub use error::{ pub use error::{BeaconConsensusEngineError, BeaconEngineResult, BeaconForkChoiceUpdateError};
BeaconConsensusEngineError, BeaconEngineResult, BeaconForkChoiceUpdateError,
BeaconOnNewPayloadError,
};
mod invalid_headers; mod invalid_headers;
pub use invalid_headers::InvalidHeaderCache; pub use invalid_headers::InvalidHeaderCache;
@ -60,9 +58,6 @@ pub use event::{BeaconConsensusEngineEvent, ConsensusEngineLiveSyncProgress};
mod handle; mod handle;
pub use handle::BeaconConsensusEngineHandle; pub use handle::BeaconConsensusEngineHandle;
mod forkchoice;
pub use forkchoice::{ForkchoiceStateHash, ForkchoiceStateTracker, ForkchoiceStatus};
mod metrics; mod metrics;
use metrics::EngineMetrics; use metrics::EngineMetrics;

View File

@ -2,7 +2,7 @@
use crate::{ use crate::{
engine::hooks::PruneHook, hooks::EngineHooks, BeaconConsensusEngine, engine::hooks::PruneHook, hooks::EngineHooks, BeaconConsensusEngine,
BeaconConsensusEngineError, BeaconConsensusEngineHandle, BeaconForkChoiceUpdateError, BeaconConsensusEngineError, BeaconConsensusEngineHandle, BeaconForkChoiceUpdateError,
BeaconOnNewPayloadError, EthBeaconConsensus, MIN_BLOCKS_FOR_PIPELINE_RUN, EthBeaconConsensus, MIN_BLOCKS_FOR_PIPELINE_RUN,
}; };
use alloy_primitives::{BlockNumber, B256}; use alloy_primitives::{BlockNumber, B256};
use alloy_rpc_types_engine::{ use alloy_rpc_types_engine::{
@ -19,7 +19,7 @@ use reth_downloaders::{
bodies::bodies::BodiesDownloaderBuilder, bodies::bodies::BodiesDownloaderBuilder,
headers::reverse_headers::ReverseHeadersDownloaderBuilder, headers::reverse_headers::ReverseHeadersDownloaderBuilder,
}; };
use reth_engine_primitives::EngineApiMessageVersion; use reth_engine_primitives::{BeaconOnNewPayloadError, EngineApiMessageVersion};
use reth_ethereum_engine_primitives::EthEngineTypes; use reth_ethereum_engine_primitives::EthEngineTypes;
use reth_evm::{either::Either, test_utils::MockExecutorProvider}; use reth_evm::{either::Either, test_utils::MockExecutorProvider};
use reth_evm_ethereum::execute::EthExecutorProvider; use reth_evm_ethereum::execute::EthExecutorProvider;

View File

@ -4,9 +4,8 @@ use alloy_primitives::{TxHash, B256};
use alloy_rpc_types_engine::{CancunPayloadFields, ExecutionPayloadSidecar, ForkchoiceState}; use alloy_rpc_types_engine::{CancunPayloadFields, ExecutionPayloadSidecar, ForkchoiceState};
use eyre::OptionExt; use eyre::OptionExt;
use futures_util::{stream::Fuse, StreamExt}; use futures_util::{stream::Fuse, StreamExt};
use reth_beacon_consensus::BeaconEngineMessage;
use reth_chainspec::EthereumHardforks; use reth_chainspec::EthereumHardforks;
use reth_engine_primitives::{EngineApiMessageVersion, EngineTypes}; use reth_engine_primitives::{BeaconEngineMessage, EngineApiMessageVersion, EngineTypes};
use reth_payload_builder::PayloadBuilderHandle; use reth_payload_builder::PayloadBuilderHandle;
use reth_payload_primitives::{ use reth_payload_primitives::{
BuiltPayload, PayloadAttributesBuilder, PayloadBuilder, PayloadKind, PayloadTypes, BuiltPayload, PayloadAttributesBuilder, PayloadBuilder, PayloadKind, PayloadTypes,

View File

@ -16,9 +16,10 @@ use std::{
use crate::miner::{LocalMiner, MiningMode}; use crate::miner::{LocalMiner, MiningMode};
use futures_util::{Stream, StreamExt}; use futures_util::{Stream, StreamExt};
use reth_beacon_consensus::{BeaconConsensusEngineEvent, BeaconEngineMessage, EngineNodeTypes}; use reth_beacon_consensus::{BeaconConsensusEngineEvent, EngineNodeTypes};
use reth_chainspec::EthChainSpec; use reth_chainspec::EthChainSpec;
use reth_consensus::Consensus; use reth_consensus::Consensus;
use reth_engine_primitives::BeaconEngineMessage;
use reth_engine_service::service::EngineMessageStream; use reth_engine_service::service::EngineMessageStream;
use reth_engine_tree::{ use reth_engine_tree::{
chain::{ChainEvent, HandlerEvent}, chain::{ChainEvent, HandlerEvent},

View File

@ -16,9 +16,16 @@ reth-execution-types.workspace = true
reth-payload-primitives.workspace = true reth-payload-primitives.workspace = true
reth-primitives.workspace = true reth-primitives.workspace = true
reth-trie.workspace = true reth-trie.workspace = true
reth-errors.workspace = true
# alloy # alloy
alloy-primitives.workspace = true alloy-primitives.workspace = true
alloy-rpc-types-engine.workspace = true
# async
tokio = { workspace = true, features = ["sync"] }
futures.workspace = true
# misc # misc
serde.workspace = true serde.workspace = true
thiserror.workspace = true

View File

@ -0,0 +1,20 @@
/// Represents all error cases when handling a new payload.
///
/// This represents all possible error cases that must be returned as JSON RCP errors back to the
/// beacon node.
#[derive(Debug, thiserror::Error)]
pub enum BeaconOnNewPayloadError {
/// Thrown when the engine task is unavailable/stopped.
#[error("beacon consensus engine task stopped")]
EngineUnavailable,
/// An internal error occurred, not necessarily related to the payload.
#[error(transparent)]
Internal(Box<dyn core::error::Error + Send + Sync>),
}
impl BeaconOnNewPayloadError {
/// Create a new internal error.
pub fn internal<E: core::error::Error + Send + Sync + 'static>(e: E) -> Self {
Self::Internal(Box::new(e))
}
}

View File

@ -58,13 +58,13 @@ impl ForkchoiceStateTracker {
/// Returns whether the latest received FCU is syncing: [`ForkchoiceStatus::Invalid`] /// Returns whether the latest received FCU is syncing: [`ForkchoiceStatus::Invalid`]
#[allow(dead_code)] #[allow(dead_code)]
pub(crate) fn is_latest_invalid(&self) -> bool { pub fn is_latest_invalid(&self) -> bool {
self.latest_status().map_or(false, |s| s.is_invalid()) self.latest_status().map_or(false, |s| s.is_invalid())
} }
/// Returns the last valid head hash. /// Returns the last valid head hash.
#[allow(dead_code)] #[allow(dead_code)]
pub(crate) fn last_valid_head(&self) -> Option<B256> { pub fn last_valid_head(&self) -> Option<B256> {
self.last_valid.as_ref().map(|s| s.head_block_hash) self.last_valid.as_ref().map(|s| s.head_block_hash)
} }
@ -188,7 +188,7 @@ pub enum ForkchoiceStateHash {
impl ForkchoiceStateHash { impl ForkchoiceStateHash {
/// Tries to find a matching hash in the given [`ForkchoiceState`]. /// Tries to find a matching hash in the given [`ForkchoiceState`].
pub(crate) fn find(state: &ForkchoiceState, hash: B256) -> Option<Self> { pub fn find(state: &ForkchoiceState, hash: B256) -> Option<Self> {
if state.head_block_hash == hash { if state.head_block_hash == hash {
Some(Self::Head(hash)) Some(Self::Head(hash))
} else if state.safe_block_hash == hash { } else if state.safe_block_hash == hash {
@ -201,7 +201,7 @@ impl ForkchoiceStateHash {
} }
/// Returns true if this is the head hash of the [`ForkchoiceState`] /// Returns true if this is the head hash of the [`ForkchoiceState`]
pub(crate) const fn is_head(&self) -> bool { pub const fn is_head(&self) -> bool {
matches!(self, Self::Head(_)) matches!(self, Self::Head(_))
} }
} }

View File

@ -8,6 +8,15 @@
#![cfg_attr(not(test), warn(unused_crate_dependencies))] #![cfg_attr(not(test), warn(unused_crate_dependencies))]
#![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))] #![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))]
mod error;
pub use error::BeaconOnNewPayloadError;
mod forkchoice;
pub use forkchoice::{ForkchoiceStateHash, ForkchoiceStateTracker, ForkchoiceStatus};
mod message;
pub use message::{BeaconEngineMessage, OnForkChoiceUpdated};
mod invalid_block_hook; mod invalid_block_hook;
pub use invalid_block_hook::InvalidBlockHook; pub use invalid_block_hook::InvalidBlockHook;

View File

@ -1,10 +1,9 @@
use crate::engine::{error::BeaconOnNewPayloadError, forkchoice::ForkchoiceStatus}; use crate::{BeaconOnNewPayloadError, EngineApiMessageVersion, EngineTypes, ForkchoiceStatus};
use alloy_rpc_types_engine::{ use alloy_rpc_types_engine::{
ExecutionPayload, ExecutionPayloadSidecar, ForkChoiceUpdateResult, ForkchoiceState, ExecutionPayload, ExecutionPayloadSidecar, ForkChoiceUpdateResult, ForkchoiceState,
ForkchoiceUpdateError, ForkchoiceUpdated, PayloadId, PayloadStatus, PayloadStatusEnum, ForkchoiceUpdateError, ForkchoiceUpdated, PayloadId, PayloadStatus, PayloadStatusEnum,
}; };
use futures::{future::Either, FutureExt}; use futures::{future::Either, FutureExt};
use reth_engine_primitives::{EngineApiMessageVersion, EngineTypes};
use reth_errors::RethResult; use reth_errors::RethResult;
use reth_payload_primitives::PayloadBuilderError; use reth_payload_primitives::PayloadBuilderError;
use std::{ use std::{

View File

@ -25,6 +25,7 @@ reth-stages-api.workspace = true
reth-tasks.workspace = true reth-tasks.workspace = true
reth-node-types.workspace = true reth-node-types.workspace = true
reth-chainspec.workspace = true reth-chainspec.workspace = true
reth-engine-primitives.workspace = true
# async # async
futures.workspace = true futures.workspace = true

View File

@ -1,8 +1,9 @@
use futures::{Stream, StreamExt}; use futures::{Stream, StreamExt};
use pin_project::pin_project; use pin_project::pin_project;
use reth_beacon_consensus::{BeaconConsensusEngineEvent, BeaconEngineMessage, EngineNodeTypes}; use reth_beacon_consensus::{BeaconConsensusEngineEvent, EngineNodeTypes};
use reth_chainspec::EthChainSpec; use reth_chainspec::EthChainSpec;
use reth_consensus::Consensus; use reth_consensus::Consensus;
use reth_engine_primitives::BeaconEngineMessage;
use reth_engine_tree::{ use reth_engine_tree::{
backfill::PipelineSync, backfill::PipelineSync,
download::BasicBlockDownloader, download::BasicBlockDownloader,
@ -145,6 +146,7 @@ mod tests {
use super::*; use super::*;
use reth_beacon_consensus::EthBeaconConsensus; use reth_beacon_consensus::EthBeaconConsensus;
use reth_chainspec::{ChainSpecBuilder, MAINNET}; use reth_chainspec::{ChainSpecBuilder, MAINNET};
use reth_engine_primitives::BeaconEngineMessage;
use reth_engine_tree::{test_utils::TestPipelineBuilder, tree::NoopInvalidBlockHook}; use reth_engine_tree::{test_utils::TestPipelineBuilder, tree::NoopInvalidBlockHook};
use reth_ethereum_engine_primitives::EthEngineTypes; use reth_ethereum_engine_primitives::EthEngineTypes;
use reth_evm_ethereum::execute::EthExecutorProvider; use reth_evm_ethereum::execute::EthExecutorProvider;

View File

@ -7,9 +7,9 @@ use crate::{
}; };
use alloy_primitives::B256; use alloy_primitives::B256;
use futures::{Stream, StreamExt}; use futures::{Stream, StreamExt};
use reth_beacon_consensus::{BeaconConsensusEngineEvent, BeaconEngineMessage}; use reth_beacon_consensus::BeaconConsensusEngineEvent;
use reth_chain_state::ExecutedBlock; use reth_chain_state::ExecutedBlock;
use reth_engine_primitives::EngineTypes; use reth_engine_primitives::{BeaconEngineMessage, EngineTypes};
use reth_primitives::SealedBlockWithSenders; use reth_primitives::SealedBlockWithSenders;
use std::{ use std::{
collections::HashSet, collections::HashSet,

View File

@ -15,8 +15,7 @@ use alloy_rpc_types_engine::{
PayloadValidationError, PayloadValidationError,
}; };
use reth_beacon_consensus::{ use reth_beacon_consensus::{
BeaconConsensusEngineEvent, BeaconEngineMessage, ForkchoiceStateTracker, InvalidHeaderCache, BeaconConsensusEngineEvent, InvalidHeaderCache, MIN_BLOCKS_FOR_PIPELINE_RUN,
OnForkChoiceUpdated, MIN_BLOCKS_FOR_PIPELINE_RUN,
}; };
use reth_blockchain_tree::{ use reth_blockchain_tree::{
error::{InsertBlockErrorKindTwo, InsertBlockErrorTwo, InsertBlockFatalError}, error::{InsertBlockErrorKindTwo, InsertBlockErrorTwo, InsertBlockFatalError},
@ -27,7 +26,10 @@ use reth_chain_state::{
}; };
use reth_chainspec::EthereumHardforks; use reth_chainspec::EthereumHardforks;
use reth_consensus::{Consensus, PostExecutionInput}; use reth_consensus::{Consensus, PostExecutionInput};
use reth_engine_primitives::{EngineApiMessageVersion, EngineTypes}; use reth_engine_primitives::{
BeaconEngineMessage, BeaconOnNewPayloadError, EngineApiMessageVersion, EngineTypes,
ForkchoiceStateTracker, OnForkChoiceUpdated,
};
use reth_errors::{ConsensusError, ProviderResult}; use reth_errors::{ConsensusError, ProviderResult};
use reth_evm::execute::BlockExecutorProvider; use reth_evm::execute::BlockExecutorProvider;
use reth_payload_builder::PayloadBuilderHandle; use reth_payload_builder::PayloadBuilderHandle;
@ -1246,11 +1248,11 @@ where
} }
BeaconEngineMessage::NewPayload { payload, sidecar, tx } => { BeaconEngineMessage::NewPayload { payload, sidecar, tx } => {
let output = self.on_new_payload(payload, sidecar); let output = self.on_new_payload(payload, sidecar);
if let Err(err) = tx.send(output.map(|o| o.outcome).map_err(|e| { if let Err(err) =
reth_beacon_consensus::BeaconOnNewPayloadError::Internal( tx.send(output.map(|o| o.outcome).map_err(|e| {
Box::new(e), BeaconOnNewPayloadError::Internal(Box::new(e))
) }))
})) { {
error!(target: "engine::tree", "Failed to send event: {err:?}"); error!(target: "engine::tree", "Failed to send event: {err:?}");
self.metrics self.metrics
.engine .engine
@ -2600,9 +2602,10 @@ mod tests {
use alloy_rlp::Decodable; use alloy_rlp::Decodable;
use alloy_rpc_types_engine::{CancunPayloadFields, ExecutionPayloadSidecar}; use alloy_rpc_types_engine::{CancunPayloadFields, ExecutionPayloadSidecar};
use assert_matches::assert_matches; use assert_matches::assert_matches;
use reth_beacon_consensus::{EthBeaconConsensus, ForkchoiceStatus}; use reth_beacon_consensus::EthBeaconConsensus;
use reth_chain_state::{test_utils::TestBlockBuilder, BlockState}; use reth_chain_state::{test_utils::TestBlockBuilder, BlockState};
use reth_chainspec::{ChainSpec, HOLESKY, MAINNET}; use reth_chainspec::{ChainSpec, HOLESKY, MAINNET};
use reth_engine_primitives::ForkchoiceStatus;
use reth_ethereum_engine_primitives::EthEngineTypes; use reth_ethereum_engine_primitives::EthEngineTypes;
use reth_evm::test_utils::MockExecutorProvider; use reth_evm::test_utils::MockExecutorProvider;
use reth_provider::test_utils::MockEthProvider; use reth_provider::test_utils::MockEthProvider;

View File

@ -17,7 +17,6 @@ reth-errors.workspace = true
reth-fs-util.workspace = true reth-fs-util.workspace = true
reth-rpc-types-compat.workspace = true reth-rpc-types-compat.workspace = true
reth-engine-primitives.workspace = true reth-engine-primitives.workspace = true
reth-beacon-consensus.workspace = true
reth-payload-validator.workspace = true reth-payload-validator.workspace = true
reth-evm.workspace = true reth-evm.workspace = true
reth-revm.workspace = true reth-revm.workspace = true
@ -51,8 +50,7 @@ tracing.workspace = true
[features] [features]
optimism = [ optimism = [
"reth-beacon-consensus/optimism", "reth-primitives/optimism",
"reth-primitives/optimism", "reth-provider/optimism",
"reth-provider/optimism", "revm-primitives/optimism",
"revm-primitives/optimism"
] ]

View File

@ -2,8 +2,7 @@
use alloy_rpc_types_engine::{ExecutionPayload, ExecutionPayloadSidecar, ForkchoiceState}; use alloy_rpc_types_engine::{ExecutionPayload, ExecutionPayloadSidecar, ForkchoiceState};
use futures::{Stream, StreamExt}; use futures::{Stream, StreamExt};
use reth_beacon_consensus::BeaconEngineMessage; use reth_engine_primitives::{BeaconEngineMessage, EngineTypes};
use reth_engine_primitives::EngineTypes;
use reth_fs_util as fs; use reth_fs_util as fs;
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use std::{ use std::{

View File

@ -1,8 +1,7 @@
//! Collection of various stream utilities for consensus engine. //! Collection of various stream utilities for consensus engine.
use futures::Stream; use futures::Stream;
use reth_beacon_consensus::BeaconEngineMessage; use reth_engine_primitives::{BeaconEngineMessage, EngineTypes};
use reth_engine_primitives::EngineTypes;
use reth_payload_validator::ExecutionPayloadValidator; use reth_payload_validator::ExecutionPayloadValidator;
use std::path::PathBuf; use std::path::PathBuf;
use tokio_util::either::Either; use tokio_util::either::Either;

View File

@ -7,8 +7,10 @@ use alloy_rpc_types_engine::{
}; };
use futures::{stream::FuturesUnordered, Stream, StreamExt, TryFutureExt}; use futures::{stream::FuturesUnordered, Stream, StreamExt, TryFutureExt};
use itertools::Either; use itertools::Either;
use reth_beacon_consensus::{BeaconEngineMessage, BeaconOnNewPayloadError, OnForkChoiceUpdated}; use reth_engine_primitives::{
use reth_engine_primitives::{EngineApiMessageVersion, EngineTypes}; BeaconEngineMessage, BeaconOnNewPayloadError, EngineApiMessageVersion, EngineTypes,
OnForkChoiceUpdated,
};
use reth_errors::{BlockExecutionError, BlockValidationError, RethError, RethResult}; use reth_errors::{BlockExecutionError, BlockValidationError, RethError, RethResult};
use reth_ethereum_forks::EthereumHardforks; use reth_ethereum_forks::EthereumHardforks;
use reth_evm::{ use reth_evm::{

View File

@ -1,8 +1,7 @@
//! Stream wrapper that skips specified number of FCUs. //! Stream wrapper that skips specified number of FCUs.
use futures::{Stream, StreamExt}; use futures::{Stream, StreamExt};
use reth_beacon_consensus::{BeaconEngineMessage, OnForkChoiceUpdated}; use reth_engine_primitives::{BeaconEngineMessage, EngineTypes, OnForkChoiceUpdated};
use reth_engine_primitives::EngineTypes;
use std::{ use std::{
pin::Pin, pin::Pin,
task::{ready, Context, Poll}, task::{ready, Context, Poll},

View File

@ -2,8 +2,7 @@
use alloy_rpc_types_engine::{PayloadStatus, PayloadStatusEnum}; use alloy_rpc_types_engine::{PayloadStatus, PayloadStatusEnum};
use futures::{Stream, StreamExt}; use futures::{Stream, StreamExt};
use reth_beacon_consensus::BeaconEngineMessage; use reth_engine_primitives::{BeaconEngineMessage, EngineTypes};
use reth_engine_primitives::EngineTypes;
use std::{ use std::{
pin::Pin, pin::Pin,
task::{ready, Context, Poll}, task::{ready, Context, Poll},

View File

@ -19,6 +19,7 @@ reth-stages.workspace = true
reth-prune.workspace = true reth-prune.workspace = true
reth-static-file-types.workspace = true reth-static-file-types.workspace = true
reth-primitives-traits.workspace = true reth-primitives-traits.workspace = true
reth-engine-primitives.workspace = true
# ethereum # ethereum
alloy-primitives.workspace = true alloy-primitives.workspace = true

View File

@ -5,9 +5,8 @@ use alloy_consensus::constants::GWEI_TO_WEI;
use alloy_primitives::{BlockNumber, B256}; use alloy_primitives::{BlockNumber, B256};
use alloy_rpc_types_engine::ForkchoiceState; use alloy_rpc_types_engine::ForkchoiceState;
use futures::Stream; use futures::Stream;
use reth_beacon_consensus::{ use reth_beacon_consensus::{BeaconConsensusEngineEvent, ConsensusEngineLiveSyncProgress};
BeaconConsensusEngineEvent, ConsensusEngineLiveSyncProgress, ForkchoiceStatus, use reth_engine_primitives::ForkchoiceStatus;
};
use reth_network_api::{NetworkEvent, PeersInfo}; use reth_network_api::{NetworkEvent, PeersInfo};
use reth_primitives_traits::{format_gas, format_gas_throughput}; use reth_primitives_traits::{format_gas, format_gas_throughput};
use reth_prune::PrunerEvent; use reth_prune::PrunerEvent;

View File

@ -1023,8 +1023,9 @@ mod tests {
use super::*; use super::*;
use alloy_rpc_types_engine::{ClientCode, ClientVersionV1}; use alloy_rpc_types_engine::{ClientCode, ClientVersionV1};
use assert_matches::assert_matches; use assert_matches::assert_matches;
use reth_beacon_consensus::{BeaconConsensusEngineEvent, BeaconEngineMessage}; use reth_beacon_consensus::BeaconConsensusEngineEvent;
use reth_chainspec::{ChainSpec, MAINNET}; use reth_chainspec::{ChainSpec, MAINNET};
use reth_engine_primitives::BeaconEngineMessage;
use reth_ethereum_engine_primitives::{EthEngineTypes, EthereumEngineValidator}; use reth_ethereum_engine_primitives::{EthEngineTypes, EthereumEngineValidator};
use reth_payload_builder::test_utils::spawn_test_payload_service; use reth_payload_builder::test_utils::spawn_test_payload_service;
use reth_primitives::SealedBlock; use reth_primitives::SealedBlock;

View File

@ -2,7 +2,8 @@ use alloy_primitives::{B256, U256};
use jsonrpsee_types::error::{ use jsonrpsee_types::error::{
INTERNAL_ERROR_CODE, INVALID_PARAMS_CODE, INVALID_PARAMS_MSG, SERVER_ERROR_MSG, INTERNAL_ERROR_CODE, INVALID_PARAMS_CODE, INVALID_PARAMS_MSG, SERVER_ERROR_MSG,
}; };
use reth_beacon_consensus::{BeaconForkChoiceUpdateError, BeaconOnNewPayloadError}; use reth_beacon_consensus::BeaconForkChoiceUpdateError;
use reth_engine_primitives::BeaconOnNewPayloadError;
use reth_payload_primitives::{EngineObjectValidationError, PayloadBuilderError}; use reth_payload_primitives::{EngineObjectValidationError, PayloadBuilderError};
use thiserror::Error; use thiserror::Error;