feat(rpc): relax VaidationApi and EngineApi (#13241)

This commit is contained in:
Arsenii Kulikov
2024-12-10 01:48:29 +04:00
committed by GitHub
parent af5dc60867
commit eb6080863b
20 changed files with 259 additions and 193 deletions

View File

@ -31,8 +31,6 @@ reth-transaction-pool.workspace = true
reth-evm.workspace = true
reth-engine-primitives.workspace = true
alloy-consensus.workspace = true
# rpc/net
jsonrpsee = { workspace = true, features = ["server"] }
tower-http = { workspace = true, features = ["full"] }

View File

@ -1,4 +1,3 @@
use alloy_consensus::Header;
use reth_evm::ConfigureEvm;
use reth_primitives::NodePrimitives;
use reth_provider::{BlockReader, CanonStateSubscriptions, EvmEnvProvider, StateProviderFactory};
@ -62,7 +61,7 @@ where
>,
) -> Self
where
EvmConfig: ConfigureEvm<Header = Header>,
EvmConfig: ConfigureEvm<Header = Provider::Header>,
Tasks: TaskSpawner + Clone + 'static,
{
let cache = EthStateCache::spawn_with(provider.clone(), config.cache, executor.clone());

View File

@ -16,10 +16,10 @@
//! Configure only an http server with a selection of [`RethRpcModule`]s
//!
//! ```
//! use alloy_consensus::Header;
//! use reth_engine_primitives::PayloadValidator;
//! use reth_evm::{execute::BlockExecutorProvider, ConfigureEvm};
//! use reth_network_api::{NetworkInfo, Peers};
//! use reth_primitives::TransactionSigned;
//! use reth_primitives::{Header, TransactionSigned};
//! use reth_provider::{AccountReader, CanonStateSubscriptions, ChangeSetReader, FullRpcProvider};
//! use reth_rpc::EthApi;
//! use reth_rpc_builder::{
@ -27,8 +27,18 @@
//! };
//! use reth_tasks::TokioTaskExecutor;
//! use reth_transaction_pool::{PoolTransaction, TransactionPool};
//! use std::sync::Arc;
//!
//! pub async fn launch<Provider, Pool, Network, Events, EvmConfig, BlockExecutor, Consensus>(
//! pub async fn launch<
//! Provider,
//! Pool,
//! Network,
//! Events,
//! EvmConfig,
//! BlockExecutor,
//! Consensus,
//! Validator,
//! >(
//! provider: Provider,
//! pool: Pool,
//! network: Network,
@ -36,6 +46,7 @@
//! evm_config: EvmConfig,
//! block_executor: BlockExecutor,
//! consensus: Consensus,
//! validator: Validator,
//! ) where
//! Provider: FullRpcProvider<
//! Transaction = TransactionSigned,
@ -53,6 +64,7 @@
//! EvmConfig: ConfigureEvm<Header = Header, Transaction = TransactionSigned>,
//! BlockExecutor: BlockExecutorProvider<Primitives = Events::Primitives>,
//! Consensus: reth_consensus::FullConsensus + Clone + 'static,
//! Validator: PayloadValidator<Block = reth_primitives::Block>,
//! {
//! // configure the rpc module per transport
//! let transports = TransportRpcModuleConfig::default().with_http(vec![
@ -71,7 +83,7 @@
//! block_executor,
//! consensus,
//! )
//! .build(transports, Box::new(EthApi::with_spawner));
//! .build(transports, Box::new(EthApi::with_spawner), Arc::new(validator));
//! let handle = RpcServerConfig::default()
//! .with_http(ServerBuilder::default())
//! .start(&transport_modules)
@ -83,11 +95,10 @@
//!
//!
//! ```
//! use alloy_consensus::Header;
//! use reth_engine_primitives::EngineTypes;
//! use reth_engine_primitives::{EngineTypes, PayloadValidator};
//! use reth_evm::{execute::BlockExecutorProvider, ConfigureEvm};
//! use reth_network_api::{NetworkInfo, Peers};
//! use reth_primitives::TransactionSigned;
//! use reth_primitives::{Header, TransactionSigned};
//! use reth_provider::{AccountReader, CanonStateSubscriptions, ChangeSetReader, FullRpcProvider};
//! use reth_rpc::EthApi;
//! use reth_rpc_api::EngineApiServer;
@ -98,6 +109,7 @@
//! use reth_rpc_layer::JwtSecret;
//! use reth_tasks::TokioTaskExecutor;
//! use reth_transaction_pool::{PoolTransaction, TransactionPool};
//! use std::sync::Arc;
//! use tokio::try_join;
//!
//! pub async fn launch<
@ -110,6 +122,7 @@
//! EvmConfig,
//! BlockExecutor,
//! Consensus,
//! Validator,
//! >(
//! provider: Provider,
//! pool: Pool,
@ -119,6 +132,7 @@
//! evm_config: EvmConfig,
//! block_executor: BlockExecutor,
//! consensus: Consensus,
//! validator: Validator,
//! ) where
//! Provider: FullRpcProvider<
//! Transaction = TransactionSigned,
@ -138,6 +152,7 @@
//! EvmConfig: ConfigureEvm<Header = Header, Transaction = TransactionSigned>,
//! BlockExecutor: BlockExecutorProvider<Primitives = Events::Primitives>,
//! Consensus: reth_consensus::FullConsensus + Clone + 'static,
//! Validator: PayloadValidator<Block = reth_primitives::Block>,
//! {
//! // configure the rpc module per transport
//! let transports = TransportRpcModuleConfig::default().with_http(vec![
@ -158,8 +173,12 @@
//! );
//!
//! // configure the server modules
//! let (modules, auth_module, _registry) =
//! builder.build_with_auth_server(transports, engine_api, Box::new(EthApi::with_spawner));
//! let (modules, auth_module, _registry) = builder.build_with_auth_server(
//! transports,
//! engine_api,
//! Box::new(EthApi::with_spawner),
//! Arc::new(validator),
//! );
//!
//! // start the servers
//! let auth_config = AuthServerConfig::builder(JwtSecret::random()).build();
@ -187,7 +206,6 @@ use std::{
};
use crate::{auth::AuthRpcModule, error::WsHttpSamePortError, metrics::RpcRequestMetrics};
use alloy_consensus::Header;
use error::{ConflictingModules, RpcError, ServerKind};
use eth::DynEthApiBuilder;
use http::{header::AUTHORIZATION, HeaderMap};
@ -201,7 +219,7 @@ use jsonrpsee::{
};
use reth_chainspec::EthereumHardforks;
use reth_consensus::FullConsensus;
use reth_engine_primitives::EngineTypes;
use reth_engine_primitives::{EngineTypes, PayloadValidator};
use reth_evm::{execute::BlockExecutorProvider, ConfigureEvm};
use reth_network_api::{noop::NoopNetwork, NetworkInfo, Peers};
use reth_primitives::NodePrimitives;
@ -274,6 +292,7 @@ pub async fn launch<Provider, Pool, Network, Tasks, Events, EvmConfig, EthApi, B
eth: DynEthApiBuilder<Provider, Pool, EvmConfig, Network, Tasks, Events, EthApi>,
block_executor: BlockExecutor,
consensus: Arc<dyn FullConsensus<BlockExecutor::Primitives>>,
payload_validator: Arc<dyn PayloadValidator<Block = Provider::Block>>,
) -> Result<RpcServerHandle, RpcError>
where
Provider: FullRpcProvider<
@ -297,12 +316,7 @@ where
Header = <BlockExecutor::Primitives as NodePrimitives>::BlockHeader,
>,
>,
BlockExecutor: BlockExecutorProvider<
Primitives: NodePrimitives<
BlockHeader = reth_primitives::Header,
BlockBody = reth_primitives::BlockBody,
>,
>,
BlockExecutor: BlockExecutorProvider,
{
let module_config = module_config.into();
server_config
@ -318,7 +332,7 @@ where
block_executor,
consensus,
)
.build(module_config, eth),
.build(module_config, eth, payload_validator),
)
.await
}
@ -651,6 +665,7 @@ where
Provider: FullRpcProvider<
Block = <Events::Primitives as NodePrimitives>::Block,
Receipt = <Events::Primitives as NodePrimitives>::Receipt,
Header = <Events::Primitives as NodePrimitives>::BlockHeader,
> + AccountReader
+ ChangeSetReader,
Pool: TransactionPool + 'static,
@ -661,12 +676,7 @@ where
Header = <BlockExecutor::Primitives as NodePrimitives>::BlockHeader,
Transaction = <BlockExecutor::Primitives as NodePrimitives>::SignedTx,
>,
BlockExecutor: BlockExecutorProvider<
Primitives: NodePrimitives<
BlockHeader = reth_primitives::Header,
BlockBody = reth_primitives::BlockBody,
>,
>,
BlockExecutor: BlockExecutorProvider,
Consensus: reth_consensus::FullConsensus<BlockExecutor::Primitives> + Clone + 'static,
{
/// Configures all [`RpcModule`]s specific to the given [`TransportRpcModuleConfig`] which can
@ -681,6 +691,7 @@ where
module_config: TransportRpcModuleConfig,
engine: EngineApi,
eth: DynEthApiBuilder<Provider, Pool, EvmConfig, Network, Tasks, Events, EthApi>,
payload_validator: Arc<dyn PayloadValidator<Block = Provider::Block>>,
) -> (
TransportRpcModules,
AuthRpcModule,
@ -721,6 +732,7 @@ where
evm_config,
eth,
block_executor,
payload_validator,
);
let modules = registry.create_transport_rpc_modules(module_config);
@ -738,21 +750,24 @@ where
/// # Example
///
/// ```no_run
/// use alloy_consensus::Header;
/// use reth_consensus::noop::NoopConsensus;
/// use reth_engine_primitives::PayloadValidator;
/// use reth_evm::ConfigureEvm;
/// use reth_evm_ethereum::execute::EthExecutorProvider;
/// use reth_network_api::noop::NoopNetwork;
/// use reth_primitives::TransactionSigned;
/// use reth_primitives::{Header, TransactionSigned};
/// use reth_provider::test_utils::{NoopProvider, TestCanonStateSubscriptions};
/// use reth_rpc::EthApi;
/// use reth_rpc_builder::RpcModuleBuilder;
/// use reth_tasks::TokioTaskExecutor;
/// use reth_transaction_pool::noop::NoopTransactionPool;
/// use std::sync::Arc;
///
/// fn init<Evm: ConfigureEvm<Header = Header, Transaction = TransactionSigned> + 'static>(
/// evm: Evm,
/// ) {
/// fn init<Evm, Validator>(evm: Evm, validator: Validator)
/// where
/// Evm: ConfigureEvm<Header = Header, Transaction = TransactionSigned> + 'static,
/// Validator: PayloadValidator<Block = reth_primitives::Block> + 'static,
/// {
/// let mut registry = RpcModuleBuilder::default()
/// .with_provider(NoopProvider::default())
/// .with_pool(NoopTransactionPool::default())
@ -762,7 +777,7 @@ where
/// .with_evm_config(evm)
/// .with_block_executor(EthExecutorProvider::mainnet())
/// .with_consensus(NoopConsensus::default())
/// .into_registry(Default::default(), Box::new(EthApi::with_spawner));
/// .into_registry(Default::default(), Box::new(EthApi::with_spawner), Arc::new(validator));
///
/// let eth_api = registry.eth_api();
/// }
@ -771,6 +786,7 @@ where
self,
config: RpcModuleConfig,
eth: DynEthApiBuilder<Provider, Pool, EvmConfig, Network, Tasks, Events, EthApi>,
payload_validator: Arc<dyn PayloadValidator<Block = Provider::Block>>,
) -> RpcRegistryInner<Provider, Pool, Network, Tasks, Events, EthApi, BlockExecutor, Consensus>
where
EthApi: EthApiTypes + 'static,
@ -796,6 +812,7 @@ where
evm_config,
eth,
block_executor,
payload_validator,
)
}
@ -805,6 +822,7 @@ where
self,
module_config: TransportRpcModuleConfig,
eth: DynEthApiBuilder<Provider, Pool, EvmConfig, Network, Tasks, Events, EthApi>,
payload_validator: Arc<dyn PayloadValidator<Block = Provider::Block>>,
) -> TransportRpcModules<()>
where
EthApi: FullEthApiServer<
@ -843,6 +861,7 @@ where
evm_config,
eth,
block_executor,
payload_validator,
);
modules.config = module_config;
@ -957,6 +976,7 @@ pub struct RpcRegistryInner<
events: Events,
block_executor: BlockExecutor,
consensus: Consensus,
payload_validator: Arc<dyn PayloadValidator<Block = Provider::Block>>,
/// Holds the configuration for the RPC modules
config: RpcModuleConfig,
/// Holds a all `eth_` namespace handlers
@ -1008,9 +1028,10 @@ where
EthApi,
>,
block_executor: BlockExecutor,
payload_validator: Arc<dyn PayloadValidator<Block = Provider::Block>>,
) -> Self
where
EvmConfig: ConfigureEvm<Header = Header>,
EvmConfig: ConfigureEvm<Header = Provider::Header>,
{
let blocking_pool_guard = BlockingTaskGuard::new(config.eth.max_tracing_requests);
@ -1037,6 +1058,7 @@ where
blocking_pool_guard,
events,
block_executor,
payload_validator,
}
}
}
@ -1320,6 +1342,7 @@ where
pub fn validation_api(&self) -> ValidationApi<Provider, BlockExecutor>
where
Consensus: reth_consensus::FullConsensus<BlockExecutor::Primitives> + Clone + 'static,
Provider: BlockReader<Block = <BlockExecutor::Primitives as NodePrimitives>::Block>,
{
ValidationApi::new(
self.provider.clone(),
@ -1327,6 +1350,7 @@ where
self.block_executor.clone(),
self.config.flashbots.clone(),
Box::new(self.executor.clone()),
self.payload_validator.clone(),
)
}
}
@ -1334,7 +1358,9 @@ where
impl<Provider, Pool, Network, Tasks, Events, EthApi, BlockExecutor, Consensus>
RpcRegistryInner<Provider, Pool, Network, Tasks, Events, EthApi, BlockExecutor, Consensus>
where
Provider: FullRpcProvider + AccountReader + ChangeSetReader,
Provider: FullRpcProvider<Block = <BlockExecutor::Primitives as NodePrimitives>::Block>
+ AccountReader
+ ChangeSetReader,
Pool: TransactionPool + 'static,
Network: NetworkInfo + Peers + Clone + 'static,
Tasks: TaskSpawner + Clone + 'static,
@ -1346,12 +1372,7 @@ where
Header = <BlockExecutor::Primitives as NodePrimitives>::BlockHeader,
>,
>,
BlockExecutor: BlockExecutorProvider<
Primitives: NodePrimitives<
BlockHeader = reth_primitives::Header,
BlockBody = reth_primitives::BlockBody,
>,
>,
BlockExecutor: BlockExecutorProvider,
Consensus: reth_consensus::FullConsensus<BlockExecutor::Primitives> + Clone + 'static,
{
/// Configures the auth module that includes the
@ -1500,6 +1521,7 @@ where
self.block_executor.clone(),
self.config.flashbots.clone(),
Box::new(self.executor.clone()),
self.payload_validator.clone(),
)
.into_rpc()
.into(),

View File

@ -5,6 +5,8 @@ use jsonrpsee::{
types::Request,
MethodResponse,
};
use reth_chainspec::MAINNET;
use reth_ethereum_engine_primitives::EthereumEngineValidator;
use reth_rpc::EthApi;
use reth_rpc_builder::{RpcServerConfig, TransportRpcModuleConfig};
use reth_rpc_eth_api::EthApiClient;
@ -63,6 +65,7 @@ async fn test_rpc_middleware() {
let modules = builder.build(
TransportRpcModuleConfig::set_http(RpcModuleSelection::All),
Box::new(EthApi::with_spawner),
Arc::new(EthereumEngineValidator::new(MAINNET.clone())),
);
let mylayer = MyMiddlewareLayer::default();

View File

@ -1,7 +1,9 @@
//! Startup tests
use std::io;
use std::{io, sync::Arc};
use reth_chainspec::MAINNET;
use reth_ethereum_engine_primitives::EthereumEngineValidator;
use reth_rpc::EthApi;
use reth_rpc_builder::{
error::{RpcError, ServerKind, WsHttpSamePortError},
@ -30,6 +32,7 @@ async fn test_http_addr_in_use() {
let server = builder.build(
TransportRpcModuleConfig::set_http(vec![RethRpcModule::Admin]),
Box::new(EthApi::with_spawner),
Arc::new(EthereumEngineValidator::new(MAINNET.clone())),
);
let result =
RpcServerConfig::http(Default::default()).with_http_address(addr).start(&server).await;
@ -45,6 +48,7 @@ async fn test_ws_addr_in_use() {
let server = builder.build(
TransportRpcModuleConfig::set_ws(vec![RethRpcModule::Admin]),
Box::new(EthApi::with_spawner),
Arc::new(EthereumEngineValidator::new(MAINNET.clone())),
);
let result = RpcServerConfig::ws(Default::default()).with_ws_address(addr).start(&server).await;
let err = result.unwrap_err();
@ -66,6 +70,7 @@ async fn test_launch_same_port_different_modules() {
TransportRpcModuleConfig::set_ws(vec![RethRpcModule::Admin])
.with_http(vec![RethRpcModule::Eth]),
Box::new(EthApi::with_spawner),
Arc::new(EthereumEngineValidator::new(MAINNET.clone())),
);
let addr = test_address();
let res = RpcServerConfig::ws(Default::default())
@ -88,6 +93,7 @@ async fn test_launch_same_port_same_cors() {
TransportRpcModuleConfig::set_ws(vec![RethRpcModule::Eth])
.with_http(vec![RethRpcModule::Eth]),
Box::new(EthApi::with_spawner),
Arc::new(EthereumEngineValidator::new(MAINNET.clone())),
);
let addr = test_address();
let res = RpcServerConfig::ws(Default::default())
@ -108,6 +114,7 @@ async fn test_launch_same_port_different_cors() {
TransportRpcModuleConfig::set_ws(vec![RethRpcModule::Eth])
.with_http(vec![RethRpcModule::Eth]),
Box::new(EthApi::with_spawner),
Arc::new(EthereumEngineValidator::new(MAINNET.clone())),
);
let addr = test_address();
let res = RpcServerConfig::ws(Default::default())

View File

@ -1,4 +1,7 @@
use std::net::{Ipv4Addr, SocketAddr, SocketAddrV4};
use std::{
net::{Ipv4Addr, SocketAddr, SocketAddrV4},
sync::Arc,
};
use alloy_rpc_types_engine::{ClientCode, ClientVersionV1};
use reth_beacon_consensus::BeaconConsensusEngineHandle;
@ -61,8 +64,11 @@ pub async fn launch_auth(secret: JwtSecret) -> AuthServerHandle {
/// Launches a new server with http only with the given modules
pub async fn launch_http(modules: impl Into<RpcModuleSelection>) -> RpcServerHandle {
let builder = test_rpc_builder();
let server =
builder.build(TransportRpcModuleConfig::set_http(modules), Box::new(EthApi::with_spawner));
let server = builder.build(
TransportRpcModuleConfig::set_http(modules),
Box::new(EthApi::with_spawner),
Arc::new(EthereumEngineValidator::new(MAINNET.clone())),
);
RpcServerConfig::http(Default::default())
.with_http_address(test_address())
.start(&server)
@ -73,8 +79,11 @@ pub async fn launch_http(modules: impl Into<RpcModuleSelection>) -> RpcServerHan
/// Launches a new server with ws only with the given modules
pub async fn launch_ws(modules: impl Into<RpcModuleSelection>) -> RpcServerHandle {
let builder = test_rpc_builder();
let server =
builder.build(TransportRpcModuleConfig::set_ws(modules), Box::new(EthApi::with_spawner));
let server = builder.build(
TransportRpcModuleConfig::set_ws(modules),
Box::new(EthApi::with_spawner),
Arc::new(EthereumEngineValidator::new(MAINNET.clone())),
);
RpcServerConfig::ws(Default::default())
.with_ws_address(test_address())
.start(&server)
@ -89,6 +98,7 @@ pub async fn launch_http_ws(modules: impl Into<RpcModuleSelection>) -> RpcServer
let server = builder.build(
TransportRpcModuleConfig::set_ws(modules.clone()).with_http(modules),
Box::new(EthApi::with_spawner),
Arc::new(EthereumEngineValidator::new(MAINNET.clone())),
);
RpcServerConfig::ws(Default::default())
.with_ws_address(test_address())
@ -107,6 +117,7 @@ pub async fn launch_http_ws_same_port(modules: impl Into<RpcModuleSelection>) ->
let server = builder.build(
TransportRpcModuleConfig::set_ws(modules.clone()).with_http(modules),
Box::new(EthApi::with_spawner),
Arc::new(EthereumEngineValidator::new(MAINNET.clone())),
);
let addr = test_address();
RpcServerConfig::ws(Default::default())

View File

@ -25,7 +25,7 @@ use reth_payload_primitives::{
validate_payload_timestamp, EngineApiMessageVersion, PayloadBuilderAttributes,
PayloadOrAttributes,
};
use reth_primitives::{Block, EthereumHardfork};
use reth_primitives::EthereumHardfork;
use reth_rpc_api::EngineApiServer;
use reth_rpc_types_compat::engine::payload::{
convert_payload_input_v2_to_payload, convert_to_payload_body_v1,
@ -80,11 +80,7 @@ struct EngineApiInner<Provider, EngineT: EngineTypes, Pool, Validator, ChainSpec
impl<Provider, EngineT, Pool, Validator, ChainSpec>
EngineApi<Provider, EngineT, Pool, Validator, ChainSpec>
where
Provider: HeaderProvider
+ BlockReader<Block = reth_primitives::Block>
+ StateProviderFactory
+ EvmEnvProvider
+ 'static,
Provider: HeaderProvider + BlockReader + StateProviderFactory + EvmEnvProvider + 'static,
EngineT: EngineTypes,
Pool: TransactionPool + 'static,
Validator: EngineValidator<EngineT>,
@ -573,7 +569,7 @@ where
f: F,
) -> EngineApiResult<Vec<Option<R>>>
where
F: Fn(Block) -> R + Send + 'static,
F: Fn(Provider::Block) -> R + Send + 'static,
R: Send + 'static,
{
let len = hashes.len() as u64;
@ -748,11 +744,7 @@ where
impl<Provider, EngineT, Pool, Validator, ChainSpec> EngineApiServer<EngineT>
for EngineApi<Provider, EngineT, Pool, Validator, ChainSpec>
where
Provider: HeaderProvider
+ BlockReader<Block = reth_primitives::Block>
+ StateProviderFactory
+ EvmEnvProvider
+ 'static,
Provider: HeaderProvider + BlockReader + StateProviderFactory + EvmEnvProvider + 'static,
EngineT: EngineTypes,
Pool: TransactionPool + 'static,
Validator: EngineValidator<EngineT>,
@ -1045,7 +1037,7 @@ mod tests {
use reth_engine_primitives::BeaconEngineMessage;
use reth_ethereum_engine_primitives::{EthEngineTypes, EthereumEngineValidator};
use reth_payload_builder::test_utils::spawn_test_payload_service;
use reth_primitives::SealedBlock;
use reth_primitives::{Block, SealedBlock};
use reth_provider::test_utils::MockEthProvider;
use reth_rpc_types_compat::engine::payload::execution_payload_from_sealed_block;
use reth_tasks::TokioTaskExecutor;
@ -1171,7 +1163,7 @@ mod tests {
let expected = blocks
.iter()
.cloned()
.map(|b| Some(convert_to_payload_body_v1(b.unseal())))
.map(|b| Some(convert_to_payload_body_v1(b.unseal::<Block>())))
.collect::<Vec<_>>();
let res = api.get_payload_bodies_by_range_v1(start, count).await.unwrap();
@ -1213,7 +1205,7 @@ mod tests {
if first_missing_range.contains(&b.number) {
None
} else {
Some(convert_to_payload_body_v1(b.unseal()))
Some(convert_to_payload_body_v1(b.unseal::<Block>()))
}
})
.collect::<Vec<_>>();
@ -1232,7 +1224,7 @@ mod tests {
{
None
} else {
Some(convert_to_payload_body_v1(b.unseal()))
Some(convert_to_payload_body_v1(b.unseal::<Block>()))
}
})
.collect::<Vec<_>>();

View File

@ -38,7 +38,7 @@ fn payload_body_roundtrip() {
0..=99,
BlockRangeParams { tx_count: 0..2, ..Default::default() },
) {
let unsealed = block.clone().unseal();
let unsealed = block.clone().unseal::<Block>();
let payload_body: ExecutionPayloadBodyV1 = convert_to_payload_body_v1(unsealed);
assert_eq!(

View File

@ -16,6 +16,7 @@ use reth_primitives::{
proofs::{self},
Block, BlockBody, BlockExt, SealedBlock, TransactionSigned,
};
use reth_primitives_traits::BlockBody as _;
/// Converts [`ExecutionPayloadV1`] to [`Block`]
pub fn try_payload_v1_to_block(payload: ExecutionPayloadV1) -> Result<Block, PayloadError> {
@ -320,15 +321,13 @@ pub fn validate_block_hash(
}
/// Converts [`Block`] to [`ExecutionPayloadBodyV1`]
pub fn convert_to_payload_body_v1(value: Block) -> ExecutionPayloadBodyV1 {
let transactions = value.body.transactions.into_iter().map(|tx| {
let mut out = Vec::new();
tx.encode_2718(&mut out);
out.into()
});
pub fn convert_to_payload_body_v1(
value: impl reth_primitives_traits::Block,
) -> ExecutionPayloadBodyV1 {
let transactions = value.body().transactions().iter().map(|tx| tx.encoded_2718().into());
ExecutionPayloadBodyV1 {
transactions: transactions.collect(),
withdrawals: value.body.withdrawals.map(Withdrawals::into_inner),
withdrawals: value.body().withdrawals().cloned().map(Withdrawals::into_inner),
}
}

View File

@ -18,6 +18,7 @@ reth-primitives = { workspace = true, features = ["secp256k1"] }
reth-primitives-traits.workspace = true
reth-rpc-api.workspace = true
reth-rpc-eth-api.workspace = true
reth-engine-primitives.workspace = true
reth-errors.workspace = true
reth-ethereum-consensus.workspace = true
reth-provider.workspace = true
@ -35,7 +36,6 @@ reth-rpc-eth-types.workspace = true
reth-rpc-server-types.workspace = true
reth-network-types.workspace = true
reth-consensus.workspace = true
reth-payload-validator.workspace = true
# ethereum
alloy-consensus.workspace = true

View File

@ -14,10 +14,10 @@ use async_trait::async_trait;
use jsonrpsee::core::RpcResult;
use reth_chainspec::{ChainSpecProvider, EthereumHardforks};
use reth_consensus::{Consensus, FullConsensus, PostExecutionInput};
use reth_engine_primitives::PayloadValidator;
use reth_errors::{BlockExecutionError, ConsensusError, ProviderError};
use reth_ethereum_consensus::GAS_LIMIT_BOUND_DIVISOR;
use reth_evm::execute::{BlockExecutorProvider, Executor};
use reth_payload_validator::ExecutionPayloadValidator;
use reth_primitives::{GotExpected, NodePrimitives, SealedBlockWithSenders, SealedHeader};
use reth_primitives_traits::{Block as _, BlockBody};
use reth_provider::{
@ -34,14 +34,13 @@ use tokio::sync::{oneshot, RwLock};
/// The type that implements the `validation` rpc namespace trait
#[derive(Clone, Debug, derive_more::Deref)]
pub struct ValidationApi<Provider: ChainSpecProvider, E: BlockExecutorProvider> {
pub struct ValidationApi<Provider, E: BlockExecutorProvider> {
#[deref]
inner: Arc<ValidationApiInner<Provider, E>>,
}
impl<Provider, E> ValidationApi<Provider, E>
where
Provider: ChainSpecProvider,
E: BlockExecutorProvider,
{
/// Create a new instance of the [`ValidationApi`]
@ -51,10 +50,12 @@ where
executor_provider: E,
config: ValidationApiConfig,
task_spawner: Box<dyn TaskSpawner>,
payload_validator: Arc<
dyn PayloadValidator<Block = <E::Primitives as NodePrimitives>::Block>,
>,
) -> Self {
let ValidationApiConfig { disallow } = config;
let payload_validator = ExecutionPayloadValidator::new(provider.chain_spec());
let inner = Arc::new(ValidationApiInner {
provider,
consensus,
@ -91,16 +92,11 @@ where
impl<Provider, E> ValidationApi<Provider, E>
where
Provider: BlockReaderIdExt<Header = reth_primitives::Header>
Provider: BlockReaderIdExt<Header = <E::Primitives as NodePrimitives>::BlockHeader>
+ ChainSpecProvider<ChainSpec: EthereumHardforks>
+ StateProviderFactory
+ 'static,
E: BlockExecutorProvider<
Primitives: NodePrimitives<
BlockHeader = Provider::Header,
BlockBody = reth_primitives::BlockBody,
>,
>,
E: BlockExecutorProvider,
{
/// Validates the given block and a [`BidTrace`] against it.
pub async fn validate_message_against_block(
@ -116,8 +112,8 @@ where
self.consensus.validate_block_pre_execution(&block)?;
if !self.disallow.is_empty() {
if self.disallow.contains(&block.beneficiary) {
return Err(ValidationApiError::Blacklist(block.beneficiary))
if self.disallow.contains(&block.beneficiary()) {
return Err(ValidationApiError::Blacklist(block.beneficiary()))
}
if self.disallow.contains(&message.proposer_fee_recipient) {
return Err(ValidationApiError::Blacklist(message.proposer_fee_recipient))
@ -137,9 +133,9 @@ where
let latest_header =
self.provider.latest_header()?.ok_or_else(|| ValidationApiError::MissingLatestBlock)?;
if latest_header.hash() != block.header.parent_hash {
if latest_header.hash() != block.header.parent_hash() {
return Err(ConsensusError::ParentHashMismatch(
GotExpected { got: block.header.parent_hash, expected: latest_header.hash() }
GotExpected { got: block.header.parent_hash(), expected: latest_header.hash() }
.into(),
)
.into())
@ -200,7 +196,7 @@ where
/// Ensures that fields of [`BidTrace`] match the fields of the [`SealedHeader`].
fn validate_message_against_header(
&self,
header: &SealedHeader,
header: &SealedHeader<<E::Primitives as NodePrimitives>::BlockHeader>,
message: &BidTrace,
) -> Result<(), ValidationApiError> {
if header.hash() != message.block_hash {
@ -208,20 +204,20 @@ where
got: message.block_hash,
expected: header.hash(),
}))
} else if header.parent_hash != message.parent_hash {
} else if header.parent_hash() != message.parent_hash {
Err(ValidationApiError::ParentHashMismatch(GotExpected {
got: message.parent_hash,
expected: header.parent_hash,
expected: header.parent_hash(),
}))
} else if header.gas_limit != message.gas_limit {
} else if header.gas_limit() != message.gas_limit {
Err(ValidationApiError::GasLimitMismatch(GotExpected {
got: message.gas_limit,
expected: header.gas_limit,
expected: header.gas_limit(),
}))
} else if header.gas_used != message.gas_used {
} else if header.gas_used() != message.gas_used {
return Err(ValidationApiError::GasUsedMismatch(GotExpected {
got: message.gas_used,
expected: header.gas_used,
expected: header.gas_used(),
}))
} else {
Ok(())
@ -235,20 +231,20 @@ where
fn validate_gas_limit(
&self,
registered_gas_limit: u64,
parent_header: &SealedHeader,
header: &SealedHeader,
parent_header: &SealedHeader<<E::Primitives as NodePrimitives>::BlockHeader>,
header: &SealedHeader<<E::Primitives as NodePrimitives>::BlockHeader>,
) -> Result<(), ValidationApiError> {
let max_gas_limit =
parent_header.gas_limit + parent_header.gas_limit / GAS_LIMIT_BOUND_DIVISOR - 1;
parent_header.gas_limit() + parent_header.gas_limit() / GAS_LIMIT_BOUND_DIVISOR - 1;
let min_gas_limit =
parent_header.gas_limit - parent_header.gas_limit / GAS_LIMIT_BOUND_DIVISOR + 1;
parent_header.gas_limit() - parent_header.gas_limit() / GAS_LIMIT_BOUND_DIVISOR + 1;
let best_gas_limit =
std::cmp::max(min_gas_limit, std::cmp::min(max_gas_limit, registered_gas_limit));
if best_gas_limit != header.gas_limit {
if best_gas_limit != header.gas_limit() {
return Err(ValidationApiError::GasLimitMismatch(GotExpected {
got: header.gas_limit,
got: header.gas_limit(),
expected: best_gas_limit,
}))
}
@ -409,17 +405,12 @@ where
#[async_trait]
impl<Provider, E> BlockSubmissionValidationApiServer for ValidationApi<Provider, E>
where
Provider: BlockReaderIdExt<Header = reth_primitives::Header>
Provider: BlockReaderIdExt<Header = <E::Primitives as NodePrimitives>::BlockHeader>
+ ChainSpecProvider<ChainSpec: EthereumHardforks>
+ StateProviderFactory
+ Clone
+ 'static,
E: BlockExecutorProvider<
Primitives: NodePrimitives<
BlockHeader = Provider::Header,
BlockBody = reth_primitives::BlockBody,
>,
>,
E: BlockExecutorProvider,
{
async fn validate_builder_submission_v1(
&self,
@ -473,13 +464,13 @@ where
}
#[derive(Debug)]
pub struct ValidationApiInner<Provider: ChainSpecProvider, E: BlockExecutorProvider> {
pub struct ValidationApiInner<Provider, E: BlockExecutorProvider> {
/// The provider that can interact with the chain.
provider: Provider,
/// Consensus implementation.
consensus: Arc<dyn FullConsensus<E::Primitives>>,
/// Execution payload validator.
payload_validator: ExecutionPayloadValidator<Provider::ChainSpec>,
payload_validator: Arc<dyn PayloadValidator<Block = <E::Primitives as NodePrimitives>::Block>>,
/// Block executor factory.
executor_provider: E,
/// Set of disallowed addresses