mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 19:09:54 +00:00
feat(sync): beacon consensus engine (#1845)
This commit is contained in:
@ -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"
|
||||
|
||||
@ -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]
|
||||
|
||||
@ -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>),
|
||||
}
|
||||
|
||||
@ -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.
|
||||
|
||||
Reference in New Issue
Block a user