Compare commits

...

1 Commits

Author SHA1 Message Date
terence tsao
9e0119d51f gloas: implement modified process withdrawals 2026-01-22 20:52:08 -08:00
11 changed files with 472 additions and 0 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"],
@@ -20,6 +22,8 @@ go_library(
"//consensus-types/primitives:go_default_library",
"//crypto/bls:go_default_library",
"//crypto/bls/common:go_default_library",
"//encoding/bytesutil:go_default_library",
"//proto/engine/v1:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//time/slots:go_default_library",
"@com_github_pkg_errors//:go_default_library",

View File

@@ -0,0 +1,79 @@
package gloas
import (
"github.com/OffchainLabs/prysm/v7/beacon-chain/state"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
)
// IsBuilderIndex returns true when the BuilderIndex flag is set on a validator index.
// Spec v1.6.1 (pseudocode):
// def is_builder_index(validator_index: ValidatorIndex) -> bool:
//
// return (validator_index & BUILDER_INDEX_FLAG) != 0
func IsBuilderIndex(validatorIndex primitives.ValidatorIndex) bool {
return uint64(validatorIndex)&params.BeaconConfig().BuilderIndexFlag != 0
}
// ConvertValidatorIndexToBuilderIndex 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 ConvertValidatorIndexToBuilderIndex(validatorIndex primitives.ValidatorIndex) primitives.BuilderIndex {
return primitives.BuilderIndex(uint64(validatorIndex) & ^params.BeaconConfig().BuilderIndexFlag)
}
// ConvertBuilderIndexToValidatorIndex 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 ConvertBuilderIndexToValidatorIndex(builderIndex primitives.BuilderIndex) primitives.ValidatorIndex {
return primitives.ValidatorIndex(uint64(builderIndex) | params.BeaconConfig().BuilderIndexFlag)
}
// 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
}
// IsActiveBuilder checks if the builder is active for the given state.
// Spec v1.6.1 (pseudocode):
// def is_active_builder(state: BeaconState, builder_index: BuilderIndex) -> bool:
//
// """
// Check if the builder at ``builder_index`` is active for the given ``state``.
// """
// builder = state.builders[builder_index]
// return (
// # Placement in builder list is finalized
// builder.deposit_epoch < state.finalized_checkpoint.epoch
// # Has not initiated exit
// and builder.withdrawable_epoch == FAR_FUTURE_EPOCH
// )
func IsActiveBuilder(st state.ReadOnlyBeaconState, builderIndex primitives.BuilderIndex) bool {
builders, err := st.Builders()
if err != nil {
return false
}
if builderIndex >= primitives.BuilderIndex(len(builders)) {
return false
}
builder := builders[builderIndex]
if builder == nil {
return false
}
finalizedEpoch := st.FinalizedCheckpointEpoch()
return builder.DepositEpoch < finalizedEpoch &&
builder.WithdrawableEpoch == params.BeaconConfig().FarFutureEpoch
}

View File

@@ -0,0 +1,201 @@
package gloas
import (
"fmt"
"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/OffchainLabs/prysm/v7/encoding/bytesutil"
enginev1 "github.com/OffchainLabs/prysm/v7/proto/engine/v1"
"github.com/OffchainLabs/prysm/v7/time/slots"
)
// getBuilderWithdrawals returns builder withdrawals derived from pending builder withdrawals.
// Spec v1.6.1 (pseudocode):
// def get_builder_withdrawals(state, withdrawal_index, prior_withdrawals):
//
// withdrawals_limit = MAX_WITHDRAWALS_PER_PAYLOAD
//
// processed_count = 0
// withdrawals = []
// for withdrawal in state.builder_pending_withdrawals:
// all_withdrawals = prior_withdrawals + withdrawals
// has_reached_limit = len(all_withdrawals) == withdrawals_limit
// if has_reached_limit:
// break
//
// builder_index = withdrawal.builder_index
// withdrawals.append(
// Withdrawal(
// index=withdrawal_index,
// validator_index=convert_builder_index_to_validator_index(builder_index),
// address=withdrawal.fee_recipient,
// amount=withdrawal.amount,
// )
// )
// withdrawal_index += WithdrawalIndex(1)
// processed_count += 1
//
// return withdrawals, withdrawal_index, processed_count
func getBuilderWithdrawals(
st state.ReadOnlyBeaconState,
withdrawalIndex uint64,
) ([]*enginev1.Withdrawal, uint64, uint64, error) {
pendingWithdrawals, err := st.BuilderPendingWithdrawals()
if err != nil {
return nil, withdrawalIndex, 0, err
}
withdrawalsLimit := params.BeaconConfig().MaxWithdrawalsPerPayload
withdrawals := make([]*enginev1.Withdrawal, 0, len(pendingWithdrawals))
var processedCount uint64
for _, withdrawal := range pendingWithdrawals {
if uint64(+len(withdrawals)) == withdrawalsLimit {
break
}
withdrawals = append(withdrawals, &enginev1.Withdrawal{
Index: withdrawalIndex,
ValidatorIndex: ConvertBuilderIndexToValidatorIndex(primitives.BuilderIndex(withdrawal.BuilderIndex)),
Address: bytesutil.SafeCopyBytes(withdrawal.FeeRecipient),
Amount: uint64(withdrawal.Amount),
})
withdrawalIndex++
processedCount++
}
return withdrawals, withdrawalIndex, processedCount, nil
}
// getBuildersSweepWithdrawals returns builder withdrawals selected by a sweep over the builders registry.
// Spec v1.6.1 (pseudocode):
// def get_builders_sweep_withdrawals(state, withdrawal_index, prior_withdrawals):
//
// epoch = get_current_epoch(state)
// builders_limit = min(len(state.builders), MAX_BUILDERS_PER_WITHDRAWALS_SWEEP)
// withdrawals_limit = MAX_WITHDRAWALS_PER_PAYLOAD
//
// processed_count = 0
// withdrawals = []
// builder_index = state.next_withdrawal_builder_index
// for _ in range(builders_limit):
// all_withdrawals = prior_withdrawals + withdrawals
// has_reached_limit = len(all_withdrawals) == withdrawals_limit
// if has_reached_limit:
// break
//
// builder = state.builders[builder_index]
// if builder.withdrawable_epoch <= epoch and builder.balance > 0:
// withdrawals.append(
// Withdrawal(
// index=withdrawal_index,
// validator_index=convert_builder_index_to_validator_index(builder_index),
// address=builder.execution_address,
// amount=builder.balance,
// )
// )
// withdrawal_index += WithdrawalIndex(1)
//
// builder_index = BuilderIndex((builder_index + 1) % len(state.builders))
// processed_count += 1
//
// return withdrawals, withdrawal_index, processed_count
func getBuildersSweepWithdrawals(
st state.ReadOnlyBeaconState,
withdrawalIndex uint64,
priorWithdrawals []*enginev1.Withdrawal,
) ([]*enginev1.Withdrawal, uint64, uint64, error) {
builders, err := st.Builders()
if err != nil {
return nil, withdrawalIndex, 0, err
}
if len(builders) == 0 {
return nil, withdrawalIndex, 0, nil
}
builderIndex, err := st.NextWithdrawalBuilderIndex()
if err != nil {
return nil, withdrawalIndex, 0, err
}
if uint64(builderIndex) >= uint64(len(builders)) {
return nil, withdrawalIndex, 0, fmt.Errorf("next withdrawal builder index %d out of range", builderIndex)
}
cfg := params.BeaconConfig()
buildersLimit := len(builders)
if maxBuilders := int(cfg.MaxBuildersPerWithdrawalsSweep); buildersLimit > maxBuilders {
buildersLimit = maxBuilders
}
withdrawalsLimit := cfg.MaxWithdrawalsPerPayload
epoch := slots.ToEpoch(st.Slot())
withdrawals := make([]*enginev1.Withdrawal, 0, buildersLimit)
var processedCount uint64
for i := 0; i < buildersLimit; i++ {
if uint64(len(priorWithdrawals)+len(withdrawals)) == withdrawalsLimit {
break
}
builder := builders[builderIndex]
if builder != nil && builder.WithdrawableEpoch <= epoch && builder.Balance > 0 {
withdrawals = append(withdrawals, &enginev1.Withdrawal{
Index: withdrawalIndex,
ValidatorIndex: ConvertBuilderIndexToValidatorIndex(builderIndex),
Address: bytesutil.SafeCopyBytes(builder.ExecutionAddress),
Amount: uint64(builder.Balance),
})
withdrawalIndex++
}
builderIndex = primitives.BuilderIndex((uint64(builderIndex) + 1) % uint64(len(builders)))
processedCount++
}
return withdrawals, withdrawalIndex, processedCount, nil
}
// updatePayloadExpectedWithdrawals stores the expected withdrawals for the next payload.
// Spec v1.6.1 (pseudocode):
// def update_payload_expected_withdrawals(state, withdrawals):
//
// state.payload_expected_withdrawals = List[Withdrawal, MAX_WITHDRAWALS_PER_PAYLOAD](withdrawals)
func updatePayloadExpectedWithdrawals(st state.BeaconState, withdrawals []*enginev1.Withdrawal) error {
return st.SetPayloadExpectedWithdrawals(withdrawals)
}
// updateBuilderPendingWithdrawals removes processed builder pending withdrawals.
// Spec v1.6.1 (pseudocode):
// def update_builder_pending_withdrawals(state, processed_builder_withdrawals_count):
//
// state.builder_pending_withdrawals = state.builder_pending_withdrawals[processed_builder_withdrawals_count:]
func updateBuilderPendingWithdrawals(st state.BeaconState, processedBuilderWithdrawalsCount uint64) error {
if processedBuilderWithdrawalsCount == 0 {
return nil
}
return st.DequeueBuilderPendingWithdrawals(processedBuilderWithdrawalsCount)
}
// updateNextWithdrawalBuilderIndex advances the sweep position for builders.
// Spec v1.6.1 (pseudocode):
// def update_next_withdrawal_builder_index(state, processed_builders_sweep_count):
//
// if len(state.builders) > 0:
// next_index = state.next_withdrawal_builder_index + processed_builders_sweep_count
// next_builder_index = BuilderIndex(next_index % len(state.builders))
// state.next_withdrawal_builder_index = next_builder_index
func updateNextWithdrawalBuilderIndex(st state.BeaconState, processedBuildersSweepCount uint64) error {
buildersCount := st.BuildersCount()
if buildersCount == 0 {
return nil
}
nextIndex, err := st.NextWithdrawalBuilderIndex()
if err != nil {
return err
}
nextBuilderIndex := primitives.BuilderIndex((uint64(nextIndex) + processedBuildersSweepCount) % uint64(buildersCount))
return st.SetNextWithdrawalBuilderIndex(nextBuilderIndex)
}

View File

@@ -60,6 +60,7 @@ type ReadOnlyBeaconState interface {
ReadOnlyCheckpoint
ReadOnlyAttestations
ReadOnlyWithdrawals
ReadOnlyGloas
ReadOnlyParticipation
ReadOnlyInactivity
ReadOnlySyncCommittee
@@ -103,6 +104,7 @@ type WriteOnlyBeaconState interface {
WriteOnlyDeposits
WriteOnlyProposerLookahead
writeOnlyGloasFields
WriteOnlyGloas
SetGenesisTime(val time.Time) error
SetGenesisValidatorsRoot(val []byte) error
SetSlot(val primitives.Slot) error

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"
)
@@ -22,3 +23,27 @@ type readOnlyGloasFields interface {
LatestBlockHash() ([32]byte, error)
BuilderPendingPayments() ([]*ethpb.BuilderPendingPayment, error)
}
// Builder represents a builder registry entry.
type Builder struct {
ExecutionAddress []byte
Balance uint64
DepositEpoch primitives.Epoch
WithdrawableEpoch primitives.Epoch
}
// ReadOnlyGloas defines read access to Gloas-specific state fields.
type ReadOnlyGloas interface {
IsParentBlockFull() bool
BuildersCount() int
Builders() ([]*Builder, error)
NextWithdrawalBuilderIndex() (primitives.BuilderIndex, error)
BuilderPendingWithdrawals() ([]*ethpb.BuilderPendingWithdrawal, error)
}
// WriteOnlyGloas defines write access to Gloas-specific state fields.
type WriteOnlyGloas interface {
SetPayloadExpectedWithdrawals(withdrawals []*enginev1.Withdrawal) error
DequeueBuilderPendingWithdrawals(num uint64) error
SetNextWithdrawalBuilderIndex(idx primitives.BuilderIndex) error
}

View File

@@ -1,15 +1,34 @@
package state_native
import (
"bytes"
"fmt"
state "github.com/OffchainLabs/prysm/v7/beacon-chain/state"
fieldparams "github.com/OffchainLabs/prysm/v7/config/fieldparams"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/encoding/bytesutil"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v7/runtime/version"
)
// IsParentBlockFull returns true when the latest bid was fulfilled with a payload.
func (b *BeaconState) IsParentBlockFull() bool {
if b.version < version.Gloas {
return false
}
b.lock.RLock()
defer b.lock.RUnlock()
if b.latestExecutionPayloadBid == nil {
return false
}
return bytes.Equal(b.latestExecutionPayloadBid.BlockHash, b.latestBlockHash)
}
// LatestBlockHash returns the hash of the latest execution block.
func (b *BeaconState) LatestBlockHash() ([32]byte, error) {
if b.version < version.Gloas {
@@ -147,3 +166,68 @@ 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() ([]*state.Builder, error) {
if b.version < version.Gloas {
return nil, errNotSupported("Builders", b.version)
}
b.lock.RLock()
defer b.lock.RUnlock()
if b.builders == nil {
return nil, nil
}
builders := make([]*state.Builder, len(b.builders))
for i, builder := range b.builders {
if builder == nil {
continue
}
builders[i] = &state.Builder{
ExecutionAddress: bytesutil.SafeCopyBytes(builder.ExecutionAddress),
Balance: uint64(builder.Balance),
DepositEpoch: primitives.Epoch(builder.DepositEpoch),
WithdrawableEpoch: primitives.Epoch(builder.WithdrawableEpoch),
}
}
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

@@ -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,17 @@ 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
}

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)