From 29e5a1f599844007b9a7eed869a14ba140ddb1e5 Mon Sep 17 00:00:00 2001 From: Mark Mackey Date: Wed, 15 Oct 2025 10:15:44 -0500 Subject: [PATCH 1/9] hold for now --- .../beacon_chain/src/envelope_verification.rs | 449 ++++++++++++++++++ .../src/envelope_verification_types.rs | 136 ++++++ beacon_node/beacon_chain/src/lib.rs | 2 + .../src/envelope_processing.rs | 65 +++ consensus/state_processing/src/lib.rs | 1 + .../per_block_processing/signature_sets.rs | 30 +- .../src/signed_execution_payload_envelope.rs | 30 ++ 7 files changed, 712 insertions(+), 1 deletion(-) create mode 100644 beacon_node/beacon_chain/src/envelope_verification.rs create mode 100644 beacon_node/beacon_chain/src/envelope_verification_types.rs create mode 100644 consensus/state_processing/src/envelope_processing.rs diff --git a/beacon_node/beacon_chain/src/envelope_verification.rs b/beacon_node/beacon_chain/src/envelope_verification.rs new file mode 100644 index 00000000000..6f927c63874 --- /dev/null +++ b/beacon_node/beacon_chain/src/envelope_verification.rs @@ -0,0 +1,449 @@ +//! The incremental processing steps (e.g., signatures verified but not the state transition) is +//! represented as a sequence of wrapper-types around the block. There is a linear progression of +//! types, starting at a `SignedBeaconBlock` and finishing with a `Fully VerifiedBlock` (see +//! diagram below). +//! +//! ```ignore +//! START +//! | +//! ▼ +//! SignedExecutionPayloadEnvelope +//! | +//! |--------------- +//! | | +//! | ▼ +//! | GossipVerifiedEnvelope +//! | | +//! |--------------- +//! | +//! ▼ +//! ExecutionPendingEnvelope +//! | +//! await +//! | +//! ▼ +//! END +//! +//! ``` + +use crate::block_verification::{PayloadVerificationHandle, PayloadVerificationOutcome}; +use crate::data_availability_checker::MaybeAvailableEnvelope; +use crate::envelope_verification_types::EnvelopeImportData; +use crate::execution_payload::PayloadNotifier; +use crate::NotifyExecutionLayer; +use crate::{BeaconChain, BeaconChainError, BeaconChainTypes}; +use derivative::Derivative; +use safe_arith::ArithError; +use slot_clock::SlotClock; +use state_processing::envelope_processing::{envelope_processing, EnvelopeProcessingError}; +use state_processing::per_block_processing::compute_timestamp_at_slot; +use state_processing::{BlockProcessingError, VerifySignatures}; +use std::sync::Arc; +use tree_hash::TreeHash; +use types::{ + BeaconState, BeaconStateError, EthSpec, ExecutionBlockHash, Hash256, SignedBlindedBeaconBlock, + SignedExecutionPayloadEnvelope, +}; + +// TODO(EIP7732): don't use this redefinition.. +macro_rules! envelope_verify { + ($condition: expr, $result: expr) => { + if !$condition { + return Err($result); + } + }; +} + +#[derive(Debug)] +pub enum EnvelopeError { + /// The envelope's block root is unknown. + BlockRootUnknown { + block_root: Hash256, + }, + /// The signature is invalid. + BadSignature, + /// Envelope doesn't match latest beacon block header + LatestBlockHeaderMismatch { + envelope_root: Hash256, + block_header_root: Hash256, + }, + /// The builder index doesn't match the committed bid + BuilderIndexMismatch { + committed_bid: u64, + envelope: u64, + }, + /// The blob KZG commitments root doesn't match the committed bid + BlobKzgCommitmentsRootMismatch { + committed_bid: Hash256, + envelope: Hash256, + }, + /// The withdrawals root doesn't match the state's latest withdrawals root + WithdrawalsRootMismatch { + state: Hash256, + envelope: Hash256, + }, + // The gas limit doesn't match the committed bid + GasLimitMismatch { + committed_bid: u64, + envelope: u64, + }, + // The block hash doesn't match the committed bid + BlockHashMismatch { + committed_bid: ExecutionBlockHash, + envelope: ExecutionBlockHash, + }, + // The parent hash doesn't match the previous execution payload + ParentHashMismatch { + state: ExecutionBlockHash, + envelope: ExecutionBlockHash, + }, + // The previous randao didn't match the payload + PrevRandaoMismatch { + state: Hash256, + envelope: Hash256, + }, + // The timestamp didn't match the payload + TimestampMismatch { + state: u64, + envelope: u64, + }, + // Blob committments exceeded the maximum + BlobLimitExceeded { + max: usize, + envelope: usize, + }, + // Invalid state root + InvalidStateRoot { + state: Hash256, + envelope: Hash256, + }, + // The payload was withheld but the block hash + // matched the committed bid + PayloadWithheldBlockHashMismatch, + // Some Beacon Chain Error + BeaconChainError(BeaconChainError), + // Some Beacon State error + BeaconStateError(BeaconStateError), + // Some ArithError + ArithError(ArithError), + // Some BlockProcessingError (for electra operations) + BlockProcessingError(BlockProcessingError), +} + +impl From for EnvelopeError { + fn from(e: BeaconChainError) -> Self { + EnvelopeError::BeaconChainError(e) + } +} + +impl From for EnvelopeError { + fn from(e: BeaconStateError) -> Self { + EnvelopeError::BeaconStateError(e) + } +} + +impl From for EnvelopeError { + fn from(e: ArithError) -> Self { + EnvelopeError::ArithError(e) + } +} + +impl From for EnvelopeError { + fn from(e: EnvelopeProcessingError) -> Self { + match e { + EnvelopeProcessingError::BadSignature => EnvelopeError::BadSignature, + EnvelopeProcessingError::BeaconStateError(e) => EnvelopeError::BeaconStateError(e), + EnvelopeProcessingError::BlockProcessingError(e) => { + EnvelopeError::BlockProcessingError(e) + } + } + } +} + +/// A wrapper around a `SignedExecutionPayloadEnvelope` that indicates it has been approved for re-gossiping on +/// the p2p network. +#[derive(Derivative)] +#[derivative(Debug(bound = "T: BeaconChainTypes"))] +pub struct GossipVerifiedEnvelope { + pub signed_envelope: Arc>, + pub parent_block: Arc>, + pub pre_state: Box>, +} + +impl GossipVerifiedEnvelope { + pub fn new( + signed_envelope: Arc>, + chain: &BeaconChain, + ) -> Result { + let envelope = signed_envelope.message(); + let payload = envelope.payload(); + let block_root = envelope.beacon_block_root(); + + // TODO(EIP7732): this check would fail if the block didn't pass validation right? + + // check that we've seen the parent block of this envelope + let fork_choice_read_lock = chain.canonical_head.fork_choice_read_lock(); + if !fork_choice_read_lock.contains_block(&block_root) { + return Err(EnvelopeError::BlockRootUnknown { block_root }); + } + drop(fork_choice_read_lock); + + let parent_block = chain + .get_blinded_block(&block_root)? + .ok_or_else(|| EnvelopeError::from(BeaconChainError::MissingBeaconBlock(block_root))) + .map(Arc::new)?; + let execution_bid = &parent_block + .message() + .body() + .signed_execution_bid()? + .message; + + // TODO(EIP7732): check we're within the bounds of the slot (probably) + + // TODO(EIP7732): check that we haven't seen another valid `SignedExecutionPayloadEnvelope` + // for this block root from this builder + + // builder index matches committed bid + if envelope.builder_index() != execution_bid.builder_index { + return Err(EnvelopeError::BuilderIndexMismatch { + committed_bid: execution_bid.builder_index, + envelope: envelope.builder_index(), + }); + } + + // if payload is withheld, the block hash should not match the committed bid + if !envelope.payload_withheld() && payload.block_hash() == execution_bid.block_hash { + return Err(EnvelopeError::PayloadWithheldBlockHashMismatch); + } + + let parent_state = chain + .get_state( + &parent_block.message().state_root(), + Some(parent_block.slot()), + )? + .ok_or_else(|| { + EnvelopeError::from(BeaconChainError::MissingBeaconState( + parent_block.message().state_root(), + )) + })?; + + // verify the signature + if !signed_envelope.verify_signature(&parent_state, &chain.spec)? { + return Err(EnvelopeError::BadSignature); + } + + Ok(Self { + signed_envelope, + parent_block, + pre_state: Box::new(parent_state), + }) + } + + pub fn envelope_cloned(&self) -> Arc> { + self.signed_envelope.clone() + } +} + +pub trait IntoExecutionPendingEnvelope: Sized { + fn into_execution_pending_envelope( + self, + chain: &Arc>, + notify_execution_layer: NotifyExecutionLayer, + ) -> Result, EnvelopeError>; +} + +pub struct ExecutionPendingEnvelope { + pub signed_envelope: MaybeAvailableEnvelope, + pub import_data: EnvelopeImportData, + pub payload_verification_handle: PayloadVerificationHandle, +} + +impl IntoExecutionPendingEnvelope for GossipVerifiedEnvelope { + fn into_execution_pending_envelope( + self, + chain: &Arc>, + notify_execution_layer: NotifyExecutionLayer, + ) -> Result, EnvelopeError> { + let signed_envelope = self.signed_envelope; + let envelope = signed_envelope.message(); + let payload = &envelope.payload(); + + // verify signature already done + let mut state = *self.pre_state; + + // setting state.latest_block_header happens in envelope_processing + + // Verify consistency with the beacon block + if !envelope.tree_hash_root() == state.latest_block_header().tree_hash_root() { + return Err(EnvelopeError::LatestBlockHeaderMismatch { + envelope_root: envelope.tree_hash_root(), + block_header_root: state.latest_block_header().tree_hash_root(), + }); + }; + + // Verify consistency with the committed bid + let committed_bid = state.latest_execution_bid()?; + // builder index match already verified + if committed_bid.blob_kzg_commitments_root + != envelope.blob_kzg_commitments().tree_hash_root() + { + return Err(EnvelopeError::BlobKzgCommitmentsRootMismatch { + committed_bid: committed_bid.blob_kzg_commitments_root, + envelope: envelope.blob_kzg_commitments().tree_hash_root(), + }); + }; + + if !envelope.payload_withheld() { + // Verify the withdrawals root + envelope_verify!( + payload.withdrawals()?.tree_hash_root() == state.latest_withdrawals_root()?, + EnvelopeError::WithdrawalsRootMismatch { + state: state.latest_withdrawals_root()?, + envelope: payload.withdrawals()?.tree_hash_root(), + } + .into() + ); + + // Verify the gas limit + envelope_verify!( + payload.gas_limit() == committed_bid.gas_limit, + EnvelopeError::GasLimitMismatch { + committed_bid: committed_bid.gas_limit, + envelope: payload.gas_limit(), + } + .into() + ); + // Verify the block hash + envelope_verify!( + committed_bid.block_hash == payload.block_hash(), + EnvelopeError::BlockHashMismatch { + committed_bid: committed_bid.block_hash, + envelope: payload.block_hash(), + } + .into() + ); + + // Verify consistency of the parent hash with respect to the previous execution payload + envelope_verify!( + payload.parent_hash() == state.latest_block_hash()?, + EnvelopeError::ParentHashMismatch { + state: state.latest_block_hash()?, + envelope: payload.parent_hash(), + } + .into() + ); + + // Verify prev_randao + envelope_verify!( + payload.prev_randao() == *state.get_randao_mix(state.current_epoch())?, + EnvelopeError::PrevRandaoMismatch { + state: *state.get_randao_mix(state.current_epoch())?, + envelope: payload.prev_randao(), + } + .into() + ); + + // Verify the timestamp + let state_timestamp = + compute_timestamp_at_slot(&state, state.slot(), chain.spec.as_ref())?; + envelope_verify!( + payload.timestamp() == state_timestamp, + EnvelopeError::TimestampMismatch { + state: state_timestamp, + envelope: payload.timestamp(), + } + .into() + ); + + // Verify the commitments are under limit + envelope_verify!( + envelope.blob_kzg_commitments().len() + <= T::EthSpec::max_blob_commitments_per_block(), + EnvelopeError::BlobLimitExceeded { + max: T::EthSpec::max_blob_commitments_per_block(), + envelope: envelope.blob_kzg_commitments().len(), + } + .into() + ); + } + + // Verify the execution payload is valid + let payload_notifier = + PayloadNotifier::from_envelope(chain.clone(), envelope, notify_execution_layer)?; + let block_root = envelope.beacon_block_root(); + let slot = self.parent_block.slot(); + + let payload_verification_future = async move { + let chain = payload_notifier.chain.clone(); + // TODO:(EIP7732): timing + if let Some(started_execution) = chain.slot_clock.now_duration() { + chain.block_times_cache.write().set_time_started_execution( + block_root, + slot, + started_execution, + ); + } + + let payload_verification_status = payload_notifier.notify_new_payload().await?; + Ok(PayloadVerificationOutcome { + payload_verification_status, + // This fork is after the merge so it'll never be the merge transition block + is_valid_merge_transition_block: false, + }) + }; + // Spawn the payload verification future as a new task, but don't wait for it to complete. + // The `payload_verification_future` will be awaited later to ensure verification completed + // successfully. + let payload_verification_handle = chain + .task_executor + .spawn_handle( + payload_verification_future, + "execution_payload_verification", + ) + .ok_or(BeaconChainError::RuntimeShutdown)?; + + // All the state modifications are done in envelope_processing + envelope_processing( + &mut state, + &signed_envelope, + VerifySignatures::False, + &chain.spec, + )?; + + // TODO(EIP7732): if verify + envelope_verify!( + state.canonical_root()? == envelope.state_root(), + EnvelopeError::InvalidStateRoot { + state: state.canonical_root()?, + envelope: envelope.state_root(), + } + ); + + Ok(ExecutionPendingEnvelope { + signed_envelope: MaybeAvailableEnvelope::AvailabilityPending { + block_root, + envelope: signed_envelope, + }, + import_data: EnvelopeImportData { + block_root, + parent_block: self.parent_block, + post_state: Box::new(state), + }, + payload_verification_handle, + }) + } +} + +impl IntoExecutionPendingEnvelope + for Arc> +{ + fn into_execution_pending_envelope( + self, + chain: &Arc>, + notify_execution_layer: NotifyExecutionLayer, + ) -> Result, EnvelopeError> { + // TODO(EIP7732): figure out how this should be refactored.. + GossipVerifiedEnvelope::new(self, chain)? + .into_execution_pending_envelope(chain, notify_execution_layer) + } +} \ No newline at end of file diff --git a/beacon_node/beacon_chain/src/envelope_verification_types.rs b/beacon_node/beacon_chain/src/envelope_verification_types.rs new file mode 100644 index 00000000000..ac58c985822 --- /dev/null +++ b/beacon_node/beacon_chain/src/envelope_verification_types.rs @@ -0,0 +1,136 @@ +use crate::data_availability_checker::{AvailableEnvelope, MaybeAvailableEnvelope}; +use crate::PayloadVerificationOutcome; +use std::sync::Arc; +use types::{ + BeaconState, BlobIdentifier, EthSpec, Hash256, SignedBlindedBeaconBlock, + SignedExecutionPayloadEnvelope, +}; + +/// A block that has completed all pre-deneb block processing checks including verification +/// by an EL client **and** has all requisite blob data to be imported into fork choice. +#[derive(PartialEq)] +pub struct AvailableExecutedEnvelope { + pub envelope: AvailableEnvelope, + pub import_data: EnvelopeImportData, + pub payload_verification_outcome: PayloadVerificationOutcome, +} + +impl AvailableExecutedEnvelope { + pub fn new( + envelope: AvailableEnvelope, + import_data: EnvelopeImportData, + payload_verification_outcome: PayloadVerificationOutcome, + ) -> Self { + Self { + envelope, + import_data, + payload_verification_outcome, + } + } + + pub fn get_all_blob_ids(&self) -> Vec { + let num_blobs_expected = self + .envelope + .envelope() + .message() + .blob_kzg_commitments() + .len(); + let mut blob_ids = Vec::with_capacity(num_blobs_expected); + for i in 0..num_blobs_expected { + blob_ids.push(BlobIdentifier { + block_root: self.import_data.block_root, + index: i as u64, + }); + } + blob_ids + } +} + +#[derive(PartialEq)] +pub struct EnvelopeImportData { + pub block_root: Hash256, + pub parent_block: Arc>, + pub post_state: Box>, +} + +pub struct AvailabilityPendingExecutedEnvelope { + pub envelope: Arc>, + pub import_data: EnvelopeImportData, + pub payload_verification_outcome: PayloadVerificationOutcome, +} + +impl AvailabilityPendingExecutedEnvelope { + pub fn new( + envelope: Arc>, + import_data: EnvelopeImportData, + payload_verification_outcome: PayloadVerificationOutcome, + ) -> Self { + Self { + envelope, + import_data, + payload_verification_outcome, + } + } + + pub fn as_envelope(&self) -> &SignedExecutionPayloadEnvelope { + self.envelope.as_ref() + } + + pub fn num_blobs_expected(&self) -> usize { + self.envelope.message().blob_kzg_commitments().len() + } +} + +/// An envelope that has gone through all envelope processing checks including envelope processing +/// and execution by an EL client. This block hasn't necessarily completed data availability checks. +/// +/// +/// It contains 2 variants: +/// 1. `Available`: This envelope has been executed and also contains all data to consider it a +/// fully available envelope. +/// 2. `AvailabilityPending`: This envelope hasn't received all required blobs to consider it a +/// fully available envelope. +pub enum ExecutedEnvelope { + Available(AvailableExecutedEnvelope), + AvailabilityPending(AvailabilityPendingExecutedEnvelope), +} + +impl ExecutedEnvelope { + pub fn new( + envelope: MaybeAvailableEnvelope, + import_data: EnvelopeImportData, + payload_verification_outcome: PayloadVerificationOutcome, + ) -> Self { + match envelope { + MaybeAvailableEnvelope::Available(available_envelope) => { + Self::Available(AvailableExecutedEnvelope::new( + available_envelope, + import_data, + payload_verification_outcome, + )) + } + MaybeAvailableEnvelope::AvailabilityPending { + block_root: _, + envelope, + } => Self::AvailabilityPending(AvailabilityPendingExecutedEnvelope::new( + envelope, + import_data, + payload_verification_outcome, + )), + } + } + + pub fn as_envelope(&self) -> &SignedExecutionPayloadEnvelope { + match self { + Self::Available(available) => available.envelope.envelope(), + Self::AvailabilityPending(pending) => pending.envelope.as_ref(), + } + } + + pub fn block_root(&self) -> Hash256 { + match self { + Self::Available(available) => available.import_data.block_root, + Self::AvailabilityPending(pending) => pending.import_data.block_root, + } + } +} \ No newline at end of file diff --git a/beacon_node/beacon_chain/src/lib.rs b/beacon_node/beacon_chain/src/lib.rs index 4ac3e54742d..8df88532c22 100644 --- a/beacon_node/beacon_chain/src/lib.rs +++ b/beacon_node/beacon_chain/src/lib.rs @@ -21,6 +21,8 @@ pub mod custody_context; pub mod data_availability_checker; pub mod data_column_verification; mod early_attester_cache; +pub mod envelope_verification; +pub mod envelope_verification_types; mod errors; pub mod events; pub mod execution_payload; diff --git a/consensus/state_processing/src/envelope_processing.rs b/consensus/state_processing/src/envelope_processing.rs new file mode 100644 index 00000000000..aafc6bb3e85 --- /dev/null +++ b/consensus/state_processing/src/envelope_processing.rs @@ -0,0 +1,65 @@ +use crate::per_block_processing::process_operations::{ + process_consolidation_requests, process_deposit_requests, process_withdrawal_requests, +}; +use crate::BlockProcessingError; +use crate::VerifySignatures; +use types::{BeaconState, BeaconStateError, ChainSpec, EthSpec, Hash256, SignedExecutionPayloadEnvelope}; + +#[derive(Debug)] +pub enum EnvelopeProcessingError { + /// Bad Signature + BadSignature, + BeaconStateError(BeaconStateError), + BlockProcessingError(BlockProcessingError), +} + +impl From for EnvelopeProcessingError { + fn from(e: BeaconStateError) -> Self { + EnvelopeProcessingError::BeaconStateError(e) + } +} + +impl From for EnvelopeProcessingError { + fn from(e: BlockProcessingError) -> Self { + EnvelopeProcessingError::BlockProcessingError(e) + } +} + +/// Processes a `SignedExecutionPayloadEnvelope` +/// +/// This function does all the state modifications inside `process_execution_payload()` +pub fn envelope_processing( + state: &mut BeaconState, + signed_envelope: &SignedExecutionPayloadEnvelope, + verify_signatures: VerifySignatures, + spec: &ChainSpec, +) -> Result<(), EnvelopeProcessingError> { + if verify_signatures.is_true() { + // Verify Signed Envelope Signature + if !signed_envelope.verify_signature(&state, spec)? { + return Err(EnvelopeProcessingError::BadSignature); + } + } + + // Cache latest block header state root + let previous_state_root = state.canonical_root()?; + if state.latest_block_header().state_root == Hash256::default() { + state.latest_block_header_mut().state_root = previous_state_root; + } + + // Verify consistency with the beacon block + + // process electra operations + let envelope = signed_envelope.message(); + let payload = envelope.payload(); + let execution_requests = envelope.execution_requests(); + process_deposit_requests(state, &execution_requests.deposits, spec)?; + process_withdrawal_requests(state, &execution_requests.withdrawals, spec)?; + process_consolidation_requests(state, &execution_requests.consolidations, spec)?; + + // cache the latest block hash and full slot + *state.latest_block_hash_mut()? = payload.block_hash(); + + todo!("the rest of process_execution_payload()"); + //Ok(()) +} \ No newline at end of file diff --git a/consensus/state_processing/src/lib.rs b/consensus/state_processing/src/lib.rs index 9b2696c6d59..e37c5265799 100644 --- a/consensus/state_processing/src/lib.rs +++ b/consensus/state_processing/src/lib.rs @@ -20,6 +20,7 @@ pub mod all_caches; pub mod block_replayer; pub mod common; pub mod consensus_context; +pub mod envelope_processing; pub mod epoch_cache; pub mod genesis; pub mod per_block_processing; diff --git a/consensus/state_processing/src/per_block_processing/signature_sets.rs b/consensus/state_processing/src/per_block_processing/signature_sets.rs index dafd0d79ea9..af1b75a00f1 100644 --- a/consensus/state_processing/src/per_block_processing/signature_sets.rs +++ b/consensus/state_processing/src/per_block_processing/signature_sets.rs @@ -12,7 +12,7 @@ use types::{ InconsistentFork, IndexedAttestation, IndexedAttestationRef, ProposerSlashing, PublicKey, PublicKeyBytes, Signature, SignedAggregateAndProof, SignedBeaconBlock, SignedBeaconBlockHeader, SignedBlsToExecutionChange, SignedContributionAndProof, SignedRoot, SignedVoluntaryExit, - SigningData, Slot, SyncAggregate, SyncAggregatorSelectionData, Unsigned, + SigningData, Slot, SyncAggregate, SyncAggregatorSelectionData, Unsigned, SignedExecutionPayloadEnvelope, }; pub type Result = std::result::Result; @@ -331,6 +331,34 @@ where Ok(SignatureSet::multiple_pubkeys(signature, pubkeys, message)) } +pub fn execution_envelope_signature_set<'a, E, F>( + state: &'a BeaconState, + get_pubkey: F, + signed_envelope: &'a SignedExecutionPayloadEnvelope, + spec: &'a ChainSpec, +) -> Result> +where + E: EthSpec, + F: Fn(usize) -> Option>, +{ + let domain = spec.get_domain( + state.current_epoch(), + Domain::BeaconBuilder, + &state.fork(), + state.genesis_validators_root(), + ); + let message = signed_envelope.message().signing_root(domain); + let pubkey = get_pubkey(signed_envelope.message().builder_index() as usize).ok_or( + Error::ValidatorUnknown(signed_envelope.message().builder_index()), + )?; + + Ok(SignatureSet::single_pubkey( + signed_envelope.signature(), + pubkey, + message, + )) +} + /// Returns the signature set for the given `attester_slashing` and corresponding `pubkeys`. pub fn attester_slashing_signature_sets<'a, E, F>( state: &'a BeaconState, diff --git a/consensus/types/src/signed_execution_payload_envelope.rs b/consensus/types/src/signed_execution_payload_envelope.rs index 96276a764ba..f8b6e48bcea 100644 --- a/consensus/types/src/signed_execution_payload_envelope.rs +++ b/consensus/types/src/signed_execution_payload_envelope.rs @@ -74,6 +74,36 @@ impl SignedExecutionPayloadEnvelope { Self::NextFork(signed) => ExecutionPayloadEnvelopeRef::NextFork(&signed.message), } } + + /// Verify `self.signature`. + /// + /// The `parent_state` is the post-state of the beacon block with + /// block_root = self.message.beacon_block_root + pub fn verify_signature( + &self, + parent_state: &BeaconState, + spec: &ChainSpec, + ) -> Result { + let domain = spec.get_domain( + parent_state.current_epoch(), + Domain::BeaconBuilder, + &parent_state.fork(), + parent_state.genesis_validators_root(), + ); + let pubkey = parent_state + .validators() + .get(self.message().builder_index() as usize) + .and_then(|v| { + let pk: Option = v.pubkey.decompress().ok(); + pk + }) + .ok_or_else(|| { + BeaconStateError::UnknownValidator(self.message().builder_index() as usize) + })?; + let message = self.message().signing_root(domain); + + Ok(self.signature().verify(&pubkey, message)) + } } impl<'de, E: EthSpec> ContextDeserialize<'de, ForkName> for SignedExecutionPayloadEnvelope { From f3b79839a198223c0f0dc25ce2148e1f0d3f32da Mon Sep 17 00:00:00 2001 From: Mark Mackey Date: Mon, 20 Oct 2025 12:51:21 -0500 Subject: [PATCH 2/9] Added NewPayloadCache --- beacon_node/execution_layer/src/lib.rs | 195 ++++++++++++++---- .../execution_layer/src/versioned_hashes.rs | 2 +- 2 files changed, 159 insertions(+), 38 deletions(-) diff --git a/beacon_node/execution_layer/src/lib.rs b/beacon_node/execution_layer/src/lib.rs index 1983db57eba..d361b6840fc 100644 --- a/beacon_node/execution_layer/src/lib.rs +++ b/beacon_node/execution_layer/src/lib.rs @@ -39,7 +39,7 @@ use std::time::{Duration, Instant, SystemTime, UNIX_EPOCH}; use strum::AsRefStr; use task_executor::TaskExecutor; use tokio::{ - sync::{Mutex, MutexGuard, RwLock}, + sync::{Mutex, MutexGuard, RwLock, broadcast}, time::sleep, }; use tokio_stream::wrappers::WatchStream; @@ -138,15 +138,15 @@ impl TryFrom> for ProvenancedPayload), + Builder(Arc), NoHeaderFromBuilder, CannotProduceHeader, - EngineError(Box), + EngineError(Arc), NotSynced, ShuttingDown, FeeRecipientUnspecified, @@ -177,7 +177,7 @@ impl From for Error { impl From for Error { fn from(e: ApiError) -> Self { - Error::ApiError(e) + Error::ApiError(Arc::new(e)) } } @@ -186,12 +186,18 @@ impl From for Error { match e { // This removes an unnecessary layer of indirection. // TODO (mark): consider refactoring these error enums - EngineError::Api { error } => Error::ApiError(error), - _ => Error::EngineError(Box::new(e)), + EngineError::Api { error } => Error::ApiError(Arc::new(error)), + _ => Error::EngineError(Arc::new(e)), } } } +impl From for Error { + fn from(e: builder_client::Error) -> Self { + Error::Builder(Arc::new(e)) + } +} + pub enum BlockProposalContentsType { Full(BlockProposalContents>), Blinded(BlockProposalContents>), @@ -418,6 +424,108 @@ pub enum SubmitBlindedBlockResponse { type PayloadContentsRefTuple<'a, E> = (ExecutionPayloadRef<'a, E>, Option<&'a BlobsBundle>); +/// Cache for deduplicating new payload requests. +/// +/// Handles both in-flight requests and recently completed requests to avoid +/// duplicate network calls to the execution engine. +struct NewPayloadCache { + inner: Mutex, +} + +struct NewPayloadCacheInner { + /// In-flight requests mapped by block hash + in_flight: HashMap>>, + /// Recently completed requests with their completion time + completed: LruCache)>, +} + +impl NewPayloadCache { + /// Cache TTL for completed requests (12 seconds) + const COMPLETED_TTL: Duration = Duration::from_secs(12); + /// Maximum number of completed requests to cache + const COMPLETED_CACHE_SIZE: NonZeroUsize = new_non_zero_usize(32); + + fn new() -> Self { + Self { + inner: Mutex::new(NewPayloadCacheInner { + in_flight: HashMap::new(), + completed: LruCache::new(Self::COMPLETED_CACHE_SIZE), + }), + } + } + + /// Get cached result or execute the provided function. + /// + /// Returns a future that resolves to the payload status. Handles: + /// 1. Returning cached completed results (if not expired) + /// 2. Joining in-flight requests + /// 3. Executing new requests and caching results + async fn get_or_execute( + &self, + block_hash: ExecutionBlockHash, + execute_fn: F, + ) -> Result + where + F: FnOnce() -> Fut, + Fut: Future>, + { + let now = Instant::now(); + + // Single lock acquisition to handle all cases + let mut cache = self.inner.lock().await; + + // 1. Check completed cache first + if let Some((timestamp, result)) = cache.completed.get(&block_hash) { + if now.duration_since(*timestamp) < Self::COMPLETED_TTL { + return result.clone(); + } else { + // Entry expired, remove it + cache.completed.pop(&block_hash); + } + } + + // 2. Check in-flight requests + if let Some(sender) = cache.in_flight.get(&block_hash) { + let mut receiver = sender.subscribe(); + drop(cache); // Release lock early + + match receiver.recv().await { + Ok(result) => return result, + Err(_) => { + // Sender was dropped, fall through to execute new request + error!( + "NewPayloadCache: Sender was dropped for block hash {}. This shouldn't happen.", + block_hash + ); + // just call the execute_fn again + return execute_fn().await; + } + } + } + + // 3. Start new request + let (sender, _receiver) = broadcast::channel(1); + cache.in_flight.insert(block_hash, sender.clone()); + drop(cache); // Release lock for execution + + // Execute the function + let result = execute_fn().await; + + // Update cache with result + let mut cache = self.inner.lock().await; + cache.in_flight.remove(&block_hash); + cache + .completed + .put(block_hash, (Instant::now(), result.clone())); + drop(cache); + + // Broadcast result to any waiting receivers + let _ = sender.send(result.clone()); + + result + } +} + struct Inner { engine: Arc, builder: ArcSwapOption, @@ -433,6 +541,10 @@ struct Inner { /// This is used *only* in the informational sync status endpoint, so that a VC using this /// node can prefer another node with a healthier EL. last_new_payload_errored: RwLock, + /// Cache for deduplicating `notify_new_payload` requests. + /// + /// Handles both in-flight requests and recently completed requests. + new_payload_cache: NewPayloadCache, } #[derive(Debug, Default, Clone, Serialize, Deserialize)] @@ -523,8 +635,8 @@ impl ExecutionLayer { let engine: Engine = { let auth = Auth::new(jwt_key, jwt_id, jwt_version); debug!(endpoint = %execution_url, jwt_path = ?secret_file.as_path(),"Loaded execution endpoint"); - let api = HttpJsonRpc::new_with_auth(execution_url, auth, execution_timeout_multiplier) - .map_err(Error::ApiError)?; + let api = + HttpJsonRpc::new_with_auth(execution_url, auth, execution_timeout_multiplier)?; Engine::new(api, executor.clone()) }; @@ -539,6 +651,7 @@ impl ExecutionLayer { executor, payload_cache: PayloadCache::default(), last_new_payload_errored: RwLock::new(false), + new_payload_cache: NewPayloadCache::new(), }; let el = Self { @@ -582,7 +695,7 @@ impl ExecutionLayer { builder_header_timeout, disable_ssz, ) - .map_err(Error::Builder)?; + .map_err(Into::::into)?; info!( ?builder_url, local_user_agent = builder_client.get_user_agent(), @@ -1349,15 +1462,34 @@ impl ExecutionLayer { Ok(GetPayloadResponseType::Full(payload_response)) }) .await - .map_err(Box::new) - .map_err(Error::EngineError) + .map_err(Into::into) } /// Maps to the `engine_newPayload` JSON-RPC call. + /// + /// Deduplicates concurrent requests with the same block hash - if multiple threads + /// call this function with the same block hash simultaneously, only one request + /// is sent to the execution engine, but all threads receive the same response. + /// Also caches recent results for a short time to avoid duplicate requests. /// TODO(EIP-7732) figure out how and why Mark relaxed new_payload_request param's typ to NewPayloadRequest pub async fn notify_new_payload( &self, new_payload_request: NewPayloadRequest<'_, E>, + ) -> Result { + let block_hash = new_payload_request.block_hash(); + + self.inner + .new_payload_cache + .get_or_execute(block_hash, || { + self.notify_new_payload_impl(new_payload_request) + }) + .await + } + + /// Internal implementation of notify_new_payload without deduplication logic. + async fn notify_new_payload_impl( + &self, + new_payload_request: NewPayloadRequest<'_, E>, ) -> Result { let _timer = metrics::start_timer_vec( &metrics::EXECUTION_LAYER_REQUEST_TIMES, @@ -1391,9 +1523,7 @@ impl ExecutionLayer { } *self.inner.last_new_payload_errored.write().await = result.is_err(); - process_payload_status(block_hash, result) - .map_err(Box::new) - .map_err(Error::EngineError) + process_payload_status(block_hash, result).map_err(Into::into) } /// Update engine sync status. @@ -1529,8 +1659,7 @@ impl ExecutionLayer { head_block_hash, result.map(|response| response.payload_status), ) - .map_err(Box::new) - .map_err(Error::EngineError) + .map_err(Into::into) } /// Returns the execution engine capabilities resulting from a call to @@ -1622,9 +1751,7 @@ impl ExecutionLayer { } Ok(block.map(|b| b.block_hash)) }) - .await - .map_err(Box::new) - .map_err(Error::EngineError)?; + .await?; if let Some(hash) = &hash_opt { info!( @@ -1734,8 +1861,7 @@ impl ExecutionLayer { Ok(None) }) .await - .map_err(Box::new) - .map_err(Error::EngineError) + .map_err(Into::into) } /// This function should remain internal. @@ -1786,8 +1912,7 @@ impl ExecutionLayer { engine.api.get_payload_bodies_by_hash_v1(hashes).await }) .await - .map_err(Box::new) - .map_err(Error::EngineError) + .map_err(Into::into) } pub async fn get_payload_bodies_by_range( @@ -1804,8 +1929,7 @@ impl ExecutionLayer { .await }) .await - .map_err(Box::new) - .map_err(Error::EngineError) + .map_err(Into::into) } /// Fetch a full payload from the execution node. @@ -1867,8 +1991,7 @@ impl ExecutionLayer { self.engine() .request(|engine| async move { engine.api.get_blobs_v1(query).await }) .await - .map_err(Box::new) - .map_err(Error::EngineError) + .map_err(Into::into) } else { Err(Error::GetBlobsNotSupported) } @@ -1884,8 +2007,7 @@ impl ExecutionLayer { self.engine() .request(|engine| async move { engine.api.get_blobs_v2(query).await }) .await - .map_err(Box::new) - .map_err(Error::EngineError) + .map_err(Into::into) } else { Err(Error::GetBlobsNotSupported) } @@ -1898,8 +2020,7 @@ impl ExecutionLayer { self.engine() .request(|engine| async move { engine.api.get_block_by_number(query).await }) .await - .map_err(Box::new) - .map_err(Error::EngineError) + .map_err(Into::into) } pub async fn propose_blinded_beacon_block( @@ -1948,12 +2069,12 @@ impl ExecutionLayer { builder .post_builder_blinded_blocks_v1_ssz(block) .await - .map_err(Error::Builder) + .map_err(Into::into) } else { builder .post_builder_blinded_blocks_v1(block) .await - .map_err(Error::Builder) + .map_err(Into::into) .map(|d| d.data) } }) @@ -2018,12 +2139,12 @@ impl ExecutionLayer { builder .post_builder_blinded_blocks_v2_ssz(block) .await - .map_err(Error::Builder) + .map_err(Into::into) } else { builder .post_builder_blinded_blocks_v2(block) .await - .map_err(Error::Builder) + .map_err(Into::into) } }) .await; diff --git a/beacon_node/execution_layer/src/versioned_hashes.rs b/beacon_node/execution_layer/src/versioned_hashes.rs index 97c3100de99..b895db6388c 100644 --- a/beacon_node/execution_layer/src/versioned_hashes.rs +++ b/beacon_node/execution_layer/src/versioned_hashes.rs @@ -2,7 +2,7 @@ use alloy_consensus::TxEnvelope; use alloy_rlp::Decodable; use types::{EthSpec, ExecutionPayloadRef, Hash256, Unsigned, VersionedHash}; -#[derive(Debug)] +#[derive(Debug, Clone)] pub enum Error { DecodingTransaction(String), LengthMismatch { expected: usize, found: usize }, From ccb519b71c7e410fb21249a28f5950f23c2d9f7a Mon Sep 17 00:00:00 2001 From: Mark Mackey Date: Mon, 20 Oct 2025 15:45:01 -0500 Subject: [PATCH 3/9] Moar Progress --- beacon_node/beacon_chain/src/beacon_chain.rs | 17 ++ .../beacon_chain/src/block_verification.rs | 3 +- .../beacon_chain/src/envelope_verification.rs | 269 +++++++++--------- .../src/envelope_verification_types.rs | 140 ++------- beacon_node/beacon_chain/src/errors.rs | 1 + .../beacon_chain/src/execution_payload.rs | 12 +- beacon_node/beacon_chain/src/lib.rs | 1 + .../src/envelope_processing.rs | 12 +- .../per_block_processing/signature_sets.rs | 5 +- 9 files changed, 200 insertions(+), 260 deletions(-) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 5a82dc70c5f..c985f937a5d 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -1135,6 +1135,23 @@ impl BeaconChain { } } + /// Returns the full block at the given root, if it's available in the database. + /// + /// Should always return a full block for pre-merge and post-gloas blocks. + /// An + pub fn get_full_block( + &self, + block_root: &Hash256, + ) -> Result>, Error> { + match self.store.try_get_full_block(block_root)? { + Some(DatabaseBlock::Full(block)) => Ok(Some(block)), + Some(DatabaseBlock::Blinded(_)) => { + Err(Error::ExecutionPayloadMissingFromDatabase(*block_root)) + } + None => Ok(None), + } + } + /// Returns the block at the given root, if any. /// /// ## Errors diff --git a/beacon_node/beacon_chain/src/block_verification.rs b/beacon_node/beacon_chain/src/block_verification.rs index 691293b2000..e4f523c8dda 100644 --- a/beacon_node/beacon_chain/src/block_verification.rs +++ b/beacon_node/beacon_chain/src/block_verification.rs @@ -708,7 +708,8 @@ pub struct SignatureVerifiedBlock { } /// Used to await the result of executing payload with an EE. -type PayloadVerificationHandle = JoinHandle>>; +pub type PayloadVerificationHandle = + JoinHandle>>; /// A wrapper around a `SignedBeaconBlock` that indicates that this block is fully verified and /// ready to import into the `BeaconChain`. The validation includes: diff --git a/beacon_node/beacon_chain/src/envelope_verification.rs b/beacon_node/beacon_chain/src/envelope_verification.rs index 6f927c63874..de40bac6020 100644 --- a/beacon_node/beacon_chain/src/envelope_verification.rs +++ b/beacon_node/beacon_chain/src/envelope_verification.rs @@ -4,48 +4,50 @@ //! diagram below). //! //! ```ignore -//! START -//! | -//! ▼ -//! SignedExecutionPayloadEnvelope -//! | -//! |--------------- -//! | | -//! | ▼ -//! | GossipVerifiedEnvelope -//! | | -//! |--------------- -//! | -//! ▼ -//! ExecutionPendingEnvelope -//! | -//! await -//! | -//! ▼ -//! END +//! START +//! | +//! ▼ +//! SignedExecutionPayloadEnvelope +//! | +//! |--------------- +//! | | +//! | ▼ +//! | GossipVerifiedEnvelope +//! | | +//! |--------------- +//! | +//! ▼ +//! SignatureVerifiedEnvelope +//! | +//! ▼ +//! ExecutionPendingEnvelope +//! | +//! await +//! | +//! ▼ +//! END //! //! ``` +use crate::NotifyExecutionLayer; use crate::block_verification::{PayloadVerificationHandle, PayloadVerificationOutcome}; -use crate::data_availability_checker::MaybeAvailableEnvelope; -use crate::envelope_verification_types::EnvelopeImportData; +use crate::envelope_verification_types::{EnvelopeImportData, MaybeAvailableEnvelope}; use crate::execution_payload::PayloadNotifier; -use crate::NotifyExecutionLayer; use crate::{BeaconChain, BeaconChainError, BeaconChainTypes}; use derivative::Derivative; use safe_arith::ArithError; use slot_clock::SlotClock; -use state_processing::envelope_processing::{envelope_processing, EnvelopeProcessingError}; +use state_processing::envelope_processing::{EnvelopeProcessingError, envelope_processing}; use state_processing::per_block_processing::compute_timestamp_at_slot; use state_processing::{BlockProcessingError, VerifySignatures}; use std::sync::Arc; use tree_hash::TreeHash; use types::{ - BeaconState, BeaconStateError, EthSpec, ExecutionBlockHash, Hash256, SignedBlindedBeaconBlock, - SignedExecutionPayloadEnvelope, + BeaconState, BeaconStateError, ExecutionBlockHash, Hash256, SignedBeaconBlock, + SignedExecutionPayloadEnvelope, Slot, }; -// TODO(EIP7732): don't use this redefinition.. +// TODO(gloas): don't use this redefinition.. macro_rules! envelope_verify { ($condition: expr, $result: expr) => { if !$condition { @@ -54,7 +56,7 @@ macro_rules! envelope_verify { }; } -#[derive(Debug)] +#[derive(Debug, Clone)] pub enum EnvelopeError { /// The envelope's block root is unknown. BlockRootUnknown { @@ -117,11 +119,13 @@ pub enum EnvelopeError { state: Hash256, envelope: Hash256, }, - // The payload was withheld but the block hash - // matched the committed bid - PayloadWithheldBlockHashMismatch, + // The slot doesn't match the parent block + SlotMismatch { + parent_block: Slot, + envelope: Slot, + }, // Some Beacon Chain Error - BeaconChainError(BeaconChainError), + BeaconChainError(Arc), // Some Beacon State error BeaconStateError(BeaconStateError), // Some ArithError @@ -132,7 +136,7 @@ pub enum EnvelopeError { impl From for EnvelopeError { fn from(e: BeaconChainError) -> Self { - EnvelopeError::BeaconChainError(e) + EnvelopeError::BeaconChainError(Arc::new(e)) } } @@ -166,7 +170,7 @@ impl From for EnvelopeError { #[derivative(Debug(bound = "T: BeaconChainTypes"))] pub struct GossipVerifiedEnvelope { pub signed_envelope: Arc>, - pub parent_block: Arc>, + pub parent_block: Arc>, pub pre_state: Box>, } @@ -179,7 +183,7 @@ impl GossipVerifiedEnvelope { let payload = envelope.payload(); let block_root = envelope.beacon_block_root(); - // TODO(EIP7732): this check would fail if the block didn't pass validation right? + // TODO(gloas): this check would fail if the block didn't pass validation right? // check that we've seen the parent block of this envelope let fork_choice_read_lock = chain.canonical_head.fork_choice_read_lock(); @@ -189,7 +193,7 @@ impl GossipVerifiedEnvelope { drop(fork_choice_read_lock); let parent_block = chain - .get_blinded_block(&block_root)? + .get_full_block(&block_root)? .ok_or_else(|| EnvelopeError::from(BeaconChainError::MissingBeaconBlock(block_root))) .map(Arc::new)?; let execution_bid = &parent_block @@ -198,10 +202,17 @@ impl GossipVerifiedEnvelope { .signed_execution_bid()? .message; - // TODO(EIP7732): check we're within the bounds of the slot (probably) + // TODO(gloas): check we're within the bounds of the slot (probably) + // I think a timestamp check like this is on the beacon block but need to check. + if envelope.slot() != parent_block.slot() { + return Err(EnvelopeError::SlotMismatch { + parent_block: parent_block.slot(), + envelope: envelope.slot(), + }); + } - // TODO(EIP7732): check that we haven't seen another valid `SignedExecutionPayloadEnvelope` - // for this block root from this builder + // TODO(gloas): check that we haven't seen another valid `SignedExecutionPayloadEnvelope` + // for this block root from this builder - envelope status table check // builder index matches committed bid if envelope.builder_index() != execution_bid.builder_index { @@ -211,15 +222,21 @@ impl GossipVerifiedEnvelope { }); } - // if payload is withheld, the block hash should not match the committed bid - if !envelope.payload_withheld() && payload.block_hash() == execution_bid.block_hash { - return Err(EnvelopeError::PayloadWithheldBlockHashMismatch); + // the block hash should match the block hash of the execution bid + if payload.block_hash() != execution_bid.block_hash { + return Err(EnvelopeError::BlockHashMismatch { + committed_bid: execution_bid.block_hash, + envelope: payload.block_hash(), + }); } + // TODO(gloas): expensive load here.. check this let parent_state = chain + // TODO(gloas): may need a get_block_state to get the right state here.. .get_state( &parent_block.message().state_root(), Some(parent_block.slot()), + true, )? .ok_or_else(|| { EnvelopeError::from(BeaconChainError::MissingBeaconState( @@ -271,15 +288,24 @@ impl IntoExecutionPendingEnvelope for GossipVerifiedEnve // verify signature already done let mut state = *self.pre_state; - // setting state.latest_block_header happens in envelope_processing + // all state modifications are done in envelope_processing (called at the bottom of this function) + // so here perform the consistency check with the beacon block on a copy of the latest block header + // and let it be modified later in envelope_processing + let previous_state_root = state.canonical_root()?; + if state.latest_block_header().state_root == Hash256::default() { + let mut copy_of_latest_block_header = state.latest_block_header().clone(); + copy_of_latest_block_header.state_root = previous_state_root; + + // Verify consistency with the beacon block + if !envelope.beacon_block_root() == copy_of_latest_block_header.tree_hash_root() { + return Err(EnvelopeError::LatestBlockHeaderMismatch { + envelope_root: envelope.beacon_block_root(), + block_header_root: copy_of_latest_block_header.tree_hash_root(), + }); + }; + } - // Verify consistency with the beacon block - if !envelope.tree_hash_root() == state.latest_block_header().tree_hash_root() { - return Err(EnvelopeError::LatestBlockHeaderMismatch { - envelope_root: envelope.tree_hash_root(), - block_header_root: state.latest_block_header().tree_hash_root(), - }); - }; + // the check about the slots matching is already done in the GossipVerifiedEnvelope // Verify consistency with the committed bid let committed_bid = state.latest_execution_bid()?; @@ -293,79 +319,68 @@ impl IntoExecutionPendingEnvelope for GossipVerifiedEnve }); }; - if !envelope.payload_withheld() { - // Verify the withdrawals root - envelope_verify!( - payload.withdrawals()?.tree_hash_root() == state.latest_withdrawals_root()?, - EnvelopeError::WithdrawalsRootMismatch { - state: state.latest_withdrawals_root()?, - envelope: payload.withdrawals()?.tree_hash_root(), - } - .into() - ); - - // Verify the gas limit - envelope_verify!( - payload.gas_limit() == committed_bid.gas_limit, - EnvelopeError::GasLimitMismatch { - committed_bid: committed_bid.gas_limit, - envelope: payload.gas_limit(), - } - .into() - ); - // Verify the block hash - envelope_verify!( - committed_bid.block_hash == payload.block_hash(), - EnvelopeError::BlockHashMismatch { - committed_bid: committed_bid.block_hash, - envelope: payload.block_hash(), - } - .into() - ); - - // Verify consistency of the parent hash with respect to the previous execution payload - envelope_verify!( - payload.parent_hash() == state.latest_block_hash()?, - EnvelopeError::ParentHashMismatch { - state: state.latest_block_hash()?, - envelope: payload.parent_hash(), - } - .into() - ); - - // Verify prev_randao - envelope_verify!( - payload.prev_randao() == *state.get_randao_mix(state.current_epoch())?, - EnvelopeError::PrevRandaoMismatch { - state: *state.get_randao_mix(state.current_epoch())?, - envelope: payload.prev_randao(), - } - .into() - ); - - // Verify the timestamp - let state_timestamp = - compute_timestamp_at_slot(&state, state.slot(), chain.spec.as_ref())?; - envelope_verify!( - payload.timestamp() == state_timestamp, - EnvelopeError::TimestampMismatch { - state: state_timestamp, - envelope: payload.timestamp(), - } - .into() - ); - - // Verify the commitments are under limit - envelope_verify!( - envelope.blob_kzg_commitments().len() - <= T::EthSpec::max_blob_commitments_per_block(), - EnvelopeError::BlobLimitExceeded { - max: T::EthSpec::max_blob_commitments_per_block(), - envelope: envelope.blob_kzg_commitments().len(), - } - .into() - ); - } + // Verify the withdrawals root + envelope_verify!( + payload.withdrawals()?.tree_hash_root() == *state.latest_withdrawals_root()?, + EnvelopeError::WithdrawalsRootMismatch { + state: *state.latest_withdrawals_root()?, + envelope: payload.withdrawals()?.tree_hash_root(), + } + .into() + ); + + // Verify the gas limit + envelope_verify!( + payload.gas_limit() == committed_bid.gas_limit, + EnvelopeError::GasLimitMismatch { + committed_bid: committed_bid.gas_limit, + envelope: payload.gas_limit(), + } + .into() + ); + // Verify the block hash already done in the GossipVerifiedEnvelope + + // Verify consistency of the parent hash with respect to the previous execution payload + envelope_verify!( + payload.parent_hash() == *state.latest_block_hash()?, + EnvelopeError::ParentHashMismatch { + state: *state.latest_block_hash()?, + envelope: payload.parent_hash(), + } + .into() + ); + + // Verify prev_randao + envelope_verify!( + payload.prev_randao() == *state.get_randao_mix(state.current_epoch())?, + EnvelopeError::PrevRandaoMismatch { + state: *state.get_randao_mix(state.current_epoch())?, + envelope: payload.prev_randao(), + } + .into() + ); + + // Verify the timestamp + let state_timestamp = compute_timestamp_at_slot(&state, state.slot(), chain.spec.as_ref())?; + envelope_verify!( + payload.timestamp() == state_timestamp, + EnvelopeError::TimestampMismatch { + state: state_timestamp, + envelope: payload.timestamp(), + } + .into() + ); + + // Verify the commitments are under limit + let max_blobs = chain.spec.max_blobs_per_block(state.current_epoch()) as usize; + envelope_verify!( + envelope.blob_kzg_commitments().len() <= max_blobs, + EnvelopeError::BlobLimitExceeded { + max: max_blobs, + envelope: envelope.blob_kzg_commitments().len(), + } + .into() + ); // Verify the execution payload is valid let payload_notifier = @@ -375,7 +390,7 @@ impl IntoExecutionPendingEnvelope for GossipVerifiedEnve let payload_verification_future = async move { let chain = payload_notifier.chain.clone(); - // TODO:(EIP7732): timing + // TODO:(gloas): timing if let Some(started_execution) = chain.slot_clock.now_duration() { chain.block_times_cache.write().set_time_started_execution( block_root, @@ -410,7 +425,7 @@ impl IntoExecutionPendingEnvelope for GossipVerifiedEnve &chain.spec, )?; - // TODO(EIP7732): if verify + // TODO(gloas): if verify envelope_verify!( state.canonical_root()? == envelope.state_root(), EnvelopeError::InvalidStateRoot { @@ -421,7 +436,7 @@ impl IntoExecutionPendingEnvelope for GossipVerifiedEnve Ok(ExecutionPendingEnvelope { signed_envelope: MaybeAvailableEnvelope::AvailabilityPending { - block_root, + block_hash: payload.block_hash(), envelope: signed_envelope, }, import_data: EnvelopeImportData { @@ -442,8 +457,8 @@ impl IntoExecutionPendingEnvelope chain: &Arc>, notify_execution_layer: NotifyExecutionLayer, ) -> Result, EnvelopeError> { - // TODO(EIP7732): figure out how this should be refactored.. + // TODO(gloas): figure out how this should be refactored.. GossipVerifiedEnvelope::new(self, chain)? .into_execution_pending_envelope(chain, notify_execution_layer) } -} \ No newline at end of file +} diff --git a/beacon_node/beacon_chain/src/envelope_verification_types.rs b/beacon_node/beacon_chain/src/envelope_verification_types.rs index ac58c985822..3d639752ff0 100644 --- a/beacon_node/beacon_chain/src/envelope_verification_types.rs +++ b/beacon_node/beacon_chain/src/envelope_verification_types.rs @@ -1,136 +1,30 @@ -use crate::data_availability_checker::{AvailableEnvelope, MaybeAvailableEnvelope}; -use crate::PayloadVerificationOutcome; use std::sync::Arc; use types::{ - BeaconState, BlobIdentifier, EthSpec, Hash256, SignedBlindedBeaconBlock, - SignedExecutionPayloadEnvelope, + BeaconState, ChainSpec, DataColumnSidecarList, EthSpec, ExecutionBlockHash, Hash256, + SignedBeaconBlock, SignedExecutionPayloadEnvelope, }; -/// A block that has completed all pre-deneb block processing checks including verification -/// by an EL client **and** has all requisite blob data to be imported into fork choice. -#[derive(PartialEq)] -pub struct AvailableExecutedEnvelope { - pub envelope: AvailableEnvelope, - pub import_data: EnvelopeImportData, - pub payload_verification_outcome: PayloadVerificationOutcome, -} - -impl AvailableExecutedEnvelope { - pub fn new( - envelope: AvailableEnvelope, - import_data: EnvelopeImportData, - payload_verification_outcome: PayloadVerificationOutcome, - ) -> Self { - Self { - envelope, - import_data, - payload_verification_outcome, - } - } - - pub fn get_all_blob_ids(&self) -> Vec { - let num_blobs_expected = self - .envelope - .envelope() - .message() - .blob_kzg_commitments() - .len(); - let mut blob_ids = Vec::with_capacity(num_blobs_expected); - for i in 0..num_blobs_expected { - blob_ids.push(BlobIdentifier { - block_root: self.import_data.block_root, - index: i as u64, - }); - } - blob_ids - } -} - #[derive(PartialEq)] pub struct EnvelopeImportData { pub block_root: Hash256, - pub parent_block: Arc>, + pub parent_block: Arc>, pub post_state: Box>, } -pub struct AvailabilityPendingExecutedEnvelope { - pub envelope: Arc>, - pub import_data: EnvelopeImportData, - pub payload_verification_outcome: PayloadVerificationOutcome, +#[derive(Debug)] +#[allow(dead_code)] +pub struct AvailableEnvelope { + block_hash: ExecutionBlockHash, + envelope: Arc>, + columns: DataColumnSidecarList, + /// Timestamp at which this block first became available (UNIX timestamp, time since 1970). + columns_available_timestamp: Option, + pub spec: Arc, } - -impl AvailabilityPendingExecutedEnvelope { - pub fn new( +pub enum MaybeAvailableEnvelope { + Available(AvailableEnvelope), + AvailabilityPending { + block_hash: ExecutionBlockHash, envelope: Arc>, - import_data: EnvelopeImportData, - payload_verification_outcome: PayloadVerificationOutcome, - ) -> Self { - Self { - envelope, - import_data, - payload_verification_outcome, - } - } - - pub fn as_envelope(&self) -> &SignedExecutionPayloadEnvelope { - self.envelope.as_ref() - } - - pub fn num_blobs_expected(&self) -> usize { - self.envelope.message().blob_kzg_commitments().len() - } -} - -/// An envelope that has gone through all envelope processing checks including envelope processing -/// and execution by an EL client. This block hasn't necessarily completed data availability checks. -/// -/// -/// It contains 2 variants: -/// 1. `Available`: This envelope has been executed and also contains all data to consider it a -/// fully available envelope. -/// 2. `AvailabilityPending`: This envelope hasn't received all required blobs to consider it a -/// fully available envelope. -pub enum ExecutedEnvelope { - Available(AvailableExecutedEnvelope), - AvailabilityPending(AvailabilityPendingExecutedEnvelope), + }, } - -impl ExecutedEnvelope { - pub fn new( - envelope: MaybeAvailableEnvelope, - import_data: EnvelopeImportData, - payload_verification_outcome: PayloadVerificationOutcome, - ) -> Self { - match envelope { - MaybeAvailableEnvelope::Available(available_envelope) => { - Self::Available(AvailableExecutedEnvelope::new( - available_envelope, - import_data, - payload_verification_outcome, - )) - } - MaybeAvailableEnvelope::AvailabilityPending { - block_root: _, - envelope, - } => Self::AvailabilityPending(AvailabilityPendingExecutedEnvelope::new( - envelope, - import_data, - payload_verification_outcome, - )), - } - } - - pub fn as_envelope(&self) -> &SignedExecutionPayloadEnvelope { - match self { - Self::Available(available) => available.envelope.envelope(), - Self::AvailabilityPending(pending) => pending.envelope.as_ref(), - } - } - - pub fn block_root(&self) -> Hash256 { - match self { - Self::Available(available) => available.import_data.block_root, - Self::AvailabilityPending(pending) => pending.import_data.block_root, - } - } -} \ No newline at end of file diff --git a/beacon_node/beacon_chain/src/errors.rs b/beacon_node/beacon_chain/src/errors.rs index d4eba2b0ea2..9523c1ec6ca 100644 --- a/beacon_node/beacon_chain/src/errors.rs +++ b/beacon_node/beacon_chain/src/errors.rs @@ -148,6 +148,7 @@ pub enum BeaconChainError { EngineGetCapabilititesFailed(Box), ExecutionLayerGetBlockByNumberFailed(Box), ExecutionLayerGetBlockByHashFailed(Box), + ExecutionPayloadMissingFromDatabase(Hash256), BlockHashMissingFromExecutionLayer(ExecutionBlockHash), InconsistentPayloadReconstructed { slot: Slot, diff --git a/beacon_node/beacon_chain/src/execution_payload.rs b/beacon_node/beacon_chain/src/execution_payload.rs index f0cab06ca3d..7cad2104ebc 100644 --- a/beacon_node/beacon_chain/src/execution_payload.rs +++ b/beacon_node/beacon_chain/src/execution_payload.rs @@ -9,7 +9,7 @@ use crate::{ BeaconChain, BeaconChainError, BeaconChainTypes, BlockError, BlockProductionError, - ExecutionPayloadError, + EnvelopeError, ExecutionPayloadError, }; use execution_layer::{ BlockProposalContents, BlockProposalContentsType, BuilderParams, NewPayloadRequest, @@ -108,6 +108,16 @@ impl PayloadNotifier { }) } + pub fn from_envelope( + _chain: Arc>, + _envelope: ExecutionPayloadEnvelopeRef, + _notify_execution_layer: NotifyExecutionLayer, + ) -> Result { + todo!( + "this isn't a real method but something like this will be needed after refactoring this a bit" + ); + } + pub async fn notify_new_payload(self) -> Result { if let Some(precomputed_status) = self.payload_verification_status { Ok(precomputed_status) diff --git a/beacon_node/beacon_chain/src/lib.rs b/beacon_node/beacon_chain/src/lib.rs index 8df88532c22..37e8945a254 100644 --- a/beacon_node/beacon_chain/src/lib.rs +++ b/beacon_node/beacon_chain/src/lib.rs @@ -87,6 +87,7 @@ pub use block_verification_types::AvailabilityPendingExecutedBlock; pub use block_verification_types::ExecutedBlock; pub use canonical_head::{CachedHead, CanonicalHead, CanonicalHeadRwLock}; pub use custody_context::CustodyContext; +pub use envelope_verification::{EnvelopeError, GossipVerifiedEnvelope}; pub use events::ServerSentEventHandler; pub use execution_layer::EngineState; pub use execution_payload::NotifyExecutionLayer; diff --git a/consensus/state_processing/src/envelope_processing.rs b/consensus/state_processing/src/envelope_processing.rs index aafc6bb3e85..01f49c14388 100644 --- a/consensus/state_processing/src/envelope_processing.rs +++ b/consensus/state_processing/src/envelope_processing.rs @@ -1,9 +1,11 @@ +use crate::BlockProcessingError; +use crate::VerifySignatures; use crate::per_block_processing::process_operations::{ process_consolidation_requests, process_deposit_requests, process_withdrawal_requests, }; -use crate::BlockProcessingError; -use crate::VerifySignatures; -use types::{BeaconState, BeaconStateError, ChainSpec, EthSpec, Hash256, SignedExecutionPayloadEnvelope}; +use types::{ + BeaconState, BeaconStateError, ChainSpec, EthSpec, Hash256, SignedExecutionPayloadEnvelope, +}; #[derive(Debug)] pub enum EnvelopeProcessingError { @@ -47,8 +49,6 @@ pub fn envelope_processing( state.latest_block_header_mut().state_root = previous_state_root; } - // Verify consistency with the beacon block - // process electra operations let envelope = signed_envelope.message(); let payload = envelope.payload(); @@ -62,4 +62,4 @@ pub fn envelope_processing( todo!("the rest of process_execution_payload()"); //Ok(()) -} \ No newline at end of file +} diff --git a/consensus/state_processing/src/per_block_processing/signature_sets.rs b/consensus/state_processing/src/per_block_processing/signature_sets.rs index af1b75a00f1..1dc53ce1a89 100644 --- a/consensus/state_processing/src/per_block_processing/signature_sets.rs +++ b/consensus/state_processing/src/per_block_processing/signature_sets.rs @@ -11,8 +11,9 @@ use types::{ BeaconStateError, ChainSpec, DepositData, Domain, Epoch, EthSpec, Fork, Hash256, InconsistentFork, IndexedAttestation, IndexedAttestationRef, ProposerSlashing, PublicKey, PublicKeyBytes, Signature, SignedAggregateAndProof, SignedBeaconBlock, SignedBeaconBlockHeader, - SignedBlsToExecutionChange, SignedContributionAndProof, SignedRoot, SignedVoluntaryExit, - SigningData, Slot, SyncAggregate, SyncAggregatorSelectionData, Unsigned, SignedExecutionPayloadEnvelope, + SignedBlsToExecutionChange, SignedContributionAndProof, SignedExecutionPayloadEnvelope, + SignedRoot, SignedVoluntaryExit, SigningData, Slot, SyncAggregate, SyncAggregatorSelectionData, + Unsigned, }; pub type Result = std::result::Result; From 76b0330b4c4ccf11f5fa49ceb188d68b08daaf6c Mon Sep 17 00:00:00 2001 From: Mark Mackey Date: Wed, 22 Oct 2025 16:01:47 -0500 Subject: [PATCH 4/9] Refactor for Organization --- .../beacon_chain/src/envelope_verification.rs | 405 +++++++++--------- .../src/envelope_processing.rs | 239 ++++++++++- .../types/src/execution_payload_envelope.rs | 4 + .../src/signed_execution_payload_envelope.rs | 42 +- 4 files changed, 469 insertions(+), 221 deletions(-) diff --git a/beacon_node/beacon_chain/src/envelope_verification.rs b/beacon_node/beacon_chain/src/envelope_verification.rs index de40bac6020..a2d0dbcf531 100644 --- a/beacon_node/beacon_chain/src/envelope_verification.rs +++ b/beacon_node/beacon_chain/src/envelope_verification.rs @@ -17,9 +17,6 @@ //! |--------------- //! | //! ▼ -//! SignatureVerifiedEnvelope -//! | -//! ▼ //! ExecutionPendingEnvelope //! | //! await @@ -35,27 +32,16 @@ use crate::envelope_verification_types::{EnvelopeImportData, MaybeAvailableEnvel use crate::execution_payload::PayloadNotifier; use crate::{BeaconChain, BeaconChainError, BeaconChainTypes}; use derivative::Derivative; -use safe_arith::ArithError; use slot_clock::SlotClock; use state_processing::envelope_processing::{EnvelopeProcessingError, envelope_processing}; -use state_processing::per_block_processing::compute_timestamp_at_slot; use state_processing::{BlockProcessingError, VerifySignatures}; use std::sync::Arc; -use tree_hash::TreeHash; +use tracing::{debug, instrument}; use types::{ - BeaconState, BeaconStateError, ExecutionBlockHash, Hash256, SignedBeaconBlock, + BeaconState, BeaconStateError, EthSpec, ExecutionBlockHash, Hash256, SignedBeaconBlock, SignedExecutionPayloadEnvelope, Slot, }; -// TODO(gloas): don't use this redefinition.. -macro_rules! envelope_verify { - ($condition: expr, $result: expr) => { - if !$condition { - return Err($result); - } - }; -} - #[derive(Debug, Clone)] pub enum EnvelopeError { /// The envelope's block root is unknown. @@ -64,74 +50,33 @@ pub enum EnvelopeError { }, /// The signature is invalid. BadSignature, - /// Envelope doesn't match latest beacon block header - LatestBlockHeaderMismatch { - envelope_root: Hash256, - block_header_root: Hash256, - }, /// The builder index doesn't match the committed bid BuilderIndexMismatch { committed_bid: u64, envelope: u64, }, - /// The blob KZG commitments root doesn't match the committed bid - BlobKzgCommitmentsRootMismatch { - committed_bid: Hash256, - envelope: Hash256, - }, - /// The withdrawals root doesn't match the state's latest withdrawals root - WithdrawalsRootMismatch { - state: Hash256, - envelope: Hash256, + // The slot doesn't match the parent block + SlotMismatch { + parent_block: Slot, + envelope: Slot, }, - // The gas limit doesn't match the committed bid - GasLimitMismatch { - committed_bid: u64, - envelope: u64, + // The validator index is unknown + UnknownValidator { + builder_index: u64, }, // The block hash doesn't match the committed bid BlockHashMismatch { committed_bid: ExecutionBlockHash, envelope: ExecutionBlockHash, }, - // The parent hash doesn't match the previous execution payload - ParentHashMismatch { - state: ExecutionBlockHash, - envelope: ExecutionBlockHash, - }, - // The previous randao didn't match the payload - PrevRandaoMismatch { - state: Hash256, - envelope: Hash256, - }, - // The timestamp didn't match the payload - TimestampMismatch { - state: u64, - envelope: u64, - }, - // Blob committments exceeded the maximum - BlobLimitExceeded { - max: usize, - envelope: usize, - }, - // Invalid state root - InvalidStateRoot { - state: Hash256, - envelope: Hash256, - }, - // The slot doesn't match the parent block - SlotMismatch { - parent_block: Slot, - envelope: Slot, - }, // Some Beacon Chain Error BeaconChainError(Arc), // Some Beacon State error BeaconStateError(BeaconStateError), - // Some ArithError - ArithError(ArithError), // Some BlockProcessingError (for electra operations) BlockProcessingError(BlockProcessingError), + // Some EnvelopeProcessingError + EnvelopeProcessingError(EnvelopeProcessingError), } impl From for EnvelopeError { @@ -146,24 +91,120 @@ impl From for EnvelopeError { } } -impl From for EnvelopeError { - fn from(e: ArithError) -> Self { - EnvelopeError::ArithError(e) - } -} - +/// Pull errors up from EnvelopeProcessingError to EnvelopeError impl From for EnvelopeError { fn from(e: EnvelopeProcessingError) -> Self { match e { EnvelopeProcessingError::BadSignature => EnvelopeError::BadSignature, EnvelopeProcessingError::BeaconStateError(e) => EnvelopeError::BeaconStateError(e), + EnvelopeProcessingError::BlockHashMismatch { + committed_bid, + envelope, + } => EnvelopeError::BlockHashMismatch { + committed_bid, + envelope, + }, EnvelopeProcessingError::BlockProcessingError(e) => { EnvelopeError::BlockProcessingError(e) } + e => EnvelopeError::EnvelopeProcessingError(e), } } } +/// This snapshot is to be used for verifying a envelope of the block. +#[derive(Debug, Clone)] +pub struct EnvelopeProcessingSnapshot { + /// This state is equivalent to the `self.beacon_block.state_root()` before applying the envelope. + pub pre_state: BeaconState, + pub state_root: Hash256, + pub beacon_block_root: Hash256, +} + +#[allow(clippy::type_complexity)] +#[instrument(skip_all, level = "debug", fields(beacon_block_root = %envelope.beacon_block_root()))] +fn load_snapshot( + envelope: &SignedExecutionPayloadEnvelope, + chain: &BeaconChain, +) -> Result, EnvelopeError> { + // Reject any block if its parent is not known to fork choice. + // + // A block that is not in fork choice is either: + // + // - Not yet imported: we should reject this block because we should only import a child + // after its parent has been fully imported. + // - Pre-finalized: if the parent block is _prior_ to finalization, we should ignore it + // because it will revert finalization. Note that the finalized block is stored in fork + // choice, so we will not reject any child of the finalized block (this is relevant during + // genesis). + + let beacon_block_root = envelope.beacon_block_root(); + if !chain + .canonical_head + .fork_choice_read_lock() + .contains_block(&beacon_block_root) + { + return Err(EnvelopeError::BlockRootUnknown { + block_root: beacon_block_root, + }); + } + + let fork_choice_read_lock = chain.canonical_head.fork_choice_read_lock(); + let Some(proto_beacon_block) = fork_choice_read_lock.get_block(&beacon_block_root) else { + return Err(EnvelopeError::BlockRootUnknown { + block_root: beacon_block_root, + }); + }; + drop(fork_choice_read_lock); + + // TODO(gloas): add metrics here + + let result = { + // Load the parent block's state from the database, returning an error if it is not found. + // It is an error because if we know the parent block we should also know the parent state. + // Retrieve any state that is advanced through to at most `block.slot()`: this is + // particularly important if `block` descends from the finalized/split block, but at a slot + // prior to the finalized slot (which is invalid and inaccessible in our DB schema). + let (parent_state_root, state) = chain + .store + // TODO(gloas): the state doesn't need to be advanced here because we're applying an envelope + // but this function does use a lot of caches that could be more efficient. Is there + // a better way to do this? + .get_advanced_hot_state( + beacon_block_root, + proto_beacon_block.slot, + proto_beacon_block.state_root, + ) + .map_err(|e| EnvelopeError::BeaconChainError(Arc::new(e.into())))? + .ok_or_else(|| { + BeaconChainError::DBInconsistent(format!( + "Missing state for parent block {beacon_block_root:?}", + )) + })?; + + if state.slot() == proto_beacon_block.slot { + // Sanity check. + if parent_state_root != proto_beacon_block.state_root { + return Err(BeaconChainError::DBInconsistent(format!( + "Parent state at slot {} has the wrong state root: {:?} != {:?}", + state.slot(), + parent_state_root, + proto_beacon_block.state_root, + )) + .into()); + } + } + + Ok(EnvelopeProcessingSnapshot { + pre_state: state, + state_root: parent_state_root, + beacon_block_root, + }) + }; + + result +} + /// A wrapper around a `SignedExecutionPayloadEnvelope` that indicates it has been approved for re-gossiping on /// the p2p network. #[derive(Derivative)] @@ -171,7 +212,7 @@ impl From for EnvelopeError { pub struct GossipVerifiedEnvelope { pub signed_envelope: Arc>, pub parent_block: Arc>, - pub pre_state: Box>, + pub parent: Option>>, } impl GossipVerifiedEnvelope { @@ -181,29 +222,43 @@ impl GossipVerifiedEnvelope { ) -> Result { let envelope = signed_envelope.message(); let payload = envelope.payload(); - let block_root = envelope.beacon_block_root(); + let beacon_block_root = envelope.beacon_block_root(); + // check that we've seen the parent block of this envelope and that it passes validation // TODO(gloas): this check would fail if the block didn't pass validation right? - - // check that we've seen the parent block of this envelope let fork_choice_read_lock = chain.canonical_head.fork_choice_read_lock(); - if !fork_choice_read_lock.contains_block(&block_root) { - return Err(EnvelopeError::BlockRootUnknown { block_root }); - } + let Some(parent_proto_block) = fork_choice_read_lock.get_block(&beacon_block_root) else { + return Err(EnvelopeError::BlockRootUnknown { + block_root: beacon_block_root, + }); + }; drop(fork_choice_read_lock); + // TODO(gloas): check that we haven't seen another valid `SignedExecutionPayloadEnvelope` + // for this block root from this builder - envelope status table check + + // TODO(gloas): this should probably be obtained from the ProtoBlock instead of the DB + // but this means the ProtoBlock needs to include something like the ExecutionBid + // will need to answer this question later. let parent_block = chain - .get_full_block(&block_root)? - .ok_or_else(|| EnvelopeError::from(BeaconChainError::MissingBeaconBlock(block_root))) + .get_full_block(&beacon_block_root)? + .ok_or_else(|| { + EnvelopeError::from(BeaconChainError::MissingBeaconBlock(beacon_block_root)) + }) .map(Arc::new)?; let execution_bid = &parent_block .message() .body() - .signed_execution_bid()? + .signed_execution_payload_bid()? .message; - // TODO(gloas): check we're within the bounds of the slot (probably) - // I think a timestamp check like this is on the beacon block but need to check. + // TODO(gloas): Gossip rules for the beacon block contain the following: + // https://github.com/ethereum/consensus-specs/blob/master/specs/phase0/p2p-interface.md#beacon_block + // [IGNORE] The block is not from a future slot (with a MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance) + // [IGNORE] The block is from a slot greater than the latest finalized slot + // should these kinds of checks be included for envelopes as well? + + // check that the slot of the envelope matches the slot of the parent block if envelope.slot() != parent_block.slot() { return Err(EnvelopeError::SlotMismatch { parent_block: parent_block.slot(), @@ -211,9 +266,6 @@ impl GossipVerifiedEnvelope { }); } - // TODO(gloas): check that we haven't seen another valid `SignedExecutionPayloadEnvelope` - // for this block root from this builder - envelope status table check - // builder index matches committed bid if envelope.builder_index() != execution_bid.builder_index { return Err(EnvelopeError::BuilderIndexMismatch { @@ -230,29 +282,59 @@ impl GossipVerifiedEnvelope { }); } - // TODO(gloas): expensive load here.. check this - let parent_state = chain - // TODO(gloas): may need a get_block_state to get the right state here.. - .get_state( - &parent_block.message().state_root(), - Some(parent_block.slot()), - true, - )? - .ok_or_else(|| { - EnvelopeError::from(BeaconChainError::MissingBeaconState( - parent_block.message().state_root(), - )) - })?; + // TODO(gloas): check these assumptions.. exactly what the most efficient way to verify the signatures + // in this case isn't clear. There are questions about the proposer cache, the pubkey cache, + // and so on. + + // get the fork from the cache so we can verify the signature + let block_slot = envelope.slot(); + let block_epoch = block_slot.epoch(T::EthSpec::slots_per_epoch()); + let proposer_shuffling_decision_block = + parent_proto_block.proposer_shuffling_root_for_child_block(block_epoch, &chain.spec); + let mut opt_parent = None; + let envelope_ref = signed_envelope.as_ref(); + let proposer = chain.with_proposer_cache::<_, EnvelopeError>( + proposer_shuffling_decision_block, + block_epoch, + |proposers| proposers.get_slot::(block_slot), + || { + debug!( + %beacon_block_root, + block_hash = %envelope_ref.block_hash(), + "Proposer shuffling cache miss for envelope verification" + ); + // The proposer index was *not* cached and we must load the parent in order to + // determine the proposer index. + let snapshot = load_snapshot(envelope_ref, chain)?; + opt_parent = Some(Box::new(snapshot.clone())); + Ok((snapshot.state_root, snapshot.pre_state)) + }, + )?; + let fork = proposer.fork; + + let signature_is_valid = { + let pubkey_cache = chain.validator_pubkey_cache.read(); + let builder_pubkey = pubkey_cache + .get(envelope.builder_index() as usize) + .ok_or_else(|| EnvelopeError::UnknownValidator { + builder_index: envelope.builder_index(), + })?; + signed_envelope.verify_signature( + &builder_pubkey, + &fork, + chain.genesis_validators_root, + &chain.spec, + ) + }; - // verify the signature - if !signed_envelope.verify_signature(&parent_state, &chain.spec)? { + if !signature_is_valid { return Err(EnvelopeError::BadSignature); } Ok(Self { signed_envelope, parent_block, - pre_state: Box::new(parent_state), + parent: opt_parent, }) } @@ -285,103 +367,6 @@ impl IntoExecutionPendingEnvelope for GossipVerifiedEnve let envelope = signed_envelope.message(); let payload = &envelope.payload(); - // verify signature already done - let mut state = *self.pre_state; - - // all state modifications are done in envelope_processing (called at the bottom of this function) - // so here perform the consistency check with the beacon block on a copy of the latest block header - // and let it be modified later in envelope_processing - let previous_state_root = state.canonical_root()?; - if state.latest_block_header().state_root == Hash256::default() { - let mut copy_of_latest_block_header = state.latest_block_header().clone(); - copy_of_latest_block_header.state_root = previous_state_root; - - // Verify consistency with the beacon block - if !envelope.beacon_block_root() == copy_of_latest_block_header.tree_hash_root() { - return Err(EnvelopeError::LatestBlockHeaderMismatch { - envelope_root: envelope.beacon_block_root(), - block_header_root: copy_of_latest_block_header.tree_hash_root(), - }); - }; - } - - // the check about the slots matching is already done in the GossipVerifiedEnvelope - - // Verify consistency with the committed bid - let committed_bid = state.latest_execution_bid()?; - // builder index match already verified - if committed_bid.blob_kzg_commitments_root - != envelope.blob_kzg_commitments().tree_hash_root() - { - return Err(EnvelopeError::BlobKzgCommitmentsRootMismatch { - committed_bid: committed_bid.blob_kzg_commitments_root, - envelope: envelope.blob_kzg_commitments().tree_hash_root(), - }); - }; - - // Verify the withdrawals root - envelope_verify!( - payload.withdrawals()?.tree_hash_root() == *state.latest_withdrawals_root()?, - EnvelopeError::WithdrawalsRootMismatch { - state: *state.latest_withdrawals_root()?, - envelope: payload.withdrawals()?.tree_hash_root(), - } - .into() - ); - - // Verify the gas limit - envelope_verify!( - payload.gas_limit() == committed_bid.gas_limit, - EnvelopeError::GasLimitMismatch { - committed_bid: committed_bid.gas_limit, - envelope: payload.gas_limit(), - } - .into() - ); - // Verify the block hash already done in the GossipVerifiedEnvelope - - // Verify consistency of the parent hash with respect to the previous execution payload - envelope_verify!( - payload.parent_hash() == *state.latest_block_hash()?, - EnvelopeError::ParentHashMismatch { - state: *state.latest_block_hash()?, - envelope: payload.parent_hash(), - } - .into() - ); - - // Verify prev_randao - envelope_verify!( - payload.prev_randao() == *state.get_randao_mix(state.current_epoch())?, - EnvelopeError::PrevRandaoMismatch { - state: *state.get_randao_mix(state.current_epoch())?, - envelope: payload.prev_randao(), - } - .into() - ); - - // Verify the timestamp - let state_timestamp = compute_timestamp_at_slot(&state, state.slot(), chain.spec.as_ref())?; - envelope_verify!( - payload.timestamp() == state_timestamp, - EnvelopeError::TimestampMismatch { - state: state_timestamp, - envelope: payload.timestamp(), - } - .into() - ); - - // Verify the commitments are under limit - let max_blobs = chain.spec.max_blobs_per_block(state.current_epoch()) as usize; - envelope_verify!( - envelope.blob_kzg_commitments().len() <= max_blobs, - EnvelopeError::BlobLimitExceeded { - max: max_blobs, - envelope: envelope.blob_kzg_commitments().len(), - } - .into() - ); - // Verify the execution payload is valid let payload_notifier = PayloadNotifier::from_envelope(chain.clone(), envelope, notify_execution_layer)?; @@ -390,7 +375,7 @@ impl IntoExecutionPendingEnvelope for GossipVerifiedEnve let payload_verification_future = async move { let chain = payload_notifier.chain.clone(); - // TODO:(gloas): timing + // TODO:(gloas): timing metrics if let Some(started_execution) = chain.slot_clock.now_duration() { chain.block_times_cache.write().set_time_started_execution( block_root, @@ -417,23 +402,23 @@ impl IntoExecutionPendingEnvelope for GossipVerifiedEnve ) .ok_or(BeaconChainError::RuntimeShutdown)?; + let parent = if let Some(snapshot) = self.parent { + *snapshot + } else { + load_snapshot(signed_envelope.as_ref(), chain)? + }; + let mut state = parent.pre_state; + // All the state modifications are done in envelope_processing envelope_processing( &mut state, + Some(parent.state_root), &signed_envelope, + // verify signature already done for GossipVerifiedEnvelope VerifySignatures::False, &chain.spec, )?; - // TODO(gloas): if verify - envelope_verify!( - state.canonical_root()? == envelope.state_root(), - EnvelopeError::InvalidStateRoot { - state: state.canonical_root()?, - envelope: envelope.state_root(), - } - ); - Ok(ExecutionPendingEnvelope { signed_envelope: MaybeAvailableEnvelope::AvailabilityPending { block_hash: payload.block_hash(), diff --git a/consensus/state_processing/src/envelope_processing.rs b/consensus/state_processing/src/envelope_processing.rs index 01f49c14388..77d97c1f0e0 100644 --- a/consensus/state_processing/src/envelope_processing.rs +++ b/consensus/state_processing/src/envelope_processing.rs @@ -1,18 +1,91 @@ use crate::BlockProcessingError; use crate::VerifySignatures; +use crate::per_block_processing::compute_timestamp_at_slot; use crate::per_block_processing::process_operations::{ process_consolidation_requests, process_deposit_requests, process_withdrawal_requests, }; +use safe_arith::{ArithError, SafeArith}; +use tree_hash::TreeHash; use types::{ - BeaconState, BeaconStateError, ChainSpec, EthSpec, Hash256, SignedExecutionPayloadEnvelope, + BeaconState, BeaconStateError, BuilderPendingPayment, ChainSpec, EthSpec, ExecutionBlockHash, + Hash256, SignedExecutionPayloadEnvelope, Slot, }; -#[derive(Debug)] +// TODO(gloas): don't use this redefinition.. +macro_rules! envelope_verify { + ($condition: expr, $result: expr) => { + if !$condition { + return Err($result); + } + }; +} + +#[derive(Debug, Clone)] pub enum EnvelopeProcessingError { /// Bad Signature BadSignature, BeaconStateError(BeaconStateError), BlockProcessingError(BlockProcessingError), + ArithError(ArithError), + /// Envelope doesn't match latest beacon block header + LatestBlockHeaderMismatch { + envelope_root: Hash256, + block_header_root: Hash256, + }, + /// Envelope doesn't match latest beacon block slot + SlotMismatch { + envelope_slot: Slot, + parent_state_slot: Slot, + }, + /// The withdrawals root doesn't match the state's latest withdrawals root + WithdrawalsRootMismatch { + state: Hash256, + envelope: Hash256, + }, + // The gas limit doesn't match the committed bid + GasLimitMismatch { + committed_bid: u64, + envelope: u64, + }, + // The block hash doesn't match the committed bid + BlockHashMismatch { + committed_bid: ExecutionBlockHash, + envelope: ExecutionBlockHash, + }, + // The parent hash doesn't match the previous execution payload + ParentHashMismatch { + state: ExecutionBlockHash, + envelope: ExecutionBlockHash, + }, + /// The blob KZG commitments root doesn't match the committed bid + BlobKzgCommitmentsRootMismatch { + committed_bid: Hash256, + envelope: Hash256, + }, + // The previous randao didn't match the payload + PrevRandaoMismatch { + state: Hash256, + envelope: Hash256, + }, + // The timestamp didn't match the payload + TimestampMismatch { + state: u64, + envelope: u64, + }, + // Blob committments exceeded the maximum + BlobLimitExceeded { + max: usize, + envelope: usize, + }, + // Invalid state root + InvalidStateRoot { + state: Hash256, + envelope: Hash256, + }, + // BitFieldError + BitFieldError(ssz::BitfieldError), + // Some kind of error calculating the builder payment index + BuilderPaymentIndexOutOfBounds(usize), } impl From for EnvelopeProcessingError { @@ -27,39 +100,185 @@ impl From for EnvelopeProcessingError { } } +impl From for EnvelopeProcessingError { + fn from(e: ArithError) -> Self { + EnvelopeProcessingError::ArithError(e) + } +} + /// Processes a `SignedExecutionPayloadEnvelope` /// /// This function does all the state modifications inside `process_execution_payload()` pub fn envelope_processing( state: &mut BeaconState, + parent_state_root: Option, signed_envelope: &SignedExecutionPayloadEnvelope, verify_signatures: VerifySignatures, spec: &ChainSpec, ) -> Result<(), EnvelopeProcessingError> { if verify_signatures.is_true() { // Verify Signed Envelope Signature - if !signed_envelope.verify_signature(&state, spec)? { + // TODO(gloas): there is probably a more efficient way to do this.. + if !signed_envelope.verify_signature_with_state(&state, spec)? { return Err(EnvelopeProcessingError::BadSignature); } } + let envelope = signed_envelope.message(); + let payload = envelope.payload(); + let execution_requests = envelope.execution_requests(); + // Cache latest block header state root - let previous_state_root = state.canonical_root()?; if state.latest_block_header().state_root == Hash256::default() { + let previous_state_root = parent_state_root + .map(Ok) + .unwrap_or_else(|| state.canonical_root())?; state.latest_block_header_mut().state_root = previous_state_root; } + // Verify consistency with the beacon block + envelope_verify!( + envelope.beacon_block_root() == state.latest_block_header().tree_hash_root(), + EnvelopeProcessingError::LatestBlockHeaderMismatch { + envelope_root: envelope.beacon_block_root(), + block_header_root: state.latest_block_header().tree_hash_root(), + } + ); + envelope_verify!( + envelope.slot() == state.slot(), + EnvelopeProcessingError::SlotMismatch { + envelope_slot: envelope.slot(), + parent_state_slot: state.slot(), + } + ); + + // Verify consistency with the committed bid + let committed_bid = state.latest_execution_payload_bid()?; + // builder index match already verified + if committed_bid.blob_kzg_commitments_root != envelope.blob_kzg_commitments().tree_hash_root() { + return Err(EnvelopeProcessingError::BlobKzgCommitmentsRootMismatch { + committed_bid: committed_bid.blob_kzg_commitments_root, + envelope: envelope.blob_kzg_commitments().tree_hash_root(), + }); + }; + + // Verify the withdrawals root + envelope_verify!( + payload.withdrawals()?.tree_hash_root() == *state.latest_withdrawals_root()?, + EnvelopeProcessingError::WithdrawalsRootMismatch { + state: *state.latest_withdrawals_root()?, + envelope: payload.withdrawals()?.tree_hash_root(), + } + ); + + // Verify the gas limit + envelope_verify!( + payload.gas_limit() == committed_bid.gas_limit, + EnvelopeProcessingError::GasLimitMismatch { + committed_bid: committed_bid.gas_limit, + envelope: payload.gas_limit(), + } + ); + + // Verify the block hash + envelope_verify!( + committed_bid.block_hash == payload.block_hash(), + EnvelopeProcessingError::BlockHashMismatch { + committed_bid: committed_bid.block_hash, + envelope: payload.block_hash(), + } + ); + + // Verify consistency of the parent hash with respect to the previous execution payload + envelope_verify!( + payload.parent_hash() == *state.latest_block_hash()?, + EnvelopeProcessingError::ParentHashMismatch { + state: *state.latest_block_hash()?, + envelope: payload.parent_hash(), + } + ); + + // Verify prev_randao + envelope_verify!( + payload.prev_randao() == *state.get_randao_mix(state.current_epoch())?, + EnvelopeProcessingError::PrevRandaoMismatch { + state: *state.get_randao_mix(state.current_epoch())?, + envelope: payload.prev_randao(), + } + .into() + ); + + // Verify the timestamp + let state_timestamp = compute_timestamp_at_slot(&state, state.slot(), spec)?; + envelope_verify!( + payload.timestamp() == state_timestamp, + EnvelopeProcessingError::TimestampMismatch { + state: state_timestamp, + envelope: payload.timestamp(), + } + .into() + ); + + // Verify the commitments are under limit + let max_blobs = spec.max_blobs_per_block(state.current_epoch()) as usize; + envelope_verify!( + envelope.blob_kzg_commitments().len() <= max_blobs, + EnvelopeProcessingError::BlobLimitExceeded { + max: max_blobs, + envelope: envelope.blob_kzg_commitments().len(), + } + .into() + ); + // process electra operations - let envelope = signed_envelope.message(); - let payload = envelope.payload(); - let execution_requests = envelope.execution_requests(); process_deposit_requests(state, &execution_requests.deposits, spec)?; process_withdrawal_requests(state, &execution_requests.withdrawals, spec)?; process_consolidation_requests(state, &execution_requests.consolidations, spec)?; - // cache the latest block hash and full slot + // queue the builder payment + let payment_index = E::slots_per_epoch() + .safe_add(state.slot().as_u64().safe_rem(E::slots_per_epoch())?)? + as usize; + let mut payment = state + .builder_pending_payments()? + .get(payment_index) + .ok_or_else(|| EnvelopeProcessingError::BuilderPaymentIndexOutOfBounds(payment_index))? + .clone(); + let amount = payment.withdrawal.amount; + if amount > 0 { + let exit_queue_epoch = state.compute_exit_epoch_and_update_churn(amount, spec)?; + payment.withdrawal.withdrawable_epoch = + exit_queue_epoch.safe_add(spec.min_validator_withdrawability_delay)?; + state + .builder_pending_withdrawals_mut()? + .push(payment.withdrawal) + .map_err(|e| EnvelopeProcessingError::BeaconStateError(e.into()))?; + } + *state + .builder_pending_payments_mut()? + .get_mut(payment_index) + .ok_or_else(|| EnvelopeProcessingError::BuilderPaymentIndexOutOfBounds(payment_index))? = + BuilderPendingPayment::default(); + + // cache the execution payload hash + let availability_index = state + .slot() + .safe_rem(E::slots_per_historical_root() as u64)? + .as_usize(); + state + .execution_payload_availability_mut()? + .set(availability_index, true) + .map_err(|e| EnvelopeProcessingError::BitFieldError(e))?; *state.latest_block_hash_mut()? = payload.block_hash(); - todo!("the rest of process_execution_payload()"); - //Ok(()) + // verify the state root + envelope_verify!( + envelope.state_root() == state.canonical_root()?, + EnvelopeProcessingError::InvalidStateRoot { + state: state.canonical_root()?, + envelope: envelope.state_root(), + } + ); + + Ok(()) } diff --git a/consensus/types/src/execution_payload_envelope.rs b/consensus/types/src/execution_payload_envelope.rs index 81539e519a7..422825f2de9 100644 --- a/consensus/types/src/execution_payload_envelope.rs +++ b/consensus/types/src/execution_payload_envelope.rs @@ -69,6 +69,10 @@ impl<'a, E: EthSpec> ExecutionPayloadEnvelopeRef<'a, E> { Self::NextFork(envelope) => ExecutionPayloadRef::Gloas(&envelope.payload), } } + + pub fn block_hash(&self) -> ExecutionBlockHash { + self.payload().block_hash() + } } impl<'de, E: EthSpec> ContextDeserialize<'de, ForkName> for ExecutionPayloadEnvelope { diff --git a/consensus/types/src/signed_execution_payload_envelope.rs b/consensus/types/src/signed_execution_payload_envelope.rs index f8b6e48bcea..9b625287902 100644 --- a/consensus/types/src/signed_execution_payload_envelope.rs +++ b/consensus/types/src/signed_execution_payload_envelope.rs @@ -75,11 +75,28 @@ impl SignedExecutionPayloadEnvelope { } } + pub fn slot(&self) -> Slot { + self.message().slot() + } + + pub fn epoch(&self) -> Epoch { + self.slot().epoch(E::slots_per_epoch()) + } + + pub fn beacon_block_root(&self) -> Hash256 { + self.message().beacon_block_root() + } + + pub fn block_hash(&self) -> ExecutionBlockHash { + self.message().block_hash() + } + /// Verify `self.signature`. /// /// The `parent_state` is the post-state of the beacon block with /// block_root = self.message.beacon_block_root - pub fn verify_signature( + /// todo(gloas): maybe delete this function later + pub fn verify_signature_with_state( &self, parent_state: &BeaconState, spec: &ChainSpec, @@ -104,6 +121,29 @@ impl SignedExecutionPayloadEnvelope { Ok(self.signature().verify(&pubkey, message)) } + + /// Verify `self.signature`. + /// + /// If the root of `block.message` is already known it can be passed in via `object_root_opt`. + /// Otherwise, it will be computed locally. + pub fn verify_signature( + &self, + pubkey: &PublicKey, + fork: &Fork, + genesis_validators_root: Hash256, + spec: &ChainSpec, + ) -> bool { + let domain = spec.get_domain( + self.epoch(), + Domain::BeaconProposer, + fork, + genesis_validators_root, + ); + + let message = self.message().signing_root(domain); + + self.signature().verify(pubkey, message) + } } impl<'de, E: EthSpec> ContextDeserialize<'de, ForkName> for SignedExecutionPayloadEnvelope { From ea95246f8bc8bc00cb6959792ccdd16819caccfe Mon Sep 17 00:00:00 2001 From: Mark Mackey Date: Thu, 23 Oct 2025 09:59:06 -0500 Subject: [PATCH 5/9] rename TODOs --- .../beacon_chain/src/envelope_verification.rs | 30 +++++++++---------- .../src/envelope_processing.rs | 4 +-- .../src/signed_execution_payload_envelope.rs | 2 +- 3 files changed, 18 insertions(+), 18 deletions(-) diff --git a/beacon_node/beacon_chain/src/envelope_verification.rs b/beacon_node/beacon_chain/src/envelope_verification.rs index a2d0dbcf531..8406e1971fd 100644 --- a/beacon_node/beacon_chain/src/envelope_verification.rs +++ b/beacon_node/beacon_chain/src/envelope_verification.rs @@ -157,7 +157,7 @@ fn load_snapshot( }; drop(fork_choice_read_lock); - // TODO(gloas): add metrics here + // TODO(EIP-7732): add metrics here let result = { // Load the parent block's state from the database, returning an error if it is not found. @@ -167,9 +167,9 @@ fn load_snapshot( // prior to the finalized slot (which is invalid and inaccessible in our DB schema). let (parent_state_root, state) = chain .store - // TODO(gloas): the state doesn't need to be advanced here because we're applying an envelope - // but this function does use a lot of caches that could be more efficient. Is there - // a better way to do this? + // TODO(EIP-7732): the state doesn't need to be advanced here because we're applying an envelope + // but this function does use a lot of caches that could be more efficient. Is there + // a better way to do this? .get_advanced_hot_state( beacon_block_root, proto_beacon_block.slot, @@ -225,7 +225,7 @@ impl GossipVerifiedEnvelope { let beacon_block_root = envelope.beacon_block_root(); // check that we've seen the parent block of this envelope and that it passes validation - // TODO(gloas): this check would fail if the block didn't pass validation right? + // TODO(EIP-7732): this check would fail if the block didn't pass validation right? let fork_choice_read_lock = chain.canonical_head.fork_choice_read_lock(); let Some(parent_proto_block) = fork_choice_read_lock.get_block(&beacon_block_root) else { return Err(EnvelopeError::BlockRootUnknown { @@ -234,12 +234,12 @@ impl GossipVerifiedEnvelope { }; drop(fork_choice_read_lock); - // TODO(gloas): check that we haven't seen another valid `SignedExecutionPayloadEnvelope` - // for this block root from this builder - envelope status table check + // TODO(EIP-7732): check that we haven't seen another valid `SignedExecutionPayloadEnvelope` + // for this block root from this builder - envelope status table check - // TODO(gloas): this should probably be obtained from the ProtoBlock instead of the DB - // but this means the ProtoBlock needs to include something like the ExecutionBid - // will need to answer this question later. + // TODO(EIP-7732): this should probably be obtained from the ProtoBlock instead of the DB + // but this means the ProtoBlock needs to include something like the ExecutionBid + // will need to answer this question later. let parent_block = chain .get_full_block(&beacon_block_root)? .ok_or_else(|| { @@ -252,7 +252,7 @@ impl GossipVerifiedEnvelope { .signed_execution_payload_bid()? .message; - // TODO(gloas): Gossip rules for the beacon block contain the following: + // TODO(EIP-7732): Gossip rules for the beacon block contain the following: // https://github.com/ethereum/consensus-specs/blob/master/specs/phase0/p2p-interface.md#beacon_block // [IGNORE] The block is not from a future slot (with a MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance) // [IGNORE] The block is from a slot greater than the latest finalized slot @@ -282,9 +282,9 @@ impl GossipVerifiedEnvelope { }); } - // TODO(gloas): check these assumptions.. exactly what the most efficient way to verify the signatures - // in this case isn't clear. There are questions about the proposer cache, the pubkey cache, - // and so on. + // TODO(EIP-7732): check these assumptions.. exactly what the most efficient way to verify the signatures + // in this case isn't clear. There are questions about the proposer cache, the pubkey cache, + // and so on. // get the fork from the cache so we can verify the signature let block_slot = envelope.slot(); @@ -442,7 +442,7 @@ impl IntoExecutionPendingEnvelope chain: &Arc>, notify_execution_layer: NotifyExecutionLayer, ) -> Result, EnvelopeError> { - // TODO(gloas): figure out how this should be refactored.. + // TODO(EIP-7732): figure out how this should be refactored.. GossipVerifiedEnvelope::new(self, chain)? .into_execution_pending_envelope(chain, notify_execution_layer) } diff --git a/consensus/state_processing/src/envelope_processing.rs b/consensus/state_processing/src/envelope_processing.rs index 77d97c1f0e0..6de94f1c805 100644 --- a/consensus/state_processing/src/envelope_processing.rs +++ b/consensus/state_processing/src/envelope_processing.rs @@ -11,7 +11,7 @@ use types::{ Hash256, SignedExecutionPayloadEnvelope, Slot, }; -// TODO(gloas): don't use this redefinition.. +// TODO(EIP-7732): don't use this redefinition.. macro_rules! envelope_verify { ($condition: expr, $result: expr) => { if !$condition { @@ -118,7 +118,7 @@ pub fn envelope_processing( ) -> Result<(), EnvelopeProcessingError> { if verify_signatures.is_true() { // Verify Signed Envelope Signature - // TODO(gloas): there is probably a more efficient way to do this.. + // TODO(EIP-7732): there is probably a more efficient way to do this.. if !signed_envelope.verify_signature_with_state(&state, spec)? { return Err(EnvelopeProcessingError::BadSignature); } diff --git a/consensus/types/src/signed_execution_payload_envelope.rs b/consensus/types/src/signed_execution_payload_envelope.rs index 9b625287902..ffaf316dfcd 100644 --- a/consensus/types/src/signed_execution_payload_envelope.rs +++ b/consensus/types/src/signed_execution_payload_envelope.rs @@ -95,7 +95,7 @@ impl SignedExecutionPayloadEnvelope { /// /// The `parent_state` is the post-state of the beacon block with /// block_root = self.message.beacon_block_root - /// todo(gloas): maybe delete this function later + /// TODO(EIP-7732): maybe delete this function later pub fn verify_signature_with_state( &self, parent_state: &BeaconState, From 150b117cf06e82600ebd743800d1af058fe5f556 Mon Sep 17 00:00:00 2001 From: Shane K Moore <41407272+shane-moore@users.noreply.github.com> Date: Fri, 31 Oct 2025 17:13:46 -0400 Subject: [PATCH 6/9] modify get_pending_balance_to_withdraw per gloas spec (#8302) --- consensus/types/src/beacon_state.rs | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/consensus/types/src/beacon_state.rs b/consensus/types/src/beacon_state.rs index 085b2fb9886..4db17aea5c4 100644 --- a/consensus/types/src/beacon_state.rs +++ b/consensus/types/src/beacon_state.rs @@ -2456,8 +2456,13 @@ impl BeaconState { .map_err(Into::into) } + // TODO(EIP-7732): The consensus spec PR for this change mentions that some EF tests will be needed but haven't been created yet. + // We should integrate them once they are available. + // https://github.com/ethereum/consensus-specs/pull/4513 pub fn get_pending_balance_to_withdraw(&self, validator_index: usize) -> Result { let mut pending_balance = 0; + + // Sum pending partial withdrawals for withdrawal in self .pending_partial_withdrawals()? .iter() @@ -2465,6 +2470,27 @@ impl BeaconState { { pending_balance.safe_add_assign(withdrawal.amount)?; } + + // Sum builder pending withdrawals + if let Ok(builder_pending_withdrawals) = self.builder_pending_withdrawals() { + for withdrawal in builder_pending_withdrawals + .iter() + .filter(|withdrawal| withdrawal.builder_index as usize == validator_index) + { + pending_balance.safe_add_assign(withdrawal.amount)?; + } + } + + // Sum builder pending payments + if let Ok(builder_pending_payments) = self.builder_pending_payments() { + for payment in builder_pending_payments + .iter() + .filter(|payment| payment.withdrawal.builder_index as usize == validator_index) + { + pending_balance.safe_add_assign(payment.withdrawal.amount)?; + } + } + Ok(pending_balance) } From 4ab5a77361ae9ca4a2b6c7929e41d7e08a66f218 Mon Sep 17 00:00:00 2001 From: Shane K Moore <41407272+shane-moore@users.noreply.github.com> Date: Mon, 3 Nov 2025 12:25:32 -0800 Subject: [PATCH 7/9] Gloas modify process_withdrawals (#8281) * add process withdrawals logic * fix process_withdrawals test * updates per consensus spec v1.6.0-beta.1 release * add todo for is_parent_block_full --- beacon_node/beacon_chain/src/beacon_chain.rs | 4 +- beacon_node/http_api/src/builder_states.rs | 2 +- .../src/per_block_processing.rs | 150 ++++++++-------- .../process_withdrawals.rs | 166 ++++++++++++++++++ consensus/types/src/beacon_state.rs | 1 + testing/ef_tests/src/cases/operations.rs | 11 +- 6 files changed, 253 insertions(+), 81 deletions(-) create mode 100644 consensus/state_processing/src/per_block_processing/process_withdrawals.rs diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index c985f937a5d..853b8def76c 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -4832,7 +4832,7 @@ impl BeaconChain { let proposal_epoch = proposal_slot.epoch(T::EthSpec::slots_per_epoch()); if head_state.current_epoch() == proposal_epoch { return get_expected_withdrawals(&unadvanced_state, &self.spec) - .map(|(withdrawals, _)| withdrawals) + .map(|(withdrawals, _, _)| withdrawals) .map_err(Error::PrepareProposerFailed); } @@ -4850,7 +4850,7 @@ impl BeaconChain { &self.spec, )?; get_expected_withdrawals(&advanced_state, &self.spec) - .map(|(withdrawals, _)| withdrawals) + .map(|(withdrawals, _, _)| withdrawals) .map_err(Error::PrepareProposerFailed) } diff --git a/beacon_node/http_api/src/builder_states.rs b/beacon_node/http_api/src/builder_states.rs index 7c05dd00d26..74228961fb5 100644 --- a/beacon_node/http_api/src/builder_states.rs +++ b/beacon_node/http_api/src/builder_states.rs @@ -32,7 +32,7 @@ pub fn get_next_withdrawals( } match get_expected_withdrawals(&state, &chain.spec) { - Ok((withdrawals, _)) => Ok(withdrawals), + Ok((withdrawals, _, _)) => Ok(withdrawals), Err(e) => Err(warp_utils::reject::custom_server_error(format!( "failed to get expected withdrawal: {:?}", e diff --git a/consensus/state_processing/src/per_block_processing.rs b/consensus/state_processing/src/per_block_processing.rs index 5335c917cb3..7b16dbe8042 100644 --- a/consensus/state_processing/src/per_block_processing.rs +++ b/consensus/state_processing/src/per_block_processing.rs @@ -30,6 +30,7 @@ pub mod deneb; pub mod errors; mod is_valid_indexed_attestation; pub mod process_operations; +pub mod process_withdrawals; pub mod signature_sets; pub mod tests; mod verify_attestation; @@ -39,7 +40,6 @@ mod verify_deposit; mod verify_exit; mod verify_proposer_slashing; -use crate::common::decrease_balance; use crate::common::update_progressive_balances_cache::{ initialize_progressive_balances_cache, update_progressive_balances_metrics, }; @@ -171,13 +171,20 @@ pub fn per_block_processing>( // previous block. if is_execution_enabled(state, block.body()) { let body = block.body(); - // TODO(EIP-7732): build out process_withdrawals variant for gloas - process_withdrawals::(state, body.execution_payload()?, spec)?; - process_execution_payload::(state, body, spec)?; - } + if state.fork_name_unchecked().gloas_enabled() { + process_withdrawals::gloas::process_withdrawals::(state, spec)?; - // TODO(EIP-7732): build out process_execution_bid - // process_execution_bid(state, block, verify_signatures, spec)?; + // TODO(EIP-7732): build out process_execution_bid + // process_execution_bid(state, block, verify_signatures, spec)?; + } else { + process_withdrawals::capella::process_withdrawals::( + state, + body.execution_payload()?, + spec, + )?; + process_execution_payload::(state, body, spec)?; + } + } process_randao(state, block, verify_randao, ctxt, spec)?; process_eth1_data(state, block.body().eth1_data())?; @@ -515,17 +522,70 @@ pub fn compute_timestamp_at_slot( /// Compute the next batch of withdrawals which should be included in a block. /// -/// https://github.com/ethereum/consensus-specs/blob/dev/specs/electra/beacon-chain.md#new-get_expected_withdrawals +/// https://ethereum.github.io/consensus-specs/specs/gloas/beacon-chain/#modified-get_expected_withdrawals +#[allow(clippy::type_complexity)] pub fn get_expected_withdrawals( state: &BeaconState, spec: &ChainSpec, -) -> Result<(Withdrawals, Option), BlockProcessingError> { +) -> Result<(Withdrawals, Option, Option), BlockProcessingError> { let epoch = state.current_epoch(); let mut withdrawal_index = state.next_withdrawal_index()?; let mut validator_index = state.next_withdrawal_validator_index()?; let mut withdrawals = Vec::::with_capacity(E::max_withdrawals_per_payload()); let fork_name = state.fork_name_unchecked(); + // [New in Gloas:EIP7732] + // Sweep for builder payments + let processed_builder_withdrawals_count = + if let Ok(builder_pending_withdrawals) = state.builder_pending_withdrawals() { + let mut processed_builder_withdrawals_count = 0; + for withdrawal in builder_pending_withdrawals { + if withdrawal.withdrawable_epoch > epoch + || withdrawals.len().safe_add(1)? == E::max_withdrawals_per_payload() + { + break; + } + + if process_withdrawals::is_builder_payment_withdrawable(state, withdrawal)? { + let total_withdrawn = withdrawals + .iter() + .filter_map(|w| { + (w.validator_index == withdrawal.builder_index).then_some(w.amount) + }) + .safe_sum()?; + let balance = state + .get_balance(withdrawal.builder_index as usize)? + .safe_sub(total_withdrawn)?; + let builder = state.get_validator(withdrawal.builder_index as usize)?; + + let withdrawable_balance = if builder.slashed { + std::cmp::min(balance, withdrawal.amount) + } else if balance > spec.min_activation_balance { + std::cmp::min( + balance.safe_sub(spec.min_activation_balance)?, + withdrawal.amount, + ) + } else { + 0 + }; + + if withdrawable_balance > 0 { + withdrawals.push(Withdrawal { + index: withdrawal_index, + validator_index: withdrawal.builder_index, + address: withdrawal.fee_recipient, + amount: withdrawable_balance, + }); + withdrawal_index.safe_add_assign(1)?; + } + } + processed_builder_withdrawals_count.safe_add_assign(1)?; + } + Some(processed_builder_withdrawals_count) + } else { + None + }; + // [New in Electra:EIP7251] // Consume pending partial withdrawals let processed_partial_withdrawals_count = @@ -626,71 +686,9 @@ pub fn get_expected_withdrawals( .safe_rem(state.validators().len() as u64)?; } - Ok((withdrawals.into(), processed_partial_withdrawals_count)) -} - -/// Apply withdrawals to the state. -/// TODO(EIP-7732): abstract this out and create gloas variant -pub fn process_withdrawals>( - state: &mut BeaconState, - payload: Payload::Ref<'_>, - spec: &ChainSpec, -) -> Result<(), BlockProcessingError> { - if state.fork_name_unchecked().capella_enabled() { - let (expected_withdrawals, processed_partial_withdrawals_count) = - get_expected_withdrawals(state, spec)?; - let expected_root = expected_withdrawals.tree_hash_root(); - let withdrawals_root = payload.withdrawals_root()?; - - if expected_root != withdrawals_root { - return Err(BlockProcessingError::WithdrawalsRootMismatch { - expected: expected_root, - found: withdrawals_root, - }); - } - - for withdrawal in expected_withdrawals.iter() { - decrease_balance( - state, - withdrawal.validator_index as usize, - withdrawal.amount, - )?; - } - - // Update pending partial withdrawals [New in Electra:EIP7251] - if let Some(processed_partial_withdrawals_count) = processed_partial_withdrawals_count { - state - .pending_partial_withdrawals_mut()? - .pop_front(processed_partial_withdrawals_count)?; - } - - // Update the next withdrawal index if this block contained withdrawals - if let Some(latest_withdrawal) = expected_withdrawals.last() { - *state.next_withdrawal_index_mut()? = latest_withdrawal.index.safe_add(1)?; - - // Update the next validator index to start the next withdrawal sweep - if expected_withdrawals.len() == E::max_withdrawals_per_payload() { - // Next sweep starts after the latest withdrawal's validator index - let next_validator_index = latest_withdrawal - .validator_index - .safe_add(1)? - .safe_rem(state.validators().len() as u64)?; - *state.next_withdrawal_validator_index_mut()? = next_validator_index; - } - } - - // Advance sweep by the max length of the sweep if there was not a full set of withdrawals - if expected_withdrawals.len() != E::max_withdrawals_per_payload() { - let next_validator_index = state - .next_withdrawal_validator_index()? - .safe_add(spec.max_validators_per_withdrawals_sweep)? - .safe_rem(state.validators().len() as u64)?; - *state.next_withdrawal_validator_index_mut()? = next_validator_index; - } - - Ok(()) - } else { - // these shouldn't even be encountered but they're here for completeness - Ok(()) - } + Ok(( + withdrawals.into(), + processed_builder_withdrawals_count, + processed_partial_withdrawals_count, + )) } diff --git a/consensus/state_processing/src/per_block_processing/process_withdrawals.rs b/consensus/state_processing/src/per_block_processing/process_withdrawals.rs new file mode 100644 index 00000000000..4082403dc16 --- /dev/null +++ b/consensus/state_processing/src/per_block_processing/process_withdrawals.rs @@ -0,0 +1,166 @@ +use super::errors::BlockProcessingError; +use super::get_expected_withdrawals; +use crate::common::decrease_balance; +use safe_arith::SafeArith; +use tree_hash::TreeHash; +use types::{ + AbstractExecPayload, BeaconState, BuilderPendingWithdrawal, ChainSpec, EthSpec, ExecPayload, + List, Withdrawals, +}; + +/// Check if a builder payment is withdrawable. +/// A builder payment is withdrawable if the builder is not slashed or +/// the builder's withdrawable epoch has been reached. +pub fn is_builder_payment_withdrawable( + state: &BeaconState, + withdrawal: &BuilderPendingWithdrawal, +) -> Result { + let builder = state.get_validator(withdrawal.builder_index as usize)?; + let current_epoch = state.current_epoch(); + + Ok(builder.withdrawable_epoch >= current_epoch || !builder.slashed) +} + +fn process_withdrawals_common( + state: &mut BeaconState, + expected_withdrawals: Withdrawals, + partial_withdrawals_count: Option, + spec: &ChainSpec, +) -> Result<(), BlockProcessingError> { + match state { + BeaconState::Capella(_) + | BeaconState::Deneb(_) + | BeaconState::Electra(_) + | BeaconState::Fulu(_) + | BeaconState::Gloas(_) => { + // Update pending partial withdrawals [New in Electra:EIP7251] + if let Some(partial_withdrawals_count) = partial_withdrawals_count { + state + .pending_partial_withdrawals_mut()? + .pop_front(partial_withdrawals_count)?; + } + + // Update the next withdrawal index if this block contained withdrawals + if let Some(latest_withdrawal) = expected_withdrawals.last() { + *state.next_withdrawal_index_mut()? = latest_withdrawal.index.safe_add(1)?; + + // Update the next validator index to start the next withdrawal sweep + if expected_withdrawals.len() == E::max_withdrawals_per_payload() { + // Next sweep starts after the latest withdrawal's validator index + let next_validator_index = latest_withdrawal + .validator_index + .safe_add(1)? + .safe_rem(state.validators().len() as u64)?; + *state.next_withdrawal_validator_index_mut()? = next_validator_index; + } + } + + // Advance sweep by the max length of the sweep if there was not a full set of withdrawals + if expected_withdrawals.len() != E::max_withdrawals_per_payload() { + let next_validator_index = state + .next_withdrawal_validator_index()? + .safe_add(spec.max_validators_per_withdrawals_sweep)? + .safe_rem(state.validators().len() as u64)?; + *state.next_withdrawal_validator_index_mut()? = next_validator_index; + } + + Ok(()) + } + // these shouldn't even be encountered but they're here for completeness + BeaconState::Base(_) | BeaconState::Altair(_) | BeaconState::Bellatrix(_) => Ok(()), + } +} + +pub mod capella { + use super::*; + /// Apply withdrawals to the state. + pub fn process_withdrawals>( + state: &mut BeaconState, + payload: Payload::Ref<'_>, + spec: &ChainSpec, + ) -> Result<(), BlockProcessingError> { + // check if capella enabled because this function will run on the merge block where the fork is technically still Bellatrix + if state.fork_name_unchecked().capella_enabled() { + let (expected_withdrawals, _, partial_withdrawals_count) = + get_expected_withdrawals(state, spec)?; + + let expected_root = expected_withdrawals.tree_hash_root(); + let withdrawals_root = payload.withdrawals_root()?; + if expected_root != withdrawals_root { + return Err(BlockProcessingError::WithdrawalsRootMismatch { + expected: expected_root, + found: withdrawals_root, + }); + } + + for withdrawal in expected_withdrawals.iter() { + decrease_balance( + state, + withdrawal.validator_index as usize, + withdrawal.amount, + )?; + } + + process_withdrawals_common(state, expected_withdrawals, partial_withdrawals_count, spec) + } else { + // these shouldn't even be encountered but they're here for completeness + Ok(()) + } + } +} +pub mod gloas { + use super::*; + + // TODO(EIP-7732): Add comprehensive tests for Gloas `process_withdrawals`: + // Similar to Capella version, these will be tested via: + // 1. EF consensus-spec tests in `testing/ef_tests/src/cases/operations.rs` + // 2. Integration tests via full block processing + // These tests would currently fail due to incomplete Gloas block structure as mentioned here, so we will implement them after block and payload processing is in a good state. + // https://github.com/sigp/lighthouse/pull/8273 + /// Apply withdrawals to the state. + pub fn process_withdrawals( + state: &mut BeaconState, + spec: &ChainSpec, + ) -> Result<(), BlockProcessingError> { + if !state.is_parent_block_full() { + return Ok(()); + } + + let (expected_withdrawals, builder_withdrawals_count, partial_withdrawals_count) = + get_expected_withdrawals(state, spec)?; + + *state.latest_withdrawals_root_mut()? = expected_withdrawals.tree_hash_root(); + + for withdrawal in expected_withdrawals.iter() { + decrease_balance( + state, + withdrawal.validator_index as usize, + withdrawal.amount, + )?; + } + + if let (Ok(builder_pending_withdrawals), Some(builder_count)) = ( + state.builder_pending_withdrawals(), + builder_withdrawals_count, + ) { + let mut updated_builder_withdrawals = + Vec::with_capacity(E::builder_pending_withdrawals_limit()); + + for (i, withdrawal) in builder_pending_withdrawals.iter().enumerate() { + if i < builder_count { + if !is_builder_payment_withdrawable(state, withdrawal)? { + updated_builder_withdrawals.push(withdrawal.clone()); + } + } else { + updated_builder_withdrawals.push(withdrawal.clone()); + } + } + + *state.builder_pending_withdrawals_mut()? = List::new(updated_builder_withdrawals)?; + } + + process_withdrawals_common(state, expected_withdrawals, partial_withdrawals_count, spec)?; + + Ok(()) + } +} diff --git a/consensus/types/src/beacon_state.rs b/consensus/types/src/beacon_state.rs index 4db17aea5c4..4aaf8998446 100644 --- a/consensus/types/src/beacon_state.rs +++ b/consensus/types/src/beacon_state.rs @@ -2204,6 +2204,7 @@ impl BeaconState { } } + /// Return true if the parent block was full (both beacon block and execution payload were present). pub fn is_parent_block_full(&self) -> bool { match self { BeaconState::Base(_) | BeaconState::Altair(_) => false, diff --git a/testing/ef_tests/src/cases/operations.rs b/testing/ef_tests/src/cases/operations.rs index a53bce927cb..63b46945c2f 100644 --- a/testing/ef_tests/src/cases/operations.rs +++ b/testing/ef_tests/src/cases/operations.rs @@ -419,8 +419,15 @@ impl Operation for WithdrawalsPayload { spec: &ChainSpec, _: &Operations, ) -> Result<(), BlockProcessingError> { - // TODO(EIP-7732): implement separate gloas and non-gloas variants of process_withdrawals - process_withdrawals::<_, FullPayload<_>>(state, self.payload.to_ref(), spec) + if state.fork_name_unchecked().gloas_enabled() { + process_withdrawals::gloas::process_withdrawals(state, spec) + } else { + process_withdrawals::capella::process_withdrawals::<_, FullPayload<_>>( + state, + self.payload.to_ref(), + spec, + ) + } } } From 4dfc31c0a9153748b8bb790feacb4e7d186246a6 Mon Sep 17 00:00:00 2001 From: Shane K Moore <41407272+shane-moore@users.noreply.github.com> Date: Thu, 6 Nov 2025 14:02:37 -0800 Subject: [PATCH 8/9] Gloas process execution payload bid (#8355) * add proces_execution_bid * add has_builder_withdrawal_credential * process_execution_payload_bid signature is infinity check for self-build * process_execution_payload_bid updates per consensus spec v1.6.0-beta.1 release * process_execution_bid to avoid expensive lookups for 0 amount bids * verify builder not slashed even for self-building --- .../src/bls_to_execution_changes.rs | 22 ++- beacon_node/operation_pool/src/lib.rs | 7 +- .../src/per_block_processing.rs | 172 +++++++++++++++++- .../src/per_block_processing/errors.rs | 41 +++++ .../process_operations.rs | 13 +- .../per_block_processing/signature_sets.rs | 34 +++- .../state_processing/src/upgrade/electra.rs | 2 +- consensus/types/src/beacon_state.rs | 1 + consensus/types/src/validator.rs | 60 +++++- 9 files changed, 316 insertions(+), 36 deletions(-) diff --git a/beacon_node/operation_pool/src/bls_to_execution_changes.rs b/beacon_node/operation_pool/src/bls_to_execution_changes.rs index cc8809c43e6..c3817808052 100644 --- a/beacon_node/operation_pool/src/bls_to_execution_changes.rs +++ b/beacon_node/operation_pool/src/bls_to_execution_changes.rs @@ -113,16 +113,18 @@ impl BlsToExecutionChanges { .validators() .get(validator_index as usize) .is_none_or(|validator| { - let prune = validator.has_execution_withdrawal_credential(spec) - && head_block - .message() - .body() - .bls_to_execution_changes() - .map_or(true, |recent_changes| { - !recent_changes - .iter() - .any(|c| c.message.validator_index == validator_index) - }); + let prune = validator.has_execution_withdrawal_credential( + spec, + head_state.fork_name_unchecked(), + ) && head_block + .message() + .body() + .bls_to_execution_changes() + .map_or(true, |recent_changes| { + !recent_changes + .iter() + .any(|c| c.message.validator_index == validator_index) + }); if prune { validator_indices_pruned.push(validator_index); } diff --git a/beacon_node/operation_pool/src/lib.rs b/beacon_node/operation_pool/src/lib.rs index 24e2cfbbb5d..cdcb66f91a2 100644 --- a/beacon_node/operation_pool/src/lib.rs +++ b/beacon_node/operation_pool/src/lib.rs @@ -582,7 +582,12 @@ impl OperationPool { address_change.signature_is_still_valid(&state.fork()) && state .get_validator(address_change.as_inner().message.validator_index as usize) - .is_ok_and(|validator| !validator.has_execution_withdrawal_credential(spec)) + .is_ok_and(|validator| { + !validator.has_execution_withdrawal_credential( + spec, + state.fork_name_unchecked(), + ) + }) }, |address_change| address_change.as_inner().clone(), E::MaxBlsToExecutionChanges::to_usize(), diff --git a/consensus/state_processing/src/per_block_processing.rs b/consensus/state_processing/src/per_block_processing.rs index 7b16dbe8042..189e055d711 100644 --- a/consensus/state_processing/src/per_block_processing.rs +++ b/consensus/state_processing/src/per_block_processing.rs @@ -1,8 +1,12 @@ +use self::errors::ExecutionPayloadBidInvalid; use crate::consensus_context::ConsensusContext; use errors::{BlockOperationError, BlockProcessingError, HeaderInvalid}; use rayon::prelude::*; use safe_arith::{ArithError, SafeArith, SafeArithIter}; -use signature_sets::{block_proposal_signature_set, get_pubkey_from_state, randao_signature_set}; +use signature_sets::{ + block_proposal_signature_set, execution_payload_bid_signature_set, get_pubkey_from_state, + randao_signature_set, +}; use std::borrow::Cow; use tree_hash::TreeHash; use types::*; @@ -173,9 +177,7 @@ pub fn per_block_processing>( let body = block.body(); if state.fork_name_unchecked().gloas_enabled() { process_withdrawals::gloas::process_withdrawals::(state, spec)?; - - // TODO(EIP-7732): build out process_execution_bid - // process_execution_bid(state, block, verify_signatures, spec)?; + process_execution_payload_bid(state, block, verify_signatures, spec)?; } else { process_withdrawals::capella::process_withdrawals::( state, @@ -625,7 +627,7 @@ pub fn get_expected_withdrawals( index: withdrawal_index, validator_index: withdrawal.validator_index, address: validator - .get_execution_withdrawal_address(spec) + .get_execution_withdrawal_address(spec, state.fork_name_unchecked()) .ok_or(BeaconStateError::NonExecutionAddressWithdrawalCredential)?, amount: withdrawable_balance, }); @@ -662,7 +664,7 @@ pub fn get_expected_withdrawals( index: withdrawal_index, validator_index, address: validator - .get_execution_withdrawal_address(spec) + .get_execution_withdrawal_address(spec, state.fork_name_unchecked()) .ok_or(BlockProcessingError::WithdrawalCredentialsInvalid)?, amount: balance, }); @@ -672,7 +674,7 @@ pub fn get_expected_withdrawals( index: withdrawal_index, validator_index, address: validator - .get_execution_withdrawal_address(spec) + .get_execution_withdrawal_address(spec, state.fork_name_unchecked()) .ok_or(BlockProcessingError::WithdrawalCredentialsInvalid)?, amount: balance.safe_sub(validator.get_max_effective_balance(spec, fork_name))?, }); @@ -692,3 +694,159 @@ pub fn get_expected_withdrawals( processed_partial_withdrawals_count, )) } + +pub fn process_execution_payload_bid>( + state: &mut BeaconState, + block: BeaconBlockRef<'_, E, Payload>, + verify_signatures: VerifySignatures, + spec: &ChainSpec, +) -> Result<(), BlockProcessingError> { + // Verify the bid signature + let signed_bid = block.body().signed_execution_payload_bid()?; + + let bid = &signed_bid.message; + let amount = bid.value; + let builder_index = bid.builder_index; + let builder = state.get_validator(builder_index as usize)?; + + // For self-builds, amount must be zero regardless of withdrawal credential prefix + if builder_index == block.proposer_index() { + block_verify!(amount == 0, ExecutionPayloadBidInvalid::BadAmount.into()); + // TODO(EIP-7732): check with team if we should use ExecutionPayloadBidInvalid::BadSignature or a new error variant for this, like BadSelfBuildSignature + block_verify!( + signed_bid.signature.is_infinity(), + ExecutionPayloadBidInvalid::BadSignature.into() + ); + } else { + // Non-self builds require builder withdrawal credential + block_verify!( + builder.has_builder_withdrawal_credential(spec), + ExecutionPayloadBidInvalid::BadWithdrawalCredentials.into() + ); + if verify_signatures.is_true() { + block_verify!( + execution_payload_bid_signature_set( + state, + |i| get_pubkey_from_state(state, i), + signed_bid, + spec + )? + .verify(), + ExecutionPayloadBidInvalid::BadSignature.into() + ); + } + } + + // Verify builder is active and not slashed + block_verify!( + builder.is_active_at(state.current_epoch()), + ExecutionPayloadBidInvalid::BuilderNotActive(builder_index).into() + ); + block_verify!( + !builder.slashed, + ExecutionPayloadBidInvalid::BuilderSlashed(builder_index).into() + ); + + // Only perform payment related checks if amount > 0 + if amount > 0 { + // Check that the builder has funds to cover the bid + let pending_payments = state + .builder_pending_payments()? + .iter() + .filter_map(|payment| { + if payment.withdrawal.builder_index == builder_index { + Some(payment.withdrawal.amount) + } else { + None + } + }) + .safe_sum()?; + + let pending_withdrawals = state + .builder_pending_withdrawals()? + .iter() + .filter_map(|withdrawal| { + if withdrawal.builder_index == builder_index { + Some(withdrawal.amount) + } else { + None + } + }) + .safe_sum()?; + + let builder_balance = state.get_balance(builder_index as usize)?; + + block_verify!( + builder_balance + >= amount + .safe_add(pending_payments)? + .safe_add(pending_withdrawals)? + .safe_add(spec.min_activation_balance)?, + ExecutionPayloadBidInvalid::InsufficientBalance { + builder_index, + builder_balance, + bid_value: amount, + } + .into() + ); + } + + // Verify that the bid is for the current slot + block_verify!( + bid.slot == block.slot(), + ExecutionPayloadBidInvalid::SlotMismatch { + state_slot: block.slot(), + bid_slot: bid.slot, + } + .into() + ); + + // Verify that the bid is for the right parent block + let latest_block_hash = state.latest_block_hash()?; + block_verify!( + bid.parent_block_hash == *latest_block_hash, + ExecutionPayloadBidInvalid::ParentBlockHashMismatch { + state_block_hash: *latest_block_hash, + bid_parent_hash: bid.parent_block_hash, + } + .into() + ); + + block_verify!( + bid.parent_block_root == block.parent_root(), + ExecutionPayloadBidInvalid::ParentBlockRootMismatch { + block_parent_root: block.parent_root(), + bid_parent_root: bid.parent_block_root, + } + .into() + ); + + // Record the pending payment if there is some payment + if amount > 0 { + let pending_payment = BuilderPendingPayment { + weight: 0, + withdrawal: BuilderPendingWithdrawal { + fee_recipient: bid.fee_recipient, + amount, + builder_index, + withdrawable_epoch: spec.far_future_epoch, + }, + }; + + let payment_index = (E::slots_per_epoch() + .safe_add(bid.slot.as_u64().safe_rem(E::slots_per_epoch())?)?) + as usize; + + *state + .builder_pending_payments_mut()? + .get_mut(payment_index) + .ok_or(BlockProcessingError::BeaconStateError( + BeaconStateError::BuilderPendingPaymentsIndexNotSupported(payment_index), + ))? = pending_payment; + } + + // Cache the execution bid + *state.latest_execution_payload_bid_mut()? = bid.clone(); + + Ok(()) +} diff --git a/consensus/state_processing/src/per_block_processing/errors.rs b/consensus/state_processing/src/per_block_processing/errors.rs index ff7c0204e24..0374547a207 100644 --- a/consensus/state_processing/src/per_block_processing/errors.rs +++ b/consensus/state_processing/src/per_block_processing/errors.rs @@ -91,6 +91,9 @@ pub enum BlockProcessingError { }, WithdrawalCredentialsInvalid, PendingAttestationInElectra, + ExecutionPayloadBidInvalid { + reason: ExecutionPayloadBidInvalid, + }, } impl From for BlockProcessingError { @@ -147,6 +150,12 @@ impl From for BlockProcessingError { } } +impl From for BlockProcessingError { + fn from(reason: ExecutionPayloadBidInvalid) -> Self { + Self::ExecutionPayloadBidInvalid { reason } + } +} + impl From> for BlockProcessingError { fn from(e: BlockOperationError) -> BlockProcessingError { match e { @@ -440,6 +449,38 @@ pub enum ExitInvalid { PendingWithdrawalInQueue(u64), } +#[derive(Debug, PartialEq, Clone)] +pub enum ExecutionPayloadBidInvalid { + /// The builder sent a 0 amount + BadAmount, + /// The signature is invalid. + BadSignature, + /// The builder's withdrawal credential is invalid + BadWithdrawalCredentials, + /// The builder is not an active validator. + BuilderNotActive(u64), + /// The builder is slashed + BuilderSlashed(u64), + /// The builder has insufficient balance to cover the bid + InsufficientBalance { + builder_index: u64, + builder_balance: u64, + bid_value: u64, + }, + /// Bid slot doesn't match state slot + SlotMismatch { state_slot: Slot, bid_slot: Slot }, + /// The bid's parent block hash doesn't match the state's latest block hash + ParentBlockHashMismatch { + state_block_hash: ExecutionBlockHash, + bid_parent_hash: ExecutionBlockHash, + }, + /// The bid's parent block root doesn't match the block's parent root + ParentBlockRootMismatch { + block_parent_root: Hash256, + bid_parent_root: Hash256, + }, +} + #[derive(Debug, PartialEq, Clone)] pub enum BlsExecutionChangeInvalid { /// The specified validator is not in the state's validator registry. diff --git a/consensus/state_processing/src/per_block_processing/process_operations.rs b/consensus/state_processing/src/per_block_processing/process_operations.rs index 9a1c6c2f6ad..9f5b916743f 100644 --- a/consensus/state_processing/src/per_block_processing/process_operations.rs +++ b/consensus/state_processing/src/per_block_processing/process_operations.rs @@ -513,9 +513,10 @@ pub fn process_withdrawal_requests( let validator = state.get_validator(validator_index)?; // Verify withdrawal credentials - let has_correct_credential = validator.has_execution_withdrawal_credential(spec); + let has_correct_credential = + validator.has_execution_withdrawal_credential(spec, state.fork_name_unchecked()); let is_correct_source_address = validator - .get_execution_withdrawal_address(spec) + .get_execution_withdrawal_address(spec, state.fork_name_unchecked()) .map(|addr| addr == request.source_address) .unwrap_or(false); @@ -560,7 +561,7 @@ pub fn process_withdrawal_requests( .safe_add(pending_balance_to_withdraw)?; // Only allow partial withdrawals with compounding withdrawal credentials - if validator.has_compounding_withdrawal_credential(spec) + if validator.has_compounding_withdrawal_credential(spec, state.fork_name_unchecked()) && has_sufficient_effective_balance && has_excess_balance { @@ -729,7 +730,9 @@ pub fn process_consolidation_request( let source_validator = state.get_validator(source_index)?; // Verify the source withdrawal credentials - if let Some(withdrawal_address) = source_validator.get_execution_withdrawal_address(spec) { + if let Some(withdrawal_address) = + source_validator.get_execution_withdrawal_address(spec, state.fork_name_unchecked()) + { if withdrawal_address != consolidation_request.source_address { return Ok(()); } @@ -740,7 +743,7 @@ pub fn process_consolidation_request( let target_validator = state.get_validator(target_index)?; // Verify the target has compounding withdrawal credentials - if !target_validator.has_compounding_withdrawal_credential(spec) { + if !target_validator.has_compounding_withdrawal_credential(spec, state.fork_name_unchecked()) { return Ok(()); } diff --git a/consensus/state_processing/src/per_block_processing/signature_sets.rs b/consensus/state_processing/src/per_block_processing/signature_sets.rs index 1dc53ce1a89..c18cbbe4b53 100644 --- a/consensus/state_processing/src/per_block_processing/signature_sets.rs +++ b/consensus/state_processing/src/per_block_processing/signature_sets.rs @@ -11,9 +11,9 @@ use types::{ BeaconStateError, ChainSpec, DepositData, Domain, Epoch, EthSpec, Fork, Hash256, InconsistentFork, IndexedAttestation, IndexedAttestationRef, ProposerSlashing, PublicKey, PublicKeyBytes, Signature, SignedAggregateAndProof, SignedBeaconBlock, SignedBeaconBlockHeader, - SignedBlsToExecutionChange, SignedContributionAndProof, SignedExecutionPayloadEnvelope, - SignedRoot, SignedVoluntaryExit, SigningData, Slot, SyncAggregate, SyncAggregatorSelectionData, - Unsigned, + SignedBlsToExecutionChange, SignedContributionAndProof, SignedExecutionPayloadBid, + SignedExecutionPayloadEnvelope, SignedRoot, SignedVoluntaryExit, SigningData, Slot, + SyncAggregate, SyncAggregatorSelectionData, Unsigned, }; pub type Result = std::result::Result; @@ -360,6 +360,34 @@ where )) } +pub fn execution_payload_bid_signature_set<'a, E, F>( + state: &'a BeaconState, + get_pubkey: F, + signed_execution_payload_bid: &'a SignedExecutionPayloadBid, + spec: &'a ChainSpec, +) -> Result> +where + E: EthSpec, + F: Fn(usize) -> Option>, +{ + let domain = spec.get_domain( + state.current_epoch(), + Domain::BeaconBuilder, + &state.fork(), + state.genesis_validators_root(), + ); + let execution_payload_bid = &signed_execution_payload_bid.message; + let pubkey = get_pubkey(execution_payload_bid.builder_index as usize) + .ok_or(Error::ValidatorUnknown(execution_payload_bid.builder_index))?; + let message = execution_payload_bid.signing_root(domain); + + Ok(SignatureSet::single_pubkey( + &signed_execution_payload_bid.signature, + pubkey, + message, + )) +} + /// Returns the signature set for the given `attester_slashing` and corresponding `pubkeys`. pub fn attester_slashing_signature_sets<'a, E, F>( state: &'a BeaconState, diff --git a/consensus/state_processing/src/upgrade/electra.rs b/consensus/state_processing/src/upgrade/electra.rs index 258b28a45bd..a84b81d85c1 100644 --- a/consensus/state_processing/src/upgrade/electra.rs +++ b/consensus/state_processing/src/upgrade/electra.rs @@ -82,7 +82,7 @@ pub fn upgrade_to_electra( // Ensure early adopters of compounding credentials go through the activation churn let validators = post.validators().clone(); for (index, validator) in validators.iter().enumerate() { - if validator.has_compounding_withdrawal_credential(spec) { + if validator.has_compounding_withdrawal_credential(spec, post.fork_name_unchecked()) { post.queue_excess_active_balance(index, spec)?; } } diff --git a/consensus/types/src/beacon_state.rs b/consensus/types/src/beacon_state.rs index 4aaf8998446..af31f5f1c59 100644 --- a/consensus/types/src/beacon_state.rs +++ b/consensus/types/src/beacon_state.rs @@ -161,6 +161,7 @@ pub enum Error { TotalActiveBalanceDiffUninitialized, GeneralizedIndexNotSupported(usize), IndexNotSupported(usize), + BuilderPendingPaymentsIndexNotSupported(usize), InvalidFlagIndex(usize), MerkleTreeError(merkle_proof::MerkleTreeError), PartialWithdrawalCountInvalid(usize), diff --git a/consensus/types/src/validator.rs b/consensus/types/src/validator.rs index dec8bba627f..6d48f92a83b 100644 --- a/consensus/types/src/validator.rs +++ b/consensus/types/src/validator.rs @@ -159,13 +159,41 @@ impl Validator { } /// Check if ``validator`` has an 0x02 prefixed "compounding" withdrawal credential. - pub fn has_compounding_withdrawal_credential(&self, spec: &ChainSpec) -> bool { + pub fn has_compounding_withdrawal_credential( + &self, + spec: &ChainSpec, + current_fork: ForkName, + ) -> bool { + if current_fork.gloas_enabled() { + self.has_compounding_withdrawal_credential_gloas(spec) + } else { + self.has_compounding_withdrawal_credential_electra(spec) + } + } + + /// Check if ``validator`` has an 0x02 prefixed "compounding" withdrawal credential + pub fn has_compounding_withdrawal_credential_electra(&self, spec: &ChainSpec) -> bool { + is_compounding_withdrawal_credential(self.withdrawal_credentials, spec) + } + + /// Check if ``validator`` has an 0x02 prefixed "compounding" withdrawal credential or an 0x03 prefixed "builder" withdrawal credential + pub fn has_compounding_withdrawal_credential_gloas(&self, spec: &ChainSpec) -> bool { is_compounding_withdrawal_credential(self.withdrawal_credentials, spec) + || is_builder_withdrawal_credential(self.withdrawal_credentials, spec) + } + + /// Check if ``validator`` has an 0x03 prefixed "builder" withdrawal credential. + pub fn has_builder_withdrawal_credential(&self, spec: &ChainSpec) -> bool { + is_builder_withdrawal_credential(self.withdrawal_credentials, spec) } /// Get the execution withdrawal address if this validator has one initialized. - pub fn get_execution_withdrawal_address(&self, spec: &ChainSpec) -> Option
{ - self.has_execution_withdrawal_credential(spec) + pub fn get_execution_withdrawal_address( + &self, + spec: &ChainSpec, + current_fork: ForkName, + ) -> Option
{ + self.has_execution_withdrawal_credential(spec, current_fork) .then(|| { self.withdrawal_credentials .as_slice() @@ -196,7 +224,7 @@ impl Validator { current_fork: ForkName, ) -> bool { if current_fork.electra_enabled() { - self.is_fully_withdrawable_validator_electra(balance, epoch, spec) + self.is_fully_withdrawable_validator_electra(balance, epoch, spec, current_fork) } else { self.is_fully_withdrawable_validator_capella(balance, epoch, spec) } @@ -220,8 +248,9 @@ impl Validator { balance: u64, epoch: Epoch, spec: &ChainSpec, + current_fork: ForkName, ) -> bool { - self.has_execution_withdrawal_credential(spec) + self.has_execution_withdrawal_credential(spec, current_fork) && self.withdrawable_epoch <= epoch && balance > 0 } @@ -261,21 +290,25 @@ impl Validator { let max_effective_balance = self.get_max_effective_balance(spec, current_fork); let has_max_effective_balance = self.effective_balance == max_effective_balance; let has_excess_balance = balance > max_effective_balance; - self.has_execution_withdrawal_credential(spec) + self.has_execution_withdrawal_credential(spec, current_fork) && has_max_effective_balance && has_excess_balance } /// Returns `true` if the validator has a 0x01 or 0x02 prefixed withdrawal credential. - pub fn has_execution_withdrawal_credential(&self, spec: &ChainSpec) -> bool { - self.has_compounding_withdrawal_credential(spec) + pub fn has_execution_withdrawal_credential( + &self, + spec: &ChainSpec, + current_fork: ForkName, + ) -> bool { + self.has_compounding_withdrawal_credential(spec, current_fork) || self.has_eth1_withdrawal_credential(spec) } /// Returns the max effective balance for a validator in gwei. pub fn get_max_effective_balance(&self, spec: &ChainSpec, current_fork: ForkName) -> u64 { if current_fork >= ForkName::Electra { - if self.has_compounding_withdrawal_credential(spec) { + if self.has_compounding_withdrawal_credential(spec, current_fork) { spec.max_effective_balance_electra } else { spec.min_activation_balance @@ -313,6 +346,15 @@ pub fn is_compounding_withdrawal_credential( .unwrap_or(false) } +/// Check if the withdrawal credential has the builder withdrawal prefix (0x03). +pub fn is_builder_withdrawal_credential(withdrawal_credentials: Hash256, spec: &ChainSpec) -> bool { + withdrawal_credentials + .as_slice() + .first() + .map(|prefix_byte| *prefix_byte == spec.builder_withdrawal_prefix_byte) + .unwrap_or(false) +} + #[cfg(test)] mod tests { use super::*; From 0a972d1ff145aaff428de38fce4c419da91ca943 Mon Sep 17 00:00:00 2001 From: Mark Mackey Date: Fri, 7 Nov 2025 14:42:00 -0600 Subject: [PATCH 9/9] Educe migration --- beacon_node/beacon_chain/src/envelope_verification.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/beacon_node/beacon_chain/src/envelope_verification.rs b/beacon_node/beacon_chain/src/envelope_verification.rs index 8406e1971fd..3c0a8363e24 100644 --- a/beacon_node/beacon_chain/src/envelope_verification.rs +++ b/beacon_node/beacon_chain/src/envelope_verification.rs @@ -31,7 +31,7 @@ use crate::block_verification::{PayloadVerificationHandle, PayloadVerificationOu use crate::envelope_verification_types::{EnvelopeImportData, MaybeAvailableEnvelope}; use crate::execution_payload::PayloadNotifier; use crate::{BeaconChain, BeaconChainError, BeaconChainTypes}; -use derivative::Derivative; +use educe::Educe; use slot_clock::SlotClock; use state_processing::envelope_processing::{EnvelopeProcessingError, envelope_processing}; use state_processing::{BlockProcessingError, VerifySignatures}; @@ -207,8 +207,8 @@ fn load_snapshot( /// A wrapper around a `SignedExecutionPayloadEnvelope` that indicates it has been approved for re-gossiping on /// the p2p network. -#[derive(Derivative)] -#[derivative(Debug(bound = "T: BeaconChainTypes"))] +#[derive(Educe)] +#[educe(Debug(bound = "T: BeaconChainTypes"))] pub struct GossipVerifiedEnvelope { pub signed_envelope: Arc>, pub parent_block: Arc>,