feat(sync): beacon consensus engine (#1845)

This commit is contained in:
Roman Krasiuk
2023-03-23 20:18:19 +02:00
committed by GitHub
parent 84af91737d
commit ce40bea46e
27 changed files with 1918 additions and 1141 deletions

View File

@ -8,6 +8,7 @@ description = "Helpers for configuring RPC"
[dependencies]
# reth
reth-primitives = { path = "../../primitives" }
reth-ipc = { path = "../ipc" }
reth-interfaces = { path = "../../interfaces" }
reth-network-api = { path = "../../net/network-api" }
@ -33,7 +34,6 @@ tracing = "0.1"
[dev-dependencies]
reth-tracing = { path = "../../tracing" }
reth-primitives = { path = "../../primitives" }
reth-rpc-api = { path = "../rpc-api", features = ["client"] }
reth-transaction-pool = { path = "../../transaction-pool", features = ["test-utils"] }
reth-provider = { path = "../../storage/provider", features = ["test-utils"] }

View File

@ -1,6 +1,7 @@
pub use jsonrpsee::server::ServerBuilder;
use jsonrpsee::{core::Error as RpcError, server::ServerHandle, RpcModule};
use reth_network_api::{NetworkInfo, Peers};
use reth_primitives::ChainSpec;
use reth_provider::{BlockProvider, EvmEnvProvider, HeaderProvider, StateProviderFactory};
use reth_rpc::{
eth::cache::EthStateCache, AuthLayer, EngineApi, EthApi, JwtAuthValidator, JwtSecret,
@ -9,14 +10,16 @@ use reth_rpc_api::servers::*;
use reth_rpc_engine_api::EngineApiHandle;
use reth_tasks::TaskSpawner;
use reth_transaction_pool::TransactionPool;
use std::net::SocketAddr;
use std::{net::SocketAddr, sync::Arc};
/// Configure and launch an auth server with `engine` and a _new_ `eth` namespace.
#[allow(clippy::too_many_arguments)]
pub async fn launch<Client, Pool, Network, Tasks>(
client: Client,
pool: Pool,
network: Network,
executor: Tasks,
chain_spec: Arc<ChainSpec>,
handle: EngineApiHandle,
socket_addr: SocketAddr,
secret: JwtSecret,
@ -35,13 +38,20 @@ where
{
// spawn a new cache task
let eth_cache = EthStateCache::spawn_with(client.clone(), Default::default(), executor);
launch_with_eth_api(EthApi::new(client, pool, network, eth_cache), handle, socket_addr, secret)
.await
launch_with_eth_api(
EthApi::new(client, pool, network, eth_cache),
chain_spec,
handle,
socket_addr,
secret,
)
.await
}
/// Configure and launch an auth server with existing EthApi implementation.
pub async fn launch_with_eth_api<Client, Pool, Network>(
eth_api: EthApi<Client, Pool, Network>,
chain_spec: Arc<ChainSpec>,
handle: EngineApiHandle,
socket_addr: SocketAddr,
secret: JwtSecret,
@ -59,7 +69,7 @@ where
{
// Configure the module and start the server.
let mut module = RpcModule::new(());
module.merge(EngineApi::new(handle).into_rpc()).expect("No conflicting methods");
module.merge(EngineApi::new(chain_spec, handle).into_rpc()).expect("No conflicting methods");
module.merge(eth_api.into_rpc()).expect("No conflicting methods");
// Create auth middleware.

View File

@ -11,10 +11,10 @@ description = "Implementation of Engine API"
reth-primitives = { path = "../../primitives" }
reth-interfaces = { path = "../../interfaces" }
reth-provider = { path = "../../storage/provider" }
reth-rlp = { path = "../../rlp" }
reth-executor = { path = "../../executor" }
reth-revm = { path = "../../revm" }
reth-rpc-types = { path = "../rpc-types" }
reth-beacon-consensus = { path = "../../consensus/beacon" }
# async
futures = "0.3"

View File

@ -1,27 +1,19 @@
use crate::{message::EngineApiMessageVersion, EngineApiError, EngineApiMessage, EngineApiResult};
use crate::{EngineApiError, EngineApiMessage, EngineApiResult};
use futures::StreamExt;
use reth_interfaces::consensus::ForkchoiceState;
use reth_primitives::{
proofs::{self, EMPTY_LIST_HASH},
BlockHash, BlockId, BlockNumber, ChainSpec, Hardfork, Header, SealedBlock, TransactionSigned,
H256, H64, U256,
};
use reth_provider::{
BlockExecutor, BlockProvider, EvmEnvProvider, ExecutorFactory, HeaderProvider,
StateProviderFactory,
};
use reth_rlp::Decodable;
use reth_rpc_types::engine::{
ExecutionPayload, ExecutionPayloadBodies, ForkchoiceUpdated, PayloadAttributes, PayloadStatus,
PayloadStatusEnum, TransitionConfiguration,
};
use reth_beacon_consensus::BeaconEngineMessage;
use reth_primitives::{BlockHash, BlockId, BlockNumber, ChainSpec, Hardfork, H64};
use reth_provider::{BlockProvider, EvmEnvProvider, HeaderProvider, StateProviderFactory};
use reth_rpc_types::engine::{ExecutionPayload, ExecutionPayloadBodies, TransitionConfiguration};
use std::{
future::Future,
pin::Pin,
sync::Arc,
task::{ready, Context, Poll},
};
use tokio::sync::{mpsc, oneshot, watch};
use tokio::sync::{
mpsc::{self, UnboundedSender},
oneshot,
};
use tokio_stream::wrappers::UnboundedReceiverStream;
/// The Engine API handle.
@ -41,10 +33,7 @@ pub struct EngineApi<Client> {
/// Consensus configuration
chain_spec: Arc<ChainSpec>,
message_rx: UnboundedReceiverStream<EngineApiMessage>,
forkchoice_state_tx: watch::Sender<ForkchoiceState>,
// TODO: Placeholder for storing future blocks. Make cache bounded. Use lru
// local_store: HashMap<H64, ExecutionPayload>,
// remote_store: HashMap<H64, ExecutionPayload>,
engine_tx: UnboundedSender<BeaconEngineMessage>,
}
impl<Client: HeaderProvider + BlockProvider + StateProviderFactory + EvmEnvProvider>
@ -55,18 +44,16 @@ impl<Client: HeaderProvider + BlockProvider + StateProviderFactory + EvmEnvProvi
client: Client,
chain_spec: Arc<ChainSpec>,
message_rx: mpsc::UnboundedReceiver<EngineApiMessage>,
forkchoice_state_tx: watch::Sender<ForkchoiceState>,
engine_tx: UnboundedSender<BeaconEngineMessage>,
) -> Self {
Self {
client,
chain_spec,
message_rx: UnboundedReceiverStream::new(message_rx),
forkchoice_state_tx,
}
Self { client, chain_spec, message_rx: UnboundedReceiverStream::new(message_rx), engine_tx }
}
fn on_message(&mut self, msg: EngineApiMessage) {
match msg {
EngineApiMessage::ExchangeTransitionConfiguration(config, tx) => {
let _ = tx.send(self.exchange_transition_configuration(config));
}
EngineApiMessage::GetPayload(payload_id, tx) => {
let _ = tx.send(self.get_payload(payload_id).ok_or(EngineApiError::PayloadUnknown));
}
@ -76,130 +63,21 @@ impl<Client: HeaderProvider + BlockProvider + StateProviderFactory + EvmEnvProvi
EngineApiMessage::GetPayloadBodiesByRange(start, count, tx) => {
let _ = tx.send(self.get_payload_bodies_by_range(start, count));
}
EngineApiMessage::NewPayload(version, payload, tx) => {
if let Err(err) = self.validate_withdrawals_presence(
version,
payload.timestamp.as_u64(),
payload.withdrawals.is_some(),
) {
let _ = tx.send(Err(err));
return
}
let _ = tx.send(self.new_payload(payload));
EngineApiMessage::NewPayload(payload, tx) => {
// forward message to the consensus engine
let _ = self.engine_tx.send(BeaconEngineMessage::NewPayload { payload, tx });
}
EngineApiMessage::ForkchoiceUpdated(version, state, attrs, tx) => {
if let Some(attributes) = &attrs {
if let Err(err) = self.validate_withdrawals_presence(
version,
attributes.timestamp.as_u64(),
attributes.withdrawals.is_some(),
) {
let _ = tx.send(Err(err));
return
}
}
let _ = tx.send(self.fork_choice_updated(state, attrs));
}
EngineApiMessage::ExchangeTransitionConfiguration(config, tx) => {
let _ = tx.send(self.exchange_transition_configuration(config));
EngineApiMessage::ForkchoiceUpdated(state, payload_attrs, tx) => {
// forward message to the consensus engine
let _ = self.engine_tx.send(BeaconEngineMessage::ForkchoiceUpdated {
state,
payload_attrs,
tx,
});
}
}
}
/// Validates the presence of the `withdrawals` field according to the payload timestamp.
/// After Shanghai, withdrawals field must be [Some].
/// Before Shanghai, withdrawals field must be [None];
fn validate_withdrawals_presence(
&self,
version: EngineApiMessageVersion,
timestamp: u64,
has_withdrawals: bool,
) -> EngineApiResult<()> {
let is_shanghai = self.chain_spec.fork(Hardfork::Shanghai).active_at_timestamp(timestamp);
match version {
EngineApiMessageVersion::V1 => {
if is_shanghai || has_withdrawals {
return Err(EngineApiError::InvalidParams)
}
}
EngineApiMessageVersion::V2 => {
let shanghai_with_no_withdrawals = is_shanghai && !has_withdrawals;
let not_shanghai_with_withdrawals = !is_shanghai && has_withdrawals;
if shanghai_with_no_withdrawals || not_shanghai_with_withdrawals {
return Err(EngineApiError::InvalidParams)
}
}
};
Ok(())
}
/// Try to construct a block from given payload. Perform addition validation of `extra_data` and
/// `base_fee_per_gas` fields.
///
/// NOTE: The log bloom is assumed to be validated during serialization.
/// NOTE: Empty ommers, nonce and difficulty values are validated upon computing block hash and
/// comparing the value with `payload.block_hash`.
///
/// See <https://github.com/ethereum/go-ethereum/blob/79a478bb6176425c2400e949890e668a3d9a3d05/core/beacon/types.go#L145>
fn try_construct_block(&self, payload: ExecutionPayload) -> EngineApiResult<SealedBlock> {
if payload.extra_data.len() > 32 {
return Err(EngineApiError::PayloadExtraData(payload.extra_data))
}
if payload.base_fee_per_gas == U256::ZERO {
return Err(EngineApiError::PayloadBaseFee(payload.base_fee_per_gas))
}
let transactions = payload
.transactions
.iter()
.map(|tx| TransactionSigned::decode(&mut tx.as_ref()))
.collect::<std::result::Result<Vec<_>, _>>()?;
let transactions_root = proofs::calculate_transaction_root(transactions.iter());
let withdrawals_root =
payload.withdrawals.as_ref().map(|w| proofs::calculate_withdrawals_root(w.iter()));
let header = Header {
parent_hash: payload.parent_hash,
beneficiary: payload.fee_recipient,
state_root: payload.state_root,
transactions_root,
receipts_root: payload.receipts_root,
withdrawals_root,
logs_bloom: payload.logs_bloom,
number: payload.block_number.as_u64(),
gas_limit: payload.gas_limit.as_u64(),
gas_used: payload.gas_used.as_u64(),
timestamp: payload.timestamp.as_u64(),
mix_hash: payload.prev_randao,
base_fee_per_gas: Some(payload.base_fee_per_gas.to::<u64>()),
extra_data: payload.extra_data,
// Defaults
ommers_hash: EMPTY_LIST_HASH,
difficulty: Default::default(),
nonce: Default::default(),
}
.seal_slow();
if payload.block_hash != header.hash() {
return Err(EngineApiError::PayloadBlockHash {
execution: header.hash(),
consensus: payload.block_hash,
})
}
Ok(SealedBlock {
header,
body: transactions,
withdrawals: payload.withdrawals,
ommers: Default::default(),
})
}
/// Called to retrieve the latest state of the network, validate new blocks, and maintain
/// consistency between the Consensus and Execution layers.
///
@ -225,7 +103,10 @@ impl<Client: HeaderProvider + BlockProvider + StateProviderFactory + EvmEnvProvi
let mut result = Vec::with_capacity(count as usize);
for num in start..start + count {
let block = self.client.block(BlockId::Number(num.into()))?;
let block = self
.client
.block(BlockId::Number(num.into()))
.map_err(|err| EngineApiError::Internal(Box::new(err)))?;
result.push(block.map(Into::into));
}
@ -244,181 +125,16 @@ impl<Client: HeaderProvider + BlockProvider + StateProviderFactory + EvmEnvProvi
let mut result = Vec::with_capacity(hashes.len());
for hash in hashes {
let block = self.client.block(BlockId::Hash(hash.into()))?;
let block = self
.client
.block(BlockId::Hash(hash.into()))
.map_err(|err| EngineApiError::Internal(Box::new(err)))?;
result.push(block.map(Into::into));
}
Ok(result)
}
/// When the Consensus layer receives a new block via the consensus gossip protocol,
/// the transactions in the block are sent to the execution layer in the form of a
/// `ExecutionPayload`. The Execution layer executes the transactions and validates the
/// state in the block header, then passes validation data back to Consensus layer, that
/// adds the block to the head of its own blockchain and attests to it. The block is then
/// broadcasted over the consensus p2p network in the form of a "Beacon block".
///
/// These responses should adhere to the [Engine API Spec for
/// `engine_newPayload`](https://github.com/ethereum/execution-apis/blob/main/src/engine/paris.md#specification).
pub fn new_payload(&mut self, payload: ExecutionPayload) -> EngineApiResult<PayloadStatus> {
let block = match self.try_construct_block(payload) {
Ok(b) => b,
Err(err) => {
return Ok(PayloadStatus::from_status(PayloadStatusEnum::InvalidBlockHash {
validation_error: err.to_string(),
}))
}
};
let block_hash = block.header.hash();
let parent_hash = block.parent_hash;
// The block already exists in our database
if self.client.is_known(&block_hash)? {
return Ok(PayloadStatus::new(PayloadStatusEnum::Valid, block_hash))
}
let Some(parent) = self.client.block_by_hash(parent_hash)? else {
// TODO: cache block for storing later
return Ok(PayloadStatus::from_status(PayloadStatusEnum::Syncing))
};
let Some(parent_td) = self.client.header_td(&block.parent_hash)? else { return Ok(PayloadStatus::from_status(PayloadStatusEnum::Invalid {
validation_error: EngineApiError::PayloadPreMerge.to_string(),
})) };
// Short circuit the check by passing parent total difficulty.
if !self.chain_spec.fork(Hardfork::Paris).active_at_ttd(parent_td, U256::ZERO) {
// This case returns a `latestValidHash` of zero because it is required by the engine
// api spec:
//
// Client software MUST respond to this method call in the following way:
// {
// status: INVALID,
// latestValidHash:
// 0x0000000000000000000000000000000000000000000000000000000000000000,
// validationError: errorMessage | null
// }
//
// if terminal block conditions are not satisfied
return Ok(PayloadStatus::from_status(PayloadStatusEnum::Invalid {
validation_error: EngineApiError::PayloadPreMerge.to_string(),
})
.with_latest_valid_hash(H256::zero()))
}
if block.timestamp <= parent.timestamp {
return Ok(PayloadStatus::from_status(PayloadStatusEnum::Invalid {
validation_error: EngineApiError::PayloadTimestamp {
invalid: block.timestamp,
latest: parent.timestamp,
}
.to_string(),
}))
}
let state_provider = self.client.latest()?;
let total_difficulty = parent_td + block.header.difficulty;
let factory = reth_executor::Factory::new(self.chain_spec.clone());
let mut executor = factory.with_sp(&state_provider);
match executor.execute_and_verify_receipt(&block.unseal(), total_difficulty, None) {
Ok(_) => Ok(PayloadStatus::new(PayloadStatusEnum::Valid, block_hash)),
Err(err) => Ok(PayloadStatus::new(
PayloadStatusEnum::Invalid { validation_error: err.to_string() },
parent_hash, // The parent hash is already in our database hence it is valid
)),
}
}
/// Called to resolve chain forks and ensure that the Execution layer is working with the latest
/// valid chain.
///
/// These responses should adhere to the [Engine API Spec for
/// `engine_forkchoiceUpdated`](https://github.com/ethereum/execution-apis/blob/main/src/engine/paris.md#specification-1).
pub fn fork_choice_updated(
&self,
fork_choice_state: ForkchoiceState,
payload_attributes: Option<PayloadAttributes>,
) -> EngineApiResult<ForkchoiceUpdated> {
let ForkchoiceState { head_block_hash, finalized_block_hash, .. } = fork_choice_state;
if head_block_hash.is_zero() {
return Ok(ForkchoiceUpdated::from_status(PayloadStatusEnum::Invalid {
validation_error: EngineApiError::ForkchoiceEmptyHead.to_string(),
}))
}
let Some(head) = self.client.header(&head_block_hash)? else {
// Block is not known, nothing to do
return Ok(ForkchoiceUpdated::from_status(PayloadStatusEnum::Syncing))
};
// The finalized block hash is not known, we are still syncing
if !finalized_block_hash.is_zero() && !self.client.is_known(&finalized_block_hash)? {
return Ok(ForkchoiceUpdated::from_status(PayloadStatusEnum::Syncing))
}
let Some(head_td) = self.client.header_td(&head_block_hash)? else {
// internal error - we have the head block but not the total difficulty
return Ok(ForkchoiceUpdated::from_status(PayloadStatusEnum::Invalid {
validation_error: EngineApiError::Internal(
reth_interfaces::provider::ProviderError::TotalDifficulty {
number: head.number,
}
.into(),
)
.to_string(),
}))
};
// From the Engine API spec:
//
// If forkchoiceState.headBlockHash references a PoW block, client software MUST validate
// this block with respect to terminal block conditions according to EIP-3675. This check
// maps to the transition block validity section of the EIP. Additionally, if this
// validation fails, client software MUST NOT update the forkchoice state and MUST NOT
// begin a payload build process.
//
// We use ZERO here because as long as the total difficulty is above the ttd, we are sure
// that the block is EITHER:
// * The terminal PoW block, or
// * A child of the terminal PoW block
//
// Using the head.difficulty instead of U256::ZERO here would be incorrect because it would
// not return true on the terminal PoW block. For the terminal PoW block, head_td -
// head.difficulty would be less than the TTD, causing active_at_ttd to return false.
if !self.chain_spec.fork(Hardfork::Paris).active_at_ttd(head_td, U256::ZERO) {
// This case returns a `latestValidHash` of zero because it is required by the engine
// api spec:
//
// Client software MUST respond to this method call in the following way:
// {
// status: INVALID,
// latestValidHash:
// 0x0000000000000000000000000000000000000000000000000000000000000000,
// validationError: errorMessage | null
// }
// obtained either from the Payload validation process or as a result of validating a
// terminal PoW block referenced by forkchoiceState.headBlockHash
return Ok(ForkchoiceUpdated::from_status(PayloadStatusEnum::Invalid {
validation_error: EngineApiError::PayloadPreMerge.to_string(),
})
.with_latest_valid_hash(H256::zero()))
}
if let Err(error) = self.forkchoice_state_tx.send(fork_choice_state) {
tracing::error!(target: "rpc::engine_api", ?error, "Failed to update forkchoice state");
}
if let Some(_attr) = payload_attributes {
// TODO: optionally build the block
}
let chain_info = self.client.chain_info()?;
Ok(ForkchoiceUpdated::from_status(PayloadStatusEnum::Valid)
.with_latest_valid_hash(chain_info.best_hash))
}
/// Called to verify network configuration parameters and ensure that Consensus and Execution
/// layers are using the latest configuration.
pub fn exchange_transition_configuration(
@ -454,7 +170,10 @@ impl<Client: HeaderProvider + BlockProvider + StateProviderFactory + EvmEnvProvi
}
// Attempt to look up terminal block hash
let local_hash = self.client.block_hash(terminal_block_number.as_u64())?;
let local_hash = self
.client
.block_hash(terminal_block_number.as_u64())
.map_err(|err| EngineApiError::Internal(Box::new(err)))?;
// Transition configuration exchange is successful if block hashes match
match local_hash {
@ -495,27 +214,24 @@ where
mod tests {
use super::*;
use assert_matches::assert_matches;
use reth_interfaces::test_utils::generators::random_block;
use reth_primitives::{H256, MAINNET};
use reth_interfaces::{consensus::ForkchoiceState, test_utils::generators::random_block};
use reth_primitives::{SealedBlock, H256, MAINNET};
use reth_provider::test_utils::MockEthProvider;
use std::sync::Arc;
use tokio::sync::{
mpsc::{unbounded_channel, UnboundedSender},
watch::Receiver as WatchReceiver,
};
use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver, UnboundedSender};
fn setup_engine_api() -> (EngineApiTestHandle, EngineApi<Arc<MockEthProvider>>) {
let chain_spec = Arc::new(MAINNET.clone());
let client = Arc::new(MockEthProvider::default());
let (msg_tx, msg_rx) = unbounded_channel();
let (forkchoice_state_tx, forkchoice_state_rx) = watch::channel(ForkchoiceState::default());
let (engine_tx, engine_rx) = mpsc::unbounded_channel();
let api = EngineApi {
client: client.clone(),
chain_spec: chain_spec.clone(),
message_rx: UnboundedReceiverStream::new(msg_rx),
forkchoice_state_tx,
engine_tx,
};
let handle = EngineApiTestHandle { chain_spec, client, msg_tx, forkchoice_state_rx };
let handle = EngineApiTestHandle { chain_spec, client, msg_tx, engine_rx };
(handle, api)
}
@ -523,254 +239,37 @@ mod tests {
chain_spec: Arc<ChainSpec>,
client: Arc<MockEthProvider>,
msg_tx: UnboundedSender<EngineApiMessage>,
forkchoice_state_rx: WatchReceiver<ForkchoiceState>,
engine_rx: UnboundedReceiver<BeaconEngineMessage>,
}
impl EngineApiTestHandle {
fn send_message(&self, msg: EngineApiMessage) {
self.msg_tx.send(msg).expect("failed to send engine msg");
}
fn forkchoice_state(&self) -> ForkchoiceState {
self.forkchoice_state_rx.borrow().clone()
}
fn forkchoice_state_has_changed(&self) -> bool {
self.forkchoice_state_rx.has_changed().unwrap()
}
}
mod new_payload {
use super::*;
use reth_interfaces::test_utils::generators::random_header;
use reth_primitives::{
bytes::{Bytes, BytesMut},
Block,
};
use reth_rlp::DecodeError;
#[tokio::test]
async fn forwards_responses_to_consensus_engine() {
let (mut handle, api) = setup_engine_api();
tokio::spawn(api);
fn transform_block<F: FnOnce(Block) -> Block>(src: SealedBlock, f: F) -> SealedBlock {
let unsealed = src.unseal();
let mut transformed: Block = f(unsealed);
// Recalculate roots
transformed.header.transactions_root =
proofs::calculate_transaction_root(transformed.body.iter());
transformed.header.ommers_hash =
proofs::calculate_ommers_root(transformed.ommers.iter());
SealedBlock {
header: transformed.header.seal_slow(),
body: transformed.body,
ommers: transformed.ommers.into_iter().map(Header::seal_slow).collect(),
withdrawals: transformed.withdrawals,
}
}
let (result_tx, _result_rx) = oneshot::channel();
handle.send_message(EngineApiMessage::NewPayload(SealedBlock::default().into(), result_tx));
assert_matches!(
handle.engine_rx.recv().await,
Some(BeaconEngineMessage::NewPayload { .. })
);
#[tokio::test]
async fn payload_validation() {
let (_, api) = setup_engine_api();
let block = random_block(100, Some(H256::random()), Some(3), Some(0));
// Valid extra data
let block_with_valid_extra_data = transform_block(block.clone(), |mut b| {
b.header.extra_data = BytesMut::zeroed(32).freeze().into();
b
});
assert_matches!(api.try_construct_block(block_with_valid_extra_data.into()), Ok(_));
// Invalid extra data
let block_with_invalid_extra_data: Bytes = BytesMut::zeroed(33).freeze();
let invalid_extra_data_block = transform_block(block.clone(), |mut b| {
b.header.extra_data = block_with_invalid_extra_data.clone().into();
b
});
assert_matches!(
api.try_construct_block(invalid_extra_data_block.into()),
Err(EngineApiError::PayloadExtraData(data)) if data == block_with_invalid_extra_data
);
// Zero base fee
let block_with_zero_base_fee = transform_block(block.clone(), |mut b| {
b.header.base_fee_per_gas = Some(0);
b
});
assert_matches!(
api.try_construct_block(block_with_zero_base_fee.into()),
Err(EngineApiError::PayloadBaseFee(val)) if val == U256::ZERO
);
// Invalid encoded transactions
let mut payload_with_invalid_txs: ExecutionPayload = block.clone().into();
payload_with_invalid_txs.transactions.iter_mut().for_each(|tx| {
*tx = Bytes::new().into();
});
assert_matches!(
api.try_construct_block(payload_with_invalid_txs),
Err(EngineApiError::Decode(DecodeError::InputTooShort))
);
// Non empty ommers
let block_with_ommers = transform_block(block.clone(), |mut b| {
b.ommers.push(random_header(100, None).unseal());
b
});
assert_matches!(
api.try_construct_block(block_with_ommers.clone().into()),
Err(EngineApiError::PayloadBlockHash { consensus, .. })
if consensus == block_with_ommers.hash()
);
// None zero difficulty
let block_with_difficulty = transform_block(block.clone(), |mut b| {
b.header.difficulty = U256::from(1);
b
});
assert_matches!(
api.try_construct_block(block_with_difficulty.clone().into()),
Err(EngineApiError::PayloadBlockHash { consensus, .. })
if consensus == block_with_difficulty.hash()
);
// None zero nonce
let block_with_nonce = transform_block(block.clone(), |mut b| {
b.header.nonce = 1;
b
});
assert_matches!(
api.try_construct_block(block_with_nonce.clone().into()),
Err(EngineApiError::PayloadBlockHash { consensus, .. })
if consensus == block_with_nonce.hash()
);
// Valid block
let valid_block = block;
assert_matches!(api.try_construct_block(valid_block.into()), Ok(_));
}
#[tokio::test]
async fn payload_known() {
let (handle, api) = setup_engine_api();
tokio::spawn(api);
let block = random_block(100, Some(H256::random()), None, Some(0)); // payload must have no ommers
let block_hash = block.hash();
let execution_payload = block.clone().into();
handle.client.add_header(block_hash, block.header.unseal());
let (result_tx, result_rx) = oneshot::channel();
handle.send_message(EngineApiMessage::NewPayload(
EngineApiMessageVersion::V1,
execution_payload,
result_tx,
));
let expected_result = PayloadStatus::new(PayloadStatusEnum::Valid, block_hash);
assert_matches!(result_rx.await, Ok(Ok(result)) => assert_eq!(result, expected_result));
}
#[tokio::test]
async fn payload_parent_unknown() {
let (handle, api) = setup_engine_api();
tokio::spawn(api);
let (result_tx, result_rx) = oneshot::channel();
let block = random_block(100, Some(H256::random()), None, Some(0)); // payload must have no ommers
handle.send_message(EngineApiMessage::NewPayload(
EngineApiMessageVersion::V1,
block.into(),
result_tx,
));
let expected_result = PayloadStatus::from_status(PayloadStatusEnum::Syncing);
assert_matches!(result_rx.await, Ok(Ok(result)) => assert_eq!(result, expected_result));
}
#[tokio::test]
async fn payload_pre_merge() {
let (handle, api) = setup_engine_api();
tokio::spawn(api);
let parent = transform_block(random_block(100, None, None, Some(0)), |mut b| {
b.header.difficulty =
handle.chain_spec.fork(Hardfork::Paris).ttd().unwrap() - U256::from(1);
b
});
let block = random_block(101, Some(parent.hash()), None, Some(0));
handle.client.add_block(parent.hash(), parent.clone().unseal());
let (result_tx, result_rx) = oneshot::channel();
handle.send_message(EngineApiMessage::NewPayload(
EngineApiMessageVersion::V1,
block.clone().into(),
result_tx,
));
let expected_result = PayloadStatus::from_status(PayloadStatusEnum::Invalid {
validation_error: EngineApiError::PayloadPreMerge.to_string(),
})
.with_latest_valid_hash(H256::zero());
assert_matches!(result_rx.await, Ok(Ok(result)) => assert_eq!(result, expected_result));
}
#[tokio::test]
async fn invalid_payload_timestamp() {
let (handle, api) = setup_engine_api();
tokio::spawn(api);
let block_timestamp = 100;
let parent_timestamp = block_timestamp + 10;
let parent = transform_block(random_block(100, None, None, Some(0)), |mut b| {
b.header.timestamp = parent_timestamp;
b.header.difficulty =
handle.chain_spec.fork(Hardfork::Paris).ttd().unwrap() + U256::from(1);
b
});
let block =
transform_block(random_block(101, Some(parent.hash()), None, Some(0)), |mut b| {
b.header.timestamp = block_timestamp;
b
});
handle.client.add_block(parent.hash(), parent.clone().unseal());
let (result_tx, result_rx) = oneshot::channel();
handle.send_message(EngineApiMessage::NewPayload(
EngineApiMessageVersion::V1,
block.clone().into(),
result_tx,
));
let expected_result = PayloadStatus::from_status(PayloadStatusEnum::Invalid {
validation_error: EngineApiError::PayloadTimestamp {
invalid: block_timestamp,
latest: parent_timestamp,
}
.to_string(),
});
assert_matches!( result_rx.await, Ok(Ok(result)) => assert_eq!(result, expected_result));
}
// TODO: add execution tests
}
// non exhaustive tests for engine_getPayload
// TODO: amend when block building is implemented
mod get_payload {
use super::*;
#[tokio::test]
async fn payload_unknown() {
let (handle, api) = setup_engine_api();
tokio::spawn(api);
let payload_id = H64::random();
let (result_tx, result_rx) = oneshot::channel();
handle.send_message(EngineApiMessage::GetPayload(payload_id, result_tx));
assert_matches!(result_rx.await, Ok(Err(EngineApiError::PayloadUnknown)));
}
let (result_tx, _result_rx) = oneshot::channel();
handle.send_message(EngineApiMessage::ForkchoiceUpdated(
ForkchoiceState::default(),
None,
result_tx,
));
assert_matches!(
handle.engine_rx.recv().await,
Some(BeaconEngineMessage::ForkchoiceUpdated { .. })
);
}
// tests covering `engine_getPayloadBodiesByRange` and `engine_getPayloadBodiesByHash`
@ -892,177 +391,10 @@ mod tests {
}
}
mod fork_choice_updated {
use super::*;
use reth_interfaces::test_utils::generators::random_header;
#[tokio::test]
async fn empty_head() {
let (handle, api) = setup_engine_api();
tokio::spawn(api);
let (result_tx, result_rx) = oneshot::channel();
handle.send_message(EngineApiMessage::ForkchoiceUpdated(
EngineApiMessageVersion::V1,
ForkchoiceState::default(),
None,
result_tx,
));
let expected_result = ForkchoiceUpdated::from_status(PayloadStatusEnum::Invalid {
validation_error: EngineApiError::ForkchoiceEmptyHead.to_string(),
});
assert_matches!(result_rx.await, Ok(Ok(result)) => assert_eq!(result, expected_result));
assert!(!handle.forkchoice_state_has_changed());
}
#[tokio::test]
async fn unknown_head_hash() {
let (handle, api) = setup_engine_api();
tokio::spawn(api);
let state = ForkchoiceState { head_block_hash: H256::random(), ..Default::default() };
let (result_tx, result_rx) = oneshot::channel();
handle.send_message(EngineApiMessage::ForkchoiceUpdated(
EngineApiMessageVersion::V1,
state,
None,
result_tx,
));
let expected_result = ForkchoiceUpdated::from_status(PayloadStatusEnum::Syncing);
assert_matches!(result_rx.await, Ok(Ok(result)) => assert_eq!(result, expected_result));
assert!(!handle.forkchoice_state_has_changed());
}
#[tokio::test]
async fn unknown_finalized_hash() {
let (handle, api) = setup_engine_api();
tokio::spawn(api);
let head = random_header(100, None);
handle.client.add_header(head.hash(), head.clone().unseal());
let state = ForkchoiceState {
head_block_hash: head.hash(),
finalized_block_hash: H256::random(),
..Default::default()
};
let (result_tx, result_rx) = oneshot::channel();
handle.send_message(EngineApiMessage::ForkchoiceUpdated(
EngineApiMessageVersion::V1,
state,
None,
result_tx,
));
let expected_result = ForkchoiceUpdated::from_status(PayloadStatusEnum::Syncing);
assert_matches!(result_rx.await, Ok(Ok(result)) => assert_eq!(result, expected_result));
assert!(!handle.forkchoice_state_has_changed());
}
#[tokio::test]
async fn forkchoice_state_is_updated() {
let (handle, api) = setup_engine_api();
tokio::spawn(api);
let ttd = handle.chain_spec.fork(Hardfork::Paris).ttd().unwrap();
let finalized = random_header(90, None);
let mut head = random_header(100, None).unseal();
// set the difficulty so we know it is post-merge
head.difficulty = ttd;
let head = head.seal_slow();
handle.client.extend_headers([
(head.hash(), head.clone().unseal()),
(finalized.hash(), finalized.clone().unseal()),
]);
let state = ForkchoiceState {
head_block_hash: head.hash(),
finalized_block_hash: finalized.hash(),
..Default::default()
};
let (result_tx, result_rx) = oneshot::channel();
handle.send_message(EngineApiMessage::ForkchoiceUpdated(
EngineApiMessageVersion::V1,
state.clone(),
None,
result_tx,
));
let expected_result = ForkchoiceUpdated {
payload_id: None,
payload_status: PayloadStatus {
status: PayloadStatusEnum::Valid,
latest_valid_hash: Some(head.hash()),
},
};
assert_matches!(result_rx.await, Ok(Ok(result)) => assert_eq!(result, expected_result));
assert!(handle.forkchoice_state_has_changed());
assert_eq!(handle.forkchoice_state(), state);
}
#[tokio::test]
async fn forkchoice_updated_invalid_pow() {
let (handle, api) = setup_engine_api();
tokio::spawn(api);
let finalized = random_header(90, None);
let mut head = random_header(100, None).unseal();
// ensure we don't mess up when subtracting just in case
let ttd = handle.chain_spec.fork(Hardfork::Paris).ttd().unwrap();
assert!(ttd > finalized.difficulty);
// set the difficulty so we know it is post-merge
head.difficulty = ttd - U256::from(1) - finalized.difficulty;
let head = head.seal_slow();
handle.client.extend_headers([
(head.hash(), head.clone().unseal()),
(finalized.hash(), finalized.clone().unseal()),
]);
let state = ForkchoiceState {
head_block_hash: head.hash(),
finalized_block_hash: finalized.hash(),
..Default::default()
};
let (result_tx, result_rx) = oneshot::channel();
handle.send_message(EngineApiMessage::ForkchoiceUpdated(
EngineApiMessageVersion::V1,
state.clone(),
None,
result_tx,
));
let expected_result = ForkchoiceUpdated {
payload_id: None,
payload_status: PayloadStatus {
status: PayloadStatusEnum::Invalid {
validation_error: EngineApiError::PayloadPreMerge.to_string(),
},
latest_valid_hash: Some(H256::zero()),
},
};
assert_matches!(result_rx.await, Ok(Ok(result)) => assert_eq!(result, expected_result));
// From the engine API spec:
//
// Additionally, if this validation fails, client software MUST NOT update the
// forkchoice state and MUST NOT begin a payload build process.
assert!(!handle.forkchoice_state_has_changed());
}
}
// https://github.com/ethereum/execution-apis/blob/main/src/engine/paris.md#specification-3
mod exchange_transition_configuration {
use super::*;
use reth_primitives::U256;
#[tokio::test]
async fn terminal_td_mismatch() {
@ -1081,11 +413,11 @@ mod tests {
result_tx,
));
let expected_error = EngineApiError::TerminalTD {
execution: handle.chain_spec.fork(Hardfork::Paris).ttd().unwrap(),
consensus: U256::from(transition_config.terminal_total_difficulty),
};
assert_matches!(result_rx.await, Ok(Err(error)) => assert_eq!(error, expected_error));
assert_matches!(
result_rx.await,
Ok(Err(EngineApiError::TerminalTD { execution, consensus }))
if execution == handle.chain_spec.fork(Hardfork::Paris).ttd().unwrap() && consensus == U256::from(transition_config.terminal_total_difficulty)
);
}
#[tokio::test]
@ -1109,12 +441,11 @@ mod tests {
transition_config.clone(),
result_tx,
));
let expected_error = EngineApiError::TerminalBlockHash {
execution: None,
consensus: transition_config.terminal_block_hash,
};
assert_matches!(result_rx.await, Ok(Err(error)) => assert_eq!(error, expected_error));
assert_matches!(
result_rx.await,
Ok(Err(EngineApiError::TerminalBlockHash { execution, consensus }))
if execution == None && consensus == transition_config.terminal_block_hash
);
// Add block and to provider local store and test for mismatch
handle.client.add_block(
@ -1128,11 +459,11 @@ mod tests {
result_tx,
));
let expected_error = EngineApiError::TerminalBlockHash {
execution: Some(execution_terminal_block.hash()),
consensus: transition_config.terminal_block_hash,
};
assert_matches!(result_rx.await, Ok(Err(error)) => assert_eq!(error, expected_error));
assert_matches!(
result_rx.await,
Ok(Err(EngineApiError::TerminalBlockHash { execution, consensus }))
if execution == Some(execution_terminal_block.hash()) && consensus == transition_config.terminal_block_hash
);
}
#[tokio::test]

View File

@ -1,4 +1,5 @@
use reth_primitives::{Bytes, H256, U256};
use reth_beacon_consensus::BeaconEngineError;
use reth_primitives::{H256, U256};
use thiserror::Error;
/// The Engine API result type
@ -10,39 +11,8 @@ pub const UNKNOWN_PAYLOAD_CODE: i32 = -38001;
pub const REQUEST_TOO_LARGE_CODE: i32 = -38004;
/// Error returned by [`EngineApi`][crate::EngineApi]
#[derive(Error, PartialEq, Debug)]
#[derive(Error, Debug)]
pub enum EngineApiError {
/// Invalid payload extra data.
#[error("Invalid payload extra data: {0}")]
PayloadExtraData(Bytes),
/// Invalid payload base fee.
#[error("Invalid payload base fee: {0}")]
PayloadBaseFee(U256),
/// Invalid payload block hash.
#[error("Invalid payload block hash. Execution: {execution}. Consensus: {consensus}")]
PayloadBlockHash {
/// The block hash computed from the payload.
execution: H256,
/// The block hash provided with the payload.
consensus: H256,
},
/// Invalid payload block hash.
#[error("Invalid payload timestamp: {invalid}. Latest: {latest}")]
PayloadTimestamp {
/// The payload timestamp.
invalid: u64,
/// Latest available timestamp.
latest: u64,
},
/// Failed to recover transaction signer.
#[error("Failed to recover signer for payload transaction: {hash:?}")]
PayloadSignerRecovery {
/// The hash of the failed transaction
hash: H256,
},
/// Received pre-merge payload.
#[error("Received pre-merge payload.")]
PayloadPreMerge,
/// Unknown payload requested.
#[error("Unknown payload")]
PayloadUnknown,
@ -75,16 +45,10 @@ pub enum EngineApiError {
/// Consensus terminal block hash.
consensus: H256,
},
/// Forkchoice zero hash head received.
#[error("Received zero hash as forkchoice head")]
ForkchoiceEmptyHead,
/// Chain spec merge terminal total difficulty is not set
#[error("The merge terminal total difficulty is not known")]
UnknownMergeTerminalTotalDifficulty,
/// Encountered decoding error.
/// Beacon consensus engine error.
#[error(transparent)]
Decode(#[from] reth_rlp::DecodeError),
/// API encountered an internal error.
ConsensusEngine(#[from] BeaconEngineError),
/// Encountered an internal error.
#[error(transparent)]
Internal(#[from] reth_interfaces::Error),
Internal(Box<dyn std::error::Error + Send + Sync>),
}

View File

@ -1,4 +1,5 @@
use crate::EngineApiSender;
use reth_beacon_consensus::BeaconEngineSender;
use reth_interfaces::consensus::ForkchoiceState;
use reth_primitives::{BlockHash, BlockNumber, H64};
use reth_rpc_types::engine::{
@ -9,26 +10,25 @@ use reth_rpc_types::engine::{
/// Message type for communicating with [`EngineApi`][crate::EngineApi].
#[derive(Debug)]
pub enum EngineApiMessage {
/// New payload message
NewPayload(EngineApiMessageVersion, ExecutionPayload, EngineApiSender<PayloadStatus>),
/// Get payload message
GetPayload(H64, EngineApiSender<ExecutionPayload>),
/// Get payload bodies by range message
GetPayloadBodiesByRange(BlockNumber, u64, EngineApiSender<ExecutionPayloadBodies>),
/// Get payload bodies by hash message
GetPayloadBodiesByHash(Vec<BlockHash>, EngineApiSender<ExecutionPayloadBodies>),
/// Forkchoice updated message
ForkchoiceUpdated(
EngineApiMessageVersion,
ForkchoiceState,
Option<PayloadAttributes>,
EngineApiSender<ForkchoiceUpdated>,
),
/// Exchange transition configuration message
ExchangeTransitionConfiguration(
TransitionConfiguration,
EngineApiSender<TransitionConfiguration>,
),
/// New payload message
NewPayload(ExecutionPayload, BeaconEngineSender<PayloadStatus>),
/// Forkchoice updated message
ForkchoiceUpdated(
ForkchoiceState,
Option<PayloadAttributes>,
BeaconEngineSender<ForkchoiceUpdated>,
),
}
/// The version of Engine API message.

View File

@ -8,6 +8,7 @@ readme = "README.md"
description = """
Reth RPC types
"""
[dependencies]
# reth
reth-primitives = { path = "../../primitives" }
@ -27,5 +28,9 @@ jsonrpsee-types = { version = "0.16" }
lru = "0.9"
[dev-dependencies]
rand = "0.8"
# reth
reth-interfaces = { path = "../../interfaces", features = ["test-utils"] }
# misc
rand = "0.8"
assert_matches = "1.5"

View File

@ -1,235 +0,0 @@
//! Engine API types: <https://github.com/ethereum/execution-apis/blob/main/src/engine/authentication.md> and <https://eips.ethereum.org/EIPS/eip-3675> following the execution specs <https://github.com/ethereum/execution-apis/tree/6709c2a795b707202e93c4f2867fa0bf2640a84f/src/engine>
#![allow(missing_docs)]
use reth_primitives::{
Address, Block, Bloom, Bytes, SealedBlock, Withdrawal, H256, H64, U256, U64,
};
use reth_rlp::Encodable;
use serde::{Deserialize, Serialize};
/// The list of supported Engine capabilities
pub const CAPABILITIES: [&str; 9] = [
"engine_forkchoiceUpdatedV1",
"engine_forkchoiceUpdatedV2",
"engine_exchangeTransitionConfigurationV1",
"engine_getPayloadV1",
"engine_getPayloadV2",
"engine_newPayloadV1",
"engine_newPayloadV2",
"engine_getPayloadBodiesByHashV1",
"engine_getPayloadBodiesByRangeV1",
];
/// This structure maps on the ExecutionPayload structure of the beacon chain spec.
///
/// See also: <https://github.com/ethereum/execution-apis/blob/6709c2a795b707202e93c4f2867fa0bf2640a84f/src/engine/paris.md#executionpayloadv1>
#[derive(Clone, Debug, PartialEq, Eq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct ExecutionPayload {
pub parent_hash: H256,
pub fee_recipient: Address,
pub state_root: H256,
pub receipts_root: H256,
pub logs_bloom: Bloom,
pub prev_randao: H256,
pub block_number: U64,
pub gas_limit: U64,
pub gas_used: U64,
pub timestamp: U64,
pub extra_data: Bytes,
pub base_fee_per_gas: U256,
pub block_hash: H256,
pub transactions: Vec<Bytes>,
/// Array of [`Withdrawal`] enabled with V2
/// See <https://github.com/ethereum/execution-apis/blob/6709c2a795b707202e93c4f2867fa0bf2640a84f/src/engine/shanghai.md#executionpayloadv2>
#[serde(default, skip_serializing_if = "Option::is_none")]
pub withdrawals: Option<Vec<Withdrawal>>,
}
impl From<SealedBlock> for ExecutionPayload {
fn from(value: SealedBlock) -> Self {
let transactions = value
.body
.iter()
.map(|tx| {
let mut encoded = Vec::new();
tx.encode(&mut encoded);
encoded.into()
})
.collect();
ExecutionPayload {
parent_hash: value.parent_hash,
fee_recipient: value.beneficiary,
state_root: value.state_root,
receipts_root: value.receipts_root,
logs_bloom: value.logs_bloom,
prev_randao: value.mix_hash,
block_number: value.number.into(),
gas_limit: value.gas_limit.into(),
gas_used: value.gas_used.into(),
timestamp: value.timestamp.into(),
extra_data: value.extra_data.clone(),
base_fee_per_gas: U256::from(value.base_fee_per_gas.unwrap_or_default()),
block_hash: value.hash(),
transactions,
withdrawals: value.withdrawals,
}
}
}
/// This structure contains a body of an execution payload.
///
/// See also: <https://github.com/ethereum/execution-apis/blob/6452a6b194d7db269bf1dbd087a267251d3cc7f8/src/engine/shanghai.md#executionpayloadbodyv1>
#[derive(Clone, Debug, PartialEq, Eq, Serialize, Deserialize)]
pub struct ExecutionPayloadBody {
pub transactions: Vec<Bytes>,
pub withdrawals: Vec<Withdrawal>,
}
impl From<Block> for ExecutionPayloadBody {
fn from(value: Block) -> Self {
let transactions = value.body.into_iter().map(|tx| {
let mut out = Vec::new();
tx.encode(&mut out);
out.into()
});
ExecutionPayloadBody {
transactions: transactions.collect(),
withdrawals: value.withdrawals.unwrap_or_default(),
}
}
}
/// The execution payload body response that allows for `null` values.
pub type ExecutionPayloadBodies = Vec<Option<ExecutionPayloadBody>>;
/// This structure encapsulates the fork choice state
#[derive(Default, Clone, Debug, PartialEq, Eq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct ForkchoiceState {
pub head_block_hash: H256,
pub safe_block_hash: H256,
pub finalized_block_hash: H256,
}
/// This structure contains the attributes required to initiate a payload build process in the
/// context of an `engine_forkchoiceUpdated` call.
#[derive(Clone, Debug, PartialEq, Eq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct PayloadAttributes {
pub timestamp: U64,
pub prev_randao: H256,
pub suggested_fee_recipient: Address,
/// Array of [`Withdrawal`] enabled with V2
/// See <https://github.com/ethereum/execution-apis/blob/6452a6b194d7db269bf1dbd087a267251d3cc7f8/src/engine/shanghai.md#payloadattributesv2>
#[serde(default, skip_serializing_if = "Option::is_none")]
pub withdrawals: Option<Vec<Withdrawal>>,
}
/// This structure contains the result of processing a payload
#[derive(Clone, Debug, PartialEq, Eq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct PayloadStatus {
#[serde(flatten)]
pub status: PayloadStatusEnum,
/// Hash of the most recent valid block in the branch defined by payload and its ancestors
pub latest_valid_hash: Option<H256>,
}
impl PayloadStatus {
pub fn new(status: PayloadStatusEnum, latest_valid_hash: H256) -> Self {
Self { status, latest_valid_hash: Some(latest_valid_hash) }
}
pub fn from_status(status: PayloadStatusEnum) -> Self {
Self { status, latest_valid_hash: None }
}
pub fn with_latest_valid_hash(mut self, latest_valid_hash: H256) -> Self {
self.latest_valid_hash = Some(latest_valid_hash);
self
}
}
#[derive(Clone, Debug, PartialEq, Eq, Serialize, Deserialize)]
#[serde(tag = "status", rename_all = "SCREAMING_SNAKE_CASE")]
pub enum PayloadStatusEnum {
Valid,
Invalid {
#[serde(rename = "validationError")]
validation_error: String,
},
Syncing,
Accepted,
InvalidBlockHash {
#[serde(rename = "validationError")]
validation_error: String,
},
}
/// This structure contains configurable settings of the transition process.
#[derive(Default, Clone, Debug, PartialEq, Eq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct TransitionConfiguration {
/// Maps on the TERMINAL_TOTAL_DIFFICULTY parameter of EIP-3675
pub terminal_total_difficulty: U256,
/// Maps on TERMINAL_BLOCK_HASH parameter of EIP-3675
pub terminal_block_hash: H256,
/// Maps on TERMINAL_BLOCK_NUMBER parameter of EIP-3675
pub terminal_block_number: U64,
}
#[derive(Clone, Debug, PartialEq, Eq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct ForkchoiceUpdated {
pub payload_status: PayloadStatus,
pub payload_id: Option<H64>,
}
impl ForkchoiceUpdated {
pub fn new(payload_status: PayloadStatus) -> Self {
Self { payload_status, payload_id: None }
}
pub fn from_status(status: PayloadStatusEnum) -> Self {
Self { payload_status: PayloadStatus::from_status(status), payload_id: None }
}
pub fn with_latest_valid_hash(mut self, hash: H256) -> Self {
self.payload_status.latest_valid_hash = Some(hash);
self
}
pub fn with_payload_id(mut self, id: H64) -> Self {
self.payload_id = Some(id);
self
}
}
#[cfg(test)]
mod tests {
use super::*;
use reth_interfaces::test_utils::generators::random_block_range;
use reth_primitives::{TransactionSigned, H256};
use reth_rlp::Decodable;
#[test]
fn payload_body_roundtrip() {
for block in random_block_range(0..100, H256::default(), 0..2) {
let unsealed = block.clone().unseal();
let payload_body: ExecutionPayloadBody = unsealed.into();
assert_eq!(
Ok(block.body),
payload_body
.transactions
.iter()
.map(|x| TransactionSigned::decode(&mut &x[..]))
.collect::<Result<Vec<_>, _>>(),
);
assert_eq!(block.withdrawals.unwrap_or_default(), payload_body.withdrawals);
}
}
}

View File

@ -0,0 +1,39 @@
use super::{PayloadStatus, PayloadStatusEnum};
use reth_primitives::{H256, H64};
use serde::{Deserialize, Serialize};
/// This structure encapsulates the fork choice state
#[derive(Default, Clone, Copy, Debug, PartialEq, Eq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct ForkchoiceState {
pub head_block_hash: H256,
pub safe_block_hash: H256,
pub finalized_block_hash: H256,
}
#[derive(Clone, Debug, PartialEq, Eq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct ForkchoiceUpdated {
pub payload_status: PayloadStatus,
pub payload_id: Option<H64>,
}
impl ForkchoiceUpdated {
pub fn new(payload_status: PayloadStatus) -> Self {
Self { payload_status, payload_id: None }
}
pub fn from_status(status: PayloadStatusEnum) -> Self {
Self { payload_status: PayloadStatus::from_status(status), payload_id: None }
}
pub fn with_latest_valid_hash(mut self, hash: H256) -> Self {
self.payload_status.latest_valid_hash = Some(hash);
self
}
pub fn with_payload_id(mut self, id: H64) -> Self {
self.payload_id = Some(id);
self
}
}

View File

@ -0,0 +1,22 @@
//! Engine API types: <https://github.com/ethereum/execution-apis/blob/main/src/engine/authentication.md> and <https://eips.ethereum.org/EIPS/eip-3675> following the execution specs <https://github.com/ethereum/execution-apis/tree/6709c2a795b707202e93c4f2867fa0bf2640a84f/src/engine>
#![allow(missing_docs)]
mod forkchoice;
mod payload;
mod transition;
pub use self::{forkchoice::*, payload::*, transition::*};
/// The list of supported Engine capabilities
pub const CAPABILITIES: [&str; 9] = [
"engine_forkchoiceUpdatedV1",
"engine_forkchoiceUpdatedV2",
"engine_exchangeTransitionConfigurationV1",
"engine_getPayloadV1",
"engine_getPayloadV2",
"engine_newPayloadV1",
"engine_newPayloadV2",
"engine_getPayloadBodiesByHashV1",
"engine_getPayloadBodiesByRangeV1",
];

View File

@ -0,0 +1,361 @@
use reth_primitives::{
proofs::{self, EMPTY_LIST_HASH},
Address, Block, Bloom, Bytes, Header, SealedBlock, TransactionSigned, Withdrawal, H256, U256,
U64,
};
use reth_rlp::{Decodable, Encodable};
use serde::{Deserialize, Serialize};
/// This structure maps on the ExecutionPayload structure of the beacon chain spec.
///
/// See also: <https://github.com/ethereum/execution-apis/blob/6709c2a795b707202e93c4f2867fa0bf2640a84f/src/engine/paris.md#executionpayloadv1>
#[derive(Clone, Debug, PartialEq, Eq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct ExecutionPayload {
pub parent_hash: H256,
pub fee_recipient: Address,
pub state_root: H256,
pub receipts_root: H256,
pub logs_bloom: Bloom,
pub prev_randao: H256,
pub block_number: U64,
pub gas_limit: U64,
pub gas_used: U64,
pub timestamp: U64,
pub extra_data: Bytes,
pub base_fee_per_gas: U256,
pub block_hash: H256,
pub transactions: Vec<Bytes>,
/// Array of [`Withdrawal`] enabled with V2
/// See <https://github.com/ethereum/execution-apis/blob/6709c2a795b707202e93c4f2867fa0bf2640a84f/src/engine/shanghai.md#executionpayloadv2>
#[serde(default, skip_serializing_if = "Option::is_none")]
pub withdrawals: Option<Vec<Withdrawal>>,
}
impl From<SealedBlock> for ExecutionPayload {
fn from(value: SealedBlock) -> Self {
let transactions = value
.body
.iter()
.map(|tx| {
let mut encoded = Vec::new();
tx.encode(&mut encoded);
encoded.into()
})
.collect();
ExecutionPayload {
parent_hash: value.parent_hash,
fee_recipient: value.beneficiary,
state_root: value.state_root,
receipts_root: value.receipts_root,
logs_bloom: value.logs_bloom,
prev_randao: value.mix_hash,
block_number: value.number.into(),
gas_limit: value.gas_limit.into(),
gas_used: value.gas_used.into(),
timestamp: value.timestamp.into(),
extra_data: value.extra_data.clone(),
base_fee_per_gas: U256::from(value.base_fee_per_gas.unwrap_or_default()),
block_hash: value.hash(),
transactions,
withdrawals: value.withdrawals,
}
}
}
/// Try to construct a block from given payload. Perform addition validation of `extra_data` and
/// `base_fee_per_gas` fields.
///
/// NOTE: The log bloom is assumed to be validated during serialization.
/// NOTE: Empty ommers, nonce and difficulty values are validated upon computing block hash and
/// comparing the value with `payload.block_hash`.
///
/// See <https://github.com/ethereum/go-ethereum/blob/79a478bb6176425c2400e949890e668a3d9a3d05/core/beacon/types.go#L145>
impl TryFrom<ExecutionPayload> for SealedBlock {
type Error = PayloadError;
fn try_from(payload: ExecutionPayload) -> Result<Self, Self::Error> {
if payload.extra_data.len() > 32 {
return Err(PayloadError::ExtraData(payload.extra_data))
}
if payload.base_fee_per_gas == U256::ZERO {
return Err(PayloadError::BaseFee(payload.base_fee_per_gas))
}
let transactions = payload
.transactions
.iter()
.map(|tx| TransactionSigned::decode(&mut tx.as_ref()))
.collect::<Result<Vec<_>, _>>()?;
let transactions_root = proofs::calculate_transaction_root(transactions.iter());
let withdrawals_root =
payload.withdrawals.as_ref().map(|w| proofs::calculate_withdrawals_root(w.iter()));
let header = Header {
parent_hash: payload.parent_hash,
beneficiary: payload.fee_recipient,
state_root: payload.state_root,
transactions_root,
receipts_root: payload.receipts_root,
withdrawals_root,
logs_bloom: payload.logs_bloom,
number: payload.block_number.as_u64(),
gas_limit: payload.gas_limit.as_u64(),
gas_used: payload.gas_used.as_u64(),
timestamp: payload.timestamp.as_u64(),
mix_hash: payload.prev_randao,
base_fee_per_gas: Some(payload.base_fee_per_gas.to::<u64>()),
extra_data: payload.extra_data,
// Defaults
ommers_hash: EMPTY_LIST_HASH,
difficulty: Default::default(),
nonce: Default::default(),
}
.seal_slow();
if payload.block_hash != header.hash() {
return Err(PayloadError::BlockHash {
execution: header.hash(),
consensus: payload.block_hash,
})
}
Ok(SealedBlock {
header,
body: transactions,
withdrawals: payload.withdrawals,
ommers: Default::default(),
})
}
}
#[derive(thiserror::Error, Debug)]
pub enum PayloadError {
/// Invalid payload extra data.
#[error("Invalid payload extra data: {0}")]
ExtraData(Bytes),
/// Invalid payload base fee.
#[error("Invalid payload base fee: {0}")]
BaseFee(U256),
/// Invalid payload block hash.
#[error("Invalid payload block hash. Execution: {execution}. Consensus: {consensus}")]
BlockHash {
/// The block hash computed from the payload.
execution: H256,
/// The block hash provided with the payload.
consensus: H256,
},
/// Encountered decoding error.
#[error(transparent)]
Decode(#[from] reth_rlp::DecodeError),
}
/// This structure contains a body of an execution payload.
///
/// See also: <https://github.com/ethereum/execution-apis/blob/6452a6b194d7db269bf1dbd087a267251d3cc7f8/src/engine/shanghai.md#executionpayloadbodyv1>
#[derive(Clone, Debug, PartialEq, Eq, Serialize, Deserialize)]
pub struct ExecutionPayloadBody {
pub transactions: Vec<Bytes>,
pub withdrawals: Vec<Withdrawal>,
}
impl From<Block> for ExecutionPayloadBody {
fn from(value: Block) -> Self {
let transactions = value.body.into_iter().map(|tx| {
let mut out = Vec::new();
tx.encode(&mut out);
out.into()
});
ExecutionPayloadBody {
transactions: transactions.collect(),
withdrawals: value.withdrawals.unwrap_or_default(),
}
}
}
/// The execution payload body response that allows for `null` values.
pub type ExecutionPayloadBodies = Vec<Option<ExecutionPayloadBody>>;
/// This structure contains the attributes required to initiate a payload build process in the
/// context of an `engine_forkchoiceUpdated` call.
#[derive(Clone, Debug, PartialEq, Eq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct PayloadAttributes {
pub timestamp: U64,
pub prev_randao: H256,
pub suggested_fee_recipient: Address,
/// Array of [`Withdrawal`] enabled with V2
/// See <https://github.com/ethereum/execution-apis/blob/6452a6b194d7db269bf1dbd087a267251d3cc7f8/src/engine/shanghai.md#payloadattributesv2>
#[serde(default, skip_serializing_if = "Option::is_none")]
pub withdrawals: Option<Vec<Withdrawal>>,
}
/// This structure contains the result of processing a payload
#[derive(Clone, Debug, PartialEq, Eq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct PayloadStatus {
#[serde(flatten)]
pub status: PayloadStatusEnum,
/// Hash of the most recent valid block in the branch defined by payload and its ancestors
pub latest_valid_hash: Option<H256>,
}
impl PayloadStatus {
pub fn new(status: PayloadStatusEnum, latest_valid_hash: Option<H256>) -> Self {
Self { status, latest_valid_hash }
}
pub fn from_status(status: PayloadStatusEnum) -> Self {
Self { status, latest_valid_hash: None }
}
pub fn with_latest_valid_hash(mut self, latest_valid_hash: H256) -> Self {
self.latest_valid_hash = Some(latest_valid_hash);
self
}
}
#[derive(Clone, Debug, PartialEq, Eq, Serialize, Deserialize)]
#[serde(tag = "status", rename_all = "SCREAMING_SNAKE_CASE")]
pub enum PayloadStatusEnum {
Valid,
Invalid {
#[serde(rename = "validationError")]
validation_error: String,
},
Syncing,
Accepted,
InvalidBlockHash {
#[serde(rename = "validationError")]
validation_error: String,
},
}
#[cfg(test)]
mod tests {
use super::*;
use assert_matches::assert_matches;
use reth_interfaces::test_utils::generators::{
random_block, random_block_range, random_header,
};
use reth_primitives::{
bytes::{Bytes, BytesMut},
TransactionSigned, H256,
};
use reth_rlp::{Decodable, DecodeError};
fn transform_block<F: FnOnce(Block) -> Block>(src: SealedBlock, f: F) -> ExecutionPayload {
let unsealed = src.unseal();
let mut transformed: Block = f(unsealed);
// Recalculate roots
transformed.header.transactions_root =
proofs::calculate_transaction_root(transformed.body.iter());
transformed.header.ommers_hash = proofs::calculate_ommers_root(transformed.ommers.iter());
SealedBlock {
header: transformed.header.seal_slow(),
body: transformed.body,
ommers: transformed.ommers.into_iter().map(Header::seal_slow).collect(),
withdrawals: transformed.withdrawals,
}
.into()
}
#[test]
fn payload_body_roundtrip() {
for block in random_block_range(0..100, H256::default(), 0..2) {
let unsealed = block.clone().unseal();
let payload_body: ExecutionPayloadBody = unsealed.into();
assert_eq!(
Ok(block.body),
payload_body
.transactions
.iter()
.map(|x| TransactionSigned::decode(&mut &x[..]))
.collect::<Result<Vec<_>, _>>(),
);
assert_eq!(block.withdrawals.unwrap_or_default(), payload_body.withdrawals);
}
}
#[test]
fn payload_validation() {
let block = random_block(100, Some(H256::random()), Some(3), Some(0));
// Valid extra data
let block_with_valid_extra_data = transform_block(block.clone(), |mut b| {
b.header.extra_data = BytesMut::zeroed(32).freeze().into();
b
});
assert_matches!(TryInto::<SealedBlock>::try_into(block_with_valid_extra_data), Ok(_));
// Invalid extra data
let block_with_invalid_extra_data: Bytes = BytesMut::zeroed(33).freeze().into();
let invalid_extra_data_block = transform_block(block.clone(), |mut b| {
b.header.extra_data = block_with_invalid_extra_data.clone().into();
b
});
assert_matches!(
TryInto::<SealedBlock>::try_into(invalid_extra_data_block),
Err(PayloadError::ExtraData(data)) if data == block_with_invalid_extra_data
);
// Zero base fee
let block_with_zero_base_fee = transform_block(block.clone(), |mut b| {
b.header.base_fee_per_gas = Some(0);
b
});
assert_matches!(
TryInto::<SealedBlock>::try_into(block_with_zero_base_fee),
Err(PayloadError::BaseFee(val)) if val == U256::ZERO
);
// Invalid encoded transactions
let mut payload_with_invalid_txs: ExecutionPayload = block.clone().into();
payload_with_invalid_txs.transactions.iter_mut().for_each(|tx| {
*tx = Bytes::new().into();
});
assert_matches!(
TryInto::<SealedBlock>::try_into(payload_with_invalid_txs),
Err(PayloadError::Decode(DecodeError::InputTooShort))
);
// Non empty ommers
let block_with_ommers = transform_block(block.clone(), |mut b| {
b.ommers.push(random_header(100, None).unseal());
b
});
assert_matches!(
TryInto::<SealedBlock>::try_into(block_with_ommers.clone()),
Err(PayloadError::BlockHash { consensus, .. })
if consensus == block_with_ommers.block_hash
);
// None zero difficulty
let block_with_difficulty = transform_block(block.clone(), |mut b| {
b.header.difficulty = U256::from(1);
b
});
assert_matches!(
TryInto::<SealedBlock>::try_into(block_with_difficulty.clone()),
Err(PayloadError::BlockHash { consensus, .. }) if consensus == block_with_difficulty.block_hash
);
// None zero nonce
let block_with_nonce = transform_block(block.clone(), |mut b| {
b.header.nonce = 1;
b
});
assert_matches!(
TryInto::<SealedBlock>::try_into(block_with_nonce.clone()),
Err(PayloadError::BlockHash { consensus, .. }) if consensus == block_with_nonce.block_hash
);
// Valid block
let valid_block = block;
assert_matches!(TryInto::<SealedBlock>::try_into(valid_block), Ok(_));
}
}

View File

@ -0,0 +1,14 @@
use reth_primitives::{H256, U256, U64};
use serde::{Deserialize, Serialize};
/// This structure contains configurable settings of the transition process.
#[derive(Default, Clone, Debug, PartialEq, Eq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct TransitionConfiguration {
/// Maps on the TERMINAL_TOTAL_DIFFICULTY parameter of EIP-3675
pub terminal_total_difficulty: U256,
/// Maps on TERMINAL_BLOCK_HASH parameter of EIP-3675
pub terminal_block_hash: H256,
/// Maps on TERMINAL_BLOCK_NUMBER parameter of EIP-3675
pub terminal_block_number: U64,
}

View File

@ -1,3 +1,5 @@
use std::sync::Arc;
use crate::result::rpc_err;
use async_trait::async_trait;
use jsonrpsee::{
@ -5,7 +7,7 @@ use jsonrpsee::{
types::error::INVALID_PARAMS_CODE,
};
use reth_interfaces::consensus::ForkchoiceState;
use reth_primitives::{BlockHash, BlockNumber, H64};
use reth_primitives::{BlockHash, BlockNumber, ChainSpec, Hardfork, H64};
use reth_rpc_api::EngineApiServer;
use reth_rpc_engine_api::{
EngineApiError, EngineApiHandle, EngineApiMessage, EngineApiMessageVersion, EngineApiResult,
@ -17,16 +19,30 @@ use reth_rpc_types::engine::{
};
use tokio::sync::oneshot::{self, Receiver};
fn to_rpc_error<E: Into<EngineApiError>>(error: E) -> Error {
let error = error.into();
let code = match error {
EngineApiError::InvalidParams => INVALID_PARAMS_CODE,
EngineApiError::PayloadUnknown => UNKNOWN_PAYLOAD_CODE,
EngineApiError::PayloadRequestTooLarge { .. } => REQUEST_TOO_LARGE_CODE,
// Any other server error
_ => jsonrpsee::types::error::INTERNAL_ERROR_CODE,
};
rpc_err(code, error.to_string(), None)
}
/// The server implementation of Engine API
pub struct EngineApi {
/// Handle to the consensus engine
/// Chain spec
chain_spec: Arc<ChainSpec>,
/// Handle to the engine API implementation.
engine_tx: EngineApiHandle,
}
impl EngineApi {
/// Creates a new instance of [EngineApi].
pub fn new(engine_tx: EngineApiHandle) -> Self {
Self { engine_tx }
pub fn new(chain_spec: Arc<ChainSpec>, engine_tx: EngineApiHandle) -> Self {
Self { chain_spec, engine_tx }
}
}
@ -37,22 +53,42 @@ impl std::fmt::Debug for EngineApi {
}
impl EngineApi {
async fn delegate_request<T>(
/// Validates the presence of the `withdrawals` field according to the payload timestamp.
/// After Shanghai, withdrawals field must be [Some].
/// Before Shanghai, withdrawals field must be [None];
fn validate_withdrawals_presence(
&self,
version: EngineApiMessageVersion,
timestamp: u64,
has_withdrawals: bool,
) -> EngineApiResult<()> {
let is_shanghai = self.chain_spec.fork(Hardfork::Shanghai).active_at_timestamp(timestamp);
match version {
EngineApiMessageVersion::V1 => {
if is_shanghai || has_withdrawals {
return Err(EngineApiError::InvalidParams)
}
}
EngineApiMessageVersion::V2 => {
let shanghai_with_no_withdrawals = is_shanghai && !has_withdrawals;
let not_shanghai_with_withdrawals = !is_shanghai && has_withdrawals;
if shanghai_with_no_withdrawals || not_shanghai_with_withdrawals {
return Err(EngineApiError::InvalidParams)
}
}
};
Ok(())
}
async fn delegate_request<T, E: Into<EngineApiError>>(
&self,
msg: EngineApiMessage,
rx: Receiver<EngineApiResult<T>>,
rx: Receiver<std::result::Result<T, E>>,
) -> Result<T> {
let _ = self.engine_tx.send(msg);
rx.await.map_err(|err| Error::Custom(err.to_string()))?.map_err(|err| {
let code = match err {
EngineApiError::InvalidParams => INVALID_PARAMS_CODE,
EngineApiError::PayloadUnknown => UNKNOWN_PAYLOAD_CODE,
EngineApiError::PayloadRequestTooLarge { .. } => REQUEST_TOO_LARGE_CODE,
// Any other server error
_ => jsonrpsee::types::error::INTERNAL_ERROR_CODE,
};
rpc_err(code, err.to_string(), None)
})
rx.await.map_err(|err| Error::Custom(err.to_string()))?.map_err(|err| to_rpc_error(err))
}
}
@ -62,23 +98,27 @@ impl EngineApiServer for EngineApi {
/// See also <https://github.com/ethereum/execution-apis/blob/8db51dcd2f4bdfbd9ad6e4a7560aac97010ad063/src/engine/specification.md#engine_newpayloadv1>
/// Caution: This should not accept the `withdrawals` field
async fn new_payload_v1(&self, payload: ExecutionPayload) -> Result<PayloadStatus> {
let (tx, rx) = oneshot::channel();
self.delegate_request(
EngineApiMessage::NewPayload(EngineApiMessageVersion::V1, payload, tx),
rx,
self.validate_withdrawals_presence(
EngineApiMessageVersion::V1,
payload.timestamp.as_u64(),
payload.withdrawals.is_some(),
)
.await
.map_err(to_rpc_error)?;
let (tx, rx) = oneshot::channel();
self.delegate_request(EngineApiMessage::NewPayload(payload, tx), rx).await
}
/// Handler for `engine_getPayloadV2`
/// See also <https://github.com/ethereum/execution-apis/blob/8db51dcd2f4bdfbd9ad6e4a7560aac97010ad063/src/engine/specification.md#engine_newpayloadv1>
async fn new_payload_v2(&self, payload: ExecutionPayload) -> Result<PayloadStatus> {
let (tx, rx) = oneshot::channel();
self.delegate_request(
EngineApiMessage::NewPayload(EngineApiMessageVersion::V2, payload, tx),
rx,
self.validate_withdrawals_presence(
EngineApiMessageVersion::V2,
payload.timestamp.as_u64(),
payload.withdrawals.is_some(),
)
.await
.map_err(to_rpc_error)?;
let (tx, rx) = oneshot::channel();
self.delegate_request(EngineApiMessage::NewPayload(payload, tx), rx).await
}
/// Handler for `engine_forkchoiceUpdatedV1`
@ -90,14 +130,17 @@ impl EngineApiServer for EngineApi {
fork_choice_state: ForkchoiceState,
payload_attributes: Option<PayloadAttributes>,
) -> Result<ForkchoiceUpdated> {
if let Some(ref attrs) = payload_attributes {
self.validate_withdrawals_presence(
EngineApiMessageVersion::V1,
attrs.timestamp.as_u64(),
attrs.withdrawals.is_some(),
)
.map_err(to_rpc_error)?;
}
let (tx, rx) = oneshot::channel();
self.delegate_request(
EngineApiMessage::ForkchoiceUpdated(
EngineApiMessageVersion::V1,
fork_choice_state,
payload_attributes,
tx,
),
EngineApiMessage::ForkchoiceUpdated(fork_choice_state, payload_attributes, tx),
rx,
)
.await
@ -110,14 +153,17 @@ impl EngineApiServer for EngineApi {
fork_choice_state: ForkchoiceState,
payload_attributes: Option<PayloadAttributes>,
) -> Result<ForkchoiceUpdated> {
if let Some(ref attrs) = payload_attributes {
self.validate_withdrawals_presence(
EngineApiMessageVersion::V2,
attrs.timestamp.as_u64(),
attrs.withdrawals.is_some(),
)
.map_err(to_rpc_error)?;
}
let (tx, rx) = oneshot::channel();
self.delegate_request(
EngineApiMessage::ForkchoiceUpdated(
EngineApiMessageVersion::V2,
fork_choice_state,
payload_attributes,
tx,
),
EngineApiMessage::ForkchoiceUpdated(fork_choice_state, payload_attributes, tx),
rx,
)
.await