Documentation ¶
Index ¶
- Variables
- func AddValidatorToRegistry(beaconState state.BeaconState, pubKey []byte, withdrawalCredentials []byte, ...) error
- func ApplyDeposit(beaconState state.BeaconState, data *ethpb.Deposit_Data, ...) (state.BeaconState, error)
- func ApplyPendingDeposit(ctx context.Context, st state.BeaconState, deposit *ethpb.PendingDeposit) error
- func ComputeConsolidationEpochAndUpdateChurn(ctx context.Context, s state.BeaconState, consolidationBalance primitives.Gwei) (primitives.Epoch, error)
- func GetValidatorFromDeposit(pubKey []byte, withdrawalCredentials []byte, amount uint64) (*ethpb.Validator, error)
- func IsValidDepositSignature(data *ethpb.Deposit_Data) (bool, error)
- func IsValidSwitchToCompoundingRequest(st state.BeaconState, req *enginev1.ConsolidationRequest) bool
- func ProcessConsolidationRequests(ctx context.Context, st state.BeaconState, ...) error
- func ProcessDeposit(beaconState state.BeaconState, deposit *ethpb.Deposit, ...) (state.BeaconState, error)
- func ProcessDepositRequests(ctx context.Context, beaconState state.BeaconState, ...) (state.BeaconState, error)
- func ProcessDeposits(ctx context.Context, beaconState state.BeaconState, deposits []*ethpb.Deposit) (state.BeaconState, error)
- func ProcessEffectiveBalanceUpdates(st state.BeaconState) error
- func ProcessEpoch(ctx context.Context, state state.BeaconState) error
- func ProcessOperations(ctx context.Context, st state.BeaconState, ...) (state.BeaconState, error)
- func ProcessPendingConsolidations(ctx context.Context, st state.BeaconState) error
- func ProcessPendingDeposits(ctx context.Context, st state.BeaconState, activeBalance primitives.Gwei) error
- func ProcessRegistryUpdates(ctx context.Context, st state.BeaconState) error
- func ProcessWithdrawalRequests(ctx context.Context, st state.BeaconState, wrs []*enginev1.WithdrawalRequest) (state.BeaconState, error)
- func QueueEntireBalanceAndResetValidator(s state.BeaconState, idx primitives.ValidatorIndex) error
- func QueueExcessActiveBalance(s state.BeaconState, idx primitives.ValidatorIndex) error
- func SwitchToCompoundingValidator(s state.BeaconState, idx primitives.ValidatorIndex) error
- func UpgradeToElectra(beaconState state.BeaconState) (state.BeaconState, error)
- func VerifyBlockDepositLength(body interfaces.ReadOnlyBeaconBlockBody, state state.BeaconState) error
Constants ¶
This section is empty.
Variables ¶
var ( InitializePrecomputeValidators = altair.InitializePrecomputeValidators ProcessEpochParticipation = altair.ProcessEpochParticipation ProcessInactivityScores = altair.ProcessInactivityScores ProcessRewardsAndPenaltiesPrecompute = altair.ProcessRewardsAndPenaltiesPrecompute ProcessSlashings = e.ProcessSlashings ProcessEth1DataReset = e.ProcessEth1DataReset ProcessSlashingsReset = e.ProcessSlashingsReset ProcessRandaoMixesReset = e.ProcessRandaoMixesReset ProcessHistoricalDataUpdate = e.ProcessHistoricalDataUpdate ProcessParticipationFlagUpdates = altair.ProcessParticipationFlagUpdates ProcessSyncCommitteeUpdates = altair.ProcessSyncCommitteeUpdates AttestationsDelta = altair.AttestationsDelta )
Re-exports for methods that haven't changed in Electra.
var ( ProcessBLSToExecutionChanges = blocks.ProcessBLSToExecutionChanges ProcessVoluntaryExits = blocks.ProcessVoluntaryExits ProcessAttesterSlashings = blocks.ProcessAttesterSlashings ProcessProposerSlashings = blocks.ProcessProposerSlashings )
var (
ProcessAttestationsNoVerifySignature = altair.ProcessAttestationsNoVerifySignature
)
Functions ¶
func AddValidatorToRegistry ¶
func AddValidatorToRegistry(beaconState state.BeaconState, pubKey []byte, withdrawalCredentials []byte, amount uint64) error
AddValidatorToRegistry updates the beacon state with validator information def add_validator_to_registry(state: BeaconState, pubkey: BLSPubkey, withdrawal_credentials: Bytes32, amount: uint64) -> None:
index = get_index_for_new_validator(state) validator = get_validator_from_deposit(pubkey, withdrawal_credentials, amount) # [Modified in Electra:EIP7251] set_or_append_list(state.validators, index, validator) set_or_append_list(state.balances, index, amount) set_or_append_list(state.previous_epoch_participation, index, ParticipationFlags(0b0000_0000)) set_or_append_list(state.current_epoch_participation, index, ParticipationFlags(0b0000_0000)) set_or_append_list(state.inactivity_scores, index, uint64(0))
func ApplyDeposit ¶
func ApplyDeposit(beaconState state.BeaconState, data *ethpb.Deposit_Data, allSignaturesVerified bool) (state.BeaconState, error)
ApplyDeposit adds the incoming deposit as a pending deposit on the state
Spec pseudocode definition: def apply_deposit(state: BeaconState,
pubkey: BLSPubkey, withdrawal_credentials: Bytes32, amount: uint64, signature: BLSSignature) -> None: validator_pubkeys = [v.pubkey for v in state.validators] if pubkey not in validator_pubkeys: # Verify the deposit signature (proof of possession) which is not checked by the deposit contract if is_valid_deposit_signature(pubkey, withdrawal_credentials, amount, signature): add_validator_to_registry(state, pubkey, withdrawal_credentials, Gwei(0)) # [Modified in Electra:EIP7251] # [New in Electra:EIP7251] state.pending_deposits.append(PendingDeposit( pubkey=pubkey, withdrawal_credentials=withdrawal_credentials, amount=amount, signature=signature, slot=GENESIS_SLOT, # Use GENESIS_SLOT to distinguish from a pending deposit request )) else: # Increase balance by deposit amount # [Modified in Electra:EIP7251] state.pending_deposits.append(PendingDeposit( pubkey=pubkey, withdrawal_credentials=withdrawal_credentials, amount=amount, signature=signature, slot=GENESIS_SLOT # Use GENESIS_SLOT to distinguish from a pending deposit request ))
func ApplyPendingDeposit ¶ added in v5.2.0
func ApplyPendingDeposit(ctx context.Context, st state.BeaconState, deposit *ethpb.PendingDeposit) error
ApplyPendingDeposit implements the spec definition below. Note : This function is NOT used by ProcessPendingDeposits due to simplified logic for more readable batch processing
Spec Definition:
def apply_pending_deposit(state: BeaconState, deposit: PendingDeposit) -> None:
""" Applies ``deposit`` to the ``state``. """ validator_pubkeys = [v.pubkey for v in state.validators] if deposit.pubkey not in validator_pubkeys: # Verify the deposit signature (proof of possession) which is not checked by the deposit contract if is_valid_deposit_signature( deposit.pubkey, deposit.withdrawal_credentials, deposit.amount, deposit.signature ): add_validator_to_registry(state, deposit.pubkey, deposit.withdrawal_credentials, deposit.amount) else: validator_index = ValidatorIndex(validator_pubkeys.index(deposit.pubkey)) # Increase balance increase_balance(state, validator_index, deposit.amount)
func ComputeConsolidationEpochAndUpdateChurn ¶
func ComputeConsolidationEpochAndUpdateChurn(ctx context.Context, s state.BeaconState, consolidationBalance primitives.Gwei) (primitives.Epoch, error)
ComputeConsolidationEpochAndUpdateChurn fulfills the consensus spec definition below. This method calls mutating methods to the beacon state.
Spec definition:
def compute_consolidation_epoch_and_update_churn(state: BeaconState, consolidation_balance: Gwei) -> Epoch: earliest_consolidation_epoch = max( state.earliest_consolidation_epoch, compute_activation_exit_epoch(get_current_epoch(state))) per_epoch_consolidation_churn = get_consolidation_churn_limit(state) # New epoch for consolidations. if state.earliest_consolidation_epoch < earliest_consolidation_epoch: consolidation_balance_to_consume = per_epoch_consolidation_churn else: consolidation_balance_to_consume = state.consolidation_balance_to_consume # Consolidation doesn't fit in the current earliest epoch. if consolidation_balance > consolidation_balance_to_consume: balance_to_process = consolidation_balance - consolidation_balance_to_consume additional_epochs = (balance_to_process - 1) // per_epoch_consolidation_churn + 1 earliest_consolidation_epoch += additional_epochs consolidation_balance_to_consume += additional_epochs * per_epoch_consolidation_churn # Consume the balance and update state variables. state.consolidation_balance_to_consume = consolidation_balance_to_consume - consolidation_balance state.earliest_consolidation_epoch = earliest_consolidation_epoch return state.earliest_consolidation_epoch
func GetValidatorFromDeposit ¶ added in v5.2.0
func GetValidatorFromDeposit(pubKey []byte, withdrawalCredentials []byte, amount uint64) (*ethpb.Validator, error)
GetValidatorFromDeposit gets a new validator object with provided parameters
def get_validator_from_deposit(pubkey: BLSPubkey, withdrawal_credentials: Bytes32, amount: uint64) -> Validator:
validator = Validator( pubkey=pubkey, withdrawal_credentials=withdrawal_credentials, effective_balance=Gwei(0), slashed=False, activation_eligibility_epoch=FAR_FUTURE_EPOCH, activation_epoch=FAR_FUTURE_EPOCH, exit_epoch=FAR_FUTURE_EPOCH, withdrawable_epoch=FAR_FUTURE_EPOCH, ) # [Modified in Electra:EIP7251] max_effective_balance = get_max_effective_balance(validator) validator.effective_balance = min(amount - amount % EFFECTIVE_BALANCE_INCREMENT, max_effective_balance) return validator
func IsValidDepositSignature ¶
func IsValidDepositSignature(data *ethpb.Deposit_Data) (bool, error)
IsValidDepositSignature returns whether deposit_data is valid def is_valid_deposit_signature(pubkey: BLSPubkey, withdrawal_credentials: Bytes32, amount: uint64, signature: BLSSignature) -> bool:
deposit_message = DepositMessage( pubkey=pubkey, withdrawal_credentials=withdrawal_credentials, amount=amount, ) domain = compute_domain(DOMAIN_DEPOSIT) # Fork-agnostic domain since deposits are valid across forks signing_root = compute_signing_root(deposit_message, domain) return bls.Verify(pubkey, signing_root, signature)
func IsValidSwitchToCompoundingRequest ¶ added in v5.2.0
func IsValidSwitchToCompoundingRequest(st state.BeaconState, req *enginev1.ConsolidationRequest) bool
IsValidSwitchToCompoundingRequest returns true if the given consolidation request is valid for switching to compounding.
Spec code:
def is_valid_switch_to_compounding_request(
state: BeaconState, consolidation_request: ConsolidationRequest
) -> bool:
# Switch to compounding requires source and target be equal if consolidation_request.source_pubkey != consolidation_request.target_pubkey: return False # Verify pubkey exists source_pubkey = consolidation_request.source_pubkey validator_pubkeys = [v.pubkey for v in state.validators] if source_pubkey not in validator_pubkeys: return False source_validator = state.validators[ValidatorIndex(validator_pubkeys.index(source_pubkey))] # Verify request has been authorized if source_validator.withdrawal_credentials[12:] != consolidation_request.source_address: return False # Verify source withdrawal credentials if not has_eth1_withdrawal_credential(source_validator): return False # Verify the source is active current_epoch = get_current_epoch(state) if not is_active_validator(source_validator, current_epoch): return False # Verify exit for source has not been initiated if source_validator.exit_epoch != FAR_FUTURE_EPOCH: return False return True
func ProcessConsolidationRequests ¶
func ProcessConsolidationRequests(ctx context.Context, st state.BeaconState, reqs []*enginev1.ConsolidationRequest) error
ProcessConsolidationRequests implements the spec definition below. This method makes mutating calls to the beacon state.
def process_consolidation_request( state: BeaconState, consolidation_request: ConsolidationRequest ) -> None: if is_valid_switch_to_compounding_request(state, consolidation_request): validator_pubkeys = [v.pubkey for v in state.validators] request_source_pubkey = consolidation_request.source_pubkey source_index = ValidatorIndex(validator_pubkeys.index(request_source_pubkey)) switch_to_compounding_validator(state, source_index) return # Verify that source != target, so a consolidation cannot be used as an exit. if consolidation_request.source_pubkey == consolidation_request.target_pubkey: return # If the pending consolidations queue is full, consolidation requests are ignored if len(state.pending_consolidations) == PENDING_CONSOLIDATIONS_LIMIT: return # If there is too little available consolidation churn limit, consolidation requests are ignored if get_consolidation_churn_limit(state) <= MIN_ACTIVATION_BALANCE: return validator_pubkeys = [v.pubkey for v in state.validators] # Verify pubkeys exists request_source_pubkey = consolidation_request.source_pubkey request_target_pubkey = consolidation_request.target_pubkey if request_source_pubkey not in validator_pubkeys: return if request_target_pubkey not in validator_pubkeys: return source_index = ValidatorIndex(validator_pubkeys.index(request_source_pubkey)) target_index = ValidatorIndex(validator_pubkeys.index(request_target_pubkey)) source_validator = state.validators[source_index] target_validator = state.validators[target_index] # Verify source withdrawal credentials has_correct_credential = has_execution_withdrawal_credential(source_validator) is_correct_source_address = ( source_validator.withdrawal_credentials[12:] == consolidation_request.source_address ) if not (has_correct_credential and is_correct_source_address): return # Verify that target has execution withdrawal credentials if not has_execution_withdrawal_credential(target_validator): return # Verify the source and the target are active current_epoch = get_current_epoch(state) if not is_active_validator(source_validator, current_epoch): return if not is_active_validator(target_validator, current_epoch): return # Verify exits for source and target have not been initiated if source_validator.exit_epoch != FAR_FUTURE_EPOCH: return if target_validator.exit_epoch != FAR_FUTURE_EPOCH: return # Verify the source has been active long enough if current_epoch < source_validator.activation_epoch + SHARD_COMMITTEE_PERIOD: return # Verify the source has no pending withdrawals in the queue if get_pending_balance_to_withdraw(state, source_index) > 0: return # Initiate source validator exit and append pending consolidation source_validator.exit_epoch = compute_consolidation_epoch_and_update_churn( state, source_validator.effective_balance ) source_validator.withdrawable_epoch = Epoch( source_validator.exit_epoch + MIN_VALIDATOR_WITHDRAWABILITY_DELAY ) state.pending_consolidations.append(PendingConsolidation( source_index=source_index, target_index=target_index )) # Churn any target excess active balance of target and raise its max if has_eth1_withdrawal_credential(target_validator): switch_to_compounding_validator(state, target_index)
func ProcessDeposit ¶
func ProcessDeposit(beaconState state.BeaconState, deposit *ethpb.Deposit, allSignaturesVerified bool) (state.BeaconState, error)
ProcessDeposit takes in a deposit object and inserts it into the registry as a new validator or balance change. Returns the resulting state, a boolean to indicate whether or not the deposit resulted in a new validator entry into the beacon state, and any error.
Spec pseudocode definition: def process_deposit(state: BeaconState, deposit: Deposit) -> None:
# Verify the Merkle branch assert is_valid_merkle_branch( leaf=hash_tree_root(deposit.data), branch=deposit.proof, depth=DEPOSIT_CONTRACT_TREE_DEPTH + 1, # Add 1 for the List length mix-in index=state.eth1_deposit_index, root=state.eth1_data.deposit_root, ) # Deposits must be processed in order state.eth1_deposit_index += 1 apply_deposit( state=state, pubkey=deposit.data.pubkey, withdrawal_credentials=deposit.data.withdrawal_credentials, amount=deposit.data.amount, signature=deposit.data.signature, )
func ProcessDepositRequests ¶
func ProcessDepositRequests(ctx context.Context, beaconState state.BeaconState, requests []*enginev1.DepositRequest) (state.BeaconState, error)
ProcessDepositRequests is a function as part of electra to process execution layer deposits
func ProcessDeposits ¶
func ProcessDeposits( ctx context.Context, beaconState state.BeaconState, deposits []*ethpb.Deposit, ) (state.BeaconState, error)
ProcessDeposits is one of the operations performed on each processed beacon block to verify queued validators from the Ethereum 1.0 Deposit Contract into the beacon chain.
Spec pseudocode definition:
For each deposit in block.body.deposits: process_deposit(state, deposit)
func ProcessEffectiveBalanceUpdates ¶
func ProcessEffectiveBalanceUpdates(st state.BeaconState) error
ProcessEffectiveBalanceUpdates processes effective balance updates during epoch processing.
Spec pseudocode definition:
def process_effective_balance_updates(state: BeaconState) -> None: # Update effective balances with hysteresis for index, validator in enumerate(state.validators): balance = state.balances[index] HYSTERESIS_INCREMENT = uint64(EFFECTIVE_BALANCE_INCREMENT // HYSTERESIS_QUOTIENT) DOWNWARD_THRESHOLD = HYSTERESIS_INCREMENT * HYSTERESIS_DOWNWARD_MULTIPLIER UPWARD_THRESHOLD = HYSTERESIS_INCREMENT * HYSTERESIS_UPWARD_MULTIPLIER EFFECTIVE_BALANCE_LIMIT = ( MAX_EFFECTIVE_BALANCE_EIP7251 if has_compounding_withdrawal_credential(validator) else MIN_ACTIVATION_BALANCE ) if ( balance + DOWNWARD_THRESHOLD < validator.effective_balance or validator.effective_balance + UPWARD_THRESHOLD < balance ): validator.effective_balance = min(balance - balance % EFFECTIVE_BALANCE_INCREMENT, EFFECTIVE_BALANCE_LIMIT)
func ProcessEpoch ¶
func ProcessEpoch(ctx context.Context, state state.BeaconState) error
ProcessEpoch describes the per epoch operations that are performed on the beacon state. It's optimized by pre computing validator attested info and epoch total/attested balances upfront.
Spec definition:
def process_epoch(state: BeaconState) -> None: process_justification_and_finalization(state) process_inactivity_updates(state) process_rewards_and_penalties(state) process_registry_updates(state) process_slashings(state) process_eth1_data_reset(state) process_pending_deposits(state) # New in EIP7251 process_pending_consolidations(state) # New in EIP7251 process_effective_balance_updates(state) process_slashings_reset(state) process_randao_mixes_reset(state)
func ProcessOperations ¶
func ProcessOperations( ctx context.Context, st state.BeaconState, block interfaces.ReadOnlyBeaconBlock) (state.BeaconState, error)
func ProcessPendingConsolidations ¶
func ProcessPendingConsolidations(ctx context.Context, st state.BeaconState) error
ProcessPendingConsolidations implements the spec definition below. This method makes mutating calls to the beacon state.
Spec definition:
def process_pending_consolidations(state: BeaconState) -> None:
next_epoch = Epoch(get_current_epoch(state) + 1) next_pending_consolidation = 0 for pending_consolidation in state.pending_consolidations: source_validator = state.validators[pending_consolidation.source_index] if source_validator.slashed: next_pending_consolidation += 1 continue if source_validator.withdrawable_epoch > next_epoch: break # Calculate the consolidated balance max_effective_balance = get_max_effective_balance(source_validator) source_effective_balance = min(state.balances[pending_consolidation.source_index], max_effective_balance) # Move active balance to target. Excess balance is withdrawable. decrease_balance(state, pending_consolidation.source_index, source_effective_balance) increase_balance(state, pending_consolidation.target_index, source_effective_balance) next_pending_consolidation += 1 state.pending_consolidations = state.pending_consolidations[next_pending_consolidation:]
func ProcessPendingDeposits ¶ added in v5.2.0
func ProcessPendingDeposits(ctx context.Context, st state.BeaconState, activeBalance primitives.Gwei) error
ProcessPendingDeposits implements the spec definition below. This method mutates the state. Iterating over `pending_deposits` queue this function runs the following checks before applying pending deposit: 1. All Eth1 bridge deposits are processed before the first deposit request gets processed. 2. Deposit position in the queue is finalized. 3. Deposit does not exceed the `MAX_PENDING_DEPOSITS_PER_EPOCH` limit. 4. Deposit does not exceed the activation churn limit.
Spec definition:
def process_pending_deposits(state: BeaconState) -> None:
next_epoch = Epoch(get_current_epoch(state) + 1) available_for_processing = state.deposit_balance_to_consume + get_activation_exit_churn_limit(state) processed_amount = 0 next_deposit_index = 0 deposits_to_postpone = [] is_churn_limit_reached = False finalized_slot = compute_start_slot_at_epoch(state.finalized_checkpoint.epoch) for deposit in state.pending_deposits: # Do not process deposit requests if Eth1 bridge deposits are not yet applied. if ( # Is deposit request deposit.slot > GENESIS_SLOT and # There are pending Eth1 bridge deposits state.eth1_deposit_index < state.deposit_requests_start_index ): break # Check if deposit has been finalized, otherwise, stop processing. if deposit.slot > finalized_slot: break # Check if number of processed deposits has not reached the limit, otherwise, stop processing. if next_deposit_index >= MAX_PENDING_DEPOSITS_PER_EPOCH: break # Read validator state is_validator_exited = False is_validator_withdrawn = False validator_pubkeys = [v.pubkey for v in state.validators] if deposit.pubkey in validator_pubkeys: validator = state.validators[ValidatorIndex(validator_pubkeys.index(deposit.pubkey))] is_validator_exited = validator.exit_epoch < FAR_FUTURE_EPOCH is_validator_withdrawn = validator.withdrawable_epoch < next_epoch if is_validator_withdrawn: # Deposited balance will never become active. Increase balance but do not consume churn apply_pending_deposit(state, deposit) elif is_validator_exited: # Validator is exiting, postpone the deposit until after withdrawable epoch deposits_to_postpone.append(deposit) else: # Check if deposit fits in the churn, otherwise, do no more deposit processing in this epoch. is_churn_limit_reached = processed_amount + deposit.amount > available_for_processing if is_churn_limit_reached: break # Consume churn and apply deposit. processed_amount += deposit.amount apply_pending_deposit(state, deposit) # Regardless of how the deposit was handled, we move on in the queue. next_deposit_index += 1 state.pending_deposits = state.pending_deposits[next_deposit_index:] + deposits_to_postpone # Accumulate churn only if the churn limit has been hit. if is_churn_limit_reached: state.deposit_balance_to_consume = available_for_processing - processed_amount else: state.deposit_balance_to_consume = Gwei(0)
func ProcessRegistryUpdates ¶
func ProcessRegistryUpdates(ctx context.Context, st state.BeaconState) error
ProcessRegistryUpdates processes all validators eligible for the activation queue, all validators which should be ejected, and all validators which are eligible for activation from the queue.
Spec pseudocode definition:
def process_registry_updates(state: BeaconState) -> None: # Process activation eligibility and ejections for index, validator in enumerate(state.validators): if is_eligible_for_activation_queue(validator): validator.activation_eligibility_epoch = get_current_epoch(state) + 1 if ( is_active_validator(validator, get_current_epoch(state)) and validator.effective_balance <= EJECTION_BALANCE ): initiate_validator_exit(state, ValidatorIndex(index)) # Activate all eligible validators activation_epoch = compute_activation_exit_epoch(get_current_epoch(state)) for validator in state.validators: if is_eligible_for_activation(state, validator): validator.activation_epoch = activation_epoch
func ProcessWithdrawalRequests ¶
func ProcessWithdrawalRequests(ctx context.Context, st state.BeaconState, wrs []*enginev1.WithdrawalRequest) (state.BeaconState, error)
ProcessWithdrawalRequests processes the validator withdrawals from the provided execution payload into the beacon state triggered by the execution layer.
Spec pseudocode definition:
def process_withdrawal_request(
state: BeaconState, withdrawal_request: WithdrawalRequest
) -> None:
amount = withdrawal_request.amount is_full_exit_request = amount == FULL_EXIT_REQUEST_AMOUNT # If partial withdrawal queue is full, only full exits are processed if len(state.pending_partial_withdrawals) == PENDING_PARTIAL_WITHDRAWALS_LIMIT and not is_full_exit_request: return validator_pubkeys = [v.pubkey for v in state.validators] # Verify pubkey exists request_pubkey = withdrawal_request.validator_pubkey if request_pubkey not in validator_pubkeys: return index = ValidatorIndex(validator_pubkeys.index(request_pubkey)) validator = state.validators[index] # Verify withdrawal credentials has_correct_credential = has_execution_withdrawal_credential(validator) is_correct_source_address = ( validator.withdrawal_credentials[12:] == withdrawal_request.source_address ) if not (has_correct_credential and is_correct_source_address): return # Verify the validator is active if not is_active_validator(validator, get_current_epoch(state)): return # Verify exit has not been initiated if validator.exit_epoch != FAR_FUTURE_EPOCH: return # Verify the validator has been active long enough if get_current_epoch(state) < validator.activation_epoch + SHARD_COMMITTEE_PERIOD: return pending_balance_to_withdraw = get_pending_balance_to_withdraw(state, index) if is_full_exit_request: # Only exit validator if it has no pending withdrawals in the queue if pending_balance_to_withdraw == 0: initiate_validator_exit(state, index) return has_sufficient_effective_balance = validator.effective_balance >= MIN_ACTIVATION_BALANCE has_excess_balance = state.balances[index] > MIN_ACTIVATION_BALANCE + pending_balance_to_withdraw # Only allow partial withdrawals with compounding withdrawal credentials if has_compounding_withdrawal_credential(validator) and has_sufficient_effective_balance and has_excess_balance: to_withdraw = min( state.balances[index] - MIN_ACTIVATION_BALANCE - pending_balance_to_withdraw, amount ) exit_queue_epoch = compute_exit_epoch_and_update_churn(state, to_withdraw) withdrawable_epoch = Epoch(exit_queue_epoch + MIN_VALIDATOR_WITHDRAWABILITY_DELAY) state.pending_partial_withdrawals.append(PendingPartialWithdrawal( index=index, amount=to_withdraw, withdrawable_epoch=withdrawable_epoch, ))
func QueueEntireBalanceAndResetValidator ¶
func QueueEntireBalanceAndResetValidator(s state.BeaconState, idx primitives.ValidatorIndex) error
QueueEntireBalanceAndResetValidator queues the entire balance and resets the validator. This is used in electra fork logic.
Spec definition:
def queue_entire_balance_and_reset_validator(state: BeaconState, index: ValidatorIndex) -> None:
balance = state.balances[index] state.balances[index] = 0 validator = state.validators[index] validator.effective_balance = 0 validator.activation_eligibility_epoch = FAR_FUTURE_EPOCH state.pending_deposits.append(PendingDeposit( pubkey=validator.pubkey, withdrawal_credentials=validator.withdrawal_credentials, amount=balance, signature=bls.G2_POINT_AT_INFINITY, slot=GENESIS_SLOT,
))
func QueueExcessActiveBalance ¶
func QueueExcessActiveBalance(s state.BeaconState, idx primitives.ValidatorIndex) error
QueueExcessActiveBalance queues validators with balances above the min activation balance and adds to pending deposit.
Spec definition:
def queue_excess_active_balance(state: BeaconState, index: ValidatorIndex) -> None:
balance = state.balances[index] if balance > MIN_ACTIVATION_BALANCE: excess_balance = balance - MIN_ACTIVATION_BALANCE state.balances[index] = MIN_ACTIVATION_BALANCE validator = state.validators[index] state.pending_deposits.append(PendingDeposit( pubkey=validator.pubkey, withdrawal_credentials=validator.withdrawal_credentials, amount=excess_balance, signature=bls.G2_POINT_AT_INFINITY, slot=GENESIS_SLOT, ))
func SwitchToCompoundingValidator ¶
func SwitchToCompoundingValidator(s state.BeaconState, idx primitives.ValidatorIndex) error
SwitchToCompoundingValidator
Spec definition:
def switch_to_compounding_validator(state: BeaconState, index: ValidatorIndex) -> None:
validator = state.validators[index] validator.withdrawal_credentials = COMPOUNDING_WITHDRAWAL_PREFIX + validator.withdrawal_credentials[1:] queue_excess_active_balance(state, index)
func UpgradeToElectra ¶
func UpgradeToElectra(beaconState state.BeaconState) (state.BeaconState, error)
UpgradeToElectra updates inputs a generic state to return the version Electra state.
nolint:dupword Spec code: def upgrade_to_electra(pre: deneb.BeaconState) -> BeaconState:
epoch = deneb.get_current_epoch(pre) latest_execution_payload_header = pre.latest_execution_payload_header earliest_exit_epoch = compute_activation_exit_epoch(get_current_epoch(pre)) for validator in pre.validators: if validator.exit_epoch != FAR_FUTURE_EPOCH: if validator.exit_epoch > earliest_exit_epoch: earliest_exit_epoch = validator.exit_epoch earliest_exit_epoch += Epoch(1) post = BeaconState( # Versioning genesis_time=pre.genesis_time, genesis_validators_root=pre.genesis_validators_root, slot=pre.slot, fork=Fork( previous_version=pre.fork.current_version, current_version=ELECTRA_FORK_VERSION, # [Modified in Electra:EIP6110] epoch=epoch, ), # History latest_block_header=pre.latest_block_header, block_roots=pre.block_roots, state_roots=pre.state_roots, historical_roots=pre.historical_roots, # Eth1 eth1_data=pre.eth1_data, eth1_data_votes=pre.eth1_data_votes, eth1_deposit_index=pre.eth1_deposit_index, # Registry validators=pre.validators, balances=pre.balances, # Randomness randao_mixes=pre.randao_mixes, # Slashings slashings=pre.slashings, # Participation previous_epoch_participation=pre.previous_epoch_participation, current_epoch_participation=pre.current_epoch_participation, # Finality justification_bits=pre.justification_bits, previous_justified_checkpoint=pre.previous_justified_checkpoint, current_justified_checkpoint=pre.current_justified_checkpoint, finalized_checkpoint=pre.finalized_checkpoint, # Inactivity inactivity_scores=pre.inactivity_scores, # Sync current_sync_committee=pre.current_sync_committee, next_sync_committee=pre.next_sync_committee, # Execution-layer latest_execution_payload_header=latest_execution_payload_header, # [Modified in Electra:EIP6110:EIP7002] # Withdrawals next_withdrawal_index=pre.next_withdrawal_index, next_withdrawal_validator_index=pre.next_withdrawal_validator_index, # Deep history valid from Capella onwards historical_summaries=pre.historical_summaries, # [New in Electra:EIP6110] deposit_requests_start_index=UNSET_DEPOSIT_REQUESTS_START_INDEX, # [New in Electra:EIP7251] deposit_balance_to_consume=0, exit_balance_to_consume=0, earliest_exit_epoch=earliest_exit_epoch, consolidation_balance_to_consume=0, earliest_consolidation_epoch=compute_activation_exit_epoch(get_current_epoch(pre)), pending_deposits=[], pending_partial_withdrawals=[], pending_consolidations=[], ) post.exit_balance_to_consume = get_activation_exit_churn_limit(post) post.consolidation_balance_to_consume = get_consolidation_churn_limit(post) # [New in Electra:EIP7251] # add validators that are not yet active to pending balance deposits pre_activation = sorted([ index for index, validator in enumerate(post.validators) if validator.activation_epoch == FAR_FUTURE_EPOCH ], key=lambda index: ( post.validators[index].activation_eligibility_epoch, index )) for index in pre_activation: balance = post.balances[index] post.balances[index] = 0 validator = post.validators[index] validator.effective_balance = 0 validator.activation_eligibility_epoch = FAR_FUTURE_EPOCH # Use bls.G2_POINT_AT_INFINITY as a signature field placeholder # and GENESIS_SLOT to distinguish from a pending deposit request post.pending_deposits.append(PendingDeposit( pubkey=validator.pubkey, withdrawal_credentials=validator.withdrawal_credentials, amount=balance, signature=bls.G2_POINT_AT_INFINITY, slot=GENESIS_SLOT, )) # Ensure early adopters of compounding credentials go through the activation churn for index, validator in enumerate(post.validators): if has_compounding_withdrawal_credential(validator): queue_excess_active_balance(post, ValidatorIndex(index)) return post
func VerifyBlockDepositLength ¶
func VerifyBlockDepositLength(body interfaces.ReadOnlyBeaconBlockBody, state state.BeaconState) error
VerifyBlockDepositLength
Spec definition:
# [Modified in Electra:EIP6110] # Disable former deposit mechanism once all prior deposits are processed eth1_deposit_index_limit = min(state.eth1_data.deposit_count, state.deposit_requests_start_index) if state.eth1_deposit_index < eth1_deposit_index_limit: assert len(body.deposits) == min(MAX_DEPOSITS, eth1_deposit_index_limit - state.eth1_deposit_index) else: assert len(body.deposits) == 0
Types ¶
This section is empty.