Implement engine API v1.0.0-alpha.4 (#2810)

* Added ForkchoiceUpdatedV1 & GetPayloadV1

* Added ExecutePayloadV1

* Added new geth test vectors

* Separated Json Object/Serialization Code into file

* Deleted code/tests for Requests Removed from spec

* Finally fixed serialization of null '0x'

* Made Naming of JSON Structs Consistent

* Fix clippy lints

* Remove u64 payload id

* Remove unused serde impls

* Swap to [u8; 8] for payload id

* Tidy

* Adjust some block gen return vals

* Tidy

* Add fallback when payload id is unknown

* Remove comment

Co-authored-by: Mark Mackey <mark@sigmaprime.io>
This commit is contained in:
Paul Hauner 2021-11-15 17:13:38 +11:00
parent cdfd1304a5
commit 47db682d7e
No known key found for this signature in database
GPG Key ID: 5E2CFF9B75FA63DF
17 changed files with 1271 additions and 915 deletions

View File

@ -2908,10 +2908,30 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
let timestamp = let timestamp =
compute_timestamp_at_slot(&state, &self.spec).map_err(BeaconStateError::from)?; compute_timestamp_at_slot(&state, &self.spec).map_err(BeaconStateError::from)?;
let random = *state.get_randao_mix(state.current_epoch())?; let random = *state.get_randao_mix(state.current_epoch())?;
let finalized_root = state.finalized_checkpoint().root;
let finalized_block_hash =
if let Some(block) = self.fork_choice.read().get_block(&finalized_root) {
block.execution_status.block_hash()
} else {
self.store
.get_block(&finalized_root)
.map_err(BlockProductionError::FailedToReadFinalizedBlock)?
.ok_or(BlockProductionError::MissingFinalizedBlock(finalized_root))?
.message()
.body()
.execution_payload()
.map(|ep| ep.block_hash)
};
execution_layer execution_layer
.block_on(|execution_layer| { .block_on(|execution_layer| {
execution_layer.get_payload(parent_hash, timestamp, random) execution_layer.get_payload(
parent_hash,
timestamp,
random,
finalized_block_hash.unwrap_or_else(Hash256::zero),
)
}) })
.map_err(BlockProductionError::GetPayloadFailed) .map_err(BlockProductionError::GetPayloadFailed)
}; };
@ -3168,7 +3188,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
.attester_shuffling_decision_root(self.genesis_block_root, RelativeEpoch::Current); .attester_shuffling_decision_root(self.genesis_block_root, RelativeEpoch::Current);
// Used later for the execution engine. // Used later for the execution engine.
let new_head_execution_block_hash = new_head let new_head_execution_block_hash_opt = new_head
.beacon_block .beacon_block
.message() .message()
.body() .body()
@ -3404,7 +3424,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
} }
// If this is a post-merge block, update the execution layer. // If this is a post-merge block, update the execution layer.
if let Some(block_hash) = new_head_execution_block_hash { if let Some(new_head_execution_block_hash) = new_head_execution_block_hash_opt {
if is_merge_complete { if is_merge_complete {
let execution_layer = self let execution_layer = self
.execution_layer .execution_layer
@ -3420,7 +3440,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
execution_layer, execution_layer,
store, store,
new_finalized_checkpoint.root, new_finalized_checkpoint.root,
block_hash, new_head_execution_block_hash,
) )
.await .await
{ {
@ -3461,7 +3481,11 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
.unwrap_or_else(Hash256::zero); .unwrap_or_else(Hash256::zero);
execution_layer execution_layer
.forkchoice_updated(head_execution_block_hash, finalized_execution_block_hash) .notify_forkchoice_updated(
head_execution_block_hash,
finalized_execution_block_hash,
None,
)
.await .await
.map_err(Error::ExecutionForkChoiceUpdateFailed) .map_err(Error::ExecutionForkChoiceUpdateFailed)
} }

View File

@ -50,7 +50,7 @@ use crate::{
}, },
metrics, BeaconChain, BeaconChainError, BeaconChainTypes, metrics, BeaconChain, BeaconChainError, BeaconChainTypes,
}; };
use execution_layer::ExecutePayloadResponse; use execution_layer::ExecutePayloadResponseStatus;
use fork_choice::{ForkChoice, ForkChoiceStore, PayloadVerificationStatus}; use fork_choice::{ForkChoice, ForkChoiceStore, PayloadVerificationStatus};
use parking_lot::RwLockReadGuard; use parking_lot::RwLockReadGuard;
use proto_array::{Block as ProtoBlock, ExecutionStatus}; use proto_array::{Block as ProtoBlock, ExecutionStatus};
@ -1139,7 +1139,9 @@ impl<'a, T: BeaconChainTypes> FullyVerifiedBlock<'a, T> {
} }
// This is the soonest we can run these checks as they must be called AFTER per_slot_processing // This is the soonest we can run these checks as they must be called AFTER per_slot_processing
let (execute_payload_handle, payload_verification_status) = //
// TODO(merge): handle the latest_valid_hash of an invalid payload.
let (_latest_valid_hash, payload_verification_status) =
if is_execution_enabled(&state, block.message().body()) { if is_execution_enabled(&state, block.message().body()) {
let execution_layer = chain let execution_layer = chain
.execution_layer .execution_layer
@ -1159,15 +1161,15 @@ impl<'a, T: BeaconChainTypes> FullyVerifiedBlock<'a, T> {
.block_on(|execution_layer| execution_layer.execute_payload(execution_payload)); .block_on(|execution_layer| execution_layer.execute_payload(execution_payload));
match execute_payload_response { match execute_payload_response {
Ok((status, handle)) => match status { Ok((status, latest_valid_hash)) => match status {
ExecutePayloadResponse::Valid => { ExecutePayloadResponseStatus::Valid => {
(handle, PayloadVerificationStatus::Verified) (latest_valid_hash, PayloadVerificationStatus::Verified)
} }
ExecutePayloadResponse::Invalid => { ExecutePayloadResponseStatus::Invalid => {
return Err(ExecutionPayloadError::RejectedByExecutionEngine.into()); return Err(ExecutionPayloadError::RejectedByExecutionEngine.into());
} }
ExecutePayloadResponse::Syncing => { ExecutePayloadResponseStatus::Syncing => {
(handle, PayloadVerificationStatus::NotVerified) (latest_valid_hash, PayloadVerificationStatus::NotVerified)
} }
}, },
Err(_) => (None, PayloadVerificationStatus::NotVerified), Err(_) => (None, PayloadVerificationStatus::NotVerified),
@ -1274,15 +1276,6 @@ impl<'a, T: BeaconChainTypes> FullyVerifiedBlock<'a, T> {
}); });
} }
// If this block required an `executePayload` call to the execution node, inform it that the
// block is indeed valid.
//
// If the handle is dropped without explicitly declaring validity, an invalid message will
// be sent to the execution engine.
if let Some(execute_payload_handle) = execute_payload_handle {
execute_payload_handle.publish_consensus_valid();
}
Ok(Self { Ok(Self {
block, block,
block_root, block_root,

View File

@ -183,6 +183,8 @@ pub enum BlockProductionError {
ExecutionLayerMissing, ExecutionLayerMissing,
TerminalPoWBlockLookupFailed(execution_layer::Error), TerminalPoWBlockLookupFailed(execution_layer::Error),
GetPayloadFailed(execution_layer::Error), GetPayloadFailed(execution_layer::Error),
FailedToReadFinalizedBlock(store::Error),
MissingFinalizedBlock(Hash256),
} }
easy_from_to!(BlockProcessingError, BlockProductionError); easy_from_to!(BlockProcessingError, BlockProductionError);

View File

@ -4,11 +4,13 @@ use serde::{Deserialize, Serialize};
pub const LATEST_TAG: &str = "latest"; pub const LATEST_TAG: &str = "latest";
use crate::engines::ForkChoiceState;
pub use types::{Address, EthSpec, ExecutionPayload, Hash256, Uint256}; pub use types::{Address, EthSpec, ExecutionPayload, Hash256, Uint256};
pub mod http; pub mod http;
pub mod json_structures;
pub type PayloadId = u64; pub type PayloadId = [u8; 8];
#[derive(Debug)] #[derive(Debug)]
pub enum Error { pub enum Error {
@ -23,6 +25,7 @@ pub enum Error {
ExecutionBlockNotFound(Hash256), ExecutionBlockNotFound(Hash256),
ExecutionHeadBlockNotFound, ExecutionHeadBlockNotFound,
ParentHashEqualsBlockHash(Hash256), ParentHashEqualsBlockHash(Hash256),
PayloadIdUnavailable,
} }
impl From<reqwest::Error> for Error { impl From<reqwest::Error> for Error {
@ -52,50 +55,35 @@ pub trait EngineApi {
block_hash: Hash256, block_hash: Hash256,
) -> Result<Option<ExecutionBlock>, Error>; ) -> Result<Option<ExecutionBlock>, Error>;
async fn prepare_payload( async fn execute_payload_v1<T: EthSpec>(
&self,
parent_hash: Hash256,
timestamp: u64,
random: Hash256,
fee_recipient: Address,
) -> Result<PayloadId, Error>;
async fn execute_payload<T: EthSpec>(
&self, &self,
execution_payload: ExecutionPayload<T>, execution_payload: ExecutionPayload<T>,
) -> Result<ExecutePayloadResponse, Error>; ) -> Result<ExecutePayloadResponse, Error>;
async fn get_payload<T: EthSpec>( async fn get_payload_v1<T: EthSpec>(
&self, &self,
payload_id: PayloadId, payload_id: PayloadId,
) -> Result<ExecutionPayload<T>, Error>; ) -> Result<ExecutionPayload<T>, Error>;
async fn consensus_validated( async fn forkchoice_updated_v1(
&self, &self,
block_hash: Hash256, forkchoice_state: ForkChoiceState,
status: ConsensusStatus, payload_attributes: Option<PayloadAttributes>,
) -> Result<(), Error>; ) -> Result<ForkchoiceUpdatedResponse, Error>;
async fn forkchoice_updated(
&self,
head_block_hash: Hash256,
finalized_block_hash: Hash256,
) -> Result<(), Error>;
} }
#[derive(Clone, Copy, Debug, PartialEq, Serialize, Deserialize)] #[derive(Clone, Copy, Debug, PartialEq)]
#[serde(rename_all = "SCREAMING_SNAKE_CASE")] pub enum ExecutePayloadResponseStatus {
pub enum ExecutePayloadResponse {
Valid, Valid,
Invalid, Invalid,
Syncing, Syncing,
} }
#[derive(Clone, Copy, Debug, PartialEq, Serialize, Deserialize)] #[derive(Clone, Debug, PartialEq)]
#[serde(rename_all = "SCREAMING_SNAKE_CASE")] pub struct ExecutePayloadResponse {
pub enum ConsensusStatus { pub status: ExecutePayloadResponseStatus,
Valid, pub latest_valid_hash: Option<Hash256>,
Invalid, pub message: Option<String>,
} }
#[derive(Clone, Copy, Debug, PartialEq, Serialize)] #[derive(Clone, Copy, Debug, PartialEq, Serialize)]
@ -114,3 +102,21 @@ pub struct ExecutionBlock {
pub parent_hash: Hash256, pub parent_hash: Hash256,
pub total_difficulty: Uint256, pub total_difficulty: Uint256,
} }
#[derive(Clone, Copy, Debug)]
pub struct PayloadAttributes {
pub timestamp: u64,
pub random: Hash256,
pub fee_recipient: Address,
}
#[derive(Clone, Copy, Debug, PartialEq)]
pub enum ForkchoiceUpdatedResponseStatus {
Success,
Syncing,
}
#[derive(Clone, Debug, PartialEq)]
pub struct ForkchoiceUpdatedResponse {
pub status: ForkchoiceUpdatedResponseStatus,
pub payload_id: Option<PayloadId>,
}

View File

@ -1,14 +1,15 @@
//! Contains an implementation of `EngineAPI` using the JSON-RPC API via HTTP. //! Contains an implementation of `EngineAPI` using the JSON-RPC API via HTTP.
use super::*; use super::*;
use crate::json_structures::*;
use async_trait::async_trait; use async_trait::async_trait;
use eth1::http::EIP155_ERROR_STR; use eth1::http::EIP155_ERROR_STR;
use reqwest::header::CONTENT_TYPE; use reqwest::header::CONTENT_TYPE;
use sensitive_url::SensitiveUrl; use sensitive_url::SensitiveUrl;
use serde::{de::DeserializeOwned, Deserialize, Serialize}; use serde::de::DeserializeOwned;
use serde_json::json; use serde_json::json;
use std::time::Duration; use std::time::Duration;
use types::{EthSpec, FixedVector, Transaction, Unsigned, VariableList}; use types::EthSpec;
pub use reqwest::Client; pub use reqwest::Client;
@ -26,19 +27,13 @@ pub const ETH_GET_BLOCK_BY_HASH_TIMEOUT: Duration = Duration::from_secs(1);
pub const ETH_SYNCING: &str = "eth_syncing"; pub const ETH_SYNCING: &str = "eth_syncing";
pub const ETH_SYNCING_TIMEOUT: Duration = Duration::from_millis(250); pub const ETH_SYNCING_TIMEOUT: Duration = Duration::from_millis(250);
pub const ENGINE_PREPARE_PAYLOAD: &str = "engine_preparePayload"; pub const ENGINE_EXECUTE_PAYLOAD_V1: &str = "engine_executePayloadV1";
pub const ENGINE_PREPARE_PAYLOAD_TIMEOUT: Duration = Duration::from_millis(500);
pub const ENGINE_EXECUTE_PAYLOAD: &str = "engine_executePayload";
pub const ENGINE_EXECUTE_PAYLOAD_TIMEOUT: Duration = Duration::from_secs(2); pub const ENGINE_EXECUTE_PAYLOAD_TIMEOUT: Duration = Duration::from_secs(2);
pub const ENGINE_GET_PAYLOAD: &str = "engine_getPayload"; pub const ENGINE_GET_PAYLOAD_V1: &str = "engine_getPayloadV1";
pub const ENGINE_GET_PAYLOAD_TIMEOUT: Duration = Duration::from_secs(2); pub const ENGINE_GET_PAYLOAD_TIMEOUT: Duration = Duration::from_secs(2);
pub const ENGINE_CONSENSUS_VALIDATED: &str = "engine_consensusValidated"; pub const ENGINE_FORKCHOICE_UPDATED_V1: &str = "engine_forkchoiceUpdatedV1";
pub const ENGINE_CONSENSUS_VALIDATED_TIMEOUT: Duration = Duration::from_millis(500);
pub const ENGINE_FORKCHOICE_UPDATED: &str = "engine_forkchoiceUpdated";
pub const ENGINE_FORKCHOICE_UPDATED_TIMEOUT: Duration = Duration::from_millis(500); pub const ENGINE_FORKCHOICE_UPDATED_TIMEOUT: Duration = Duration::from_millis(500);
pub struct HttpJsonRpc { pub struct HttpJsonRpc {
@ -138,334 +133,55 @@ impl EngineApi for HttpJsonRpc {
.await .await
} }
async fn prepare_payload( async fn execute_payload_v1<T: EthSpec>(
&self,
parent_hash: Hash256,
timestamp: u64,
random: Hash256,
fee_recipient: Address,
) -> Result<PayloadId, Error> {
let params = json!([JsonPreparePayloadRequest {
parent_hash,
timestamp,
random,
fee_recipient
}]);
let response: JsonPayloadIdResponse = self
.rpc_request(
ENGINE_PREPARE_PAYLOAD,
params,
ENGINE_PREPARE_PAYLOAD_TIMEOUT,
)
.await?;
Ok(response.payload_id)
}
async fn execute_payload<T: EthSpec>(
&self, &self,
execution_payload: ExecutionPayload<T>, execution_payload: ExecutionPayload<T>,
) -> Result<ExecutePayloadResponse, Error> { ) -> Result<ExecutePayloadResponse, Error> {
let params = json!([JsonExecutionPayload::from(execution_payload)]); let params = json!([JsonExecutionPayloadV1::from(execution_payload)]);
let result: ExecutePayloadResponseWrapper = self let response: JsonExecutePayloadV1Response = self
.rpc_request( .rpc_request(
ENGINE_EXECUTE_PAYLOAD, ENGINE_EXECUTE_PAYLOAD_V1,
params, params,
ENGINE_EXECUTE_PAYLOAD_TIMEOUT, ENGINE_EXECUTE_PAYLOAD_TIMEOUT,
) )
.await?; .await?;
Ok(result.status) Ok(response.into())
} }
async fn get_payload<T: EthSpec>( async fn get_payload_v1<T: EthSpec>(
&self, &self,
payload_id: PayloadId, payload_id: PayloadId,
) -> Result<ExecutionPayload<T>, Error> { ) -> Result<ExecutionPayload<T>, Error> {
let params = json!([JsonPayloadIdRequest { payload_id }]); let params = json!([JsonPayloadIdRequest::from(payload_id)]);
let response: JsonExecutionPayload<T> = self let response: JsonExecutionPayloadV1<T> = self
.rpc_request(ENGINE_GET_PAYLOAD, params, ENGINE_GET_PAYLOAD_TIMEOUT) .rpc_request(ENGINE_GET_PAYLOAD_V1, params, ENGINE_GET_PAYLOAD_TIMEOUT)
.await?; .await?;
Ok(ExecutionPayload::from(response)) Ok(response.into())
} }
async fn consensus_validated( async fn forkchoice_updated_v1(
&self, &self,
block_hash: Hash256, forkchoice_state: ForkChoiceState,
status: ConsensusStatus, payload_attributes: Option<PayloadAttributes>,
) -> Result<(), Error> { ) -> Result<ForkchoiceUpdatedResponse, Error> {
let params = json!([JsonConsensusValidatedRequest { block_hash, status }]); let params = json!([
JsonForkChoiceStateV1::from(forkchoice_state),
payload_attributes.map(JsonPayloadAttributesV1::from)
]);
self.rpc_request( let response: JsonForkchoiceUpdatedV1Response = self
ENGINE_CONSENSUS_VALIDATED, .rpc_request(
params, ENGINE_FORKCHOICE_UPDATED_V1,
ENGINE_CONSENSUS_VALIDATED_TIMEOUT, params,
) ENGINE_FORKCHOICE_UPDATED_TIMEOUT,
.await )
} .await?;
async fn forkchoice_updated( Ok(response.into())
&self,
head_block_hash: Hash256,
finalized_block_hash: Hash256,
) -> Result<(), Error> {
let params = json!([JsonForkChoiceUpdatedRequest {
head_block_hash,
finalized_block_hash
}]);
self.rpc_request(
ENGINE_FORKCHOICE_UPDATED,
params,
ENGINE_FORKCHOICE_UPDATED_TIMEOUT,
)
.await
}
}
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
struct JsonRequestBody<'a> {
jsonrpc: &'a str,
method: &'a str,
params: serde_json::Value,
id: u32,
}
#[derive(Debug, PartialEq, Serialize, Deserialize)]
struct JsonError {
code: i64,
message: String,
}
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
struct JsonResponseBody {
jsonrpc: String,
#[serde(default)]
error: Option<JsonError>,
#[serde(default)]
result: serde_json::Value,
id: u32,
}
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct JsonPreparePayloadRequest {
pub parent_hash: Hash256,
#[serde(with = "eth2_serde_utils::u64_hex_be")]
pub timestamp: u64,
pub random: Hash256,
pub fee_recipient: Address,
}
/// On the request, just provide the `payload_id`, without the object wrapper (transparent).
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(transparent, rename_all = "camelCase")]
pub struct JsonPayloadIdRequest {
#[serde(with = "eth2_serde_utils::u64_hex_be")]
pub payload_id: u64,
}
/// On the response, expect without the object wrapper (non-transparent).
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct JsonPayloadIdResponse {
#[serde(with = "eth2_serde_utils::u64_hex_be")]
pub payload_id: u64,
}
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct ExecutePayloadResponseWrapper {
pub status: ExecutePayloadResponse,
}
#[derive(Debug, PartialEq, Default, Serialize, Deserialize)]
#[serde(bound = "T: EthSpec", rename_all = "camelCase")]
pub struct JsonExecutionPayload<T: EthSpec> {
pub parent_hash: Hash256,
pub coinbase: Address,
pub state_root: Hash256,
pub receipt_root: Hash256,
#[serde(with = "serde_logs_bloom")]
pub logs_bloom: FixedVector<u8, T::BytesPerLogsBloom>,
pub random: Hash256,
#[serde(with = "eth2_serde_utils::u64_hex_be")]
pub block_number: u64,
#[serde(with = "eth2_serde_utils::u64_hex_be")]
pub gas_limit: u64,
#[serde(with = "eth2_serde_utils::u64_hex_be")]
pub gas_used: u64,
#[serde(with = "eth2_serde_utils::u64_hex_be")]
pub timestamp: u64,
#[serde(with = "ssz_types::serde_utils::hex_var_list")]
pub extra_data: VariableList<u8, T::MaxExtraDataBytes>,
pub base_fee_per_gas: Uint256,
pub block_hash: Hash256,
#[serde(with = "serde_transactions")]
pub transactions:
VariableList<Transaction<T::MaxBytesPerTransaction>, T::MaxTransactionsPerPayload>,
}
impl<T: EthSpec> From<ExecutionPayload<T>> for JsonExecutionPayload<T> {
fn from(e: ExecutionPayload<T>) -> Self {
Self {
parent_hash: e.parent_hash,
coinbase: e.coinbase,
state_root: e.state_root,
receipt_root: e.receipt_root,
logs_bloom: e.logs_bloom,
random: e.random,
block_number: e.block_number,
gas_limit: e.gas_limit,
gas_used: e.gas_used,
timestamp: e.timestamp,
extra_data: e.extra_data,
base_fee_per_gas: e.base_fee_per_gas,
block_hash: e.block_hash,
transactions: e.transactions,
}
}
}
impl<T: EthSpec> From<JsonExecutionPayload<T>> for ExecutionPayload<T> {
fn from(e: JsonExecutionPayload<T>) -> Self {
Self {
parent_hash: e.parent_hash,
coinbase: e.coinbase,
state_root: e.state_root,
receipt_root: e.receipt_root,
logs_bloom: e.logs_bloom,
random: e.random,
block_number: e.block_number,
gas_limit: e.gas_limit,
gas_used: e.gas_used,
timestamp: e.timestamp,
extra_data: e.extra_data,
base_fee_per_gas: e.base_fee_per_gas,
block_hash: e.block_hash,
transactions: e.transactions,
}
}
}
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct JsonConsensusValidatedRequest {
pub block_hash: Hash256,
pub status: ConsensusStatus,
}
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct JsonForkChoiceUpdatedRequest {
pub head_block_hash: Hash256,
pub finalized_block_hash: Hash256,
}
/// Serializes the `logs_bloom` field of an `ExecutionPayload`.
pub mod serde_logs_bloom {
use super::*;
use eth2_serde_utils::hex::PrefixedHexVisitor;
use serde::{Deserializer, Serializer};
pub fn serialize<S, U>(bytes: &FixedVector<u8, U>, serializer: S) -> Result<S::Ok, S::Error>
where
S: Serializer,
U: Unsigned,
{
let mut hex_string: String = "0x".to_string();
hex_string.push_str(&hex::encode(&bytes[..]));
serializer.serialize_str(&hex_string)
}
pub fn deserialize<'de, D, U>(deserializer: D) -> Result<FixedVector<u8, U>, D::Error>
where
D: Deserializer<'de>,
U: Unsigned,
{
let vec = deserializer.deserialize_string(PrefixedHexVisitor)?;
FixedVector::new(vec)
.map_err(|e| serde::de::Error::custom(format!("invalid logs bloom: {:?}", e)))
}
}
/// Serializes the `transactions` field of an `ExecutionPayload`.
pub mod serde_transactions {
use super::*;
use eth2_serde_utils::hex;
use serde::ser::SerializeSeq;
use serde::{de, Deserializer, Serializer};
use std::marker::PhantomData;
type Value<M, N> = VariableList<Transaction<M>, N>;
#[derive(Default)]
pub struct ListOfBytesListVisitor<M, N> {
_phantom_m: PhantomData<M>,
_phantom_n: PhantomData<N>,
}
impl<'a, M: Unsigned, N: Unsigned> serde::de::Visitor<'a> for ListOfBytesListVisitor<M, N> {
type Value = Value<M, N>;
fn expecting(&self, formatter: &mut std::fmt::Formatter) -> std::fmt::Result {
write!(formatter, "a list of 0x-prefixed byte lists")
}
fn visit_seq<A>(self, mut seq: A) -> Result<Self::Value, A::Error>
where
A: serde::de::SeqAccess<'a>,
{
let mut outer = VariableList::default();
while let Some(val) = seq.next_element::<String>()? {
let inner_vec = hex::decode(&val).map_err(de::Error::custom)?;
let transaction = VariableList::new(inner_vec).map_err(|e| {
serde::de::Error::custom(format!("transaction too large: {:?}", e))
})?;
outer.push(transaction).map_err(|e| {
serde::de::Error::custom(format!("too many transactions: {:?}", e))
})?;
}
Ok(outer)
}
}
pub fn serialize<S, M: Unsigned, N: Unsigned>(
value: &Value<M, N>,
serializer: S,
) -> Result<S::Ok, S::Error>
where
S: Serializer,
{
let mut seq = serializer.serialize_seq(Some(value.len()))?;
for transaction in value {
// It's important to match on the inner values of the transaction. Serializing the
// entire `Transaction` will result in appending the SSZ union prefix byte. The
// execution node does not want that.
let hex = hex::encode(&transaction[..]);
seq.serialize_element(&hex)?;
}
seq.end()
}
pub fn deserialize<'de, D, M: Unsigned, N: Unsigned>(
deserializer: D,
) -> Result<Value<M, N>, D::Error>
where
D: Deserializer<'de>,
{
let visitor: ListOfBytesListVisitor<M, N> = <_>::default();
deserializer.deserialize_any(visitor)
} }
} }
@ -476,7 +192,7 @@ mod test {
use std::future::Future; use std::future::Future;
use std::str::FromStr; use std::str::FromStr;
use std::sync::Arc; use std::sync::Arc;
use types::MainnetEthSpec; use types::{MainnetEthSpec, Transaction, Unsigned, VariableList};
struct Tester { struct Tester {
server: MockServer<MainnetEthSpec>, server: MockServer<MainnetEthSpec>,
@ -547,6 +263,8 @@ mod test {
const ADDRESS_00: &str = "0x0000000000000000000000000000000000000000"; const ADDRESS_00: &str = "0x0000000000000000000000000000000000000000";
const ADDRESS_01: &str = "0x0101010101010101010101010101010101010101"; const ADDRESS_01: &str = "0x0101010101010101010101010101010101010101";
const JSON_NULL: serde_json::Value = serde_json::Value::Null;
const LOGS_BLOOM_00: &str = "0x00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000";
const LOGS_BLOOM_01: &str = "0x01010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101"; const LOGS_BLOOM_01: &str = "0x01010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101";
fn encode_transactions<E: EthSpec>( fn encode_transactions<E: EthSpec>(
@ -555,7 +273,7 @@ mod test {
E::MaxTransactionsPerPayload, E::MaxTransactionsPerPayload,
>, >,
) -> Result<serde_json::Value, serde_json::Error> { ) -> Result<serde_json::Value, serde_json::Error> {
let ep: JsonExecutionPayload<E> = JsonExecutionPayload { let ep: JsonExecutionPayloadV1<E> = JsonExecutionPayloadV1 {
transactions, transactions,
..<_>::default() ..<_>::default()
}; };
@ -588,7 +306,7 @@ mod test {
json.as_object_mut() json.as_object_mut()
.unwrap() .unwrap()
.insert("transactions".into(), transactions); .insert("transactions".into(), transactions);
let ep: JsonExecutionPayload<E> = serde_json::from_value(json)?; let ep: JsonExecutionPayloadV1<E> = serde_json::from_value(json)?;
Ok(ep.transactions) Ok(ep.transactions)
} }
@ -713,28 +431,38 @@ mod test {
} }
#[tokio::test] #[tokio::test]
async fn prepare_payload_request() { async fn forkchoice_updated_v1_with_payload_attributes_request() {
Tester::new() Tester::new()
.assert_request_equals( .assert_request_equals(
|client| async move { |client| async move {
let _ = client let _ = client
.prepare_payload( .forkchoice_updated_v1(
Hash256::repeat_byte(0), ForkChoiceState {
42, head_block_hash: Hash256::repeat_byte(1),
Hash256::repeat_byte(1), safe_block_hash: Hash256::repeat_byte(1),
Address::repeat_byte(0), finalized_block_hash: Hash256::zero(),
},
Some(PayloadAttributes {
timestamp: 5,
random: Hash256::zero(),
fee_recipient: Address::repeat_byte(0),
}),
) )
.await; .await;
}, },
json!({ json!({
"id": STATIC_ID, "id": STATIC_ID,
"jsonrpc": JSONRPC_VERSION, "jsonrpc": JSONRPC_VERSION,
"method": ENGINE_PREPARE_PAYLOAD, "method": ENGINE_FORKCHOICE_UPDATED_V1,
"params": [{ "params": [{
"parentHash": HASH_00, "headBlockHash": HASH_01,
"timestamp": "0x2a", "safeBlockHash": HASH_01,
"random": HASH_01, "finalizedBlockHash": HASH_00,
"feeRecipient": ADDRESS_00, },
{
"timestamp":"0x5",
"random": HASH_00,
"feeRecipient": ADDRESS_00
}] }]
}), }),
) )
@ -742,29 +470,29 @@ mod test {
} }
#[tokio::test] #[tokio::test]
async fn get_payload_request() { async fn get_payload_v1_request() {
Tester::new() Tester::new()
.assert_request_equals( .assert_request_equals(
|client| async move { |client| async move {
let _ = client.get_payload::<MainnetEthSpec>(42).await; let _ = client.get_payload_v1::<MainnetEthSpec>([42; 8]).await;
}, },
json!({ json!({
"id": STATIC_ID, "id": STATIC_ID,
"jsonrpc": JSONRPC_VERSION, "jsonrpc": JSONRPC_VERSION,
"method": ENGINE_GET_PAYLOAD, "method": ENGINE_GET_PAYLOAD_V1,
"params": ["0x2a"] "params": ["0x2a2a2a2a2a2a2a2a"]
}), }),
) )
.await; .await;
} }
#[tokio::test] #[tokio::test]
async fn execute_payload_request() { async fn execute_payload_v1_request() {
Tester::new() Tester::new()
.assert_request_equals( .assert_request_equals(
|client| async move { |client| async move {
let _ = client let _ = client
.execute_payload::<MainnetEthSpec>(ExecutionPayload { .execute_payload_v1::<MainnetEthSpec>(ExecutionPayload {
parent_hash: Hash256::repeat_byte(0), parent_hash: Hash256::repeat_byte(0),
coinbase: Address::repeat_byte(1), coinbase: Address::repeat_byte(1),
state_root: Hash256::repeat_byte(1), state_root: Hash256::repeat_byte(1),
@ -785,7 +513,7 @@ mod test {
json!({ json!({
"id": STATIC_ID, "id": STATIC_ID,
"jsonrpc": JSONRPC_VERSION, "jsonrpc": JSONRPC_VERSION,
"method": ENGINE_EXECUTE_PAYLOAD, "method": ENGINE_EXECUTE_PAYLOAD_V1,
"params": [{ "params": [{
"parentHash": HASH_00, "parentHash": HASH_00,
"coinbase": ADDRESS_01, "coinbase": ADDRESS_01,
@ -808,64 +536,47 @@ mod test {
} }
#[tokio::test] #[tokio::test]
async fn consensus_validated_request() { async fn forkchoice_updated_v1_request() {
Tester::new() Tester::new()
.assert_request_equals( .assert_request_equals(
|client| async move { |client| async move {
let _ = client let _ = client
.consensus_validated(Hash256::repeat_byte(0), ConsensusStatus::Valid) .forkchoice_updated_v1(
ForkChoiceState {
head_block_hash: Hash256::repeat_byte(0),
safe_block_hash: Hash256::repeat_byte(0),
finalized_block_hash: Hash256::repeat_byte(1),
},
None,
)
.await; .await;
}, },
json!({ json!({
"id": STATIC_ID, "id": STATIC_ID,
"jsonrpc": JSONRPC_VERSION, "jsonrpc": JSONRPC_VERSION,
"method": ENGINE_CONSENSUS_VALIDATED, "method": ENGINE_FORKCHOICE_UPDATED_V1,
"params": [{ "params": [{
"blockHash": HASH_00, "headBlockHash": HASH_00,
"status": "VALID", "safeBlockHash": HASH_00,
}] "finalizedBlockHash": HASH_01,
}), }, JSON_NULL]
)
.await
.assert_request_equals(
|client| async move {
let _ = client
.consensus_validated(Hash256::repeat_byte(1), ConsensusStatus::Invalid)
.await;
},
json!({
"id": STATIC_ID,
"jsonrpc": JSONRPC_VERSION,
"method": ENGINE_CONSENSUS_VALIDATED,
"params": [{
"blockHash": HASH_01,
"status": "INVALID",
}]
}), }),
) )
.await; .await;
} }
#[tokio::test] fn str_to_payload_id(s: &str) -> PayloadId {
async fn forkchoice_updated_request() { serde_json::from_str::<TransparentJsonPayloadId>(&format!("\"{}\"", s))
Tester::new() .unwrap()
.assert_request_equals( .into()
|client| async move { }
let _ = client
.forkchoice_updated(Hash256::repeat_byte(0), Hash256::repeat_byte(1)) #[test]
.await; fn str_payload_id() {
}, assert_eq!(
json!({ str_to_payload_id("0x002a2a2a2a2a2a01"),
"id": STATIC_ID, [0, 42, 42, 42, 42, 42, 42, 1]
"jsonrpc": JSONRPC_VERSION, );
"method": ENGINE_FORKCHOICE_UPDATED,
"params": [{
"headBlockHash": HASH_00,
"finalizedBlockHash": HASH_01,
}]
}),
)
.await;
} }
/// Test vectors provided by Geth: /// Test vectors provided by Geth:
@ -877,70 +588,143 @@ mod test {
async fn geth_test_vectors() { async fn geth_test_vectors() {
Tester::new() Tester::new()
.assert_request_equals( .assert_request_equals(
// engine_forkchoiceUpdatedV1 (prepare payload) REQUEST validation
|client| async move { |client| async move {
let _ = client let _ = client
.prepare_payload( .forkchoice_updated_v1(
Hash256::from_str("0xa0513a503d5bd6e89a144c3268e5b7e9da9dbf63df125a360e3950a7d0d67131").unwrap(), ForkChoiceState {
5, head_block_hash: Hash256::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
Hash256::zero(), safe_block_hash: Hash256::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
Address::zero(), finalized_block_hash: Hash256::zero(),
},
Some(PayloadAttributes {
timestamp: 5,
random: Hash256::zero(),
fee_recipient: Address::from_str("0xa94f5374fce5edbc8e2a8697c15331677e6ebf0b").unwrap(),
})
) )
.await; .await;
}, },
serde_json::from_str(r#"{"jsonrpc":"2.0","method":"engine_preparePayload","params":[{"parentHash":"0xa0513a503d5bd6e89a144c3268e5b7e9da9dbf63df125a360e3950a7d0d67131", "timestamp":"0x5", "random":"0x0000000000000000000000000000000000000000000000000000000000000000", "feeRecipient":"0x0000000000000000000000000000000000000000"}],"id": 1}"#).unwrap() json!({
"id": STATIC_ID,
"jsonrpc": JSONRPC_VERSION,
"method": ENGINE_FORKCHOICE_UPDATED_V1,
"params": [{
"headBlockHash": "0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a",
"safeBlockHash": "0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a",
"finalizedBlockHash": HASH_00,
},
{
"timestamp":"0x5",
"random": HASH_00,
"feeRecipient":"0xa94f5374fce5edbc8e2a8697c15331677e6ebf0b"
}]
})
) )
.await .await
.with_preloaded_responses( .with_preloaded_responses(
vec![serde_json::from_str(r#"{"jsonrpc":"2.0","id":1,"result":{"payloadId":"0x0"}}"#).unwrap()], // engine_forkchoiceUpdatedV1 (prepare payload) RESPONSE validation
//
// NOTE THIS HAD TO BE MODIFIED FROM ORIGINAL RESPONSE
// {
// "jsonrpc":"2.0",
// "id":67,
// "result":{
// "status":"VALID", // <- This must be SUCCESS
// "payloadId":"0xa247243752eb10b4"
// }
// }
// see spec for engine_forkchoiceUpdatedV1 response:
// https://github.com/ethereum/execution-apis/blob/v1.0.0-alpha.4/src/engine/specification.md#response-1
vec![json!({
"id": STATIC_ID,
"jsonrpc": JSONRPC_VERSION,
"result": {
"status": "SUCCESS",
"payloadId": "0xa247243752eb10b4"
}
})],
|client| async move { |client| async move {
let payload_id = client let response = client
.prepare_payload( .forkchoice_updated_v1(
Hash256::from_str("0xa0513a503d5bd6e89a144c3268e5b7e9da9dbf63df125a360e3950a7d0d67131").unwrap(), ForkChoiceState {
5, head_block_hash: Hash256::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
Hash256::zero(), safe_block_hash: Hash256::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
Address::zero(), finalized_block_hash: Hash256::zero(),
},
Some(PayloadAttributes {
timestamp: 5,
random: Hash256::zero(),
fee_recipient: Address::from_str("0xa94f5374fce5edbc8e2a8697c15331677e6ebf0b").unwrap(),
})
) )
.await .await
.unwrap(); .unwrap();
assert_eq!(response, ForkchoiceUpdatedResponse {
assert_eq!(payload_id, 0); status: ForkchoiceUpdatedResponseStatus::Success,
payload_id:
Some(str_to_payload_id("0xa247243752eb10b4")),
});
}, },
) )
.await .await
.assert_request_equals( .assert_request_equals(
// engine_getPayloadV1 REQUEST validation
|client| async move { |client| async move {
let _ = client let _ = client
.get_payload::<MainnetEthSpec>(0) .get_payload_v1::<MainnetEthSpec>(str_to_payload_id("0xa247243752eb10b4"))
.await; .await;
}, },
serde_json::from_str(r#"{"jsonrpc":"2.0","method":"engine_getPayload","params":["0x0"],"id":1}"#).unwrap() json!({
"id": STATIC_ID,
"jsonrpc": JSONRPC_VERSION,
"method": ENGINE_GET_PAYLOAD_V1,
"params": ["0xa247243752eb10b4"]
})
) )
.await .await
.with_preloaded_responses( .with_preloaded_responses(
// Note: this response has been modified due to errors in the test vectors: // engine_getPayloadV1 RESPONSE validation
// vec![json!({
// https://github.com/ethereum/go-ethereum/pull/23607#issuecomment-930668512 "jsonrpc":JSONRPC_VERSION,
vec![serde_json::from_str(r#"{"jsonrpc":"2.0","id":67,"result":{"blockHash":"0xb084c10440f05f5a23a55d1d7ebcb1b3892935fb56f23cdc9a7f42c348eed174","parentHash":"0xa0513a503d5bd6e89a144c3268e5b7e9da9dbf63df125a360e3950a7d0d67131","coinbase":"0xa94f5374fce5edbc8e2a8697c15331677e6ebf0b","stateRoot":"0xca3149fa9e37db08d1cd49c9061db1002ef1cd58db2210f2115c8c989b2bdf45","receiptRoot":"0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421","logsBloom":"0x00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000","random":"0x0000000000000000000000000000000000000000000000000000000000000000","blockNumber":"0x1","gasLimit":"0x989680","gasUsed":"0x0","timestamp":"0x5","extraData":"0x","baseFeePerGas":"0x0","transactions":[]}}"#).unwrap()], "id":STATIC_ID,
"result":{
"parentHash":"0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a",
"coinbase":"0xa94f5374fce5edbc8e2a8697c15331677e6ebf0b",
"stateRoot":"0xca3149fa9e37db08d1cd49c9061db1002ef1cd58db2210f2115c8c989b2bdf45",
"receiptRoot":"0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421",
"logsBloom": LOGS_BLOOM_00,
"random": HASH_00,
"blockNumber":"0x1",
"gasLimit":"0x1c9c380",
"gasUsed":"0x0",
"timestamp":"0x5",
"extraData":"0x",
"baseFeePerGas":"0x7",
"blockHash":"0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858",
"transactions":[]
}
})],
|client| async move { |client| async move {
let payload = client let payload = client
.get_payload::<MainnetEthSpec>(0) .get_payload_v1::<MainnetEthSpec>(str_to_payload_id("0xa247243752eb10b4"))
.await .await
.unwrap(); .unwrap();
let expected = ExecutionPayload { let expected = ExecutionPayload {
parent_hash: Hash256::from_str("0xa0513a503d5bd6e89a144c3268e5b7e9da9dbf63df125a360e3950a7d0d67131").unwrap(), parent_hash: Hash256::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
coinbase: Address::from_str("0xa94f5374fce5edbc8e2a8697c15331677e6ebf0b").unwrap(), coinbase: Address::from_str("0xa94f5374fce5edbc8e2a8697c15331677e6ebf0b").unwrap(),
state_root: Hash256::from_str("0xca3149fa9e37db08d1cd49c9061db1002ef1cd58db2210f2115c8c989b2bdf45").unwrap(), state_root: Hash256::from_str("0xca3149fa9e37db08d1cd49c9061db1002ef1cd58db2210f2115c8c989b2bdf45").unwrap(),
receipt_root: Hash256::from_str("0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421").unwrap(), receipt_root: Hash256::from_str("0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421").unwrap(),
logs_bloom: vec![0; 256].into(), logs_bloom: vec![0; 256].into(),
random: Hash256::zero(), random: Hash256::zero(),
block_number: 1, block_number: 1,
gas_limit: 10000000, gas_limit: u64::from_str_radix("1c9c380",16).unwrap(),
gas_used: 0, gas_used: 0,
timestamp: 5, timestamp: 5,
extra_data: vec![].into(), extra_data: vec![].into(),
base_fee_per_gas: Uint256::from(0), base_fee_per_gas: Uint256::from(7),
block_hash: Hash256::from_str("0xb084c10440f05f5a23a55d1d7ebcb1b3892935fb56f23cdc9a7f42c348eed174").unwrap(), block_hash: Hash256::from_str("0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858").unwrap(),
transactions: vec![].into(), transactions: vec![].into(),
}; };
@ -949,96 +733,144 @@ mod test {
) )
.await .await
.assert_request_equals( .assert_request_equals(
// engine_executePayloadV1 REQUEST validation
|client| async move { |client| async move {
let _ = client let _ = client
.execute_payload::<MainnetEthSpec>(ExecutionPayload { .execute_payload_v1::<MainnetEthSpec>(ExecutionPayload {
parent_hash: Hash256::from_str("0xa0513a503d5bd6e89a144c3268e5b7e9da9dbf63df125a360e3950a7d0d67131").unwrap(), parent_hash: Hash256::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
coinbase: Address::from_str("0xa94f5374fce5edbc8e2a8697c15331677e6ebf0b").unwrap(), coinbase: Address::from_str("0xa94f5374fce5edbc8e2a8697c15331677e6ebf0b").unwrap(),
state_root: Hash256::from_str("0xca3149fa9e37db08d1cd49c9061db1002ef1cd58db2210f2115c8c989b2bdf45").unwrap(), state_root: Hash256::from_str("0xca3149fa9e37db08d1cd49c9061db1002ef1cd58db2210f2115c8c989b2bdf45").unwrap(),
receipt_root: Hash256::from_str("0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421").unwrap(), receipt_root: Hash256::from_str("0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421").unwrap(),
logs_bloom: vec![0; 256].into(), logs_bloom: vec![0; 256].into(),
random: Hash256::zero(), random: Hash256::zero(),
block_number: 1, block_number: 1,
gas_limit: 10000000, gas_limit: u64::from_str_radix("1c9c380",16).unwrap(),
gas_used: 0, gas_used: 0,
timestamp: 5, timestamp: 5,
extra_data: vec![].into(), extra_data: vec![].into(),
base_fee_per_gas: Uint256::from(0), base_fee_per_gas: Uint256::from(7),
block_hash: Hash256::from_str("0xb084c10440f05f5a23a55d1d7ebcb1b3892935fb56f23cdc9a7f42c348eed174").unwrap(), block_hash: Hash256::from_str("0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858").unwrap(),
transactions: vec![].into(), transactions: vec![].into(),
}) })
.await; .await;
}, },
// Note: I have renamed the `recieptsRoot` field to `recieptRoot` and `number` to `blockNumber` since I think json!({
// Geth has an issue. See: "id": STATIC_ID,
// "jsonrpc": JSONRPC_VERSION,
// https://github.com/ethereum/go-ethereum/pull/23607#issuecomment-930668512 "method": ENGINE_EXECUTE_PAYLOAD_V1,
serde_json::from_str(r#"{"jsonrpc":"2.0","method":"engine_executePayload","params":[{"blockHash":"0xb084c10440f05f5a23a55d1d7ebcb1b3892935fb56f23cdc9a7f42c348eed174","parentHash":"0xa0513a503d5bd6e89a144c3268e5b7e9da9dbf63df125a360e3950a7d0d67131","coinbase":"0xa94f5374fce5edbc8e2a8697c15331677e6ebf0b","stateRoot":"0xca3149fa9e37db08d1cd49c9061db1002ef1cd58db2210f2115c8c989b2bdf45","receiptRoot":"0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421","logsBloom":"0x00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000","random":"0x0000000000000000000000000000000000000000000000000000000000000000","blockNumber":"0x1","gasLimit":"0x989680","gasUsed":"0x0","timestamp":"0x5","extraData":"0x","baseFeePerGas":"0x0","transactions":[]}],"id":1}"#).unwrap() "params": [{
"parentHash":"0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a",
"coinbase":"0xa94f5374fce5edbc8e2a8697c15331677e6ebf0b",
"stateRoot":"0xca3149fa9e37db08d1cd49c9061db1002ef1cd58db2210f2115c8c989b2bdf45",
"receiptRoot":"0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421",
"logsBloom": LOGS_BLOOM_00,
"random": HASH_00,
"blockNumber":"0x1",
"gasLimit":"0x1c9c380",
"gasUsed":"0x0",
"timestamp":"0x5",
"extraData":"0x",
"baseFeePerGas":"0x7",
"blockHash":"0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858",
"transactions":[]
}],
})
) )
.await .await
.with_preloaded_responses( .with_preloaded_responses(
vec![serde_json::from_str(r#"{"jsonrpc":"2.0","id":67,"result":{"status":"VALID"}}"#).unwrap()], // engine_executePayloadV1 RESPONSE validation
//
// NOTE THIS HAD TO BE MODIFIED FROM ORIGINAL RESPONSE
// {
// "jsonrpc":"2.0",
// "id":67,
// "result":{
// "status":"SUCCESS", // <- This must be VALID
// "latestValidHash":"0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858"
// }
// }
// see spec for engine_executePayloadV1 response:
// https://github.com/ethereum/execution-apis/blob/v1.0.0-alpha.4/src/engine/specification.md#response
vec![json!({
"jsonrpc": JSONRPC_VERSION,
"id": STATIC_ID,
"result":{
"status":"VALID",
"latestValidHash":"0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858"
}
})],
|client| async move { |client| async move {
let response = client let response = client
.execute_payload::<MainnetEthSpec>(ExecutionPayload::default()) .execute_payload_v1::<MainnetEthSpec>(ExecutionPayload::default())
.await .await
.unwrap(); .unwrap();
assert_eq!(response, ExecutePayloadResponse::Valid); assert_eq!(response,
ExecutePayloadResponse {
status: ExecutePayloadResponseStatus::Valid,
latest_valid_hash: Some(Hash256::from_str("0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858").unwrap()),
message: None
}
);
}, },
) )
.await .await
.assert_request_equals( .assert_request_equals(
// engine_forkchoiceUpdatedV1 REQUEST validation
|client| async move { |client| async move {
let _ = client let _ = client
.consensus_validated( .forkchoice_updated_v1(
Hash256::from_str("0xb084c10440f05f5a23a55d1d7ebcb1b3892935fb56f23cdc9a7f42c348eed174").unwrap(), ForkChoiceState {
ConsensusStatus::Valid head_block_hash: Hash256::from_str("0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858").unwrap(),
safe_block_hash: Hash256::from_str("0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858").unwrap(),
finalized_block_hash: Hash256::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
},
None,
) )
.await; .await;
}, },
serde_json::from_str(r#"{"jsonrpc":"2.0","method":"engine_consensusValidated","params":[{"blockHash":"0xb084c10440f05f5a23a55d1d7ebcb1b3892935fb56f23cdc9a7f42c348eed174", "status":"VALID"}],"id":1}"#).unwrap() json!({
"jsonrpc": JSONRPC_VERSION,
"method": ENGINE_FORKCHOICE_UPDATED_V1,
"params": [
{
"headBlockHash":"0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858",
"safeBlockHash":"0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858",
"finalizedBlockHash":"0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a"
}, JSON_NULL],
"id": STATIC_ID
})
) )
.await .await
.with_preloaded_responses( .with_preloaded_responses(
vec![serde_json::from_str(r#"{"jsonrpc":"2.0","id":67,"result":null}"#).unwrap()], // engine_forkchoiceUpdatedV1 RESPONSE validation
|client| async move {
let _: () = client
.consensus_validated(
Hash256::zero(),
ConsensusStatus::Valid
)
.await
.unwrap();
},
)
.await
.assert_request_equals(
|client| async move {
let _ = client
.forkchoice_updated(
Hash256::from_str("0xb084c10440f05f5a23a55d1d7ebcb1b3892935fb56f23cdc9a7f42c348eed174").unwrap(),
Hash256::from_str("0xb084c10440f05f5a23a55d1d7ebcb1b3892935fb56f23cdc9a7f42c348eed174").unwrap(),
)
.await;
},
// Note: Geth incorrectly uses `engine_forkChoiceUpdated` (capital `C`). I've
// modified this vector to correct this. See:
// //
// https://github.com/ethereum/go-ethereum/pull/23607#issuecomment-930668512 // Note: this test was modified to provide `null` rather than `0x`. The geth vectors
serde_json::from_str(r#"{"jsonrpc":"2.0","method":"engine_forkchoiceUpdated","params":[{"headBlockHash":"0xb084c10440f05f5a23a55d1d7ebcb1b3892935fb56f23cdc9a7f42c348eed174", "finalizedBlockHash":"0xb084c10440f05f5a23a55d1d7ebcb1b3892935fb56f23cdc9a7f42c348eed174"}],"id":1}"#).unwrap() // are invalid.
) vec![json!({
.await "jsonrpc": JSONRPC_VERSION,
.with_preloaded_responses( "id": STATIC_ID,
vec![serde_json::from_str(r#"{"jsonrpc":"2.0","id":67,"result":null}"#).unwrap()], "result": {
"status":"SUCCESS",
"payloadId": serde_json::Value::Null
}
})],
|client| async move { |client| async move {
let _: () = client let response = client
.forkchoice_updated( .forkchoice_updated_v1(
Hash256::zero(), ForkChoiceState {
Hash256::zero(), head_block_hash: Hash256::from_str("0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858").unwrap(),
safe_block_hash: Hash256::from_str("0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858").unwrap(),
finalized_block_hash: Hash256::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
},
None,
) )
.await .await
.unwrap(); .unwrap();
assert_eq!(response, ForkchoiceUpdatedResponse {
status: ForkchoiceUpdatedResponseStatus::Success,
payload_id: None,
});
}, },
) )
.await; .await;

View File

@ -0,0 +1,476 @@
use super::*;
use serde::{Deserialize, Serialize};
use types::{EthSpec, FixedVector, Transaction, Unsigned, VariableList};
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct JsonRequestBody<'a> {
pub jsonrpc: &'a str,
pub method: &'a str,
pub params: serde_json::Value,
pub id: u32,
}
#[derive(Debug, PartialEq, Serialize, Deserialize)]
pub struct JsonError {
pub code: i64,
pub message: String,
}
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct JsonResponseBody {
pub jsonrpc: String,
#[serde(default)]
pub error: Option<JsonError>,
#[serde(default)]
pub result: serde_json::Value,
pub id: u32,
}
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(transparent)]
pub struct TransparentJsonPayloadId(#[serde(with = "eth2_serde_utils::bytes_8_hex")] pub PayloadId);
impl From<PayloadId> for TransparentJsonPayloadId {
fn from(id: PayloadId) -> Self {
Self(id)
}
}
impl From<TransparentJsonPayloadId> for PayloadId {
fn from(wrapper: TransparentJsonPayloadId) -> Self {
wrapper.0
}
}
/// On the request, use a transparent wrapper.
pub type JsonPayloadIdRequest = TransparentJsonPayloadId;
/// On the response, expect without the object wrapper (non-transparent).
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct JsonPayloadIdResponse {
#[serde(with = "eth2_serde_utils::bytes_8_hex")]
pub payload_id: PayloadId,
}
#[derive(Debug, PartialEq, Default, Serialize, Deserialize)]
#[serde(bound = "T: EthSpec", rename_all = "camelCase")]
pub struct JsonExecutionPayloadV1<T: EthSpec> {
pub parent_hash: Hash256,
pub coinbase: Address,
pub state_root: Hash256,
pub receipt_root: Hash256,
#[serde(with = "serde_logs_bloom")]
pub logs_bloom: FixedVector<u8, T::BytesPerLogsBloom>,
pub random: Hash256,
#[serde(with = "eth2_serde_utils::u64_hex_be")]
pub block_number: u64,
#[serde(with = "eth2_serde_utils::u64_hex_be")]
pub gas_limit: u64,
#[serde(with = "eth2_serde_utils::u64_hex_be")]
pub gas_used: u64,
#[serde(with = "eth2_serde_utils::u64_hex_be")]
pub timestamp: u64,
#[serde(with = "ssz_types::serde_utils::hex_var_list")]
pub extra_data: VariableList<u8, T::MaxExtraDataBytes>,
pub base_fee_per_gas: Uint256,
pub block_hash: Hash256,
#[serde(with = "serde_transactions")]
pub transactions:
VariableList<Transaction<T::MaxBytesPerTransaction>, T::MaxTransactionsPerPayload>,
}
impl<T: EthSpec> From<ExecutionPayload<T>> for JsonExecutionPayloadV1<T> {
fn from(e: ExecutionPayload<T>) -> Self {
// Use this verbose deconstruction pattern to ensure no field is left unused.
let ExecutionPayload {
parent_hash,
coinbase,
state_root,
receipt_root,
logs_bloom,
random,
block_number,
gas_limit,
gas_used,
timestamp,
extra_data,
base_fee_per_gas,
block_hash,
transactions,
} = e;
Self {
parent_hash,
coinbase,
state_root,
receipt_root,
logs_bloom,
random,
block_number,
gas_limit,
gas_used,
timestamp,
extra_data,
base_fee_per_gas,
block_hash,
transactions,
}
}
}
impl<T: EthSpec> From<JsonExecutionPayloadV1<T>> for ExecutionPayload<T> {
fn from(e: JsonExecutionPayloadV1<T>) -> Self {
// Use this verbose deconstruction pattern to ensure no field is left unused.
let JsonExecutionPayloadV1 {
parent_hash,
coinbase,
state_root,
receipt_root,
logs_bloom,
random,
block_number,
gas_limit,
gas_used,
timestamp,
extra_data,
base_fee_per_gas,
block_hash,
transactions,
} = e;
Self {
parent_hash,
coinbase,
state_root,
receipt_root,
logs_bloom,
random,
block_number,
gas_limit,
gas_used,
timestamp,
extra_data,
base_fee_per_gas,
block_hash,
transactions,
}
}
}
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct JsonPayloadAttributesV1 {
#[serde(with = "eth2_serde_utils::u64_hex_be")]
pub timestamp: u64,
pub random: Hash256,
pub fee_recipient: Address,
}
impl From<PayloadAttributes> for JsonPayloadAttributesV1 {
fn from(p: PayloadAttributes) -> Self {
// Use this verbose deconstruction pattern to ensure no field is left unused.
let PayloadAttributes {
timestamp,
random,
fee_recipient,
} = p;
Self {
timestamp,
random,
fee_recipient,
}
}
}
impl From<JsonPayloadAttributesV1> for PayloadAttributes {
fn from(j: JsonPayloadAttributesV1) -> Self {
// Use this verbose deconstruction pattern to ensure no field is left unused.
let JsonPayloadAttributesV1 {
timestamp,
random,
fee_recipient,
} = j;
Self {
timestamp,
random,
fee_recipient,
}
}
}
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct JsonForkChoiceStateV1 {
pub head_block_hash: Hash256,
pub safe_block_hash: Hash256,
pub finalized_block_hash: Hash256,
}
impl From<ForkChoiceState> for JsonForkChoiceStateV1 {
fn from(f: ForkChoiceState) -> Self {
// Use this verbose deconstruction pattern to ensure no field is left unused.
let ForkChoiceState {
head_block_hash,
safe_block_hash,
finalized_block_hash,
} = f;
Self {
head_block_hash,
safe_block_hash,
finalized_block_hash,
}
}
}
impl From<JsonForkChoiceStateV1> for ForkChoiceState {
fn from(j: JsonForkChoiceStateV1) -> Self {
// Use this verbose deconstruction pattern to ensure no field is left unused.
let JsonForkChoiceStateV1 {
head_block_hash,
safe_block_hash,
finalized_block_hash,
} = j;
Self {
head_block_hash,
safe_block_hash,
finalized_block_hash,
}
}
}
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "SCREAMING_SNAKE_CASE")]
pub enum JsonExecutePayloadV1ResponseStatus {
Valid,
Invalid,
Syncing,
}
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct JsonExecutePayloadV1Response {
pub status: JsonExecutePayloadV1ResponseStatus,
pub latest_valid_hash: Option<Hash256>,
pub message: Option<String>,
}
impl From<ExecutePayloadResponseStatus> for JsonExecutePayloadV1ResponseStatus {
fn from(e: ExecutePayloadResponseStatus) -> Self {
match e {
ExecutePayloadResponseStatus::Valid => JsonExecutePayloadV1ResponseStatus::Valid,
ExecutePayloadResponseStatus::Invalid => JsonExecutePayloadV1ResponseStatus::Invalid,
ExecutePayloadResponseStatus::Syncing => JsonExecutePayloadV1ResponseStatus::Syncing,
}
}
}
impl From<JsonExecutePayloadV1ResponseStatus> for ExecutePayloadResponseStatus {
fn from(j: JsonExecutePayloadV1ResponseStatus) -> Self {
match j {
JsonExecutePayloadV1ResponseStatus::Valid => ExecutePayloadResponseStatus::Valid,
JsonExecutePayloadV1ResponseStatus::Invalid => ExecutePayloadResponseStatus::Invalid,
JsonExecutePayloadV1ResponseStatus::Syncing => ExecutePayloadResponseStatus::Syncing,
}
}
}
impl From<ExecutePayloadResponse> for JsonExecutePayloadV1Response {
fn from(e: ExecutePayloadResponse) -> Self {
// Use this verbose deconstruction pattern to ensure no field is left unused.
let ExecutePayloadResponse {
status,
latest_valid_hash,
message,
} = e;
Self {
status: status.into(),
latest_valid_hash,
message,
}
}
}
impl From<JsonExecutePayloadV1Response> for ExecutePayloadResponse {
fn from(j: JsonExecutePayloadV1Response) -> Self {
// Use this verbose deconstruction pattern to ensure no field is left unused.
let JsonExecutePayloadV1Response {
status,
latest_valid_hash,
message,
} = j;
Self {
status: status.into(),
latest_valid_hash,
message,
}
}
}
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "SCREAMING_SNAKE_CASE")]
pub enum JsonForkchoiceUpdatedV1ResponseStatus {
Success,
Syncing,
}
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct JsonForkchoiceUpdatedV1Response {
pub status: JsonForkchoiceUpdatedV1ResponseStatus,
pub payload_id: Option<TransparentJsonPayloadId>,
}
impl From<JsonForkchoiceUpdatedV1ResponseStatus> for ForkchoiceUpdatedResponseStatus {
fn from(j: JsonForkchoiceUpdatedV1ResponseStatus) -> Self {
match j {
JsonForkchoiceUpdatedV1ResponseStatus::Success => {
ForkchoiceUpdatedResponseStatus::Success
}
JsonForkchoiceUpdatedV1ResponseStatus::Syncing => {
ForkchoiceUpdatedResponseStatus::Syncing
}
}
}
}
impl From<ForkchoiceUpdatedResponseStatus> for JsonForkchoiceUpdatedV1ResponseStatus {
fn from(f: ForkchoiceUpdatedResponseStatus) -> Self {
match f {
ForkchoiceUpdatedResponseStatus::Success => {
JsonForkchoiceUpdatedV1ResponseStatus::Success
}
ForkchoiceUpdatedResponseStatus::Syncing => {
JsonForkchoiceUpdatedV1ResponseStatus::Syncing
}
}
}
}
impl From<JsonForkchoiceUpdatedV1Response> for ForkchoiceUpdatedResponse {
fn from(j: JsonForkchoiceUpdatedV1Response) -> Self {
// Use this verbose deconstruction pattern to ensure no field is left unused.
let JsonForkchoiceUpdatedV1Response { status, payload_id } = j;
Self {
status: status.into(),
payload_id: payload_id.map(Into::into),
}
}
}
impl From<ForkchoiceUpdatedResponse> for JsonForkchoiceUpdatedV1Response {
fn from(f: ForkchoiceUpdatedResponse) -> Self {
// Use this verbose deconstruction pattern to ensure no field is left unused.
let ForkchoiceUpdatedResponse { status, payload_id } = f;
Self {
status: status.into(),
payload_id: payload_id.map(Into::into),
}
}
}
/// Serializes the `logs_bloom` field of an `ExecutionPayload`.
pub mod serde_logs_bloom {
use super::*;
use eth2_serde_utils::hex::PrefixedHexVisitor;
use serde::{Deserializer, Serializer};
pub fn serialize<S, U>(bytes: &FixedVector<u8, U>, serializer: S) -> Result<S::Ok, S::Error>
where
S: Serializer,
U: Unsigned,
{
let mut hex_string: String = "0x".to_string();
hex_string.push_str(&hex::encode(&bytes[..]));
serializer.serialize_str(&hex_string)
}
pub fn deserialize<'de, D, U>(deserializer: D) -> Result<FixedVector<u8, U>, D::Error>
where
D: Deserializer<'de>,
U: Unsigned,
{
let vec = deserializer.deserialize_string(PrefixedHexVisitor)?;
FixedVector::new(vec)
.map_err(|e| serde::de::Error::custom(format!("invalid logs bloom: {:?}", e)))
}
}
/// Serializes the `transactions` field of an `ExecutionPayload`.
pub mod serde_transactions {
use super::*;
use eth2_serde_utils::hex;
use serde::ser::SerializeSeq;
use serde::{de, Deserializer, Serializer};
use std::marker::PhantomData;
type Value<M, N> = VariableList<Transaction<M>, N>;
#[derive(Default)]
pub struct ListOfBytesListVisitor<M, N> {
_phantom_m: PhantomData<M>,
_phantom_n: PhantomData<N>,
}
impl<'a, M: Unsigned, N: Unsigned> serde::de::Visitor<'a> for ListOfBytesListVisitor<M, N> {
type Value = Value<M, N>;
fn expecting(&self, formatter: &mut std::fmt::Formatter) -> std::fmt::Result {
write!(formatter, "a list of 0x-prefixed byte lists")
}
fn visit_seq<A>(self, mut seq: A) -> Result<Self::Value, A::Error>
where
A: serde::de::SeqAccess<'a>,
{
let mut outer = VariableList::default();
while let Some(val) = seq.next_element::<String>()? {
let inner_vec = hex::decode(&val).map_err(de::Error::custom)?;
let transaction = VariableList::new(inner_vec).map_err(|e| {
serde::de::Error::custom(format!("transaction too large: {:?}", e))
})?;
outer.push(transaction).map_err(|e| {
serde::de::Error::custom(format!("too many transactions: {:?}", e))
})?;
}
Ok(outer)
}
}
pub fn serialize<S, M: Unsigned, N: Unsigned>(
value: &Value<M, N>,
serializer: S,
) -> Result<S::Ok, S::Error>
where
S: Serializer,
{
let mut seq = serializer.serialize_seq(Some(value.len()))?;
for transaction in value {
// It's important to match on the inner values of the transaction. Serializing the
// entire `Transaction` will result in appending the SSZ union prefix byte. The
// execution node does not want that.
let hex = hex::encode(&transaction[..]);
seq.serialize_element(&hex)?;
}
seq.end()
}
pub fn deserialize<'de, D, M: Unsigned, N: Unsigned>(
deserializer: D,
) -> Result<Value<M, N>, D::Error>
where
D: Deserializer<'de>,
{
let visitor: ListOfBytesListVisitor<M, N> = <_>::default();
deserializer.deserialize_any(visitor)
}
}

View File

@ -1,11 +1,17 @@
//! Provides generic behaviour for multiple execution engines, specifically fallback behaviour. //! Provides generic behaviour for multiple execution engines, specifically fallback behaviour.
use crate::engine_api::{EngineApi, Error as EngineApiError}; use crate::engine_api::{EngineApi, Error as EngineApiError, PayloadAttributes, PayloadId};
use futures::future::join_all; use futures::future::join_all;
use lru::LruCache;
use slog::{crit, debug, info, warn, Logger}; use slog::{crit, debug, info, warn, Logger};
use std::future::Future; use std::future::Future;
use tokio::sync::RwLock; use tokio::sync::{Mutex, RwLock};
use types::Hash256; use types::{Address, Hash256};
/// The number of payload IDs that will be stored for each `Engine`.
///
/// Since the size of each value is small (~100 bytes) a large number is used for safety.
const PAYLOAD_ID_LRU_CACHE_SIZE: usize = 512;
/// Stores the remembered state of a engine. /// Stores the remembered state of a engine.
#[derive(Copy, Clone, PartialEq)] #[derive(Copy, Clone, PartialEq)]
@ -16,8 +22,9 @@ enum EngineState {
} }
#[derive(Copy, Clone, PartialEq, Debug)] #[derive(Copy, Clone, PartialEq, Debug)]
pub struct ForkChoiceHead { pub struct ForkChoiceState {
pub head_block_hash: Hash256, pub head_block_hash: Hash256,
pub safe_block_hash: Hash256,
pub finalized_block_hash: Hash256, pub finalized_block_hash: Hash256,
} }
@ -37,10 +44,19 @@ impl Logging {
} }
} }
#[derive(Hash, PartialEq, std::cmp::Eq)]
struct PayloadIdCacheKey {
pub head_block_hash: Hash256,
pub timestamp: u64,
pub random: Hash256,
pub fee_recipient: Address,
}
/// An execution engine. /// An execution engine.
pub struct Engine<T> { pub struct Engine<T> {
pub id: String, pub id: String,
pub api: T, pub api: T,
payload_id_cache: Mutex<LruCache<PayloadIdCacheKey, PayloadId>>,
state: RwLock<EngineState>, state: RwLock<EngineState>,
} }
@ -50,16 +66,66 @@ impl<T> Engine<T> {
Self { Self {
id, id,
api, api,
payload_id_cache: Mutex::new(LruCache::new(PAYLOAD_ID_LRU_CACHE_SIZE)),
state: RwLock::new(EngineState::Offline), state: RwLock::new(EngineState::Offline),
} }
} }
pub async fn get_payload_id(
&self,
head_block_hash: Hash256,
timestamp: u64,
random: Hash256,
fee_recipient: Address,
) -> Option<PayloadId> {
self.payload_id_cache
.lock()
.await
.get(&PayloadIdCacheKey {
head_block_hash,
timestamp,
random,
fee_recipient,
})
.cloned()
}
}
impl<T: EngineApi> Engine<T> {
pub async fn notify_forkchoice_updated(
&self,
forkchoice_state: ForkChoiceState,
payload_attributes: Option<PayloadAttributes>,
log: &Logger,
) -> Result<Option<PayloadId>, EngineApiError> {
let response = self
.api
.forkchoice_updated_v1(forkchoice_state, payload_attributes)
.await?;
if let Some(payload_id) = response.payload_id {
if let Some(key) =
payload_attributes.map(|pa| PayloadIdCacheKey::new(&forkchoice_state, &pa))
{
self.payload_id_cache.lock().await.put(key, payload_id);
} else {
debug!(
log,
"Engine returned unexpected payload_id";
"payload_id" => ?payload_id
);
}
}
Ok(response.payload_id)
}
} }
/// Holds multiple execution engines and provides functionality for managing them in a fallback /// Holds multiple execution engines and provides functionality for managing them in a fallback
/// manner. /// manner.
pub struct Engines<T> { pub struct Engines<T> {
pub engines: Vec<Engine<T>>, pub engines: Vec<Engine<T>>,
pub latest_head: RwLock<Option<ForkChoiceHead>>, pub latest_forkchoice_state: RwLock<Option<ForkChoiceState>>,
pub log: Logger, pub log: Logger,
} }
@ -70,23 +136,30 @@ pub enum EngineError {
} }
impl<T: EngineApi> Engines<T> { impl<T: EngineApi> Engines<T> {
pub async fn set_latest_head(&self, latest_head: ForkChoiceHead) { async fn get_latest_forkchoice_state(&self) -> Option<ForkChoiceState> {
*self.latest_head.write().await = Some(latest_head); *self.latest_forkchoice_state.read().await
} }
async fn send_latest_head(&self, engine: &Engine<T>) { pub async fn set_latest_forkchoice_state(&self, state: ForkChoiceState) {
let latest_head: Option<ForkChoiceHead> = *self.latest_head.read().await; *self.latest_forkchoice_state.write().await = Some(state);
if let Some(head) = latest_head { }
async fn send_latest_forkchoice_state(&self, engine: &Engine<T>) {
let latest_forkchoice_state = self.get_latest_forkchoice_state().await;
if let Some(forkchoice_state) = latest_forkchoice_state {
info!( info!(
self.log, self.log,
"Issuing forkchoiceUpdated"; "Issuing forkchoiceUpdated";
"head" => ?head, "forkchoice_state" => ?forkchoice_state,
"id" => &engine.id, "id" => &engine.id,
); );
// For simplicity, payload attributes are never included in this call. It may be
// reasonable to include them in the future.
if let Err(e) = engine if let Err(e) = engine
.api .api
.forkchoice_updated(head.head_block_hash, head.finalized_block_hash) .forkchoice_updated_v1(forkchoice_state, None)
.await .await
{ {
debug!( debug!(
@ -132,8 +205,8 @@ impl<T: EngineApi> Engines<T> {
); );
} }
// Send the node our latest head. // Send the node our latest forkchoice_state.
self.send_latest_head(engine).await; self.send_latest_forkchoice_state(engine).await;
*state_lock = EngineState::Synced *state_lock = EngineState::Synced
} }
@ -146,8 +219,8 @@ impl<T: EngineApi> Engines<T> {
) )
} }
// Send the node our latest head, it may assist with syncing. // Send the node our latest forkchoice_state, it may assist with syncing.
self.send_latest_head(engine).await; self.send_latest_forkchoice_state(engine).await;
*state_lock = EngineState::Syncing *state_lock = EngineState::Syncing
} }
@ -312,3 +385,14 @@ impl<T: EngineApi> Engines<T> {
join_all(futures).await join_all(futures).await
} }
} }
impl PayloadIdCacheKey {
fn new(state: &ForkChoiceState, attributes: &PayloadAttributes) -> Self {
Self {
head_block_hash: state.head_block_hash,
timestamp: attributes.timestamp,
random: attributes.random,
fee_recipient: attributes.fee_recipient,
}
}
}

View File

@ -1,103 +0,0 @@
use crate::{ConsensusStatus, ExecutionLayer};
use slog::{crit, error, Logger};
use types::Hash256;
/// Provides a "handle" which should be returned after an `engine_executePayload` call.
///
/// This handle allows the holder to send a valid or invalid message to the execution nodes to
/// indicate the consensus verification status of `self.block_hash`.
///
/// Most notably, this `handle` will send an "invalid" message when it is dropped unless it has
/// already sent a "valid" or "invalid" message. This is to help ensure that any accidental
/// dropping of this handle results in an "invalid" message. Such dropping would be expected when a
/// block verification returns early with an error.
pub struct ExecutePayloadHandle {
pub(crate) block_hash: Hash256,
pub(crate) execution_layer: Option<ExecutionLayer>,
pub(crate) log: Logger,
}
impl ExecutePayloadHandle {
/// Publish a "valid" message to all nodes for `self.block_hash`.
pub fn publish_consensus_valid(mut self) {
self.publish_blocking(ConsensusStatus::Valid)
}
/// Publish an "invalid" message to all nodes for `self.block_hash`.
pub fn publish_consensus_invalid(mut self) {
self.publish_blocking(ConsensusStatus::Invalid)
}
/// Publish the `status` message to all nodes for `self.block_hash`.
pub async fn publish_async(&mut self, status: ConsensusStatus) {
if let Some(execution_layer) = self.execution_layer() {
publish(&execution_layer, self.block_hash, status, &self.log).await
}
}
/// Publishes a message, suitable for running in a non-async context.
fn publish_blocking(&mut self, status: ConsensusStatus) {
if let Some(execution_layer) = self.execution_layer() {
let log = &self.log.clone();
let block_hash = self.block_hash;
if let Err(e) = execution_layer.block_on(|execution_layer| async move {
publish(execution_layer, block_hash, status, log).await;
Ok(())
}) {
error!(
self.log,
"Failed to spawn payload status task";
"error" => ?e,
"block_hash" => ?block_hash,
"status" => ?status,
);
}
}
}
/// Takes `self.execution_layer`, it cannot be used to send another duplicate or conflicting
/// message. Creates a log message if such an attempt is made.
fn execution_layer(&mut self) -> Option<ExecutionLayer> {
let execution_layer = self.execution_layer.take();
if execution_layer.is_none() {
crit!(
self.log,
"Double usage of ExecutePayloadHandle";
"block_hash" => ?self.block_hash,
);
}
execution_layer
}
}
/// Publish a `status`, creating a log message if it fails.
async fn publish(
execution_layer: &ExecutionLayer,
block_hash: Hash256,
status: ConsensusStatus,
log: &Logger,
) {
if let Err(e) = execution_layer
.consensus_validated(block_hash, status)
.await
{
// TODO(paul): consider how to recover when we are temporarily unable to tell a node
// that the block was valid.
crit!(
log,
"Failed to update execution consensus status";
"error" => ?e,
"block_hash" => ?block_hash,
"status" => ?status,
);
}
}
/// See the struct-level documentation for the reasoning for this `Drop` implementation.
impl Drop for ExecutePayloadHandle {
fn drop(&mut self) {
if self.execution_layer.is_some() {
self.publish_blocking(ConsensusStatus::Invalid)
}
}
}

View File

@ -5,7 +5,7 @@
//! deposit-contract functionality that the `beacon_node/eth1` crate already provides. //! deposit-contract functionality that the `beacon_node/eth1` crate already provides.
use engine_api::{Error as ApiError, *}; use engine_api::{Error as ApiError, *};
use engines::{Engine, EngineError, Engines, ForkChoiceHead, Logging}; use engines::{Engine, EngineError, Engines, ForkChoiceState, Logging};
use lru::LruCache; use lru::LruCache;
use sensitive_url::SensitiveUrl; use sensitive_url::SensitiveUrl;
use slog::{crit, debug, error, info, Logger}; use slog::{crit, debug, error, info, Logger};
@ -19,12 +19,10 @@ use tokio::{
time::{sleep, sleep_until, Instant}, time::{sleep, sleep_until, Instant},
}; };
pub use engine_api::{http::HttpJsonRpc, ConsensusStatus, ExecutePayloadResponse}; pub use engine_api::{http::HttpJsonRpc, ExecutePayloadResponseStatus};
pub use execute_payload_handle::ExecutePayloadHandle;
mod engine_api; mod engine_api;
mod engines; mod engines;
mod execute_payload_handle;
pub mod test_utils; pub mod test_utils;
/// Each time the `ExecutionLayer` retrieves a block from an execution node, it stores that block /// Each time the `ExecutionLayer` retrieves a block from an execution node, it stores that block
@ -97,7 +95,7 @@ impl ExecutionLayer {
let inner = Inner { let inner = Inner {
engines: Engines { engines: Engines {
engines, engines,
latest_head: <_>::default(), latest_forkchoice_state: <_>::default(),
log: log.clone(), log: log.clone(),
}, },
terminal_total_difficulty, terminal_total_difficulty,
@ -236,39 +234,6 @@ impl ExecutionLayer {
self.engines().any_synced().await self.engines().any_synced().await
} }
/// Maps to the `engine_preparePayload` JSON-RPC function.
///
/// ## Fallback Behavior
///
/// The result will be returned from the first node that returns successfully. No more nodes
/// will be contacted.
pub async fn prepare_payload(
&self,
parent_hash: Hash256,
timestamp: u64,
random: Hash256,
) -> Result<PayloadId, Error> {
let fee_recipient = self.fee_recipient()?;
debug!(
self.log(),
"Issuing engine_preparePayload";
"fee_recipient" => ?fee_recipient,
"random" => ?random,
"timestamp" => timestamp,
"parent_hash" => ?parent_hash,
);
self.engines()
.first_success(|engine| {
// TODO(merge): make a cache for these IDs, so we don't always have to perform this
// request.
engine
.api
.prepare_payload(parent_hash, timestamp, random, fee_recipient)
})
.await
.map_err(Error::EngineErrors)
}
/// Maps to the `engine_getPayload` JSON-RPC call. /// Maps to the `engine_getPayload` JSON-RPC call.
/// ///
/// However, it will attempt to call `self.prepare_payload` if it cannot find an existing /// However, it will attempt to call `self.prepare_payload` if it cannot find an existing
@ -283,6 +248,7 @@ impl ExecutionLayer {
parent_hash: Hash256, parent_hash: Hash256,
timestamp: u64, timestamp: u64,
random: Hash256, random: Hash256,
finalized_block_hash: Hash256,
) -> Result<ExecutionPayload<T>, Error> { ) -> Result<ExecutionPayload<T>, Error> {
let fee_recipient = self.fee_recipient()?; let fee_recipient = self.fee_recipient()?;
debug!( debug!(
@ -295,14 +261,41 @@ impl ExecutionLayer {
); );
self.engines() self.engines()
.first_success(|engine| async move { .first_success(|engine| async move {
// TODO(merge): make a cache for these IDs, so we don't always have to perform this let payload_id = if let Some(id) = engine
// request. .get_payload_id(parent_hash, timestamp, random, fee_recipient)
let payload_id = engine .await
.api {
.prepare_payload(parent_hash, timestamp, random, fee_recipient) // The payload id has been cached for this engine.
.await?; id
} else {
// The payload id has *not* been cached for this engine. Trigger an artificial
// fork choice update to retrieve a payload ID.
//
// TODO(merge): a better algorithm might try to favour a node that already had a
// cached payload id, since a payload that has had more time to produce is
// likely to be more profitable.
let fork_choice_state = ForkChoiceState {
head_block_hash: parent_hash,
safe_block_hash: parent_hash,
finalized_block_hash,
};
let payload_attributes = PayloadAttributes {
timestamp,
random,
fee_recipient,
};
engine.api.get_payload(payload_id).await engine
.notify_forkchoice_updated(
fork_choice_state,
Some(payload_attributes),
self.log(),
)
.await?
.ok_or(ApiError::PayloadIdUnavailable)?
};
engine.api.get_payload_v1(payload_id).await
}) })
.await .await
.map_err(Error::EngineErrors) .map_err(Error::EngineErrors)
@ -323,7 +316,7 @@ impl ExecutionLayer {
pub async fn execute_payload<T: EthSpec>( pub async fn execute_payload<T: EthSpec>(
&self, &self,
execution_payload: &ExecutionPayload<T>, execution_payload: &ExecutionPayload<T>,
) -> Result<(ExecutePayloadResponse, Option<ExecutePayloadHandle>), Error> { ) -> Result<(ExecutePayloadResponseStatus, Option<Hash256>), Error> {
debug!( debug!(
self.log(), self.log(),
"Issuing engine_executePayload"; "Issuing engine_executePayload";
@ -334,18 +327,46 @@ impl ExecutionLayer {
let broadcast_results = self let broadcast_results = self
.engines() .engines()
.broadcast(|engine| engine.api.execute_payload(execution_payload.clone())) .broadcast(|engine| engine.api.execute_payload_v1(execution_payload.clone()))
.await; .await;
let mut errors = vec![]; let mut errors = vec![];
let mut valid = 0; let mut valid = 0;
let mut invalid = 0; let mut invalid = 0;
let mut syncing = 0; let mut syncing = 0;
let mut invalid_latest_valid_hash = vec![];
for result in broadcast_results { for result in broadcast_results {
match result { match result.map(|response| (response.latest_valid_hash, response.status)) {
Ok(ExecutePayloadResponse::Valid) => valid += 1, Ok((Some(latest_hash), ExecutePayloadResponseStatus::Valid)) => {
Ok(ExecutePayloadResponse::Invalid) => invalid += 1, if latest_hash == execution_payload.block_hash {
Ok(ExecutePayloadResponse::Syncing) => syncing += 1, valid += 1;
} else {
invalid += 1;
errors.push(EngineError::Api {
id: "unknown".to_string(),
error: engine_api::Error::BadResponse(
format!(
"execute_payload: response.status = Valid but invalid latest_valid_hash. Expected({:?}) Found({:?})",
execution_payload.block_hash,
latest_hash,
)
),
});
invalid_latest_valid_hash.push(latest_hash);
}
}
Ok((Some(latest_hash), ExecutePayloadResponseStatus::Invalid)) => {
invalid += 1;
invalid_latest_valid_hash.push(latest_hash);
}
Ok((_, ExecutePayloadResponseStatus::Syncing)) => syncing += 1,
Ok((None, status)) => errors.push(EngineError::Api {
id: "unknown".to_string(),
error: engine_api::Error::BadResponse(format!(
"execute_payload: status {:?} returned with null latest_valid_hash",
status
)),
}),
Err(e) => errors.push(e), Err(e) => errors.push(e),
} }
} }
@ -359,16 +380,14 @@ impl ExecutionLayer {
} }
if valid > 0 { if valid > 0 {
let handle = ExecutePayloadHandle { Ok((
block_hash: execution_payload.block_hash, ExecutePayloadResponseStatus::Valid,
execution_layer: Some(self.clone()), Some(execution_payload.block_hash),
log: self.log().clone(), ))
};
Ok((ExecutePayloadResponse::Valid, Some(handle)))
} else if invalid > 0 { } else if invalid > 0 {
Ok((ExecutePayloadResponse::Invalid, None)) Ok((ExecutePayloadResponseStatus::Invalid, None))
} else if syncing > 0 { } else if syncing > 0 {
Ok((ExecutePayloadResponse::Syncing, None)) Ok((ExecutePayloadResponseStatus::Syncing, None))
} else { } else {
Err(Error::EngineErrors(errors)) Err(Error::EngineErrors(errors))
} }
@ -384,48 +403,11 @@ impl ExecutionLayer {
/// ///
/// - Ok, if any node returns successfully. /// - Ok, if any node returns successfully.
/// - An error, if all nodes return an error. /// - An error, if all nodes return an error.
pub async fn consensus_validated( pub async fn notify_forkchoice_updated(
&self,
block_hash: Hash256,
status: ConsensusStatus,
) -> Result<(), Error> {
debug!(
self.log(),
"Issuing engine_consensusValidated";
"status" => ?status,
"block_hash" => ?block_hash,
);
let broadcast_results = self
.engines()
.broadcast(|engine| engine.api.consensus_validated(block_hash, status))
.await;
if broadcast_results.iter().any(Result::is_ok) {
Ok(())
} else {
Err(Error::EngineErrors(
broadcast_results
.into_iter()
.filter_map(Result::err)
.collect(),
))
}
}
/// Maps to the `engine_consensusValidated` JSON-RPC call.
///
/// ## Fallback Behaviour
///
/// The request will be broadcast to all nodes, simultaneously. It will await a response (or
/// failure) from all nodes and then return based on the first of these conditions which
/// returns true:
///
/// - Ok, if any node returns successfully.
/// - An error, if all nodes return an error.
pub async fn forkchoice_updated(
&self, &self,
head_block_hash: Hash256, head_block_hash: Hash256,
finalized_block_hash: Hash256, finalized_block_hash: Hash256,
payload_attributes: Option<PayloadAttributes>,
) -> Result<(), Error> { ) -> Result<(), Error> {
debug!( debug!(
self.log(), self.log(),
@ -434,33 +416,35 @@ impl ExecutionLayer {
"head_block_hash" => ?head_block_hash, "head_block_hash" => ?head_block_hash,
); );
// Update the cached version of the latest head so it can be sent to new or reconnecting // see https://hackmd.io/@n0ble/kintsugi-spec#Engine-API
// execution nodes. // for now, we must set safe_block_hash = head_block_hash
let forkchoice_state = ForkChoiceState {
head_block_hash,
safe_block_hash: head_block_hash,
finalized_block_hash,
};
self.engines() self.engines()
.set_latest_head(ForkChoiceHead { .set_latest_forkchoice_state(forkchoice_state)
head_block_hash,
finalized_block_hash,
})
.await; .await;
let broadcast_results = self let broadcast_results = self
.engines() .engines()
.broadcast(|engine| { .broadcast(|engine| async move {
engine engine
.api .notify_forkchoice_updated(forkchoice_state, payload_attributes, self.log())
.forkchoice_updated(head_block_hash, finalized_block_hash) .await
}) })
.await; .await;
if broadcast_results.iter().any(Result::is_ok) { if broadcast_results.iter().any(Result::is_ok) {
Ok(()) Ok(())
} else { } else {
Err(Error::EngineErrors( let errors = broadcast_results
broadcast_results .into_iter()
.into_iter() .filter_map(Result::err)
.filter_map(Result::err) .collect();
.collect(), Err(Error::EngineErrors(errors))
))
} }
} }

View File

@ -1,6 +1,8 @@
use crate::engine_api::{ use crate::engine_api::{
http::JsonPreparePayloadRequest, ConsensusStatus, ExecutePayloadResponse, ExecutionBlock, ExecutePayloadResponse, ExecutePayloadResponseStatus, ExecutionBlock, PayloadAttributes,
PayloadId,
}; };
use crate::engines::ForkChoiceState;
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use std::collections::HashMap; use std::collections::HashMap;
use tree_hash::TreeHash; use tree_hash::TreeHash;
@ -90,7 +92,7 @@ pub struct ExecutionBlockGenerator<T: EthSpec> {
*/ */
pub pending_payloads: HashMap<Hash256, ExecutionPayload<T>>, pub pending_payloads: HashMap<Hash256, ExecutionPayload<T>>,
pub next_payload_id: u64, pub next_payload_id: u64,
pub payload_ids: HashMap<u64, ExecutionPayload<T>>, pub payload_ids: HashMap<PayloadId, ExecutionPayload<T>>,
} }
impl<T: EthSpec> ExecutionBlockGenerator<T> { impl<T: EthSpec> ExecutionBlockGenerator<T> {
@ -222,104 +224,128 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
Ok(()) Ok(())
} }
pub fn prepare_payload(&mut self, payload: JsonPreparePayloadRequest) -> Result<u64, String> { pub fn get_payload(&mut self, id: &PayloadId) -> Option<ExecutionPayload<T>> {
if !self.blocks.iter().any(|(_, block)| { self.payload_ids.remove(id)
block.block_hash() == self.terminal_block_hash
|| block.block_number() == self.terminal_block_number
}) {
return Err("refusing to create payload id before terminal block".to_string());
}
let parent = self
.blocks
.get(&payload.parent_hash)
.ok_or_else(|| format!("unknown parent block {:?}", payload.parent_hash))?;
let id = self.next_payload_id;
self.next_payload_id += 1;
let mut execution_payload = ExecutionPayload {
parent_hash: payload.parent_hash,
coinbase: payload.fee_recipient,
receipt_root: Hash256::repeat_byte(42),
state_root: Hash256::repeat_byte(43),
logs_bloom: vec![0; 256].into(),
random: payload.random,
block_number: parent.block_number() + 1,
gas_limit: GAS_LIMIT,
gas_used: GAS_USED,
timestamp: payload.timestamp,
extra_data: "block gen was here".as_bytes().to_vec().into(),
base_fee_per_gas: Uint256::one(),
block_hash: Hash256::zero(),
transactions: vec![].into(),
};
execution_payload.block_hash = execution_payload.tree_hash_root();
self.payload_ids.insert(id, execution_payload);
Ok(id)
}
pub fn get_payload(&mut self, id: u64) -> Option<ExecutionPayload<T>> {
self.payload_ids.remove(&id)
} }
pub fn execute_payload(&mut self, payload: ExecutionPayload<T>) -> ExecutePayloadResponse { pub fn execute_payload(&mut self, payload: ExecutionPayload<T>) -> ExecutePayloadResponse {
let parent = if let Some(parent) = self.blocks.get(&payload.parent_hash) { let parent = if let Some(parent) = self.blocks.get(&payload.parent_hash) {
parent parent
} else { } else {
return ExecutePayloadResponse::Invalid; return ExecutePayloadResponse {
status: ExecutePayloadResponseStatus::Syncing,
latest_valid_hash: None,
message: None,
};
}; };
if payload.block_number != parent.block_number() + 1 { if payload.block_number != parent.block_number() + 1 {
return ExecutePayloadResponse::Invalid; return ExecutePayloadResponse {
status: ExecutePayloadResponseStatus::Invalid,
latest_valid_hash: Some(parent.block_hash()),
message: Some("invalid block number".to_string()),
};
} }
let valid_hash = payload.block_hash;
self.pending_payloads.insert(payload.block_hash, payload); self.pending_payloads.insert(payload.block_hash, payload);
ExecutePayloadResponse::Valid ExecutePayloadResponse {
status: ExecutePayloadResponseStatus::Valid,
latest_valid_hash: Some(valid_hash),
message: None,
}
} }
pub fn consensus_validated( pub fn forkchoice_updated_v1(
&mut self, &mut self,
block_hash: Hash256, forkchoice_state: ForkChoiceState,
status: ConsensusStatus, payload_attributes: Option<PayloadAttributes>,
) -> Result<(), String> { ) -> Result<Option<PayloadId>, String> {
let payload = self if let Some(payload) = self
.pending_payloads .pending_payloads
.remove(&block_hash) .remove(&forkchoice_state.head_block_hash)
.ok_or_else(|| format!("no pending payload for {:?}", block_hash))?;
match status {
ConsensusStatus::Valid => self.insert_block(Block::PoS(payload)),
ConsensusStatus::Invalid => Ok(()),
}
}
pub fn forkchoice_updated(
&mut self,
block_hash: Hash256,
finalized_block_hash: Hash256,
) -> Result<(), String> {
if !self.blocks.contains_key(&block_hash) {
return Err(format!("block hash {:?} unknown", block_hash));
}
if finalized_block_hash != Hash256::zero()
&& !self.blocks.contains_key(&finalized_block_hash)
{ {
self.insert_block(Block::PoS(payload))?;
}
if !self.blocks.contains_key(&forkchoice_state.head_block_hash) {
return Err(format!( return Err(format!(
"finalized block hash {:?} is unknown", "block hash {:?} unknown",
finalized_block_hash forkchoice_state.head_block_hash
));
}
if !self.blocks.contains_key(&forkchoice_state.safe_block_hash) {
return Err(format!(
"block hash {:?} unknown",
forkchoice_state.head_block_hash
)); ));
} }
Ok(()) if forkchoice_state.finalized_block_hash != Hash256::zero()
&& !self
.blocks
.contains_key(&forkchoice_state.finalized_block_hash)
{
return Err(format!(
"finalized block hash {:?} is unknown",
forkchoice_state.finalized_block_hash
));
}
match payload_attributes {
None => Ok(None),
Some(attributes) => {
if !self.blocks.iter().any(|(_, block)| {
block.block_hash() == self.terminal_block_hash
|| block.block_number() == self.terminal_block_number
}) {
return Err("refusing to create payload id before terminal block".to_string());
}
let parent = self
.blocks
.get(&forkchoice_state.head_block_hash)
.ok_or_else(|| {
format!(
"unknown parent block {:?}",
forkchoice_state.head_block_hash
)
})?;
let id = payload_id_from_u64(self.next_payload_id);
self.next_payload_id += 1;
let mut execution_payload = ExecutionPayload {
parent_hash: forkchoice_state.head_block_hash,
coinbase: attributes.fee_recipient,
receipt_root: Hash256::repeat_byte(42),
state_root: Hash256::repeat_byte(43),
logs_bloom: vec![0; 256].into(),
random: attributes.random,
block_number: parent.block_number() + 1,
gas_limit: GAS_LIMIT,
gas_used: GAS_USED,
timestamp: attributes.timestamp,
extra_data: "block gen was here".as_bytes().to_vec().into(),
base_fee_per_gas: Uint256::one(),
block_hash: Hash256::zero(),
transactions: vec![].into(),
};
execution_payload.block_hash = execution_payload.tree_hash_root();
self.payload_ids.insert(id, execution_payload);
Ok(Some(id))
}
}
} }
} }
fn payload_id_from_u64(n: u64) -> PayloadId {
n.to_le_bytes()
}
pub fn generate_pow_block( pub fn generate_pow_block(
terminal_total_difficulty: Uint256, terminal_total_difficulty: Uint256,
terminal_block_number: u64, terminal_block_number: u64,

View File

@ -1,5 +1,6 @@
use super::Context; use super::Context;
use crate::engine_api::http::*; use crate::engine_api::{http::*, ExecutePayloadResponse, ExecutePayloadResponseStatus};
use crate::json_structures::*;
use serde::de::DeserializeOwned; use serde::de::DeserializeOwned;
use serde_json::Value as JsonValue; use serde_json::Value as JsonValue;
use std::sync::Arc; use std::sync::Arc;
@ -53,57 +54,59 @@ pub async fn handle_rpc<T: EthSpec>(
) )
.unwrap()) .unwrap())
} }
ENGINE_PREPARE_PAYLOAD => { ENGINE_EXECUTE_PAYLOAD_V1 => {
let request = get_param_0(params)?; let request: JsonExecutionPayloadV1<T> = get_param(params, 0)?;
let payload_id = ctx
.execution_block_generator
.write()
.prepare_payload(request)?;
Ok(serde_json::to_value(JsonPayloadIdResponse { payload_id }).unwrap()) let response = if let Some(status) = *ctx.static_execute_payload_response.lock() {
match status {
ExecutePayloadResponseStatus::Valid => ExecutePayloadResponse {
status,
latest_valid_hash: Some(request.block_hash),
message: None,
},
ExecutePayloadResponseStatus::Syncing => ExecutePayloadResponse {
status,
latest_valid_hash: None,
message: None,
},
_ => unimplemented!("invalid static executePayloadResponse"),
}
} else {
ctx.execution_block_generator
.write()
.execute_payload(request.into())
};
Ok(serde_json::to_value(JsonExecutePayloadV1Response::from(response)).unwrap())
} }
ENGINE_EXECUTE_PAYLOAD => { ENGINE_GET_PAYLOAD_V1 => {
let request: JsonExecutionPayload<T> = get_param_0(params)?; let request: JsonPayloadIdRequest = get_param(params, 0)?;
let id = request.into();
let status = ctx
.static_execute_payload_response
.lock()
.unwrap_or_else(|| {
ctx.execution_block_generator
.write()
.execute_payload(request.into())
});
Ok(serde_json::to_value(ExecutePayloadResponseWrapper { status }).unwrap())
}
ENGINE_GET_PAYLOAD => {
let request: JsonPayloadIdRequest = get_param_0(params)?;
let id = request.payload_id;
let response = ctx let response = ctx
.execution_block_generator .execution_block_generator
.write() .write()
.get_payload(id) .get_payload(&id)
.ok_or_else(|| format!("no payload for id {}", id))?; .ok_or_else(|| format!("no payload for id {:?}", id))?;
Ok(serde_json::to_value(JsonExecutionPayload::from(response)).unwrap()) Ok(serde_json::to_value(JsonExecutionPayloadV1::from(response)).unwrap())
} }
ENGINE_FORKCHOICE_UPDATED_V1 => {
ENGINE_CONSENSUS_VALIDATED => { let forkchoice_state: JsonForkChoiceStateV1 = get_param(params, 0)?;
let request: JsonConsensusValidatedRequest = get_param_0(params)?; let payload_attributes: Option<JsonPayloadAttributesV1> = get_param(params, 1)?;
ctx.execution_block_generator let id = ctx
.execution_block_generator
.write() .write()
.consensus_validated(request.block_hash, request.status)?; .forkchoice_updated_v1(
forkchoice_state.into(),
payload_attributes.map(|json| json.into()),
)?;
Ok(JsonValue::Null) Ok(serde_json::to_value(JsonForkchoiceUpdatedV1Response {
} status: JsonForkchoiceUpdatedV1ResponseStatus::Success,
ENGINE_FORKCHOICE_UPDATED => { payload_id: id.map(Into::into),
let request: JsonForkChoiceUpdatedRequest = get_param_0(params)?; })
ctx.execution_block_generator .unwrap())
.write()
.forkchoice_updated(request.head_block_hash, request.finalized_block_hash)?;
Ok(JsonValue::Null)
} }
other => Err(format!( other => Err(format!(
"The method {} does not exist/is not available", "The method {} does not exist/is not available",
@ -112,12 +115,12 @@ pub async fn handle_rpc<T: EthSpec>(
} }
} }
fn get_param_0<T: DeserializeOwned>(params: &JsonValue) -> Result<T, String> { fn get_param<T: DeserializeOwned>(params: &JsonValue, index: usize) -> Result<T, String> {
params params
.get(0) .get(index)
.ok_or_else(|| "missing/invalid params[0] value".to_string()) .ok_or_else(|| format!("missing/invalid params[{}] value", index))
.and_then(|param| { .and_then(|param| {
serde_json::from_value(param.clone()) serde_json::from_value(param.clone())
.map_err(|e| format!("failed to deserialize param[0]: {:?}", e)) .map_err(|e| format!("failed to deserialize param[{}]: {:?}", index, e))
}) })
} }

View File

@ -105,16 +105,24 @@ impl<T: EthSpec> MockExecutionLayer<T> {
let block_number = latest_execution_block.block_number() + 1; let block_number = latest_execution_block.block_number() + 1;
let timestamp = block_number; let timestamp = block_number;
let random = Hash256::from_low_u64_be(block_number); let random = Hash256::from_low_u64_be(block_number);
let finalized_block_hash = parent_hash;
let _payload_id = self self.el
.el .notify_forkchoice_updated(
.prepare_payload(parent_hash, timestamp, random) parent_hash,
Hash256::zero(),
Some(PayloadAttributes {
timestamp,
random,
fee_recipient: Address::repeat_byte(42),
}),
)
.await .await
.unwrap(); .unwrap();
let payload = self let payload = self
.el .el
.get_payload::<T>(parent_hash, timestamp, random) .get_payload::<T>(parent_hash, timestamp, random, finalized_block_hash)
.await .await
.unwrap(); .unwrap();
let block_hash = payload.block_hash; let block_hash = payload.block_hash;
@ -123,16 +131,13 @@ impl<T: EthSpec> MockExecutionLayer<T> {
assert_eq!(payload.timestamp, timestamp); assert_eq!(payload.timestamp, timestamp);
assert_eq!(payload.random, random); assert_eq!(payload.random, random);
let (payload_response, payload_handle) = self.el.execute_payload(&payload).await.unwrap(); let (payload_response, latest_valid_hash) =
assert_eq!(payload_response, ExecutePayloadResponse::Valid); self.el.execute_payload(&payload).await.unwrap();
assert_eq!(payload_response, ExecutePayloadResponseStatus::Valid);
payload_handle assert_eq!(latest_valid_hash, Some(payload.block_hash));
.unwrap()
.publish_async(ConsensusStatus::Valid)
.await;
self.el self.el
.forkchoice_updated(block_hash, Hash256::zero()) .notify_forkchoice_updated(block_hash, Hash256::zero(), None)
.await .await
.unwrap(); .unwrap();

View File

@ -1,7 +1,7 @@
//! Provides a mock execution engine HTTP JSON-RPC API for use in testing. //! Provides a mock execution engine HTTP JSON-RPC API for use in testing.
use crate::engine_api::http::JSONRPC_VERSION; use crate::engine_api::http::JSONRPC_VERSION;
use crate::engine_api::ExecutePayloadResponse; use crate::engine_api::ExecutePayloadResponseStatus;
use bytes::Bytes; use bytes::Bytes;
use environment::null_logger; use environment::null_logger;
use handle_rpc::handle_rpc; use handle_rpc::handle_rpc;
@ -116,7 +116,7 @@ impl<T: EthSpec> MockServer<T> {
} }
pub fn all_payloads_valid(&self) { pub fn all_payloads_valid(&self) {
*self.ctx.static_execute_payload_response.lock() = Some(ExecutePayloadResponse::Valid) *self.ctx.static_execute_payload_response.lock() = Some(ExecutePayloadResponseStatus::Valid)
} }
} }
@ -152,7 +152,7 @@ pub struct Context<T: EthSpec> {
pub last_echo_request: Arc<RwLock<Option<Bytes>>>, pub last_echo_request: Arc<RwLock<Option<Bytes>>>,
pub execution_block_generator: RwLock<ExecutionBlockGenerator<T>>, pub execution_block_generator: RwLock<ExecutionBlockGenerator<T>>,
pub preloaded_responses: Arc<Mutex<Vec<serde_json::Value>>>, pub preloaded_responses: Arc<Mutex<Vec<serde_json::Value>>>,
pub static_execute_payload_response: Arc<Mutex<Option<ExecutePayloadResponse>>>, pub static_execute_payload_response: Arc<Mutex<Option<ExecutePayloadResponseStatus>>>,
pub _phantom: PhantomData<T>, pub _phantom: PhantomData<T>,
} }

View File

@ -40,6 +40,15 @@ impl ExecutionStatus {
pub fn irrelevant() -> Self { pub fn irrelevant() -> Self {
ExecutionStatus::Irrelevant(false) ExecutionStatus::Irrelevant(false)
} }
pub fn block_hash(&self) -> Option<Hash256> {
match self {
ExecutionStatus::Valid(hash)
| ExecutionStatus::Invalid(hash)
| ExecutionStatus::Unknown(hash) => Some(*hash),
ExecutionStatus::Irrelevant(_) => None,
}
}
} }
/// A block that is to be applied to the fork choice. /// A block that is to be applied to the fork choice.

View File

@ -1,38 +0,0 @@
//! Formats `[u8; 4]` as a 0x-prefixed hex string.
//!
//! E.g., `[0, 1, 2, 3]` serializes as `"0x00010203"`.
use crate::hex::PrefixedHexVisitor;
use serde::de::Error;
use serde::{Deserializer, Serializer};
const BYTES_LEN: usize = 4;
pub fn serialize<S>(bytes: &[u8; BYTES_LEN], serializer: S) -> Result<S::Ok, S::Error>
where
S: Serializer,
{
let mut hex_string: String = "0x".to_string();
hex_string.push_str(&hex::encode(&bytes));
serializer.serialize_str(&hex_string)
}
pub fn deserialize<'de, D>(deserializer: D) -> Result<[u8; BYTES_LEN], D::Error>
where
D: Deserializer<'de>,
{
let decoded = deserializer.deserialize_str(PrefixedHexVisitor)?;
if decoded.len() != BYTES_LEN {
return Err(D::Error::custom(format!(
"expected {} bytes for array, got {}",
BYTES_LEN,
decoded.len()
)));
}
let mut array = [0; BYTES_LEN];
array.copy_from_slice(&decoded);
Ok(array)
}

View File

@ -0,0 +1,52 @@
//! Formats `[u8; n]` as a 0x-prefixed hex string.
//!
//! E.g., `[0, 1, 2, 3]` serializes as `"0x00010203"`.
use crate::hex::PrefixedHexVisitor;
use serde::de::Error;
use serde::{Deserializer, Serializer};
macro_rules! bytes_hex {
($num_bytes: tt) => {
use super::*;
const BYTES_LEN: usize = $num_bytes;
pub fn serialize<S>(bytes: &[u8; BYTES_LEN], serializer: S) -> Result<S::Ok, S::Error>
where
S: Serializer,
{
let mut hex_string: String = "0x".to_string();
hex_string.push_str(&hex::encode(&bytes));
serializer.serialize_str(&hex_string)
}
pub fn deserialize<'de, D>(deserializer: D) -> Result<[u8; BYTES_LEN], D::Error>
where
D: Deserializer<'de>,
{
let decoded = deserializer.deserialize_str(PrefixedHexVisitor)?;
if decoded.len() != BYTES_LEN {
return Err(D::Error::custom(format!(
"expected {} bytes for array, got {}",
BYTES_LEN,
decoded.len()
)));
}
let mut array = [0; BYTES_LEN];
array.copy_from_slice(&decoded);
Ok(array)
}
};
}
pub mod bytes_4_hex {
bytes_hex!(4);
}
pub mod bytes_8_hex {
bytes_hex!(8);
}

View File

@ -1,6 +1,6 @@
mod quoted_int; mod quoted_int;
pub mod bytes_4_hex; pub mod fixed_bytes_hex;
pub mod hex; pub mod hex;
pub mod hex_vec; pub mod hex_vec;
pub mod list_of_bytes_lists; pub mod list_of_bytes_lists;
@ -9,4 +9,5 @@ pub mod u32_hex;
pub mod u64_hex_be; pub mod u64_hex_be;
pub mod u8_hex; pub mod u8_hex;
pub use fixed_bytes_hex::{bytes_4_hex, bytes_8_hex};
pub use quoted_int::{quoted_u256, quoted_u32, quoted_u64, quoted_u8}; pub use quoted_int::{quoted_u256, quoted_u32, quoted_u64, quoted_u8};