[caplin] abstract transition (#7661)

start of abstracting the transition layer. 

incremental movements.
This commit is contained in:
a 2023-06-11 16:50:02 -05:00 committed by GitHub
parent e380ee4bdd
commit 4d35e776da
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
69 changed files with 1694 additions and 1538 deletions

View File

@ -1,211 +0,0 @@
package transition
import (
"errors"
"fmt"
"github.com/ledgerwatch/erigon/cl/cltypes/solid"
state2 "github.com/ledgerwatch/erigon/cl/phase1/core/state"
libcommon "github.com/ledgerwatch/erigon-lib/common"
"github.com/ledgerwatch/erigon/cl/clparams"
"github.com/ledgerwatch/erigon/cl/cltypes"
"github.com/ledgerwatch/erigon/metrics/methelp"
)
// processBlock takes a block and transitions the state to the next slot, using the provided execution payload if enabled.
func processBlock(state *state2.BeaconState, signedBlock *cltypes.SignedBeaconBlock, fullValidation bool) error {
block := signedBlock.Block
version := state.Version()
// Check the state version is correct.
if signedBlock.Version() != version {
return fmt.Errorf("processBlock: wrong state version for block at slot %d", block.Slot)
}
h := methelp.NewHistTimer("beacon_process_block")
c := h.Tag("process_step", "block_header")
// Process the block header.
if err := ProcessBlockHeader(state, block, fullValidation); err != nil {
return fmt.Errorf("processBlock: failed to process block header: %v", err)
}
c.PutSince()
// Process execution payload if enabled.
if version >= clparams.BellatrixVersion && executionEnabled(state, block.Body.ExecutionPayload) {
if state.Version() >= clparams.CapellaVersion {
// Process withdrawals in the execution payload.
c = h.Tag("process_step", "withdrawals")
if err := ProcessWithdrawals(state, block.Body.ExecutionPayload.Withdrawals, fullValidation); err != nil {
return fmt.Errorf("processBlock: failed to process withdrawals: %v", err)
}
c.PutSince()
}
// Process the execution payload.
c = h.Tag("process_step", "execution_payload")
if err := ProcessExecutionPayload(state, block.Body.ExecutionPayload); err != nil {
return fmt.Errorf("processBlock: failed to process execution payload: %v", err)
}
c.PutSince()
}
// Process RANDAO reveal.
c = h.Tag("process_step", "randao_reveal")
if err := ProcessRandao(state, block.Body.RandaoReveal, block.ProposerIndex, fullValidation); err != nil {
return fmt.Errorf("processBlock: failed to process RANDAO reveal: %v", err)
}
c.PutSince()
// Process Eth1 data.
c = h.Tag("process_step", "eth1_data")
if err := ProcessEth1Data(state, block.Body.Eth1Data); err != nil {
return fmt.Errorf("processBlock: failed to process Eth1 data: %v", err)
}
c.PutSince()
// Process block body operations.
c = h.Tag("process_step", "operations")
if err := processOperations(state, block.Body, fullValidation); err != nil {
return fmt.Errorf("processBlock: failed to process block body operations: %v", err)
}
c.PutSince()
// Process sync aggregate in case of Altair version.
if version >= clparams.AltairVersion {
c = h.Tag("process_step", "sync_aggregate")
if err := ProcessSyncAggregate(state, block.Body.SyncAggregate, fullValidation); err != nil {
return fmt.Errorf("processBlock: failed to process sync aggregate: %v", err)
}
c.PutSince()
}
if version >= clparams.DenebVersion && fullValidation {
c = h.Tag("process_step", "blob_kzg_commitments")
verified, err := VerifyKzgCommitmentsAgainstTransactions(block.Body.ExecutionPayload.Transactions, block.Body.BlobKzgCommitments)
if err != nil {
return fmt.Errorf("processBlock: failed to process blob kzg commitments: %w", err)
}
if !verified {
return fmt.Errorf("processBlock: failed to process blob kzg commitments: commitments are not equal")
}
c.PutSince()
}
h.PutSince()
return nil
}
func processOperations(state *state2.BeaconState, blockBody *cltypes.BeaconBody, fullValidation bool) error {
if blockBody.Deposits.Len() != int(maximumDeposits(state)) {
return errors.New("outstanding deposits do not match maximum deposits")
}
h := methelp.NewHistTimer("beacon_process_block_operations")
// Process each proposer slashing
var err error
c := h.Tag("operation", "proposer_slashings")
if err := solid.RangeErr[*cltypes.ProposerSlashing](blockBody.ProposerSlashings, func(index int, slashing *cltypes.ProposerSlashing, length int) error {
if err = ProcessProposerSlashing(state, slashing); err != nil {
return fmt.Errorf("ProcessProposerSlashing: %s", err)
}
return nil
}); err != nil {
return err
}
c.PutSince()
c = h.Tag("operation", "attester_slashings")
if err := solid.RangeErr[*cltypes.AttesterSlashing](blockBody.AttesterSlashings, func(index int, slashing *cltypes.AttesterSlashing, length int) error {
if err = ProcessAttesterSlashing(state, slashing); err != nil {
return fmt.Errorf("ProcessAttesterSlashing: %s", err)
}
return nil
}); err != nil {
return err
}
c.PutSince()
// Process each attestations
c = h.Tag("operation", "attestations", "validation", "false")
if fullValidation {
c = h.Tag("operation", "attestations", "validation", "true")
}
if err := ProcessAttestations(state, blockBody.Attestations, fullValidation); err != nil {
return fmt.Errorf("ProcessAttestation: %s", err)
}
c.PutSince()
// Process each deposit
c = h.Tag("operation", "deposit")
if err := solid.RangeErr[*cltypes.Deposit](blockBody.Deposits, func(index int, deposit *cltypes.Deposit, length int) error {
if err = ProcessDeposit(state, deposit, fullValidation); err != nil {
return fmt.Errorf("ProcessDeposit: %s", err)
}
return nil
}); err != nil {
return err
}
c.PutSince()
// Process each voluntary exit.
c = h.Tag("operation", "voluntary_exit")
if err := solid.RangeErr[*cltypes.SignedVoluntaryExit](blockBody.VoluntaryExits, func(index int, exit *cltypes.SignedVoluntaryExit, length int) error {
if err = ProcessVoluntaryExit(state, exit, fullValidation); err != nil {
return fmt.Errorf("ProcessVoluntaryExit: %s", err)
}
return nil
}); err != nil {
return err
}
c.PutSince()
if state.Version() < clparams.CapellaVersion {
return nil
}
// Process each execution change. this will only have entries after the capella fork.
c = h.Tag("operation", "execution_change")
if err := solid.RangeErr[*cltypes.SignedBLSToExecutionChange](blockBody.ExecutionChanges, func(index int, addressChange *cltypes.SignedBLSToExecutionChange, length int) error {
if err := ProcessBlsToExecutionChange(state, addressChange, fullValidation); err != nil {
return fmt.Errorf("ProcessBlsToExecutionChange: %s", err)
}
return nil
}); err != nil {
return err
}
c.PutSince()
return nil
}
func maximumDeposits(state *state2.BeaconState) (maxDeposits uint64) {
maxDeposits = state.Eth1Data().DepositCount - state.Eth1DepositIndex()
if maxDeposits > state.BeaconConfig().MaxDeposits {
maxDeposits = state.BeaconConfig().MaxDeposits
}
return
}
// ProcessExecutionPayload sets the latest payload header accordinly.
func ProcessExecutionPayload(s *state2.BeaconState, payload *cltypes.Eth1Block) error {
if state2.IsMergeTransitionComplete(s.BeaconState) {
if payload.ParentHash != s.LatestExecutionPayloadHeader().BlockHash {
return fmt.Errorf("ProcessExecutionPayload: invalid eth1 chain. mismatching parent")
}
}
if payload.PrevRandao != s.GetRandaoMixes(state2.Epoch(s.BeaconState)) {
return fmt.Errorf("ProcessExecutionPayload: randao mix mismatches with mix digest")
}
if payload.Time != state2.ComputeTimestampAtSlot(s.BeaconState, s.Slot()) {
return fmt.Errorf("ProcessExecutionPayload: invalid Eth1 timestamp")
}
payloadHeader, err := payload.PayloadHeader()
if err != nil {
return err
}
s.SetLatestExecutionPayloadHeader(payloadHeader)
return nil
}
func executionEnabled(s *state2.BeaconState, payload *cltypes.Eth1Block) bool {
return (!state2.IsMergeTransitionComplete(s.BeaconState) && payload.BlockHash != libcommon.Hash{}) || state2.IsMergeTransitionComplete(s.BeaconState)
}

View File

@ -1,281 +0,0 @@
package transition
import (
"errors"
"fmt"
"github.com/ledgerwatch/erigon-lib/common"
"github.com/ledgerwatch/erigon/cl/cltypes/solid"
state2 "github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/Giulio2002/bls"
"github.com/ledgerwatch/log/v3"
"github.com/ledgerwatch/erigon/cl/clparams"
"github.com/ledgerwatch/erigon/cl/cltypes"
"github.com/ledgerwatch/erigon/cl/fork"
"github.com/ledgerwatch/erigon/cl/utils"
"github.com/ledgerwatch/erigon/core/types"
)
func ProcessProposerSlashing(s *state2.BeaconState, propSlashing *cltypes.ProposerSlashing) error {
h1 := propSlashing.Header1.Header
h2 := propSlashing.Header2.Header
if h1.Slot != h2.Slot {
return fmt.Errorf("non-matching slots on proposer slashing: %d != %d", h1.Slot, h2.Slot)
}
if h1.ProposerIndex != h2.ProposerIndex {
return fmt.Errorf("non-matching proposer indices proposer slashing: %d != %d", h1.ProposerIndex, h2.ProposerIndex)
}
h1Root, err := h1.HashSSZ()
if err != nil {
return fmt.Errorf("unable to hash header1: %v", err)
}
h2Root, err := h2.HashSSZ()
if err != nil {
return fmt.Errorf("unable to hash header2: %v", err)
}
if h1Root == h2Root {
return fmt.Errorf("propose slashing headers are the same: %v == %v", h1Root, h2Root)
}
proposer, err := s.ValidatorForValidatorIndex(int(h1.ProposerIndex))
if err != nil {
return err
}
if !proposer.IsSlashable(state2.Epoch(s.BeaconState)) {
return fmt.Errorf("proposer is not slashable: %v", proposer)
}
for _, signedHeader := range []*cltypes.SignedBeaconBlockHeader{propSlashing.Header1, propSlashing.Header2} {
domain, err := s.GetDomain(s.BeaconConfig().DomainBeaconProposer, state2.GetEpochAtSlot(s.BeaconConfig(), signedHeader.Header.Slot))
if err != nil {
return fmt.Errorf("unable to get domain: %v", err)
}
signingRoot, err := fork.ComputeSigningRoot(signedHeader.Header, domain)
if err != nil {
return fmt.Errorf("unable to compute signing root: %v", err)
}
pk := proposer.PublicKey()
valid, err := bls.Verify(signedHeader.Signature[:], signingRoot[:], pk[:])
if err != nil {
return fmt.Errorf("unable to verify signature: %v", err)
}
if !valid {
return fmt.Errorf("invalid signature: signature %v, root %v, pubkey %v", signedHeader.Signature[:], signingRoot[:], pk)
}
}
// Set whistleblower index to 0 so current proposer gets reward.
s.SlashValidator(h1.ProposerIndex, nil)
return nil
}
func ProcessAttesterSlashing(s *state2.BeaconState, attSlashing *cltypes.AttesterSlashing) error {
att1 := attSlashing.Attestation_1
att2 := attSlashing.Attestation_2
if !cltypes.IsSlashableAttestationData(att1.Data, att2.Data) {
return fmt.Errorf("attestation data not slashable: %+v; %+v", att1.Data, att2.Data)
}
valid, err := state2.IsValidIndexedAttestation(s.BeaconState, att1)
if err != nil {
return fmt.Errorf("error calculating indexed attestation 1 validity: %v", err)
}
if !valid {
return fmt.Errorf("invalid indexed attestation 1")
}
valid, err = state2.IsValidIndexedAttestation(s.BeaconState, att2)
if err != nil {
return fmt.Errorf("error calculating indexed attestation 2 validity: %v", err)
}
if !valid {
return fmt.Errorf("invalid indexed attestation 2")
}
slashedAny := false
currentEpoch := state2.GetEpochAtSlot(s.BeaconConfig(), s.Slot())
for _, ind := range solid.IntersectionOfSortedSets(att1.AttestingIndices, att2.AttestingIndices) {
validator, err := s.ValidatorForValidatorIndex(int(ind))
if err != nil {
return err
}
if validator.IsSlashable(currentEpoch) {
err := s.SlashValidator(ind, nil)
if err != nil {
return fmt.Errorf("unable to slash validator: %d", ind)
}
slashedAny = true
}
}
if !slashedAny {
return fmt.Errorf("no validators slashed")
}
return nil
}
func ProcessDeposit(s *state2.BeaconState, deposit *cltypes.Deposit, fullValidation bool) error {
if deposit == nil {
return nil
}
depositLeaf, err := deposit.Data.HashSSZ()
if err != nil {
return err
}
depositIndex := s.Eth1DepositIndex()
eth1Data := s.Eth1Data()
rawProof := []common.Hash{}
deposit.Proof.Range(func(_ int, l common.Hash, _ int) bool {
rawProof = append(rawProof, l)
return true
})
// Validate merkle proof for deposit leaf.
if fullValidation && !utils.IsValidMerkleBranch(
depositLeaf,
rawProof,
s.BeaconConfig().DepositContractTreeDepth+1,
depositIndex,
eth1Data.Root,
) {
return fmt.Errorf("processDepositForAltair: Could not validate deposit root")
}
// Increment index
s.SetEth1DepositIndex(depositIndex + 1)
publicKey := deposit.Data.PubKey
amount := deposit.Data.Amount
// Check if pub key is in validator set
validatorIndex, has := s.ValidatorIndexByPubkey(publicKey)
if !has {
// Agnostic domain.
domain, err := fork.ComputeDomain(s.BeaconConfig().DomainDeposit[:], utils.Uint32ToBytes4(s.BeaconConfig().GenesisForkVersion), [32]byte{})
if err != nil {
return err
}
depositMessageRoot, err := deposit.Data.MessageHash()
if err != nil {
return err
}
signedRoot := utils.Keccak256(depositMessageRoot[:], domain)
// Perform BLS verification and if successful noice.
valid, err := bls.Verify(deposit.Data.Signature[:], signedRoot[:], publicKey[:])
// Literally you can input it trash.
if !valid || err != nil {
log.Debug("Validator BLS verification failed", "valid", valid, "err", err)
return nil
}
// Append validator
s.AddValidator(state2.ValidatorFromDeposit(s.BeaconConfig(), deposit), amount)
// Altair forward
if s.Version() >= clparams.AltairVersion {
s.AddCurrentEpochParticipationFlags(cltypes.ParticipationFlags(0))
s.AddPreviousEpochParticipationFlags(cltypes.ParticipationFlags(0))
s.AddInactivityScore(0)
}
return nil
}
// Increase the balance if exists already
return state2.IncreaseBalance(s.BeaconState, validatorIndex, amount)
}
// ProcessVoluntaryExit takes a voluntary exit and applies state transition.
func ProcessVoluntaryExit(s *state2.BeaconState, signedVoluntaryExit *cltypes.SignedVoluntaryExit, fullValidation bool) error {
// Sanity checks so that we know it is good.
voluntaryExit := signedVoluntaryExit.VolunaryExit
currentEpoch := state2.Epoch(s.BeaconState)
validator, err := s.ValidatorForValidatorIndex(int(voluntaryExit.ValidatorIndex))
if err != nil {
return err
}
if !validator.Active(currentEpoch) {
return errors.New("ProcessVoluntaryExit: validator is not active")
}
if validator.ExitEpoch() != s.BeaconConfig().FarFutureEpoch {
return errors.New("ProcessVoluntaryExit: another exit for the same validator is already getting processed")
}
if currentEpoch < voluntaryExit.Epoch {
return errors.New("ProcessVoluntaryExit: exit is happening in the future")
}
if currentEpoch < validator.ActivationEpoch()+s.BeaconConfig().ShardCommitteePeriod {
return errors.New("ProcessVoluntaryExit: exit is happening too fast")
}
// We can skip it in some instances if we want to optimistically sync up.
if fullValidation {
domain, err := s.GetDomain(s.BeaconConfig().DomainVoluntaryExit, voluntaryExit.Epoch)
if err != nil {
return err
}
signingRoot, err := fork.ComputeSigningRoot(voluntaryExit, domain)
if err != nil {
return err
}
pk := validator.PublicKey()
valid, err := bls.Verify(signedVoluntaryExit.Signature[:], signingRoot[:], pk[:])
if err != nil {
return err
}
if !valid {
return errors.New("ProcessVoluntaryExit: BLS verification failed")
}
}
// Do the exit (same process in slashing).
return s.InitiateValidatorExit(voluntaryExit.ValidatorIndex)
}
// ProcessWithdrawals processes withdrawals by decreasing the balance of each validator
// and updating the next withdrawal index and validator index.
func ProcessWithdrawals(s *state2.BeaconState, withdrawals *solid.ListSSZ[*types.Withdrawal], fullValidation bool) error {
// Get the list of withdrawals, the expected withdrawals (if performing full validation),
// and the beacon configuration.
beaconConfig := s.BeaconConfig()
numValidators := uint64(s.ValidatorLength())
// Check if full validation is required and verify expected withdrawals.
if fullValidation {
expectedWithdrawals := state2.ExpectedWithdrawals(s.BeaconState)
if len(expectedWithdrawals) != withdrawals.Len() {
return fmt.Errorf("ProcessWithdrawals: expected %d withdrawals, but got %d", len(expectedWithdrawals), withdrawals.Len())
}
if err := solid.RangeErr[*types.Withdrawal](withdrawals, func(i int, w *types.Withdrawal, _ int) error {
if !expectedWithdrawals[i].Equal(w) {
return fmt.Errorf("ProcessWithdrawals: withdrawal %d does not match expected withdrawal", i)
}
return nil
}); err != nil {
return err
}
}
if err := solid.RangeErr[*types.Withdrawal](withdrawals, func(_ int, w *types.Withdrawal, _ int) error {
if err := state2.DecreaseBalance(s.BeaconState, w.Validator, w.Amount); err != nil {
return err
}
return nil
}); err != nil {
return err
}
// Update next withdrawal index based on number of withdrawals.
if withdrawals.Len() > 0 {
lastWithdrawalIndex := withdrawals.Get(withdrawals.Len() - 1).Index
s.SetNextWithdrawalIndex(lastWithdrawalIndex + 1)
}
// Update next withdrawal validator index based on number of withdrawals.
if withdrawals.Len() == int(beaconConfig.MaxWithdrawalsPerPayload) {
lastWithdrawalValidatorIndex := withdrawals.Get(withdrawals.Len()-1).Validator + 1
s.SetNextWithdrawalValidatorIndex(lastWithdrawalValidatorIndex % numValidators)
} else {
nextIndex := s.NextWithdrawalValidatorIndex() + beaconConfig.MaxValidatorsPerWithdrawalsSweep
s.SetNextWithdrawalValidatorIndex(nextIndex % numValidators)
}
return nil
}

View File

@ -1,260 +0,0 @@
package transition
import (
"errors"
"fmt"
"github.com/ledgerwatch/erigon/cl/cltypes/solid"
state2 "github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/ledgerwatch/erigon/cl/clparams"
"github.com/ledgerwatch/erigon/cl/utils"
"github.com/ledgerwatch/erigon/metrics/methelp"
"golang.org/x/exp/slices"
)
func ProcessAttestations(s *state2.BeaconState, attestations *solid.ListSSZ[*solid.Attestation], fullValidation bool) error {
attestingIndiciesSet := make([][]uint64, attestations.Len())
h := methelp.NewHistTimer("beacon_process_attestations")
baseRewardPerIncrement := s.BaseRewardPerIncrement()
c := h.Tag("attestation_step", "process")
var err error
if err := solid.RangeErr[*solid.Attestation](attestations, func(i int, a *solid.Attestation, _ int) error {
if attestingIndiciesSet[i], err = processAttestation(s, a, baseRewardPerIncrement); err != nil {
return err
}
return nil
}); err != nil {
return err
}
if err != nil {
return err
}
var valid bool
c.PutSince()
if fullValidation {
c = h.Tag("attestation_step", "validate")
valid, err = verifyAttestations(s, attestations, attestingIndiciesSet)
if err != nil {
return err
}
if !valid {
return errors.New("ProcessAttestation: wrong bls data")
}
c.PutSince()
}
return nil
}
func processAttestationPostAltair(s *state2.BeaconState, attestation *solid.Attestation, baseRewardPerIncrement uint64) ([]uint64, error) {
data := attestation.AttestantionData()
currentEpoch := state2.Epoch(s.BeaconState)
stateSlot := s.Slot()
beaconConfig := s.BeaconConfig()
h := methelp.NewHistTimer("beacon_process_attestation_post_altair")
c := h.Tag("step", "get_participation_flag")
participationFlagsIndicies, err := s.GetAttestationParticipationFlagIndicies(attestation.AttestantionData(), stateSlot-data.Slot())
if err != nil {
return nil, err
}
c.PutSince()
c = h.Tag("step", "get_attesting_indices")
attestingIndicies, err := s.GetAttestingIndicies(attestation.AttestantionData(), attestation.AggregationBits(), true)
if err != nil {
return nil, err
}
c.PutSince()
var proposerRewardNumerator uint64
isCurrentEpoch := data.Target().Epoch() == currentEpoch
c = h.Tag("step", "update_attestation")
for _, attesterIndex := range attestingIndicies {
val, err := s.ValidatorEffectiveBalance(int(attesterIndex))
if err != nil {
return nil, err
}
baseReward := (val / beaconConfig.EffectiveBalanceIncrement) * baseRewardPerIncrement
for flagIndex, weight := range beaconConfig.ParticipationWeights() {
flagParticipation := s.EpochParticipationForValidatorIndex(isCurrentEpoch, int(attesterIndex))
if !slices.Contains(participationFlagsIndicies, uint8(flagIndex)) || flagParticipation.HasFlag(flagIndex) {
continue
}
s.SetEpochParticipationForValidatorIndex(isCurrentEpoch, int(attesterIndex), flagParticipation.Add(flagIndex))
proposerRewardNumerator += baseReward * weight
}
}
c.PutSince()
// Reward proposer
c = h.Tag("step", "get_proposer_index")
proposer, err := s.GetBeaconProposerIndex()
if err != nil {
return nil, err
}
c.PutSince()
proposerRewardDenominator := (beaconConfig.WeightDenominator - beaconConfig.ProposerWeight) * beaconConfig.WeightDenominator / beaconConfig.ProposerWeight
reward := proposerRewardNumerator / proposerRewardDenominator
return attestingIndicies, state2.IncreaseBalance(s.BeaconState, proposer, reward)
}
// processAttestationsPhase0 implements the rules for phase0 processing.
func processAttestationPhase0(s *state2.BeaconState, attestation *solid.Attestation) ([]uint64, error) {
data := attestation.AttestantionData()
committee, err := s.GetBeaconCommitee(data.Slot(), data.ValidatorIndex())
if err != nil {
return nil, err
}
if len(committee) != utils.GetBitlistLength(attestation.AggregationBits()) {
return nil, fmt.Errorf("processAttestationPhase0: mismatching aggregation bits size")
}
// Cached so it is performant.
proposerIndex, err := s.GetBeaconProposerIndex()
if err != nil {
return nil, err
}
// Create the attestation to add to pending attestations
pendingAttestation := solid.NewPendingAttestionFromParameters(
attestation.AggregationBits(),
data,
s.Slot()-data.Slot(),
proposerIndex,
)
isCurrentAttestation := data.Target().Epoch() == state2.Epoch(s.BeaconState)
// Depending of what slot we are on we put in either the current justified or previous justified.
if isCurrentAttestation {
if !data.Source().Equal(s.CurrentJustifiedCheckpoint()) {
return nil, fmt.Errorf("processAttestationPhase0: mismatching sources")
}
s.AddCurrentEpochAtteastation(pendingAttestation)
} else {
if !data.Source().Equal(s.PreviousJustifiedCheckpoint()) {
return nil, fmt.Errorf("processAttestationPhase0: mismatching sources")
}
s.AddPreviousEpochAttestation(pendingAttestation)
}
// Not required by specs but needed if we want performant epoch transition.
indicies, err := s.GetAttestingIndicies(attestation.AttestantionData(), attestation.AggregationBits(), true)
if err != nil {
return nil, err
}
epochRoot, err := state2.GetBlockRoot(s.BeaconState, attestation.AttestantionData().Target().Epoch())
if err != nil {
return nil, err
}
slotRoot, err := s.GetBlockRootAtSlot(attestation.AttestantionData().Slot())
if err != nil {
return nil, err
}
// Basically we flag all validators we are currently attesting. will be important for rewards/finalization processing.
for _, index := range indicies {
minCurrentInclusionDelayAttestation, err := s.ValidatorMinCurrentInclusionDelayAttestation(int(index))
if err != nil {
return nil, err
}
minPreviousInclusionDelayAttestation, err := s.ValidatorMinPreviousInclusionDelayAttestation(int(index))
if err != nil {
return nil, err
}
// NOTE: does not affect state root.
// We need to set it to currents or previouses depending on which attestation we process.
if isCurrentAttestation {
if minCurrentInclusionDelayAttestation == nil ||
minCurrentInclusionDelayAttestation.InclusionDelay() > pendingAttestation.InclusionDelay() {
if err := s.SetValidatorMinCurrentInclusionDelayAttestation(int(index), pendingAttestation); err != nil {
return nil, err
}
}
if err := s.SetValidatorIsCurrentMatchingSourceAttester(int(index), true); err != nil {
return nil, err
}
if attestation.AttestantionData().Target().BlockRoot() == epochRoot {
if err := s.SetValidatorIsCurrentMatchingTargetAttester(int(index), true); err != nil {
return nil, err
}
} else {
continue
}
if attestation.AttestantionData().BeaconBlockRoot() == slotRoot {
if err := s.SetValidatorIsCurrentMatchingHeadAttester(int(index), true); err != nil {
return nil, err
}
}
} else {
if minPreviousInclusionDelayAttestation == nil ||
minPreviousInclusionDelayAttestation.InclusionDelay() > pendingAttestation.InclusionDelay() {
if err := s.SetValidatorMinPreviousInclusionDelayAttestation(int(index), pendingAttestation); err != nil {
return nil, err
}
}
if err := s.SetValidatorIsPreviousMatchingSourceAttester(int(index), true); err != nil {
return nil, err
}
if attestation.AttestantionData().Target().BlockRoot() != epochRoot {
continue
}
if err := s.SetValidatorIsPreviousMatchingTargetAttester(int(index), true); err != nil {
return nil, err
}
if attestation.AttestantionData().BeaconBlockRoot() == slotRoot {
if err := s.SetValidatorIsPreviousMatchingHeadAttester(int(index), true); err != nil {
return nil, err
}
}
}
}
return indicies, nil
}
// ProcessAttestation takes an attestation and process it.
func processAttestation(s *state2.BeaconState, attestation *solid.Attestation, baseRewardPerIncrement uint64) ([]uint64, error) {
data := attestation.AttestantionData()
currentEpoch := state2.Epoch(s.BeaconState)
previousEpoch := state2.PreviousEpoch(s.BeaconState)
stateSlot := s.Slot()
beaconConfig := s.BeaconConfig()
// Prelimary checks.
if (data.Target().Epoch() != currentEpoch && data.Target().Epoch() != previousEpoch) || data.Target().Epoch() != state2.GetEpochAtSlot(s.BeaconConfig(), data.Slot()) {
return nil, errors.New("ProcessAttestation: attestation with invalid epoch")
}
if data.Slot()+beaconConfig.MinAttestationInclusionDelay > stateSlot || stateSlot > data.Slot()+beaconConfig.SlotsPerEpoch {
return nil, errors.New("ProcessAttestation: attestation slot not in range")
}
if data.ValidatorIndex() >= s.CommitteeCount(data.Target().Epoch()) {
return nil, errors.New("ProcessAttestation: attester index out of range")
}
// check if we need to use rules for phase0 or post-altair.
if s.Version() == clparams.Phase0Version {
return processAttestationPhase0(s, attestation)
}
return processAttestationPostAltair(s, attestation, baseRewardPerIncrement)
}
func verifyAttestations(s *state2.BeaconState, attestations *solid.ListSSZ[*solid.Attestation], attestingIndicies [][]uint64) (bool, error) {
var err error
valid := true
attestations.Range(func(idx int, a *solid.Attestation, _ int) bool {
indexedAttestation := state2.GetIndexedAttestation(a, attestingIndicies[idx])
valid, err = state2.IsValidIndexedAttestation(s.BeaconState, indexedAttestation)
if err != nil {
return false
}
if !valid {
return false
}
return true
})
return valid, err
}

View File

@ -1,65 +0,0 @@
package transition
import (
"bytes"
"fmt"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/Giulio2002/bls"
"github.com/ledgerwatch/erigon/cl/cltypes"
"github.com/ledgerwatch/erigon/cl/fork"
"github.com/ledgerwatch/erigon/cl/utils"
)
// ProcessBlsToExecutionChange processes a BLSToExecutionChange message by updating a validator's withdrawal credentials.
func ProcessBlsToExecutionChange(state *state.BeaconState, signedChange *cltypes.SignedBLSToExecutionChange, fullValidation bool) error {
change := signedChange.Message
beaconConfig := state.BeaconConfig()
validator, err := state.ValidatorForValidatorIndex(int(change.ValidatorIndex))
if err != nil {
return err
}
// Perform full validation if requested.
wc := validator.WithdrawalCredentials()
if fullValidation {
// Check the validator's withdrawal credentials prefix.
if wc[0] != beaconConfig.BLSWithdrawalPrefixByte {
return fmt.Errorf("invalid withdrawal credentials prefix")
}
// Check the validator's withdrawal credentials against the provided message.
hashedFrom := utils.Keccak256(change.From[:])
if !bytes.Equal(hashedFrom[1:], wc[1:]) {
return fmt.Errorf("invalid withdrawal credentials")
}
// Compute the signing domain and verify the message signature.
domain, err := fork.ComputeDomain(beaconConfig.DomainBLSToExecutionChange[:], utils.Uint32ToBytes4(beaconConfig.GenesisForkVersion), state.GenesisValidatorsRoot())
if err != nil {
return err
}
signedRoot, err := fork.ComputeSigningRoot(change, domain)
if err != nil {
return err
}
valid, err := bls.Verify(signedChange.Signature[:], signedRoot[:], change.From[:])
if err != nil {
return err
}
if !valid {
return fmt.Errorf("invalid signature")
}
}
credentials := wc
// Reset the validator's withdrawal credentials.
credentials[0] = beaconConfig.ETH1AddressWithdrawalPrefixByte
copy(credentials[1:], make([]byte, 11))
copy(credentials[12:], change.To[:])
// Update the state with the modified validator.
state.SetWithdrawalCredentialForValidatorAtIndex(int(change.ValidatorIndex), credentials)
return nil
}

View File

@ -1,103 +0,0 @@
package transition
import (
"github.com/ledgerwatch/erigon/cl/clparams"
"github.com/ledgerwatch/erigon/cl/cltypes/solid"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
)
// ProcessEpoch process epoch transition.
func ProcessEpoch(state *state.BeaconState) error {
if err := ProcessJustificationBitsAndFinality(state); err != nil {
return err
}
if state.Version() >= clparams.AltairVersion {
if err := ProcessInactivityScores(state); err != nil {
return err
}
}
if err := ProcessRewardsAndPenalties(state); err != nil {
return err
}
if err := ProcessRegistryUpdates(state); err != nil {
return err
}
if err := ProcessSlashings(state); err != nil {
return err
}
ProcessEth1DataReset(state)
if err := ProcessEffectiveBalanceUpdates(state); err != nil {
return err
}
ProcessSlashingsReset(state)
ProcessRandaoMixesReset(state)
if err := ProcessHistoricalRootsUpdate(state); err != nil {
return err
}
if state.Version() == clparams.Phase0Version {
if err := ProcessParticipationRecordUpdates(state); err != nil {
return err
}
}
if state.Version() >= clparams.AltairVersion {
ProcessParticipationFlagUpdates(state)
if err := ProcessSyncCommitteeUpdate(state); err != nil {
return err
}
}
return nil
}
func ProcessParticipationRecordUpdates(state *state.BeaconState) error {
state.SetPreviousEpochAttestations(state.CurrentEpochAttestations())
state.ResetCurrentEpochAttestations()
var err error
// Also mark all current attesters as previous
state.ForEachValidator(func(_ solid.Validator, idx, total int) bool {
var oldCurrentMatchingSourceAttester, oldCurrentMatchingTargetAttester, oldCurrentMatchingHeadAttester bool
var oldMinCurrentInclusionDelayAttestation *solid.PendingAttestation
if oldCurrentMatchingSourceAttester, err = state.ValidatorIsCurrentMatchingSourceAttester(idx); err != nil {
return false
}
if oldCurrentMatchingTargetAttester, err = state.ValidatorIsCurrentMatchingTargetAttester(idx); err != nil {
return false
}
if oldCurrentMatchingHeadAttester, err = state.ValidatorIsCurrentMatchingHeadAttester(idx); err != nil {
return false
}
if oldMinCurrentInclusionDelayAttestation, err = state.ValidatorMinCurrentInclusionDelayAttestation(idx); err != nil {
return false
}
// Previous sources/target/head
if err = state.SetValidatorIsPreviousMatchingSourceAttester(idx, oldCurrentMatchingSourceAttester); err != nil {
return false
}
if err = state.SetValidatorIsPreviousMatchingTargetAttester(idx, oldCurrentMatchingTargetAttester); err != nil {
return false
}
if err = state.SetValidatorIsPreviousMatchingHeadAttester(idx, oldCurrentMatchingHeadAttester); err != nil {
return false
}
if err = state.SetValidatorMinPreviousInclusionDelayAttestation(idx, oldMinCurrentInclusionDelayAttestation); err != nil {
return false
}
// Current sources/target/head
if err = state.SetValidatorIsCurrentMatchingSourceAttester(idx, false); err != nil {
return false
}
if err = state.SetValidatorIsCurrentMatchingTargetAttester(idx, false); err != nil {
return false
}
if err = state.SetValidatorIsCurrentMatchingHeadAttester(idx, false); err != nil {
return false
}
if err = state.SetValidatorMinCurrentInclusionDelayAttestation(idx, nil); err != nil {
return false
}
return true
})
return err
}

View File

@ -1,56 +0,0 @@
package transition
import (
"github.com/ledgerwatch/erigon/cl/clparams"
"github.com/ledgerwatch/erigon/cl/cltypes/solid"
state2 "github.com/ledgerwatch/erigon/cl/phase1/core/state"
)
func processSlashings(s *state2.BeaconState, slashingMultiplier uint64) error {
// Get the current epoch
epoch := state2.Epoch(s.BeaconState)
// Get the total active balance
totalBalance := s.GetTotalActiveBalance()
// Calculate the total slashing amount
// by summing all slashings and multiplying by the provided multiplier
slashing := state2.GetTotalSlashingAmount(s.BeaconState) * slashingMultiplier
// Adjust the total slashing amount to be no greater than the total active balance
if totalBalance < slashing {
slashing = totalBalance
}
beaconConfig := s.BeaconConfig()
// Apply penalties to validators who have been slashed and reached the withdrawable epoch
var err error
s.ForEachValidator(func(validator solid.Validator, i, total int) bool {
if !validator.Slashed() || epoch+beaconConfig.EpochsPerSlashingsVector/2 != validator.WithdrawableEpoch() {
return true
}
// Get the effective balance increment
increment := beaconConfig.EffectiveBalanceIncrement
// Calculate the penalty numerator by multiplying the validator's effective balance by the total slashing amount
penaltyNumerator := validator.EffectiveBalance() / increment * slashing
// Calculate the penalty by dividing the penalty numerator by the total balance and multiplying by the increment
penalty := penaltyNumerator / totalBalance * increment
// Decrease the validator's balance by the calculated penalty
if err = state2.DecreaseBalance(s.BeaconState, uint64(i), penalty); err != nil {
return false
}
return true
})
if err != nil {
return err
}
return nil
}
func ProcessSlashings(state *state2.BeaconState) error {
// Depending on the version of the state, use different multipliers
switch state.Version() {
case clparams.Phase0Version:
return processSlashings(state, state.BeaconConfig().ProportionalSlashingMultiplier)
case clparams.AltairVersion:
return processSlashings(state, state.BeaconConfig().ProportionalSlashingMultiplierAltair)
default:
return processSlashings(state, state.BeaconConfig().ProportionalSlashingMultiplierBellatrix)
}
}

View File

@ -1,183 +0,0 @@
package transition
import (
"fmt"
"time"
state2 "github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/Giulio2002/bls"
libcommon "github.com/ledgerwatch/erigon-lib/common"
"github.com/ledgerwatch/erigon/cl/clparams"
"github.com/ledgerwatch/erigon/cl/cltypes"
"github.com/ledgerwatch/erigon/cl/fork"
"github.com/ledgerwatch/erigon/cl/utils"
"github.com/ledgerwatch/log/v3"
)
func TransitionState(s *state2.BeaconState, block *cltypes.SignedBeaconBlock, fullValidation bool) error {
currentBlock := block.Block
if err := ProcessSlots(s, currentBlock.Slot); err != nil {
return err
}
if fullValidation {
valid, err := verifyBlockSignature(s, block)
if err != nil {
return fmt.Errorf("error validating block signature: %v", err)
}
if !valid {
return fmt.Errorf("block not valid")
}
}
// Transition block
if err := processBlock(s, block, fullValidation); err != nil {
return err
}
if fullValidation {
expectedStateRoot, err := s.HashSSZ()
if err != nil {
return fmt.Errorf("unable to generate state root: %v", err)
}
if expectedStateRoot != currentBlock.StateRoot {
return fmt.Errorf("expected state root differs from received state root")
}
}
s.SetPreviousStateRoot(currentBlock.StateRoot)
return nil
}
// transitionSlot is called each time there is a new slot to process
func transitionSlot(s *state2.BeaconState) error {
slot := s.Slot()
previousStateRoot := s.PreviousStateRoot()
var err error
if previousStateRoot == (libcommon.Hash{}) {
previousStateRoot, err = s.HashSSZ()
if err != nil {
return err
}
}
beaconConfig := s.BeaconConfig()
s.SetStateRootAt(int(slot%beaconConfig.SlotsPerHistoricalRoot), previousStateRoot)
latestBlockHeader := s.LatestBlockHeader()
if latestBlockHeader.Root == [32]byte{} {
latestBlockHeader.Root = previousStateRoot
s.SetLatestBlockHeader(&latestBlockHeader)
}
blockHeader := s.LatestBlockHeader()
previousBlockRoot, err := (&blockHeader).HashSSZ()
if err != nil {
return err
}
s.SetBlockRootAt(int(slot%beaconConfig.SlotsPerHistoricalRoot), previousBlockRoot)
return nil
}
func ProcessSlots(s *state2.BeaconState, slot uint64) error {
beaconConfig := s.BeaconConfig()
sSlot := s.Slot()
if slot <= sSlot {
return fmt.Errorf("new slot: %d not greater than s slot: %d", slot, sSlot)
}
// Process each slot.
for i := sSlot; i < slot; i++ {
err := transitionSlot(s)
if err != nil {
return fmt.Errorf("unable to process slot transition: %v", err)
}
// TODO(Someone): Add epoch transition.
if (sSlot+1)%beaconConfig.SlotsPerEpoch == 0 {
start := time.Now()
if err := ProcessEpoch(s); err != nil {
return err
}
log.Debug("Processed new epoch successfully", "epoch", state2.Epoch(s.BeaconState), "process_epoch_elpsed", time.Since(start))
}
// TODO: add logic to process epoch updates.
sSlot += 1
s.SetSlot(sSlot)
if sSlot%beaconConfig.SlotsPerEpoch != 0 {
continue
}
if state2.Epoch(s.BeaconState) == beaconConfig.AltairForkEpoch {
if err := s.UpgradeToAltair(); err != nil {
return err
}
}
if state2.Epoch(s.BeaconState) == beaconConfig.BellatrixForkEpoch {
if err := s.UpgradeToBellatrix(); err != nil {
return err
}
}
if state2.Epoch(s.BeaconState) == beaconConfig.CapellaForkEpoch {
if err := s.UpgradeToCapella(); err != nil {
return err
}
}
if state2.Epoch(s.BeaconState) == beaconConfig.DenebForkEpoch {
if err := s.UpgradeToDeneb(); err != nil {
return err
}
}
}
return nil
}
func verifyBlockSignature(s *state2.BeaconState, block *cltypes.SignedBeaconBlock) (bool, error) {
proposer, err := s.ValidatorForValidatorIndex(int(block.Block.ProposerIndex))
if err != nil {
return false, err
}
domain, err := s.GetDomain(s.BeaconConfig().DomainBeaconProposer, state2.Epoch(s.BeaconState))
if err != nil {
return false, err
}
sigRoot, err := fork.ComputeSigningRoot(block.Block, domain)
if err != nil {
return false, err
}
pk := proposer.PublicKey()
return bls.Verify(block.Signature[:], sigRoot[:], pk[:])
}
// ProcessHistoricalRootsUpdate updates the historical root data structure by computing a new historical root batch when it is time to do so.
func ProcessHistoricalRootsUpdate(s *state2.BeaconState) error {
nextEpoch := state2.Epoch(s.BeaconState) + 1
beaconConfig := s.BeaconConfig()
blockRoots := s.BlockRoots()
stateRoots := s.StateRoots()
// Check if it's time to compute the historical root batch.
if nextEpoch%(beaconConfig.SlotsPerHistoricalRoot/beaconConfig.SlotsPerEpoch) != 0 {
return nil
}
// Compute historical root batch.
blockRootsLeaf, err := blockRoots.HashSSZ()
if err != nil {
return err
}
stateRootsLeaf, err := stateRoots.HashSSZ()
if err != nil {
return err
}
// Add the historical summary or root to the s.
if s.Version() >= clparams.CapellaVersion {
s.AddHistoricalSummary(&cltypes.HistoricalSummary{
BlockSummaryRoot: blockRootsLeaf,
StateSummaryRoot: stateRootsLeaf,
})
} else {
historicalRoot := utils.Keccak256(blockRootsLeaf[:], stateRootsLeaf[:])
s.AddHistoricalRoot(historicalRoot)
}
return nil
}

View File

@ -1,93 +0,0 @@
package transition
import (
"errors"
state2 "github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/Giulio2002/bls"
"github.com/ledgerwatch/erigon/cl/cltypes"
"github.com/ledgerwatch/erigon/cl/fork"
"github.com/ledgerwatch/erigon/cl/utils"
)
// processSyncAggregate applies all the logic in the spec function `process_sync_aggregate` except
// verifying the BLS signatures. It returns the modified beacons state and the list of validators'
// public keys that voted, for future signature verification.
func processSyncAggregate(s *state2.BeaconState, sync *cltypes.SyncAggregate) ([][]byte, error) {
currentSyncCommittee := s.CurrentSyncCommittee()
if currentSyncCommittee == nil {
return nil, errors.New("nil current sync committee in s")
}
committeeKeys := currentSyncCommittee.GetCommittee()
if len(sync.SyncCommiteeBits)*8 > len(committeeKeys) {
return nil, errors.New("bits length exceeds committee length")
}
var votedKeys [][]byte
proposerReward, participantReward, err := s.SyncRewards()
if err != nil {
return nil, err
}
proposerIndex, err := s.GetBeaconProposerIndex()
if err != nil {
return nil, err
}
syncAggregateBits := sync.SyncCommiteeBits
earnedProposerReward := uint64(0)
currPubKeyIndex := 0
for i := range syncAggregateBits {
for bit := 1; bit <= 128; bit *= 2 {
vIdx, exists := s.ValidatorIndexByPubkey(committeeKeys[currPubKeyIndex])
// Impossible scenario.
if !exists {
return nil, errors.New("validator public key does not exist in state")
}
if syncAggregateBits[i]&byte(bit) > 0 {
votedKeys = append(votedKeys, committeeKeys[currPubKeyIndex][:])
if err := state2.IncreaseBalance(s.BeaconState, vIdx, participantReward); err != nil {
return nil, err
}
earnedProposerReward += proposerReward
} else {
if err := state2.DecreaseBalance(s.BeaconState, vIdx, participantReward); err != nil {
return nil, err
}
}
currPubKeyIndex++
}
}
return votedKeys, state2.IncreaseBalance(s.BeaconState, proposerIndex, earnedProposerReward)
}
func ProcessSyncAggregate(s *state2.BeaconState, sync *cltypes.SyncAggregate, fullValidation bool) error {
votedKeys, err := processSyncAggregate(s, sync)
if err != nil {
return err
}
if fullValidation {
previousSlot := s.PreviousSlot()
domain, err := fork.Domain(s.Fork(), state2.GetEpochAtSlot(s.BeaconConfig(), previousSlot), s.BeaconConfig().DomainSyncCommittee, s.GenesisValidatorsRoot())
if err != nil {
return nil
}
blockRoot, err := s.GetBlockRootAtSlot(previousSlot)
if err != nil {
return err
}
msg := utils.Keccak256(blockRoot[:], domain)
isValid, err := bls.VerifyAggregate(sync.SyncCommiteeSignature[:], msg[:], votedKeys)
if err != nil {
return err
}
if !isValid {
return errors.New("ProcessSyncAggregate: cannot validate sync committee signature")
}
}
return nil
}

View File

@ -1,120 +0,0 @@
package transition
import (
"encoding/binary"
"fmt"
state2 "github.com/ledgerwatch/erigon/cl/phase1/core/state"
libcommon "github.com/ledgerwatch/erigon-lib/common"
"github.com/Giulio2002/bls"
"github.com/ledgerwatch/erigon/cl/cltypes"
"github.com/ledgerwatch/erigon/cl/utils"
)
func computeSigningRootEpoch(epoch uint64, domain []byte) (libcommon.Hash, error) {
b := make([]byte, 32)
binary.LittleEndian.PutUint64(b, epoch)
return utils.Keccak256(b, domain), nil
}
func ProcessBlockHeader(state *state2.BeaconState, block *cltypes.BeaconBlock, fullValidation bool) error {
if fullValidation {
if block.Slot != state.Slot() {
return fmt.Errorf("state slot: %d, not equal to block slot: %d", state.Slot(), block.Slot)
}
if block.Slot <= state.LatestBlockHeader().Slot {
return fmt.Errorf("slock slot: %d, not greater than latest block slot: %d", block.Slot, state.LatestBlockHeader().Slot)
}
propInd, err := state.GetBeaconProposerIndex()
if err != nil {
return fmt.Errorf("error in GetBeaconProposerIndex: %v", err)
}
if block.ProposerIndex != propInd {
return fmt.Errorf("block proposer index: %d, does not match beacon proposer index: %d", block.ProposerIndex, propInd)
}
blockHeader := state.LatestBlockHeader()
latestRoot, err := (&blockHeader).HashSSZ()
if err != nil {
return fmt.Errorf("unable to hash tree root of latest block header: %v", err)
}
if block.ParentRoot != latestRoot {
return fmt.Errorf("block parent root: %x, does not match latest block root: %x", block.ParentRoot, latestRoot)
}
}
bodyRoot, err := block.Body.HashSSZ()
if err != nil {
return fmt.Errorf("unable to hash tree root of block body: %v", err)
}
state.SetLatestBlockHeader(&cltypes.BeaconBlockHeader{
Slot: block.Slot,
ProposerIndex: block.ProposerIndex,
ParentRoot: block.ParentRoot,
BodyRoot: bodyRoot,
})
proposer, err := state.ValidatorForValidatorIndex(int(block.ProposerIndex))
if err != nil {
return err
}
if proposer.Slashed() {
return fmt.Errorf("proposer: %d is slashed", block.ProposerIndex)
}
return nil
}
func ProcessRandao(s *state2.BeaconState, randao [96]byte, proposerIndex uint64, fullValidation bool) error {
epoch := state2.Epoch(s.BeaconState)
proposer, err := s.ValidatorForValidatorIndex(int(proposerIndex))
if err != nil {
return err
}
if fullValidation {
domain, err := s.GetDomain(s.BeaconConfig().DomainRandao, epoch)
if err != nil {
return fmt.Errorf("ProcessRandao: unable to get domain: %v", err)
}
signingRoot, err := computeSigningRootEpoch(epoch, domain)
if err != nil {
return fmt.Errorf("ProcessRandao: unable to compute signing root: %v", err)
}
pk := proposer.PublicKey()
valid, err := bls.Verify(randao[:], signingRoot[:], pk[:])
if err != nil {
return fmt.Errorf("ProcessRandao: unable to verify public key: %x, with signing root: %x, and signature: %x, %v", pk[:], signingRoot[:], randao[:], err)
}
if !valid {
return fmt.Errorf("ProcessRandao: invalid signature: public key: %x, signing root: %x, signature: %x", pk[:], signingRoot[:], randao[:])
}
}
randaoMixes := s.GetRandaoMixes(epoch)
randaoHash := utils.Keccak256(randao[:])
mix := [32]byte{}
for i := range mix {
mix[i] = randaoMixes[i] ^ randaoHash[i]
}
s.SetRandaoMixAt(int(epoch%s.BeaconConfig().EpochsPerHistoricalVector), mix)
return nil
}
func ProcessEth1Data(state *state2.BeaconState, eth1Data *cltypes.Eth1Data) error {
state.AddEth1DataVote(eth1Data)
newVotes := state.Eth1DataVotes()
// Count how many times body.Eth1Data appears in the votes.
numVotes := 0
newVotes.Range(func(index int, value *cltypes.Eth1Data, length int) bool {
if eth1Data.Equal(value) {
numVotes += 1
}
return true
})
if uint64(numVotes*2) > state.BeaconConfig().EpochsPerEth1VotingPeriod*state.BeaconConfig().SlotsPerEpoch {
state.SetEth1Data(eth1Data)
}
return nil
}

View File

@ -6,7 +6,7 @@ import (
"github.com/ledgerwatch/erigon/cl/cltypes"
"github.com/ledgerwatch/erigon/cl/cltypes/solid"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/ledgerwatch/erigon/cl/phase1/core/transition"
"github.com/ledgerwatch/erigon/cl/transition"
"github.com/ledgerwatch/log/v3"
"golang.org/x/exp/slices"
)

View File

@ -2,10 +2,9 @@ package forkchoice
import (
"fmt"
"github.com/ledgerwatch/erigon/cl/freezer"
"github.com/ledgerwatch/erigon/cl/phase1/core/transition"
"github.com/ledgerwatch/erigon/cl/phase1/forkchoice/fork_graph"
"github.com/ledgerwatch/erigon/cl/transition/impl/eth2/statechange"
"github.com/ledgerwatch/erigon/cl/cltypes"
"github.com/ledgerwatch/log/v3"
@ -74,7 +73,7 @@ func (f *ForkChoiceStore) OnBlock(block *cltypes.SignedBeaconBlock, newPayload,
justificationBits = lastProcessedState.JustificationBits().Copy()
)
// Eagerly compute unrealized justification and finality
if err := transition.ProcessJustificationBitsAndFinality(lastProcessedState); err != nil {
if err := statechange.ProcessJustificationBitsAndFinality(lastProcessedState); err != nil {
return err
}
f.updateUnrealizedCheckpoints(lastProcessedState.CurrentJustifiedCheckpoint().Copy(), lastProcessedState.FinalizedCheckpoint().Copy())

View File

@ -2,12 +2,11 @@ package forkchoice
import (
"fmt"
"github.com/ledgerwatch/erigon/cl/cltypes/solid"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/ledgerwatch/erigon/cl/phase1/core/transition"
"github.com/ledgerwatch/erigon/cl/transition"
libcommon "github.com/ledgerwatch/erigon-lib/common"
"github.com/ledgerwatch/erigon/cl/cltypes/solid"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/ledgerwatch/log/v3"
)
@ -85,7 +84,7 @@ func (f *ForkChoiceStore) getCheckpointState(checkpoint solid.Checkpoint) (*chec
if baseState.Slot() < f.computeStartSlotAtEpoch(checkpoint.Epoch()) {
log.Debug("Long checkpoint detected")
// If we require to change it then process the future epoch
if err := transition.ProcessSlots(baseState, f.computeStartSlotAtEpoch(checkpoint.Epoch())); err != nil {
if err := transition.DefaultMachine.ProcessSlots(baseState, f.computeStartSlotAtEpoch(checkpoint.Epoch())); err != nil {
return nil, err
}
}

View File

@ -2,10 +2,10 @@ package stages
import (
"context"
"github.com/ledgerwatch/erigon/cl/transition"
"github.com/ledgerwatch/erigon/cl/phase1/core/rawdb"
state2 "github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/ledgerwatch/erigon/cl/phase1/core/transition"
"github.com/ledgerwatch/erigon/cl/phase1/execution_client"
libcommon "github.com/ledgerwatch/erigon-lib/common"

View File

@ -1,13 +1,12 @@
package consensus_tests
import (
"github.com/ledgerwatch/erigon/cl/transition/impl/eth2/statechange"
"io/fs"
"os"
"testing"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
transition2 "github.com/ledgerwatch/erigon/cl/phase1/core/transition"
"github.com/ledgerwatch/erigon/spectest"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
@ -52,58 +51,57 @@ func (b *EpochProcessing) Run(t *testing.T, root fs.FS, c spectest.TestCase) (er
}
var effectiveBalancesUpdateTest = NewEpochProcessing(func(s *state.BeaconState) error {
return transition2.ProcessEffectiveBalanceUpdates(s)
return statechange.ProcessEffectiveBalanceUpdates(s)
})
var eth1DataResetTest = NewEpochProcessing(func(s *state.BeaconState) error {
transition2.ProcessEth1DataReset(s)
statechange.ProcessEth1DataReset(s)
return nil
})
var historicalRootsUpdateTest = NewEpochProcessing(func(s *state.BeaconState) error {
transition2.ProcessHistoricalRootsUpdate(s)
return nil
return statechange.ProcessHistoricalRootsUpdate(s)
})
var inactivityUpdateTest = NewEpochProcessing(func(s *state.BeaconState) error {
return transition2.ProcessInactivityScores(s)
return statechange.ProcessInactivityScores(s)
})
var justificationFinalizationTest = NewEpochProcessing(func(s *state.BeaconState) error {
return transition2.ProcessJustificationBitsAndFinality(s)
return statechange.ProcessJustificationBitsAndFinality(s)
})
var participationFlagUpdatesTest = NewEpochProcessing(func(s *state.BeaconState) error {
transition2.ProcessParticipationFlagUpdates(s)
statechange.ProcessParticipationFlagUpdates(s)
return nil
})
var participationRecordUpdatesTest = NewEpochProcessing(func(s *state.BeaconState) error {
return transition2.ProcessParticipationRecordUpdates(s)
return statechange.ProcessParticipationRecordUpdates(s)
})
var randaoMixesTest = NewEpochProcessing(func(s *state.BeaconState) error {
transition2.ProcessRandaoMixesReset(s)
statechange.ProcessRandaoMixesReset(s)
return nil
})
var registryUpdatesTest = NewEpochProcessing(func(s *state.BeaconState) error {
return transition2.ProcessRegistryUpdates(s)
return statechange.ProcessRegistryUpdates(s)
})
var rewardsAndPenaltiesTest = NewEpochProcessing(func(s *state.BeaconState) error {
return transition2.ProcessRewardsAndPenalties(s)
return statechange.ProcessRewardsAndPenalties(s)
})
var slashingsTest = NewEpochProcessing(func(s *state.BeaconState) error {
return transition2.ProcessSlashings(s)
return statechange.ProcessSlashings(s)
})
var slashingsResetTest = NewEpochProcessing(func(s *state.BeaconState) error {
transition2.ProcessSlashingsReset(s)
statechange.ProcessSlashingsReset(s)
return nil
})
var recordsResetTest = NewEpochProcessing(func(s *state.BeaconState) error {
transition2.ProcessParticipationRecordUpdates(s)
statechange.ProcessParticipationRecordUpdates(s)
return nil
})

View File

@ -2,11 +2,10 @@ package consensus_tests
import (
"fmt"
"github.com/ledgerwatch/erigon/cl/transition/machine"
"io/fs"
"testing"
"github.com/ledgerwatch/erigon/cl/phase1/core/transition"
"github.com/ledgerwatch/erigon/spectest"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
@ -26,7 +25,7 @@ var FinalityFinality = spectest.HandlerFunc(func(t *testing.T, root fs.FS, c spe
}
startSlot := testState.Slot()
for _, block := range blocks {
if err := transition.TransitionState(testState, block, true); err != nil {
if err := machine.TransitionState(c.Machine, testState, block); err != nil {
require.NoError(t, fmt.Errorf("cannot transition state: %w. slot=%d. start_slot=%d", err, block.Block.Slot, startSlot))
}
}

View File

@ -7,7 +7,6 @@ import (
"testing"
"github.com/ledgerwatch/erigon/cl/cltypes/solid"
transition2 "github.com/ledgerwatch/erigon/cl/phase1/core/transition"
"github.com/ledgerwatch/erigon/cl/cltypes"
"github.com/ledgerwatch/erigon/spectest"
@ -39,7 +38,7 @@ func operationAttestationHandler(t *testing.T, root fs.FS, c spectest.TestCase)
if err := spectest.ReadSszOld(root, att, c.Version(), attestationFileName); err != nil {
return err
}
if err := transition2.ProcessAttestations(preState, solid.NewDynamicListSSZFromList([]*solid.Attestation{att}, 128), true); err != nil {
if err := c.Machine.ProcessAttestations(preState, solid.NewDynamicListSSZFromList([]*solid.Attestation{att}, 128)); err != nil {
if expectedError {
return nil
}
@ -69,7 +68,7 @@ func operationAttesterSlashingHandler(t *testing.T, root fs.FS, c spectest.TestC
if err := spectest.ReadSszOld(root, att, c.Version(), attesterSlashingFileName); err != nil {
return err
}
if err := transition2.ProcessAttesterSlashing(preState, att); err != nil {
if err := c.Machine.ProcessAttesterSlashing(preState, att); err != nil {
if expectedError {
return nil
}
@ -99,7 +98,7 @@ func operationProposerSlashingHandler(t *testing.T, root fs.FS, c spectest.TestC
if err := spectest.ReadSszOld(root, att, c.Version(), proposerSlashingFileName); err != nil {
return err
}
if err := transition2.ProcessProposerSlashing(preState, att); err != nil {
if err := c.Machine.ProcessProposerSlashing(preState, att); err != nil {
if expectedError {
return nil
}
@ -129,7 +128,7 @@ func operationBlockHeaderHandler(t *testing.T, root fs.FS, c spectest.TestCase)
if err := spectest.ReadSszOld(root, block, c.Version(), blockFileName); err != nil {
return err
}
if err := transition2.ProcessBlockHeader(preState, block, true); err != nil {
if err := c.Machine.ProcessBlockHeader(preState, block); err != nil {
if expectedError {
return nil
}
@ -159,7 +158,7 @@ func operationDepositHandler(t *testing.T, root fs.FS, c spectest.TestCase) erro
if err := spectest.ReadSszOld(root, deposit, c.Version(), depositFileName); err != nil {
return err
}
if err := transition2.ProcessDeposit(preState, deposit, true); err != nil {
if err := c.Machine.ProcessDeposit(preState, deposit); err != nil {
if expectedError {
return nil
}
@ -189,7 +188,7 @@ func operationSyncAggregateHandler(t *testing.T, root fs.FS, c spectest.TestCase
if err := spectest.ReadSszOld(root, agg, c.Version(), syncAggregateFileName); err != nil {
return err
}
if err := transition2.ProcessSyncAggregate(preState, agg, true); err != nil {
if err := c.Machine.ProcessSyncAggregate(preState, agg); err != nil {
if expectedError {
return nil
}
@ -219,7 +218,7 @@ func operationVoluntaryExitHandler(t *testing.T, root fs.FS, c spectest.TestCase
if err := spectest.ReadSszOld(root, vo, c.Version(), voluntaryExitFileName); err != nil {
return err
}
if err := transition2.ProcessVoluntaryExit(preState, vo, true); err != nil {
if err := c.Machine.ProcessVoluntaryExit(preState, vo); err != nil {
if expectedError {
return nil
}
@ -249,7 +248,7 @@ func operationWithdrawalHandler(t *testing.T, root fs.FS, c spectest.TestCase) e
if err := spectest.ReadSszOld(root, executionPayload, c.Version(), executionPayloadFileName); err != nil {
return err
}
if err := transition2.ProcessWithdrawals(preState, executionPayload.Withdrawals, true); err != nil {
if err := c.Machine.ProcessWithdrawals(preState, executionPayload.Withdrawals); err != nil {
if expectedError {
return nil
}
@ -279,7 +278,7 @@ func operationSignedBlsChangeHandler(t *testing.T, root fs.FS, c spectest.TestCa
if err := spectest.ReadSszOld(root, change, c.Version(), addressChangeFileName); err != nil {
return err
}
if err := transition2.ProcessBlsToExecutionChange(preState, change, true); err != nil {
if err := c.Machine.ProcessBlsToExecutionChange(preState, change); err != nil {
if expectedError {
return nil
}

View File

@ -1,12 +1,11 @@
package consensus_tests
import (
"github.com/ledgerwatch/erigon/cl/transition/machine"
"io/fs"
"os"
"testing"
"github.com/ledgerwatch/erigon/cl/phase1/core/transition"
"github.com/ledgerwatch/erigon/cl/cltypes"
"github.com/ledgerwatch/erigon/spectest"
"github.com/stretchr/testify/assert"
@ -25,7 +24,7 @@ var SanitySlots = spectest.HandlerFunc(func(t *testing.T, root fs.FS, c spectest
expectedState, err := spectest.ReadBeaconState(root, c.Version(), spectest.PostSsz)
require.NoError(t, err)
err = transition.ProcessSlots(testState, expectedState.Slot())
err = c.Machine.ProcessSlots(testState, expectedState.Slot())
require.NoError(t, err)
expectedRoot, err := expectedState.HashSSZ()
@ -65,7 +64,7 @@ var SanityBlocks = spectest.HandlerFunc(func(t *testing.T, root fs.FS, c spectes
var block *cltypes.SignedBeaconBlock
for _, block = range blocks {
err = transition.TransitionState(testState, block, true)
err = machine.TransitionState(c.Machine, testState, block)
if err != nil {
break
}

View File

@ -2,11 +2,10 @@ package consensus_tests
import (
"fmt"
"github.com/ledgerwatch/erigon/cl/transition/machine"
"io/fs"
"testing"
"github.com/ledgerwatch/erigon/cl/phase1/core/transition"
"github.com/ledgerwatch/erigon/cl/clparams"
"github.com/ledgerwatch/erigon/cl/cltypes"
"github.com/ledgerwatch/erigon/spectest"
@ -60,7 +59,7 @@ func (b *TransitionCore) Run(t *testing.T, root fs.FS, c spectest.TestCase) (err
break
}
blockIndex++
if err := transition.TransitionState(startState, block, true); err != nil {
if err := machine.TransitionState(c.Machine, startState, block); err != nil {
return fmt.Errorf("cannot transition state: %s. slot=%d. start_slot=%d", err, block.Block.Slot, startSlot)
}
}

View File

@ -5,6 +5,7 @@
package spectest
import (
"github.com/ledgerwatch/erigon/cl/transition"
"os"
"testing"
@ -14,5 +15,5 @@ import (
)
func Test(t *testing.T) {
spectest.RunCases(t, consensus_tests.TestFormats, os.DirFS("./tests"))
spectest.RunCases(t, consensus_tests.TestFormats, transition.ValidatingMachine, os.DirFS("./tests"))
}

19
cl/transition/compat.go Normal file
View File

@ -0,0 +1,19 @@
package transition
import (
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/ledgerwatch/erigon/cl/transition/impl/eth2"
machine2 "github.com/ledgerwatch/erigon/cl/transition/machine"
"github.com/ledgerwatch/erigon/cl/cltypes"
)
var _ machine2.Interface = (*eth2.Impl)(nil)
var DefaultMachine = &eth2.Impl{}
var ValidatingMachine = &eth2.Impl{FullValidation: true}
func TransitionState(s *state.BeaconState, block *cltypes.SignedBeaconBlock, fullValidation bool) error {
cvm := &eth2.Impl{FullValidation: fullValidation}
return machine2.TransitionState(cvm, s, block)
}

View File

@ -1,7 +1,8 @@
package transition
package eth2_test
import (
_ "embed"
"github.com/ledgerwatch/erigon/cl/transition"
"testing"
"github.com/ledgerwatch/erigon/cl/clparams"
@ -11,16 +12,16 @@ import (
"github.com/stretchr/testify/require"
)
//go:embed test_data/block_processing/capella_block.ssz_snappy
//go:embed statechange/test_data/block_processing/capella_block.ssz_snappy
var capellaBlock []byte
//go:embed test_data/block_processing/capella_state.ssz_snappy
//go:embed statechange/test_data/block_processing/capella_state.ssz_snappy
var capellaState []byte
func TestBlockProcessing(t *testing.T) {
state := state.New(&clparams.MainnetBeaconConfig)
require.NoError(t, utils.DecodeSSZSnappy(state, capellaState, int(clparams.CapellaVersion)))
s := state.New(&clparams.MainnetBeaconConfig)
require.NoError(t, utils.DecodeSSZSnappy(s, capellaState, int(clparams.CapellaVersion)))
block := &cltypes.SignedBeaconBlock{}
require.NoError(t, utils.DecodeSSZSnappy(block, capellaBlock, int(clparams.CapellaVersion)))
require.NoError(t, TransitionState(state, block, true)) // All checks already made in transition state
require.NoError(t, transition.TransitionState(s, block, true)) // All checks already made in transition state
}

View File

@ -0,0 +1,11 @@
package eth2
import "github.com/ledgerwatch/erigon/cl/transition/machine"
type Impl = impl
var _ machine.Interface = (*impl)(nil)
type impl struct {
FullValidation bool
}

View File

@ -0,0 +1,872 @@
package eth2
import (
"bytes"
"errors"
"fmt"
"github.com/ledgerwatch/erigon/cl/transition/impl/eth2/statechange"
"github.com/ledgerwatch/erigon/metrics/methelp"
"golang.org/x/exp/slices"
"reflect"
"time"
"github.com/ledgerwatch/erigon-lib/common"
"github.com/ledgerwatch/erigon/cl/cltypes/solid"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/Giulio2002/bls"
"github.com/ledgerwatch/log/v3"
"github.com/ledgerwatch/erigon/cl/clparams"
"github.com/ledgerwatch/erigon/cl/cltypes"
"github.com/ledgerwatch/erigon/cl/fork"
"github.com/ledgerwatch/erigon/cl/utils"
"github.com/ledgerwatch/erigon/core/types"
)
func (I *impl) ProcessProposerSlashing(s *state.BeaconState, propSlashing *cltypes.ProposerSlashing) error {
h1 := propSlashing.Header1.Header
h2 := propSlashing.Header2.Header
if h1.Slot != h2.Slot {
return fmt.Errorf("non-matching slots on proposer slashing: %d != %d", h1.Slot, h2.Slot)
}
if h1.ProposerIndex != h2.ProposerIndex {
return fmt.Errorf("non-matching proposer indices proposer slashing: %d != %d", h1.ProposerIndex, h2.ProposerIndex)
}
h1Root, err := h1.HashSSZ()
if err != nil {
return fmt.Errorf("unable to hash header1: %v", err)
}
h2Root, err := h2.HashSSZ()
if err != nil {
return fmt.Errorf("unable to hash header2: %v", err)
}
if h1Root == h2Root {
return fmt.Errorf("propose slashing headers are the same: %v == %v", h1Root, h2Root)
}
proposer, err := s.ValidatorForValidatorIndex(int(h1.ProposerIndex))
if err != nil {
return err
}
if !proposer.IsSlashable(state.Epoch(s.BeaconState)) {
return fmt.Errorf("proposer is not slashable: %v", proposer)
}
for _, signedHeader := range []*cltypes.SignedBeaconBlockHeader{propSlashing.Header1, propSlashing.Header2} {
domain, err := s.GetDomain(s.BeaconConfig().DomainBeaconProposer, state.GetEpochAtSlot(s.BeaconConfig(), signedHeader.Header.Slot))
if err != nil {
return fmt.Errorf("unable to get domain: %v", err)
}
signingRoot, err := fork.ComputeSigningRoot(signedHeader.Header, domain)
if err != nil {
return fmt.Errorf("unable to compute signing root: %v", err)
}
pk := proposer.PublicKey()
valid, err := bls.Verify(signedHeader.Signature[:], signingRoot[:], pk[:])
if err != nil {
return fmt.Errorf("unable to verify signature: %v", err)
}
if !valid {
return fmt.Errorf("invalid signature: signature %v, root %v, pubkey %v", signedHeader.Signature[:], signingRoot[:], pk)
}
}
// Set whistleblower index to 0 so current proposer gets reward.
s.SlashValidator(h1.ProposerIndex, nil)
return nil
}
func (I *impl) ProcessAttesterSlashing(s *state.BeaconState, attSlashing *cltypes.AttesterSlashing) error {
att1 := attSlashing.Attestation_1
att2 := attSlashing.Attestation_2
if !cltypes.IsSlashableAttestationData(att1.Data, att2.Data) {
return fmt.Errorf("attestation data not slashable: %+v; %+v", att1.Data, att2.Data)
}
valid, err := state.IsValidIndexedAttestation(s.BeaconState, att1)
if err != nil {
return fmt.Errorf("error calculating indexed attestation 1 validity: %v", err)
}
if !valid {
return fmt.Errorf("invalid indexed attestation 1")
}
valid, err = state.IsValidIndexedAttestation(s.BeaconState, att2)
if err != nil {
return fmt.Errorf("error calculating indexed attestation 2 validity: %v", err)
}
if !valid {
return fmt.Errorf("invalid indexed attestation 2")
}
slashedAny := false
currentEpoch := state.GetEpochAtSlot(s.BeaconConfig(), s.Slot())
for _, ind := range solid.IntersectionOfSortedSets(
solid.IterableSSZ[uint64](att1.AttestingIndices),
solid.IterableSSZ[uint64](att2.AttestingIndices)) {
validator, err := s.ValidatorForValidatorIndex(int(ind))
if err != nil {
return err
}
if validator.IsSlashable(currentEpoch) {
err := s.SlashValidator(ind, nil)
if err != nil {
return fmt.Errorf("unable to slash validator: %d", ind)
}
slashedAny = true
}
}
if !slashedAny {
return fmt.Errorf("no validators slashed")
}
return nil
}
func (I *impl) ProcessDeposit(s *state.BeaconState, deposit *cltypes.Deposit) error {
if deposit == nil {
return nil
}
depositLeaf, err := deposit.Data.HashSSZ()
if err != nil {
return err
}
depositIndex := s.Eth1DepositIndex()
eth1Data := s.Eth1Data()
rawProof := []common.Hash{}
deposit.Proof.Range(func(_ int, l common.Hash, _ int) bool {
rawProof = append(rawProof, l)
return true
})
// Validate merkle proof for deposit leaf.
if I.FullValidation && !utils.IsValidMerkleBranch(
depositLeaf,
rawProof,
s.BeaconConfig().DepositContractTreeDepth+1,
depositIndex,
eth1Data.Root,
) {
return fmt.Errorf("processDepositForAltair: Could not validate deposit root")
}
// Increment index
s.SetEth1DepositIndex(depositIndex + 1)
publicKey := deposit.Data.PubKey
amount := deposit.Data.Amount
// Check if pub key is in validator set
validatorIndex, has := s.ValidatorIndexByPubkey(publicKey)
if !has {
// Agnostic domain.
domain, err := fork.ComputeDomain(s.BeaconConfig().DomainDeposit[:], utils.Uint32ToBytes4(s.BeaconConfig().GenesisForkVersion), [32]byte{})
if err != nil {
return err
}
depositMessageRoot, err := deposit.Data.MessageHash()
if err != nil {
return err
}
signedRoot := utils.Keccak256(depositMessageRoot[:], domain)
// Perform BLS verification and if successful noice.
valid, err := bls.Verify(deposit.Data.Signature[:], signedRoot[:], publicKey[:])
// Literally you can input it trash.
if !valid || err != nil {
log.Debug("Validator BLS verification failed", "valid", valid, "err", err)
return nil
}
// Append validator
s.AddValidator(state.ValidatorFromDeposit(s.BeaconConfig(), deposit), amount)
// Altair forward
if s.Version() >= clparams.AltairVersion {
s.AddCurrentEpochParticipationFlags(cltypes.ParticipationFlags(0))
s.AddPreviousEpochParticipationFlags(cltypes.ParticipationFlags(0))
s.AddInactivityScore(0)
}
return nil
}
// Increase the balance if exists already
return state.IncreaseBalance(s.BeaconState, validatorIndex, amount)
}
// ProcessVoluntaryExit takes a voluntary exit and applies state transition.
func (I *impl) ProcessVoluntaryExit(s *state.BeaconState, signedVoluntaryExit *cltypes.SignedVoluntaryExit) error {
// Sanity checks so that we know it is good.
voluntaryExit := signedVoluntaryExit.VolunaryExit
currentEpoch := state.Epoch(s.BeaconState)
validator, err := s.ValidatorForValidatorIndex(int(voluntaryExit.ValidatorIndex))
if err != nil {
return err
}
if !validator.Active(currentEpoch) {
return errors.New("ProcessVoluntaryExit: validator is not active")
}
if validator.ExitEpoch() != s.BeaconConfig().FarFutureEpoch {
return errors.New("ProcessVoluntaryExit: another exit for the same validator is already getting processed")
}
if currentEpoch < voluntaryExit.Epoch {
return errors.New("ProcessVoluntaryExit: exit is happening in the future")
}
if currentEpoch < validator.ActivationEpoch()+s.BeaconConfig().ShardCommitteePeriod {
return errors.New("ProcessVoluntaryExit: exit is happening too fast")
}
// We can skip it in some instances if we want to optimistically sync up.
if I.FullValidation {
domain, err := s.GetDomain(s.BeaconConfig().DomainVoluntaryExit, voluntaryExit.Epoch)
if err != nil {
return err
}
signingRoot, err := fork.ComputeSigningRoot(voluntaryExit, domain)
if err != nil {
return err
}
pk := validator.PublicKey()
valid, err := bls.Verify(signedVoluntaryExit.Signature[:], signingRoot[:], pk[:])
if err != nil {
return err
}
if !valid {
return errors.New("ProcessVoluntaryExit: BLS verification failed")
}
}
// Do the exit (same process in slashing).
return s.InitiateValidatorExit(voluntaryExit.ValidatorIndex)
}
// ProcessWithdrawals processes withdrawals by decreasing the balance of each validator
// and updating the next withdrawal index and validator index.
func (I *impl) ProcessWithdrawals(s *state.BeaconState, withdrawals *solid.ListSSZ[*types.Withdrawal]) error {
// Get the list of withdrawals, the expected withdrawals (if performing full validation),
// and the beacon configuration.
beaconConfig := s.BeaconConfig()
numValidators := uint64(s.ValidatorLength())
// Check if full validation is required and verify expected withdrawals.
if I.FullValidation {
expectedWithdrawals := state.ExpectedWithdrawals(s.BeaconState)
if len(expectedWithdrawals) != withdrawals.Len() {
return fmt.Errorf("ProcessWithdrawals: expected %d withdrawals, but got %d", len(expectedWithdrawals), withdrawals.Len())
}
if err := solid.RangeErr[*types.Withdrawal](withdrawals, func(i int, w *types.Withdrawal, _ int) error {
if !expectedWithdrawals[i].Equal(w) {
return fmt.Errorf("ProcessWithdrawals: withdrawal %d does not match expected withdrawal", i)
}
return nil
}); err != nil {
return err
}
}
if err := solid.RangeErr[*types.Withdrawal](withdrawals, func(_ int, w *types.Withdrawal, _ int) error {
if err := state.DecreaseBalance(s.BeaconState, w.Validator, w.Amount); err != nil {
return err
}
return nil
}); err != nil {
return err
}
// Update next withdrawal index based on number of withdrawals.
if withdrawals.Len() > 0 {
lastWithdrawalIndex := withdrawals.Get(withdrawals.Len() - 1).Index
s.SetNextWithdrawalIndex(lastWithdrawalIndex + 1)
}
// Update next withdrawal validator index based on number of withdrawals.
if withdrawals.Len() == int(beaconConfig.MaxWithdrawalsPerPayload) {
lastWithdrawalValidatorIndex := withdrawals.Get(withdrawals.Len()-1).Validator + 1
s.SetNextWithdrawalValidatorIndex(lastWithdrawalValidatorIndex % numValidators)
} else {
nextIndex := s.NextWithdrawalValidatorIndex() + beaconConfig.MaxValidatorsPerWithdrawalsSweep
s.SetNextWithdrawalValidatorIndex(nextIndex % numValidators)
}
return nil
}
// ProcessExecutionPayload sets the latest payload header accordinly.
func (I *impl) ProcessExecutionPayload(s *state.BeaconState, payload *cltypes.Eth1Block) error {
if state.IsMergeTransitionComplete(s.BeaconState) {
if payload.ParentHash != s.LatestExecutionPayloadHeader().BlockHash {
return fmt.Errorf("ProcessExecutionPayload: invalid eth1 chain. mismatching parent")
}
}
if payload.PrevRandao != s.GetRandaoMixes(state.Epoch(s.BeaconState)) {
return fmt.Errorf("ProcessExecutionPayload: randao mix mismatches with mix digest")
}
if payload.Time != state.ComputeTimestampAtSlot(s.BeaconState, s.Slot()) {
return fmt.Errorf("ProcessExecutionPayload: invalid Eth1 timestamp")
}
payloadHeader, err := payload.PayloadHeader()
if err != nil {
return err
}
s.SetLatestExecutionPayloadHeader(payloadHeader)
return nil
}
func (I *impl) ProcessSyncAggregate(s *state.BeaconState, sync *cltypes.SyncAggregate) error {
votedKeys, err := processSyncAggregate(s, sync)
if err != nil {
return err
}
if I.FullValidation {
previousSlot := s.PreviousSlot()
domain, err := fork.Domain(s.Fork(), state.GetEpochAtSlot(s.BeaconConfig(), previousSlot), s.BeaconConfig().DomainSyncCommittee, s.GenesisValidatorsRoot())
if err != nil {
return nil
}
blockRoot, err := s.GetBlockRootAtSlot(previousSlot)
if err != nil {
return err
}
msg := utils.Keccak256(blockRoot[:], domain)
isValid, err := bls.VerifyAggregate(sync.SyncCommiteeSignature[:], msg[:], votedKeys)
if err != nil {
return err
}
if !isValid {
return errors.New("ProcessSyncAggregate: cannot validate sync committee signature")
}
}
return nil
}
// processSyncAggregate applies all the logic in the spec function `process_sync_aggregate` except
// verifying the BLS signatures. It returns the modified beacons state and the list of validators'
// public keys that voted, for future signature verification.
func processSyncAggregate(s *state.BeaconState, sync *cltypes.SyncAggregate) ([][]byte, error) {
currentSyncCommittee := s.CurrentSyncCommittee()
if currentSyncCommittee == nil {
return nil, errors.New("nil current sync committee in s")
}
committeeKeys := currentSyncCommittee.GetCommittee()
if len(sync.SyncCommiteeBits)*8 > len(committeeKeys) {
return nil, errors.New("bits length exceeds committee length")
}
var votedKeys [][]byte
proposerReward, participantReward, err := s.SyncRewards()
if err != nil {
return nil, err
}
proposerIndex, err := s.GetBeaconProposerIndex()
if err != nil {
return nil, err
}
syncAggregateBits := sync.SyncCommiteeBits
earnedProposerReward := uint64(0)
currPubKeyIndex := 0
for i := range syncAggregateBits {
for bit := 1; bit <= 128; bit *= 2 {
vIdx, exists := s.ValidatorIndexByPubkey(committeeKeys[currPubKeyIndex])
// Impossible scenario.
if !exists {
return nil, errors.New("validator public key does not exist in state")
}
if syncAggregateBits[i]&byte(bit) > 0 {
votedKeys = append(votedKeys, committeeKeys[currPubKeyIndex][:])
if err := state.IncreaseBalance(s.BeaconState, vIdx, participantReward); err != nil {
return nil, err
}
earnedProposerReward += proposerReward
} else {
if err := state.DecreaseBalance(s.BeaconState, vIdx, participantReward); err != nil {
return nil, err
}
}
currPubKeyIndex++
}
}
return votedKeys, state.IncreaseBalance(s.BeaconState, proposerIndex, earnedProposerReward)
}
// ProcessBlsToExecutionChange processes a BLSToExecutionChange message by updating a validator's withdrawal credentials.
func (I *impl) ProcessBlsToExecutionChange(s *state.BeaconState, signedChange *cltypes.SignedBLSToExecutionChange) error {
change := signedChange.Message
beaconConfig := s.BeaconConfig()
validator, err := s.ValidatorForValidatorIndex(int(change.ValidatorIndex))
if err != nil {
return err
}
// Perform full validation if requested.
wc := validator.WithdrawalCredentials()
if I.FullValidation {
// Check the validator's withdrawal credentials prefix.
if wc[0] != beaconConfig.BLSWithdrawalPrefixByte {
return fmt.Errorf("invalid withdrawal credentials prefix")
}
// Check the validator's withdrawal credentials against the provided message.
hashedFrom := utils.Keccak256(change.From[:])
if !bytes.Equal(hashedFrom[1:], wc[1:]) {
return fmt.Errorf("invalid withdrawal credentials")
}
// Compute the signing domain and verify the message signature.
domain, err := fork.ComputeDomain(beaconConfig.DomainBLSToExecutionChange[:], utils.Uint32ToBytes4(beaconConfig.GenesisForkVersion), s.GenesisValidatorsRoot())
if err != nil {
return err
}
signedRoot, err := fork.ComputeSigningRoot(change, domain)
if err != nil {
return err
}
valid, err := bls.Verify(signedChange.Signature[:], signedRoot[:], change.From[:])
if err != nil {
return err
}
if !valid {
return fmt.Errorf("invalid signature")
}
}
credentials := wc
// Reset the validator's withdrawal credentials.
credentials[0] = beaconConfig.ETH1AddressWithdrawalPrefixByte
copy(credentials[1:], make([]byte, 11))
copy(credentials[12:], change.To[:])
// Update the state with the modified validator.
s.SetWithdrawalCredentialForValidatorAtIndex(int(change.ValidatorIndex), credentials)
return nil
}
func (I *impl) VerifyKzgCommitmentsAgainstTransactions(transactions *solid.TransactionsSSZ, kzgCommitments *solid.ListSSZ[*cltypes.KZGCommitment]) (bool, error) {
if I.FullValidation {
return true, nil
}
allVersionedHashes := []common.Hash{}
transactions.ForEach(func(tx []byte, idx, total int) bool {
if tx[0] != types.BlobTxType {
return true
}
allVersionedHashes = append(allVersionedHashes, txPeekBlobVersionedHashes(tx)...)
return true
})
commitmentVersionedHash := []common.Hash{}
var err error
var versionedHash common.Hash
kzgCommitments.Range(func(index int, value *cltypes.KZGCommitment, length int) bool {
versionedHash, err = kzgCommitmentToVersionedHash(value)
if err != nil {
return false
}
commitmentVersionedHash = append(commitmentVersionedHash, versionedHash)
return true
})
if err != nil {
return false, err
}
return reflect.DeepEqual(allVersionedHashes, commitmentVersionedHash), nil
}
func (I *impl) ProcessAttestations(s *state.BeaconState, attestations *solid.ListSSZ[*solid.Attestation]) error {
attestingIndiciesSet := make([][]uint64, attestations.Len())
h := methelp.NewHistTimer("beacon_process_attestations")
baseRewardPerIncrement := s.BaseRewardPerIncrement()
c := h.Tag("attestation_step", "process")
var err error
if err := solid.RangeErr[*solid.Attestation](attestations, func(i int, a *solid.Attestation, _ int) error {
if attestingIndiciesSet[i], err = processAttestation(s, a, baseRewardPerIncrement); err != nil {
return err
}
return nil
}); err != nil {
return err
}
if err != nil {
return err
}
var valid bool
c.PutSince()
if I.FullValidation {
c = h.Tag("attestation_step", "validate")
valid, err = verifyAttestations(s, attestations, attestingIndiciesSet)
if err != nil {
return err
}
if !valid {
return errors.New("ProcessAttestation: wrong bls data")
}
c.PutSince()
}
return nil
}
func processAttestationPostAltair(s *state.BeaconState, attestation *solid.Attestation, baseRewardPerIncrement uint64) ([]uint64, error) {
data := attestation.AttestantionData()
currentEpoch := state.Epoch(s.BeaconState)
stateSlot := s.Slot()
beaconConfig := s.BeaconConfig()
h := methelp.NewHistTimer("beacon_process_attestation_post_altair")
c := h.Tag("step", "get_participation_flag")
participationFlagsIndicies, err := s.GetAttestationParticipationFlagIndicies(attestation.AttestantionData(), stateSlot-data.Slot())
if err != nil {
return nil, err
}
c.PutSince()
c = h.Tag("step", "get_attesting_indices")
attestingIndicies, err := s.GetAttestingIndicies(attestation.AttestantionData(), attestation.AggregationBits(), true)
if err != nil {
return nil, err
}
c.PutSince()
var proposerRewardNumerator uint64
isCurrentEpoch := data.Target().Epoch() == currentEpoch
c = h.Tag("step", "update_attestation")
for _, attesterIndex := range attestingIndicies {
val, err := s.ValidatorEffectiveBalance(int(attesterIndex))
if err != nil {
return nil, err
}
baseReward := (val / beaconConfig.EffectiveBalanceIncrement) * baseRewardPerIncrement
for flagIndex, weight := range beaconConfig.ParticipationWeights() {
flagParticipation := s.EpochParticipationForValidatorIndex(isCurrentEpoch, int(attesterIndex))
if !slices.Contains(participationFlagsIndicies, uint8(flagIndex)) || flagParticipation.HasFlag(flagIndex) {
continue
}
s.SetEpochParticipationForValidatorIndex(isCurrentEpoch, int(attesterIndex), flagParticipation.Add(flagIndex))
proposerRewardNumerator += baseReward * weight
}
}
c.PutSince()
// Reward proposer
c = h.Tag("step", "get_proposer_index")
proposer, err := s.GetBeaconProposerIndex()
if err != nil {
return nil, err
}
c.PutSince()
proposerRewardDenominator := (beaconConfig.WeightDenominator - beaconConfig.ProposerWeight) * beaconConfig.WeightDenominator / beaconConfig.ProposerWeight
reward := proposerRewardNumerator / proposerRewardDenominator
return attestingIndicies, state.IncreaseBalance(s.BeaconState, proposer, reward)
}
// processAttestationsPhase0 implements the rules for phase0 processing.
func processAttestationPhase0(s *state.BeaconState, attestation *solid.Attestation) ([]uint64, error) {
data := attestation.AttestantionData()
committee, err := s.GetBeaconCommitee(data.Slot(), data.ValidatorIndex())
if err != nil {
return nil, err
}
if len(committee) != utils.GetBitlistLength(attestation.AggregationBits()) {
return nil, fmt.Errorf("processAttestationPhase0: mismatching aggregation bits size")
}
// Cached so it is performant.
proposerIndex, err := s.GetBeaconProposerIndex()
if err != nil {
return nil, err
}
// Create the attestation to add to pending attestations
pendingAttestation := solid.NewPendingAttestionFromParameters(
attestation.AggregationBits(),
data,
s.Slot()-data.Slot(),
proposerIndex,
)
isCurrentAttestation := data.Target().Epoch() == state.Epoch(s.BeaconState)
// Depending of what slot we are on we put in either the current justified or previous justified.
if isCurrentAttestation {
if !data.Source().Equal(s.CurrentJustifiedCheckpoint()) {
return nil, fmt.Errorf("processAttestationPhase0: mismatching sources")
}
s.AddCurrentEpochAtteastation(pendingAttestation)
} else {
if !data.Source().Equal(s.PreviousJustifiedCheckpoint()) {
return nil, fmt.Errorf("processAttestationPhase0: mismatching sources")
}
s.AddPreviousEpochAttestation(pendingAttestation)
}
// Not required by specs but needed if we want performant epoch transition.
indicies, err := s.GetAttestingIndicies(attestation.AttestantionData(), attestation.AggregationBits(), true)
if err != nil {
return nil, err
}
epochRoot, err := state.GetBlockRoot(s.BeaconState, attestation.AttestantionData().Target().Epoch())
if err != nil {
return nil, err
}
slotRoot, err := s.GetBlockRootAtSlot(attestation.AttestantionData().Slot())
if err != nil {
return nil, err
}
// Basically we flag all validators we are currently attesting. will be important for rewards/finalization processing.
for _, index := range indicies {
minCurrentInclusionDelayAttestation, err := s.ValidatorMinCurrentInclusionDelayAttestation(int(index))
if err != nil {
return nil, err
}
minPreviousInclusionDelayAttestation, err := s.ValidatorMinPreviousInclusionDelayAttestation(int(index))
if err != nil {
return nil, err
}
// NOTE: does not affect state root.
// We need to set it to currents or previouses depending on which attestation we process.
if isCurrentAttestation {
if minCurrentInclusionDelayAttestation == nil ||
minCurrentInclusionDelayAttestation.InclusionDelay() > pendingAttestation.InclusionDelay() {
if err := s.SetValidatorMinCurrentInclusionDelayAttestation(int(index), pendingAttestation); err != nil {
return nil, err
}
}
if err := s.SetValidatorIsCurrentMatchingSourceAttester(int(index), true); err != nil {
return nil, err
}
if attestation.AttestantionData().Target().BlockRoot() == epochRoot {
if err := s.SetValidatorIsCurrentMatchingTargetAttester(int(index), true); err != nil {
return nil, err
}
} else {
continue
}
if attestation.AttestantionData().BeaconBlockRoot() == slotRoot {
if err := s.SetValidatorIsCurrentMatchingHeadAttester(int(index), true); err != nil {
return nil, err
}
}
} else {
if minPreviousInclusionDelayAttestation == nil ||
minPreviousInclusionDelayAttestation.InclusionDelay() > pendingAttestation.InclusionDelay() {
if err := s.SetValidatorMinPreviousInclusionDelayAttestation(int(index), pendingAttestation); err != nil {
return nil, err
}
}
if err := s.SetValidatorIsPreviousMatchingSourceAttester(int(index), true); err != nil {
return nil, err
}
if attestation.AttestantionData().Target().BlockRoot() != epochRoot {
continue
}
if err := s.SetValidatorIsPreviousMatchingTargetAttester(int(index), true); err != nil {
return nil, err
}
if attestation.AttestantionData().BeaconBlockRoot() == slotRoot {
if err := s.SetValidatorIsPreviousMatchingHeadAttester(int(index), true); err != nil {
return nil, err
}
}
}
}
return indicies, nil
}
// ProcessAttestation takes an attestation and process it.
func processAttestation(s *state.BeaconState, attestation *solid.Attestation, baseRewardPerIncrement uint64) ([]uint64, error) {
data := attestation.AttestantionData()
currentEpoch := state.Epoch(s.BeaconState)
previousEpoch := state.PreviousEpoch(s.BeaconState)
stateSlot := s.Slot()
beaconConfig := s.BeaconConfig()
// Prelimary checks.
if (data.Target().Epoch() != currentEpoch && data.Target().Epoch() != previousEpoch) || data.Target().Epoch() != state.GetEpochAtSlot(s.BeaconConfig(), data.Slot()) {
return nil, errors.New("ProcessAttestation: attestation with invalid epoch")
}
if data.Slot()+beaconConfig.MinAttestationInclusionDelay > stateSlot || stateSlot > data.Slot()+beaconConfig.SlotsPerEpoch {
return nil, errors.New("ProcessAttestation: attestation slot not in range")
}
if data.ValidatorIndex() >= s.CommitteeCount(data.Target().Epoch()) {
return nil, errors.New("ProcessAttestation: attester index out of range")
}
// check if we need to use rules for phase0 or post-altair.
if s.Version() == clparams.Phase0Version {
return processAttestationPhase0(s, attestation)
}
return processAttestationPostAltair(s, attestation, baseRewardPerIncrement)
}
func verifyAttestations(s *state.BeaconState, attestations *solid.ListSSZ[*solid.Attestation], attestingIndicies [][]uint64) (bool, error) {
var err error
valid := true
attestations.Range(func(idx int, a *solid.Attestation, _ int) bool {
indexedAttestation := state.GetIndexedAttestation(a, attestingIndicies[idx])
valid, err = state.IsValidIndexedAttestation(s.BeaconState, indexedAttestation)
if err != nil {
return false
}
if !valid {
return false
}
return true
})
return valid, err
}
func (I *impl) ProcessBlockHeader(s *state.BeaconState, block *cltypes.BeaconBlock) error {
if I.FullValidation {
if block.Slot != s.Slot() {
return fmt.Errorf("state slot: %d, not equal to block slot: %d", s.Slot(), block.Slot)
}
if block.Slot <= s.LatestBlockHeader().Slot {
return fmt.Errorf("slock slot: %d, not greater than latest block slot: %d", block.Slot, s.LatestBlockHeader().Slot)
}
propInd, err := s.GetBeaconProposerIndex()
if err != nil {
return fmt.Errorf("error in GetBeaconProposerIndex: %v", err)
}
if block.ProposerIndex != propInd {
return fmt.Errorf("block proposer index: %d, does not match beacon proposer index: %d", block.ProposerIndex, propInd)
}
blockHeader := s.LatestBlockHeader()
latestRoot, err := (&blockHeader).HashSSZ()
if err != nil {
return fmt.Errorf("unable to hash tree root of latest block header: %v", err)
}
if block.ParentRoot != latestRoot {
return fmt.Errorf("block parent root: %x, does not match latest block root: %x", block.ParentRoot, latestRoot)
}
}
bodyRoot, err := block.Body.HashSSZ()
if err != nil {
return fmt.Errorf("unable to hash tree root of block body: %v", err)
}
s.SetLatestBlockHeader(&cltypes.BeaconBlockHeader{
Slot: block.Slot,
ProposerIndex: block.ProposerIndex,
ParentRoot: block.ParentRoot,
BodyRoot: bodyRoot,
})
proposer, err := s.ValidatorForValidatorIndex(int(block.ProposerIndex))
if err != nil {
return err
}
if proposer.Slashed() {
return fmt.Errorf("proposer: %d is slashed", block.ProposerIndex)
}
return nil
}
func (I *impl) ProcessRandao(s *state.BeaconState, randao [96]byte, proposerIndex uint64) error {
epoch := state.Epoch(s.BeaconState)
proposer, err := s.ValidatorForValidatorIndex(int(proposerIndex))
if err != nil {
return err
}
if I.FullValidation {
domain, err := s.GetDomain(s.BeaconConfig().DomainRandao, epoch)
if err != nil {
return fmt.Errorf("ProcessRandao: unable to get domain: %v", err)
}
signingRoot, err := computeSigningRootEpoch(epoch, domain)
if err != nil {
return fmt.Errorf("ProcessRandao: unable to compute signing root: %v", err)
}
pk := proposer.PublicKey()
valid, err := bls.Verify(randao[:], signingRoot[:], pk[:])
if err != nil {
return fmt.Errorf("ProcessRandao: unable to verify public key: %x, with signing root: %x, and signature: %x, %v", pk[:], signingRoot[:], randao[:], err)
}
if !valid {
return fmt.Errorf("ProcessRandao: invalid signature: public key: %x, signing root: %x, signature: %x", pk[:], signingRoot[:], randao[:])
}
}
randaoMixes := s.GetRandaoMixes(epoch)
randaoHash := utils.Keccak256(randao[:])
mix := [32]byte{}
for i := range mix {
mix[i] = randaoMixes[i] ^ randaoHash[i]
}
s.SetRandaoMixAt(int(epoch%s.BeaconConfig().EpochsPerHistoricalVector), mix)
return nil
}
func (I *impl) ProcessEth1Data(state *state.BeaconState, eth1Data *cltypes.Eth1Data) error {
state.AddEth1DataVote(eth1Data)
newVotes := state.Eth1DataVotes()
// Count how many times body.Eth1Data appears in the votes.
numVotes := 0
newVotes.Range(func(index int, value *cltypes.Eth1Data, length int) bool {
if eth1Data.Equal(value) {
numVotes += 1
}
return true
})
if uint64(numVotes*2) > state.BeaconConfig().EpochsPerEth1VotingPeriod*state.BeaconConfig().SlotsPerEpoch {
state.SetEth1Data(eth1Data)
}
return nil
}
func (I *impl) ProcessSlots(s *state.BeaconState, slot uint64) error {
beaconConfig := s.BeaconConfig()
sSlot := s.Slot()
if slot <= sSlot {
return fmt.Errorf("new slot: %d not greater than s slot: %d", slot, sSlot)
}
// Process each slot.
for i := sSlot; i < slot; i++ {
err := transitionSlot(s)
if err != nil {
return fmt.Errorf("unable to process slot transition: %v", err)
}
// TODO(Someone): Add epoch transition.
if (sSlot+1)%beaconConfig.SlotsPerEpoch == 0 {
start := time.Now()
if err := statechange.ProcessEpoch(s); err != nil {
return err
}
log.Debug("Processed new epoch successfully", "epoch", state.Epoch(s.BeaconState), "process_epoch_elpsed", time.Since(start))
}
// TODO: add logic to process epoch updates.
sSlot += 1
s.SetSlot(sSlot)
if sSlot%beaconConfig.SlotsPerEpoch != 0 {
continue
}
if state.Epoch(s.BeaconState) == beaconConfig.AltairForkEpoch {
if err := s.UpgradeToAltair(); err != nil {
return err
}
}
if state.Epoch(s.BeaconState) == beaconConfig.BellatrixForkEpoch {
if err := s.UpgradeToBellatrix(); err != nil {
return err
}
}
if state.Epoch(s.BeaconState) == beaconConfig.CapellaForkEpoch {
if err := s.UpgradeToCapella(); err != nil {
return err
}
}
if state.Epoch(s.BeaconState) == beaconConfig.DenebForkEpoch {
if err := s.UpgradeToDeneb(); err != nil {
return err
}
}
}
return nil
}

View File

@ -1,17 +1,17 @@
package transition
package statechange
import (
"github.com/ledgerwatch/erigon/cl/clparams"
"github.com/ledgerwatch/erigon/cl/cltypes"
"github.com/ledgerwatch/erigon/cl/cltypes/solid"
state2 "github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
)
// weighJustificationAndFinalization checks justification and finality of epochs and adds records to the state as needed.
func weighJustificationAndFinalization(s *state2.BeaconState, previousEpochTargetBalance, currentEpochTargetBalance uint64) error {
func weighJustificationAndFinalization(s *state.BeaconState, previousEpochTargetBalance, currentEpochTargetBalance uint64) error {
totalActiveBalance := s.GetTotalActiveBalance()
currentEpoch := state2.Epoch(s.BeaconState)
previousEpoch := state2.PreviousEpoch(s.BeaconState)
currentEpoch := state.Epoch(s.BeaconState)
previousEpoch := state.PreviousEpoch(s.BeaconState)
oldPreviousJustifiedCheckpoint := s.PreviousJustifiedCheckpoint()
oldCurrentJustifiedCheckpoint := s.CurrentJustifiedCheckpoint()
justificationBits := s.JustificationBits()
@ -23,7 +23,7 @@ func weighJustificationAndFinalization(s *state2.BeaconState, previousEpochTarge
justificationBits[0] = false
// Update justified checkpoint if super majority is reached on previous epoch
if previousEpochTargetBalance*3 >= totalActiveBalance*2 {
checkPointRoot, err := state2.GetBlockRoot(s.BeaconState, previousEpoch)
checkPointRoot, err := state.GetBlockRoot(s.BeaconState, previousEpoch)
if err != nil {
return err
}
@ -32,7 +32,7 @@ func weighJustificationAndFinalization(s *state2.BeaconState, previousEpochTarge
justificationBits[1] = true
}
if currentEpochTargetBalance*3 >= totalActiveBalance*2 {
checkPointRoot, err := state2.GetBlockRoot(s.BeaconState, currentEpoch)
checkPointRoot, err := state.GetBlockRoot(s.BeaconState, currentEpoch)
if err != nil {
return err
}
@ -58,9 +58,9 @@ func weighJustificationAndFinalization(s *state2.BeaconState, previousEpochTarge
return nil
}
func ProcessJustificationBitsAndFinality(s *state2.BeaconState) error {
currentEpoch := state2.Epoch(s.BeaconState)
previousEpoch := state2.PreviousEpoch(s.BeaconState)
func ProcessJustificationBitsAndFinality(s *state.BeaconState) error {
currentEpoch := state.Epoch(s.BeaconState)
previousEpoch := state.PreviousEpoch(s.BeaconState)
beaconConfig := s.BeaconConfig()
// Skip for first 2 epochs
if currentEpoch <= beaconConfig.GenesisEpoch+1 {

View File

@ -1,4 +1,4 @@
package transition
package statechange
import (
"github.com/ledgerwatch/erigon/cl/cltypes/solid"

View File

@ -0,0 +1,49 @@
package statechange
import (
"github.com/ledgerwatch/erigon/cl/clparams"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
)
// ProcessEpoch process epoch transition.
func ProcessEpoch(state *state.BeaconState) error {
if err := ProcessJustificationBitsAndFinality(state); err != nil {
return err
}
if state.Version() >= clparams.AltairVersion {
if err := ProcessInactivityScores(state); err != nil {
return err
}
}
if err := ProcessRewardsAndPenalties(state); err != nil {
return err
}
if err := ProcessRegistryUpdates(state); err != nil {
return err
}
if err := ProcessSlashings(state); err != nil {
return err
}
ProcessEth1DataReset(state)
if err := ProcessEffectiveBalanceUpdates(state); err != nil {
return err
}
ProcessSlashingsReset(state)
ProcessRandaoMixesReset(state)
if err := ProcessHistoricalRootsUpdate(state); err != nil {
return err
}
if state.Version() == clparams.Phase0Version {
if err := ProcessParticipationRecordUpdates(state); err != nil {
return err
}
}
if state.Version() >= clparams.AltairVersion {
ProcessParticipationFlagUpdates(state)
if err := ProcessSyncCommitteeUpdate(state); err != nil {
return err
}
}
return nil
}

View File

@ -1,13 +1,11 @@
package transition_test
package statechange
import (
_ "embed"
"testing"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
transition2 "github.com/ledgerwatch/erigon/cl/phase1/core/transition"
"github.com/ledgerwatch/erigon/cl/clparams"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/ledgerwatch/erigon/cl/utils"
"github.com/stretchr/testify/require"
)
@ -91,64 +89,64 @@ var startingSlashingsResetState []byte
func TestProcessRewardsAndPenalties(t *testing.T) {
runEpochTransitionConsensusTest(t, startingRewardsPenaltyState, expectedRewardsPenaltyState, func(s *state.BeaconState) error {
return transition2.ProcessRewardsAndPenalties(s)
return ProcessRewardsAndPenalties(s)
})
}
func TestProcessRegistryUpdates(t *testing.T) {
runEpochTransitionConsensusTest(t, startingRegistryUpdatesState, expectedRegistryUpdatesState, func(s *state.BeaconState) error {
return transition2.ProcessRegistryUpdates(s)
return ProcessRegistryUpdates(s)
})
}
func TestProcessEffectiveBalances(t *testing.T) {
runEpochTransitionConsensusTest(t, startingEffectiveBalancesState, expectedEffectiveBalancesState, func(s *state.BeaconState) error {
return transition2.ProcessEffectiveBalanceUpdates(s)
return ProcessEffectiveBalanceUpdates(s)
})
}
func TestProcessHistoricalRoots(t *testing.T) {
runEpochTransitionConsensusTest(t, startingHistoricalRootsState, expectedHistoricalRootsState, func(s *state.BeaconState) error {
return transition2.ProcessHistoricalRootsUpdate(s)
return ProcessHistoricalRootsUpdate(s)
})
}
func TestProcessParticipationFlagUpdates(t *testing.T) {
runEpochTransitionConsensusTest(t, startingParticipationFlagState, expectedParticipationFlagState, func(s *state.BeaconState) error {
transition2.ProcessParticipationFlagUpdates(s)
ProcessParticipationFlagUpdates(s)
return nil
})
}
func TestProcessSlashings(t *testing.T) {
runEpochTransitionConsensusTest(t, startingSlashingsState, expectedSlashingsState, func(s *state.BeaconState) error {
return transition2.ProcessSlashings(s)
return ProcessSlashings(s)
})
}
func TestProcessJustificationAndFinality(t *testing.T) {
runEpochTransitionConsensusTest(t, startingJustificationAndFinalityState, expectedJustificationAndFinalityState, func(s *state.BeaconState) error {
return transition2.ProcessJustificationBitsAndFinality(s)
return ProcessJustificationBitsAndFinality(s)
})
}
func TestEth1DataReset(t *testing.T) {
runEpochTransitionConsensusTest(t, startingEth1DataResetState, expectedEth1DataResetState, func(s *state.BeaconState) error {
transition2.ProcessEth1DataReset(s)
ProcessEth1DataReset(s)
return nil
})
}
func TestRandaoMixesReset(t *testing.T) {
runEpochTransitionConsensusTest(t, startingRandaoMixesResetState, expectedRandaoMixesResetState, func(s *state.BeaconState) error {
transition2.ProcessRandaoMixesReset(s)
ProcessRandaoMixesReset(s)
return nil
})
}
func TestSlashingsReset(t *testing.T) {
runEpochTransitionConsensusTest(t, startingSlashingsResetState, expectedSlashingsResetState, func(s *state.BeaconState) error {
transition2.ProcessSlashingsReset(s)
ProcessSlashingsReset(s)
return nil
})
}
@ -161,6 +159,6 @@ var startingInactivityScoresState []byte
func TestInactivityScores(t *testing.T) {
runEpochTransitionConsensusTest(t, startingInactivityScoresState, expectedInactivityScoresState, func(s *state.BeaconState) error {
return transition2.ProcessInactivityScores(s)
return ProcessInactivityScores(s)
})
}

View File

@ -0,0 +1,59 @@
package statechange
import (
"github.com/ledgerwatch/erigon/cl/cltypes/solid"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
)
func ProcessParticipationRecordUpdates(s *state.BeaconState) error {
s.SetPreviousEpochAttestations(s.CurrentEpochAttestations())
s.ResetCurrentEpochAttestations()
var err error
// Also mark all current attesters as previous
s.ForEachValidator(func(_ solid.Validator, idx, total int) bool {
var oldCurrentMatchingSourceAttester, oldCurrentMatchingTargetAttester, oldCurrentMatchingHeadAttester bool
var oldMinCurrentInclusionDelayAttestation *solid.PendingAttestation
if oldCurrentMatchingSourceAttester, err = s.ValidatorIsCurrentMatchingSourceAttester(idx); err != nil {
return false
}
if oldCurrentMatchingTargetAttester, err = s.ValidatorIsCurrentMatchingTargetAttester(idx); err != nil {
return false
}
if oldCurrentMatchingHeadAttester, err = s.ValidatorIsCurrentMatchingHeadAttester(idx); err != nil {
return false
}
if oldMinCurrentInclusionDelayAttestation, err = s.ValidatorMinCurrentInclusionDelayAttestation(idx); err != nil {
return false
}
// Previous sources/target/head
if err = s.SetValidatorIsPreviousMatchingSourceAttester(idx, oldCurrentMatchingSourceAttester); err != nil {
return false
}
if err = s.SetValidatorIsPreviousMatchingTargetAttester(idx, oldCurrentMatchingTargetAttester); err != nil {
return false
}
if err = s.SetValidatorIsPreviousMatchingHeadAttester(idx, oldCurrentMatchingHeadAttester); err != nil {
return false
}
if err = s.SetValidatorMinPreviousInclusionDelayAttestation(idx, oldMinCurrentInclusionDelayAttestation); err != nil {
return false
}
// Current sources/target/head
if err = s.SetValidatorIsCurrentMatchingSourceAttester(idx, false); err != nil {
return false
}
if err = s.SetValidatorIsCurrentMatchingTargetAttester(idx, false); err != nil {
return false
}
if err = s.SetValidatorIsCurrentMatchingHeadAttester(idx, false); err != nil {
return false
}
if err = s.SetValidatorMinCurrentInclusionDelayAttestation(idx, nil); err != nil {
return false
}
return true
})
return err
}

View File

@ -0,0 +1,44 @@
package statechange
import (
"github.com/ledgerwatch/erigon/cl/clparams"
"github.com/ledgerwatch/erigon/cl/cltypes"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/ledgerwatch/erigon/cl/utils"
)
// ProcessHistoricalRootsUpdate updates the historical root data structure by computing a new historical root batch when it is time to do so.
func ProcessHistoricalRootsUpdate(s *state.BeaconState) error {
nextEpoch := state.Epoch(s.BeaconState) + 1
beaconConfig := s.BeaconConfig()
blockRoots := s.BlockRoots()
stateRoots := s.StateRoots()
// Check if it's time to compute the historical root batch.
if nextEpoch%(beaconConfig.SlotsPerHistoricalRoot/beaconConfig.SlotsPerEpoch) != 0 {
return nil
}
// Compute historical root batch.
blockRootsLeaf, err := blockRoots.HashSSZ()
if err != nil {
return err
}
stateRootsLeaf, err := stateRoots.HashSSZ()
if err != nil {
return err
}
// Add the historical summary or root to the s.
if s.Version() >= clparams.CapellaVersion {
s.AddHistoricalSummary(&cltypes.HistoricalSummary{
BlockSummaryRoot: blockRootsLeaf,
StateSummaryRoot: stateRootsLeaf,
})
} else {
historicalRoot := utils.Keccak256(blockRootsLeaf[:], stateRootsLeaf[:])
s.AddHistoricalRoot(historicalRoot)
}
return nil
}

View File

@ -1,28 +1,28 @@
package transition
package statechange
import (
state2 "github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/ledgerwatch/erigon/cl/utils"
)
// ProcessInactivityScores will updates the inactivity registry of each validator.
func ProcessInactivityScores(s *state2.BeaconState) error {
if state2.Epoch(s.BeaconState) == s.BeaconConfig().GenesisEpoch {
func ProcessInactivityScores(s *state.BeaconState) error {
if state.Epoch(s.BeaconState) == s.BeaconConfig().GenesisEpoch {
return nil
}
previousEpoch := state2.PreviousEpoch(s.BeaconState)
for _, validatorIndex := range state2.EligibleValidatorsIndicies(s.BeaconState) {
previousEpoch := state.PreviousEpoch(s.BeaconState)
for _, validatorIndex := range state.EligibleValidatorsIndicies(s.BeaconState) {
// retrieve validator inactivity score index.
score, err := s.ValidatorInactivityScore(int(validatorIndex))
if err != nil {
return err
}
if state2.IsUnslashedParticipatingIndex(s.BeaconState, previousEpoch, validatorIndex, int(s.BeaconConfig().TimelyTargetFlagIndex)) {
if state.IsUnslashedParticipatingIndex(s.BeaconState, previousEpoch, validatorIndex, int(s.BeaconConfig().TimelyTargetFlagIndex)) {
score -= utils.Min64(1, score)
} else {
score += s.BeaconConfig().InactivityScoreBias
}
if !state2.InactivityLeaking(s.BeaconState) {
if !state.InactivityLeaking(s.BeaconState) {
score -= utils.Min64(s.BeaconConfig().InactivityScoreRecoveryRate, score)
}
if err := s.SetValidatorInactivityScore(int(validatorIndex), score); err != nil {

View File

@ -1,10 +1,10 @@
package transition
package statechange
import (
"sort"
"github.com/ledgerwatch/erigon/cl/cltypes/solid"
state2 "github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/ledgerwatch/erigon/cl/clparams"
)
@ -15,15 +15,15 @@ func computeActivationExitEpoch(beaconConfig *clparams.BeaconChainConfig, epoch
}
// ProcessRegistyUpdates updates every epoch the activation status of validators. Specs at: https://github.com/ethereum/consensus-specs/blob/dev/specs/phase0/beacon-chain.md#registry-updates.
func ProcessRegistryUpdates(s *state2.BeaconState) error {
func ProcessRegistryUpdates(s *state.BeaconState) error {
beaconConfig := s.BeaconConfig()
currentEpoch := state2.Epoch(s.BeaconState)
currentEpoch := state.Epoch(s.BeaconState)
// start also initializing the activation queue.
activationQueue := make([]uint64, 0)
// Process activation eligibility and ejections.
var err error
s.ForEachValidator(func(validator solid.Validator, validatorIndex, total int) bool {
if state2.IsValidatorEligibleForActivationQueue(s.BeaconState, validator) {
if state.IsValidatorEligibleForActivationQueue(s.BeaconState, validator) {
s.SetActivationEligibilityEpochForValidatorAtIndex(validatorIndex, currentEpoch+1)
}
if validator.Active(currentEpoch) && validator.EffectiveBalance() <= beaconConfig.EjectionBalance {
@ -32,7 +32,7 @@ func ProcessRegistryUpdates(s *state2.BeaconState) error {
}
}
// Insert in the activation queue in case.
if state2.IsValidatorEligibleForActivation(s.BeaconState, validator) {
if state.IsValidatorEligibleForActivation(s.BeaconState, validator) {
activationQueue = append(activationQueue, uint64(validatorIndex))
}
return true

View File

@ -1,18 +1,18 @@
package transition
package statechange
import (
"github.com/ledgerwatch/erigon/cl/clparams"
"github.com/ledgerwatch/erigon/cl/cltypes/solid"
state2 "github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
)
func processRewardsAndPenaltiesPostAltair(s *state2.BeaconState) (err error) {
func processRewardsAndPenaltiesPostAltair(s *state.BeaconState) (err error) {
beaconConfig := s.BeaconConfig()
weights := beaconConfig.ParticipationWeights()
eligibleValidators := state2.EligibleValidatorsIndicies(s.BeaconState)
eligibleValidators := state.EligibleValidatorsIndicies(s.BeaconState)
// Initialize variables
totalActiveBalance := s.GetTotalActiveBalance()
previousEpoch := state2.PreviousEpoch(s.BeaconState)
previousEpoch := state.PreviousEpoch(s.BeaconState)
// Inactivity penalties denominator.
inactivityPenaltyDenominator := beaconConfig.InactivityScoreBias * beaconConfig.GetPenaltyQuotient(s.Version())
// Make buffer for flag indexes total balances.
@ -20,7 +20,7 @@ func processRewardsAndPenaltiesPostAltair(s *state2.BeaconState) (err error) {
// Compute all total balances for each enable unslashed validator indicies with all flags on.
s.ForEachValidator(func(validator solid.Validator, validatorIndex, total int) bool {
for i := range weights {
if state2.IsUnslashedParticipatingIndex(s.BeaconState, previousEpoch, uint64(validatorIndex), i) {
if state.IsUnslashedParticipatingIndex(s.BeaconState, previousEpoch, uint64(validatorIndex), i) {
flagsTotalBalances[i] += validator.EffectiveBalance()
}
}
@ -40,20 +40,20 @@ func processRewardsAndPenaltiesPostAltair(s *state2.BeaconState) (err error) {
return
}
for flagIdx := range weights {
if state2.IsUnslashedParticipatingIndex(s.BeaconState, previousEpoch, index, flagIdx) {
if !state2.InactivityLeaking(s.BeaconState) {
if state.IsUnslashedParticipatingIndex(s.BeaconState, previousEpoch, index, flagIdx) {
if !state.InactivityLeaking(s.BeaconState) {
rewardNumerator := baseReward * rewardMultipliers[flagIdx]
if err := state2.IncreaseBalance(s.BeaconState, index, rewardNumerator/rewardDenominator); err != nil {
if err := state.IncreaseBalance(s.BeaconState, index, rewardNumerator/rewardDenominator); err != nil {
return err
}
}
} else if flagIdx != int(beaconConfig.TimelyHeadFlagIndex) {
if err := state2.DecreaseBalance(s.BeaconState, index, baseReward*weights[flagIdx]/beaconConfig.WeightDenominator); err != nil {
if err := state.DecreaseBalance(s.BeaconState, index, baseReward*weights[flagIdx]/beaconConfig.WeightDenominator); err != nil {
return err
}
}
}
if !state2.IsUnslashedParticipatingIndex(s.BeaconState, previousEpoch, index, int(beaconConfig.TimelyTargetFlagIndex)) {
if !state.IsUnslashedParticipatingIndex(s.BeaconState, previousEpoch, index, int(beaconConfig.TimelyTargetFlagIndex)) {
inactivityScore, err := s.ValidatorInactivityScore(int(index))
if err != nil {
return err
@ -63,19 +63,19 @@ func processRewardsAndPenaltiesPostAltair(s *state2.BeaconState) (err error) {
if err != nil {
return err
}
state2.DecreaseBalance(s.BeaconState, index, (effectiveBalance*inactivityScore)/inactivityPenaltyDenominator)
state.DecreaseBalance(s.BeaconState, index, (effectiveBalance*inactivityScore)/inactivityPenaltyDenominator)
}
}
return
}
// processRewardsAndPenaltiesPhase0 process rewards and penalties for phase0 state.
func processRewardsAndPenaltiesPhase0(s *state2.BeaconState) (err error) {
func processRewardsAndPenaltiesPhase0(s *state.BeaconState) (err error) {
beaconConfig := s.BeaconConfig()
if state2.Epoch(s.BeaconState) == beaconConfig.GenesisEpoch {
if state.Epoch(s.BeaconState) == beaconConfig.GenesisEpoch {
return nil
}
eligibleValidators := state2.EligibleValidatorsIndicies(s.BeaconState)
eligibleValidators := state.EligibleValidatorsIndicies(s.BeaconState)
// Initialize variables
rewardDenominator := s.GetTotalActiveBalance() / beaconConfig.EffectiveBalanceIncrement
// Make buffer for flag indexes totTargetal balances.
@ -151,47 +151,47 @@ func processRewardsAndPenaltiesPhase0(s *state2.BeaconState) (err error) {
}
// If we attested then we reward the validator.
if state2.InactivityLeaking(s.BeaconState) {
if err := state2.IncreaseBalance(s.BeaconState, index, baseReward*attested); err != nil {
if state.InactivityLeaking(s.BeaconState) {
if err := state.IncreaseBalance(s.BeaconState, index, baseReward*attested); err != nil {
return err
}
} else {
if !currentValidator.Slashed() && previousMatchingSourceAttester {
rewardNumerator := baseReward * unslashedMatchingSourceBalanceIncrements
if err := state2.IncreaseBalance(s.BeaconState, index, rewardNumerator/rewardDenominator); err != nil {
if err := state.IncreaseBalance(s.BeaconState, index, rewardNumerator/rewardDenominator); err != nil {
return err
}
}
if !currentValidator.Slashed() && previousMatchingTargetAttester {
rewardNumerator := baseReward * unslashedMatchingTargetBalanceIncrements
if err := state2.IncreaseBalance(s.BeaconState, index, rewardNumerator/rewardDenominator); err != nil {
if err := state.IncreaseBalance(s.BeaconState, index, rewardNumerator/rewardDenominator); err != nil {
return err
}
}
if !currentValidator.Slashed() && previousMatchingHeadAttester {
rewardNumerator := baseReward * unslashedMatchingHeadBalanceIncrements
if err := state2.IncreaseBalance(s.BeaconState, index, rewardNumerator/rewardDenominator); err != nil {
if err := state.IncreaseBalance(s.BeaconState, index, rewardNumerator/rewardDenominator); err != nil {
return err
}
}
}
// Process inactivity of the network as a whole finalities.
if state2.InactivityLeaking(s.BeaconState) {
if state.InactivityLeaking(s.BeaconState) {
proposerReward := baseReward / beaconConfig.ProposerRewardQuotient
// Neutralize rewards.
if state2.DecreaseBalance(s.BeaconState, index, beaconConfig.BaseRewardsPerEpoch*baseReward-proposerReward); err != nil {
if state.DecreaseBalance(s.BeaconState, index, beaconConfig.BaseRewardsPerEpoch*baseReward-proposerReward); err != nil {
return err
}
if currentValidator.Slashed() || !previousMatchingTargetAttester {
// Increase penalities linearly if network is leaking.
if state2.DecreaseBalance(s.BeaconState, index, currentValidator.EffectiveBalance()*state2.FinalityDelay(s.BeaconState)/beaconConfig.InactivityPenaltyQuotient); err != nil {
if state.DecreaseBalance(s.BeaconState, index, currentValidator.EffectiveBalance()*state.FinalityDelay(s.BeaconState)/beaconConfig.InactivityPenaltyQuotient); err != nil {
return err
}
}
}
// For each missed duty we penalize the validator.
if state2.DecreaseBalance(s.BeaconState, index, baseReward*missed); err != nil {
if state.DecreaseBalance(s.BeaconState, index, baseReward*missed); err != nil {
return err
}
@ -217,11 +217,11 @@ func processRewardsAndPenaltiesPhase0(s *state2.BeaconState) (err error) {
}
// Compute proposer reward.
proposerReward := (baseReward / beaconConfig.ProposerRewardQuotient)
if err = state2.IncreaseBalance(s.BeaconState, attestation.ProposerIndex(), proposerReward); err != nil {
if err = state.IncreaseBalance(s.BeaconState, attestation.ProposerIndex(), proposerReward); err != nil {
return false
}
maxAttesterReward := baseReward - proposerReward
if err = state2.IncreaseBalance(s.BeaconState, uint64(index), maxAttesterReward/attestation.InclusionDelay()); err != nil {
if err = state.IncreaseBalance(s.BeaconState, uint64(index), maxAttesterReward/attestation.InclusionDelay()); err != nil {
return false
}
return true
@ -233,8 +233,8 @@ func processRewardsAndPenaltiesPhase0(s *state2.BeaconState) (err error) {
}
// ProcessRewardsAndPenalties applies rewards/penalties accumulated during previous epoch.
func ProcessRewardsAndPenalties(s *state2.BeaconState) error {
if state2.Epoch(s.BeaconState) == s.BeaconConfig().GenesisEpoch {
func ProcessRewardsAndPenalties(s *state.BeaconState) error {
if state.Epoch(s.BeaconState) == s.BeaconConfig().GenesisEpoch {
return nil
}
if s.Version() == clparams.Phase0Version {

View File

@ -0,0 +1,93 @@
package statechange
import (
"github.com/ledgerwatch/erigon/cl/clparams"
"github.com/ledgerwatch/erigon/cl/cltypes/solid"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
)
func processSlashings(s *state.BeaconState, slashingMultiplier uint64) error {
// Get the current epoch
epoch := state.Epoch(s.BeaconState)
// Get the total active balance
totalBalance := s.GetTotalActiveBalance()
// Calculate the total slashing amount
// by summing all slashings and multiplying by the provided multiplier
slashing := state.GetTotalSlashingAmount(s.BeaconState) * slashingMultiplier
// Adjust the total slashing amount to be no greater than the total active balance
if totalBalance < slashing {
slashing = totalBalance
}
beaconConfig := s.BeaconConfig()
// Apply penalties to validators who have been slashed and reached the withdrawable epoch
var err error
s.ForEachValidator(func(validator solid.Validator, i, total int) bool {
if !validator.Slashed() || epoch+beaconConfig.EpochsPerSlashingsVector/2 != validator.WithdrawableEpoch() {
return true
}
// Get the effective balance increment
increment := beaconConfig.EffectiveBalanceIncrement
// Calculate the penalty numerator by multiplying the validator's effective balance by the total slashing amount
penaltyNumerator := validator.EffectiveBalance() / increment * slashing
// Calculate the penalty by dividing the penalty numerator by the total balance and multiplying by the increment
penalty := penaltyNumerator / totalBalance * increment
// Decrease the validator's balance by the calculated penalty
if err = state.DecreaseBalance(s.BeaconState, uint64(i), penalty); err != nil {
return false
}
return true
})
if err != nil {
return err
}
return nil
}
func processSlashings2(s *state.BeaconState, slashingMultiplier uint64) error {
// Get the current epoch
epoch := state.Epoch(s.BeaconState)
// Get the total active balance
totalBalance := s.GetTotalActiveBalance()
// Calculate the total slashing amount
// by summing all slashings and multiplying by the provided multiplier
slashing := state.GetTotalSlashingAmount(s.BeaconState) * slashingMultiplier
// Adjust the total slashing amount to be no greater than the total active balance
if totalBalance < slashing {
slashing = totalBalance
}
beaconConfig := s.BeaconConfig()
// Apply penalties to validators who have been slashed and reached the withdrawable epoch
var err error
s.ForEachValidator(func(validator solid.Validator, i, total int) bool {
if !validator.Slashed() || epoch+beaconConfig.EpochsPerSlashingsVector/2 != validator.WithdrawableEpoch() {
return true
}
// Get the effective balance increment
increment := beaconConfig.EffectiveBalanceIncrement
// Calculate the penalty numerator by multiplying the validator's effective balance by the total slashing amount
penaltyNumerator := validator.EffectiveBalance() / increment * slashing
// Calculate the penalty by dividing the penalty numerator by the total balance and multiplying by the increment
penalty := penaltyNumerator / totalBalance * increment
// Decrease the validator's balance by the calculated penalty
if err = state.DecreaseBalance(s.BeaconState, uint64(i), penalty); err != nil {
return false
}
return true
})
if err != nil {
return err
}
return nil
}
func ProcessSlashings(state *state.BeaconState) error {
// Depending on the version of the state, use different multipliers
switch state.Version() {
case clparams.Phase0Version:
return processSlashings(state, state.BeaconConfig().ProportionalSlashingMultiplier)
case clparams.AltairVersion:
return processSlashings(state, state.BeaconConfig().ProportionalSlashingMultiplierAltair)
default:
return processSlashings(state, state.BeaconConfig().ProportionalSlashingMultiplierBellatrix)
}
}

View File

@ -1,12 +1,12 @@
package transition
package statechange
import (
state2 "github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
)
// ProcessSyncCommitteeUpdate implements processing for the sync committee update. unfortunately there is no easy way to test it.
func ProcessSyncCommitteeUpdate(s *state2.BeaconState) error {
if (state2.Epoch(s.BeaconState)+1)%s.BeaconConfig().EpochsPerSyncCommitteePeriod != 0 {
func ProcessSyncCommitteeUpdate(s *state.BeaconState) error {
if (state.Epoch(s.BeaconState)+1)%s.BeaconConfig().EpochsPerSyncCommitteePeriod != 0 {
return nil
}
// Set new current sync committee.

View File

@ -1,14 +1,13 @@
package transition_test
package statechange_test
import (
"encoding/binary"
"github.com/ledgerwatch/erigon/cl/transition/impl/eth2/statechange"
"testing"
"github.com/ledgerwatch/erigon/cl/clparams"
"github.com/ledgerwatch/erigon/cl/cltypes/solid"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/ledgerwatch/erigon/cl/phase1/core/transition"
"github.com/ledgerwatch/erigon/cl/clparams"
"github.com/ledgerwatch/erigon/common"
"github.com/stretchr/testify/require"
)
@ -18,7 +17,7 @@ func TestProcessSyncCommittee(t *testing.T) {
var pk [48]byte
copy(pk[:], pkBytes)
validatorNum := 10_000
state := state.New(&clparams.MainnetBeaconConfig)
s := state.New(&clparams.MainnetBeaconConfig)
currentCommittee := &solid.SyncCommittee{}
nextCommittee := &solid.SyncCommittee{}
for i := 0; i < validatorNum; i++ {
@ -28,13 +27,13 @@ func TestProcessSyncCommittee(t *testing.T) {
v.SetExitEpoch(clparams.MainnetBeaconConfig.FarFutureEpoch)
v.SetPublicKey(pk)
v.SetEffectiveBalance(2000000000)
state.AddValidator(v, 2000000000)
s.AddValidator(v, 2000000000)
}
state.SetCurrentSyncCommittee(currentCommittee)
state.SetNextSyncCommittee(nextCommittee)
prevNextSyncCommittee := state.NextSyncCommittee()
state.SetSlot(8160)
require.NoError(t, transition.ProcessSyncCommitteeUpdate(state))
require.Equal(t, state.CurrentSyncCommittee(), prevNextSyncCommittee)
require.NotEqual(t, state.NextSyncCommittee(), prevNextSyncCommittee)
s.SetCurrentSyncCommittee(currentCommittee)
s.SetNextSyncCommittee(nextCommittee)
prevNextSyncCommittee := s.NextSyncCommittee()
s.SetSlot(8160)
require.NoError(t, statechange.ProcessSyncCommitteeUpdate(s))
require.Equal(t, s.CurrentSyncCommittee(), prevNextSyncCommittee)
require.NotEqual(t, s.NextSyncCommittee(), prevNextSyncCommittee)
}

View File

@ -1,4 +1,4 @@
package transition
package statechange
import (
state2 "github.com/ledgerwatch/erigon/cl/phase1/core/state"

View File

@ -1,12 +1,10 @@
package transition
package eth2
import (
"encoding/binary"
"reflect"
libcommon "github.com/ledgerwatch/erigon-lib/common"
"github.com/ledgerwatch/erigon/cl/cltypes"
"github.com/ledgerwatch/erigon/cl/cltypes/solid"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/ledgerwatch/erigon/cl/utils"
"github.com/ledgerwatch/erigon/core/types"
)
@ -58,32 +56,39 @@ func txPeekBlobVersionedHashes(txBytes []byte) []libcommon.Hash {
return versionedHashes
}
func VerifyKzgCommitmentsAgainstTransactions(transactions *solid.TransactionsSSZ, kzgCommitments *solid.ListSSZ[*cltypes.KZGCommitment]) (bool, error) {
allVersionedHashes := []libcommon.Hash{}
transactions.ForEach(func(tx []byte, idx, total int) bool {
if tx[0] != types.BlobTxType {
return true
func computeSigningRootEpoch(epoch uint64, domain []byte) (libcommon.Hash, error) {
b := make([]byte, 32)
binary.LittleEndian.PutUint64(b, epoch)
return utils.Keccak256(b, domain), nil
}
allVersionedHashes = append(allVersionedHashes, txPeekBlobVersionedHashes(tx)...)
return true
})
commitmentVersionedHash := []libcommon.Hash{}
// transitionSlot is called each time there is a new slot to process
func transitionSlot(s *state.BeaconState) error {
slot := s.Slot()
previousStateRoot := s.PreviousStateRoot()
var err error
var versionedHash libcommon.Hash
kzgCommitments.Range(func(index int, value *cltypes.KZGCommitment, length int) bool {
versionedHash, err = kzgCommitmentToVersionedHash(value)
if previousStateRoot == (libcommon.Hash{}) {
previousStateRoot, err = s.HashSSZ()
if err != nil {
return false
return err
}
}
commitmentVersionedHash = append(commitmentVersionedHash, versionedHash)
return true
})
if err != nil {
return false, err
}
beaconConfig := s.BeaconConfig()
return reflect.DeepEqual(allVersionedHashes, commitmentVersionedHash), nil
s.SetStateRootAt(int(slot%beaconConfig.SlotsPerHistoricalRoot), previousStateRoot)
latestBlockHeader := s.LatestBlockHeader()
if latestBlockHeader.Root == [32]byte{} {
latestBlockHeader.Root = previousStateRoot
s.SetLatestBlockHeader(&latestBlockHeader)
}
blockHeader := s.LatestBlockHeader()
previousBlockRoot, err := (&blockHeader).HashSSZ()
if err != nil {
return err
}
s.SetBlockRootAt(int(slot%beaconConfig.SlotsPerHistoricalRoot), previousBlockRoot)
return nil
}

View File

@ -0,0 +1,54 @@
package eth2
import (
"fmt"
"github.com/Giulio2002/bls"
"github.com/ledgerwatch/erigon/cl/cltypes"
"github.com/ledgerwatch/erigon/cl/fork"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
)
func (I *impl) VerifyTransition(s *state.BeaconState, currentBlock *cltypes.BeaconBlock) error {
if !I.FullValidation {
return nil
}
expectedStateRoot, err := s.HashSSZ()
if err != nil {
return fmt.Errorf("unable to generate state root: %v", err)
}
if expectedStateRoot != currentBlock.StateRoot {
return fmt.Errorf("expected state root differs from received state root")
}
return nil
}
func (I *impl) VerifyBlockSignature(s *state.BeaconState, block *cltypes.SignedBeaconBlock) error {
if !I.FullValidation {
return nil
}
valid, err := verifyBlockSignature(s, block)
if err != nil {
return fmt.Errorf("error validating block signature: %v", err)
}
if !valid {
return fmt.Errorf("block not valid")
}
return nil
}
func verifyBlockSignature(s *state.BeaconState, block *cltypes.SignedBeaconBlock) (bool, error) {
proposer, err := s.ValidatorForValidatorIndex(int(block.Block.ProposerIndex))
if err != nil {
return false, err
}
domain, err := s.GetDomain(s.BeaconConfig().DomainBeaconProposer, state.Epoch(s.BeaconState))
if err != nil {
return false, err
}
sigRoot, err := fork.ComputeSigningRoot(block.Block, domain)
if err != nil {
return false, err
}
pk := proposer.PublicKey()
return bls.Verify(block.Signature[:], sigRoot[:], pk[:])
}

View File

@ -0,0 +1,94 @@
package funcmap
import (
"github.com/ledgerwatch/erigon/cl/cltypes"
"github.com/ledgerwatch/erigon/cl/cltypes/solid"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/ledgerwatch/erigon/cl/transition/machine"
"github.com/ledgerwatch/erigon/core/types"
)
var _ machine.Interface = (*Impl)(nil)
type Impl struct {
FnVerifyBlockSignature func(s *state.BeaconState, block *cltypes.SignedBeaconBlock) error
FnVerifyTransition func(s *state.BeaconState, block *cltypes.BeaconBlock) error
FnProcessSlots func(s *state.BeaconState, slot uint64) error
FnProcessBlockHeader func(s *state.BeaconState, block *cltypes.BeaconBlock) error
FnProcessWithdrawals func(s *state.BeaconState, withdrawals *solid.ListSSZ[*types.Withdrawal]) error
FnProcessExecutionPayload func(s *state.BeaconState, payload *cltypes.Eth1Block) error
FnProcessRandao func(s *state.BeaconState, randao [96]byte, proposerIndex uint64) error
FnProcessEth1Data func(state *state.BeaconState, eth1Data *cltypes.Eth1Data) error
FnProcessSyncAggregate func(s *state.BeaconState, sync *cltypes.SyncAggregate) error
FnVerifyKzgCommitmentsAgainstTransactions func(transactions *solid.TransactionsSSZ, kzgCommitments *solid.ListSSZ[*cltypes.KZGCommitment]) (bool, error)
FnProcessProposerSlashing func(s *state.BeaconState, propSlashing *cltypes.ProposerSlashing) error
FnProcessAttesterSlashing func(s *state.BeaconState, attSlashing *cltypes.AttesterSlashing) error
FnProcessAttestations func(s *state.BeaconState, attestations *solid.ListSSZ[*solid.Attestation]) error
FnProcessDeposit func(s *state.BeaconState, deposit *cltypes.Deposit) error
FnProcessVoluntaryExit func(s *state.BeaconState, signedVoluntaryExit *cltypes.SignedVoluntaryExit) error
FnProcessBlsToExecutionChange func(state *state.BeaconState, signedChange *cltypes.SignedBLSToExecutionChange) error
}
func (i Impl) VerifyBlockSignature(s *state.BeaconState, block *cltypes.SignedBeaconBlock) error {
return i.FnVerifyBlockSignature(s, block)
}
func (i Impl) VerifyTransition(s *state.BeaconState, block *cltypes.BeaconBlock) error {
return i.FnVerifyTransition(s, block)
}
func (i Impl) ProcessBlockHeader(s *state.BeaconState, block *cltypes.BeaconBlock) error {
return i.FnProcessBlockHeader(s, block)
}
func (i Impl) ProcessWithdrawals(s *state.BeaconState, withdrawals *solid.ListSSZ[*types.Withdrawal]) error {
return i.FnProcessWithdrawals(s, withdrawals)
}
func (i Impl) ProcessExecutionPayload(s *state.BeaconState, payload *cltypes.Eth1Block) error {
return i.FnProcessExecutionPayload(s, payload)
}
func (i Impl) ProcessRandao(s *state.BeaconState, randao [96]byte, proposerIndex uint64) error {
return i.FnProcessRandao(s, randao, proposerIndex)
}
func (i Impl) ProcessEth1Data(state *state.BeaconState, eth1Data *cltypes.Eth1Data) error {
return i.FnProcessEth1Data(state, eth1Data)
}
func (i Impl) ProcessSyncAggregate(s *state.BeaconState, sync *cltypes.SyncAggregate) error {
return i.FnProcessSyncAggregate(s, sync)
}
func (i Impl) VerifyKzgCommitmentsAgainstTransactions(transactions *solid.TransactionsSSZ, kzgCommitments *solid.ListSSZ[*cltypes.KZGCommitment]) (bool, error) {
return i.FnVerifyKzgCommitmentsAgainstTransactions(transactions, kzgCommitments)
}
func (i Impl) ProcessProposerSlashing(s *state.BeaconState, propSlashing *cltypes.ProposerSlashing) error {
return i.FnProcessProposerSlashing(s, propSlashing)
}
func (i Impl) ProcessAttesterSlashing(s *state.BeaconState, attSlashing *cltypes.AttesterSlashing) error {
return i.FnProcessAttesterSlashing(s, attSlashing)
}
func (i Impl) ProcessAttestations(s *state.BeaconState, attestations *solid.ListSSZ[*solid.Attestation]) error {
return i.FnProcessAttestations(s, attestations)
}
func (i Impl) ProcessDeposit(s *state.BeaconState, deposit *cltypes.Deposit) error {
return i.FnProcessDeposit(s, deposit)
}
func (i Impl) ProcessVoluntaryExit(s *state.BeaconState, signedVoluntaryExit *cltypes.SignedVoluntaryExit) error {
return i.FnProcessVoluntaryExit(s, signedVoluntaryExit)
}
func (i Impl) ProcessBlsToExecutionChange(state *state.BeaconState, signedChange *cltypes.SignedBLSToExecutionChange) error {
return i.FnProcessBlsToExecutionChange(state, signedChange)
}
func (i Impl) ProcessSlots(s *state.BeaconState, slot uint64) error {
return i.FnProcessSlots(s, slot)
}

View File

@ -0,0 +1,141 @@
package machine
import (
"errors"
"fmt"
"github.com/ledgerwatch/erigon/cl/clparams"
"github.com/ledgerwatch/erigon/cl/cltypes"
"github.com/ledgerwatch/erigon/cl/cltypes/solid"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/ledgerwatch/erigon/metrics/methelp"
)
// ProcessBlock processes a block with the block processor
func ProcessBlock(impl BlockProcessor, s *state.BeaconState, signedBlock *cltypes.SignedBeaconBlock) error {
block := signedBlock.Block
version := s.Version()
// Check the state version is correct.
if signedBlock.Version() != version {
return fmt.Errorf("processBlock: wrong state version for block at slot %d", block.Slot)
}
h := methelp.NewHistTimer("beacon_process_block")
// Process the block header.
if err := impl.ProcessBlockHeader(s, block); err != nil {
return fmt.Errorf("processBlock: failed to process block header: %v", err)
}
// Process execution payload if enabled.
if version >= clparams.BellatrixVersion && executionEnabled(s, block.Body.ExecutionPayload) {
if s.Version() >= clparams.CapellaVersion {
// Process withdrawals in the execution payload.
if err := impl.ProcessWithdrawals(s, block.Body.ExecutionPayload.Withdrawals); err != nil {
return fmt.Errorf("processBlock: failed to process withdrawals: %v", err)
}
}
// Process the execution payload.
if err := impl.ProcessExecutionPayload(s, block.Body.ExecutionPayload); err != nil {
return fmt.Errorf("processBlock: failed to process execution payload: %v", err)
}
}
// Process RANDAO reveal.
if err := impl.ProcessRandao(s, block.Body.RandaoReveal, block.ProposerIndex); err != nil {
return fmt.Errorf("processBlock: failed to process RANDAO reveal: %v", err)
}
// Process Eth1 data.
if err := impl.ProcessEth1Data(s, block.Body.Eth1Data); err != nil {
return fmt.Errorf("processBlock: failed to process Eth1 data: %v", err)
}
// Process block body operations.
if err := ProcessOperations(impl, s, block.Body); err != nil {
return fmt.Errorf("processBlock: failed to process block body operations: %v", err)
}
// Process sync aggregate in case of Altair version.
if version >= clparams.AltairVersion {
if err := impl.ProcessSyncAggregate(s, block.Body.SyncAggregate); err != nil {
return fmt.Errorf("processBlock: failed to process sync aggregate: %v", err)
}
}
if version >= clparams.DenebVersion {
verified, err := impl.VerifyKzgCommitmentsAgainstTransactions(block.Body.ExecutionPayload.Transactions, block.Body.BlobKzgCommitments)
if err != nil {
return fmt.Errorf("processBlock: failed to process blob kzg commitments: %w", err)
}
if !verified {
return fmt.Errorf("processBlock: failed to process blob kzg commitments: commitments are not equal")
}
}
h.PutSince()
return nil
}
// ProcessOperations is called by ProcessBlock and prcesses the block body operations
func ProcessOperations(impl BlockOperationProcessor, s *state.BeaconState, blockBody *cltypes.BeaconBody) error {
if blockBody.Deposits.Len() != int(maximumDeposits(s)) {
return errors.New("outstanding deposits do not match maximum deposits")
}
// Process each proposer slashing
var err error
if err := solid.RangeErr[*cltypes.ProposerSlashing](blockBody.ProposerSlashings, func(index int, slashing *cltypes.ProposerSlashing, length int) error {
if err = impl.ProcessProposerSlashing(s, slashing); err != nil {
return fmt.Errorf("ProcessProposerSlashing: %s", err)
}
return nil
}); err != nil {
return err
}
if err := solid.RangeErr[*cltypes.AttesterSlashing](blockBody.AttesterSlashings, func(index int, slashing *cltypes.AttesterSlashing, length int) error {
if err = impl.ProcessAttesterSlashing(s, slashing); err != nil {
return fmt.Errorf("ProcessAttesterSlashing: %s", err)
}
return nil
}); err != nil {
return err
}
// Process each attestations
if err := impl.ProcessAttestations(s, blockBody.Attestations); err != nil {
return fmt.Errorf("ProcessAttestation: %s", err)
}
// Process each deposit
if err := solid.RangeErr[*cltypes.Deposit](blockBody.Deposits, func(index int, deposit *cltypes.Deposit, length int) error {
if err = impl.ProcessDeposit(s, deposit); err != nil {
return fmt.Errorf("ProcessDeposit: %s", err)
}
return nil
}); err != nil {
return err
}
// Process each voluntary exit.
if err := solid.RangeErr[*cltypes.SignedVoluntaryExit](blockBody.VoluntaryExits, func(index int, exit *cltypes.SignedVoluntaryExit, length int) error {
if err = impl.ProcessVoluntaryExit(s, exit); err != nil {
return fmt.Errorf("ProcessVoluntaryExit: %s", err)
}
return nil
}); err != nil {
return err
}
if s.Version() < clparams.CapellaVersion {
return nil
}
// Process each execution change. this will only have entries after the capella fork.
if err := solid.RangeErr[*cltypes.SignedBLSToExecutionChange](blockBody.ExecutionChanges, func(index int, addressChange *cltypes.SignedBLSToExecutionChange, length int) error {
if err := impl.ProcessBlsToExecutionChange(s, addressChange); err != nil {
return fmt.Errorf("ProcessBlsToExecutionChange: %s", err)
}
return nil
}); err != nil {
return err
}
return nil
}
func maximumDeposits(s *state.BeaconState) (maxDeposits uint64) {
maxDeposits = s.Eth1Data().DepositCount - s.Eth1DepositIndex()
if maxDeposits > s.BeaconConfig().MaxDeposits {
maxDeposits = s.BeaconConfig().MaxDeposits
}
return
}

View File

@ -0,0 +1,11 @@
package machine
import (
"github.com/ledgerwatch/erigon-lib/common"
"github.com/ledgerwatch/erigon/cl/cltypes"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
)
func executionEnabled(s *state.BeaconState, payload *cltypes.Eth1Block) bool {
return (!state.IsMergeTransitionComplete(s.BeaconState) && payload.BlockHash != common.Hash{}) || state.IsMergeTransitionComplete(s.BeaconState)
}

View File

@ -0,0 +1,48 @@
// Package machine is the interface for eth2 state transition
package machine
import (
"github.com/ledgerwatch/erigon/cl/cltypes"
"github.com/ledgerwatch/erigon/cl/cltypes/solid"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/ledgerwatch/erigon/core/types"
)
type Interface interface {
BlockValidator
BlockProcessor
SlotProcessor
}
type BlockProcessor interface {
BlockHeaderProcessor
BlockOperationProcessor
}
type BlockValidator interface {
VerifyBlockSignature(s *state.BeaconState, block *cltypes.SignedBeaconBlock) error
VerifyTransition(s *state.BeaconState, block *cltypes.BeaconBlock) error
}
type SlotProcessor interface {
ProcessSlots(s *state.BeaconState, slot uint64) error
}
type BlockHeaderProcessor interface {
ProcessBlockHeader(s *state.BeaconState, block *cltypes.BeaconBlock) error
ProcessWithdrawals(s *state.BeaconState, withdrawals *solid.ListSSZ[*types.Withdrawal]) error
ProcessExecutionPayload(s *state.BeaconState, payload *cltypes.Eth1Block) error
ProcessRandao(s *state.BeaconState, randao [96]byte, proposerIndex uint64) error
ProcessEth1Data(state *state.BeaconState, eth1Data *cltypes.Eth1Data) error
ProcessSyncAggregate(s *state.BeaconState, sync *cltypes.SyncAggregate) error
VerifyKzgCommitmentsAgainstTransactions(transactions *solid.TransactionsSSZ, kzgCommitments *solid.ListSSZ[*cltypes.KZGCommitment]) (bool, error)
}
type BlockOperationProcessor interface {
ProcessProposerSlashing(s *state.BeaconState, propSlashing *cltypes.ProposerSlashing) error
ProcessAttesterSlashing(s *state.BeaconState, attSlashing *cltypes.AttesterSlashing) error
ProcessAttestations(s *state.BeaconState, attestations *solid.ListSSZ[*solid.Attestation]) error
ProcessDeposit(s *state.BeaconState, deposit *cltypes.Deposit) error
ProcessVoluntaryExit(s *state.BeaconState, signedVoluntaryExit *cltypes.SignedVoluntaryExit) error
ProcessBlsToExecutionChange(state *state.BeaconState, signedChange *cltypes.SignedBLSToExecutionChange) error
}

View File

@ -0,0 +1,32 @@
package machine
import (
"github.com/ledgerwatch/erigon/cl/cltypes"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
)
// TransitionState will call impl..ProcessSlots, then impl.VerifyBlockSignature, then ProcessBlock, then impl.VerifyTransition
func TransitionState(impl Interface, s *state.BeaconState, block *cltypes.SignedBeaconBlock) error {
currentBlock := block.Block
if err := impl.ProcessSlots(s, currentBlock.Slot); err != nil {
return err
}
if err := impl.VerifyBlockSignature(s, block); err != nil {
return err
}
// Transition block
if err := ProcessBlock(impl, s, block); err != nil {
return err
}
// perform validation
if err := impl.VerifyTransition(s, currentBlock); err != nil {
return err
}
// if validation is successful, transition
s.SetPreviousStateRoot(currentBlock.StateRoot)
return nil
}

View File

@ -1,6 +1,7 @@
package spectest
import (
"github.com/ledgerwatch/erigon/cl/transition/machine"
"io/fs"
"os"
"strings"
@ -16,6 +17,8 @@ type TestCase struct {
HandlerName string
SuiteName string
CaseName string
Machine machine.Interface
}
func (t *TestCase) Version() clparams.StateVersion {

View File

@ -1,6 +1,7 @@
package spectest
import (
"github.com/ledgerwatch/erigon/cl/transition/machine"
"io/fs"
"path/filepath"
"testing"
@ -9,7 +10,7 @@ import (
"github.com/stretchr/testify/require"
)
func RunCases(t *testing.T, app Appendix, root fs.FS) {
func RunCases(t *testing.T, app Appendix, machineImpl machine.Interface, root fs.FS) {
cases, err := ReadTestCases(root)
require.Nil(t, err, "reading cases")
// prepare for gore.....
@ -59,6 +60,7 @@ func RunCases(t *testing.T, app Appendix, root fs.FS) {
value.SuiteName,
value.CaseName,
))
value.Machine = machineImpl
require.NoError(t, err)
err = handler.Run(t, subfs, value)
require.NoError(t, err)