Capella eip 4844 cleanup (#3652)
* add capella gossip boiler plate * get everything compiling Co-authored-by: realbigsean <sean@sigmaprime.io Co-authored-by: Mark Mackey <mark@sigmaprime.io> * small cleanup * small cleanup * cargo fix + some test cleanup * improve block production * add fixme for potential panic Co-authored-by: Mark Mackey <mark@sigmaprime.io>
This commit is contained in:
parent
221c433d62
commit
137f230344
1
Cargo.lock
generated
1
Cargo.lock
generated
@ -2097,6 +2097,7 @@ dependencies = [
|
||||
"ssz-rs",
|
||||
"state_processing",
|
||||
"strum",
|
||||
"superstruct 0.6.0",
|
||||
"task_executor",
|
||||
"tempfile",
|
||||
"tokio",
|
||||
|
@ -57,7 +57,8 @@ use crate::BeaconSnapshot;
|
||||
use crate::{metrics, BeaconChainError};
|
||||
use eth2::types::{EventKind, SseBlock, SyncDuty};
|
||||
use execution_layer::{
|
||||
BuilderParams, ChainHealth, ExecutionLayer, FailedCondition, PayloadAttributes, PayloadStatus,
|
||||
BlockProposalContents, BuilderParams, ChainHealth, ExecutionLayer, FailedCondition,
|
||||
PayloadAttributes, PayloadAttributesV1, PayloadAttributesV2, PayloadStatus,
|
||||
};
|
||||
pub use fork_choice::CountUnrealized;
|
||||
use fork_choice::{
|
||||
@ -241,7 +242,7 @@ pub trait BeaconChainTypes: Send + Sync + 'static {
|
||||
}
|
||||
|
||||
/// Used internally to split block production into discrete functions.
|
||||
struct PartialBeaconBlock<E: EthSpec, Payload> {
|
||||
struct PartialBeaconBlock<E: EthSpec, Payload: AbstractExecPayload<E>> {
|
||||
state: BeaconState<E>,
|
||||
slot: Slot,
|
||||
proposer_index: u64,
|
||||
@ -255,7 +256,7 @@ struct PartialBeaconBlock<E: EthSpec, Payload> {
|
||||
deposits: Vec<Deposit>,
|
||||
voluntary_exits: Vec<SignedVoluntaryExit>,
|
||||
sync_aggregate: Option<SyncAggregate<E>>,
|
||||
prepare_payload_handle: Option<PreparePayloadHandle<Payload, E>>,
|
||||
prepare_payload_handle: Option<PreparePayloadHandle<E, Payload>>,
|
||||
}
|
||||
|
||||
pub type BeaconForkChoice<T> = ForkChoice<
|
||||
@ -928,12 +929,12 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
||||
|
||||
// If we only have a blinded block, load the execution payload from the EL.
|
||||
let block_message = blinded_block.message();
|
||||
let execution_payload_header = &block_message
|
||||
let execution_payload_header = block_message
|
||||
.execution_payload()
|
||||
.map_err(|_| Error::BlockVariantLacksExecutionPayload(*block_root))?
|
||||
.execution_payload_header;
|
||||
.to_execution_payload_header();
|
||||
|
||||
let exec_block_hash = execution_payload_header.block_hash;
|
||||
let exec_block_hash = execution_payload_header.block_hash();
|
||||
|
||||
let execution_payload = self
|
||||
.execution_layer
|
||||
@ -944,10 +945,13 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
||||
.map_err(|e| Error::ExecutionLayerErrorPayloadReconstruction(exec_block_hash, e))?
|
||||
.ok_or(Error::BlockHashMissingFromExecutionLayer(exec_block_hash))?;
|
||||
|
||||
//FIXME(sean) avoid the clone by comparing refs to headers (`as_execution_payload_header` method ?)
|
||||
let full_payload: FullPayload<T::EthSpec> = execution_payload.clone().into();
|
||||
|
||||
// Verify payload integrity.
|
||||
let header_from_payload = ExecutionPayloadHeader::from(&execution_payload);
|
||||
if header_from_payload != *execution_payload_header {
|
||||
for txn in &execution_payload.transactions {
|
||||
let header_from_payload = full_payload.to_execution_payload_header();
|
||||
if header_from_payload != execution_payload_header {
|
||||
for txn in execution_payload.transactions() {
|
||||
debug!(
|
||||
self.log,
|
||||
"Reconstructed txn";
|
||||
@ -960,8 +964,8 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
||||
exec_block_hash,
|
||||
canonical_payload_root: execution_payload_header.tree_hash_root(),
|
||||
reconstructed_payload_root: header_from_payload.tree_hash_root(),
|
||||
canonical_transactions_root: execution_payload_header.transactions_root,
|
||||
reconstructed_transactions_root: header_from_payload.transactions_root,
|
||||
canonical_transactions_root: execution_payload_header.transactions_root(),
|
||||
reconstructed_transactions_root: header_from_payload.transactions_root(),
|
||||
});
|
||||
}
|
||||
|
||||
@ -3126,7 +3130,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
||||
///
|
||||
/// The produced block will not be inherently valid, it must be signed by a block producer.
|
||||
/// Block signing is out of the scope of this function and should be done by a separate program.
|
||||
pub async fn produce_block<Payload: ExecPayload<T::EthSpec>>(
|
||||
pub async fn produce_block<Payload: AbstractExecPayload<T::EthSpec> + 'static>(
|
||||
self: &Arc<Self>,
|
||||
randao_reveal: Signature,
|
||||
slot: Slot,
|
||||
@ -3142,7 +3146,9 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
||||
}
|
||||
|
||||
/// Same as `produce_block` but allowing for configuration of RANDAO-verification.
|
||||
pub async fn produce_block_with_verification<Payload: ExecPayload<T::EthSpec>>(
|
||||
pub async fn produce_block_with_verification<
|
||||
Payload: AbstractExecPayload<T::EthSpec> + 'static,
|
||||
>(
|
||||
self: &Arc<Self>,
|
||||
randao_reveal: Signature,
|
||||
slot: Slot,
|
||||
@ -3256,7 +3262,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
||||
/// The provided `state_root_opt` should only ever be set to `Some` if the contained value is
|
||||
/// equal to the root of `state`. Providing this value will serve as an optimization to avoid
|
||||
/// performing a tree hash in some scenarios.
|
||||
pub async fn produce_block_on_state<Payload: ExecPayload<T::EthSpec>>(
|
||||
pub async fn produce_block_on_state<Payload: AbstractExecPayload<T::EthSpec> + 'static>(
|
||||
self: &Arc<Self>,
|
||||
state: BeaconState<T::EthSpec>,
|
||||
state_root_opt: Option<Hash256>,
|
||||
@ -3291,16 +3297,17 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
||||
//
|
||||
// Wait for the execution layer to return an execution payload (if one is required).
|
||||
let prepare_payload_handle = partial_beacon_block.prepare_payload_handle.take();
|
||||
let (execution_payload, kzg_commitments, blobs) =
|
||||
if let Some(prepare_payload_handle) = prepare_payload_handle {
|
||||
let (execution_payload, commitments, blobs) = prepare_payload_handle
|
||||
.await
|
||||
.map_err(BlockProductionError::TokioJoin)?
|
||||
.ok_or(BlockProductionError::ShuttingDown)??;
|
||||
(execution_payload, commitments, blobs)
|
||||
} else {
|
||||
return Err(BlockProductionError::MissingExecutionPayload);
|
||||
};
|
||||
let execution_payload = if let Some(prepare_payload_handle) = prepare_payload_handle {
|
||||
prepare_payload_handle
|
||||
.await
|
||||
.map_err(BlockProductionError::TokioJoin)?
|
||||
.ok_or(BlockProductionError::ShuttingDown)??
|
||||
} else {
|
||||
return Err(BlockProductionError::MissingExecutionPayload);
|
||||
};
|
||||
|
||||
//FIXME(sean) waiting for the BN<>EE api for this to stabilize
|
||||
let kzg_commitments = vec![];
|
||||
|
||||
// Part 3/3 (blocking)
|
||||
//
|
||||
@ -3323,7 +3330,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
||||
.map_err(BlockProductionError::TokioJoin)?
|
||||
}
|
||||
|
||||
fn produce_partial_beacon_block<Payload: ExecPayload<T::EthSpec>>(
|
||||
fn produce_partial_beacon_block<Payload: AbstractExecPayload<T::EthSpec> + 'static>(
|
||||
self: &Arc<Self>,
|
||||
mut state: BeaconState<T::EthSpec>,
|
||||
state_root_opt: Option<Hash256>,
|
||||
@ -3383,7 +3390,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
||||
// allows it to run concurrently with things like attestation packing.
|
||||
let prepare_payload_handle = match &state {
|
||||
BeaconState::Base(_) | BeaconState::Altair(_) => None,
|
||||
BeaconState::Merge(_) | BeaconState::Eip4844(_) => {
|
||||
BeaconState::Merge(_) | BeaconState::Capella(_) | BeaconState::Eip4844(_) => {
|
||||
let prepare_payload_handle =
|
||||
get_execution_payload(self.clone(), &state, proposer_index, builder_params)?;
|
||||
Some(prepare_payload_handle)
|
||||
@ -3556,10 +3563,10 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
||||
})
|
||||
}
|
||||
|
||||
fn complete_partial_beacon_block<Payload: ExecPayload<T::EthSpec>>(
|
||||
fn complete_partial_beacon_block<Payload: AbstractExecPayload<T::EthSpec>>(
|
||||
&self,
|
||||
partial_beacon_block: PartialBeaconBlock<T::EthSpec, Payload>,
|
||||
execution_payload: Payload,
|
||||
block_contents: BlockProposalContents<T::EthSpec, Payload>,
|
||||
kzg_commitments: Vec<KzgCommitment>,
|
||||
verification: ProduceBlockVerification,
|
||||
) -> Result<BeaconBlockAndState<T::EthSpec, Payload>, BlockProductionError> {
|
||||
@ -3636,7 +3643,32 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
||||
voluntary_exits: voluntary_exits.into(),
|
||||
sync_aggregate: sync_aggregate
|
||||
.ok_or(BlockProductionError::MissingSyncAggregate)?,
|
||||
execution_payload,
|
||||
execution_payload: block_contents
|
||||
.to_payload()
|
||||
.try_into()
|
||||
.map_err(|_| BlockProductionError::InvalidPayloadFork)?,
|
||||
},
|
||||
}),
|
||||
BeaconState::Capella(_) => BeaconBlock::Capella(BeaconBlockCapella {
|
||||
slot,
|
||||
proposer_index,
|
||||
parent_root,
|
||||
state_root: Hash256::zero(),
|
||||
body: BeaconBlockBodyCapella {
|
||||
randao_reveal,
|
||||
eth1_data,
|
||||
graffiti,
|
||||
proposer_slashings: proposer_slashings.into(),
|
||||
attester_slashings: attester_slashings.into(),
|
||||
attestations: attestations.into(),
|
||||
deposits: deposits.into(),
|
||||
voluntary_exits: voluntary_exits.into(),
|
||||
sync_aggregate: sync_aggregate
|
||||
.ok_or(BlockProductionError::MissingSyncAggregate)?,
|
||||
execution_payload: block_contents
|
||||
.to_payload()
|
||||
.try_into()
|
||||
.map_err(|_| BlockProductionError::InvalidPayloadFork)?,
|
||||
},
|
||||
}),
|
||||
BeaconState::Eip4844(_) => BeaconBlock::Eip4844(BeaconBlockEip4844 {
|
||||
@ -3655,7 +3687,10 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
||||
voluntary_exits: voluntary_exits.into(),
|
||||
sync_aggregate: sync_aggregate
|
||||
.ok_or(BlockProductionError::MissingSyncAggregate)?,
|
||||
execution_payload,
|
||||
execution_payload: block_contents
|
||||
.to_payload()
|
||||
.try_into()
|
||||
.map_err(|_| BlockProductionError::InvalidPayloadFork)?,
|
||||
//FIXME(sean) get blobs
|
||||
blob_kzg_commitments: VariableList::from(kzg_commitments),
|
||||
},
|
||||
@ -3973,16 +4008,33 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
||||
return Ok(());
|
||||
}
|
||||
|
||||
let payload_attributes = PayloadAttributes {
|
||||
timestamp: self
|
||||
.slot_clock
|
||||
.start_of(prepare_slot)
|
||||
.ok_or(Error::InvalidSlot(prepare_slot))?
|
||||
.as_secs(),
|
||||
prev_randao: head_random,
|
||||
suggested_fee_recipient: execution_layer
|
||||
.get_suggested_fee_recipient(proposer as u64)
|
||||
.await,
|
||||
let payload_attributes = match self.spec.fork_name_at_epoch(prepare_epoch) {
|
||||
ForkName::Base | ForkName::Altair | ForkName::Merge => {
|
||||
PayloadAttributes::V1(PayloadAttributesV1 {
|
||||
timestamp: self
|
||||
.slot_clock
|
||||
.start_of(prepare_slot)
|
||||
.ok_or(Error::InvalidSlot(prepare_slot))?
|
||||
.as_secs(),
|
||||
prev_randao: head_random,
|
||||
suggested_fee_recipient: execution_layer
|
||||
.get_suggested_fee_recipient(proposer as u64)
|
||||
.await,
|
||||
})
|
||||
}
|
||||
ForkName::Capella | ForkName::Eip4844 => PayloadAttributes::V2(PayloadAttributesV2 {
|
||||
timestamp: self
|
||||
.slot_clock
|
||||
.start_of(prepare_slot)
|
||||
.ok_or(Error::InvalidSlot(prepare_slot))?
|
||||
.as_secs(),
|
||||
prev_randao: head_random,
|
||||
suggested_fee_recipient: execution_layer
|
||||
.get_suggested_fee_recipient(proposer as u64)
|
||||
.await,
|
||||
//FIXME(sean)
|
||||
withdrawals: vec![],
|
||||
}),
|
||||
};
|
||||
|
||||
debug!(
|
||||
@ -4122,7 +4174,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
||||
{
|
||||
// We are a proposer, check for terminal_pow_block_hash
|
||||
if let Some(terminal_pow_block_hash) = execution_layer
|
||||
.get_terminal_pow_block_hash(&self.spec, payload_attributes.timestamp)
|
||||
.get_terminal_pow_block_hash(&self.spec, payload_attributes.timestamp())
|
||||
.await
|
||||
.map_err(Error::ForkchoiceUpdate)?
|
||||
{
|
||||
@ -4297,7 +4349,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
||||
/// Returns `Ok(false)` if the block is pre-Bellatrix, or has `ExecutionStatus::Valid`.
|
||||
/// Returns `Ok(true)` if the block has `ExecutionStatus::Optimistic` or has
|
||||
/// `ExecutionStatus::Invalid`.
|
||||
pub fn is_optimistic_or_invalid_block<Payload: ExecPayload<T::EthSpec>>(
|
||||
pub fn is_optimistic_or_invalid_block<Payload: AbstractExecPayload<T::EthSpec>>(
|
||||
&self,
|
||||
block: &SignedBeaconBlock<T::EthSpec, Payload>,
|
||||
) -> Result<bool, BeaconChainError> {
|
||||
@ -4323,7 +4375,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
||||
///
|
||||
/// There is a potential race condition when syncing where the block_root of `head_block` could
|
||||
/// be pruned from the fork choice store before being read.
|
||||
pub fn is_optimistic_or_invalid_head_block<Payload: ExecPayload<T::EthSpec>>(
|
||||
pub fn is_optimistic_or_invalid_head_block<Payload: AbstractExecPayload<T::EthSpec>>(
|
||||
&self,
|
||||
head_block: &SignedBeaconBlock<T::EthSpec, Payload>,
|
||||
) -> Result<bool, BeaconChainError> {
|
||||
|
@ -14,7 +14,7 @@ use std::sync::Arc;
|
||||
use store::{Error as StoreError, HotColdDB, ItemStore};
|
||||
use superstruct::superstruct;
|
||||
use types::{
|
||||
BeaconBlockRef, BeaconState, BeaconStateError, Checkpoint, Epoch, EthSpec, ExecPayload,
|
||||
AbstractExecPayload, BeaconBlockRef, BeaconState, BeaconStateError, Checkpoint, Epoch, EthSpec,
|
||||
Hash256, Slot,
|
||||
};
|
||||
|
||||
@ -268,7 +268,7 @@ where
|
||||
self.time = slot
|
||||
}
|
||||
|
||||
fn on_verified_block<Payload: ExecPayload<E>>(
|
||||
fn on_verified_block<Payload: AbstractExecPayload<E>>(
|
||||
&mut self,
|
||||
_block: BeaconBlockRef<E, Payload>,
|
||||
block_root: Hash256,
|
||||
|
@ -1,20 +1,20 @@
|
||||
use serde_derive::Serialize;
|
||||
use std::sync::Arc;
|
||||
use types::{
|
||||
beacon_state::CloneConfig, BeaconState, EthSpec, ExecPayload, FullPayload, Hash256,
|
||||
beacon_state::CloneConfig, AbstractExecPayload, BeaconState, EthSpec, FullPayload, Hash256,
|
||||
SignedBeaconBlock,
|
||||
};
|
||||
|
||||
/// Represents some block and its associated state. Generally, this will be used for tracking the
|
||||
/// head, justified head and finalized head.
|
||||
#[derive(Clone, Serialize, PartialEq, Debug)]
|
||||
pub struct BeaconSnapshot<E: EthSpec, Payload: ExecPayload<E> = FullPayload<E>> {
|
||||
pub struct BeaconSnapshot<E: EthSpec, Payload: AbstractExecPayload<E> = FullPayload<E>> {
|
||||
pub beacon_block: Arc<SignedBeaconBlock<E, Payload>>,
|
||||
pub beacon_block_root: Hash256,
|
||||
pub beacon_state: BeaconState<E>,
|
||||
}
|
||||
|
||||
impl<E: EthSpec, Payload: ExecPayload<E>> BeaconSnapshot<E, Payload> {
|
||||
impl<E: EthSpec, Payload: AbstractExecPayload<E>> BeaconSnapshot<E, Payload> {
|
||||
/// Create a new checkpoint.
|
||||
pub fn new(
|
||||
beacon_block: Arc<SignedBeaconBlock<E, Payload>>,
|
||||
|
@ -5,10 +5,10 @@ use state_processing::{
|
||||
common::get_attesting_indices_from_state,
|
||||
per_block_processing::altair::sync_committee::compute_sync_aggregate_rewards,
|
||||
};
|
||||
use types::{BeaconBlockRef, BeaconState, EthSpec, ExecPayload, Hash256};
|
||||
use types::{AbstractExecPayload, BeaconBlockRef, BeaconState, EthSpec, Hash256};
|
||||
|
||||
impl<T: BeaconChainTypes> BeaconChain<T> {
|
||||
pub fn compute_block_reward<Payload: ExecPayload<T::EthSpec>>(
|
||||
pub fn compute_block_reward<Payload: AbstractExecPayload<T::EthSpec>>(
|
||||
&self,
|
||||
block: BeaconBlockRef<'_, T::EthSpec, Payload>,
|
||||
block_root: Hash256,
|
||||
|
@ -81,6 +81,7 @@ use std::time::Duration;
|
||||
use store::{Error as DBError, HotStateSummary, KeyValueStore, StoreOp};
|
||||
use task_executor::JoinHandle;
|
||||
use tree_hash::TreeHash;
|
||||
use types::ExecPayload;
|
||||
use types::{
|
||||
BeaconBlockRef, BeaconState, BeaconStateError, BlindedPayload, ChainSpec, CloneConfig, Epoch,
|
||||
EthSpec, ExecutionBlockHash, Hash256, InconsistentFork, PublicKey, PublicKeyBytes,
|
||||
@ -1235,7 +1236,7 @@ impl<T: BeaconChainTypes> ExecutionPendingBlock<T> {
|
||||
.message()
|
||||
.body()
|
||||
.execution_payload()
|
||||
.map(|full_payload| full_payload.execution_payload.block_hash);
|
||||
.map(|full_payload| full_payload.block_hash());
|
||||
|
||||
// Ensure the block is a candidate for optimistic import.
|
||||
if !is_optimistic_candidate_block(&chain, block.slot(), block.parent_root()).await?
|
||||
|
@ -263,6 +263,7 @@ pub enum BlockProductionError {
|
||||
MissingExecutionPayload,
|
||||
TokioJoin(tokio::task::JoinError),
|
||||
BeaconChain(BeaconChainError),
|
||||
InvalidPayloadFork,
|
||||
}
|
||||
|
||||
easy_from_to!(BlockProcessingError, BlockProductionError);
|
||||
|
@ -12,7 +12,7 @@ use crate::{
|
||||
BeaconChain, BeaconChainError, BeaconChainTypes, BlockError, BlockProductionError,
|
||||
ExecutionPayloadError,
|
||||
};
|
||||
use execution_layer::{BuilderParams, PayloadStatus};
|
||||
use execution_layer::{BlockProposalContents, BuilderParams, PayloadStatus};
|
||||
use fork_choice::{InvalidationOperation, PayloadVerificationStatus};
|
||||
use proto_array::{Block as ProtoBlock, ExecutionStatus};
|
||||
use slog::debug;
|
||||
@ -24,14 +24,11 @@ use state_processing::per_block_processing::{
|
||||
use std::sync::Arc;
|
||||
use tokio::task::JoinHandle;
|
||||
use tree_hash::TreeHash;
|
||||
use types::{
|
||||
BeaconBlockRef, BeaconState, BeaconStateError, Blob, EthSpec, ExecPayload, ExecutionBlockHash,
|
||||
Hash256, KzgCommitment, SignedBeaconBlock, Slot,
|
||||
};
|
||||
use types::*;
|
||||
|
||||
pub type PreparePayloadResult<Payload, E> =
|
||||
Result<(Payload, Vec<KzgCommitment>, Vec<Blob<E>>), BlockProductionError>;
|
||||
pub type PreparePayloadHandle<Payload, E> = JoinHandle<Option<PreparePayloadResult<Payload, E>>>;
|
||||
pub type PreparePayloadResult<E, Payload> =
|
||||
Result<BlockProposalContents<E, Payload>, BlockProductionError>;
|
||||
pub type PreparePayloadHandle<E, Payload> = JoinHandle<Option<PreparePayloadResult<E, Payload>>>;
|
||||
|
||||
#[derive(PartialEq)]
|
||||
pub enum AllowOptimisticImport {
|
||||
@ -57,7 +54,7 @@ impl<T: BeaconChainTypes> PayloadNotifier<T> {
|
||||
//
|
||||
// We will duplicate these checks again during `per_block_processing`, however these checks
|
||||
// are cheap and doing them here ensures we protect the execution engine from junk.
|
||||
partially_verify_execution_payload(
|
||||
partially_verify_execution_payload::<T::EthSpec, FullPayload<T::EthSpec>>(
|
||||
state,
|
||||
block.message().execution_payload()?,
|
||||
&chain.spec,
|
||||
@ -107,7 +104,7 @@ async fn notify_new_payload<'a, T: BeaconChainTypes>(
|
||||
.ok_or(ExecutionPayloadError::NoExecutionConnection)?;
|
||||
|
||||
let new_payload_response = execution_layer
|
||||
.notify_new_payload(&execution_payload.execution_payload)
|
||||
.notify_new_payload(&execution_payload.into())
|
||||
.await;
|
||||
|
||||
match new_payload_response {
|
||||
@ -125,7 +122,7 @@ async fn notify_new_payload<'a, T: BeaconChainTypes>(
|
||||
"Invalid execution payload";
|
||||
"validation_error" => ?validation_error,
|
||||
"latest_valid_hash" => ?latest_valid_hash,
|
||||
"execution_block_hash" => ?execution_payload.execution_payload.block_hash,
|
||||
"execution_block_hash" => ?execution_payload.block_hash(),
|
||||
"root" => ?block.tree_hash_root(),
|
||||
"graffiti" => block.body().graffiti().as_utf8_lossy(),
|
||||
"proposer_index" => block.proposer_index(),
|
||||
@ -158,7 +155,7 @@ async fn notify_new_payload<'a, T: BeaconChainTypes>(
|
||||
chain.log,
|
||||
"Invalid execution payload block hash";
|
||||
"validation_error" => ?validation_error,
|
||||
"execution_block_hash" => ?execution_payload.execution_payload.block_hash,
|
||||
"execution_block_hash" => ?execution_payload.block_hash(),
|
||||
"root" => ?block.tree_hash_root(),
|
||||
"graffiti" => block.body().graffiti().as_utf8_lossy(),
|
||||
"proposer_index" => block.proposer_index(),
|
||||
@ -311,7 +308,7 @@ pub fn validate_execution_payload_for_gossip<T: BeaconChainTypes>(
|
||||
}
|
||||
};
|
||||
|
||||
if is_merge_transition_complete || execution_payload != &<_>::default() {
|
||||
if is_merge_transition_complete || !execution_payload.is_default() {
|
||||
let expected_timestamp = chain
|
||||
.slot_clock
|
||||
.start_of(block.slot())
|
||||
@ -349,13 +346,13 @@ pub fn validate_execution_payload_for_gossip<T: BeaconChainTypes>(
|
||||
/// https://github.com/ethereum/consensus-specs/blob/v1.1.5/specs/merge/validator.md#block-proposal
|
||||
pub fn get_execution_payload<
|
||||
T: BeaconChainTypes,
|
||||
Payload: ExecPayload<T::EthSpec> + Default + Send + 'static,
|
||||
Payload: AbstractExecPayload<T::EthSpec> + 'static,
|
||||
>(
|
||||
chain: Arc<BeaconChain<T>>,
|
||||
state: &BeaconState<T::EthSpec>,
|
||||
proposer_index: u64,
|
||||
builder_params: BuilderParams,
|
||||
) -> Result<PreparePayloadHandle<Payload, T::EthSpec>, BlockProductionError> {
|
||||
) -> Result<PreparePayloadHandle<T::EthSpec, Payload>, BlockProductionError> {
|
||||
// Compute all required values from the `state` now to avoid needing to pass it into a spawned
|
||||
// task.
|
||||
let spec = &chain.spec;
|
||||
@ -364,7 +361,7 @@ pub fn get_execution_payload<
|
||||
let timestamp = compute_timestamp_at_slot(state, spec).map_err(BeaconStateError::from)?;
|
||||
let random = *state.get_randao_mix(current_epoch)?;
|
||||
let latest_execution_payload_header_block_hash =
|
||||
state.latest_execution_payload_header()?.block_hash;
|
||||
state.latest_execution_payload_header()?.block_hash();
|
||||
|
||||
// Spawn a task to obtain the execution payload from the EL via a series of async calls. The
|
||||
// `join_handle` can be used to await the result of the function.
|
||||
@ -414,13 +411,14 @@ pub async fn prepare_execution_payload<T, Payload>(
|
||||
proposer_index: u64,
|
||||
latest_execution_payload_header_block_hash: ExecutionBlockHash,
|
||||
builder_params: BuilderParams,
|
||||
) -> PreparePayloadResult<Payload, T::EthSpec>
|
||||
) -> Result<BlockProposalContents<T::EthSpec, Payload>, BlockProductionError>
|
||||
where
|
||||
T: BeaconChainTypes,
|
||||
Payload: ExecPayload<T::EthSpec> + Default,
|
||||
Payload: AbstractExecPayload<T::EthSpec>,
|
||||
{
|
||||
let current_epoch = builder_params.slot.epoch(T::EthSpec::slots_per_epoch());
|
||||
let spec = &chain.spec;
|
||||
let fork = spec.fork_name_at_slot::<T::EthSpec>(builder_params.slot);
|
||||
let execution_layer = chain
|
||||
.execution_layer
|
||||
.as_ref()
|
||||
@ -434,7 +432,7 @@ where
|
||||
if is_terminal_block_hash_set && !is_activation_epoch_reached {
|
||||
// Use the "empty" payload if there's a terminal block hash, but we haven't reached the
|
||||
// terminal block epoch yet.
|
||||
return Ok(<_>::default());
|
||||
return Ok(BlockProposalContents::default_at_fork(fork));
|
||||
}
|
||||
|
||||
let terminal_pow_block_hash = execution_layer
|
||||
@ -447,7 +445,7 @@ where
|
||||
} else {
|
||||
// If the merge transition hasn't occurred yet and the EL hasn't found the terminal
|
||||
// block, return an "empty" payload.
|
||||
return Ok(<_>::default());
|
||||
return Ok(BlockProposalContents::default_at_fork(fork));
|
||||
}
|
||||
} else {
|
||||
latest_execution_payload_header_block_hash
|
||||
@ -474,8 +472,8 @@ where
|
||||
// Note: the suggested_fee_recipient is stored in the `execution_layer`, it will add this parameter.
|
||||
//
|
||||
// This future is not executed here, it's up to the caller to await it.
|
||||
let (execution_payload_result, blobs_result) = tokio::join!(
|
||||
execution_layer.get_payload::<Payload>(
|
||||
let block_contents = execution_layer
|
||||
.get_payload::<Payload>(
|
||||
parent_hash,
|
||||
timestamp,
|
||||
random,
|
||||
@ -483,20 +481,9 @@ where
|
||||
forkchoice_update_params,
|
||||
builder_params,
|
||||
&chain.spec,
|
||||
),
|
||||
execution_layer.get_blob_bundles(parent_hash, timestamp, random, proposer_index)
|
||||
);
|
||||
)
|
||||
.await
|
||||
.map_err(BlockProductionError::GetPayloadFailed)?;
|
||||
|
||||
let execution_payload =
|
||||
execution_payload_result.map_err(BlockProductionError::GetPayloadFailed)?;
|
||||
let blobs = blobs_result.map_err(BlockProductionError::GetPayloadFailed)?;
|
||||
|
||||
if execution_payload.block_hash() != blobs.block_hash {
|
||||
return Err(BlockProductionError::BlobPayloadMismatch {
|
||||
blob_block_hash: blobs.block_hash,
|
||||
payload_block_hash: execution_payload.block_hash(),
|
||||
});
|
||||
}
|
||||
|
||||
Ok((execution_payload, blobs.kzgs, blobs.blobs))
|
||||
Ok(block_contents)
|
||||
}
|
||||
|
@ -1,6 +1,6 @@
|
||||
use eth2::types::builder_bid::SignedBuilderBid;
|
||||
use eth2::types::{
|
||||
BlindedPayload, EthSpec, ExecPayload, ExecutionBlockHash, ExecutionPayload,
|
||||
AbstractExecPayload, BlindedPayload, EthSpec, ExecutionBlockHash, ExecutionPayload,
|
||||
ForkVersionedResponse, PublicKeyBytes, SignedBeaconBlock, SignedValidatorRegistrationData,
|
||||
Slot,
|
||||
};
|
||||
@ -160,7 +160,7 @@ impl BuilderHttpClient {
|
||||
}
|
||||
|
||||
/// `GET /eth/v1/builder/header`
|
||||
pub async fn get_builder_header<E: EthSpec, Payload: ExecPayload<E>>(
|
||||
pub async fn get_builder_header<E: EthSpec, Payload: AbstractExecPayload<E>>(
|
||||
&self,
|
||||
slot: Slot,
|
||||
parent_hash: ExecutionBlockHash,
|
||||
|
@ -26,6 +26,7 @@ eth2_ssz = "0.4.1"
|
||||
eth2_ssz_types = "0.2.2"
|
||||
eth2 = { path = "../../common/eth2" }
|
||||
state_processing = { path = "../../consensus/state_processing" }
|
||||
superstruct = "0.6.0"
|
||||
lru = "0.7.1"
|
||||
exit-future = "0.2.0"
|
||||
tree_hash = "0.4.1"
|
||||
|
@ -1,13 +1,16 @@
|
||||
use crate::engines::ForkChoiceState;
|
||||
pub use ethers_core::types::Transaction;
|
||||
use ethers_core::utils::rlp::{Decodable, Rlp};
|
||||
use http::deposit_methods::RpcError;
|
||||
pub use json_structures::TransitionConfigurationV1;
|
||||
use reqwest::StatusCode;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use strum::IntoStaticStr;
|
||||
use superstruct::superstruct;
|
||||
use types::Withdrawal;
|
||||
pub use types::{
|
||||
blob::Blob, Address, EthSpec, ExecutionBlockHash, ExecutionPayload, ExecutionPayloadHeader,
|
||||
FixedVector, Hash256, KzgCommitment, KzgProof, Uint256, VariableList,
|
||||
Address, EthSpec, ExecutionBlockHash, ExecutionPayload, ExecutionPayloadHeader, FixedVector,
|
||||
Hash256, Uint256, VariableList,
|
||||
};
|
||||
|
||||
pub mod auth;
|
||||
@ -38,7 +41,9 @@ pub enum Error {
|
||||
PayloadConversionLogicFlaw,
|
||||
DeserializeTransaction(ssz_types::Error),
|
||||
DeserializeTransactions(ssz_types::Error),
|
||||
DeserializeWithdrawals(ssz_types::Error),
|
||||
BuilderApi(builder_client::Error),
|
||||
IncorrectStateVariant,
|
||||
}
|
||||
|
||||
impl From<reqwest::Error> for Error {
|
||||
@ -111,9 +116,18 @@ pub struct ExecutionBlock {
|
||||
pub timestamp: u64,
|
||||
}
|
||||
|
||||
/// Representation of an exection block with enough detail to reconstruct a payload.
|
||||
/// Representation of an execution block with enough detail to reconstruct a payload.
|
||||
#[superstruct(
|
||||
variants(Merge, Capella, Eip4844),
|
||||
variant_attributes(
|
||||
derive(Clone, Debug, PartialEq, Serialize, Deserialize,),
|
||||
serde(bound = "T: EthSpec", rename_all = "camelCase"),
|
||||
),
|
||||
cast_error(ty = "Error", expr = "Error::IncorrectStateVariant"),
|
||||
partial_getter_error(ty = "Error", expr = "Error::IncorrectStateVariant")
|
||||
)]
|
||||
#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
#[serde(bound = "T: EthSpec", rename_all = "camelCase", untagged)]
|
||||
pub struct ExecutionBlockWithTransactions<T: EthSpec> {
|
||||
pub parent_hash: ExecutionBlockHash,
|
||||
#[serde(alias = "miner")]
|
||||
@ -135,16 +149,120 @@ pub struct ExecutionBlockWithTransactions<T: EthSpec> {
|
||||
#[serde(with = "ssz_types::serde_utils::hex_var_list")]
|
||||
pub extra_data: VariableList<u8, T::MaxExtraDataBytes>,
|
||||
pub base_fee_per_gas: Uint256,
|
||||
#[superstruct(only(Eip4844))]
|
||||
#[serde(with = "eth2_serde_utils::u64_hex_be")]
|
||||
pub excess_blobs: u64,
|
||||
#[serde(rename = "hash")]
|
||||
pub block_hash: ExecutionBlockHash,
|
||||
pub transactions: Vec<Transaction>,
|
||||
#[superstruct(only(Capella, Eip4844))]
|
||||
pub withdrawals: Vec<Withdrawal>,
|
||||
}
|
||||
|
||||
#[derive(Clone, Copy, Debug, PartialEq)]
|
||||
impl<T: EthSpec> From<ExecutionPayload<T>> for ExecutionBlockWithTransactions<T> {
|
||||
fn from(payload: ExecutionPayload<T>) -> Self {
|
||||
match payload {
|
||||
ExecutionPayload::Merge(block) => Self::Merge(ExecutionBlockWithTransactionsMerge {
|
||||
parent_hash: block.parent_hash,
|
||||
fee_recipient: block.fee_recipient,
|
||||
state_root: block.state_root,
|
||||
receipts_root: block.receipts_root,
|
||||
logs_bloom: block.logs_bloom,
|
||||
prev_randao: block.prev_randao,
|
||||
block_number: block.block_number,
|
||||
gas_limit: block.gas_limit,
|
||||
gas_used: block.gas_used,
|
||||
timestamp: block.timestamp,
|
||||
extra_data: block.extra_data,
|
||||
base_fee_per_gas: block.base_fee_per_gas,
|
||||
block_hash: block.block_hash,
|
||||
transactions: block
|
||||
.transactions
|
||||
.iter()
|
||||
.map(|tx| Transaction::decode(&Rlp::new(tx)))
|
||||
.collect::<Result<Vec<_>, _>>()
|
||||
.unwrap_or_else(|_| Vec::new()),
|
||||
}),
|
||||
ExecutionPayload::Capella(block) => {
|
||||
Self::Capella(ExecutionBlockWithTransactionsCapella {
|
||||
parent_hash: block.parent_hash,
|
||||
fee_recipient: block.fee_recipient,
|
||||
state_root: block.state_root,
|
||||
receipts_root: block.receipts_root,
|
||||
logs_bloom: block.logs_bloom,
|
||||
prev_randao: block.prev_randao,
|
||||
block_number: block.block_number,
|
||||
gas_limit: block.gas_limit,
|
||||
gas_used: block.gas_used,
|
||||
timestamp: block.timestamp,
|
||||
extra_data: block.extra_data,
|
||||
base_fee_per_gas: block.base_fee_per_gas,
|
||||
block_hash: block.block_hash,
|
||||
transactions: block
|
||||
.transactions
|
||||
.iter()
|
||||
.map(|tx| Transaction::decode(&Rlp::new(tx)))
|
||||
.collect::<Result<Vec<_>, _>>()
|
||||
.unwrap_or_else(|_| Vec::new()),
|
||||
withdrawals: block.withdrawals.into(),
|
||||
})
|
||||
}
|
||||
ExecutionPayload::Eip4844(block) => {
|
||||
Self::Eip4844(ExecutionBlockWithTransactionsEip4844 {
|
||||
parent_hash: block.parent_hash,
|
||||
fee_recipient: block.fee_recipient,
|
||||
state_root: block.state_root,
|
||||
receipts_root: block.receipts_root,
|
||||
logs_bloom: block.logs_bloom,
|
||||
prev_randao: block.prev_randao,
|
||||
block_number: block.block_number,
|
||||
gas_limit: block.gas_limit,
|
||||
gas_used: block.gas_used,
|
||||
timestamp: block.timestamp,
|
||||
extra_data: block.extra_data,
|
||||
base_fee_per_gas: block.base_fee_per_gas,
|
||||
excess_blobs: block.excess_blobs,
|
||||
block_hash: block.block_hash,
|
||||
transactions: block
|
||||
.transactions
|
||||
.iter()
|
||||
.map(|tx| Transaction::decode(&Rlp::new(tx)))
|
||||
.collect::<Result<Vec<_>, _>>()
|
||||
.unwrap_or_else(|_| Vec::new()),
|
||||
withdrawals: block.withdrawals.into(),
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
impl<T: EthSpec> From<ExecutionBlockWithTransactions<T>> for ExecutionPayload<T> {
|
||||
fn from(block: ExecutionBlockWithTransactions<T>) -> Self {
|
||||
map_execution_block_with_transactions!(block, |inner, cons| {
|
||||
let block = inner.into();
|
||||
cons(block)
|
||||
})
|
||||
}
|
||||
}
|
||||
*/
|
||||
|
||||
#[superstruct(
|
||||
variants(V1, V2),
|
||||
variant_attributes(derive(Clone, Debug, PartialEq),),
|
||||
cast_error(ty = "Error", expr = "Error::IncorrectStateVariant"),
|
||||
partial_getter_error(ty = "Error", expr = "Error::IncorrectStateVariant")
|
||||
)]
|
||||
#[derive(Clone, Debug, PartialEq)]
|
||||
pub struct PayloadAttributes {
|
||||
#[superstruct(getter(copy))]
|
||||
pub timestamp: u64,
|
||||
#[superstruct(getter(copy))]
|
||||
pub prev_randao: Hash256,
|
||||
#[superstruct(getter(copy))]
|
||||
pub suggested_fee_recipient: Address,
|
||||
#[superstruct(only(V2))]
|
||||
pub withdrawals: Vec<Withdrawal>,
|
||||
}
|
||||
|
||||
#[derive(Clone, Debug, PartialEq)]
|
||||
|
@ -7,6 +7,7 @@ use reqwest::header::CONTENT_TYPE;
|
||||
use sensitive_url::SensitiveUrl;
|
||||
use serde::de::DeserializeOwned;
|
||||
use serde_json::json;
|
||||
|
||||
use std::time::Duration;
|
||||
use types::EthSpec;
|
||||
|
||||
@ -46,7 +47,7 @@ pub const ENGINE_EXCHANGE_TRANSITION_CONFIGURATION_V1_TIMEOUT: Duration = Durati
|
||||
/// This error is returned during a `chainId` call by Geth.
|
||||
pub const EIP155_ERROR_STR: &str = "chain not synced beyond EIP-155 replay-protection fork block";
|
||||
|
||||
/// Contains methods to convert arbitary bytes to an ETH2 deposit contract object.
|
||||
/// Contains methods to convert arbitrary bytes to an ETH2 deposit contract object.
|
||||
pub mod deposit_log {
|
||||
use ssz::Decode;
|
||||
use state_processing::per_block_processing::signature_sets::deposit_pubkey_signature_message;
|
||||
@ -644,7 +645,7 @@ impl HttpJsonRpc {
|
||||
&self,
|
||||
execution_payload: ExecutionPayload<T>,
|
||||
) -> Result<PayloadStatusV1, Error> {
|
||||
let params = json!([JsonExecutionPayloadV1::from(execution_payload)]);
|
||||
let params = json!([JsonExecutionPayload::from(execution_payload)]);
|
||||
|
||||
let response: JsonPayloadStatusV1 = self
|
||||
.rpc_request(ENGINE_NEW_PAYLOAD_V1, params, ENGINE_NEW_PAYLOAD_TIMEOUT)
|
||||
@ -659,7 +660,7 @@ impl HttpJsonRpc {
|
||||
) -> Result<ExecutionPayload<T>, Error> {
|
||||
let params = json!([JsonPayloadIdRequest::from(payload_id)]);
|
||||
|
||||
let response: JsonExecutionPayloadV1<T> = self
|
||||
let response: JsonExecutionPayload<T> = self
|
||||
.rpc_request(ENGINE_GET_PAYLOAD_V1, params, ENGINE_GET_PAYLOAD_TIMEOUT)
|
||||
.await?;
|
||||
|
||||
@ -669,10 +670,10 @@ impl HttpJsonRpc {
|
||||
pub async fn get_blobs_bundle_v1<T: EthSpec>(
|
||||
&self,
|
||||
payload_id: PayloadId,
|
||||
) -> Result<JsonBlobBundlesV1<T>, Error> {
|
||||
) -> Result<JsonBlobBundles<T>, Error> {
|
||||
let params = json!([JsonPayloadIdRequest::from(payload_id)]);
|
||||
|
||||
let response: JsonBlobBundlesV1<T> = self
|
||||
let response: JsonBlobBundles<T> = self
|
||||
.rpc_request(
|
||||
ENGINE_GET_BLOBS_BUNDLE_V1,
|
||||
params,
|
||||
@ -690,7 +691,7 @@ impl HttpJsonRpc {
|
||||
) -> Result<ForkchoiceUpdatedResponse, Error> {
|
||||
let params = json!([
|
||||
JsonForkChoiceStateV1::from(forkchoice_state),
|
||||
payload_attributes.map(JsonPayloadAttributesV1::from)
|
||||
payload_attributes.map(JsonPayloadAttributes::from)
|
||||
]);
|
||||
|
||||
let response: JsonForkchoiceUpdatedV1Response = self
|
||||
@ -730,7 +731,10 @@ mod test {
|
||||
use std::future::Future;
|
||||
use std::str::FromStr;
|
||||
use std::sync::Arc;
|
||||
use types::{MainnetEthSpec, Transactions, Unsigned, VariableList};
|
||||
use types::{
|
||||
AbstractExecPayload, ExecutionPayloadMerge, ForkName, FullPayload, MainnetEthSpec,
|
||||
Transactions, Unsigned, VariableList,
|
||||
};
|
||||
|
||||
struct Tester {
|
||||
server: MockServer<MainnetEthSpec>,
|
||||
@ -836,10 +840,10 @@ mod test {
|
||||
fn encode_transactions<E: EthSpec>(
|
||||
transactions: Transactions<E>,
|
||||
) -> Result<serde_json::Value, serde_json::Error> {
|
||||
let ep: JsonExecutionPayloadV1<E> = JsonExecutionPayloadV1 {
|
||||
let ep: JsonExecutionPayload<E> = JsonExecutionPayload::V1(JsonExecutionPayloadV1 {
|
||||
transactions,
|
||||
..<_>::default()
|
||||
};
|
||||
});
|
||||
let json = serde_json::to_value(&ep)?;
|
||||
Ok(json.get("transactions").unwrap().clone())
|
||||
}
|
||||
@ -866,8 +870,8 @@ mod test {
|
||||
json.as_object_mut()
|
||||
.unwrap()
|
||||
.insert("transactions".into(), transactions);
|
||||
let ep: JsonExecutionPayloadV1<E> = serde_json::from_value(json)?;
|
||||
Ok(ep.transactions)
|
||||
let ep: JsonExecutionPayload<E> = serde_json::from_value(json)?;
|
||||
Ok(ep.transactions().clone())
|
||||
}
|
||||
|
||||
fn assert_transactions_serde<E: EthSpec>(
|
||||
@ -1018,11 +1022,11 @@ mod test {
|
||||
safe_block_hash: ExecutionBlockHash::repeat_byte(1),
|
||||
finalized_block_hash: ExecutionBlockHash::zero(),
|
||||
},
|
||||
Some(PayloadAttributes {
|
||||
Some(PayloadAttributes::V1(PayloadAttributesV1 {
|
||||
timestamp: 5,
|
||||
prev_randao: Hash256::zero(),
|
||||
suggested_fee_recipient: Address::repeat_byte(0),
|
||||
}),
|
||||
})),
|
||||
)
|
||||
.await;
|
||||
},
|
||||
@ -1053,11 +1057,11 @@ mod test {
|
||||
safe_block_hash: ExecutionBlockHash::repeat_byte(1),
|
||||
finalized_block_hash: ExecutionBlockHash::zero(),
|
||||
},
|
||||
Some(PayloadAttributes {
|
||||
Some(PayloadAttributes::V1(PayloadAttributesV1 {
|
||||
timestamp: 5,
|
||||
prev_randao: Hash256::zero(),
|
||||
suggested_fee_recipient: Address::repeat_byte(0),
|
||||
}),
|
||||
})),
|
||||
)
|
||||
.await
|
||||
})
|
||||
@ -1093,22 +1097,24 @@ mod test {
|
||||
.assert_request_equals(
|
||||
|client| async move {
|
||||
let _ = client
|
||||
.new_payload_v1::<MainnetEthSpec>(ExecutionPayload {
|
||||
parent_hash: ExecutionBlockHash::repeat_byte(0),
|
||||
fee_recipient: Address::repeat_byte(1),
|
||||
state_root: Hash256::repeat_byte(1),
|
||||
receipts_root: Hash256::repeat_byte(0),
|
||||
logs_bloom: vec![1; 256].into(),
|
||||
prev_randao: Hash256::repeat_byte(1),
|
||||
block_number: 0,
|
||||
gas_limit: 1,
|
||||
gas_used: 2,
|
||||
timestamp: 42,
|
||||
extra_data: vec![].into(),
|
||||
base_fee_per_gas: Uint256::from(1),
|
||||
block_hash: ExecutionBlockHash::repeat_byte(1),
|
||||
transactions: vec![].into(),
|
||||
})
|
||||
.new_payload_v1::<MainnetEthSpec>(ExecutionPayload::Merge(
|
||||
ExecutionPayloadMerge {
|
||||
parent_hash: ExecutionBlockHash::repeat_byte(0),
|
||||
fee_recipient: Address::repeat_byte(1),
|
||||
state_root: Hash256::repeat_byte(1),
|
||||
receipts_root: Hash256::repeat_byte(0),
|
||||
logs_bloom: vec![1; 256].into(),
|
||||
prev_randao: Hash256::repeat_byte(1),
|
||||
block_number: 0,
|
||||
gas_limit: 1,
|
||||
gas_used: 2,
|
||||
timestamp: 42,
|
||||
extra_data: vec![].into(),
|
||||
base_fee_per_gas: Uint256::from(1),
|
||||
block_hash: ExecutionBlockHash::repeat_byte(1),
|
||||
transactions: vec![].into(),
|
||||
},
|
||||
))
|
||||
.await;
|
||||
},
|
||||
json!({
|
||||
@ -1138,22 +1144,24 @@ mod test {
|
||||
Tester::new(false)
|
||||
.assert_auth_failure(|client| async move {
|
||||
client
|
||||
.new_payload_v1::<MainnetEthSpec>(ExecutionPayload {
|
||||
parent_hash: ExecutionBlockHash::repeat_byte(0),
|
||||
fee_recipient: Address::repeat_byte(1),
|
||||
state_root: Hash256::repeat_byte(1),
|
||||
receipts_root: Hash256::repeat_byte(0),
|
||||
logs_bloom: vec![1; 256].into(),
|
||||
prev_randao: Hash256::repeat_byte(1),
|
||||
block_number: 0,
|
||||
gas_limit: 1,
|
||||
gas_used: 2,
|
||||
timestamp: 42,
|
||||
extra_data: vec![].into(),
|
||||
base_fee_per_gas: Uint256::from(1),
|
||||
block_hash: ExecutionBlockHash::repeat_byte(1),
|
||||
transactions: vec![].into(),
|
||||
})
|
||||
.new_payload_v1::<MainnetEthSpec>(ExecutionPayload::Merge(
|
||||
ExecutionPayloadMerge {
|
||||
parent_hash: ExecutionBlockHash::repeat_byte(0),
|
||||
fee_recipient: Address::repeat_byte(1),
|
||||
state_root: Hash256::repeat_byte(1),
|
||||
receipts_root: Hash256::repeat_byte(0),
|
||||
logs_bloom: vec![1; 256].into(),
|
||||
prev_randao: Hash256::repeat_byte(1),
|
||||
block_number: 0,
|
||||
gas_limit: 1,
|
||||
gas_used: 2,
|
||||
timestamp: 42,
|
||||
extra_data: vec![].into(),
|
||||
base_fee_per_gas: Uint256::from(1),
|
||||
block_hash: ExecutionBlockHash::repeat_byte(1),
|
||||
transactions: vec![].into(),
|
||||
},
|
||||
))
|
||||
.await
|
||||
})
|
||||
.await;
|
||||
@ -1236,11 +1244,11 @@ mod test {
|
||||
safe_block_hash: ExecutionBlockHash::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
|
||||
finalized_block_hash: ExecutionBlockHash::zero(),
|
||||
},
|
||||
Some(PayloadAttributes {
|
||||
Some(PayloadAttributes::V1(PayloadAttributesV1 {
|
||||
timestamp: 5,
|
||||
prev_randao: Hash256::zero(),
|
||||
suggested_fee_recipient: Address::from_str("0xa94f5374fce5edbc8e2a8697c15331677e6ebf0b").unwrap(),
|
||||
})
|
||||
}))
|
||||
)
|
||||
.await;
|
||||
},
|
||||
@ -1283,11 +1291,11 @@ mod test {
|
||||
safe_block_hash: ExecutionBlockHash::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
|
||||
finalized_block_hash: ExecutionBlockHash::zero(),
|
||||
},
|
||||
Some(PayloadAttributes {
|
||||
Some(PayloadAttributes::V1(PayloadAttributesV1 {
|
||||
timestamp: 5,
|
||||
prev_randao: Hash256::zero(),
|
||||
suggested_fee_recipient: Address::from_str("0xa94f5374fce5edbc8e2a8697c15331677e6ebf0b").unwrap(),
|
||||
})
|
||||
}))
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
@ -1346,7 +1354,7 @@ mod test {
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let expected = ExecutionPayload {
|
||||
let expected = ExecutionPayload::Merge(ExecutionPayloadMerge {
|
||||
parent_hash: ExecutionBlockHash::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
|
||||
fee_recipient: Address::from_str("0xa94f5374fce5edbc8e2a8697c15331677e6ebf0b").unwrap(),
|
||||
state_root: Hash256::from_str("0xca3149fa9e37db08d1cd49c9061db1002ef1cd58db2210f2115c8c989b2bdf45").unwrap(),
|
||||
@ -1361,7 +1369,7 @@ mod test {
|
||||
base_fee_per_gas: Uint256::from(7),
|
||||
block_hash: ExecutionBlockHash::from_str("0x6359b8381a370e2f54072a5784ddd78b6ed024991558c511d4452eb4f6ac898c").unwrap(),
|
||||
transactions: vec![].into(),
|
||||
};
|
||||
});
|
||||
|
||||
assert_eq!(payload, expected);
|
||||
},
|
||||
@ -1371,7 +1379,7 @@ mod test {
|
||||
// engine_newPayloadV1 REQUEST validation
|
||||
|client| async move {
|
||||
let _ = client
|
||||
.new_payload_v1::<MainnetEthSpec>(ExecutionPayload {
|
||||
.new_payload_v1::<MainnetEthSpec>(ExecutionPayload::Merge(ExecutionPayloadMerge{
|
||||
parent_hash: ExecutionBlockHash::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
|
||||
fee_recipient: Address::from_str("0xa94f5374fce5edbc8e2a8697c15331677e6ebf0b").unwrap(),
|
||||
state_root: Hash256::from_str("0xca3149fa9e37db08d1cd49c9061db1002ef1cd58db2210f2115c8c989b2bdf45").unwrap(),
|
||||
@ -1386,7 +1394,7 @@ mod test {
|
||||
base_fee_per_gas: Uint256::from(7),
|
||||
block_hash: ExecutionBlockHash::from_str("0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858").unwrap(),
|
||||
transactions: vec![].into(),
|
||||
})
|
||||
}))
|
||||
.await;
|
||||
},
|
||||
json!({
|
||||
@ -1425,7 +1433,7 @@ mod test {
|
||||
})],
|
||||
|client| async move {
|
||||
let response = client
|
||||
.new_payload_v1::<MainnetEthSpec>(ExecutionPayload::default())
|
||||
.new_payload_v1::<MainnetEthSpec>(FullPayload::default_at_fork(ForkName::Merge).into())
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
|
@ -1,6 +1,12 @@
|
||||
use super::*;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use types::{Blob, EthSpec, ExecutionBlockHash, FixedVector, Transaction, Unsigned, VariableList};
|
||||
use superstruct::superstruct;
|
||||
use types::{
|
||||
Blob, EthSpec, ExecutionBlockHash, ExecutionPayloadEip4844, ExecutionPayloadHeaderEip4844,
|
||||
FixedVector, KzgCommitment, Transaction, Unsigned, VariableList,
|
||||
};
|
||||
use types::{ExecutionPayload, ExecutionPayloadCapella, ExecutionPayloadMerge};
|
||||
use types::{ExecutionPayloadHeader, ExecutionPayloadHeaderCapella, ExecutionPayloadHeaderMerge};
|
||||
|
||||
#[derive(Debug, PartialEq, Serialize, Deserialize)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
@ -55,9 +61,19 @@ pub struct JsonPayloadIdResponse {
|
||||
pub payload_id: PayloadId,
|
||||
}
|
||||
|
||||
#[derive(Debug, PartialEq, Default, Serialize, Deserialize)]
|
||||
#[serde(bound = "T: EthSpec", rename_all = "camelCase")]
|
||||
pub struct JsonExecutionPayloadHeaderV1<T: EthSpec> {
|
||||
// (V1,V2,V3) -> (Merge,Capella,EIP4844)
|
||||
#[superstruct(
|
||||
variants(V1, V2, V3),
|
||||
variant_attributes(
|
||||
derive(Debug, PartialEq, Default, Serialize, Deserialize,),
|
||||
serde(bound = "T: EthSpec", rename_all = "camelCase"),
|
||||
),
|
||||
cast_error(ty = "Error", expr = "Error::IncorrectStateVariant"),
|
||||
partial_getter_error(ty = "Error", expr = "Error::IncorrectStateVariant")
|
||||
)]
|
||||
#[derive(Debug, PartialEq, Serialize, Deserialize)]
|
||||
#[serde(bound = "T: EthSpec", rename_all = "camelCase", untagged)]
|
||||
pub struct JsonExecutionPayloadHeader<T: EthSpec> {
|
||||
pub parent_hash: ExecutionBlockHash,
|
||||
pub fee_recipient: Address,
|
||||
pub state_root: Hash256,
|
||||
@ -77,52 +93,144 @@ pub struct JsonExecutionPayloadHeaderV1<T: EthSpec> {
|
||||
pub extra_data: VariableList<u8, T::MaxExtraDataBytes>,
|
||||
#[serde(with = "eth2_serde_utils::u256_hex_be")]
|
||||
pub base_fee_per_gas: Uint256,
|
||||
#[serde(with = "eth2_serde_utils::u64_hex_be")]
|
||||
#[superstruct(only(V3))]
|
||||
pub excess_blobs: u64,
|
||||
pub block_hash: ExecutionBlockHash,
|
||||
pub transactions_root: Hash256,
|
||||
#[superstruct(only(V2, V3))]
|
||||
pub withdrawals_root: Hash256,
|
||||
}
|
||||
|
||||
impl<T: EthSpec> From<JsonExecutionPayloadHeaderV1<T>> for ExecutionPayloadHeader<T> {
|
||||
fn from(e: JsonExecutionPayloadHeaderV1<T>) -> Self {
|
||||
// Use this verbose deconstruction pattern to ensure no field is left unused.
|
||||
let JsonExecutionPayloadHeaderV1 {
|
||||
parent_hash,
|
||||
fee_recipient,
|
||||
state_root,
|
||||
receipts_root,
|
||||
logs_bloom,
|
||||
prev_randao,
|
||||
block_number,
|
||||
gas_limit,
|
||||
gas_used,
|
||||
timestamp,
|
||||
extra_data,
|
||||
base_fee_per_gas,
|
||||
block_hash,
|
||||
transactions_root,
|
||||
} = e;
|
||||
|
||||
Self {
|
||||
parent_hash,
|
||||
fee_recipient,
|
||||
state_root,
|
||||
receipts_root,
|
||||
logs_bloom,
|
||||
prev_randao,
|
||||
block_number,
|
||||
gas_limit,
|
||||
gas_used,
|
||||
timestamp,
|
||||
extra_data,
|
||||
base_fee_per_gas,
|
||||
block_hash,
|
||||
transactions_root,
|
||||
impl<T: EthSpec> From<JsonExecutionPayloadHeader<T>> for ExecutionPayloadHeader<T> {
|
||||
fn from(json_header: JsonExecutionPayloadHeader<T>) -> Self {
|
||||
match json_header {
|
||||
JsonExecutionPayloadHeader::V1(v1) => Self::Merge(ExecutionPayloadHeaderMerge {
|
||||
parent_hash: v1.parent_hash,
|
||||
fee_recipient: v1.fee_recipient,
|
||||
state_root: v1.state_root,
|
||||
receipts_root: v1.receipts_root,
|
||||
logs_bloom: v1.logs_bloom,
|
||||
prev_randao: v1.prev_randao,
|
||||
block_number: v1.block_number,
|
||||
gas_limit: v1.gas_limit,
|
||||
gas_used: v1.gas_used,
|
||||
timestamp: v1.timestamp,
|
||||
extra_data: v1.extra_data,
|
||||
base_fee_per_gas: v1.base_fee_per_gas,
|
||||
block_hash: v1.block_hash,
|
||||
transactions_root: v1.transactions_root,
|
||||
}),
|
||||
JsonExecutionPayloadHeader::V2(v2) => Self::Capella(ExecutionPayloadHeaderCapella {
|
||||
parent_hash: v2.parent_hash,
|
||||
fee_recipient: v2.fee_recipient,
|
||||
state_root: v2.state_root,
|
||||
receipts_root: v2.receipts_root,
|
||||
logs_bloom: v2.logs_bloom,
|
||||
prev_randao: v2.prev_randao,
|
||||
block_number: v2.block_number,
|
||||
gas_limit: v2.gas_limit,
|
||||
gas_used: v2.gas_used,
|
||||
timestamp: v2.timestamp,
|
||||
extra_data: v2.extra_data,
|
||||
base_fee_per_gas: v2.base_fee_per_gas,
|
||||
block_hash: v2.block_hash,
|
||||
transactions_root: v2.transactions_root,
|
||||
withdrawals_root: v2.withdrawals_root,
|
||||
}),
|
||||
JsonExecutionPayloadHeader::V3(v3) => Self::Eip4844(ExecutionPayloadHeaderEip4844 {
|
||||
parent_hash: v3.parent_hash,
|
||||
fee_recipient: v3.fee_recipient,
|
||||
state_root: v3.state_root,
|
||||
receipts_root: v3.receipts_root,
|
||||
logs_bloom: v3.logs_bloom,
|
||||
prev_randao: v3.prev_randao,
|
||||
block_number: v3.block_number,
|
||||
gas_limit: v3.gas_limit,
|
||||
gas_used: v3.gas_used,
|
||||
timestamp: v3.timestamp,
|
||||
extra_data: v3.extra_data,
|
||||
base_fee_per_gas: v3.base_fee_per_gas,
|
||||
excess_blobs: v3.excess_blobs,
|
||||
block_hash: v3.block_hash,
|
||||
transactions_root: v3.transactions_root,
|
||||
withdrawals_root: v3.withdrawals_root,
|
||||
}),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug, PartialEq, Default, Serialize, Deserialize)]
|
||||
#[serde(bound = "T: EthSpec", rename_all = "camelCase")]
|
||||
pub struct JsonExecutionPayloadV1<T: EthSpec> {
|
||||
impl<T: EthSpec> From<ExecutionPayloadHeader<T>> for JsonExecutionPayloadHeader<T> {
|
||||
fn from(header: ExecutionPayloadHeader<T>) -> Self {
|
||||
match header {
|
||||
ExecutionPayloadHeader::Merge(merge) => Self::V1(JsonExecutionPayloadHeaderV1 {
|
||||
parent_hash: merge.parent_hash,
|
||||
fee_recipient: merge.fee_recipient,
|
||||
state_root: merge.state_root,
|
||||
receipts_root: merge.receipts_root,
|
||||
logs_bloom: merge.logs_bloom,
|
||||
prev_randao: merge.prev_randao,
|
||||
block_number: merge.block_number,
|
||||
gas_limit: merge.gas_limit,
|
||||
gas_used: merge.gas_used,
|
||||
timestamp: merge.timestamp,
|
||||
extra_data: merge.extra_data,
|
||||
base_fee_per_gas: merge.base_fee_per_gas,
|
||||
block_hash: merge.block_hash,
|
||||
transactions_root: merge.transactions_root,
|
||||
}),
|
||||
ExecutionPayloadHeader::Capella(capella) => Self::V2(JsonExecutionPayloadHeaderV2 {
|
||||
parent_hash: capella.parent_hash,
|
||||
fee_recipient: capella.fee_recipient,
|
||||
state_root: capella.state_root,
|
||||
receipts_root: capella.receipts_root,
|
||||
logs_bloom: capella.logs_bloom,
|
||||
prev_randao: capella.prev_randao,
|
||||
block_number: capella.block_number,
|
||||
gas_limit: capella.gas_limit,
|
||||
gas_used: capella.gas_used,
|
||||
timestamp: capella.timestamp,
|
||||
extra_data: capella.extra_data,
|
||||
base_fee_per_gas: capella.base_fee_per_gas,
|
||||
block_hash: capella.block_hash,
|
||||
transactions_root: capella.transactions_root,
|
||||
withdrawals_root: capella.withdrawals_root,
|
||||
}),
|
||||
ExecutionPayloadHeader::Eip4844(eip4844) => Self::V3(JsonExecutionPayloadHeaderV3 {
|
||||
parent_hash: eip4844.parent_hash,
|
||||
fee_recipient: eip4844.fee_recipient,
|
||||
state_root: eip4844.state_root,
|
||||
receipts_root: eip4844.receipts_root,
|
||||
logs_bloom: eip4844.logs_bloom,
|
||||
prev_randao: eip4844.prev_randao,
|
||||
block_number: eip4844.block_number,
|
||||
gas_limit: eip4844.gas_limit,
|
||||
gas_used: eip4844.gas_used,
|
||||
timestamp: eip4844.timestamp,
|
||||
extra_data: eip4844.extra_data,
|
||||
base_fee_per_gas: eip4844.base_fee_per_gas,
|
||||
excess_blobs: eip4844.excess_blobs,
|
||||
block_hash: eip4844.block_hash,
|
||||
transactions_root: eip4844.transactions_root,
|
||||
withdrawals_root: eip4844.withdrawals_root,
|
||||
}),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// (V1,V2, V2) -> (Merge,Capella,EIP4844)
|
||||
#[superstruct(
|
||||
variants(V1, V2, V3),
|
||||
variant_attributes(
|
||||
derive(Debug, PartialEq, Default, Serialize, Deserialize,),
|
||||
serde(bound = "T: EthSpec", rename_all = "camelCase"),
|
||||
),
|
||||
cast_error(ty = "Error", expr = "Error::IncorrectStateVariant"),
|
||||
partial_getter_error(ty = "Error", expr = "Error::IncorrectStateVariant")
|
||||
)]
|
||||
#[derive(Debug, PartialEq, Serialize, Deserialize)]
|
||||
#[serde(bound = "T: EthSpec", rename_all = "camelCase", untagged)]
|
||||
pub struct JsonExecutionPayload<T: EthSpec> {
|
||||
pub parent_hash: ExecutionBlockHash,
|
||||
pub fee_recipient: Address,
|
||||
pub state_root: Hash256,
|
||||
@ -142,136 +250,219 @@ pub struct JsonExecutionPayloadV1<T: EthSpec> {
|
||||
pub extra_data: VariableList<u8, T::MaxExtraDataBytes>,
|
||||
#[serde(with = "eth2_serde_utils::u256_hex_be")]
|
||||
pub base_fee_per_gas: Uint256,
|
||||
#[superstruct(only(V3))]
|
||||
#[serde(with = "eth2_serde_utils::u64_hex_be")]
|
||||
pub excess_blobs: u64,
|
||||
pub block_hash: ExecutionBlockHash,
|
||||
#[serde(with = "ssz_types::serde_utils::list_of_hex_var_list")]
|
||||
pub transactions:
|
||||
VariableList<Transaction<T::MaxBytesPerTransaction>, T::MaxTransactionsPerPayload>,
|
||||
#[superstruct(only(V2, V3))]
|
||||
pub withdrawals: VariableList<Withdrawal, T::MaxWithdrawalsPerPayload>,
|
||||
}
|
||||
|
||||
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,
|
||||
fee_recipient,
|
||||
state_root,
|
||||
receipts_root,
|
||||
logs_bloom,
|
||||
prev_randao,
|
||||
block_number,
|
||||
gas_limit,
|
||||
gas_used,
|
||||
timestamp,
|
||||
extra_data,
|
||||
base_fee_per_gas,
|
||||
block_hash,
|
||||
transactions,
|
||||
} = e;
|
||||
|
||||
Self {
|
||||
parent_hash,
|
||||
fee_recipient,
|
||||
state_root,
|
||||
receipts_root,
|
||||
logs_bloom,
|
||||
prev_randao,
|
||||
block_number,
|
||||
gas_limit,
|
||||
gas_used,
|
||||
timestamp,
|
||||
extra_data,
|
||||
base_fee_per_gas,
|
||||
block_hash,
|
||||
transactions,
|
||||
impl<T: EthSpec> From<JsonExecutionPayload<T>> for ExecutionPayload<T> {
|
||||
fn from(json_payload: JsonExecutionPayload<T>) -> Self {
|
||||
match json_payload {
|
||||
JsonExecutionPayload::V1(v1) => Self::Merge(ExecutionPayloadMerge {
|
||||
parent_hash: v1.parent_hash,
|
||||
fee_recipient: v1.fee_recipient,
|
||||
state_root: v1.state_root,
|
||||
receipts_root: v1.receipts_root,
|
||||
logs_bloom: v1.logs_bloom,
|
||||
prev_randao: v1.prev_randao,
|
||||
block_number: v1.block_number,
|
||||
gas_limit: v1.gas_limit,
|
||||
gas_used: v1.gas_used,
|
||||
timestamp: v1.timestamp,
|
||||
extra_data: v1.extra_data,
|
||||
base_fee_per_gas: v1.base_fee_per_gas,
|
||||
block_hash: v1.block_hash,
|
||||
transactions: v1.transactions,
|
||||
}),
|
||||
JsonExecutionPayload::V2(v2) => Self::Capella(ExecutionPayloadCapella {
|
||||
parent_hash: v2.parent_hash,
|
||||
fee_recipient: v2.fee_recipient,
|
||||
state_root: v2.state_root,
|
||||
receipts_root: v2.receipts_root,
|
||||
logs_bloom: v2.logs_bloom,
|
||||
prev_randao: v2.prev_randao,
|
||||
block_number: v2.block_number,
|
||||
gas_limit: v2.gas_limit,
|
||||
gas_used: v2.gas_used,
|
||||
timestamp: v2.timestamp,
|
||||
extra_data: v2.extra_data,
|
||||
base_fee_per_gas: v2.base_fee_per_gas,
|
||||
block_hash: v2.block_hash,
|
||||
transactions: v2.transactions,
|
||||
withdrawals: v2.withdrawals,
|
||||
}),
|
||||
JsonExecutionPayload::V3(v3) => Self::Eip4844(ExecutionPayloadEip4844 {
|
||||
parent_hash: v3.parent_hash,
|
||||
fee_recipient: v3.fee_recipient,
|
||||
state_root: v3.state_root,
|
||||
receipts_root: v3.receipts_root,
|
||||
logs_bloom: v3.logs_bloom,
|
||||
prev_randao: v3.prev_randao,
|
||||
block_number: v3.block_number,
|
||||
gas_limit: v3.gas_limit,
|
||||
gas_used: v3.gas_used,
|
||||
timestamp: v3.timestamp,
|
||||
extra_data: v3.extra_data,
|
||||
base_fee_per_gas: v3.base_fee_per_gas,
|
||||
excess_blobs: v3.excess_blobs,
|
||||
block_hash: v3.block_hash,
|
||||
transactions: v3.transactions,
|
||||
withdrawals: v3.withdrawals,
|
||||
}),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
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,
|
||||
fee_recipient,
|
||||
state_root,
|
||||
receipts_root,
|
||||
logs_bloom,
|
||||
prev_randao,
|
||||
block_number,
|
||||
gas_limit,
|
||||
gas_used,
|
||||
timestamp,
|
||||
extra_data,
|
||||
base_fee_per_gas,
|
||||
block_hash,
|
||||
transactions,
|
||||
} = e;
|
||||
|
||||
Self {
|
||||
parent_hash,
|
||||
fee_recipient,
|
||||
state_root,
|
||||
receipts_root,
|
||||
logs_bloom,
|
||||
prev_randao,
|
||||
block_number,
|
||||
gas_limit,
|
||||
gas_used,
|
||||
timestamp,
|
||||
extra_data,
|
||||
base_fee_per_gas,
|
||||
block_hash,
|
||||
transactions,
|
||||
impl<T: EthSpec> From<ExecutionPayload<T>> for JsonExecutionPayload<T> {
|
||||
fn from(payload: ExecutionPayload<T>) -> Self {
|
||||
match payload {
|
||||
ExecutionPayload::Merge(merge) => Self::V1(JsonExecutionPayloadV1 {
|
||||
parent_hash: merge.parent_hash,
|
||||
fee_recipient: merge.fee_recipient,
|
||||
state_root: merge.state_root,
|
||||
receipts_root: merge.receipts_root,
|
||||
logs_bloom: merge.logs_bloom,
|
||||
prev_randao: merge.prev_randao,
|
||||
block_number: merge.block_number,
|
||||
gas_limit: merge.gas_limit,
|
||||
gas_used: merge.gas_used,
|
||||
timestamp: merge.timestamp,
|
||||
extra_data: merge.extra_data,
|
||||
base_fee_per_gas: merge.base_fee_per_gas,
|
||||
block_hash: merge.block_hash,
|
||||
transactions: merge.transactions,
|
||||
}),
|
||||
ExecutionPayload::Capella(capella) => Self::V2(JsonExecutionPayloadV2 {
|
||||
parent_hash: capella.parent_hash,
|
||||
fee_recipient: capella.fee_recipient,
|
||||
state_root: capella.state_root,
|
||||
receipts_root: capella.receipts_root,
|
||||
logs_bloom: capella.logs_bloom,
|
||||
prev_randao: capella.prev_randao,
|
||||
block_number: capella.block_number,
|
||||
gas_limit: capella.gas_limit,
|
||||
gas_used: capella.gas_used,
|
||||
timestamp: capella.timestamp,
|
||||
extra_data: capella.extra_data,
|
||||
base_fee_per_gas: capella.base_fee_per_gas,
|
||||
block_hash: capella.block_hash,
|
||||
transactions: capella.transactions,
|
||||
withdrawals: capella.withdrawals,
|
||||
}),
|
||||
ExecutionPayload::Eip4844(eip4844) => Self::V3(JsonExecutionPayloadV3 {
|
||||
parent_hash: eip4844.parent_hash,
|
||||
fee_recipient: eip4844.fee_recipient,
|
||||
state_root: eip4844.state_root,
|
||||
receipts_root: eip4844.receipts_root,
|
||||
logs_bloom: eip4844.logs_bloom,
|
||||
prev_randao: eip4844.prev_randao,
|
||||
block_number: eip4844.block_number,
|
||||
gas_limit: eip4844.gas_limit,
|
||||
gas_used: eip4844.gas_used,
|
||||
timestamp: eip4844.timestamp,
|
||||
extra_data: eip4844.extra_data,
|
||||
base_fee_per_gas: eip4844.base_fee_per_gas,
|
||||
excess_blobs: eip4844.excess_blobs,
|
||||
block_hash: eip4844.block_hash,
|
||||
transactions: eip4844.transactions,
|
||||
withdrawals: eip4844.withdrawals,
|
||||
}),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)]
|
||||
pub struct JsonWithdrawal {
|
||||
#[serde(with = "eth2_serde_utils::u64_hex_be")]
|
||||
pub index: u64,
|
||||
pub address: Address,
|
||||
#[serde(with = "eth2_serde_utils::u256_hex_be")]
|
||||
pub amount: Uint256,
|
||||
}
|
||||
|
||||
impl From<Withdrawal> for JsonWithdrawal {
|
||||
fn from(withdrawal: Withdrawal) -> Self {
|
||||
Self {
|
||||
index: withdrawal.index,
|
||||
address: withdrawal.address,
|
||||
amount: Uint256::from((withdrawal.amount as u128) * 1000000000u128),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl From<JsonWithdrawal> for Withdrawal {
|
||||
fn from(jw: JsonWithdrawal) -> Self {
|
||||
Self {
|
||||
index: jw.index,
|
||||
address: jw.address,
|
||||
//FIXME(sean) if EE gives us too large a number this panics
|
||||
amount: (jw.amount / 1000000000).as_u64(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[superstruct(
|
||||
variants(V1, V2),
|
||||
variant_attributes(derive(Clone, Debug, PartialEq, Serialize, Deserialize),),
|
||||
cast_error(ty = "Error", expr = "Error::IncorrectStateVariant"),
|
||||
partial_getter_error(ty = "Error", expr = "Error::IncorrectStateVariant")
|
||||
)]
|
||||
#[derive(Debug, PartialEq, Serialize, Deserialize)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
pub struct JsonPayloadAttributesV1 {
|
||||
#[serde(rename_all = "camelCase", untagged)]
|
||||
pub struct JsonPayloadAttributes {
|
||||
#[serde(with = "eth2_serde_utils::u64_hex_be")]
|
||||
pub timestamp: u64,
|
||||
pub prev_randao: Hash256,
|
||||
pub suggested_fee_recipient: Address,
|
||||
#[superstruct(only(V2))]
|
||||
pub withdrawals: Vec<JsonWithdrawal>,
|
||||
}
|
||||
|
||||
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,
|
||||
prev_randao,
|
||||
suggested_fee_recipient,
|
||||
} = p;
|
||||
|
||||
Self {
|
||||
timestamp,
|
||||
prev_randao,
|
||||
suggested_fee_recipient,
|
||||
impl From<PayloadAttributes> for JsonPayloadAttributes {
|
||||
fn from(payload_atributes: PayloadAttributes) -> Self {
|
||||
match payload_atributes {
|
||||
PayloadAttributes::V1(pa) => Self::V1(JsonPayloadAttributesV1 {
|
||||
timestamp: pa.timestamp,
|
||||
prev_randao: pa.prev_randao,
|
||||
suggested_fee_recipient: pa.suggested_fee_recipient,
|
||||
}),
|
||||
PayloadAttributes::V2(pa) => Self::V2(JsonPayloadAttributesV2 {
|
||||
timestamp: pa.timestamp,
|
||||
prev_randao: pa.prev_randao,
|
||||
suggested_fee_recipient: pa.suggested_fee_recipient,
|
||||
withdrawals: pa.withdrawals.into_iter().map(Into::into).collect(),
|
||||
}),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
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,
|
||||
prev_randao,
|
||||
suggested_fee_recipient,
|
||||
} = j;
|
||||
|
||||
Self {
|
||||
timestamp,
|
||||
prev_randao,
|
||||
suggested_fee_recipient,
|
||||
impl From<JsonPayloadAttributes> for PayloadAttributes {
|
||||
fn from(json_payload_attributes: JsonPayloadAttributes) -> Self {
|
||||
match json_payload_attributes {
|
||||
JsonPayloadAttributes::V1(jpa) => Self::V1(PayloadAttributesV1 {
|
||||
timestamp: jpa.timestamp,
|
||||
prev_randao: jpa.prev_randao,
|
||||
suggested_fee_recipient: jpa.suggested_fee_recipient,
|
||||
}),
|
||||
JsonPayloadAttributes::V2(jpa) => Self::V2(PayloadAttributesV2 {
|
||||
timestamp: jpa.timestamp,
|
||||
prev_randao: jpa.prev_randao,
|
||||
suggested_fee_recipient: jpa.suggested_fee_recipient,
|
||||
withdrawals: jpa.withdrawals.into_iter().map(Into::into).collect(),
|
||||
}),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug, PartialEq, Serialize, Deserialize)]
|
||||
#[serde(bound = "T: EthSpec", rename_all = "camelCase")]
|
||||
pub struct JsonBlobBundlesV1<T: EthSpec> {
|
||||
pub struct JsonBlobBundles<T: EthSpec> {
|
||||
pub block_hash: ExecutionBlockHash,
|
||||
pub kzgs: Vec<KzgCommitment>,
|
||||
pub blobs: Vec<Blob<T>>,
|
||||
|
@ -167,7 +167,7 @@ impl Engine {
|
||||
) -> Result<ForkchoiceUpdatedResponse, EngineApiError> {
|
||||
let response = self
|
||||
.api
|
||||
.forkchoice_updated_v1(forkchoice_state, payload_attributes)
|
||||
.forkchoice_updated_v1(forkchoice_state, payload_attributes.clone())
|
||||
.await?;
|
||||
|
||||
if let Some(payload_id) = response.payload_id {
|
||||
@ -347,13 +347,14 @@ impl Engine {
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: revisit this - do we need to key on withdrawals as well here?
|
||||
impl PayloadIdCacheKey {
|
||||
fn new(state: &ForkChoiceState, attributes: &PayloadAttributes) -> Self {
|
||||
Self {
|
||||
head_block_hash: state.head_block_hash,
|
||||
timestamp: attributes.timestamp,
|
||||
prev_randao: attributes.prev_randao,
|
||||
suggested_fee_recipient: attributes.suggested_fee_recipient,
|
||||
timestamp: attributes.timestamp(),
|
||||
prev_randao: attributes.prev_randao(),
|
||||
suggested_fee_recipient: attributes.suggested_fee_recipient(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -4,7 +4,7 @@
|
||||
//! This crate only provides useful functionality for "The Merge", it does not provide any of the
|
||||
//! deposit-contract functionality that the `beacon_node/eth1` crate already provides.
|
||||
|
||||
use crate::json_structures::JsonBlobBundlesV1;
|
||||
use crate::json_structures::JsonBlobBundles;
|
||||
use crate::payload_cache::PayloadCache;
|
||||
use auth::{strip_prefix, Auth, JwtKey};
|
||||
use builder_client::BuilderHttpClient;
|
||||
@ -33,10 +33,12 @@ use tokio::{
|
||||
time::sleep,
|
||||
};
|
||||
use tokio_stream::wrappers::WatchStream;
|
||||
use types::{AbstractExecPayload, Blob, ExecPayload, ExecutionPayloadEip4844, KzgCommitment};
|
||||
use types::{
|
||||
BlindedPayload, BlockType, ChainSpec, Epoch, ExecPayload, ExecutionBlockHash, ForkName,
|
||||
BlindedPayload, BlockType, ChainSpec, Epoch, ExecutionBlockHash, ForkName,
|
||||
ProposerPreparationData, PublicKeyBytes, SignedBeaconBlock, Slot,
|
||||
};
|
||||
use types::{ExecutionPayload, ExecutionPayloadCapella, ExecutionPayloadMerge};
|
||||
|
||||
mod engine_api;
|
||||
mod engines;
|
||||
@ -88,6 +90,70 @@ impl From<ApiError> for Error {
|
||||
}
|
||||
}
|
||||
|
||||
pub enum BlockProposalContents<T: EthSpec, Payload: AbstractExecPayload<T>> {
|
||||
Payload(Payload),
|
||||
PayloadAndBlobs {
|
||||
payload: Payload,
|
||||
kzg_commitments: Vec<KzgCommitment>,
|
||||
blobs: Vec<Blob<T>>,
|
||||
},
|
||||
}
|
||||
|
||||
impl<T: EthSpec, Payload: AbstractExecPayload<T>> BlockProposalContents<T, Payload> {
|
||||
pub fn payload(&self) -> &Payload {
|
||||
match self {
|
||||
Self::Payload(payload) => payload,
|
||||
Self::PayloadAndBlobs {
|
||||
payload,
|
||||
kzg_commitments: _,
|
||||
blobs: _,
|
||||
} => payload,
|
||||
}
|
||||
}
|
||||
pub fn to_payload(self) -> Payload {
|
||||
match self {
|
||||
Self::Payload(payload) => payload,
|
||||
Self::PayloadAndBlobs {
|
||||
payload,
|
||||
kzg_commitments: _,
|
||||
blobs: _,
|
||||
} => payload,
|
||||
}
|
||||
}
|
||||
pub fn kzg_commitments(&self) -> Option<&[KzgCommitment]> {
|
||||
match self {
|
||||
Self::Payload(_) => None,
|
||||
Self::PayloadAndBlobs {
|
||||
payload: _,
|
||||
kzg_commitments,
|
||||
blobs: _,
|
||||
} => Some(kzg_commitments),
|
||||
}
|
||||
}
|
||||
pub fn blobs(&self) -> Option<&[Blob<T>]> {
|
||||
match self {
|
||||
Self::Payload(_) => None,
|
||||
Self::PayloadAndBlobs {
|
||||
payload: _,
|
||||
kzg_commitments: _,
|
||||
blobs,
|
||||
} => Some(blobs),
|
||||
}
|
||||
}
|
||||
pub fn default_at_fork(fork_name: ForkName) -> Self {
|
||||
match fork_name {
|
||||
ForkName::Base | ForkName::Altair | ForkName::Merge | ForkName::Capella => {
|
||||
BlockProposalContents::Payload(Payload::default_at_fork(fork_name))
|
||||
}
|
||||
ForkName::Eip4844 => BlockProposalContents::PayloadAndBlobs {
|
||||
payload: Payload::default_at_fork(fork_name),
|
||||
blobs: vec![],
|
||||
kzg_commitments: vec![],
|
||||
},
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, PartialEq)]
|
||||
pub struct ProposerPreparationDataEntry {
|
||||
update_epoch: Epoch,
|
||||
@ -536,7 +602,7 @@ impl<T: EthSpec> ExecutionLayer<T> {
|
||||
/// The result will be returned from the first node that returns successfully. No more nodes
|
||||
/// will be contacted.
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
pub async fn get_payload<Payload: ExecPayload<T>>(
|
||||
pub async fn get_payload<Payload: AbstractExecPayload<T>>(
|
||||
&self,
|
||||
parent_hash: ExecutionBlockHash,
|
||||
timestamp: u64,
|
||||
@ -545,7 +611,7 @@ impl<T: EthSpec> ExecutionLayer<T> {
|
||||
forkchoice_update_params: ForkchoiceUpdateParameters,
|
||||
builder_params: BuilderParams,
|
||||
spec: &ChainSpec,
|
||||
) -> Result<Payload, Error> {
|
||||
) -> Result<BlockProposalContents<T, Payload>, Error> {
|
||||
let suggested_fee_recipient = self.get_suggested_fee_recipient(proposer_index).await;
|
||||
|
||||
match Payload::block_type() {
|
||||
@ -583,7 +649,7 @@ impl<T: EthSpec> ExecutionLayer<T> {
|
||||
}
|
||||
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
async fn get_blinded_payload<Payload: ExecPayload<T>>(
|
||||
async fn get_blinded_payload<Payload: AbstractExecPayload<T>>(
|
||||
&self,
|
||||
parent_hash: ExecutionBlockHash,
|
||||
timestamp: u64,
|
||||
@ -592,7 +658,7 @@ impl<T: EthSpec> ExecutionLayer<T> {
|
||||
forkchoice_update_params: ForkchoiceUpdateParameters,
|
||||
builder_params: BuilderParams,
|
||||
spec: &ChainSpec,
|
||||
) -> Result<Payload, Error> {
|
||||
) -> Result<BlockProposalContents<T, Payload>, Error> {
|
||||
if let Some(builder) = self.builder() {
|
||||
let slot = builder_params.slot;
|
||||
let pubkey = builder_params.pubkey;
|
||||
@ -635,6 +701,7 @@ impl<T: EthSpec> ExecutionLayer<T> {
|
||||
Ok(local)
|
||||
}
|
||||
(Ok(Some(relay)), Ok(local)) => {
|
||||
let local_payload = local.payload();
|
||||
let is_signature_valid = relay.data.verify_signature(spec);
|
||||
let header = relay.data.message.header;
|
||||
|
||||
@ -668,14 +735,14 @@ impl<T: EthSpec> ExecutionLayer<T> {
|
||||
falling back to local execution engine."
|
||||
);
|
||||
Ok(local)
|
||||
} else if header.timestamp() != local.timestamp() {
|
||||
} else if header.timestamp() != local_payload.timestamp() {
|
||||
warn!(
|
||||
self.log(),
|
||||
"Invalid timestamp from connected builder, \
|
||||
falling back to local execution engine."
|
||||
);
|
||||
Ok(local)
|
||||
} else if header.block_number() != local.block_number() {
|
||||
} else if header.block_number() != local_payload.block_number() {
|
||||
warn!(
|
||||
self.log(),
|
||||
"Invalid block number from connected builder, \
|
||||
@ -706,7 +773,8 @@ impl<T: EthSpec> ExecutionLayer<T> {
|
||||
not match, using it anyways."
|
||||
);
|
||||
}
|
||||
Ok(header)
|
||||
//FIXME(sean) the builder API needs to be updated
|
||||
Ok(BlockProposalContents::Payload(header))
|
||||
}
|
||||
}
|
||||
(relay_result, Err(local_error)) => {
|
||||
@ -715,7 +783,10 @@ impl<T: EthSpec> ExecutionLayer<T> {
|
||||
relay_result
|
||||
.map_err(Error::Builder)?
|
||||
.ok_or(Error::NoHeaderFromBuilder)
|
||||
.map(|d| d.data.message.header)
|
||||
.map(|d| {
|
||||
//FIXME(sean) the builder API needs to be updated
|
||||
BlockProposalContents::Payload(d.data.message.header)
|
||||
})
|
||||
}
|
||||
};
|
||||
}
|
||||
@ -743,14 +814,14 @@ impl<T: EthSpec> ExecutionLayer<T> {
|
||||
}
|
||||
|
||||
/// Get a full payload without caching its result in the execution layer's payload cache.
|
||||
async fn get_full_payload<Payload: ExecPayload<T>>(
|
||||
async fn get_full_payload<Payload: AbstractExecPayload<T>>(
|
||||
&self,
|
||||
parent_hash: ExecutionBlockHash,
|
||||
timestamp: u64,
|
||||
prev_randao: Hash256,
|
||||
suggested_fee_recipient: Address,
|
||||
forkchoice_update_params: ForkchoiceUpdateParameters,
|
||||
) -> Result<Payload, Error> {
|
||||
) -> Result<BlockProposalContents<T, Payload>, Error> {
|
||||
self.get_full_payload_with(
|
||||
parent_hash,
|
||||
timestamp,
|
||||
@ -763,14 +834,14 @@ impl<T: EthSpec> ExecutionLayer<T> {
|
||||
}
|
||||
|
||||
/// Get a full payload and cache its result in the execution layer's payload cache.
|
||||
async fn get_full_payload_caching<Payload: ExecPayload<T>>(
|
||||
async fn get_full_payload_caching<Payload: AbstractExecPayload<T>>(
|
||||
&self,
|
||||
parent_hash: ExecutionBlockHash,
|
||||
timestamp: u64,
|
||||
prev_randao: Hash256,
|
||||
suggested_fee_recipient: Address,
|
||||
forkchoice_update_params: ForkchoiceUpdateParameters,
|
||||
) -> Result<Payload, Error> {
|
||||
) -> Result<BlockProposalContents<T, Payload>, Error> {
|
||||
self.get_full_payload_with(
|
||||
parent_hash,
|
||||
timestamp,
|
||||
@ -782,51 +853,7 @@ impl<T: EthSpec> ExecutionLayer<T> {
|
||||
.await
|
||||
}
|
||||
|
||||
pub async fn get_blob_bundles(
|
||||
&self,
|
||||
parent_hash: ExecutionBlockHash,
|
||||
timestamp: u64,
|
||||
prev_randao: Hash256,
|
||||
proposer_index: u64,
|
||||
) -> Result<JsonBlobBundlesV1<T>, Error> {
|
||||
let suggested_fee_recipient = self.get_suggested_fee_recipient(proposer_index).await;
|
||||
|
||||
debug!(
|
||||
self.log(),
|
||||
"Issuing engine_getBlobsBundle";
|
||||
"suggested_fee_recipient" => ?suggested_fee_recipient,
|
||||
"prev_randao" => ?prev_randao,
|
||||
"timestamp" => timestamp,
|
||||
"parent_hash" => ?parent_hash,
|
||||
);
|
||||
self.engine()
|
||||
.request(|engine| async move {
|
||||
let payload_id = if let Some(id) = engine
|
||||
.get_payload_id(parent_hash, timestamp, prev_randao, suggested_fee_recipient)
|
||||
.await
|
||||
{
|
||||
// The payload id has been cached for this engine.
|
||||
metrics::inc_counter_vec(
|
||||
&metrics::EXECUTION_LAYER_PRE_PREPARED_PAYLOAD_ID,
|
||||
&[metrics::HIT],
|
||||
);
|
||||
id
|
||||
} else {
|
||||
error!(
|
||||
self.log(),
|
||||
"Exec engine unable to produce blobs, did you call get_payload before?",
|
||||
);
|
||||
return Err(ApiError::PayloadIdUnavailable);
|
||||
};
|
||||
|
||||
engine.api.get_blobs_bundle_v1::<T>(payload_id).await
|
||||
})
|
||||
.await
|
||||
.map_err(Box::new)
|
||||
.map_err(Error::EngineError)
|
||||
}
|
||||
|
||||
async fn get_full_payload_with<Payload: ExecPayload<T>>(
|
||||
async fn get_full_payload_with<Payload: AbstractExecPayload<T>>(
|
||||
&self,
|
||||
parent_hash: ExecutionBlockHash,
|
||||
timestamp: u64,
|
||||
@ -834,15 +861,7 @@ impl<T: EthSpec> ExecutionLayer<T> {
|
||||
suggested_fee_recipient: Address,
|
||||
forkchoice_update_params: ForkchoiceUpdateParameters,
|
||||
f: fn(&ExecutionLayer<T>, &ExecutionPayload<T>) -> Option<ExecutionPayload<T>>,
|
||||
) -> Result<Payload, Error> {
|
||||
debug!(
|
||||
self.log(),
|
||||
"Issuing engine_getPayload";
|
||||
"suggested_fee_recipient" => ?suggested_fee_recipient,
|
||||
"prev_randao" => ?prev_randao,
|
||||
"timestamp" => timestamp,
|
||||
"parent_hash" => ?parent_hash,
|
||||
);
|
||||
) -> Result<BlockProposalContents<T, Payload>, Error> {
|
||||
self.engine()
|
||||
.request(|engine| async move {
|
||||
let payload_id = if let Some(id) = engine
|
||||
@ -871,11 +890,13 @@ impl<T: EthSpec> ExecutionLayer<T> {
|
||||
.finalized_hash
|
||||
.unwrap_or_else(ExecutionBlockHash::zero),
|
||||
};
|
||||
let payload_attributes = PayloadAttributes {
|
||||
// FIXME: This will have to properly handle forks. To do that,
|
||||
// withdrawals will need to be passed into this function
|
||||
let payload_attributes = PayloadAttributes::V1(PayloadAttributesV1 {
|
||||
timestamp,
|
||||
prev_randao,
|
||||
suggested_fee_recipient,
|
||||
};
|
||||
});
|
||||
|
||||
let response = engine
|
||||
.notify_forkchoice_updated(
|
||||
@ -900,33 +921,64 @@ impl<T: EthSpec> ExecutionLayer<T> {
|
||||
}
|
||||
};
|
||||
|
||||
engine
|
||||
.api
|
||||
.get_payload_v1::<T>(payload_id)
|
||||
.await
|
||||
.map(|full_payload| {
|
||||
if full_payload.fee_recipient != suggested_fee_recipient {
|
||||
error!(
|
||||
self.log(),
|
||||
"Inconsistent fee recipient";
|
||||
"msg" => "The fee recipient returned from the Execution Engine differs \
|
||||
from the suggested_fee_recipient set on the beacon node. This could \
|
||||
indicate that fees are being diverted to another address. Please \
|
||||
ensure that the value of suggested_fee_recipient is set correctly and \
|
||||
that the Execution Engine is trusted.",
|
||||
"fee_recipient" => ?full_payload.fee_recipient,
|
||||
"suggested_fee_recipient" => ?suggested_fee_recipient,
|
||||
);
|
||||
}
|
||||
if f(self, &full_payload).is_some() {
|
||||
warn!(
|
||||
self.log(),
|
||||
"Duplicate payload cached, this might indicate redundant proposal \
|
||||
let blob_fut = async {
|
||||
//FIXME(sean) do a fork check here and return None otherwise
|
||||
debug!(
|
||||
self.log(),
|
||||
"Issuing engine_getBlobsBundle";
|
||||
"suggested_fee_recipient" => ?suggested_fee_recipient,
|
||||
"prev_randao" => ?prev_randao,
|
||||
"timestamp" => timestamp,
|
||||
"parent_hash" => ?parent_hash,
|
||||
);
|
||||
Some(engine.api.get_blobs_bundle_v1::<T>(payload_id).await)
|
||||
};
|
||||
let payload_fut = async {
|
||||
debug!(
|
||||
self.log(),
|
||||
"Issuing engine_getPayload";
|
||||
"suggested_fee_recipient" => ?suggested_fee_recipient,
|
||||
"prev_randao" => ?prev_randao,
|
||||
"timestamp" => timestamp,
|
||||
"parent_hash" => ?parent_hash,
|
||||
);
|
||||
engine.api.get_payload_v1::<T>(payload_id).await
|
||||
};
|
||||
|
||||
let (blob, payload) = tokio::join!(blob_fut, payload_fut);
|
||||
let payload = payload.map(|full_payload| {
|
||||
if full_payload.fee_recipient() != suggested_fee_recipient {
|
||||
error!(
|
||||
self.log(),
|
||||
"Inconsistent fee recipient";
|
||||
"msg" => "The fee recipient returned from the Execution Engine differs \
|
||||
from the suggested_fee_recipient set on the beacon node. This could \
|
||||
indicate that fees are being diverted to another address. Please \
|
||||
ensure that the value of suggested_fee_recipient is set correctly and \
|
||||
that the Execution Engine is trusted.",
|
||||
"fee_recipient" => ?full_payload.fee_recipient(),
|
||||
"suggested_fee_recipient" => ?suggested_fee_recipient,
|
||||
);
|
||||
}
|
||||
if f(self, &full_payload).is_some() {
|
||||
warn!(
|
||||
self.log(),
|
||||
"Duplicate payload cached, this might indicate redundant proposal \
|
||||
attempts."
|
||||
);
|
||||
}
|
||||
full_payload.into()
|
||||
);
|
||||
}
|
||||
full_payload.into()
|
||||
})?;
|
||||
if let Some(blob) = blob.transpose()? {
|
||||
// FIXME(sean) cache blobs
|
||||
Ok(BlockProposalContents::PayloadAndBlobs {
|
||||
payload,
|
||||
blobs: blob.blobs,
|
||||
kzg_commitments: blob.kzgs,
|
||||
})
|
||||
} else {
|
||||
Ok(BlockProposalContents::Payload(payload))
|
||||
}
|
||||
})
|
||||
.await
|
||||
.map_err(Box::new)
|
||||
@ -958,9 +1010,9 @@ impl<T: EthSpec> ExecutionLayer<T> {
|
||||
trace!(
|
||||
self.log(),
|
||||
"Issuing engine_newPayload";
|
||||
"parent_hash" => ?execution_payload.parent_hash,
|
||||
"block_hash" => ?execution_payload.block_hash,
|
||||
"block_number" => execution_payload.block_number,
|
||||
"parent_hash" => ?execution_payload.parent_hash(),
|
||||
"block_hash" => ?execution_payload.block_hash(),
|
||||
"block_number" => execution_payload.block_number(),
|
||||
);
|
||||
|
||||
let result = self
|
||||
@ -975,7 +1027,7 @@ impl<T: EthSpec> ExecutionLayer<T> {
|
||||
);
|
||||
}
|
||||
|
||||
process_payload_status(execution_payload.block_hash, result, self.log())
|
||||
process_payload_status(execution_payload.block_hash(), result, self.log())
|
||||
.map_err(Box::new)
|
||||
.map_err(Error::EngineError)
|
||||
}
|
||||
@ -1076,9 +1128,9 @@ impl<T: EthSpec> ExecutionLayer<T> {
|
||||
let payload_attributes = self.payload_attributes(next_slot, head_block_root).await;
|
||||
|
||||
// Compute the "lookahead", the time between when the payload will be produced and now.
|
||||
if let Some(payload_attributes) = payload_attributes {
|
||||
if let Some(ref payload_attributes) = payload_attributes {
|
||||
if let Ok(now) = SystemTime::now().duration_since(UNIX_EPOCH) {
|
||||
let timestamp = Duration::from_secs(payload_attributes.timestamp);
|
||||
let timestamp = Duration::from_secs(payload_attributes.timestamp());
|
||||
if let Some(lookahead) = timestamp.checked_sub(now) {
|
||||
metrics::observe_duration(
|
||||
&metrics::EXECUTION_LAYER_PAYLOAD_ATTRIBUTES_LOOKAHEAD,
|
||||
@ -1105,11 +1157,16 @@ impl<T: EthSpec> ExecutionLayer<T> {
|
||||
.set_latest_forkchoice_state(forkchoice_state)
|
||||
.await;
|
||||
|
||||
let payload_attributes_ref = &payload_attributes;
|
||||
let result = self
|
||||
.engine()
|
||||
.request(|engine| async move {
|
||||
engine
|
||||
.notify_forkchoice_updated(forkchoice_state, payload_attributes, self.log())
|
||||
.notify_forkchoice_updated(
|
||||
forkchoice_state,
|
||||
payload_attributes_ref.clone(),
|
||||
self.log(),
|
||||
)
|
||||
.await
|
||||
})
|
||||
.await;
|
||||
@ -1399,7 +1456,8 @@ impl<T: EthSpec> ExecutionLayer<T> {
|
||||
let _timer = metrics::start_timer(&metrics::EXECUTION_LAYER_GET_PAYLOAD_BY_BLOCK_HASH);
|
||||
|
||||
if hash == ExecutionBlockHash::zero() {
|
||||
return Ok(Some(ExecutionPayload::default()));
|
||||
// FIXME: how to handle forks properly here?
|
||||
return Ok(Some(ExecutionPayloadMerge::default().into()));
|
||||
}
|
||||
|
||||
let block = if let Some(block) = engine.api.get_block_by_hash_with_txns::<T>(hash).await? {
|
||||
@ -1410,7 +1468,7 @@ impl<T: EthSpec> ExecutionLayer<T> {
|
||||
|
||||
let transactions = VariableList::new(
|
||||
block
|
||||
.transactions
|
||||
.transactions()
|
||||
.into_iter()
|
||||
.map(|transaction| VariableList::new(transaction.rlp().to_vec()))
|
||||
.collect::<Result<_, _>>()
|
||||
@ -1418,22 +1476,73 @@ impl<T: EthSpec> ExecutionLayer<T> {
|
||||
)
|
||||
.map_err(ApiError::DeserializeTransactions)?;
|
||||
|
||||
Ok(Some(ExecutionPayload {
|
||||
parent_hash: block.parent_hash,
|
||||
fee_recipient: block.fee_recipient,
|
||||
state_root: block.state_root,
|
||||
receipts_root: block.receipts_root,
|
||||
logs_bloom: block.logs_bloom,
|
||||
prev_randao: block.prev_randao,
|
||||
block_number: block.block_number,
|
||||
gas_limit: block.gas_limit,
|
||||
gas_used: block.gas_used,
|
||||
timestamp: block.timestamp,
|
||||
extra_data: block.extra_data,
|
||||
base_fee_per_gas: block.base_fee_per_gas,
|
||||
block_hash: block.block_hash,
|
||||
transactions,
|
||||
}))
|
||||
let payload = match block {
|
||||
ExecutionBlockWithTransactions::Merge(merge_block) => {
|
||||
ExecutionPayload::Merge(ExecutionPayloadMerge {
|
||||
parent_hash: merge_block.parent_hash,
|
||||
fee_recipient: merge_block.fee_recipient,
|
||||
state_root: merge_block.state_root,
|
||||
receipts_root: merge_block.receipts_root,
|
||||
logs_bloom: merge_block.logs_bloom,
|
||||
prev_randao: merge_block.prev_randao,
|
||||
block_number: merge_block.block_number,
|
||||
gas_limit: merge_block.gas_limit,
|
||||
gas_used: merge_block.gas_used,
|
||||
timestamp: merge_block.timestamp,
|
||||
extra_data: merge_block.extra_data,
|
||||
base_fee_per_gas: merge_block.base_fee_per_gas,
|
||||
block_hash: merge_block.block_hash,
|
||||
transactions,
|
||||
})
|
||||
}
|
||||
ExecutionBlockWithTransactions::Capella(capella_block) => {
|
||||
let withdrawals = VariableList::new(capella_block.withdrawals.clone())
|
||||
.map_err(ApiError::DeserializeWithdrawals)?;
|
||||
|
||||
ExecutionPayload::Capella(ExecutionPayloadCapella {
|
||||
parent_hash: capella_block.parent_hash,
|
||||
fee_recipient: capella_block.fee_recipient,
|
||||
state_root: capella_block.state_root,
|
||||
receipts_root: capella_block.receipts_root,
|
||||
logs_bloom: capella_block.logs_bloom,
|
||||
prev_randao: capella_block.prev_randao,
|
||||
block_number: capella_block.block_number,
|
||||
gas_limit: capella_block.gas_limit,
|
||||
gas_used: capella_block.gas_used,
|
||||
timestamp: capella_block.timestamp,
|
||||
extra_data: capella_block.extra_data,
|
||||
base_fee_per_gas: capella_block.base_fee_per_gas,
|
||||
block_hash: capella_block.block_hash,
|
||||
transactions,
|
||||
withdrawals,
|
||||
})
|
||||
}
|
||||
ExecutionBlockWithTransactions::Eip4844(eip4844_block) => {
|
||||
let withdrawals = VariableList::new(eip4844_block.withdrawals.clone())
|
||||
.map_err(ApiError::DeserializeWithdrawals)?;
|
||||
|
||||
ExecutionPayload::Eip4844(ExecutionPayloadEip4844 {
|
||||
parent_hash: eip4844_block.parent_hash,
|
||||
fee_recipient: eip4844_block.fee_recipient,
|
||||
state_root: eip4844_block.state_root,
|
||||
receipts_root: eip4844_block.receipts_root,
|
||||
logs_bloom: eip4844_block.logs_bloom,
|
||||
prev_randao: eip4844_block.prev_randao,
|
||||
block_number: eip4844_block.block_number,
|
||||
gas_limit: eip4844_block.gas_limit,
|
||||
gas_used: eip4844_block.gas_used,
|
||||
timestamp: eip4844_block.timestamp,
|
||||
extra_data: eip4844_block.extra_data,
|
||||
base_fee_per_gas: eip4844_block.base_fee_per_gas,
|
||||
excess_blobs: eip4844_block.excess_blobs,
|
||||
block_hash: eip4844_block.block_hash,
|
||||
transactions,
|
||||
withdrawals,
|
||||
})
|
||||
}
|
||||
};
|
||||
|
||||
Ok(Some(payload))
|
||||
}
|
||||
|
||||
pub async fn propose_blinded_beacon_block(
|
||||
|
@ -12,7 +12,10 @@ use serde::{Deserialize, Serialize};
|
||||
use std::collections::HashMap;
|
||||
use tree_hash::TreeHash;
|
||||
use tree_hash_derive::TreeHash;
|
||||
use types::{EthSpec, ExecutionBlockHash, ExecutionPayload, Hash256, Uint256};
|
||||
use types::{
|
||||
EthSpec, ExecutionBlockHash, ExecutionPayload, ExecutionPayloadCapella, ExecutionPayloadMerge,
|
||||
Hash256, Uint256,
|
||||
};
|
||||
|
||||
const GAS_LIMIT: u64 = 16384;
|
||||
const GAS_USED: u64 = GAS_LIMIT - 1;
|
||||
@ -28,21 +31,21 @@ impl<T: EthSpec> Block<T> {
|
||||
pub fn block_number(&self) -> u64 {
|
||||
match self {
|
||||
Block::PoW(block) => block.block_number,
|
||||
Block::PoS(payload) => payload.block_number,
|
||||
Block::PoS(payload) => payload.block_number(),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn parent_hash(&self) -> ExecutionBlockHash {
|
||||
match self {
|
||||
Block::PoW(block) => block.parent_hash,
|
||||
Block::PoS(payload) => payload.parent_hash,
|
||||
Block::PoS(payload) => payload.parent_hash(),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn block_hash(&self) -> ExecutionBlockHash {
|
||||
match self {
|
||||
Block::PoW(block) => block.block_hash,
|
||||
Block::PoS(payload) => payload.block_hash,
|
||||
Block::PoS(payload) => payload.block_hash(),
|
||||
}
|
||||
}
|
||||
|
||||
@ -63,33 +66,18 @@ impl<T: EthSpec> Block<T> {
|
||||
timestamp: block.timestamp,
|
||||
},
|
||||
Block::PoS(payload) => ExecutionBlock {
|
||||
block_hash: payload.block_hash,
|
||||
block_number: payload.block_number,
|
||||
parent_hash: payload.parent_hash,
|
||||
block_hash: payload.block_hash(),
|
||||
block_number: payload.block_number(),
|
||||
parent_hash: payload.parent_hash(),
|
||||
total_difficulty,
|
||||
timestamp: payload.timestamp,
|
||||
timestamp: payload.timestamp(),
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
pub fn as_execution_block_with_tx(&self) -> Option<ExecutionBlockWithTransactions<T>> {
|
||||
match self {
|
||||
Block::PoS(payload) => Some(ExecutionBlockWithTransactions {
|
||||
parent_hash: payload.parent_hash,
|
||||
fee_recipient: payload.fee_recipient,
|
||||
state_root: payload.state_root,
|
||||
receipts_root: payload.receipts_root,
|
||||
logs_bloom: payload.logs_bloom.clone(),
|
||||
prev_randao: payload.prev_randao,
|
||||
block_number: payload.block_number,
|
||||
gas_limit: payload.gas_limit,
|
||||
gas_used: payload.gas_used,
|
||||
timestamp: payload.timestamp,
|
||||
extra_data: payload.extra_data.clone(),
|
||||
base_fee_per_gas: payload.base_fee_per_gas,
|
||||
block_hash: payload.block_hash,
|
||||
transactions: vec![],
|
||||
}),
|
||||
Block::PoS(payload) => Some(payload.clone().into()),
|
||||
Block::PoW(_) => None,
|
||||
}
|
||||
}
|
||||
@ -283,7 +271,9 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
|
||||
// Update the block hash after modifying the block
|
||||
match &mut block {
|
||||
Block::PoW(b) => b.block_hash = ExecutionBlockHash::from_root(b.tree_hash_root()),
|
||||
Block::PoS(b) => b.block_hash = ExecutionBlockHash::from_root(b.tree_hash_root()),
|
||||
Block::PoS(b) => {
|
||||
*b.block_hash_mut() = ExecutionBlockHash::from_root(b.tree_hash_root())
|
||||
}
|
||||
}
|
||||
self.block_hashes.insert(block_number, block.block_hash());
|
||||
self.blocks.insert(block.block_hash(), block);
|
||||
@ -295,7 +285,7 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
|
||||
}
|
||||
|
||||
pub fn new_payload(&mut self, payload: ExecutionPayload<T>) -> PayloadStatusV1 {
|
||||
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
|
||||
} else {
|
||||
return PayloadStatusV1 {
|
||||
@ -305,7 +295,7 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
|
||||
};
|
||||
};
|
||||
|
||||
if payload.block_number != parent.block_number() + 1 {
|
||||
if payload.block_number() != parent.block_number() + 1 {
|
||||
return PayloadStatusV1 {
|
||||
status: PayloadStatusV1Status::Invalid,
|
||||
latest_valid_hash: Some(parent.block_hash()),
|
||||
@ -313,8 +303,8 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
|
||||
};
|
||||
}
|
||||
|
||||
let valid_hash = payload.block_hash;
|
||||
self.pending_payloads.insert(payload.block_hash, payload);
|
||||
let valid_hash = payload.block_hash();
|
||||
self.pending_payloads.insert(payload.block_hash(), payload);
|
||||
|
||||
PayloadStatusV1 {
|
||||
status: PayloadStatusV1Status::Valid,
|
||||
@ -379,24 +369,52 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
|
||||
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,
|
||||
fee_recipient: attributes.suggested_fee_recipient,
|
||||
receipts_root: Hash256::repeat_byte(42),
|
||||
state_root: Hash256::repeat_byte(43),
|
||||
logs_bloom: vec![0; 256].into(),
|
||||
prev_randao: attributes.prev_randao,
|
||||
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: ExecutionBlockHash::zero(),
|
||||
transactions: vec![].into(),
|
||||
// FIXME: think about how to test different forks
|
||||
let mut execution_payload = match &attributes {
|
||||
PayloadAttributes::V1(pa) => ExecutionPayload::Merge(ExecutionPayloadMerge {
|
||||
parent_hash: forkchoice_state.head_block_hash,
|
||||
fee_recipient: pa.suggested_fee_recipient,
|
||||
receipts_root: Hash256::repeat_byte(42),
|
||||
state_root: Hash256::repeat_byte(43),
|
||||
logs_bloom: vec![0; 256].into(),
|
||||
prev_randao: pa.prev_randao,
|
||||
block_number: parent.block_number() + 1,
|
||||
gas_limit: GAS_LIMIT,
|
||||
gas_used: GAS_USED,
|
||||
timestamp: pa.timestamp,
|
||||
extra_data: "block gen was here".as_bytes().to_vec().into(),
|
||||
base_fee_per_gas: Uint256::one(),
|
||||
block_hash: ExecutionBlockHash::zero(),
|
||||
transactions: vec![].into(),
|
||||
}),
|
||||
PayloadAttributes::V2(pa) => {
|
||||
ExecutionPayload::Capella(ExecutionPayloadCapella {
|
||||
parent_hash: forkchoice_state.head_block_hash,
|
||||
fee_recipient: pa.suggested_fee_recipient,
|
||||
receipts_root: Hash256::repeat_byte(42),
|
||||
state_root: Hash256::repeat_byte(43),
|
||||
logs_bloom: vec![0; 256].into(),
|
||||
prev_randao: pa.prev_randao,
|
||||
block_number: parent.block_number() + 1,
|
||||
gas_limit: GAS_LIMIT,
|
||||
gas_used: GAS_USED,
|
||||
timestamp: pa.timestamp,
|
||||
extra_data: "block gen was here".as_bytes().to_vec().into(),
|
||||
base_fee_per_gas: Uint256::one(),
|
||||
block_hash: ExecutionBlockHash::zero(),
|
||||
transactions: vec![].into(),
|
||||
withdrawals: pa
|
||||
.withdrawals
|
||||
.iter()
|
||||
.cloned()
|
||||
.map(Into::into)
|
||||
.collect::<Vec<_>>()
|
||||
.into(),
|
||||
})
|
||||
}
|
||||
};
|
||||
|
||||
execution_payload.block_hash =
|
||||
*execution_payload.block_hash_mut() =
|
||||
ExecutionBlockHash::from_root(execution_payload.tree_hash_root());
|
||||
|
||||
self.payload_ids.insert(id, execution_payload);
|
||||
|
@ -75,12 +75,12 @@ pub async fn handle_rpc<T: EthSpec>(
|
||||
}
|
||||
}
|
||||
ENGINE_NEW_PAYLOAD_V1 => {
|
||||
let request: JsonExecutionPayloadV1<T> = get_param(params, 0)?;
|
||||
let request: JsonExecutionPayload<T> = get_param(params, 0)?;
|
||||
|
||||
let (static_response, should_import) =
|
||||
if let Some(mut response) = ctx.static_new_payload_response.lock().clone() {
|
||||
if response.status.status == PayloadStatusV1Status::Valid {
|
||||
response.status.latest_valid_hash = Some(request.block_hash)
|
||||
response.status.latest_valid_hash = Some(*request.block_hash())
|
||||
}
|
||||
|
||||
(Some(response.status), response.should_import)
|
||||
@ -112,11 +112,11 @@ pub async fn handle_rpc<T: EthSpec>(
|
||||
.get_payload(&id)
|
||||
.ok_or_else(|| format!("no payload for id {:?}", id))?;
|
||||
|
||||
Ok(serde_json::to_value(JsonExecutionPayloadV1::from(response)).unwrap())
|
||||
Ok(serde_json::to_value(JsonExecutionPayload::from(response)).unwrap())
|
||||
}
|
||||
ENGINE_FORKCHOICE_UPDATED_V1 => {
|
||||
let forkchoice_state: JsonForkChoiceStateV1 = get_param(params, 0)?;
|
||||
let payload_attributes: Option<JsonPayloadAttributesV1> = get_param(params, 1)?;
|
||||
let payload_attributes: Option<JsonPayloadAttributes> = get_param(params, 1)?;
|
||||
|
||||
let head_block_hash = forkchoice_state.head_block_hash;
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
use crate::test_utils::DEFAULT_JWT_SECRET;
|
||||
use crate::{Config, ExecutionLayer, PayloadAttributes};
|
||||
use crate::{Config, ExecutionLayer, PayloadAttributes, PayloadAttributesV1};
|
||||
use async_trait::async_trait;
|
||||
use eth2::types::{BlockId, StateId, ValidatorId};
|
||||
use eth2::{BeaconNodeHttpClient, Timeouts};
|
||||
@ -287,11 +287,12 @@ impl<E: EthSpec> mev_build_rs::BlindedBlockProvider for MockBuilder<E> {
|
||||
.get_randao_mix(head_state.current_epoch())
|
||||
.map_err(convert_err)?;
|
||||
|
||||
let payload_attributes = PayloadAttributes {
|
||||
// FIXME: think about proper fork here
|
||||
let payload_attributes = PayloadAttributes::V1(PayloadAttributesV1 {
|
||||
timestamp,
|
||||
prev_randao: *prev_randao,
|
||||
suggested_fee_recipient: fee_recipient,
|
||||
};
|
||||
});
|
||||
|
||||
self.el
|
||||
.insert_proposer(slot, head_block_root, val_index, payload_attributes)
|
||||
@ -315,6 +316,7 @@ impl<E: EthSpec> mev_build_rs::BlindedBlockProvider for MockBuilder<E> {
|
||||
)
|
||||
.await
|
||||
.map_err(convert_err)?
|
||||
.to_payload()
|
||||
.to_execution_payload_header();
|
||||
|
||||
let json_payload = serde_json::to_string(&payload).map_err(convert_err)?;
|
||||
|
@ -1,7 +1,7 @@
|
||||
use crate::{
|
||||
test_utils::{
|
||||
MockServer, DEFAULT_BUILDER_THRESHOLD_WEI, DEFAULT_JWT_SECRET, DEFAULT_TERMINAL_BLOCK,
|
||||
DEFAULT_TERMINAL_DIFFICULTY,
|
||||
Block, MockServer, DEFAULT_BUILDER_THRESHOLD_WEI, DEFAULT_JWT_SECRET,
|
||||
DEFAULT_TERMINAL_BLOCK, DEFAULT_TERMINAL_DIFFICULTY,
|
||||
},
|
||||
Config, *,
|
||||
};
|
||||
@ -99,20 +99,37 @@ impl<T: EthSpec> MockExecutionLayer<T> {
|
||||
finalized_hash: None,
|
||||
};
|
||||
|
||||
// FIXME: this is just best guess for how to deal with forks here..
|
||||
let payload_attributes = match &latest_execution_block {
|
||||
&Block::PoS(ref pos_block) => match pos_block {
|
||||
&ExecutionPayload::Merge(_) => PayloadAttributes::V1(PayloadAttributesV1 {
|
||||
timestamp,
|
||||
prev_randao,
|
||||
suggested_fee_recipient: Address::repeat_byte(42),
|
||||
}),
|
||||
&ExecutionPayload::Capella(_) | &ExecutionPayload::Eip4844(_) => {
|
||||
PayloadAttributes::V2(PayloadAttributesV2 {
|
||||
timestamp,
|
||||
prev_randao,
|
||||
suggested_fee_recipient: Address::repeat_byte(42),
|
||||
// FIXME: think about adding withdrawals here..
|
||||
withdrawals: vec![],
|
||||
})
|
||||
}
|
||||
},
|
||||
// I guess a PoW blocks means we should use Merge?
|
||||
&Block::PoW(_) => PayloadAttributes::V1(PayloadAttributesV1 {
|
||||
timestamp,
|
||||
prev_randao,
|
||||
suggested_fee_recipient: Address::repeat_byte(42),
|
||||
}),
|
||||
};
|
||||
|
||||
// Insert a proposer to ensure the fork choice updated command works.
|
||||
let slot = Slot::new(0);
|
||||
let validator_index = 0;
|
||||
self.el
|
||||
.insert_proposer(
|
||||
slot,
|
||||
head_block_root,
|
||||
validator_index,
|
||||
PayloadAttributes {
|
||||
timestamp,
|
||||
prev_randao,
|
||||
suggested_fee_recipient: Address::repeat_byte(42),
|
||||
},
|
||||
)
|
||||
.insert_proposer(slot, head_block_root, validator_index, payload_attributes)
|
||||
.await;
|
||||
|
||||
self.el
|
||||
@ -132,7 +149,7 @@ impl<T: EthSpec> MockExecutionLayer<T> {
|
||||
slot,
|
||||
chain_health: ChainHealth::Healthy,
|
||||
};
|
||||
let payload = self
|
||||
let payload: ExecutionPayload<T> = self
|
||||
.el
|
||||
.get_payload::<FullPayload<T>>(
|
||||
parent_hash,
|
||||
@ -145,12 +162,14 @@ impl<T: EthSpec> MockExecutionLayer<T> {
|
||||
)
|
||||
.await
|
||||
.unwrap()
|
||||
.execution_payload;
|
||||
let block_hash = payload.block_hash;
|
||||
assert_eq!(payload.parent_hash, parent_hash);
|
||||
assert_eq!(payload.block_number, block_number);
|
||||
assert_eq!(payload.timestamp, timestamp);
|
||||
assert_eq!(payload.prev_randao, prev_randao);
|
||||
.to_payload()
|
||||
.into();
|
||||
|
||||
let block_hash = payload.block_hash();
|
||||
assert_eq!(payload.parent_hash(), parent_hash);
|
||||
assert_eq!(payload.block_number(), block_number);
|
||||
assert_eq!(payload.timestamp(), timestamp);
|
||||
assert_eq!(payload.prev_randao(), prev_randao);
|
||||
|
||||
// Ensure the payload cache is empty.
|
||||
assert!(self
|
||||
@ -175,12 +194,13 @@ impl<T: EthSpec> MockExecutionLayer<T> {
|
||||
)
|
||||
.await
|
||||
.unwrap()
|
||||
.execution_payload_header;
|
||||
assert_eq!(payload_header.block_hash, block_hash);
|
||||
assert_eq!(payload_header.parent_hash, parent_hash);
|
||||
assert_eq!(payload_header.block_number, block_number);
|
||||
assert_eq!(payload_header.timestamp, timestamp);
|
||||
assert_eq!(payload_header.prev_randao, prev_randao);
|
||||
.to_payload();
|
||||
|
||||
assert_eq!(payload_header.block_hash(), block_hash);
|
||||
assert_eq!(payload_header.parent_hash(), parent_hash);
|
||||
assert_eq!(payload_header.block_number(), block_number);
|
||||
assert_eq!(payload_header.timestamp(), timestamp);
|
||||
assert_eq!(payload_header.prev_randao(), prev_randao);
|
||||
|
||||
// Ensure the payload cache has the correct payload.
|
||||
assert_eq!(
|
||||
|
@ -9,8 +9,8 @@ use std::sync::Arc;
|
||||
use tokio::sync::mpsc::UnboundedSender;
|
||||
use tree_hash::TreeHash;
|
||||
use types::{
|
||||
BlindedPayload, ExecPayload, ExecutionBlockHash, ExecutionPayload, FullPayload, Hash256,
|
||||
SignedBeaconBlock,
|
||||
AbstractExecPayload, BlindedPayload, EthSpec, ExecPayload, ExecutionBlockHash, FullPayload,
|
||||
Hash256, SignedBeaconBlock,
|
||||
};
|
||||
use warp::Rejection;
|
||||
|
||||
@ -158,12 +158,17 @@ async fn reconstruct_block<T: BeaconChainTypes>(
|
||||
|
||||
// If the execution block hash is zero, use an empty payload.
|
||||
let full_payload = if payload_header.block_hash() == ExecutionBlockHash::zero() {
|
||||
ExecutionPayload::default()
|
||||
FullPayload::default_at_fork(
|
||||
chain
|
||||
.spec
|
||||
.fork_name_at_epoch(block.slot().epoch(T::EthSpec::slots_per_epoch())),
|
||||
)
|
||||
.into()
|
||||
// If we already have an execution payload with this transactions root cached, use it.
|
||||
} else if let Some(cached_payload) =
|
||||
el.get_payload_by_root(&payload_header.tree_hash_root())
|
||||
{
|
||||
info!(log, "Reconstructing a full block using a local payload"; "block_hash" => ?cached_payload.block_hash);
|
||||
info!(log, "Reconstructing a full block using a local payload"; "block_hash" => ?cached_payload.block_hash());
|
||||
cached_payload
|
||||
// Otherwise, this means we are attempting a blind block proposal.
|
||||
} else {
|
||||
@ -176,7 +181,7 @@ async fn reconstruct_block<T: BeaconChainTypes>(
|
||||
e
|
||||
))
|
||||
})?;
|
||||
info!(log, "Successfully published a block to the builder network"; "block_hash" => ?full_payload.block_hash);
|
||||
info!(log, "Successfully published a block to the builder network"; "block_hash" => ?full_payload.block_hash());
|
||||
full_payload
|
||||
};
|
||||
|
||||
|
@ -296,7 +296,7 @@ pub fn gossipsub_config(network_load: u8, fork_context: Arc<ForkContext>) -> Gos
|
||||
match fork_context.current_fork() {
|
||||
// according to: https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/p2p-interface.md#the-gossip-domain-gossipsub
|
||||
// the derivation of the message-id remains the same in the merge and for eip 4844.
|
||||
ForkName::Altair | ForkName::Merge | ForkName::Eip4844 => {
|
||||
ForkName::Altair | ForkName::Merge | ForkName::Capella | ForkName::Eip4844 => {
|
||||
let topic_len_bytes = topic_bytes.len().to_le_bytes();
|
||||
let mut vec = Vec::with_capacity(
|
||||
prefix.len() + topic_len_bytes.len() + topic_bytes.len() + message.data.len(),
|
||||
|
@ -193,16 +193,19 @@ mod tests {
|
||||
let mut chain_spec = Spec::default_spec();
|
||||
let altair_fork_epoch = Epoch::new(1);
|
||||
let merge_fork_epoch = Epoch::new(2);
|
||||
let eip4844_fork_epoch = Epoch::new(3);
|
||||
let capella_fork_epoch = Epoch::new(3);
|
||||
let eip4844_fork_epoch = Epoch::new(4);
|
||||
|
||||
chain_spec.altair_fork_epoch = Some(altair_fork_epoch);
|
||||
chain_spec.bellatrix_fork_epoch = Some(merge_fork_epoch);
|
||||
chain_spec.capella_fork_epoch = Some(capella_fork_epoch);
|
||||
chain_spec.eip4844_fork_epoch = Some(eip4844_fork_epoch);
|
||||
|
||||
let current_slot = match fork_name {
|
||||
ForkName::Base => Slot::new(0),
|
||||
ForkName::Altair => altair_fork_epoch.start_slot(Spec::slots_per_epoch()),
|
||||
ForkName::Merge => merge_fork_epoch.start_slot(Spec::slots_per_epoch()),
|
||||
ForkName::Capella => capella_fork_epoch.start_slot(Spec::slots_per_epoch()),
|
||||
ForkName::Eip4844 => eip4844_fork_epoch.start_slot(Spec::slots_per_epoch()),
|
||||
};
|
||||
ForkContext::new::<Spec>(current_slot, Hash256::zero(), &chain_spec)
|
||||
|
@ -17,7 +17,8 @@ use std::sync::Arc;
|
||||
use tokio_util::codec::{Decoder, Encoder};
|
||||
use types::{
|
||||
BlobsSidecar, EthSpec, ForkContext, ForkName, SignedBeaconBlock, SignedBeaconBlockAltair,
|
||||
SignedBeaconBlockBase, SignedBeaconBlockEip4844, SignedBeaconBlockMerge,
|
||||
SignedBeaconBlockBase, SignedBeaconBlockCapella, SignedBeaconBlockEip4844,
|
||||
SignedBeaconBlockMerge,
|
||||
};
|
||||
use unsigned_varint::codec::Uvi;
|
||||
|
||||
@ -413,6 +414,10 @@ fn context_bytes<T: EthSpec>(
|
||||
// Eip4844 context being `None` implies that "merge never happened".
|
||||
fork_context.to_context_bytes(ForkName::Eip4844)
|
||||
}
|
||||
SignedBeaconBlock::Capella { .. } => {
|
||||
// Capella context being `None` implies that "merge never happened".
|
||||
fork_context.to_context_bytes(ForkName::Capella)
|
||||
}
|
||||
SignedBeaconBlock::Merge { .. } => {
|
||||
// Merge context being `None` implies that "merge never happened".
|
||||
fork_context.to_context_bytes(ForkName::Merge)
|
||||
@ -599,6 +604,11 @@ fn handle_v2_response<T: EthSpec>(
|
||||
decoded_buffer,
|
||||
)?),
|
||||
)))),
|
||||
ForkName::Capella => Ok(Some(RPCResponse::BlocksByRange(Arc::new(
|
||||
SignedBeaconBlock::Capella(SignedBeaconBlockCapella::from_ssz_bytes(
|
||||
decoded_buffer,
|
||||
)?),
|
||||
)))),
|
||||
ForkName::Eip4844 => Ok(Some(RPCResponse::BlocksByRange(Arc::new(
|
||||
SignedBeaconBlock::Eip4844(SignedBeaconBlockEip4844::from_ssz_bytes(
|
||||
decoded_buffer,
|
||||
@ -619,6 +629,11 @@ fn handle_v2_response<T: EthSpec>(
|
||||
decoded_buffer,
|
||||
)?),
|
||||
)))),
|
||||
ForkName::Capella => Ok(Some(RPCResponse::BlocksByRange(Arc::new(
|
||||
SignedBeaconBlock::Capella(SignedBeaconBlockCapella::from_ssz_bytes(
|
||||
decoded_buffer,
|
||||
)?),
|
||||
)))),
|
||||
ForkName::Eip4844 => Ok(Some(RPCResponse::BlocksByRange(Arc::new(
|
||||
SignedBeaconBlock::Eip4844(SignedBeaconBlockEip4844::from_ssz_bytes(
|
||||
decoded_buffer,
|
||||
@ -682,16 +697,19 @@ mod tests {
|
||||
let mut chain_spec = Spec::default_spec();
|
||||
let altair_fork_epoch = Epoch::new(1);
|
||||
let merge_fork_epoch = Epoch::new(2);
|
||||
let eip4844_fork_epoch = Epoch::new(3);
|
||||
let capella_fork_epoch = Epoch::new(3);
|
||||
let eip4844_fork_epoch = Epoch::new(4);
|
||||
|
||||
chain_spec.altair_fork_epoch = Some(altair_fork_epoch);
|
||||
chain_spec.bellatrix_fork_epoch = Some(merge_fork_epoch);
|
||||
chain_spec.capella_fork_epoch = Some(capella_fork_epoch);
|
||||
chain_spec.eip4844_fork_epoch = Some(eip4844_fork_epoch);
|
||||
|
||||
let current_slot = match fork_name {
|
||||
ForkName::Base => Slot::new(0),
|
||||
ForkName::Altair => altair_fork_epoch.start_slot(Spec::slots_per_epoch()),
|
||||
ForkName::Merge => merge_fork_epoch.start_slot(Spec::slots_per_epoch()),
|
||||
ForkName::Capella => capella_fork_epoch.start_slot(Spec::slots_per_epoch()),
|
||||
ForkName::Eip4844 => eip4844_fork_epoch.start_slot(Spec::slots_per_epoch()),
|
||||
};
|
||||
ForkContext::new::<Spec>(current_slot, Hash256::zero(), &chain_spec)
|
||||
|
@ -70,10 +70,18 @@ lazy_static! {
|
||||
pub static ref SIGNED_BEACON_BLOCK_MERGE_MAX: usize =
|
||||
// Size of a full altair block
|
||||
*SIGNED_BEACON_BLOCK_ALTAIR_MAX
|
||||
+ types::ExecutionPayload::<MainnetEthSpec>::max_execution_payload_size() // adding max size of execution payload (~16gb)
|
||||
+ types::ExecutionPayload::<MainnetEthSpec>::max_execution_payload_merge_size() // adding max size of execution payload (~16gb)
|
||||
+ ssz::BYTES_PER_LENGTH_OFFSET; // Adding the additional ssz offset for the `ExecutionPayload` field
|
||||
|
||||
pub static ref SIGNED_BEACON_BLOCK_EIP4844_MAX: usize = *SIGNED_BEACON_BLOCK_MERGE_MAX + (48 * <MainnetEthSpec>::max_blobs_per_block());
|
||||
pub static ref SIGNED_BEACON_BLOCK_CAPELLA_MAX: usize = *SIGNED_BEACON_BLOCK_ALTAIR_MAX
|
||||
+ types::ExecutionPayload::<MainnetEthSpec>::max_execution_payload_capella_size() // adding max size of execution payload (~16gb)
|
||||
+ ssz::BYTES_PER_LENGTH_OFFSET; // Adding the additional ssz offset for the `ExecutionPayload` field
|
||||
|
||||
pub static ref SIGNED_BEACON_BLOCK_EIP4844_MAX: usize = *SIGNED_BEACON_BLOCK_ALTAIR_MAX
|
||||
+ types::ExecutionPayload::<MainnetEthSpec>::max_execution_payload_eip4844_size() // adding max size of execution payload (~16gb)
|
||||
+ ssz::BYTES_PER_LENGTH_OFFSET // Adding the additional offsets for the `ExecutionPayload`
|
||||
+ (<types::KzgCommitment as Encode>::ssz_fixed_len() * <MainnetEthSpec>::max_blobs_per_block())
|
||||
+ ssz::BYTES_PER_LENGTH_OFFSET; // Length offset for the blob commitments field.
|
||||
|
||||
pub static ref BLOCKS_BY_ROOT_REQUEST_MIN: usize =
|
||||
VariableList::<Hash256, MaxRequestBlocks>::from(Vec::<Hash256>::new())
|
||||
@ -114,6 +122,8 @@ lazy_static! {
|
||||
pub(crate) const MAX_RPC_SIZE: usize = 1_048_576; // 1M
|
||||
/// The maximum bytes that can be sent across the RPC post-merge.
|
||||
pub(crate) const MAX_RPC_SIZE_POST_MERGE: usize = 10 * 1_048_576; // 10M
|
||||
//FIXME(sean) should these be the same?
|
||||
pub(crate) const MAX_RPC_SIZE_POST_CAPELLA: usize = 10 * 1_048_576; // 10M
|
||||
pub(crate) const MAX_RPC_SIZE_POST_EIP4844: usize = 10 * 1_048_576; // 10M
|
||||
/// The protocol prefix the RPC protocol id.
|
||||
const PROTOCOL_PREFIX: &str = "/eth2/beacon_chain/req";
|
||||
@ -126,9 +136,10 @@ const REQUEST_TIMEOUT: u64 = 15;
|
||||
/// Returns the maximum bytes that can be sent across the RPC.
|
||||
pub fn max_rpc_size(fork_context: &ForkContext) -> usize {
|
||||
match fork_context.current_fork() {
|
||||
ForkName::Eip4844 => MAX_RPC_SIZE_POST_EIP4844,
|
||||
ForkName::Merge => MAX_RPC_SIZE_POST_MERGE,
|
||||
ForkName::Altair | ForkName::Base => MAX_RPC_SIZE,
|
||||
ForkName::Merge => MAX_RPC_SIZE_POST_MERGE,
|
||||
ForkName::Capella => MAX_RPC_SIZE_POST_CAPELLA,
|
||||
ForkName::Eip4844 => MAX_RPC_SIZE_POST_EIP4844,
|
||||
}
|
||||
}
|
||||
|
||||
@ -149,9 +160,13 @@ pub fn rpc_block_limits_by_fork(current_fork: ForkName) -> RpcLimits {
|
||||
*SIGNED_BEACON_BLOCK_BASE_MIN, // Base block is smaller than altair and merge blocks
|
||||
*SIGNED_BEACON_BLOCK_MERGE_MAX, // Merge block is larger than base and altair blocks
|
||||
),
|
||||
ForkName::Capella => RpcLimits::new(
|
||||
*SIGNED_BEACON_BLOCK_BASE_MIN, // Base block is smaller than altair and merge blocks
|
||||
*SIGNED_BEACON_BLOCK_CAPELLA_MAX, // Capella block is larger than base, altair and merge blocks
|
||||
),
|
||||
ForkName::Eip4844 => RpcLimits::new(
|
||||
*SIGNED_BEACON_BLOCK_BASE_MIN, // Base block is smaller than altair and merge blocks
|
||||
*SIGNED_BEACON_BLOCK_EIP4844_MAX, // EIP 4844 block is larger than base, altair and merge blocks
|
||||
*SIGNED_BEACON_BLOCK_EIP4844_MAX, // EIP 4844 block is larger than all prior fork blocks
|
||||
),
|
||||
}
|
||||
}
|
||||
|
@ -12,8 +12,8 @@ use types::signed_blobs_sidecar::SignedBlobsSidecar;
|
||||
use types::{
|
||||
Attestation, AttesterSlashing, EthSpec, ForkContext, ForkName, ProposerSlashing,
|
||||
SignedAggregateAndProof, SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase,
|
||||
SignedBeaconBlockEip4844, SignedBeaconBlockMerge, SignedContributionAndProof,
|
||||
SignedVoluntaryExit, SubnetId, SyncCommitteeMessage, SyncSubnetId,
|
||||
SignedBeaconBlockCapella, SignedBeaconBlockEip4844, SignedBeaconBlockMerge,
|
||||
SignedContributionAndProof, SignedVoluntaryExit, SubnetId, SyncCommitteeMessage, SyncSubnetId,
|
||||
};
|
||||
|
||||
#[derive(Debug, Clone, PartialEq)]
|
||||
@ -175,6 +175,10 @@ impl<T: EthSpec> PubsubMessage<T> {
|
||||
SignedBeaconBlockEip4844::from_ssz_bytes(data)
|
||||
.map_err(|e| format!("{:?}", e))?,
|
||||
),
|
||||
Some(ForkName::Capella) => SignedBeaconBlock::<T>::Capella(
|
||||
SignedBeaconBlockCapella::from_ssz_bytes(data)
|
||||
.map_err(|e| format!("{:?}", e))?,
|
||||
),
|
||||
None => {
|
||||
return Err(format!(
|
||||
"Unknown gossipsub fork digest: {:?}",
|
||||
|
@ -32,16 +32,19 @@ pub fn fork_context(fork_name: ForkName) -> ForkContext {
|
||||
let mut chain_spec = E::default_spec();
|
||||
let altair_fork_epoch = Epoch::new(1);
|
||||
let merge_fork_epoch = Epoch::new(2);
|
||||
let eip4844_fork_epoch = Epoch::new(3);
|
||||
let capella_fork_epoch = Epoch::new(3);
|
||||
let eip4844_fork_epoch = Epoch::new(4);
|
||||
|
||||
chain_spec.altair_fork_epoch = Some(altair_fork_epoch);
|
||||
chain_spec.bellatrix_fork_epoch = Some(merge_fork_epoch);
|
||||
chain_spec.capella_fork_epoch = Some(capella_fork_epoch);
|
||||
chain_spec.eip4844_fork_epoch = Some(eip4844_fork_epoch);
|
||||
|
||||
let current_slot = match fork_name {
|
||||
ForkName::Base => Slot::new(0),
|
||||
ForkName::Altair => altair_fork_epoch.start_slot(E::slots_per_epoch()),
|
||||
ForkName::Merge => merge_fork_epoch.start_slot(E::slots_per_epoch()),
|
||||
ForkName::Capella => capella_fork_epoch.start_slot(E::slots_per_epoch()),
|
||||
ForkName::Eip4844 => eip4844_fork_epoch.start_slot(E::slots_per_epoch()),
|
||||
};
|
||||
ForkContext::new::<E>(current_slot, Hash256::zero(), &chain_spec)
|
||||
|
@ -706,6 +706,7 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
seen_timestamp: Duration,
|
||||
) {
|
||||
match self.chain.verify_blobs_sidecar_for_gossip(&blob) {
|
||||
//FIXME(sean)
|
||||
Ok(verified_sidecar) => {
|
||||
// Register with validator monitor
|
||||
// Propagate
|
||||
@ -2309,6 +2310,7 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
|
||||
return;
|
||||
}
|
||||
&BlobError::UnknownValidator(_) => todo!(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -586,7 +586,7 @@ impl BeaconNodeHttpClient {
|
||||
/// `POST beacon/blocks`
|
||||
///
|
||||
/// Returns `Ok(None)` on a 404 error.
|
||||
pub async fn post_beacon_blocks<T: EthSpec, Payload: ExecPayload<T>>(
|
||||
pub async fn post_beacon_blocks<T: EthSpec, Payload: AbstractExecPayload<T>>(
|
||||
&self,
|
||||
block: &SignedBeaconBlock<T, Payload>,
|
||||
) -> Result<(), Error> {
|
||||
@ -627,7 +627,7 @@ impl BeaconNodeHttpClient {
|
||||
/// `POST beacon/blinded_blocks`
|
||||
///
|
||||
/// Returns `Ok(None)` on a 404 error.
|
||||
pub async fn post_beacon_blinded_blocks<T: EthSpec, Payload: ExecPayload<T>>(
|
||||
pub async fn post_beacon_blinded_blocks<T: EthSpec, Payload: AbstractExecPayload<T>>(
|
||||
&self,
|
||||
block: &SignedBeaconBlock<T, Payload>,
|
||||
) -> Result<(), Error> {
|
||||
@ -1248,7 +1248,7 @@ impl BeaconNodeHttpClient {
|
||||
}
|
||||
|
||||
/// `GET v2/validator/blocks/{slot}`
|
||||
pub async fn get_validator_blocks<T: EthSpec, Payload: ExecPayload<T>>(
|
||||
pub async fn get_validator_blocks<T: EthSpec, Payload: AbstractExecPayload<T>>(
|
||||
&self,
|
||||
slot: Slot,
|
||||
randao_reveal: &SignatureBytes,
|
||||
@ -1259,7 +1259,7 @@ impl BeaconNodeHttpClient {
|
||||
}
|
||||
|
||||
/// `GET v2/validator/blocks/{slot}`
|
||||
pub async fn get_validator_blocks_modular<T: EthSpec, Payload: ExecPayload<T>>(
|
||||
pub async fn get_validator_blocks_modular<T: EthSpec, Payload: AbstractExecPayload<T>>(
|
||||
&self,
|
||||
slot: Slot,
|
||||
randao_reveal: &SignatureBytes,
|
||||
@ -1291,7 +1291,7 @@ impl BeaconNodeHttpClient {
|
||||
}
|
||||
|
||||
/// `GET v1/validator/blocks_and_blobs/{slot}`
|
||||
pub async fn get_validator_blocks_and_blobs<T: EthSpec, Payload: ExecPayload<T>>(
|
||||
pub async fn get_validator_blocks_and_blobs<T: EthSpec, Payload: AbstractExecPayload<T>>(
|
||||
&self,
|
||||
slot: Slot,
|
||||
randao_reveal: &SignatureBytes,
|
||||
@ -1317,7 +1317,7 @@ impl BeaconNodeHttpClient {
|
||||
}
|
||||
|
||||
/// `GET v2/validator/blinded_blocks/{slot}`
|
||||
pub async fn get_validator_blinded_blocks<T: EthSpec, Payload: ExecPayload<T>>(
|
||||
pub async fn get_validator_blinded_blocks<T: EthSpec, Payload: AbstractExecPayload<T>>(
|
||||
&self,
|
||||
slot: Slot,
|
||||
randao_reveal: &SignatureBytes,
|
||||
@ -1333,7 +1333,10 @@ impl BeaconNodeHttpClient {
|
||||
}
|
||||
|
||||
/// `GET v1/validator/blinded_blocks/{slot}`
|
||||
pub async fn get_validator_blinded_blocks_modular<T: EthSpec, Payload: ExecPayload<T>>(
|
||||
pub async fn get_validator_blinded_blocks_modular<
|
||||
T: EthSpec,
|
||||
Payload: AbstractExecPayload<T>,
|
||||
>(
|
||||
&self,
|
||||
slot: Slot,
|
||||
randao_reveal: &SignatureBytes,
|
||||
|
@ -1111,8 +1111,8 @@ pub struct LivenessResponseData {
|
||||
}
|
||||
|
||||
#[derive(PartialEq, Debug, Serialize, Deserialize)]
|
||||
#[serde(bound = "T: EthSpec, Payload: ExecPayload<T>")]
|
||||
pub struct BlocksAndBlobs<T: EthSpec, Payload: ExecPayload<T>> {
|
||||
#[serde(bound = "T: EthSpec, Payload: AbstractExecPayload<T>")]
|
||||
pub struct BlocksAndBlobs<T: EthSpec, Payload: AbstractExecPayload<T>> {
|
||||
pub block: BeaconBlock<T, Payload>,
|
||||
pub blobs: Vec<Blob<T>>,
|
||||
pub kzg_aggregate_proof: KzgProof,
|
||||
|
@ -326,9 +326,9 @@ pub fn partially_verify_execution_payload<'payload, T: EthSpec, Payload: Abstrac
|
||||
) -> Result<(), BlockProcessingError> {
|
||||
if is_merge_transition_complete(state) {
|
||||
block_verify!(
|
||||
payload.parent_hash() == *state.latest_execution_payload_header()?.block_hash(),
|
||||
payload.parent_hash() == state.latest_execution_payload_header()?.block_hash(),
|
||||
BlockProcessingError::ExecutionHashChainIncontiguous {
|
||||
expected: *state.latest_execution_payload_header()?.block_hash(),
|
||||
expected: state.latest_execution_payload_header()?.block_hash(),
|
||||
found: payload.parent_hash(),
|
||||
}
|
||||
);
|
||||
|
@ -3,7 +3,7 @@ use types::beacon_state::BeaconState;
|
||||
use types::eth_spec::EthSpec;
|
||||
|
||||
pub fn process_full_withdrawals<T: EthSpec>(
|
||||
state: &mut BeaconState<T>,
|
||||
_state: &mut BeaconState<T>,
|
||||
) -> Result<(), EpochProcessingError> {
|
||||
todo!("implement this");
|
||||
Ok(())
|
||||
|
@ -3,7 +3,7 @@ use types::beacon_state::BeaconState;
|
||||
use types::eth_spec::EthSpec;
|
||||
|
||||
pub fn process_partial_withdrawals<T: EthSpec>(
|
||||
state: &mut BeaconState<T>,
|
||||
_state: &mut BeaconState<T>,
|
||||
) -> Result<(), EpochProcessingError> {
|
||||
todo!("implement this");
|
||||
Ok(())
|
||||
|
@ -253,11 +253,14 @@ impl ChainSpec {
|
||||
pub fn fork_name_at_epoch(&self, epoch: Epoch) -> ForkName {
|
||||
match self.eip4844_fork_epoch {
|
||||
Some(fork_epoch) if epoch >= fork_epoch => ForkName::Eip4844,
|
||||
_ => match self.bellatrix_fork_epoch {
|
||||
Some(fork_epoch) if epoch >= fork_epoch => ForkName::Merge,
|
||||
_ => match self.altair_fork_epoch {
|
||||
Some(fork_epoch) if epoch >= fork_epoch => ForkName::Altair,
|
||||
_ => ForkName::Base,
|
||||
_ => match self.capella_fork_epoch {
|
||||
Some(fork_epoch) if epoch >= fork_epoch => ForkName::Capella,
|
||||
_ => match self.bellatrix_fork_epoch {
|
||||
Some(fork_epoch) if epoch >= fork_epoch => ForkName::Merge,
|
||||
_ => match self.altair_fork_epoch {
|
||||
Some(fork_epoch) if epoch >= fork_epoch => ForkName::Altair,
|
||||
_ => ForkName::Base,
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
@ -43,28 +43,40 @@ pub type Transactions<T> = VariableList<
|
||||
#[tree_hash(enum_behaviour = "transparent")]
|
||||
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
|
||||
pub struct ExecutionPayload<T: EthSpec> {
|
||||
#[superstruct(getter(copy))]
|
||||
pub parent_hash: ExecutionBlockHash,
|
||||
#[superstruct(getter(copy))]
|
||||
pub fee_recipient: Address,
|
||||
#[superstruct(getter(copy))]
|
||||
pub state_root: Hash256,
|
||||
#[superstruct(getter(copy))]
|
||||
pub receipts_root: Hash256,
|
||||
#[serde(with = "ssz_types::serde_utils::hex_fixed_vec")]
|
||||
pub logs_bloom: FixedVector<u8, T::BytesPerLogsBloom>,
|
||||
#[superstruct(getter(copy))]
|
||||
pub prev_randao: Hash256,
|
||||
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
||||
#[superstruct(getter(copy))]
|
||||
pub block_number: u64,
|
||||
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
||||
#[superstruct(getter(copy))]
|
||||
pub gas_limit: u64,
|
||||
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
||||
#[superstruct(getter(copy))]
|
||||
pub gas_used: u64,
|
||||
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
||||
#[superstruct(getter(copy))]
|
||||
pub timestamp: u64,
|
||||
#[serde(with = "ssz_types::serde_utils::hex_var_list")]
|
||||
pub extra_data: VariableList<u8, T::MaxExtraDataBytes>,
|
||||
#[serde(with = "eth2_serde_utils::quoted_u256")]
|
||||
#[superstruct(getter(copy))]
|
||||
pub base_fee_per_gas: Uint256,
|
||||
#[superstruct(only(Eip4844))]
|
||||
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
||||
#[superstruct(getter(copy))]
|
||||
pub excess_blobs: u64,
|
||||
#[superstruct(getter(copy))]
|
||||
pub block_hash: ExecutionBlockHash,
|
||||
#[serde(with = "ssz_types::serde_utils::list_of_hex_var_list")]
|
||||
pub transactions: Transactions<T>,
|
||||
@ -94,8 +106,7 @@ impl<T: EthSpec> ExecutionPayload<T> {
|
||||
// Max size of variable length `transactions` field
|
||||
+ (T::max_transactions_per_payload() * (ssz::BYTES_PER_LENGTH_OFFSET + T::max_bytes_per_transaction()))
|
||||
// Max size of variable length `withdrawals` field
|
||||
// TODO: check this
|
||||
+ (T::max_withdrawals_per_payload() * (ssz::BYTES_PER_LENGTH_OFFSET + <Withdrawal as Encode>::ssz_fixed_len()))
|
||||
+ (T::max_withdrawals_per_payload() * <Withdrawal as Encode>::ssz_fixed_len())
|
||||
}
|
||||
|
||||
#[allow(clippy::integer_arithmetic)]
|
||||
@ -108,8 +119,7 @@ impl<T: EthSpec> ExecutionPayload<T> {
|
||||
// Max size of variable length `transactions` field
|
||||
+ (T::max_transactions_per_payload() * (ssz::BYTES_PER_LENGTH_OFFSET + T::max_bytes_per_transaction()))
|
||||
// Max size of variable length `withdrawals` field
|
||||
// TODO: check this
|
||||
+ (T::max_withdrawals_per_payload() * (ssz::BYTES_PER_LENGTH_OFFSET + <Withdrawal as Encode>::ssz_fixed_len()))
|
||||
+ (T::max_withdrawals_per_payload() * <Withdrawal as Encode>::ssz_fixed_len())
|
||||
}
|
||||
|
||||
pub fn blob_txns_iter(&self) -> Iter<'_, Transaction<T::MaxBytesPerTransaction>> {
|
||||
|
@ -37,31 +37,45 @@ use BeaconStateError;
|
||||
#[ssz(enum_behaviour = "transparent")]
|
||||
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
|
||||
pub struct ExecutionPayloadHeader<T: EthSpec> {
|
||||
#[superstruct(getter(copy))]
|
||||
pub parent_hash: ExecutionBlockHash,
|
||||
#[superstruct(getter(copy))]
|
||||
pub fee_recipient: Address,
|
||||
#[superstruct(getter(copy))]
|
||||
pub state_root: Hash256,
|
||||
#[superstruct(getter(copy))]
|
||||
pub receipts_root: Hash256,
|
||||
#[serde(with = "ssz_types::serde_utils::hex_fixed_vec")]
|
||||
pub logs_bloom: FixedVector<u8, T::BytesPerLogsBloom>,
|
||||
#[superstruct(getter(copy))]
|
||||
pub prev_randao: Hash256,
|
||||
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
||||
#[superstruct(getter(copy))]
|
||||
pub block_number: u64,
|
||||
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
||||
#[superstruct(getter(copy))]
|
||||
pub gas_limit: u64,
|
||||
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
||||
#[superstruct(getter(copy))]
|
||||
pub gas_used: u64,
|
||||
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
||||
#[superstruct(getter(copy))]
|
||||
pub timestamp: u64,
|
||||
#[serde(with = "ssz_types::serde_utils::hex_var_list")]
|
||||
pub extra_data: VariableList<u8, T::MaxExtraDataBytes>,
|
||||
#[serde(with = "eth2_serde_utils::quoted_u256")]
|
||||
#[superstruct(getter(copy))]
|
||||
pub base_fee_per_gas: Uint256,
|
||||
#[superstruct(only(Eip4844))]
|
||||
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
||||
#[superstruct(getter(copy))]
|
||||
pub excess_blobs: u64,
|
||||
#[superstruct(getter(copy))]
|
||||
pub block_hash: ExecutionBlockHash,
|
||||
#[superstruct(getter(copy))]
|
||||
pub transactions_root: Hash256,
|
||||
#[superstruct(only(Capella, Eip4844))]
|
||||
#[superstruct(getter(copy))]
|
||||
pub withdrawals_root: Hash256,
|
||||
}
|
||||
|
||||
|
@ -47,6 +47,13 @@ impl ForkContext {
|
||||
));
|
||||
}
|
||||
|
||||
if spec.capella_fork_epoch.is_some() {
|
||||
fork_to_digest.push((
|
||||
ForkName::Capella,
|
||||
ChainSpec::compute_fork_digest(spec.capella_fork_version, genesis_validators_root),
|
||||
));
|
||||
}
|
||||
|
||||
if spec.eip4844_fork_epoch.is_some() {
|
||||
fork_to_digest.push((
|
||||
ForkName::Eip4844,
|
||||
|
@ -1,5 +1,4 @@
|
||||
//! Ethereum 2.0 types
|
||||
#![feature(generic_associated_types)]
|
||||
// Required for big type-level numbers
|
||||
#![recursion_limit = "128"]
|
||||
// Clippy lint set up
|
||||
@ -153,7 +152,7 @@ pub use crate::participation_list::ParticipationList;
|
||||
pub use crate::payload::{
|
||||
AbstractExecPayload, BlindedPayload, BlindedPayloadCapella, BlindedPayloadEip4844,
|
||||
BlindedPayloadMerge, BlindedPayloadRef, BlockType, ExecPayload, FullPayload,
|
||||
FullPayloadCapella, FullPayloadEip4844, FullPayloadMerge, FullPayloadRef,
|
||||
FullPayloadCapella, FullPayloadEip4844, FullPayloadMerge, FullPayloadRef, OwnedExecPayload,
|
||||
};
|
||||
pub use crate::pending_attestation::PendingAttestation;
|
||||
pub use crate::preset::{AltairPreset, BasePreset, BellatrixPreset};
|
||||
@ -164,8 +163,9 @@ pub use crate::selection_proof::SelectionProof;
|
||||
pub use crate::shuffling_id::AttestationShufflingId;
|
||||
pub use crate::signed_aggregate_and_proof::SignedAggregateAndProof;
|
||||
pub use crate::signed_beacon_block::{
|
||||
SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase, SignedBeaconBlockEip4844,
|
||||
SignedBeaconBlockHash, SignedBeaconBlockMerge, SignedBlindedBeaconBlock,
|
||||
SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase, SignedBeaconBlockCapella,
|
||||
SignedBeaconBlockEip4844, SignedBeaconBlockHash, SignedBeaconBlockMerge,
|
||||
SignedBlindedBeaconBlock,
|
||||
};
|
||||
pub use crate::signed_beacon_block_header::SignedBeaconBlockHeader;
|
||||
pub use crate::signed_blobs_sidecar::SignedBlobsSidecar;
|
||||
|
@ -57,7 +57,13 @@ impl<T: EthSpec, P> OwnedExecPayload<T> for P where
|
||||
}
|
||||
|
||||
pub trait AbstractExecPayload<T: EthSpec>:
|
||||
ExecPayload<T> + Sized + From<ExecutionPayload<T>> + TryFrom<ExecutionPayloadHeader<T>>
|
||||
ExecPayload<T>
|
||||
+ Sized
|
||||
+ From<ExecutionPayload<T>>
|
||||
+ TryFrom<ExecutionPayloadHeader<T>>
|
||||
+ TryInto<Self::Merge>
|
||||
+ TryInto<Self::Capella>
|
||||
+ TryInto<Self::Eip4844>
|
||||
{
|
||||
type Ref<'a>: ExecPayload<T>
|
||||
+ Copy
|
||||
@ -77,6 +83,8 @@ pub trait AbstractExecPayload<T: EthSpec>:
|
||||
+ Into<Self>
|
||||
+ From<ExecutionPayloadEip4844<T>>
|
||||
+ TryFrom<ExecutionPayloadHeaderEip4844<T>>;
|
||||
|
||||
fn default_at_fork(fork_name: ForkName) -> Self;
|
||||
}
|
||||
|
||||
#[superstruct(
|
||||
@ -128,6 +136,22 @@ impl<T: EthSpec> From<FullPayload<T>> for ExecutionPayload<T> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a, T: EthSpec> From<FullPayloadRef<'a, T>> for ExecutionPayload<T> {
|
||||
fn from(full_payload: FullPayloadRef<'a, T>) -> Self {
|
||||
match full_payload {
|
||||
FullPayloadRef::Merge(payload) => {
|
||||
ExecutionPayload::Merge(payload.execution_payload.clone())
|
||||
}
|
||||
FullPayloadRef::Capella(payload) => {
|
||||
ExecutionPayload::Capella(payload.execution_payload.clone())
|
||||
}
|
||||
FullPayloadRef::Eip4844(payload) => {
|
||||
ExecutionPayload::Eip4844(payload.execution_payload.clone())
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: EthSpec> ExecPayload<T> for FullPayloadMerge<T> {
|
||||
fn block_type() -> BlockType {
|
||||
BlockType::Full
|
||||
@ -425,6 +449,51 @@ impl<T: EthSpec> AbstractExecPayload<T> for FullPayload<T> {
|
||||
type Merge = FullPayloadMerge<T>;
|
||||
type Capella = FullPayloadCapella<T>;
|
||||
type Eip4844 = FullPayloadEip4844<T>;
|
||||
|
||||
fn default_at_fork(fork_name: ForkName) -> Self {
|
||||
match fork_name {
|
||||
//FIXME(sean) error handling
|
||||
ForkName::Base | ForkName::Altair => panic!(),
|
||||
ForkName::Merge => FullPayloadMerge::default().into(),
|
||||
ForkName::Capella => FullPayloadCapella::default().into(),
|
||||
ForkName::Eip4844 => FullPayloadEip4844::default().into(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//FIXME(sean) fix errors
|
||||
impl<T: EthSpec> TryInto<FullPayloadMerge<T>> for FullPayload<T> {
|
||||
type Error = ();
|
||||
|
||||
fn try_into(self) -> Result<FullPayloadMerge<T>, Self::Error> {
|
||||
match self {
|
||||
FullPayload::Merge(payload) => Ok(payload),
|
||||
FullPayload::Capella(_) => Err(()),
|
||||
FullPayload::Eip4844(_) => Err(()),
|
||||
}
|
||||
}
|
||||
}
|
||||
impl<T: EthSpec> TryInto<FullPayloadCapella<T>> for FullPayload<T> {
|
||||
type Error = ();
|
||||
|
||||
fn try_into(self) -> Result<FullPayloadCapella<T>, Self::Error> {
|
||||
match self {
|
||||
FullPayload::Merge(_) => Err(()),
|
||||
FullPayload::Capella(payload) => Ok(payload),
|
||||
FullPayload::Eip4844(_) => Err(()),
|
||||
}
|
||||
}
|
||||
}
|
||||
impl<T: EthSpec> TryInto<FullPayloadEip4844<T>> for FullPayload<T> {
|
||||
type Error = ();
|
||||
|
||||
fn try_into(self) -> Result<FullPayloadEip4844<T>, Self::Error> {
|
||||
match self {
|
||||
FullPayload::Merge(_) => Err(()),
|
||||
FullPayload::Capella(_) => Err(()),
|
||||
FullPayload::Eip4844(payload) => Ok(payload),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: EthSpec> From<ExecutionPayload<T>> for FullPayload<T> {
|
||||
@ -855,6 +924,51 @@ impl<T: EthSpec> AbstractExecPayload<T> for BlindedPayload<T> {
|
||||
type Merge = BlindedPayloadMerge<T>;
|
||||
type Capella = BlindedPayloadCapella<T>;
|
||||
type Eip4844 = BlindedPayloadEip4844<T>;
|
||||
|
||||
fn default_at_fork(fork_name: ForkName) -> Self {
|
||||
match fork_name {
|
||||
//FIXME(sean) error handling
|
||||
ForkName::Base | ForkName::Altair => panic!(),
|
||||
ForkName::Merge => BlindedPayloadMerge::default().into(),
|
||||
ForkName::Capella => BlindedPayloadCapella::default().into(),
|
||||
ForkName::Eip4844 => BlindedPayloadEip4844::default().into(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//FIXME(sean) fix errors
|
||||
impl<T: EthSpec> TryInto<BlindedPayloadMerge<T>> for BlindedPayload<T> {
|
||||
type Error = ();
|
||||
|
||||
fn try_into(self) -> Result<BlindedPayloadMerge<T>, Self::Error> {
|
||||
match self {
|
||||
BlindedPayload::Merge(payload) => Ok(payload),
|
||||
BlindedPayload::Capella(_) => Err(()),
|
||||
BlindedPayload::Eip4844(_) => Err(()),
|
||||
}
|
||||
}
|
||||
}
|
||||
impl<T: EthSpec> TryInto<BlindedPayloadCapella<T>> for BlindedPayload<T> {
|
||||
type Error = ();
|
||||
|
||||
fn try_into(self) -> Result<BlindedPayloadCapella<T>, Self::Error> {
|
||||
match self {
|
||||
BlindedPayload::Merge(_) => Err(()),
|
||||
BlindedPayload::Capella(payload) => Ok(payload),
|
||||
BlindedPayload::Eip4844(_) => Err(()),
|
||||
}
|
||||
}
|
||||
}
|
||||
impl<T: EthSpec> TryInto<BlindedPayloadEip4844<T>> for BlindedPayload<T> {
|
||||
type Error = ();
|
||||
|
||||
fn try_into(self) -> Result<BlindedPayloadEip4844<T>, Self::Error> {
|
||||
match self {
|
||||
BlindedPayload::Merge(_) => Err(()),
|
||||
BlindedPayload::Capella(_) => Err(()),
|
||||
BlindedPayload::Eip4844(payload) => Ok(payload),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: EthSpec> Default for FullPayloadMerge<T> {
|
||||
|
@ -4,7 +4,7 @@ use ssz::Encode;
|
||||
use std::fs::File;
|
||||
use std::io::Write;
|
||||
use std::time::{SystemTime, UNIX_EPOCH};
|
||||
use types::{EthSpec, ExecutionPayloadHeader};
|
||||
use types::{EthSpec, ExecutionPayloadHeader, ExecutionPayloadHeaderMerge};
|
||||
|
||||
pub fn run<T: EthSpec>(matches: &ArgMatches) -> Result<(), String> {
|
||||
let eth1_block_hash = parse_required(matches, "execution-block-hash")?;
|
||||
@ -18,14 +18,16 @@ pub fn run<T: EthSpec>(matches: &ArgMatches) -> Result<(), String> {
|
||||
let gas_limit = parse_required(matches, "gas-limit")?;
|
||||
let file_name = matches.value_of("file").ok_or("No file supplied")?;
|
||||
|
||||
let execution_payload_header: ExecutionPayloadHeader<T> = ExecutionPayloadHeader {
|
||||
gas_limit,
|
||||
base_fee_per_gas,
|
||||
timestamp: genesis_time,
|
||||
block_hash: eth1_block_hash,
|
||||
prev_randao: eth1_block_hash.into_root(),
|
||||
..ExecutionPayloadHeader::default()
|
||||
};
|
||||
//FIXME(sean)
|
||||
let execution_payload_header: ExecutionPayloadHeader<T> =
|
||||
ExecutionPayloadHeader::Merge(ExecutionPayloadHeaderMerge {
|
||||
gas_limit,
|
||||
base_fee_per_gas,
|
||||
timestamp: genesis_time,
|
||||
block_hash: eth1_block_hash,
|
||||
prev_randao: eth1_block_hash.into_root(),
|
||||
..ExecutionPayloadHeaderMerge::default()
|
||||
});
|
||||
let mut file = File::create(file_name).map_err(|_| "Unable to create file".to_string())?;
|
||||
let bytes = execution_payload_header.as_ssz_bytes();
|
||||
file.write_all(bytes.as_slice())
|
||||
|
@ -10,6 +10,7 @@ use std::path::PathBuf;
|
||||
use std::time::{SystemTime, UNIX_EPOCH};
|
||||
use types::{
|
||||
test_utils::generate_deterministic_keypairs, Address, Config, EthSpec, ExecutionPayloadHeader,
|
||||
ExecutionPayloadHeaderMerge,
|
||||
};
|
||||
|
||||
pub fn run<T: EthSpec>(testnet_dir_path: PathBuf, matches: &ArgMatches) -> Result<(), String> {
|
||||
@ -75,7 +76,9 @@ pub fn run<T: EthSpec>(testnet_dir_path: PathBuf, matches: &ArgMatches) -> Resul
|
||||
.map_err(|e| format!("Unable to open {}: {}", filename, e))?;
|
||||
file.read_to_end(&mut bytes)
|
||||
.map_err(|e| format!("Unable to read {}: {}", filename, e))?;
|
||||
ExecutionPayloadHeader::<T>::from_ssz_bytes(bytes.as_slice())
|
||||
//FIXME(sean)
|
||||
ExecutionPayloadHeaderMerge::<T>::from_ssz_bytes(bytes.as_slice())
|
||||
.map(ExecutionPayloadHeader::Merge)
|
||||
.map_err(|e| format!("SSZ decode failed: {:?}", e))
|
||||
})
|
||||
.transpose()?;
|
||||
@ -84,9 +87,9 @@ pub fn run<T: EthSpec>(testnet_dir_path: PathBuf, matches: &ArgMatches) -> Resul
|
||||
execution_payload_header.as_ref()
|
||||
{
|
||||
let eth1_block_hash =
|
||||
parse_optional(matches, "eth1-block-hash")?.unwrap_or(payload.block_hash);
|
||||
parse_optional(matches, "eth1-block-hash")?.unwrap_or(payload.block_hash());
|
||||
let genesis_time =
|
||||
parse_optional(matches, "genesis-time")?.unwrap_or(payload.timestamp);
|
||||
parse_optional(matches, "genesis-time")?.unwrap_or(payload.timestamp());
|
||||
(eth1_block_hash, genesis_time)
|
||||
} else {
|
||||
let eth1_block_hash = parse_required(matches, "eth1-block-hash").map_err(|_| {
|
||||
|
@ -78,6 +78,7 @@ pub fn previous_fork(fork_name: ForkName) -> ForkName {
|
||||
ForkName::Base => ForkName::Base,
|
||||
ForkName::Altair => ForkName::Base,
|
||||
ForkName::Merge => ForkName::Altair, // TODO: Check this when tests are released..
|
||||
ForkName::Eip4844 => ForkName::Merge, // TODO: Check this when tests are released..
|
||||
ForkName::Capella => ForkName::Merge, // TODO: Check this when tests are released..
|
||||
ForkName::Eip4844 => ForkName::Capella, // TODO: Check this when tests are released..
|
||||
}
|
||||
}
|
||||
|
@ -97,7 +97,10 @@ impl<E: EthSpec> EpochTransition<E> for JustificationAndFinalization {
|
||||
justification_and_finalization_state.apply_changes_to_state(state);
|
||||
Ok(())
|
||||
}
|
||||
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Eip4844(_) => {
|
||||
BeaconState::Altair(_)
|
||||
| BeaconState::Merge(_)
|
||||
| BeaconState::Capella(_)
|
||||
| BeaconState::Eip4844(_) => {
|
||||
let justification_and_finalization_state =
|
||||
altair::process_justification_and_finalization(
|
||||
state,
|
||||
@ -118,13 +121,14 @@ impl<E: EthSpec> EpochTransition<E> for RewardsAndPenalties {
|
||||
validator_statuses.process_attestations(state)?;
|
||||
base::process_rewards_and_penalties(state, &mut validator_statuses, spec)
|
||||
}
|
||||
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Eip4844(_) => {
|
||||
altair::process_rewards_and_penalties(
|
||||
state,
|
||||
&altair::ParticipationCache::new(state, spec).unwrap(),
|
||||
spec,
|
||||
)
|
||||
}
|
||||
BeaconState::Altair(_)
|
||||
| BeaconState::Merge(_)
|
||||
| BeaconState::Capella(_)
|
||||
| BeaconState::Eip4844(_) => altair::process_rewards_and_penalties(
|
||||
state,
|
||||
&altair::ParticipationCache::new(state, spec).unwrap(),
|
||||
spec,
|
||||
),
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -147,7 +151,10 @@ impl<E: EthSpec> EpochTransition<E> for Slashings {
|
||||
spec,
|
||||
)?;
|
||||
}
|
||||
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Eip4844(_) => {
|
||||
BeaconState::Altair(_)
|
||||
| BeaconState::Merge(_)
|
||||
| BeaconState::Capella(_)
|
||||
| BeaconState::Eip4844(_) => {
|
||||
process_slashings(
|
||||
state,
|
||||
altair::ParticipationCache::new(state, spec)
|
||||
@ -205,9 +212,10 @@ impl<E: EthSpec> EpochTransition<E> for SyncCommitteeUpdates {
|
||||
fn run(state: &mut BeaconState<E>, spec: &ChainSpec) -> Result<(), EpochProcessingError> {
|
||||
match state {
|
||||
BeaconState::Base(_) => Ok(()),
|
||||
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Eip4844(_) => {
|
||||
altair::process_sync_committee_updates(state, spec)
|
||||
}
|
||||
BeaconState::Altair(_)
|
||||
| BeaconState::Merge(_)
|
||||
| BeaconState::Capella(_)
|
||||
| BeaconState::Eip4844(_) => altair::process_sync_committee_updates(state, spec),
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -216,13 +224,14 @@ impl<E: EthSpec> EpochTransition<E> for InactivityUpdates {
|
||||
fn run(state: &mut BeaconState<E>, spec: &ChainSpec) -> Result<(), EpochProcessingError> {
|
||||
match state {
|
||||
BeaconState::Base(_) => Ok(()),
|
||||
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Eip4844(_) => {
|
||||
altair::process_inactivity_updates(
|
||||
state,
|
||||
&altair::ParticipationCache::new(state, spec).unwrap(),
|
||||
spec,
|
||||
)
|
||||
}
|
||||
BeaconState::Altair(_)
|
||||
| BeaconState::Merge(_)
|
||||
| BeaconState::Capella(_)
|
||||
| BeaconState::Eip4844(_) => altair::process_inactivity_updates(
|
||||
state,
|
||||
&altair::ParticipationCache::new(state, spec).unwrap(),
|
||||
spec,
|
||||
),
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -231,9 +240,10 @@ impl<E: EthSpec> EpochTransition<E> for ParticipationFlagUpdates {
|
||||
fn run(state: &mut BeaconState<E>, _: &ChainSpec) -> Result<(), EpochProcessingError> {
|
||||
match state {
|
||||
BeaconState::Base(_) => Ok(()),
|
||||
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Eip4844(_) => {
|
||||
altair::process_participation_flag_updates(state)
|
||||
}
|
||||
BeaconState::Altair(_)
|
||||
| BeaconState::Merge(_)
|
||||
| BeaconState::Capella(_)
|
||||
| BeaconState::Eip4844(_) => altair::process_participation_flag_updates(state),
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -280,6 +290,7 @@ impl<E: EthSpec, T: EpochTransition<E>> Case for EpochProcessing<E, T> {
|
||||
}
|
||||
// No phase0 tests for Altair and later.
|
||||
ForkName::Altair | ForkName::Merge => T::name() != "participation_record_updates",
|
||||
ForkName::Capella => false, // TODO: revisit when tests are out
|
||||
ForkName::Eip4844 => false, // TODO: revisit when tests are out
|
||||
}
|
||||
}
|
||||
|
@ -62,6 +62,7 @@ impl<E: EthSpec> Case for ForkTest<E> {
|
||||
ForkName::Altair => upgrade_to_altair(&mut result_state, spec).map(|_| result_state),
|
||||
ForkName::Merge => upgrade_to_bellatrix(&mut result_state, spec).map(|_| result_state),
|
||||
ForkName::Eip4844 => panic!("eip4844 not supported"),
|
||||
ForkName::Capella => panic!("capella not supported"),
|
||||
};
|
||||
|
||||
compare_beacon_state_results_without_caches(&mut result, &mut expected)
|
||||
|
@ -4,7 +4,10 @@ use crate::decode::{ssz_decode_file, ssz_decode_state, yaml_decode_file};
|
||||
use serde_derive::Deserialize;
|
||||
use state_processing::initialize_beacon_state_from_eth1;
|
||||
use std::path::PathBuf;
|
||||
use types::{BeaconState, Deposit, EthSpec, ExecutionPayloadHeader, ForkName, Hash256};
|
||||
use types::{
|
||||
BeaconState, Deposit, EthSpec, ExecutionPayloadHeader, ExecutionPayloadHeaderMerge, ForkName,
|
||||
Hash256,
|
||||
};
|
||||
|
||||
#[derive(Debug, Clone, Deserialize)]
|
||||
struct Metadata {
|
||||
@ -38,9 +41,14 @@ impl<E: EthSpec> LoadCase for GenesisInitialization<E> {
|
||||
let meta: Metadata = yaml_decode_file(&path.join("meta.yaml"))?;
|
||||
let execution_payload_header: Option<ExecutionPayloadHeader<E>> =
|
||||
if meta.execution_payload_header.unwrap_or(false) {
|
||||
Some(ssz_decode_file(
|
||||
//FIXME(sean) we could decode based on timestamp - we probably don't do decode a payload
|
||||
// without a block this elsewhere at presetn. But when we support SSZ in the builder api we may need to.
|
||||
// Although that API should include fork info. Hardcoding this for now
|
||||
Some(ExecutionPayloadHeader::Merge(ssz_decode_file::<
|
||||
ExecutionPayloadHeaderMerge<E>,
|
||||
>(
|
||||
&path.join("execution_payload_header.ssz_snappy"),
|
||||
)?)
|
||||
)?))
|
||||
} else {
|
||||
None
|
||||
};
|
||||
|
@ -5,6 +5,7 @@ use crate::decode::{ssz_decode_file, ssz_decode_file_with, ssz_decode_state, yam
|
||||
use crate::testing_spec;
|
||||
use crate::type_name::TypeName;
|
||||
use serde_derive::Deserialize;
|
||||
use ssz::Decode;
|
||||
use state_processing::per_block_processing::{
|
||||
errors::BlockProcessingError,
|
||||
process_block_header, process_execution_payload,
|
||||
@ -18,8 +19,8 @@ use std::fmt::Debug;
|
||||
use std::path::Path;
|
||||
use types::{
|
||||
Attestation, AttesterSlashing, BeaconBlock, BeaconState, BlindedPayload, ChainSpec, Deposit,
|
||||
EthSpec, ExecutionPayload, ForkName, FullPayload, ProposerSlashing, SignedVoluntaryExit,
|
||||
SyncAggregate,
|
||||
EthSpec, ExecutionPayload, ExecutionPayloadMerge, ForkName, FullPayload, ProposerSlashing,
|
||||
SignedVoluntaryExit, SyncAggregate,
|
||||
};
|
||||
|
||||
#[derive(Debug, Clone, Default, Deserialize)]
|
||||
@ -81,16 +82,17 @@ impl<E: EthSpec> Operation<E> for Attestation<E> {
|
||||
BeaconState::Base(_) => {
|
||||
base::process_attestations(state, &[self.clone()], VerifySignatures::True, spec)
|
||||
}
|
||||
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Eip4844(_) => {
|
||||
altair::process_attestation(
|
||||
state,
|
||||
self,
|
||||
0,
|
||||
proposer_index,
|
||||
VerifySignatures::True,
|
||||
spec,
|
||||
)
|
||||
}
|
||||
BeaconState::Altair(_)
|
||||
| BeaconState::Merge(_)
|
||||
| BeaconState::Capella(_)
|
||||
| BeaconState::Eip4844(_) => altair::process_attestation(
|
||||
state,
|
||||
self,
|
||||
0,
|
||||
proposer_index,
|
||||
VerifySignatures::True,
|
||||
spec,
|
||||
),
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -237,8 +239,13 @@ impl<E: EthSpec> Operation<E> for FullPayload<E> {
|
||||
fork_name != ForkName::Base && fork_name != ForkName::Altair
|
||||
}
|
||||
|
||||
//FIXME(sean) we could decode based on timestamp - we probably don't do decode a payload
|
||||
// without a block this elsewhere at presetn. But when we support SSZ in the builder api we may need to.
|
||||
// Although that API should include fork info. Hardcoding this for now
|
||||
fn decode(path: &Path, _spec: &ChainSpec) -> Result<Self, Error> {
|
||||
ssz_decode_file(path)
|
||||
ssz_decode_file::<ExecutionPayloadMerge<E>>(path)
|
||||
.map(ExecutionPayload::Merge)
|
||||
.map(Into::into)
|
||||
}
|
||||
|
||||
fn apply_to(
|
||||
@ -252,7 +259,7 @@ impl<E: EthSpec> Operation<E> for FullPayload<E> {
|
||||
.as_ref()
|
||||
.map_or(false, |e| e.execution_valid);
|
||||
if valid {
|
||||
process_execution_payload(state, self, spec)
|
||||
process_execution_payload::<E, FullPayload<E>>(state, self.to_ref(), spec)
|
||||
} else {
|
||||
Err(BlockProcessingError::ExecutionInvalid)
|
||||
}
|
||||
@ -272,7 +279,12 @@ impl<E: EthSpec> Operation<E> for BlindedPayload<E> {
|
||||
}
|
||||
|
||||
fn decode(path: &Path, _spec: &ChainSpec) -> Result<Self, Error> {
|
||||
ssz_decode_file::<ExecutionPayload<E>>(path).map(Into::into)
|
||||
//FIXME(sean) we could decode based on timestamp - we probably don't do decode a payload
|
||||
// without a block this elsewhere at presetn. But when we support SSZ in the builder api we may need to.
|
||||
// Although that API should include fork info. Hardcoding this for now
|
||||
let payload: Result<ExecutionPayload<E>, Error> =
|
||||
ssz_decode_file::<ExecutionPayloadMerge<E>>(path).map(Into::into);
|
||||
payload.map(Into::into)
|
||||
}
|
||||
|
||||
fn apply_to(
|
||||
@ -286,7 +298,7 @@ impl<E: EthSpec> Operation<E> for BlindedPayload<E> {
|
||||
.as_ref()
|
||||
.map_or(false, |e| e.execution_valid);
|
||||
if valid {
|
||||
process_execution_payload(state, self, spec)
|
||||
process_execution_payload::<E, BlindedPayload<E>>(state, self.to_ref(), spec)
|
||||
} else {
|
||||
Err(BlockProcessingError::ExecutionInvalid)
|
||||
}
|
||||
|
@ -46,6 +46,10 @@ impl<E: EthSpec> LoadCase for TransitionTest<E> {
|
||||
spec.bellatrix_fork_epoch = Some(Epoch::new(0));
|
||||
spec.eip4844_fork_epoch = Some(metadata.fork_epoch);
|
||||
}
|
||||
ForkName::Capella => {
|
||||
spec.capella_fork_epoch = Some(Epoch::new(0));
|
||||
spec.capella_fork_epoch = Some(metadata.fork_epoch);
|
||||
}
|
||||
}
|
||||
|
||||
// Load blocks
|
||||
|
@ -4,7 +4,8 @@ use crate::execution_engine::{
|
||||
use crate::transactions::transactions;
|
||||
use ethers_providers::Middleware;
|
||||
use execution_layer::{
|
||||
BuilderParams, ChainHealth, ExecutionLayer, PayloadAttributes, PayloadStatus,
|
||||
BuilderParams, ChainHealth, ExecutionLayer, PayloadAttributes, PayloadAttributesV1,
|
||||
PayloadStatus,
|
||||
};
|
||||
use fork_choice::ForkchoiceUpdateParameters;
|
||||
use reqwest::{header::CONTENT_TYPE, Client};
|
||||
@ -278,11 +279,11 @@ impl<E: GenericExecutionEngine> TestRig<E> {
|
||||
Slot::new(1), // Insert proposer for the next slot
|
||||
head_root,
|
||||
proposer_index,
|
||||
PayloadAttributes {
|
||||
PayloadAttributes::V1(PayloadAttributesV1 {
|
||||
timestamp,
|
||||
prev_randao,
|
||||
suggested_fee_recipient: Address::zero(),
|
||||
},
|
||||
}),
|
||||
)
|
||||
.await;
|
||||
|
||||
@ -329,7 +330,8 @@ impl<E: GenericExecutionEngine> TestRig<E> {
|
||||
)
|
||||
.await
|
||||
.unwrap()
|
||||
.execution_payload;
|
||||
.to_payload()
|
||||
.execution_payload();
|
||||
|
||||
/*
|
||||
* Execution Engine A:
|
||||
@ -337,7 +339,7 @@ impl<E: GenericExecutionEngine> TestRig<E> {
|
||||
* Indicate that the payload is the head of the chain, before submitting a
|
||||
* `notify_new_payload`.
|
||||
*/
|
||||
let head_block_hash = valid_payload.block_hash;
|
||||
let head_block_hash = valid_payload.block_hash();
|
||||
let finalized_block_hash = ExecutionBlockHash::zero();
|
||||
let slot = Slot::new(42);
|
||||
let head_block_root = Hash256::repeat_byte(42);
|
||||
@ -377,7 +379,7 @@ impl<E: GenericExecutionEngine> TestRig<E> {
|
||||
*
|
||||
* Do not provide payload attributes (we'll test that later).
|
||||
*/
|
||||
let head_block_hash = valid_payload.block_hash;
|
||||
let head_block_hash = valid_payload.block_hash();
|
||||
let finalized_block_hash = ExecutionBlockHash::zero();
|
||||
let slot = Slot::new(42);
|
||||
let head_block_root = Hash256::repeat_byte(42);
|
||||
@ -394,7 +396,7 @@ impl<E: GenericExecutionEngine> TestRig<E> {
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(status, PayloadStatus::Valid);
|
||||
assert_eq!(valid_payload.transactions.len(), pending_txs.len());
|
||||
assert_eq!(valid_payload.transactions().len(), pending_txs.len());
|
||||
|
||||
// Verify that all submitted txs were successful
|
||||
for pending_tx in pending_txs {
|
||||
@ -414,7 +416,7 @@ impl<E: GenericExecutionEngine> TestRig<E> {
|
||||
*/
|
||||
|
||||
let mut invalid_payload = valid_payload.clone();
|
||||
invalid_payload.prev_randao = Hash256::from_low_u64_be(42);
|
||||
*invalid_payload.prev_randao_mut() = Hash256::from_low_u64_be(42);
|
||||
let status = self
|
||||
.ee_a
|
||||
.execution_layer
|
||||
@ -429,8 +431,8 @@ impl<E: GenericExecutionEngine> TestRig<E> {
|
||||
* Produce another payload atop the previous one.
|
||||
*/
|
||||
|
||||
let parent_hash = valid_payload.block_hash;
|
||||
let timestamp = valid_payload.timestamp + 1;
|
||||
let parent_hash = valid_payload.block_hash();
|
||||
let timestamp = valid_payload.timestamp() + 1;
|
||||
let prev_randao = Hash256::zero();
|
||||
let proposer_index = 0;
|
||||
let builder_params = BuilderParams {
|
||||
@ -452,7 +454,8 @@ impl<E: GenericExecutionEngine> TestRig<E> {
|
||||
)
|
||||
.await
|
||||
.unwrap()
|
||||
.execution_payload;
|
||||
.to_payload()
|
||||
.execution_payload();
|
||||
|
||||
/*
|
||||
* Execution Engine A:
|
||||
@ -474,13 +477,13 @@ impl<E: GenericExecutionEngine> TestRig<E> {
|
||||
*
|
||||
* Indicate that the payload is the head of the chain, providing payload attributes.
|
||||
*/
|
||||
let head_block_hash = valid_payload.block_hash;
|
||||
let head_block_hash = valid_payload.block_hash();
|
||||
let finalized_block_hash = ExecutionBlockHash::zero();
|
||||
let payload_attributes = PayloadAttributes {
|
||||
timestamp: second_payload.timestamp + 1,
|
||||
let payload_attributes = PayloadAttributes::V1(PayloadAttributesV1 {
|
||||
timestamp: second_payload.timestamp() + 1,
|
||||
prev_randao: Hash256::zero(),
|
||||
suggested_fee_recipient: Address::zero(),
|
||||
};
|
||||
});
|
||||
let slot = Slot::new(42);
|
||||
let head_block_root = Hash256::repeat_byte(100);
|
||||
let validator_index = 0;
|
||||
@ -524,7 +527,7 @@ impl<E: GenericExecutionEngine> TestRig<E> {
|
||||
*
|
||||
* Set the second payload as the head, without providing payload attributes.
|
||||
*/
|
||||
let head_block_hash = second_payload.block_hash;
|
||||
let head_block_hash = second_payload.block_hash();
|
||||
let finalized_block_hash = ExecutionBlockHash::zero();
|
||||
let slot = Slot::new(42);
|
||||
let head_block_root = Hash256::repeat_byte(42);
|
||||
@ -576,7 +579,7 @@ impl<E: GenericExecutionEngine> TestRig<E> {
|
||||
*
|
||||
* Set the second payload as the head, without providing payload attributes.
|
||||
*/
|
||||
let head_block_hash = second_payload.block_hash;
|
||||
let head_block_hash = second_payload.block_hash();
|
||||
let finalized_block_hash = ExecutionBlockHash::zero();
|
||||
let slot = Slot::new(42);
|
||||
let head_block_root = Hash256::repeat_byte(42);
|
||||
@ -605,7 +608,7 @@ async fn check_payload_reconstruction<E: GenericExecutionEngine>(
|
||||
) {
|
||||
let reconstructed = ee
|
||||
.execution_layer
|
||||
.get_payload_by_block_hash(payload.block_hash)
|
||||
.get_payload_by_block_hash(payload.block_hash())
|
||||
.await
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
|
@ -1,7 +1,7 @@
|
||||
use crate::local_network::LocalNetwork;
|
||||
use node_test_rig::eth2::types::{BlockId, StateId};
|
||||
use std::time::Duration;
|
||||
use types::{Epoch, EthSpec, ExecutionBlockHash, Hash256, Slot, Unsigned};
|
||||
use types::{Epoch, EthSpec, ExecPayload, ExecutionBlockHash, Hash256, Slot, Unsigned};
|
||||
|
||||
/// Checks that all of the validators have on-boarded by the start of the second eth1 voting
|
||||
/// period.
|
||||
@ -228,7 +228,7 @@ pub async fn verify_transition_block_finalized<E: EthSpec>(
|
||||
.map_err(|e| format!("Get state root via http failed: {:?}", e))?
|
||||
.message()
|
||||
.execution_payload()
|
||||
.map(|payload| payload.execution_payload.block_hash)
|
||||
.map(|payload| payload.block_hash())
|
||||
.map_err(|e| format!("Execution payload does not exist: {:?}", e))?;
|
||||
block_hashes.push(execution_block_hash);
|
||||
}
|
||||
|
@ -13,8 +13,8 @@ use std::ops::Deref;
|
||||
use std::sync::Arc;
|
||||
use tokio::sync::mpsc;
|
||||
use types::{
|
||||
BlindedPayload, BlobsSidecar, BlockType, EthSpec, ExecPayload, ForkName, FullPayload,
|
||||
PublicKeyBytes, Slot,
|
||||
AbstractExecPayload, BlindedPayload, BlobsSidecar, BlockType, EthSpec, ExecPayload, ForkName,
|
||||
FullPayload, PublicKeyBytes, Slot,
|
||||
};
|
||||
|
||||
#[derive(Debug)]
|
||||
@ -276,7 +276,7 @@ impl<T: SlotClock + 'static, E: EthSpec> BlockService<T, E> {
|
||||
}
|
||||
|
||||
/// Produce a block at the given slot for validator_pubkey
|
||||
async fn publish_block<Payload: ExecPayload<E>>(
|
||||
async fn publish_block<Payload: AbstractExecPayload<E>>(
|
||||
self,
|
||||
slot: Slot,
|
||||
validator_pubkey: PublicKeyBytes,
|
||||
@ -444,7 +444,7 @@ impl<T: SlotClock + 'static, E: EthSpec> BlockService<T, E> {
|
||||
"slot" => signed_block.slot().as_u64(),
|
||||
);
|
||||
}
|
||||
ForkName::Eip4844 => {
|
||||
ForkName::Capella | ForkName::Eip4844 => {
|
||||
if matches!(Payload::block_type(), BlockType::Blinded) {
|
||||
//FIXME(sean)
|
||||
crit!(
|
||||
|
@ -34,7 +34,7 @@ pub enum Error {
|
||||
}
|
||||
|
||||
/// Enumerates all messages that can be signed by a validator.
|
||||
pub enum SignableMessage<'a, T: EthSpec, Payload: ExecPayload<T> = FullPayload<T>> {
|
||||
pub enum SignableMessage<'a, T: EthSpec, Payload: AbstractExecPayload<T> = FullPayload<T>> {
|
||||
RandaoReveal(Epoch),
|
||||
BeaconBlock(&'a BeaconBlock<T, Payload>),
|
||||
BlobsSidecar(&'a BlobsSidecar<T>),
|
||||
@ -50,7 +50,7 @@ pub enum SignableMessage<'a, T: EthSpec, Payload: ExecPayload<T> = FullPayload<T
|
||||
ValidatorRegistration(&'a ValidatorRegistrationData),
|
||||
}
|
||||
|
||||
impl<'a, T: EthSpec, Payload: ExecPayload<T>> SignableMessage<'a, T, Payload> {
|
||||
impl<'a, T: EthSpec, Payload: AbstractExecPayload<T>> SignableMessage<'a, T, Payload> {
|
||||
/// Returns the `SignedRoot` for the contained message.
|
||||
///
|
||||
/// The actual `SignedRoot` trait is not used since it also requires a `TreeHash` impl, which is
|
||||
@ -118,7 +118,7 @@ impl SigningContext {
|
||||
|
||||
impl SigningMethod {
|
||||
/// Return the signature of `signable_message`, with respect to the `signing_context`.
|
||||
pub async fn get_signature<T: EthSpec, Payload: ExecPayload<T>>(
|
||||
pub async fn get_signature<T: EthSpec, Payload: AbstractExecPayload<T>>(
|
||||
&self,
|
||||
signable_message: SignableMessage<'_, T, Payload>,
|
||||
signing_context: SigningContext,
|
||||
@ -143,7 +143,7 @@ impl SigningMethod {
|
||||
.await
|
||||
}
|
||||
|
||||
pub async fn get_signature_from_root<T: EthSpec, Payload: ExecPayload<T>>(
|
||||
pub async fn get_signature_from_root<T: EthSpec, Payload: AbstractExecPayload<T>>(
|
||||
&self,
|
||||
signable_message: SignableMessage<'_, T, Payload>,
|
||||
signing_root: Hash256,
|
||||
|
@ -27,6 +27,7 @@ pub enum ForkName {
|
||||
Phase0,
|
||||
Altair,
|
||||
Bellatrix,
|
||||
Capella,
|
||||
Eip4844,
|
||||
}
|
||||
|
||||
@ -38,7 +39,7 @@ pub struct ForkInfo {
|
||||
|
||||
#[derive(Debug, PartialEq, Serialize)]
|
||||
#[serde(bound = "T: EthSpec", rename_all = "snake_case")]
|
||||
pub enum Web3SignerObject<'a, T: EthSpec, Payload: ExecPayload<T>> {
|
||||
pub enum Web3SignerObject<'a, T: EthSpec, Payload: AbstractExecPayload<T>> {
|
||||
AggregationSlot {
|
||||
slot: Slot,
|
||||
},
|
||||
@ -76,7 +77,7 @@ pub enum Web3SignerObject<'a, T: EthSpec, Payload: ExecPayload<T>> {
|
||||
ValidatorRegistration(&'a ValidatorRegistrationData),
|
||||
}
|
||||
|
||||
impl<'a, T: EthSpec, Payload: ExecPayload<T>> Web3SignerObject<'a, T, Payload> {
|
||||
impl<'a, T: EthSpec, Payload: AbstractExecPayload<T>> Web3SignerObject<'a, T, Payload> {
|
||||
pub fn beacon_block(block: &'a BeaconBlock<T, Payload>) -> Result<Self, Error> {
|
||||
match block {
|
||||
BeaconBlock::Base(_) => Ok(Web3SignerObject::BeaconBlock {
|
||||
@ -94,6 +95,11 @@ impl<'a, T: EthSpec, Payload: ExecPayload<T>> Web3SignerObject<'a, T, Payload> {
|
||||
block: None,
|
||||
block_header: Some(block.block_header()),
|
||||
}),
|
||||
BeaconBlock::Capella(_) => Ok(Web3SignerObject::BeaconBlock {
|
||||
version: ForkName::Capella,
|
||||
block: None,
|
||||
block_header: Some(block.block_header()),
|
||||
}),
|
||||
BeaconBlock::Eip4844(_) => Ok(Web3SignerObject::BeaconBlock {
|
||||
version: ForkName::Eip4844,
|
||||
block: None,
|
||||
@ -126,7 +132,7 @@ impl<'a, T: EthSpec, Payload: ExecPayload<T>> Web3SignerObject<'a, T, Payload> {
|
||||
|
||||
#[derive(Debug, PartialEq, Serialize)]
|
||||
#[serde(bound = "T: EthSpec")]
|
||||
pub struct SigningRequest<'a, T: EthSpec, Payload: ExecPayload<T>> {
|
||||
pub struct SigningRequest<'a, T: EthSpec, Payload: AbstractExecPayload<T>> {
|
||||
#[serde(rename = "type")]
|
||||
pub message_type: MessageType,
|
||||
#[serde(skip_serializing_if = "Option::is_none")]
|
||||
|
@ -18,13 +18,13 @@ use std::path::Path;
|
||||
use std::sync::Arc;
|
||||
use task_executor::TaskExecutor;
|
||||
use types::{
|
||||
attestation::Error as AttestationError, graffiti::GraffitiString, Address, AggregateAndProof,
|
||||
Attestation, BeaconBlock, BlindedPayload, BlobsSidecar, ChainSpec, ContributionAndProof,
|
||||
Domain, Epoch, EthSpec, ExecPayload, Fork, FullPayload, Graffiti, Hash256, Keypair,
|
||||
PublicKeyBytes, SelectionProof, Signature, SignedAggregateAndProof, SignedBeaconBlock,
|
||||
SignedBlobsSidecar, SignedContributionAndProof, SignedRoot, SignedValidatorRegistrationData,
|
||||
Slot, SyncAggregatorSelectionData, SyncCommitteeContribution, SyncCommitteeMessage,
|
||||
SyncSelectionProof, SyncSubnetId, ValidatorRegistrationData,
|
||||
attestation::Error as AttestationError, graffiti::GraffitiString, AbstractExecPayload, Address,
|
||||
AggregateAndProof, Attestation, BeaconBlock, BlindedPayload, BlobsSidecar, ChainSpec,
|
||||
ContributionAndProof, Domain, Epoch, EthSpec, ExecPayload, Fork, FullPayload, Graffiti,
|
||||
Hash256, Keypair, PublicKeyBytes, SelectionProof, Signature, SignedAggregateAndProof,
|
||||
SignedBeaconBlock, SignedBlobsSidecar, SignedContributionAndProof, SignedRoot,
|
||||
SignedValidatorRegistrationData, Slot, SyncAggregatorSelectionData, SyncCommitteeContribution,
|
||||
SyncCommitteeMessage, SyncSelectionProof, SyncSubnetId, ValidatorRegistrationData,
|
||||
};
|
||||
use validator_dir::ValidatorDir;
|
||||
|
||||
@ -455,7 +455,7 @@ impl<T: SlotClock + 'static, E: EthSpec> ValidatorStore<T, E> {
|
||||
.unwrap_or(self.builder_proposals)
|
||||
}
|
||||
|
||||
pub async fn sign_block<Payload: ExecPayload<E>>(
|
||||
pub async fn sign_block<Payload: AbstractExecPayload<E>>(
|
||||
&self,
|
||||
validator_pubkey: PublicKeyBytes,
|
||||
block: BeaconBlock<E, Payload>,
|
||||
|
Loading…
Reference in New Issue
Block a user