Updated for queueless withdrawals spec

This commit is contained in:
Mark Mackey 2022-11-09 18:09:07 -06:00
parent bc0af72c74
commit ab13f95db5
20 changed files with 209 additions and 224 deletions

View File

@ -4,7 +4,6 @@ mod get_attesting_indices;
mod get_indexed_attestation;
mod initiate_validator_exit;
mod slash_validator;
mod withdraw_balance;
pub mod altair;
pub mod base;
@ -15,8 +14,6 @@ pub use get_attesting_indices::{get_attesting_indices, get_attesting_indices_fro
pub use get_indexed_attestation::get_indexed_attestation;
pub use initiate_validator_exit::initiate_validator_exit;
pub use slash_validator::slash_validator;
#[cfg(feature = "withdrawals")]
pub use withdraw_balance::withdraw_balance;
use safe_arith::SafeArith;
use types::{BeaconState, BeaconStateError, EthSpec};

View File

@ -1,29 +0,0 @@
use crate::common::decrease_balance;
use safe_arith::SafeArith;
use types::{BeaconStateError as Error, *};
#[cfg(feature = "withdrawals")]
pub fn withdraw_balance<T: EthSpec>(
state: &mut BeaconState<T>,
validator_index: usize,
amount: u64,
) -> Result<(), Error> {
decrease_balance(state, validator_index as usize, amount)?;
let withdrawal_address = Address::from_slice(
&state
.get_validator(validator_index)?
.withdrawal_credentials
.as_bytes()[12..],
);
let withdrawal = Withdrawal {
index: *state.next_withdrawal_index()?,
validator_index: validator_index as u64,
address: withdrawal_address,
amount,
};
state.next_withdrawal_index_mut()?.safe_add_assign(1)?;
state.withdrawal_queue_mut()?.push(withdrawal)?;
Ok(())
}

View File

@ -19,6 +19,8 @@ pub use process_operations::process_operations;
pub use verify_attestation::{
verify_attestation_for_block_inclusion, verify_attestation_for_state,
};
#[cfg(all(feature = "withdrawals", feature = "withdrawals-processing"))]
pub use verify_bls_to_execution_change::verify_bls_to_execution_change;
pub use verify_deposit::{
get_existing_validator_index, verify_deposit_merkle_proof, verify_deposit_signature,
};
@ -34,6 +36,8 @@ pub mod signature_sets;
pub mod tests;
mod verify_attestation;
mod verify_attester_slashing;
#[cfg(all(feature = "withdrawals", feature = "withdrawals-processing"))]
mod verify_bls_to_execution_change;
mod verify_deposit;
mod verify_exit;
mod verify_proposer_slashing;

View File

@ -49,6 +49,10 @@ pub enum BlockProcessingError {
index: usize,
reason: ExitInvalid,
},
BlsExecutionChangeInvalid {
index: usize,
reason: BlsExecutionChangeInvalid,
},
SyncAggregateInvalid {
reason: SyncAggregateInvalid,
},
@ -180,7 +184,8 @@ impl_into_block_processing_error_with_index!(
IndexedAttestationInvalid,
AttestationInvalid,
DepositInvalid,
ExitInvalid
ExitInvalid,
BlsExecutionChangeInvalid
);
pub type HeaderValidationError = BlockOperationError<HeaderInvalid>;
@ -190,6 +195,7 @@ pub type AttestationValidationError = BlockOperationError<AttestationInvalid>;
pub type SyncCommitteeMessageValidationError = BlockOperationError<SyncAggregateInvalid>;
pub type DepositValidationError = BlockOperationError<DepositInvalid>;
pub type ExitValidationError = BlockOperationError<ExitInvalid>;
pub type BlsExecutionChangeValidationError = BlockOperationError<BlsExecutionChangeInvalid>;
#[derive(Debug, PartialEq, Clone)]
pub enum BlockOperationError<T> {
@ -405,6 +411,18 @@ pub enum ExitInvalid {
SignatureSetError(SignatureSetError),
}
#[derive(Debug, PartialEq, Clone)]
pub enum BlsExecutionChangeInvalid {
/// The specified validator is not in the state's validator registry.
ValidatorUnknown(u64),
/// Validator does not have BLS Withdrawal credentials before this change
NonBlsWithdrawalCredentials,
/// Provided BLS pubkey does not match withdrawal credentials
WithdrawalCredentialsMismatch,
/// The signature is invalid
BadSignature,
}
#[derive(Debug, PartialEq, Clone)]
pub enum SyncAggregateInvalid {
/// One or more of the aggregate public keys is invalid.

View File

@ -33,6 +33,9 @@ pub fn process_operations<'a, T: EthSpec, Payload: AbstractExecPayload<T>>(
process_attestations(state, block_body, verify_signatures, ctxt, spec)?;
process_deposits(state, block_body.deposits(), spec)?;
process_exits(state, block_body.voluntary_exits(), verify_signatures, spec)?;
#[cfg(all(feature = "withdrawals", feature = "withdrawals-processing"))]
process_bls_to_execution_changes(state, block_body, verify_signatures, spec)?;
Ok(())
}
@ -279,6 +282,46 @@ pub fn process_exits<T: EthSpec>(
Ok(())
}
/// Validates each `bls_to_execution_change` and updates the state
///
/// Returns `Ok(())` if the validation and state updates completed successfully. Otherwise returs
/// an `Err` describing the invalid object or cause of failure.
///
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/capella/beacon-chain.md#new-process_bls_to_execution_change
#[cfg(all(feature = "withdrawals", feature = "withdrawals-processing"))]
pub fn process_bls_to_execution_changes<'a, T: EthSpec, Payload: AbstractExecPayload<T>>(
state: &mut BeaconState<T>,
block_body: BeaconBlockBodyRef<'a, T, Payload>,
verify_signatures: VerifySignatures,
spec: &ChainSpec,
) -> Result<(), BlockProcessingError> {
match block_body {
BeaconBlockBodyRef::Base(_)
| BeaconBlockBodyRef::Altair(_)
| BeaconBlockBodyRef::Merge(_) => Ok(()),
BeaconBlockBodyRef::Capella(_) | BeaconBlockBodyRef::Eip4844(_) => {
for (i, signed_address_change) in block_body.bls_to_execution_changes()?.enumerate() {
verify_bls_to_execution_change(
state,
&signed_address_change,
verify_signatures,
spec,
)
.map_err(|e| e.into_with_index(i))?;
state
.get_validator_mut(signed_address_change.message.validator_index)?
.change_withdrawal_credentials(
signed_address_change.message.to_execution_address,
spec,
);
}
Ok(())
}
}
}
/// Validates each `Deposit` and updates the state, short-circuiting on an invalid object.
///
/// Returns `Ok(())` if the validation and state updates completed successfully, otherwise returns

View File

@ -11,8 +11,8 @@ use types::{
BeaconStateError, ChainSpec, DepositData, Domain, Epoch, EthSpec, Fork, Hash256,
InconsistentFork, IndexedAttestation, ProposerSlashing, PublicKey, PublicKeyBytes, Signature,
SignedAggregateAndProof, SignedBeaconBlock, SignedBeaconBlockHeader,
SignedContributionAndProof, SignedRoot, SignedVoluntaryExit, SigningData, Slot, SyncAggregate,
SyncAggregatorSelectionData, Unsigned,
SignedBlsToExecutionChange, SignedContributionAndProof, SignedRoot, SignedVoluntaryExit,
SigningData, Slot, SyncAggregate, SyncAggregatorSelectionData, Unsigned,
};
pub type Result<T> = std::result::Result<T, Error>;
@ -156,6 +156,33 @@ where
))
}
pub fn bls_execution_change_signature_set<'a, T: EthSpec>(
state: &'a BeaconState<T>,
signed_address_change: &'a SignedBlsToExecutionChange,
spec: &'a ChainSpec,
) -> Result<SignatureSet<'a>> {
let domain = spec.get_domain(
state.current_epoch(),
Domain::BlsToExecutionChange,
&state.fork(),
state.genesis_validators_root(),
);
let message = signed_address_change.message.signing_root(domain);
let signing_key = Cow::Owned(
signed_address_change
.message
.from_bls_pubkey
.decompress()
.map_err(|_| Error::PublicKeyDecompressionFailed)?,
);
Ok(SignatureSet::single_pubkey(
&signed_address_change.signature,
signing_key,
message,
))
}
/// A signature set that is valid if the block proposers randao reveal signature is correct.
pub fn randao_signature_set<'a, T, F, Payload: AbstractExecPayload<T>>(
state: &'a BeaconState<T>,

View File

@ -11,7 +11,6 @@ pub use weigh_justification_and_finalization::weigh_justification_and_finalizati
pub mod altair;
pub mod base;
pub mod capella;
pub mod effective_balance_updates;
pub mod epoch_processing_summary;
pub mod errors;
@ -38,8 +37,10 @@ pub fn process_epoch<T: EthSpec>(
match state {
BeaconState::Base(_) => base::process_epoch(state, spec),
BeaconState::Altair(_) | BeaconState::Merge(_) => altair::process_epoch(state, spec),
BeaconState::Capella(_) | BeaconState::Eip4844(_) => capella::process_epoch(state, spec),
BeaconState::Altair(_)
| BeaconState::Merge(_)
| BeaconState::Capella(_)
| BeaconState::Eip4844(_) => altair::process_epoch(state, spec),
}
}

View File

@ -1,87 +0,0 @@
use super::{process_registry_updates, process_slashings, EpochProcessingSummary, Error};
use crate::per_epoch_processing::{
altair,
effective_balance_updates::process_effective_balance_updates,
historical_roots_update::process_historical_roots_update,
resets::{process_eth1_data_reset, process_randao_mixes_reset, process_slashings_reset},
};
#[cfg(all(feature = "withdrawals", feature = "withdrawals-processing"))]
pub use full_withdrawals::process_full_withdrawals;
#[cfg(all(feature = "withdrawals", feature = "withdrawals-processing"))]
pub use partial_withdrawals::process_partial_withdrawals;
use types::{BeaconState, ChainSpec, EthSpec, RelativeEpoch};
#[cfg(all(feature = "withdrawals", feature = "withdrawals-processing"))]
pub mod full_withdrawals;
#[cfg(all(feature = "withdrawals", feature = "withdrawals-processing"))]
pub mod partial_withdrawals;
pub fn process_epoch<T: EthSpec>(
state: &mut BeaconState<T>,
spec: &ChainSpec,
) -> Result<EpochProcessingSummary<T>, Error> {
// Ensure the committee caches are built.
state.build_committee_cache(RelativeEpoch::Previous, spec)?;
state.build_committee_cache(RelativeEpoch::Current, spec)?;
state.build_committee_cache(RelativeEpoch::Next, spec)?;
// Pre-compute participating indices and total balances.
let participation_cache = altair::ParticipationCache::new(state, spec)?;
let sync_committee = state.current_sync_committee()?.clone();
// Justification and finalization.
let justification_and_finalization_state =
altair::process_justification_and_finalization(state, &participation_cache)?;
justification_and_finalization_state.apply_changes_to_state(state);
altair::process_inactivity_updates(state, &participation_cache, spec)?;
// Rewards and Penalties.
altair::process_rewards_and_penalties(state, &participation_cache, spec)?;
// Registry Updates.
process_registry_updates(state, spec)?;
// Slashings.
process_slashings(
state,
participation_cache.current_epoch_total_active_balance(),
spec,
)?;
// Reset eth1 data votes.
process_eth1_data_reset(state)?;
// Update effective balances with hysteresis (lag).
process_effective_balance_updates(state, spec)?;
// Reset slashings
process_slashings_reset(state)?;
// Set randao mix
process_randao_mixes_reset(state)?;
// Set historical root accumulator
process_historical_roots_update(state)?;
// Rotate current/previous epoch participation
altair::process_participation_flag_updates(state)?;
altair::process_sync_committee_updates(state, spec)?;
// Withdrawals
#[cfg(all(feature = "withdrawals", feature = "withdrawals-processing"))]
process_full_withdrawals(state, spec)?;
#[cfg(all(feature = "withdrawals", feature = "withdrawals-processing"))]
process_partial_withdrawals(state, spec)?;
// Rotate the epoch caches to suit the epoch transition.
state.advance_caches(spec)?;
// FIXME: do we need a Capella variant for this?
Ok(EpochProcessingSummary::Altair {
participation_cache,
sync_committee,
})
}

View File

@ -1,25 +0,0 @@
#[cfg(all(feature = "withdrawals", feature = "withdrawals-processing"))]
use crate::common::withdraw_balance;
use crate::EpochProcessingError;
use types::{beacon_state::BeaconState, eth_spec::EthSpec, ChainSpec};
#[cfg(all(feature = "withdrawals", feature = "withdrawals-processing"))]
pub fn process_full_withdrawals<T: EthSpec>(
state: &mut BeaconState<T>,
spec: &ChainSpec,
) -> Result<(), EpochProcessingError> {
let current_epoch = state.current_epoch();
// FIXME: is this the most efficient way to do this?
for validator_index in 0..state.validators().len() {
// TODO: is this the correct way to handle validators not existing?
if let (Some(validator), Some(balance)) = (
state.validators().get(validator_index),
state.balances().get(validator_index),
) {
if validator.is_fully_withdrawable_at(*balance, current_epoch, spec) {
withdraw_balance(state, validator_index, *balance)?;
}
}
}
Ok(())
}

View File

@ -1,41 +0,0 @@
#[cfg(all(feature = "withdrawals", feature = "withdrawals-processing"))]
use crate::common::withdraw_balance;
use crate::EpochProcessingError;
use safe_arith::SafeArith;
use types::{beacon_state::BeaconState, eth_spec::EthSpec, ChainSpec};
#[cfg(all(feature = "withdrawals", feature = "withdrawals-processing"))]
pub fn process_partial_withdrawals<T: EthSpec>(
state: &mut BeaconState<T>,
spec: &ChainSpec,
) -> Result<(), EpochProcessingError> {
let mut partial_withdrawals_count = 0;
let mut validator_index = *state.next_partial_withdrawal_validator_index()? as usize;
let n_validators = state.validators().len();
// FIXME: is this the most efficient way to do this?
for _ in 0..n_validators {
// TODO: is this the correct way to handle validators not existing?
if let (Some(validator), Some(balance)) = (
state.validators().get(validator_index),
state.balances().get(validator_index),
) {
if validator.is_partially_withdrawable_validator(*balance, spec) {
withdraw_balance(
state,
validator_index,
*balance - spec.max_effective_balance,
)?;
partial_withdrawals_count.safe_add_assign(1)?;
validator_index = validator_index.safe_add(1)? % n_validators;
if partial_withdrawals_count == T::max_partial_withdrawals_per_epoch() {
break;
}
}
}
}
*state.next_partial_withdrawal_validator_index_mut()? = validator_index as u64;
Ok(())
}

View File

@ -11,7 +11,7 @@ pub fn upgrade_to_eip4844<E: EthSpec>(
// FIXME(sean) This is a hack to let us participate in testnets where capella doesn't exist.
// if we are disabling withdrawals, assume we should fork off of bellatrix.
let previous_fork_version = if cfg!(feature ="withdrawals") {
let previous_fork_version = if cfg!(feature = "withdrawals") {
pre.fork.current_version
} else {
spec.bellatrix_fork_version

View File

@ -0,0 +1,12 @@
# Mainnet preset - Capella
# Misc
# Max operations per block
# ---------------------------------------------------------------
# 2**4 (= 16)
MAX_BLS_TO_EXECUTION_CHANGES: 16
# Execution
# ---------------------------------------------------------------
# 2**4 (= 16) withdrawals
MAX_WITHDRAWALS_PER_PAYLOAD: 16

View File

@ -0,0 +1,12 @@
# Minimal preset - Capella
# Max operations per block
# ---------------------------------------------------------------
# 2**4 (= 16)
MAX_BLS_TO_EXECUTION_CHANGES: 16
# Execution
# ---------------------------------------------------------------
# [customized] 2**2 (= 4)
MAX_WITHDRAWALS_PER_PAYLOAD: 4

View File

@ -62,6 +62,10 @@ pub struct BeaconBlockBody<T: EthSpec, Payload: AbstractExecPayload<T> = FullPay
#[superstruct(only(Eip4844), partial_getter(rename = "execution_payload_eip4844"))]
#[serde(flatten)]
pub execution_payload: Payload::Eip4844,
#[cfg(feature = "withdrawals")]
#[superstruct(only(Capella, Eip4844))]
pub bls_to_execution_changes:
VariableList<SignedBlsToExecutionChange, T::MaxBlsToExecutionChanges>,
#[superstruct(only(Eip4844))]
pub blob_kzg_commitments: VariableList<KzgCommitment, T::MaxBlobsPerBlock>,
#[superstruct(only(Base, Altair))]

View File

@ -297,13 +297,10 @@ where
// Withdrawals
#[cfg(feature = "withdrawals")]
#[superstruct(only(Capella, Eip4844))]
pub withdrawal_queue: VariableList<Withdrawal, T::WithdrawalQueueLimit>,
#[cfg(feature = "withdrawals")]
#[superstruct(only(Capella, Eip4844))]
pub next_withdrawal_index: u64,
#[cfg(feature = "withdrawals")]
#[superstruct(only(Capella, Eip4844))]
pub next_partial_withdrawal_validator_index: u64,
pub latest_withdrawal_validator_index: u64,
// Caching (not in the spec)
#[serde(skip_serializing, skip_deserializing)]

View File

@ -98,8 +98,6 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq +
/*
* New in Capella
*/
type MaxPartialWithdrawalsPerEpoch: Unsigned + Clone + Sync + Send + Debug + PartialEq;
type WithdrawalQueueLimit: Unsigned + Clone + Sync + Send + Debug + PartialEq;
type MaxBlsToExecutionChanges: Unsigned + Clone + Sync + Send + Debug + PartialEq;
type MaxWithdrawalsPerPayload: Unsigned + Clone + Sync + Send + Debug + PartialEq;
/*
@ -235,16 +233,6 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq +
Self::BytesPerLogsBloom::to_usize()
}
/// Returns the `MAX_PARTIAL_WITHDRAWALS_PER_EPOCH` constant for this specification.
fn max_partial_withdrawals_per_epoch() -> usize {
Self::MaxPartialWithdrawalsPerEpoch::to_usize()
}
/// Returns the `WITHDRAWAL_QUEUE_LIMIT` constant for this specification.
fn withdrawal_queue_limit() -> usize {
Self::WithdrawalQueueLimit::to_usize()
}
/// Returns the `MAX_BLS_TO_EXECUTION_CHANGES` constant for this specification.
fn max_bls_to_execution_changes() -> usize {
Self::MaxBlsToExecutionChanges::to_usize()
@ -309,8 +297,6 @@ impl EthSpec for MainnetEthSpec {
type SyncSubcommitteeSize = U128; // 512 committee size / 4 sync committee subnet count
type MaxPendingAttestations = U4096; // 128 max attestations * 32 slots per epoch
type SlotsPerEth1VotingPeriod = U2048; // 64 epochs * 32 slots per epoch
type MaxPartialWithdrawalsPerEpoch = U256;
type WithdrawalQueueLimit = U1099511627776;
type MaxBlsToExecutionChanges = U16;
type MaxWithdrawalsPerPayload = U16;
@ -358,8 +344,6 @@ impl EthSpec for MinimalEthSpec {
GasLimitDenominator,
MinGasLimit,
MaxExtraDataBytes,
MaxPartialWithdrawalsPerEpoch,
WithdrawalQueueLimit,
MaxBlsToExecutionChanges,
MaxWithdrawalsPerPayload,
MaxBlobsPerBlock,
@ -408,8 +392,6 @@ impl EthSpec for GnosisEthSpec {
type SyncSubcommitteeSize = U128; // 512 committee size / 4 sync committee subnet count
type MaxPendingAttestations = U2048; // 128 max attestations * 16 slots per epoch
type SlotsPerEth1VotingPeriod = U1024; // 64 epochs * 16 slots per epoch
type MaxPartialWithdrawalsPerEpoch = U256;
type WithdrawalQueueLimit = U1099511627776;
type MaxBlsToExecutionChanges = U16;
type MaxWithdrawalsPerPayload = U16;
type MaxBlobsPerBlock = U16; // 2**4 = 16

View File

@ -3,7 +3,6 @@ use derivative::Derivative;
use serde_derive::{Deserialize, Serialize};
use ssz::Encode;
use ssz_derive::{Decode, Encode};
use std::slice::Iter;
use test_random_derive::TestRandom;
use tree_hash_derive::TreeHash;
@ -13,6 +12,8 @@ pub type Transactions<T> = VariableList<
<T as EthSpec>::MaxTransactionsPerPayload,
>;
pub type Withdrawals<T> = VariableList<Withdrawal, <T as EthSpec>::MaxWithdrawalsPerPayload>;
#[superstruct(
variants(Merge, Capella, Eip4844),
variant_attributes(
@ -82,7 +83,7 @@ pub struct ExecutionPayload<T: EthSpec> {
pub transactions: Transactions<T>,
#[cfg(feature = "withdrawals")]
#[superstruct(only(Capella, Eip4844))]
pub withdrawals: VariableList<Withdrawal, T::MaxWithdrawalsPerPayload>,
pub withdrawals: Withdrawals<T>,
}
impl<T: EthSpec> ExecutionPayload<T> {

View File

@ -27,6 +27,7 @@ pub mod beacon_block_body;
pub mod beacon_block_header;
pub mod beacon_committee;
pub mod beacon_state;
pub mod bls_to_execution_change;
pub mod builder_bid;
pub mod chain_spec;
pub mod checkpoint;
@ -61,6 +62,7 @@ pub mod shuffling_id;
pub mod signed_aggregate_and_proof;
pub mod signed_beacon_block;
pub mod signed_beacon_block_header;
pub mod signed_bls_to_execution_change;
pub mod signed_contribution_and_proof;
pub mod signed_voluntary_exit;
pub mod signing_data;
@ -117,6 +119,7 @@ pub use crate::beacon_block_header::BeaconBlockHeader;
pub use crate::beacon_committee::{BeaconCommittee, OwnedBeaconCommittee};
pub use crate::beacon_state::{BeaconTreeHashCache, Error as BeaconStateError, *};
pub use crate::blobs_sidecar::BlobsSidecar;
pub use crate::bls_to_execution_change::BlsToExecutionChange;
pub use crate::chain_spec::{ChainSpec, Config, Domain};
pub use crate::checkpoint::Checkpoint;
pub use crate::config_and_preset::{
@ -133,7 +136,7 @@ pub use crate::eth_spec::EthSpecId;
pub use crate::execution_block_hash::ExecutionBlockHash;
pub use crate::execution_payload::{
ExecutionPayload, ExecutionPayloadCapella, ExecutionPayloadEip4844, ExecutionPayloadMerge,
ExecutionPayloadRef, Transaction, Transactions,
ExecutionPayloadRef, Transaction, Transactions, Withdrawals,
};
pub use crate::execution_payload_header::{
ExecutionPayloadHeader, ExecutionPayloadHeaderCapella, ExecutionPayloadHeaderEip4844,
@ -170,6 +173,7 @@ pub use crate::signed_beacon_block::{
SignedBlindedBeaconBlock,
};
pub use crate::signed_beacon_block_header::SignedBeaconBlockHeader;
pub use crate::signed_bls_to_execution_change::SignedBlsToExecutionChange;
pub use crate::signed_contribution_and_proof::SignedContributionAndProof;
pub use crate::signed_voluntary_exit::SignedVoluntaryExit;
pub use crate::signing_data::{SignedRoot, SigningData};

View File

@ -36,6 +36,9 @@ pub trait ExecPayload<T: EthSpec>: Debug + Clone + PartialEq + Hash + TreeHash +
fn fee_recipient(&self) -> Address;
fn gas_limit(&self) -> u64;
fn transactions(&self) -> Option<&Transactions<T>>;
/// fork-specific fields
#[cfg(feature = "withdrawals")]
fn withdrawals(&self) -> Option<Result<&Withdrawals<T>, Error>>;
/// Is this a default payload? (pre-merge)
fn is_default(&self) -> bool;
@ -225,6 +228,15 @@ impl<T: EthSpec> ExecPayload<T> for FullPayload<T> {
})
}
#[cfg(feature = "withdrawals")]
fn withdrawals(&self) -> Option<Result<&Withdrawals<T>, Error>> {
match self {
FullPayload::Merge(_) => Some(Err(Error::IncorrectStateVariant)),
FullPayload::Capella(ref inner) => Some(Ok(&inner.execution_payload.withdrawals)),
FullPayload::Eip4844(ref inner) => Some(Ok(&inner.execution_payload.withdrawals)),
}
}
fn is_default<'a>(&'a self) -> bool {
map_full_payload_ref!(&'a _, self.to_ref(), move |payload, cons| {
cons(payload);
@ -309,6 +321,15 @@ impl<'b, T: EthSpec> ExecPayload<T> for FullPayloadRef<'b, T> {
})
}
#[cfg(feature = "withdrawals")]
fn withdrawals(&self) -> Option<Result<&Withdrawals<T>, Error>> {
match self {
FullPayloadRef::Merge(_inner) => Some(Err(Error::IncorrectStateVariant)),
FullPayloadRef::Capella(inner) => Some(Ok(&inner.execution_payload.withdrawals)),
FullPayloadRef::Eip4844(inner) => Some(Ok(&inner.execution_payload.withdrawals)),
}
}
// TODO: can this function be optimized?
fn is_default<'a>(&'a self) -> bool {
map_full_payload_ref!(&'a _, self, move |payload, cons| {
@ -463,6 +484,11 @@ impl<T: EthSpec> ExecPayload<T> for BlindedPayload<T> {
None
}
#[cfg(feature = "withdrawals")]
fn withdrawals(&self) -> Option<Result<&Withdrawals<T>, Error>> {
None
}
fn is_default<'a>(&'a self) -> bool {
map_blinded_payload_ref!(&'a _, self.to_ref(), move |payload, cons| {
cons(payload);
@ -536,6 +562,11 @@ impl<'b, T: EthSpec> ExecPayload<T> for BlindedPayloadRef<'b, T> {
None
}
#[cfg(feature = "withdrawals")]
fn withdrawals<'a>(&'a self) -> Option<Result<&Withdrawals<T>, Error>> {
None
}
// TODO: can this function be optimized?
fn is_default<'a>(&'a self) -> bool {
map_blinded_payload_ref!(&'a _, self, move |payload, cons| {
@ -546,7 +577,7 @@ impl<'b, T: EthSpec> ExecPayload<T> for BlindedPayloadRef<'b, T> {
}
macro_rules! impl_exec_payload_common {
($wrapper_type:ident, $wrapped_type_full:ident, $wrapped_header_type:ident, $wrapped_field:ident, $fork_variant:ident, $block_type_variant:ident, $f:block) => {
($wrapper_type:ident, $wrapped_type_full:ident, $wrapped_header_type:ident, $wrapped_field:ident, $fork_variant:ident, $block_type_variant:ident, $f:block, $g:block) => {
impl<T: EthSpec> ExecPayload<T> for $wrapper_type<T> {
fn block_type() -> BlockType {
BlockType::$block_type_variant
@ -594,6 +625,12 @@ macro_rules! impl_exec_payload_common {
let f = $f;
f(self)
}
#[cfg(feature = "withdrawals")]
fn withdrawals(&self) -> Option<Result<&Withdrawals<T>, Error>> {
let g = $g;
g(self)
}
}
impl<T: EthSpec> From<$wrapped_type_full<T>> for $wrapper_type<T> {
@ -605,7 +642,7 @@ macro_rules! impl_exec_payload_common {
}
macro_rules! impl_exec_payload_for_fork {
($wrapper_type_header:ident, $wrapper_type_full:ident, $wrapped_type_header:ident, $wrapped_type_full:ident, $fork_variant:ident) => {
($wrapper_type_header:ident, $wrapper_type_full:ident, $wrapped_type_header:ident, $wrapped_type_full:ident, $fork_variant:ident, $withdrawal_fn:block) => {
//*************** Blinded payload implementations ******************//
impl_exec_payload_common!(
@ -615,6 +652,7 @@ macro_rules! impl_exec_payload_for_fork {
execution_payload_header,
$fork_variant,
Blinded,
{ |_| { None } },
{ |_| { None } }
);
@ -680,7 +718,8 @@ macro_rules! impl_exec_payload_for_fork {
let c: for<'a> fn(&'a $wrapper_type_full<T>) -> Option<&'a Transactions<T>> =
|payload: &$wrapper_type_full<T>| Some(&payload.execution_payload.transactions);
c
}
},
$withdrawal_fn
);
impl<T: EthSpec> Default for $wrapper_type_full<T> {
@ -723,21 +762,36 @@ impl_exec_payload_for_fork!(
FullPayloadMerge,
ExecutionPayloadHeaderMerge,
ExecutionPayloadMerge,
Merge
Merge,
{
let c: for<'a> fn(&'a FullPayloadMerge<T>) -> Option<Result<&'a Withdrawals<T>, Error>> =
|_| Some(Err(Error::IncorrectStateVariant));
c
}
);
impl_exec_payload_for_fork!(
BlindedPayloadCapella,
FullPayloadCapella,
ExecutionPayloadHeaderCapella,
ExecutionPayloadCapella,
Capella
Capella,
{
let c: for<'a> fn(&'a FullPayloadCapella<T>) -> Option<Result<&Withdrawals<T>, Error>> =
|payload: &FullPayloadCapella<T>| Some(Ok(&payload.execution_payload.withdrawals));
c
}
);
impl_exec_payload_for_fork!(
BlindedPayloadEip4844,
FullPayloadEip4844,
ExecutionPayloadHeaderEip4844,
ExecutionPayloadEip4844,
Eip4844
Eip4844,
{
let c: for<'a> fn(&'a FullPayloadEip4844<T>) -> Option<Result<&Withdrawals<T>, Error>> =
|payload: &FullPayloadEip4844<T>| Some(Ok(&payload.execution_payload.withdrawals));
c
}
);
impl<T: EthSpec> AbstractExecPayload<T> for BlindedPayload<T> {

View File

@ -1,5 +1,6 @@
use crate::{
test_utils::TestRandom, BeaconState, ChainSpec, Epoch, EthSpec, Hash256, PublicKeyBytes,
test_utils::TestRandom, Address, BeaconState, BlsToExecutionChange, ChainSpec, Epoch, EthSpec,
Hash256, PublicKeyBytes,
};
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
@ -75,6 +76,16 @@ impl Validator {
.unwrap_or(false)
}
/// Changes withdrawal credentials to the provided eth1 execution address
///
/// WARNING: this function does NO VALIDATION - it just does it!
pub fn change_withdrawal_credentials(&mut self, execution_address: &Address, spec: &ChainSpec) {
let mut bytes = [0u8; 32];
bytes[0] = spec.eth1_address_withdrawal_prefix_byte;
bytes[12..].copy_from_slice(execution_address.as_bytes());
self.withdrawal_credentials = Hash256::from(bytes);
}
/// Returns `true` if the validator is fully withdrawable at some epoch
pub fn is_fully_withdrawable_at(&self, balance: u64, epoch: Epoch, spec: &ChainSpec) -> bool {
self.has_eth1_withdrawal_credential(spec) && self.withdrawable_epoch <= epoch && balance > 0