Compare commits

...

1 Commits

Author SHA1 Message Date
terence tsao
4e9f83558e gloas: implement modified process withdrawals 2026-01-28 10:01:17 -08:00
18 changed files with 846 additions and 5 deletions

View File

@@ -4,8 +4,10 @@ go_library(
name = "go_default_library",
srcs = [
"bid.go",
"builder.go",
"pending_payment.go",
"proposer_slashing.go",
"withdrawal.go",
],
importpath = "github.com/OffchainLabs/prysm/v7/beacon-chain/core/gloas",
visibility = ["//visibility:public"],
@@ -21,6 +23,7 @@ go_library(
"//crypto/bls:go_default_library",
"//crypto/bls/common:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//runtime/version:go_default_library",
"//time/slots:go_default_library",
"@com_github_pkg_errors//:go_default_library",
],

View File

@@ -0,0 +1,17 @@
package gloas
import (
"github.com/OffchainLabs/prysm/v7/config/params"
)
// IsBuilderWithdrawalCredential returns true when the builder withdrawal prefix is set.
// Spec v1.6.1 (pseudocode):
// def is_builder_withdrawal_credential(withdrawal_credentials: Bytes32) -> bool:
//
// return withdrawal_credentials[:1] == BUILDER_WITHDRAWAL_PREFIX
func IsBuilderWithdrawalCredential(withdrawalCredentials []byte) bool {
if len(withdrawalCredentials) == 0 {
return false
}
return withdrawalCredentials[0] == params.BeaconConfig().BuilderWithdrawalPrefixByte
}

View File

@@ -0,0 +1,117 @@
package gloas
import (
"fmt"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/helpers"
"github.com/OffchainLabs/prysm/v7/beacon-chain/state"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/pkg/errors"
)
// ProcessWithdrawals applies withdrawals to the state for Gloas.
//
// Spec v1.7.0-alpha.1 (pseudocode):
//
// def process_withdrawals(
//
// state: BeaconState,
// # [Modified in Gloas:EIP7732]
// # Removed `payload`
//
// ) -> None:
//
// # [New in Gloas:EIP7732]
// # Return early if the parent block is empty
// if not is_parent_block_full(state):
// return
//
// # Get expected withdrawals
// expected = get_expected_withdrawals(state)
//
// # Apply expected withdrawals
// apply_withdrawals(state, expected.withdrawals)
//
// # Update withdrawals fields in the state
// update_next_withdrawal_index(state, expected.withdrawals)
// # [New in Gloas:EIP7732]
// update_payload_expected_withdrawals(state, expected.withdrawals)
// # [New in Gloas:EIP7732]
// update_builder_pending_withdrawals(state, expected.processed_builder_withdrawals_count)
// update_pending_partial_withdrawals(state, expected.processed_partial_withdrawals_count)
// # [New in Gloas:EIP7732]
// update_next_withdrawal_builder_index(state, expected.processed_builders_sweep_count)
// update_next_withdrawal_validator_index(state, expected.withdrawals)
func ProcessWithdrawals(st state.BeaconState) error {
full, err := st.IsParentBlockFull()
if err != nil {
return errors.Wrap(err, "could not get parent block full status")
}
if !full {
return nil
}
expectedWithdrawals, processedBuilderWithdrawalsCount, processedPartialWithdrawalsCount, nextWithdrawalBuilderIndex, err := st.ExpectedWithdrawalsGloas()
if err != nil {
return errors.Wrap(err, "could not get expected withdrawals")
}
for _, withdrawal := range expectedWithdrawals {
if withdrawal.ValidatorIndex.IsBuilderIndex() {
builderIndex := withdrawal.ValidatorIndex.ToBuilderIndex()
err := st.DecreaseBuilderBalance(builderIndex, withdrawal.Amount)
if err != nil {
return errors.Wrap(err, "could not decrease builder balance")
}
} else {
err := helpers.DecreaseBalance(st, withdrawal.ValidatorIndex, withdrawal.Amount)
if err != nil {
return errors.Wrap(err, "could not decrease balance")
}
}
}
err = st.SetPayloadExpectedWithdrawals(expectedWithdrawals)
if err != nil {
return errors.Wrap(err, "could not set payload expected withdrawals")
}
err = st.DequeueBuilderPendingWithdrawals(processedBuilderWithdrawalsCount)
if err != nil {
return fmt.Errorf("unable to dequeue builder pending withdrawals from state: %w", err)
}
err = st.SetNextWithdrawalBuilderIndex(nextWithdrawalBuilderIndex)
if err != nil {
return errors.Wrap(err, "could not set next withdrawal builder index")
}
if err := st.DequeuePendingPartialWithdrawals(processedPartialWithdrawalsCount); err != nil {
return fmt.Errorf("unable to dequeue partial withdrawals from state: %w", err)
}
if len(expectedWithdrawals) > 0 {
if err := st.SetNextWithdrawalIndex(expectedWithdrawals[len(expectedWithdrawals)-1].Index + 1); err != nil {
return errors.Wrap(err, "could not set next withdrawal index")
}
}
var nextValidatorIndex primitives.ValidatorIndex
if uint64(len(expectedWithdrawals)) < params.BeaconConfig().MaxWithdrawalsPerPayload {
nextValidatorIndex, err = st.NextWithdrawalValidatorIndex()
if err != nil {
return errors.Wrap(err, "could not get next withdrawal validator index")
}
nextValidatorIndex += primitives.ValidatorIndex(params.BeaconConfig().MaxValidatorsPerWithdrawalsSweep)
nextValidatorIndex = nextValidatorIndex % primitives.ValidatorIndex(st.NumValidators())
} else {
nextValidatorIndex = expectedWithdrawals[len(expectedWithdrawals)-1].ValidatorIndex + 1
if nextValidatorIndex == primitives.ValidatorIndex(st.NumValidators()) {
nextValidatorIndex = 0
}
}
if err := st.SetNextWithdrawalValidatorIndex(nextValidatorIndex); err != nil {
return errors.Wrap(err, "could not set next withdrawal validator index")
}
return nil
}

View File

@@ -3,6 +3,7 @@ package state
import (
"github.com/OffchainLabs/prysm/v7/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
enginev1 "github.com/OffchainLabs/prysm/v7/proto/engine/v1"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
)
@@ -13,6 +14,11 @@ type writeOnlyGloasFields interface {
RotateBuilderPendingPayments() error
AppendBuilderPendingWithdrawals([]*ethpb.BuilderPendingWithdrawal) error
UpdateExecutionPayloadAvailabilityAtIndex(idx uint64, val byte) error
SetPayloadExpectedWithdrawals(withdrawals []*enginev1.Withdrawal) error
DequeueBuilderPendingWithdrawals(num uint64) error
SetNextWithdrawalBuilderIndex(idx primitives.BuilderIndex) error
DecreaseBuilderBalance(builderIndex primitives.BuilderIndex, amount uint64) error
}
type readOnlyGloasFields interface {
@@ -21,4 +27,7 @@ type readOnlyGloasFields interface {
CanBuilderCoverBid(primitives.BuilderIndex, primitives.Gwei) (bool, error)
LatestBlockHash() ([32]byte, error)
BuilderPendingPayments() ([]*ethpb.BuilderPendingPayment, error)
IsParentBlockFull() (bool, error)
ExpectedWithdrawalsGloas() ([]*enginev1.Withdrawal, uint64, uint64, primitives.BuilderIndex, error)
}

View File

@@ -15,6 +15,7 @@ go_library(
"getters_eth1.go",
"getters_exit.go",
"getters_gloas.go",
"getters_gloas_withdrawals.go",
"getters_misc.go",
"getters_participation.go",
"getters_payload_header.go",

View File

@@ -1,6 +1,7 @@
package state_native
import (
"bytes"
"fmt"
fieldparams "github.com/OffchainLabs/prysm/v7/config/fieldparams"
@@ -10,6 +11,22 @@ import (
"github.com/OffchainLabs/prysm/v7/runtime/version"
)
// IsParentBlockFull returns true when the latest bid was fulfilled with a payload.
func (b *BeaconState) IsParentBlockFull() (bool, error) {
if b.version < version.Gloas {
return false, errNotSupported("IsParentBlockFull", b.version)
}
b.lock.RLock()
defer b.lock.RUnlock()
if b.latestExecutionPayloadBid == nil {
return false, nil
}
return bytes.Equal(b.latestExecutionPayloadBid.BlockHash, b.latestBlockHash), nil
}
// LatestBlockHash returns the hash of the latest execution block.
func (b *BeaconState) LatestBlockHash() ([32]byte, error) {
if b.version < version.Gloas {
@@ -147,3 +164,56 @@ func (b *BeaconState) BuilderPendingPayments() ([]*ethpb.BuilderPendingPayment,
return b.builderPendingPaymentsVal(), nil
}
// BuilderPendingWithdrawals returns the builder pending withdrawals queue.
func (b *BeaconState) BuilderPendingWithdrawals() ([]*ethpb.BuilderPendingWithdrawal, error) {
if b.version < version.Gloas {
return nil, errNotSupported("BuilderPendingWithdrawals", b.version)
}
b.lock.RLock()
defer b.lock.RUnlock()
return b.builderPendingWithdrawalsVal(), nil
}
// BuildersCount returns the number of builders in the registry.
func (b *BeaconState) BuildersCount() int {
if b.version < version.Gloas {
return 0
}
b.lock.RLock()
defer b.lock.RUnlock()
return len(b.builders)
}
// Builders returns the builders registry.
func (b *BeaconState) Builders() ([]*ethpb.Builder, error) {
if b.version < version.Gloas {
return nil, errNotSupported("Builders", b.version)
}
b.lock.RLock()
defer b.lock.RUnlock()
builders := make([]*ethpb.Builder, len(b.builders))
for i := range b.builders {
builders[i] = ethpb.CopyBuilder(b.builders[i])
}
return builders, nil
}
// NextWithdrawalBuilderIndex returns the next builder index for the withdrawals sweep.
func (b *BeaconState) NextWithdrawalBuilderIndex() (primitives.BuilderIndex, error) {
if b.version < version.Gloas {
return 0, errNotSupported("NextWithdrawalBuilderIndex", b.version)
}
b.lock.RLock()
defer b.lock.RUnlock()
return b.nextWithdrawalBuilderIndex, nil
}

View File

@@ -44,6 +44,56 @@ func TestLatestBlockHash(t *testing.T) {
})
}
func TestIsParentBlockFull(t *testing.T) {
t.Run("returns error before gloas", func(t *testing.T) {
st, _ := util.DeterministicGenesisState(t, 1)
_, err := st.IsParentBlockFull()
require.ErrorContains(t, "is not supported", err)
})
t.Run("returns false when bid is unset", func(t *testing.T) {
st, err := state_native.InitializeFromProtoGloas(&ethpb.BeaconStateGloas{
LatestBlockHash: bytes.Repeat([]byte{0xAB}, 32),
})
require.NoError(t, err)
full, err := st.IsParentBlockFull()
require.NoError(t, err)
require.Equal(t, false, full)
})
t.Run("returns true when bid block hash matches latest block hash", func(t *testing.T) {
hashBytes := bytes.Repeat([]byte{0xAB}, 32)
st, err := state_native.InitializeFromProtoGloas(&ethpb.BeaconStateGloas{
LatestBlockHash: hashBytes,
LatestExecutionPayloadBid: &ethpb.ExecutionPayloadBid{
BlockHash: hashBytes,
},
})
require.NoError(t, err)
full, err := st.IsParentBlockFull()
require.NoError(t, err)
require.Equal(t, true, full)
})
t.Run("returns false when bid block hash does not match latest block hash", func(t *testing.T) {
latest := bytes.Repeat([]byte{0xAB}, 32)
bid := bytes.Repeat([]byte{0xCD}, 32)
st, err := state_native.InitializeFromProtoGloas(&ethpb.BeaconStateGloas{
LatestBlockHash: latest,
LatestExecutionPayloadBid: &ethpb.ExecutionPayloadBid{
BlockHash: bid,
},
})
require.NoError(t, err)
full, err := st.IsParentBlockFull()
require.NoError(t, err)
require.Equal(t, false, full)
})
}
func TestBuilderPubkey(t *testing.T) {
t.Run("returns error before gloas", func(t *testing.T) {
stIface, _ := util.DeterministicGenesisState(t, 1)

View File

@@ -0,0 +1,164 @@
package state_native
import (
"fmt"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/encoding/bytesutil"
enginev1 "github.com/OffchainLabs/prysm/v7/proto/engine/v1"
"github.com/OffchainLabs/prysm/v7/runtime/version"
"github.com/OffchainLabs/prysm/v7/time/slots"
)
// ExpectedWithdrawalsGloas returns the withdrawals that a proposer will need to pack in the next block
// applied to the current state. It is also used by validators to check that the execution payload carried
// the right number of withdrawals.
//
// Spec v1.7.0-alpha.1:
//
// def get_expected_withdrawals(state: BeaconState) -> ExpectedWithdrawals:
// withdrawal_index = state.next_withdrawal_index
// withdrawals: List[Withdrawal] = []
//
// # [New in Gloas:EIP7732]
// # Get builder withdrawals
// builder_withdrawals, withdrawal_index, processed_builder_withdrawals_count = (
// get_builder_withdrawals(state, withdrawal_index, withdrawals)
// )
// withdrawals.extend(builder_withdrawals)
//
// # Get partial withdrawals
// partial_withdrawals, withdrawal_index, processed_partial_withdrawals_count = (
// get_pending_partial_withdrawals(state, withdrawal_index, withdrawals)
// )
// withdrawals.extend(partial_withdrawals)
//
// # [New in Gloas:EIP7732]
// # Get builders sweep withdrawals
// builders_sweep_withdrawals, withdrawal_index, processed_builders_sweep_count = (
// get_builders_sweep_withdrawals(state, withdrawal_index, withdrawals)
// )
// withdrawals.extend(builders_sweep_withdrawals)
//
// # Get validators sweep withdrawals
// validators_sweep_withdrawals, withdrawal_index, processed_validators_sweep_count = (
// get_validators_sweep_withdrawals(state, withdrawal_index, withdrawals)
// )
// withdrawals.extend(validators_sweep_withdrawals)
//
// return ExpectedWithdrawals(
// withdrawals,
// # [New in Gloas:EIP7732]
// processed_builder_withdrawals_count,
// processed_partial_withdrawals_count,
// # [New in Gloas:EIP7732]
// processed_builders_sweep_count,
// processed_validators_sweep_count,
// )
func (b *BeaconState) ExpectedWithdrawalsGloas() ([]*enginev1.Withdrawal, uint64, uint64, primitives.BuilderIndex, error) {
if b.version < version.Gloas {
return nil, 0, 0, 0, errNotSupported("ExpectedWithdrawalsGloas", b.version)
}
b.lock.RLock()
defer b.lock.RUnlock()
cfg := params.BeaconConfig()
withdrawals := make([]*enginev1.Withdrawal, 0, cfg.MaxWithdrawalsPerPayload)
withdrawalIndex := b.nextWithdrawalIndex
withdrawalIndex, processedBuilderWithdrawalsCount, err := b.appendBuilderWithdrawalsGloas(withdrawalIndex, &withdrawals)
if err != nil {
return nil, 0, 0, 0, err
}
withdrawalIndex, processedPartialWithdrawalsCount, err := b.appendPendingPartialWithdrawals(withdrawalIndex, &withdrawals)
if err != nil {
return nil, 0, 0, 0, err
}
withdrawalIndex, processedBuildersSweepCount, err := b.appendBuildersSweepWithdrawalsGloas(withdrawalIndex, &withdrawals)
if err != nil {
return nil, 0, 0, 0, err
}
nextBuilderIndex := b.nextWithdrawalBuilderIndex + primitives.BuilderIndex(processedBuildersSweepCount)
err = b.appendValidatorsSweepWithdrawals(withdrawalIndex, &withdrawals)
if err != nil {
return nil, 0, 0, 0, err
}
return withdrawals, processedBuilderWithdrawalsCount, processedPartialWithdrawalsCount, nextBuilderIndex, nil
}
func (b *BeaconState) appendBuilderWithdrawalsGloas(withdrawalIndex uint64, withdrawals *[]*enginev1.Withdrawal) (uint64, uint64, error) {
cfg := params.BeaconConfig()
withdrawalsLimit := cfg.MaxWithdrawalsPerPayload - 1
ws := *withdrawals
var processedCount uint64
for _, w := range b.builderPendingWithdrawals {
if uint64(len(ws)) >= withdrawalsLimit {
break
}
ws = append(ws, &enginev1.Withdrawal{
Index: withdrawalIndex,
ValidatorIndex: primitives.ValidatorIndex(
uint64(primitives.BuilderIndex(w.BuilderIndex)) | cfg.BuilderIndexFlag,
),
Address: bytesutil.SafeCopyBytes(w.FeeRecipient),
Amount: uint64(w.Amount),
})
withdrawalIndex++
processedCount++
}
*withdrawals = ws
return withdrawalIndex, processedCount, nil
}
func (b *BeaconState) appendBuildersSweepWithdrawalsGloas(withdrawalIndex uint64, withdrawals *[]*enginev1.Withdrawal) (uint64, uint64, error) {
cfg := params.BeaconConfig()
withdrawalsLimit := cfg.MaxWithdrawalsPerPayload - 1
priorWithdrawalCount := uint64(len(*withdrawals))
if priorWithdrawalCount >= withdrawalsLimit || len(b.builders) == 0 {
return withdrawalIndex, 0, nil
}
ws := *withdrawals
epoch := slots.ToEpoch(b.slot)
buildersLimit := len(b.builders)
if maxBuilders := int(cfg.MaxBuildersPerWithdrawalsSweep); buildersLimit > maxBuilders {
buildersLimit = maxBuilders
}
builderIndex := b.nextWithdrawalBuilderIndex
if uint64(builderIndex) >= uint64(len(b.builders)) {
return withdrawalIndex, 0, fmt.Errorf("next withdrawal builder index %d out of range", builderIndex)
}
var processedCount uint64
for i := 0; i < buildersLimit; i++ {
if uint64(len(ws)) >= withdrawalsLimit {
break
}
builder := b.builders[builderIndex]
if builder != nil && builder.WithdrawableEpoch <= epoch && builder.Balance > 0 {
ws = append(ws, &enginev1.Withdrawal{
Index: withdrawalIndex,
ValidatorIndex: primitives.ValidatorIndex(uint64(builderIndex) | cfg.BuilderIndexFlag),
Address: bytesutil.SafeCopyBytes(builder.ExecutionAddress),
Amount: uint64(builder.Balance),
})
withdrawalIndex++
}
builderIndex = primitives.BuilderIndex((uint64(builderIndex) + 1) % uint64(len(b.builders)))
processedCount++
}
*withdrawals = ws
return withdrawalIndex, processedCount, nil
}

View File

@@ -133,11 +133,22 @@ func (b *BeaconState) appendPendingPartialWithdrawals(withdrawalIndex uint64, wi
return withdrawalIndex, 0, nil
}
cfg := params.BeaconConfig()
withdrawalsLimit := min(
uint64(len(*withdrawals))+cfg.MaxPendingPartialsPerWithdrawalsSweep,
cfg.MaxWithdrawalsPerPayload-1,
)
if uint64(len(*withdrawals)) > withdrawalsLimit {
return withdrawalIndex, 0, fmt.Errorf("prior withdrawals length %d exceeds limit %d", len(*withdrawals), withdrawalsLimit)
}
ws := *withdrawals
epoch := slots.ToEpoch(b.slot)
var processedPartialWithdrawalsCount uint64
for _, w := range b.pendingPartialWithdrawals {
if w.WithdrawableEpoch > epoch || len(ws) >= int(params.BeaconConfig().MaxPendingPartialsPerWithdrawalsSweep) {
isWithdrawable := w.WithdrawableEpoch <= epoch
hasReachedLimit := uint64(len(ws)) >= withdrawalsLimit
if !isWithdrawable || hasReachedLimit {
break
}
@@ -149,7 +160,7 @@ func (b *BeaconState) appendPendingPartialWithdrawals(withdrawalIndex uint64, wi
if err != nil {
return withdrawalIndex, 0, fmt.Errorf("could not retrieve balance at index %d: %w", w.Index, err)
}
hasSufficientEffectiveBalance := v.EffectiveBalance() >= params.BeaconConfig().MinActivationBalance
hasSufficientEffectiveBalance := v.EffectiveBalance() >= cfg.MinActivationBalance
var totalWithdrawn uint64
for _, wi := range ws {
if wi.ValidatorIndex == w.Index {
@@ -160,9 +171,9 @@ func (b *BeaconState) appendPendingPartialWithdrawals(withdrawalIndex uint64, wi
if err != nil {
return withdrawalIndex, 0, errors.Wrapf(err, "failed to subtract balance %d with total withdrawn %d", vBal, totalWithdrawn)
}
hasExcessBalance := balance > params.BeaconConfig().MinActivationBalance
if v.ExitEpoch() == params.BeaconConfig().FarFutureEpoch && hasSufficientEffectiveBalance && hasExcessBalance {
amount := min(balance-params.BeaconConfig().MinActivationBalance, w.Amount)
hasExcessBalance := balance > cfg.MinActivationBalance
if v.ExitEpoch() == cfg.FarFutureEpoch && hasSufficientEffectiveBalance && hasExcessBalance {
amount := min(balance-cfg.MinActivationBalance, w.Amount)
ws = append(ws, &enginev1.Withdrawal{
Index: withdrawalIndex,
ValidatorIndex: w.Index,

View File

@@ -1,6 +1,7 @@
package state_native
import (
"errors"
"fmt"
"github.com/OffchainLabs/prysm/v7/beacon-chain/state/state-native/types"
@@ -8,10 +9,32 @@ import (
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
enginev1 "github.com/OffchainLabs/prysm/v7/proto/engine/v1"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v7/runtime/version"
)
// SetPayloadExpectedWithdrawals stores the expected withdrawals for the next payload.
func (b *BeaconState) SetPayloadExpectedWithdrawals(withdrawals []*enginev1.Withdrawal) error {
if b.version < version.Gloas {
return errNotSupported("SetPayloadExpectedWithdrawals", b.version)
}
if withdrawals == nil {
return errors.New("cannot set nil payload expected withdrawals")
}
b.lock.Lock()
defer b.lock.Unlock()
b.sharedFieldReferences[types.PayloadExpectedWithdrawals].MinusRef()
b.sharedFieldReferences[types.PayloadExpectedWithdrawals] = stateutil.NewRef(1)
b.payloadExpectedWithdrawals = withdrawals
b.markFieldAsDirty(types.PayloadExpectedWithdrawals)
return nil
}
// RotateBuilderPendingPayments rotates the queue by dropping slots per epoch payments from the
// front and appending slots per epoch empty payments to the end.
// This implements: state.builder_pending_payments = state.builder_pending_payments[SLOTS_PER_EPOCH:] + [BuilderPendingPayment() for _ in range(SLOTS_PER_EPOCH)]
@@ -69,6 +92,38 @@ func (b *BeaconState) AppendBuilderPendingWithdrawals(withdrawals []*ethpb.Build
return nil
}
// DequeueBuilderPendingWithdrawals removes processed builder withdrawals from the front of the queue.
func (b *BeaconState) DequeueBuilderPendingWithdrawals(n uint64) error {
if b.version < version.Gloas {
return errNotSupported("DequeueBuilderPendingWithdrawals", b.version)
}
if n > uint64(len(b.builderPendingWithdrawals)) {
return errors.New("cannot dequeue more builder withdrawals than are in the queue")
}
if n == 0 {
return nil
}
b.lock.Lock()
defer b.lock.Unlock()
if b.sharedFieldReferences[types.BuilderPendingWithdrawals].Refs() > 1 {
withdrawals := make([]*ethpb.BuilderPendingWithdrawal, len(b.builderPendingWithdrawals))
copy(withdrawals, b.builderPendingWithdrawals)
b.builderPendingWithdrawals = withdrawals
b.sharedFieldReferences[types.BuilderPendingWithdrawals].MinusRef()
b.sharedFieldReferences[types.BuilderPendingWithdrawals] = stateutil.NewRef(1)
}
b.builderPendingWithdrawals = b.builderPendingWithdrawals[n:]
b.markFieldAsDirty(types.BuilderPendingWithdrawals)
b.rebuildTrie[types.BuilderPendingWithdrawals] = true
return nil
}
// SetExecutionPayloadBid sets the latest execution payload bid in the state.
func (b *BeaconState) SetExecutionPayloadBid(h interfaces.ROExecutionPayloadBid) error {
if b.version < version.Gloas {
@@ -161,3 +216,65 @@ func (b *BeaconState) UpdateExecutionPayloadAvailabilityAtIndex(idx uint64, val
b.markFieldAsDirty(types.ExecutionPayloadAvailability)
return nil
}
// SetNextWithdrawalBuilderIndex sets the next builder index for the withdrawals sweep.
func (b *BeaconState) SetNextWithdrawalBuilderIndex(index primitives.BuilderIndex) error {
if b.version < version.Gloas {
return errNotSupported("SetNextWithdrawalBuilderIndex", b.version)
}
b.lock.Lock()
defer b.lock.Unlock()
b.nextWithdrawalBuilderIndex = index
b.markFieldAsDirty(types.NextWithdrawalBuilderIndex)
return nil
}
// DecreaseBuilderBalance decreases the builder's balance by amount (saturating at 0).
func (b *BeaconState) DecreaseBuilderBalance(builderIndex primitives.BuilderIndex, amount uint64) error {
if b.version < version.Gloas {
return errNotSupported("DecreaseBuilderBalance", b.version)
}
if amount == 0 {
return nil
}
b.lock.Lock()
defer b.lock.Unlock()
idx := uint64(builderIndex)
if idx >= uint64(len(b.builders)) {
return fmt.Errorf("builder index %d out of range (len=%d)", builderIndex, len(b.builders))
}
// Copy-on-write for shared builders registry.
if b.sharedFieldReferences[types.Builders].Refs() > 1 {
builders := make([]*ethpb.Builder, len(b.builders))
copy(builders, b.builders)
b.builders = builders
b.sharedFieldReferences[types.Builders].MinusRef()
b.sharedFieldReferences[types.Builders] = stateutil.NewRef(1)
// Ensure we don't mutate a shared builder pointer.
if b.builders[idx] != nil {
b.builders[idx] = ethpb.CopyBuilder(b.builders[idx])
}
}
builder := b.builders[idx]
if builder == nil {
return fmt.Errorf("builder at index %d is nil", builderIndex)
}
bal := uint64(builder.Balance)
if amount >= bal {
builder.Balance = 0
} else {
builder.Balance = primitives.Gwei(bal - amount)
}
b.markFieldAsDirty(types.Builders)
b.addDirtyIndices(types.Builders, []uint64{idx})
return nil
}

View File

@@ -8,6 +8,7 @@ import (
"github.com/OffchainLabs/prysm/v7/beacon-chain/state/stateutil"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
enginev1 "github.com/OffchainLabs/prysm/v7/proto/engine/v1"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v7/runtime/version"
"github.com/OffchainLabs/prysm/v7/testing/require"
@@ -229,6 +230,163 @@ func TestRotateBuilderPendingPayments_UnsupportedVersion(t *testing.T) {
require.ErrorContains(t, "RotateBuilderPendingPayments", err)
}
func TestSetPayloadExpectedWithdrawals(t *testing.T) {
t.Run("previous fork returns expected error", func(t *testing.T) {
st := &BeaconState{version: version.Fulu}
err := st.SetPayloadExpectedWithdrawals([]*enginev1.Withdrawal{})
require.ErrorContains(t, "SetPayloadExpectedWithdrawals", err)
})
t.Run("rejects nil input", func(t *testing.T) {
st := &BeaconState{
version: version.Gloas,
dirtyFields: make(map[types.FieldIndex]bool),
sharedFieldReferences: map[types.FieldIndex]*stateutil.Reference{
types.PayloadExpectedWithdrawals: stateutil.NewRef(1),
},
}
err := st.SetPayloadExpectedWithdrawals(nil)
require.ErrorContains(t, "cannot set nil payload expected withdrawals", err)
require.Equal(t, false, st.dirtyFields[types.PayloadExpectedWithdrawals])
})
t.Run("sets and marks dirty", func(t *testing.T) {
oldRef := stateutil.NewRef(2)
st := &BeaconState{
version: version.Gloas,
dirtyFields: make(map[types.FieldIndex]bool),
sharedFieldReferences: map[types.FieldIndex]*stateutil.Reference{
types.PayloadExpectedWithdrawals: oldRef,
},
payloadExpectedWithdrawals: []*enginev1.Withdrawal{{Index: 1}, {Index: 2}},
}
withdrawals := []*enginev1.Withdrawal{{Index: 3}}
require.NoError(t, st.SetPayloadExpectedWithdrawals(withdrawals))
require.DeepEqual(t, withdrawals, st.payloadExpectedWithdrawals)
require.Equal(t, true, st.dirtyFields[types.PayloadExpectedWithdrawals])
require.Equal(t, uint(1), oldRef.Refs())
require.Equal(t, uint(1), st.sharedFieldReferences[types.PayloadExpectedWithdrawals].Refs())
require.Equal(t, false, st.sharedFieldReferences[types.PayloadExpectedWithdrawals] == oldRef)
})
}
func TestDequeueBuilderPendingWithdrawals(t *testing.T) {
t.Run("previous fork returns expected error", func(t *testing.T) {
st := &BeaconState{version: version.Fulu}
err := st.DequeueBuilderPendingWithdrawals(1)
require.ErrorContains(t, "DequeueBuilderPendingWithdrawals", err)
})
t.Run("returns error when dequeueing more than length", func(t *testing.T) {
st := &BeaconState{
version: version.Gloas,
dirtyFields: make(map[types.FieldIndex]bool),
sharedFieldReferences: map[types.FieldIndex]*stateutil.Reference{
types.BuilderPendingWithdrawals: stateutil.NewRef(1),
},
builderPendingWithdrawals: []*ethpb.BuilderPendingWithdrawal{{Amount: 1}},
}
err := st.DequeueBuilderPendingWithdrawals(2)
require.ErrorContains(t, "cannot dequeue more builder withdrawals", err)
require.Equal(t, 1, len(st.builderPendingWithdrawals))
require.Equal(t, false, st.dirtyFields[types.BuilderPendingWithdrawals])
})
t.Run("no-op on zero", func(t *testing.T) {
st := &BeaconState{
version: version.Gloas,
dirtyFields: make(map[types.FieldIndex]bool),
sharedFieldReferences: map[types.FieldIndex]*stateutil.Reference{
types.BuilderPendingWithdrawals: stateutil.NewRef(1),
},
builderPendingWithdrawals: []*ethpb.BuilderPendingWithdrawal{{Amount: 1}},
}
require.NoError(t, st.DequeueBuilderPendingWithdrawals(0))
require.Equal(t, 1, len(st.builderPendingWithdrawals))
require.Equal(t, false, st.dirtyFields[types.BuilderPendingWithdrawals])
require.Equal(t, false, st.rebuildTrie[types.BuilderPendingWithdrawals])
})
t.Run("dequeues and marks dirty", func(t *testing.T) {
st := &BeaconState{
version: version.Gloas,
dirtyFields: make(map[types.FieldIndex]bool),
sharedFieldReferences: map[types.FieldIndex]*stateutil.Reference{
types.BuilderPendingWithdrawals: stateutil.NewRef(1),
},
builderPendingWithdrawals: []*ethpb.BuilderPendingWithdrawal{
{Amount: 1},
{Amount: 2},
{Amount: 3},
},
rebuildTrie: make(map[types.FieldIndex]bool),
}
require.NoError(t, st.DequeueBuilderPendingWithdrawals(2))
require.Equal(t, 1, len(st.builderPendingWithdrawals))
require.Equal(t, primitives.Gwei(3), st.builderPendingWithdrawals[0].Amount)
require.Equal(t, true, st.dirtyFields[types.BuilderPendingWithdrawals])
require.Equal(t, true, st.rebuildTrie[types.BuilderPendingWithdrawals])
})
t.Run("copy-on-write preserves shared state", func(t *testing.T) {
sharedRef := stateutil.NewRef(2)
sharedWithdrawals := []*ethpb.BuilderPendingWithdrawal{
{Amount: 1},
{Amount: 2},
{Amount: 3},
}
st1 := &BeaconState{
version: version.Gloas,
dirtyFields: make(map[types.FieldIndex]bool),
sharedFieldReferences: map[types.FieldIndex]*stateutil.Reference{
types.BuilderPendingWithdrawals: sharedRef,
},
builderPendingWithdrawals: sharedWithdrawals,
rebuildTrie: make(map[types.FieldIndex]bool),
}
st2 := &BeaconState{
sharedFieldReferences: map[types.FieldIndex]*stateutil.Reference{
types.BuilderPendingWithdrawals: sharedRef,
},
builderPendingWithdrawals: sharedWithdrawals,
}
require.NoError(t, st1.DequeueBuilderPendingWithdrawals(2))
require.Equal(t, primitives.Gwei(3), st1.builderPendingWithdrawals[0].Amount)
require.Equal(t, 3, len(st2.builderPendingWithdrawals))
require.Equal(t, primitives.Gwei(1), st2.builderPendingWithdrawals[0].Amount)
require.Equal(t, uint(1), st1.sharedFieldReferences[types.BuilderPendingWithdrawals].Refs())
require.Equal(t, uint(1), st2.sharedFieldReferences[types.BuilderPendingWithdrawals].Refs())
})
}
func TestSetNextWithdrawalBuilderIndex(t *testing.T) {
t.Run("previous fork returns expected error", func(t *testing.T) {
st := &BeaconState{version: version.Fulu}
err := st.SetNextWithdrawalBuilderIndex(1)
require.ErrorContains(t, "SetNextWithdrawalBuilderIndex", err)
})
t.Run("sets and marks dirty", func(t *testing.T) {
st := &BeaconState{
version: version.Gloas,
dirtyFields: make(map[types.FieldIndex]bool),
}
require.NoError(t, st.SetNextWithdrawalBuilderIndex(7))
require.Equal(t, primitives.BuilderIndex(7), st.nextWithdrawalBuilderIndex)
require.Equal(t, true, st.dirtyFields[types.NextWithdrawalBuilderIndex])
})
}
func TestAppendBuilderPendingWithdrawal_CopyOnWrite(t *testing.T) {
wd := &ethpb.BuilderPendingWithdrawal{
FeeRecipient: make([]byte, 20),
@@ -318,6 +476,61 @@ func TestUpdateExecutionPayloadAvailabilityAtIndex_OutOfRange(t *testing.T) {
}
}
func TestDecreaseBuilderBalance(t *testing.T) {
t.Run("previous fork returns expected error", func(t *testing.T) {
st := &BeaconState{version: version.Fulu}
err := st.DecreaseBuilderBalance(0, 1)
require.ErrorContains(t, "DecreaseBuilderBalance", err)
})
t.Run("decreases and saturates", func(t *testing.T) {
st := &BeaconState{
version: version.Gloas,
dirtyFields: make(map[types.FieldIndex]bool),
dirtyIndices: make(map[types.FieldIndex][]uint64),
rebuildTrie: make(map[types.FieldIndex]bool),
sharedFieldReferences: map[types.FieldIndex]*stateutil.Reference{
types.Builders: stateutil.NewRef(1),
},
builders: []*ethpb.Builder{
{Balance: 10},
},
}
require.NoError(t, st.DecreaseBuilderBalance(0, 3))
require.Equal(t, uint64(7), uint64(st.builders[0].Balance))
require.Equal(t, true, st.dirtyFields[types.Builders])
require.NoError(t, st.DecreaseBuilderBalance(0, 100))
require.Equal(t, uint64(0), uint64(st.builders[0].Balance))
})
t.Run("copy-on-write preserves shared state", func(t *testing.T) {
sharedRef := stateutil.NewRef(2)
sharedBuilders := []*ethpb.Builder{
{Balance: 10},
}
st1 := &BeaconState{
version: version.Gloas,
dirtyFields: make(map[types.FieldIndex]bool),
dirtyIndices: make(map[types.FieldIndex][]uint64),
rebuildTrie: make(map[types.FieldIndex]bool),
sharedFieldReferences: map[types.FieldIndex]*stateutil.Reference{
types.Builders: sharedRef,
},
builders: sharedBuilders,
}
st2 := &BeaconState{
builders: sharedBuilders,
}
require.NoError(t, st1.DecreaseBuilderBalance(0, 3))
require.Equal(t, uint64(7), uint64(st1.builders[0].Balance))
require.Equal(t, uint64(10), uint64(st2.builders[0].Balance))
})
}
func newGloasStateWithAvailability(t *testing.T, availability []byte) *BeaconState {
t.Helper()

View File

@@ -47,6 +47,7 @@ type BeaconChainConfig struct {
HysteresisQuotient uint64 `yaml:"HYSTERESIS_QUOTIENT" spec:"true"` // HysteresisQuotient defines the hysteresis quotient for effective balance calculations.
HysteresisDownwardMultiplier uint64 `yaml:"HYSTERESIS_DOWNWARD_MULTIPLIER" spec:"true"` // HysteresisDownwardMultiplier defines the hysteresis downward multiplier for effective balance calculations.
HysteresisUpwardMultiplier uint64 `yaml:"HYSTERESIS_UPWARD_MULTIPLIER" spec:"true"` // HysteresisUpwardMultiplier defines the hysteresis upward multiplier for effective balance calculations.
BuilderIndexFlag uint64 `yaml:"BUILDER_INDEX_FLAG" spec:"true"` // BuilderIndexFlag marks a ValidatorIndex as a BuilderIndex when the bit is set.
// Gwei value constants.
MinDepositAmount uint64 `yaml:"MIN_DEPOSIT_AMOUNT" spec:"true"` // MinDepositAmount is the minimum amount of Gwei a validator can send to the deposit contract at once (lower amounts will be reverted).
@@ -126,6 +127,7 @@ type BeaconChainConfig struct {
MaxWithdrawalsPerPayload uint64 `yaml:"MAX_WITHDRAWALS_PER_PAYLOAD" spec:"true"` // MaxWithdrawalsPerPayload defines the maximum number of withdrawals in a block.
MaxBlsToExecutionChanges uint64 `yaml:"MAX_BLS_TO_EXECUTION_CHANGES" spec:"true"` // MaxBlsToExecutionChanges defines the maximum number of BLS-to-execution-change objects in a block.
MaxValidatorsPerWithdrawalsSweep uint64 `yaml:"MAX_VALIDATORS_PER_WITHDRAWALS_SWEEP" spec:"true"` // MaxValidatorsPerWithdrawalsSweep bounds the size of the sweep searching for withdrawals per slot.
MaxBuildersPerWithdrawalsSweep uint64 `yaml:"MAX_BUILDERS_PER_WITHDRAWALS_SWEEP" spec:"true"` // MaxBuildersPerWithdrawalsSweep bounds the size of the builder withdrawals sweep per slot.
// BLS domain values.
DomainBeaconProposer [4]byte `yaml:"DOMAIN_BEACON_PROPOSER" spec:"true"` // DomainBeaconProposer defines the BLS signature domain for beacon proposal verification.

View File

@@ -194,6 +194,8 @@ func ConfigToYaml(cfg *BeaconChainConfig) []byte {
fmt.Sprintf("SHARD_COMMITTEE_PERIOD: %d", cfg.ShardCommitteePeriod),
fmt.Sprintf("MIN_VALIDATOR_WITHDRAWABILITY_DELAY: %d", cfg.MinValidatorWithdrawabilityDelay),
fmt.Sprintf("MAX_VALIDATORS_PER_WITHDRAWALS_SWEEP: %d", cfg.MaxValidatorsPerWithdrawalsSweep),
fmt.Sprintf("MAX_BUILDERS_PER_WITHDRAWALS_SWEEP: %d", cfg.MaxBuildersPerWithdrawalsSweep),
fmt.Sprintf("BUILDER_INDEX_FLAG: %d", cfg.BuilderIndexFlag),
fmt.Sprintf("MAX_SEED_LOOKAHEAD: %d", cfg.MaxSeedLookahead),
fmt.Sprintf("EJECTION_BALANCE: %d", cfg.EjectionBalance),
fmt.Sprintf("MIN_PER_EPOCH_CHURN_LIMIT: %d", cfg.MinPerEpochChurnLimit),

View File

@@ -83,6 +83,7 @@ var mainnetBeaconConfig = &BeaconChainConfig{
HysteresisQuotient: 4,
HysteresisDownwardMultiplier: 1,
HysteresisUpwardMultiplier: 5,
BuilderIndexFlag: 1099511627776,
// Gwei value constants.
MinDepositAmount: 1 * 1e9,
@@ -169,6 +170,7 @@ var mainnetBeaconConfig = &BeaconChainConfig{
MaxWithdrawalsPerPayload: 16,
MaxBlsToExecutionChanges: 16,
MaxValidatorsPerWithdrawalsSweep: 16384,
MaxBuildersPerWithdrawalsSweep: 16384,
// BLS domain values.
DomainBeaconProposer: bytesutil.Uint32ToBytes4(0x00000000),

View File

@@ -49,6 +49,7 @@ func compareConfigs(t *testing.T, expected, actual *params.BeaconChainConfig) {
require.DeepEqual(t, expected.HysteresisQuotient, actual.HysteresisQuotient)
require.DeepEqual(t, expected.HysteresisDownwardMultiplier, actual.HysteresisDownwardMultiplier)
require.DeepEqual(t, expected.HysteresisUpwardMultiplier, actual.HysteresisUpwardMultiplier)
require.DeepEqual(t, expected.BuilderIndexFlag, actual.BuilderIndexFlag)
require.DeepEqual(t, expected.MinDepositAmount, actual.MinDepositAmount)
require.DeepEqual(t, expected.MaxEffectiveBalance, actual.MaxEffectiveBalance)
require.DeepEqual(t, expected.EjectionBalance, actual.EjectionBalance)
@@ -94,6 +95,7 @@ func compareConfigs(t *testing.T, expected, actual *params.BeaconChainConfig) {
require.DeepEqual(t, expected.MaxDeposits, actual.MaxDeposits)
require.DeepEqual(t, expected.MaxVoluntaryExits, actual.MaxVoluntaryExits)
require.DeepEqual(t, expected.MaxWithdrawalsPerPayload, actual.MaxWithdrawalsPerPayload)
require.DeepEqual(t, expected.MaxBuildersPerWithdrawalsSweep, actual.MaxBuildersPerWithdrawalsSweep)
require.DeepEqual(t, expected.DomainBeaconProposer, actual.DomainBeaconProposer)
require.DeepEqual(t, expected.DomainRandao, actual.DomainRandao)
require.DeepEqual(t, expected.DomainBeaconAttester, actual.DomainBeaconAttester)

View File

@@ -13,6 +13,15 @@ var _ fssz.Unmarshaler = (*BuilderIndex)(nil)
// BuilderIndex is an index into the builder registry.
type BuilderIndex uint64
// ToValidatorIndex sets the builder flag on a builder index.
//
// Spec v1.6.1 (pseudocode):
// def convert_builder_index_to_validator_index(builder_index: BuilderIndex) -> ValidatorIndex:
// return ValidatorIndex(builder_index | BUILDER_INDEX_FLAG)
func (b BuilderIndex) ToValidatorIndex() ValidatorIndex {
return ValidatorIndex(uint64(b) | BuilderIndexFlag)
}
// HashTreeRoot returns the SSZ hash tree root of the index.
func (b BuilderIndex) HashTreeRoot() ([32]byte, error) {
return fssz.HashWithDefaultHasher(b)

View File

@@ -10,9 +10,32 @@ var _ fssz.HashRoot = (ValidatorIndex)(0)
var _ fssz.Marshaler = (*ValidatorIndex)(nil)
var _ fssz.Unmarshaler = (*ValidatorIndex)(nil)
// BuilderIndexFlag marks a ValidatorIndex as a BuilderIndex when the bit is set.
//
// Spec v1.6.1: BUILDER_INDEX_FLAG.
const BuilderIndexFlag uint64 = 1 << 40
// ValidatorIndex in eth2.
type ValidatorIndex uint64
// IsBuilderIndex returns true when the BuilderIndex flag is set on the validator index.
//
// Spec v1.6.1 (pseudocode):
// def is_builder_index(validator_index: ValidatorIndex) -> bool:
// return (validator_index & BUILDER_INDEX_FLAG) != 0
func (v ValidatorIndex) IsBuilderIndex() bool {
return uint64(v)&BuilderIndexFlag != 0
}
// ToBuilderIndex strips the builder flag from a validator index.
//
// Spec v1.6.1 (pseudocode):
// def convert_validator_index_to_builder_index(validator_index: ValidatorIndex) -> BuilderIndex:
// return BuilderIndex(validator_index & ~BUILDER_INDEX_FLAG)
func (v ValidatorIndex) ToBuilderIndex() BuilderIndex {
return BuilderIndex(uint64(v) & ^BuilderIndexFlag)
}
// Div divides validator index by x.
// This method panics if dividing by zero!
func (v ValidatorIndex) Div(x uint64) ValidatorIndex {

View File

@@ -33,3 +33,32 @@ func TestValidatorIndex_Casting(t *testing.T) {
}
})
}
func TestValidatorIndex_BuilderIndexFlagConversions(t *testing.T) {
base := uint64(42)
unflagged := ValidatorIndex(base)
if unflagged.IsBuilderIndex() {
t.Fatalf("expected unflagged validator index to not be a builder index")
}
if got, want := unflagged.ToBuilderIndex(), BuilderIndex(base); got != want {
t.Fatalf("unexpected builder index: got %d want %d", got, want)
}
flagged := ValidatorIndex(base | BuilderIndexFlag)
if !flagged.IsBuilderIndex() {
t.Fatalf("expected flagged validator index to be a builder index")
}
if got, want := flagged.ToBuilderIndex(), BuilderIndex(base); got != want {
t.Fatalf("unexpected builder index: got %d want %d", got, want)
}
builder := BuilderIndex(base)
roundTrip := builder.ToValidatorIndex()
if !roundTrip.IsBuilderIndex() {
t.Fatalf("expected round-tripped validator index to be a builder index")
}
if got, want := roundTrip.ToBuilderIndex(), builder; got != want {
t.Fatalf("unexpected round-trip builder index: got %d want %d", got, want)
}
}