mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
feat(payload): abstract payload builder in trait (#10965)
Co-authored-by: Matthias Seitz <matthias.seitz@outlook.de>
This commit is contained in:
11
Cargo.lock
generated
11
Cargo.lock
generated
@ -6903,6 +6903,7 @@ dependencies = [
|
||||
"reth-node-builder",
|
||||
"reth-node-ethereum",
|
||||
"reth-payload-builder",
|
||||
"reth-payload-primitives",
|
||||
"reth-primitives",
|
||||
"reth-provider",
|
||||
"reth-rpc",
|
||||
@ -7748,6 +7749,7 @@ dependencies = [
|
||||
"reth-node-events",
|
||||
"reth-node-metrics",
|
||||
"reth-payload-builder",
|
||||
"reth-payload-primitives",
|
||||
"reth-payload-validator",
|
||||
"reth-primitives",
|
||||
"reth-provider",
|
||||
@ -8118,20 +8120,17 @@ name = "reth-payload-builder"
|
||||
version = "1.0.7"
|
||||
dependencies = [
|
||||
"alloy-primitives",
|
||||
"async-trait",
|
||||
"futures-util",
|
||||
"metrics",
|
||||
"pin-project",
|
||||
"reth-chain-state",
|
||||
"reth-errors",
|
||||
"reth-ethereum-engine-primitives",
|
||||
"reth-metrics",
|
||||
"reth-payload-primitives",
|
||||
"reth-primitives",
|
||||
"reth-provider",
|
||||
"reth-rpc-types",
|
||||
"reth-transaction-pool",
|
||||
"revm",
|
||||
"thiserror",
|
||||
"tokio",
|
||||
"tokio-stream",
|
||||
"tracing",
|
||||
@ -8142,6 +8141,8 @@ name = "reth-payload-primitives"
|
||||
version = "1.0.7"
|
||||
dependencies = [
|
||||
"alloy-primitives",
|
||||
"async-trait",
|
||||
"pin-project",
|
||||
"reth-chain-state",
|
||||
"reth-chainspec",
|
||||
"reth-errors",
|
||||
@ -8151,6 +8152,8 @@ dependencies = [
|
||||
"serde",
|
||||
"thiserror",
|
||||
"tokio",
|
||||
"tokio-stream",
|
||||
"tracing",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
|
||||
@ -2,7 +2,7 @@ use crate::engine::{error::BeaconOnNewPayloadError, forkchoice::ForkchoiceStatus
|
||||
use futures::{future::Either, FutureExt};
|
||||
use reth_engine_primitives::EngineTypes;
|
||||
use reth_errors::RethResult;
|
||||
use reth_payload_builder::error::PayloadBuilderError;
|
||||
use reth_payload_primitives::PayloadBuilderError;
|
||||
use reth_rpc_types::engine::{
|
||||
CancunPayloadFields, ExecutionPayload, ForkChoiceUpdateResult, ForkchoiceState,
|
||||
ForkchoiceUpdateError, ForkchoiceUpdated, PayloadId, PayloadStatus, PayloadStatusEnum,
|
||||
|
||||
@ -13,7 +13,7 @@ use reth_network_p2p::{
|
||||
};
|
||||
use reth_node_types::NodeTypesWithEngine;
|
||||
use reth_payload_builder::PayloadBuilderHandle;
|
||||
use reth_payload_primitives::{PayloadAttributes, PayloadBuilderAttributes};
|
||||
use reth_payload_primitives::{PayloadAttributes, PayloadBuilder, PayloadBuilderAttributes};
|
||||
use reth_payload_validator::ExecutionPayloadValidator;
|
||||
use reth_primitives::{
|
||||
constants::EPOCH_SLOTS, BlockNumHash, Head, Header, SealedBlock, SealedHeader,
|
||||
|
||||
@ -19,6 +19,7 @@ reth-db = { workspace = true, features = ["test-utils"] }
|
||||
reth-rpc.workspace = true
|
||||
reth-rpc-layer.workspace = true
|
||||
reth-payload-builder = { workspace = true, features = ["test-utils"] }
|
||||
reth-payload-primitives.workspace = true
|
||||
reth-provider.workspace = true
|
||||
reth-node-builder = { workspace = true, features = ["test-utils"] }
|
||||
reth-tokio-util.workspace = true
|
||||
|
||||
@ -1,6 +1,7 @@
|
||||
use futures_util::StreamExt;
|
||||
use reth::api::{BuiltPayload, EngineTypes, PayloadBuilderAttributes};
|
||||
use reth_payload_builder::{Events, PayloadBuilderHandle, PayloadId};
|
||||
use reth_payload_builder::{PayloadBuilderHandle, PayloadId};
|
||||
use reth_payload_primitives::{Events, PayloadBuilder};
|
||||
use tokio_stream::wrappers::BroadcastStream;
|
||||
|
||||
/// Helper for payload operations
|
||||
|
||||
@ -12,7 +12,7 @@ use reth_beacon_consensus::EngineNodeTypes;
|
||||
use reth_engine_tree::persistence::PersistenceHandle;
|
||||
use reth_payload_builder::PayloadBuilderHandle;
|
||||
use reth_payload_primitives::{
|
||||
BuiltPayload, PayloadAttributesBuilder, PayloadBuilderAttributes, PayloadTypes,
|
||||
BuiltPayload, PayloadAttributesBuilder, PayloadBuilder, PayloadBuilderAttributes, PayloadTypes,
|
||||
};
|
||||
use reth_provider::ProviderFactory;
|
||||
use reth_prune::PrunerWithFactory;
|
||||
|
||||
@ -23,7 +23,7 @@ use reth_engine_primitives::EngineTypes;
|
||||
use reth_errors::{ConsensusError, ProviderResult};
|
||||
use reth_evm::execute::{BlockExecutorProvider, Executor};
|
||||
use reth_payload_builder::PayloadBuilderHandle;
|
||||
use reth_payload_primitives::{PayloadAttributes, PayloadBuilderAttributes};
|
||||
use reth_payload_primitives::{PayloadAttributes, PayloadBuilder, PayloadBuilderAttributes};
|
||||
use reth_payload_validator::ExecutionPayloadValidator;
|
||||
use reth_primitives::{
|
||||
Block, GotExpected, Header, SealedBlock, SealedBlockWithSenders, SealedHeader,
|
||||
|
||||
@ -25,10 +25,8 @@ use reth_evm::{
|
||||
};
|
||||
use reth_evm_ethereum::{eip6110::parse_deposits_from_receipts, EthEvmConfig};
|
||||
use reth_execution_types::ExecutionOutcome;
|
||||
use reth_payload_builder::{
|
||||
error::PayloadBuilderError, EthBuiltPayload, EthPayloadBuilderAttributes,
|
||||
};
|
||||
use reth_payload_primitives::PayloadBuilderAttributes;
|
||||
use reth_payload_builder::{EthBuiltPayload, EthPayloadBuilderAttributes};
|
||||
use reth_payload_primitives::{PayloadBuilderAttributes, PayloadBuilderError};
|
||||
use reth_primitives::{
|
||||
constants::{eip4844::MAX_DATA_GAS_PER_BLOCK, BEACON_NONCE},
|
||||
eip4844::calculate_excess_blob_gas,
|
||||
|
||||
@ -40,6 +40,7 @@ reth-node-core.workspace = true
|
||||
reth-node-events.workspace = true
|
||||
reth-node-metrics.workspace = true
|
||||
reth-payload-builder.workspace = true
|
||||
reth-payload-primitives.workspace = true
|
||||
reth-payload-validator.workspace = true
|
||||
reth-primitives.workspace = true
|
||||
reth-provider.workspace = true
|
||||
|
||||
@ -31,6 +31,7 @@ use reth_node_core::{
|
||||
version::{CARGO_PKG_VERSION, CLIENT_CODE, NAME_CLIENT, VERGEN_GIT_SHA},
|
||||
};
|
||||
use reth_node_events::{cl::ConsensusLayerHealthEvents, node};
|
||||
use reth_payload_primitives::PayloadBuilder;
|
||||
use reth_provider::providers::BlockchainProvider2;
|
||||
use reth_rpc_engine_api::{capabilities::EngineCapabilities, EngineApi};
|
||||
use reth_rpc_types::{engine::ClientVersionV1, WithOtherFields};
|
||||
|
||||
@ -13,8 +13,7 @@ use reth_evm::{
|
||||
NextBlockEnvAttributes,
|
||||
};
|
||||
use reth_execution_types::ExecutionOutcome;
|
||||
use reth_payload_builder::error::PayloadBuilderError;
|
||||
use reth_payload_primitives::PayloadBuilderAttributes;
|
||||
use reth_payload_primitives::{PayloadBuilderAttributes, PayloadBuilderError};
|
||||
use reth_primitives::{
|
||||
constants::BEACON_NONCE,
|
||||
eip4844::calculate_excess_blob_gas,
|
||||
|
||||
@ -14,10 +14,9 @@ use futures_core::ready;
|
||||
use futures_util::FutureExt;
|
||||
use reth_chainspec::{ChainSpec, EthereumHardforks};
|
||||
use reth_payload_builder::{
|
||||
database::CachedReads, error::PayloadBuilderError, KeepPayloadJobAlive, PayloadId, PayloadJob,
|
||||
PayloadJobGenerator,
|
||||
database::CachedReads, KeepPayloadJobAlive, PayloadId, PayloadJob, PayloadJobGenerator,
|
||||
};
|
||||
use reth_payload_primitives::{BuiltPayload, PayloadBuilderAttributes};
|
||||
use reth_payload_primitives::{BuiltPayload, PayloadBuilderAttributes, PayloadBuilderError};
|
||||
use reth_primitives::{
|
||||
constants::{EMPTY_WITHDRAWALS, RETH_CLIENT_VERSION, SLOT_DURATION},
|
||||
proofs, BlockNumberOrTag, SealedBlock, Withdrawals,
|
||||
|
||||
@ -15,8 +15,6 @@ workspace = true
|
||||
# reth
|
||||
reth-primitives.workspace = true
|
||||
reth-rpc-types.workspace = true
|
||||
reth-transaction-pool.workspace = true
|
||||
reth-errors.workspace = true
|
||||
reth-provider.workspace = true
|
||||
reth-payload-primitives.workspace = true
|
||||
reth-ethereum-engine-primitives.workspace = true
|
||||
@ -26,17 +24,16 @@ reth-chain-state = { workspace = true, optional = true }
|
||||
alloy-primitives.workspace = true
|
||||
|
||||
# async
|
||||
async-trait.workspace = true
|
||||
tokio = { workspace = true, features = ["sync"] }
|
||||
tokio-stream.workspace = true
|
||||
futures-util.workspace = true
|
||||
pin-project.workspace = true
|
||||
|
||||
# metrics
|
||||
reth-metrics.workspace = true
|
||||
metrics.workspace = true
|
||||
|
||||
# misc
|
||||
thiserror.workspace = true
|
||||
tracing.workspace = true
|
||||
|
||||
[dev-dependencies]
|
||||
|
||||
@ -1,58 +0,0 @@
|
||||
//! Error types emitted by types or implementations of this crate.
|
||||
|
||||
use alloy_primitives::B256;
|
||||
use reth_errors::{ProviderError, RethError};
|
||||
use reth_primitives::revm_primitives::EVMError;
|
||||
use reth_transaction_pool::BlobStoreError;
|
||||
use tokio::sync::oneshot;
|
||||
|
||||
/// Possible error variants during payload building.
|
||||
#[derive(Debug, thiserror::Error)]
|
||||
pub enum PayloadBuilderError {
|
||||
/// Thrown when the parent block is missing.
|
||||
#[error("missing parent block {0}")]
|
||||
MissingParentBlock(B256),
|
||||
/// An oneshot channels has been closed.
|
||||
#[error("sender has been dropped")]
|
||||
ChannelClosed,
|
||||
/// If there's no payload to resolve.
|
||||
#[error("missing payload")]
|
||||
MissingPayload,
|
||||
/// Error occurring in the blob store.
|
||||
#[error(transparent)]
|
||||
BlobStore(#[from] BlobStoreError),
|
||||
/// Other internal error
|
||||
#[error(transparent)]
|
||||
Internal(#[from] RethError),
|
||||
/// Unrecoverable error during evm execution.
|
||||
#[error("evm execution error: {0}")]
|
||||
EvmExecutionError(EVMError<ProviderError>),
|
||||
/// Thrown if the payload requests withdrawals before Shanghai activation.
|
||||
#[error("withdrawals set before Shanghai activation")]
|
||||
WithdrawalsBeforeShanghai,
|
||||
/// Any other payload building errors.
|
||||
#[error(transparent)]
|
||||
Other(Box<dyn std::error::Error + Send + Sync>),
|
||||
}
|
||||
|
||||
impl PayloadBuilderError {
|
||||
/// Create a new error from a boxed error.
|
||||
pub fn other<E>(error: E) -> Self
|
||||
where
|
||||
E: std::error::Error + Send + Sync + 'static,
|
||||
{
|
||||
Self::Other(Box::new(error))
|
||||
}
|
||||
}
|
||||
|
||||
impl From<ProviderError> for PayloadBuilderError {
|
||||
fn from(error: ProviderError) -> Self {
|
||||
Self::Internal(RethError::Provider(error))
|
||||
}
|
||||
}
|
||||
|
||||
impl From<oneshot::error::RecvError> for PayloadBuilderError {
|
||||
fn from(_: oneshot::error::RecvError) -> Self {
|
||||
Self::ChannelClosed
|
||||
}
|
||||
}
|
||||
|
||||
@ -27,8 +27,7 @@
|
||||
//! use std::future::Future;
|
||||
//! use std::pin::Pin;
|
||||
//! use std::task::{Context, Poll};
|
||||
//! use reth_payload_builder::{EthBuiltPayload, KeepPayloadJobAlive, EthPayloadBuilderAttributes, PayloadJob, PayloadJobGenerator};
|
||||
//! use reth_payload_builder::error::PayloadBuilderError;
|
||||
//! use reth_payload_builder::{EthBuiltPayload, PayloadBuilderError, KeepPayloadJobAlive, EthPayloadBuilderAttributes, PayloadJob, PayloadJobGenerator};
|
||||
//! use reth_primitives::{Block, Header, U256};
|
||||
//!
|
||||
//! /// The generator type that creates new jobs that builds empty blocks.
|
||||
@ -102,8 +101,6 @@
|
||||
#![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))]
|
||||
|
||||
pub mod database;
|
||||
pub mod error;
|
||||
mod events;
|
||||
mod metrics;
|
||||
mod service;
|
||||
mod traits;
|
||||
@ -113,7 +110,7 @@ pub mod noop;
|
||||
#[cfg(any(test, feature = "test-utils"))]
|
||||
pub mod test_utils;
|
||||
|
||||
pub use events::{Events, PayloadEvents};
|
||||
pub use reth_payload_primitives::PayloadBuilderError;
|
||||
pub use reth_rpc_types::engine::PayloadId;
|
||||
pub use service::{
|
||||
PayloadBuilderHandle, PayloadBuilderService, PayloadServiceCommand, PayloadStore,
|
||||
|
||||
@ -20,7 +20,7 @@ pub struct NoopPayloadBuilderService<T: PayloadTypes> {
|
||||
|
||||
impl<T> NoopPayloadBuilderService<T>
|
||||
where
|
||||
T: PayloadTypes + 'static,
|
||||
T: PayloadTypes,
|
||||
{
|
||||
/// Creates a new [`NoopPayloadBuilderService`].
|
||||
pub fn new() -> (Self, PayloadBuilderHandle<T>) {
|
||||
|
||||
@ -4,14 +4,14 @@
|
||||
//! Once a new payload is created, it is continuously updated.
|
||||
|
||||
use crate::{
|
||||
error::PayloadBuilderError,
|
||||
events::{Events, PayloadEvents},
|
||||
metrics::PayloadBuilderServiceMetrics,
|
||||
traits::PayloadJobGenerator,
|
||||
KeepPayloadJobAlive, PayloadJob,
|
||||
metrics::PayloadBuilderServiceMetrics, traits::PayloadJobGenerator, KeepPayloadJobAlive,
|
||||
PayloadJob,
|
||||
};
|
||||
use futures_util::{future::FutureExt, Stream, StreamExt};
|
||||
use reth_payload_primitives::{BuiltPayload, PayloadBuilderAttributes, PayloadTypes};
|
||||
use reth_payload_primitives::{
|
||||
BuiltPayload, Events, PayloadBuilder, PayloadBuilderAttributes, PayloadBuilderError,
|
||||
PayloadEvents, PayloadTypes,
|
||||
};
|
||||
use reth_provider::CanonStateNotification;
|
||||
use reth_rpc_types::engine::PayloadId;
|
||||
use std::{
|
||||
@ -22,7 +22,7 @@ use std::{
|
||||
};
|
||||
use tokio::sync::{
|
||||
broadcast, mpsc,
|
||||
oneshot::{self, error::RecvError},
|
||||
oneshot::{self, Receiver},
|
||||
};
|
||||
use tokio_stream::wrappers::UnboundedReceiverStream;
|
||||
use tracing::{debug, info, trace, warn};
|
||||
@ -39,7 +39,7 @@ pub struct PayloadStore<T: PayloadTypes> {
|
||||
|
||||
impl<T> PayloadStore<T>
|
||||
where
|
||||
T: PayloadTypes + 'static,
|
||||
T: PayloadTypes,
|
||||
{
|
||||
/// Resolves the payload job and returns the best payload that has been built so far.
|
||||
///
|
||||
@ -102,9 +102,63 @@ pub struct PayloadBuilderHandle<T: PayloadTypes> {
|
||||
|
||||
// === impl PayloadBuilderHandle ===
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl<T> PayloadBuilder for PayloadBuilderHandle<T>
|
||||
where
|
||||
T: PayloadTypes,
|
||||
{
|
||||
type PayloadType = T;
|
||||
type Error = PayloadBuilderError;
|
||||
|
||||
async fn send_and_resolve_payload(
|
||||
&self,
|
||||
attr: <Self::PayloadType as PayloadTypes>::PayloadBuilderAttributes,
|
||||
) -> Result<PayloadFuture<<Self::PayloadType as PayloadTypes>::BuiltPayload>, Self::Error> {
|
||||
let rx = self.send_new_payload(attr);
|
||||
let id = rx.await??;
|
||||
|
||||
let (tx, rx) = oneshot::channel();
|
||||
let _ = self.to_service.send(PayloadServiceCommand::Resolve(id, tx));
|
||||
rx.await?.ok_or(PayloadBuilderError::MissingPayload)
|
||||
}
|
||||
|
||||
/// Note: this does not resolve the job if it's still in progress.
|
||||
async fn best_payload(
|
||||
&self,
|
||||
id: PayloadId,
|
||||
) -> Option<Result<<Self::PayloadType as PayloadTypes>::BuiltPayload, Self::Error>> {
|
||||
let (tx, rx) = oneshot::channel();
|
||||
self.to_service.send(PayloadServiceCommand::BestPayload(id, tx)).ok()?;
|
||||
rx.await.ok()?
|
||||
}
|
||||
|
||||
fn send_new_payload(
|
||||
&self,
|
||||
attr: <Self::PayloadType as PayloadTypes>::PayloadBuilderAttributes,
|
||||
) -> Receiver<Result<PayloadId, Self::Error>> {
|
||||
let (tx, rx) = oneshot::channel();
|
||||
let _ = self.to_service.send(PayloadServiceCommand::BuildNewPayload(attr, tx));
|
||||
rx
|
||||
}
|
||||
|
||||
/// Note: if there's already payload in progress with same identifier, it will be returned.
|
||||
async fn new_payload(
|
||||
&self,
|
||||
attr: <Self::PayloadType as PayloadTypes>::PayloadBuilderAttributes,
|
||||
) -> Result<PayloadId, Self::Error> {
|
||||
self.send_new_payload(attr).await?
|
||||
}
|
||||
|
||||
async fn subscribe(&self) -> Result<PayloadEvents<Self::PayloadType>, Self::Error> {
|
||||
let (tx, rx) = oneshot::channel();
|
||||
let _ = self.to_service.send(PayloadServiceCommand::Subscribe(tx));
|
||||
Ok(PayloadEvents { receiver: rx.await? })
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> PayloadBuilderHandle<T>
|
||||
where
|
||||
T: PayloadTypes + 'static,
|
||||
T: PayloadTypes,
|
||||
{
|
||||
/// Creates a new payload builder handle for the given channel.
|
||||
///
|
||||
@ -127,32 +181,6 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
/// Sends a message to the service to start building a new payload for the given payload
|
||||
/// attributes and returns a future that resolves to the payload.
|
||||
pub async fn send_and_resolve_payload(
|
||||
&self,
|
||||
attr: T::PayloadBuilderAttributes,
|
||||
) -> Result<PayloadFuture<T::BuiltPayload>, PayloadBuilderError> {
|
||||
let rx = self.send_new_payload(attr);
|
||||
let id = rx.await??;
|
||||
|
||||
let (tx, rx) = oneshot::channel();
|
||||
let _ = self.to_service.send(PayloadServiceCommand::Resolve(id, tx));
|
||||
rx.await?.ok_or(PayloadBuilderError::MissingPayload)
|
||||
}
|
||||
|
||||
/// Returns the best payload for the given identifier.
|
||||
///
|
||||
/// Note: this does not resolve the job if it's still in progress.
|
||||
pub async fn best_payload(
|
||||
&self,
|
||||
id: PayloadId,
|
||||
) -> Option<Result<T::BuiltPayload, PayloadBuilderError>> {
|
||||
let (tx, rx) = oneshot::channel();
|
||||
self.to_service.send(PayloadServiceCommand::BestPayload(id, tx)).ok()?;
|
||||
rx.await.ok()?
|
||||
}
|
||||
|
||||
/// Returns the payload attributes associated with the given identifier.
|
||||
///
|
||||
/// Note: this returns the attributes of the payload and does not resolve the job.
|
||||
@ -164,39 +192,6 @@ where
|
||||
self.to_service.send(PayloadServiceCommand::PayloadAttributes(id, tx)).ok()?;
|
||||
rx.await.ok()?
|
||||
}
|
||||
|
||||
/// Sends a message to the service to start building a new payload for the given payload.
|
||||
///
|
||||
/// This is the same as [`PayloadBuilderHandle::new_payload`] but does not wait for the result
|
||||
/// and returns the receiver instead
|
||||
pub fn send_new_payload(
|
||||
&self,
|
||||
attr: T::PayloadBuilderAttributes,
|
||||
) -> oneshot::Receiver<Result<PayloadId, PayloadBuilderError>> {
|
||||
let (tx, rx) = oneshot::channel();
|
||||
let _ = self.to_service.send(PayloadServiceCommand::BuildNewPayload(attr, tx));
|
||||
rx
|
||||
}
|
||||
|
||||
/// Starts building a new payload for the given payload attributes.
|
||||
///
|
||||
/// Returns the identifier of the payload.
|
||||
///
|
||||
/// Note: if there's already payload in progress with same identifier, it will be returned.
|
||||
pub async fn new_payload(
|
||||
&self,
|
||||
attr: T::PayloadBuilderAttributes,
|
||||
) -> Result<PayloadId, PayloadBuilderError> {
|
||||
self.send_new_payload(attr).await?
|
||||
}
|
||||
|
||||
/// Sends a message to the service to subscribe to payload events.
|
||||
/// Returns a receiver that will receive them.
|
||||
pub async fn subscribe(&self) -> Result<PayloadEvents<T>, RecvError> {
|
||||
let (tx, rx) = oneshot::channel();
|
||||
let _ = self.to_service.send(PayloadServiceCommand::Subscribe(tx));
|
||||
Ok(PayloadEvents { receiver: rx.await? })
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> Clone for PayloadBuilderHandle<T>
|
||||
@ -246,7 +241,7 @@ const PAYLOAD_EVENTS_BUFFER_SIZE: usize = 20;
|
||||
|
||||
impl<Gen, St, T> PayloadBuilderService<Gen, St, T>
|
||||
where
|
||||
T: PayloadTypes + 'static,
|
||||
T: PayloadTypes,
|
||||
Gen: PayloadJobGenerator,
|
||||
Gen::Job: PayloadJob<PayloadAttributes = T::PayloadBuilderAttributes>,
|
||||
<Gen::Job as PayloadJob>::BuiltPayload: Into<T::BuiltPayload>,
|
||||
@ -360,7 +355,7 @@ where
|
||||
|
||||
impl<Gen, St, T> Future for PayloadBuilderService<Gen, St, T>
|
||||
where
|
||||
T: PayloadTypes + 'static,
|
||||
T: PayloadTypes,
|
||||
Gen: PayloadJobGenerator + Unpin + 'static,
|
||||
<Gen as PayloadJobGenerator>::Job: Unpin + 'static,
|
||||
St: Stream<Item = CanonStateNotification> + Send + Unpin + 'static,
|
||||
|
||||
@ -1,13 +1,13 @@
|
||||
//! Utils for testing purposes.
|
||||
|
||||
use crate::{
|
||||
error::PayloadBuilderError, traits::KeepPayloadJobAlive, EthBuiltPayload,
|
||||
EthPayloadBuilderAttributes, PayloadBuilderHandle, PayloadBuilderService, PayloadJob,
|
||||
PayloadJobGenerator,
|
||||
traits::KeepPayloadJobAlive, EthBuiltPayload, EthPayloadBuilderAttributes,
|
||||
PayloadBuilderHandle, PayloadBuilderService, PayloadJob, PayloadJobGenerator,
|
||||
};
|
||||
|
||||
use alloy_primitives::U256;
|
||||
use reth_chain_state::ExecutedBlock;
|
||||
use reth_payload_primitives::PayloadTypes;
|
||||
use reth_payload_primitives::{PayloadBuilderError, PayloadTypes};
|
||||
use reth_primitives::Block;
|
||||
use reth_provider::CanonStateNotification;
|
||||
use std::{
|
||||
|
||||
@ -1,7 +1,6 @@
|
||||
//! Trait abstractions used by the payload crate.
|
||||
|
||||
use crate::error::PayloadBuilderError;
|
||||
use reth_payload_primitives::{BuiltPayload, PayloadBuilderAttributes};
|
||||
use reth_payload_primitives::{BuiltPayload, PayloadBuilderAttributes, PayloadBuilderError};
|
||||
use reth_provider::CanonStateNotification;
|
||||
use std::future::Future;
|
||||
|
||||
|
||||
@ -24,8 +24,12 @@ reth-chain-state.workspace = true
|
||||
alloy-primitives.workspace = true
|
||||
|
||||
# async
|
||||
async-trait.workspace = true
|
||||
tokio = { workspace = true, features = ["sync"] }
|
||||
tokio-stream.workspace = true
|
||||
pin-project.workspace = true
|
||||
|
||||
# misc
|
||||
thiserror.workspace = true
|
||||
serde.workspace = true
|
||||
thiserror.workspace = true
|
||||
tracing.workspace = true
|
||||
|
||||
@ -15,6 +15,12 @@ pub enum PayloadBuilderError {
|
||||
/// An oneshot channels has been closed.
|
||||
#[error("sender has been dropped")]
|
||||
ChannelClosed,
|
||||
/// If there's no payload to resolve.
|
||||
#[error("missing payload")]
|
||||
MissingPayload,
|
||||
/// Build cancelled
|
||||
#[error("build outcome cancelled")]
|
||||
BuildOutcomeCancelled,
|
||||
/// Error occurring in the blob store.
|
||||
#[error(transparent)]
|
||||
BlobStore(#[from] BlobStoreError),
|
||||
|
||||
@ -1,4 +1,4 @@
|
||||
use reth_payload_primitives::PayloadTypes;
|
||||
use crate::PayloadTypes;
|
||||
use std::{
|
||||
pin::Pin,
|
||||
task::{ready, Context, Poll},
|
||||
@ -29,7 +29,7 @@ pub struct PayloadEvents<T: PayloadTypes> {
|
||||
pub receiver: broadcast::Receiver<Events<T>>,
|
||||
}
|
||||
|
||||
impl<T: PayloadTypes + 'static> PayloadEvents<T> {
|
||||
impl<T: PayloadTypes> PayloadEvents<T> {
|
||||
/// Convert this receiver into a stream of `PayloadEvents`.
|
||||
pub fn into_stream(self) -> BroadcastStream<Events<T>> {
|
||||
BroadcastStream::new(self.receiver)
|
||||
@ -60,7 +60,7 @@ pub struct BuiltPayloadStream<T: PayloadTypes> {
|
||||
st: BroadcastStream<Events<T>>,
|
||||
}
|
||||
|
||||
impl<T: PayloadTypes + 'static> Stream for BuiltPayloadStream<T> {
|
||||
impl<T: PayloadTypes> Stream for BuiltPayloadStream<T> {
|
||||
type Item = T::BuiltPayload;
|
||||
|
||||
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
|
||||
@ -90,7 +90,7 @@ pub struct PayloadAttributeStream<T: PayloadTypes> {
|
||||
st: BroadcastStream<Events<T>>,
|
||||
}
|
||||
|
||||
impl<T: PayloadTypes + 'static> Stream for PayloadAttributeStream<T> {
|
||||
impl<T: PayloadTypes> Stream for PayloadAttributeStream<T> {
|
||||
type Item = T::PayloadBuilderAttributes;
|
||||
|
||||
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
|
||||
@ -12,11 +12,15 @@ mod error;
|
||||
|
||||
pub use error::{EngineObjectValidationError, PayloadBuilderError, VersionSpecificValidationError};
|
||||
|
||||
mod events;
|
||||
pub use crate::events::{Events, PayloadEvents};
|
||||
|
||||
/// Contains traits to abstract over payload attributes types and default implementations of the
|
||||
/// [`PayloadAttributes`] trait for ethereum mainnet and optimism types.
|
||||
mod traits;
|
||||
pub use traits::{
|
||||
BuiltPayload, PayloadAttributes, PayloadAttributesBuilder, PayloadBuilderAttributes,
|
||||
BuiltPayload, PayloadAttributes, PayloadAttributesBuilder, PayloadBuilder,
|
||||
PayloadBuilderAttributes,
|
||||
};
|
||||
|
||||
mod payload;
|
||||
@ -24,7 +28,7 @@ pub use payload::PayloadOrAttributes;
|
||||
|
||||
use reth_chainspec::{ChainSpec, EthereumHardforks};
|
||||
/// The types that are used by the engine API.
|
||||
pub trait PayloadTypes: Send + Sync + Unpin + core::fmt::Debug + Clone {
|
||||
pub trait PayloadTypes: Send + Sync + Unpin + core::fmt::Debug + Clone + 'static {
|
||||
/// The built payload type.
|
||||
type BuiltPayload: BuiltPayload + Clone + Unpin;
|
||||
|
||||
|
||||
@ -1,5 +1,6 @@
|
||||
use crate::{
|
||||
validate_version_specific_fields, EngineApiMessageVersion, EngineObjectValidationError,
|
||||
PayloadBuilderError, PayloadEvents, PayloadTypes,
|
||||
};
|
||||
use alloy_primitives::{Address, B256, U256};
|
||||
use reth_chain_state::ExecutedBlock;
|
||||
@ -10,6 +11,54 @@ use reth_rpc_types::{
|
||||
optimism::OptimismPayloadAttributes,
|
||||
Withdrawal,
|
||||
};
|
||||
use std::{future::Future, pin::Pin};
|
||||
use tokio::sync::oneshot;
|
||||
|
||||
pub(crate) type PayloadFuture<P> =
|
||||
Pin<Box<dyn Future<Output = Result<P, PayloadBuilderError>> + Send + Sync>>;
|
||||
|
||||
/// A type that can request, subscribe to and resolve payloads.
|
||||
#[async_trait::async_trait]
|
||||
pub trait PayloadBuilder: Send + Unpin {
|
||||
/// The Payload type for the builder.
|
||||
type PayloadType: PayloadTypes;
|
||||
/// The error type returned by the builder.
|
||||
type Error;
|
||||
|
||||
/// Sends a message to the service to start building a new payload for the given payload
|
||||
/// attributes and returns a future that resolves to the payload.
|
||||
async fn send_and_resolve_payload(
|
||||
&self,
|
||||
attr: <Self::PayloadType as PayloadTypes>::PayloadBuilderAttributes,
|
||||
) -> Result<PayloadFuture<<Self::PayloadType as PayloadTypes>::BuiltPayload>, Self::Error>;
|
||||
|
||||
/// Returns the best payload for the given identifier.
|
||||
async fn best_payload(
|
||||
&self,
|
||||
id: PayloadId,
|
||||
) -> Option<Result<<Self::PayloadType as PayloadTypes>::BuiltPayload, Self::Error>>;
|
||||
|
||||
/// Sends a message to the service to start building a new payload for the given payload.
|
||||
///
|
||||
/// This is the same as [`PayloadBuilder::new_payload`] but does not wait for the result
|
||||
/// and returns the receiver instead
|
||||
fn send_new_payload(
|
||||
&self,
|
||||
attr: <Self::PayloadType as PayloadTypes>::PayloadBuilderAttributes,
|
||||
) -> oneshot::Receiver<Result<PayloadId, Self::Error>>;
|
||||
|
||||
/// Starts building a new payload for the given payload attributes.
|
||||
///
|
||||
/// Returns the identifier of the payload.
|
||||
async fn new_payload(
|
||||
&self,
|
||||
attr: <Self::PayloadType as PayloadTypes>::PayloadBuilderAttributes,
|
||||
) -> Result<PayloadId, Self::Error>;
|
||||
|
||||
/// Sends a message to the service to subscribe to payload events.
|
||||
/// Returns a receiver that will receive them.
|
||||
async fn subscribe(&self) -> Result<PayloadEvents<Self::PayloadType>, Self::Error>;
|
||||
}
|
||||
|
||||
/// Represents a built payload type that contains a built [`SealedBlock`] and can be converted into
|
||||
/// engine API execution payloads.
|
||||
|
||||
@ -3,8 +3,7 @@ use jsonrpsee_types::error::{
|
||||
INTERNAL_ERROR_CODE, INVALID_PARAMS_CODE, INVALID_PARAMS_MSG, SERVER_ERROR_MSG,
|
||||
};
|
||||
use reth_beacon_consensus::{BeaconForkChoiceUpdateError, BeaconOnNewPayloadError};
|
||||
use reth_payload_builder::error::PayloadBuilderError;
|
||||
use reth_payload_primitives::EngineObjectValidationError;
|
||||
use reth_payload_primitives::{EngineObjectValidationError, PayloadBuilderError};
|
||||
use reth_rpc_types::ToRpcError;
|
||||
use thiserror::Error;
|
||||
|
||||
|
||||
@ -52,7 +52,7 @@ use reth_node_ethereum::{
|
||||
EthEvmConfig,
|
||||
};
|
||||
use reth_payload_builder::{
|
||||
error::PayloadBuilderError, EthBuiltPayload, EthPayloadBuilderAttributes, PayloadBuilderHandle,
|
||||
EthBuiltPayload, EthPayloadBuilderAttributes, PayloadBuilderError, PayloadBuilderHandle,
|
||||
PayloadBuilderService,
|
||||
};
|
||||
use reth_primitives::{Address, Withdrawals, B256};
|
||||
|
||||
@ -7,7 +7,7 @@ use reth::{
|
||||
use reth_basic_payload_builder::{BasicPayloadJobGeneratorConfig, PayloadBuilder, PayloadConfig};
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_node_api::PayloadBuilderAttributes;
|
||||
use reth_payload_builder::{error::PayloadBuilderError, PayloadJobGenerator};
|
||||
use reth_payload_builder::{PayloadBuilderError, PayloadJobGenerator};
|
||||
use reth_primitives::{BlockNumberOrTag, Bytes};
|
||||
use std::sync::Arc;
|
||||
|
||||
|
||||
@ -3,7 +3,7 @@ use reth::{
|
||||
providers::StateProviderFactory, tasks::TaskSpawner, transaction_pool::TransactionPool,
|
||||
};
|
||||
use reth_basic_payload_builder::{PayloadBuilder, PayloadConfig};
|
||||
use reth_payload_builder::{error::PayloadBuilderError, KeepPayloadJobAlive, PayloadJob};
|
||||
use reth_payload_builder::{KeepPayloadJobAlive, PayloadBuilderError, PayloadJob};
|
||||
|
||||
use std::{
|
||||
pin::Pin,
|
||||
|
||||
Reference in New Issue
Block a user