Compare commits

...

4 Commits

Author SHA1 Message Date
terence tsao
21bce63341 use gloas_test package to avoid circular dependency 2026-01-22 08:31:55 -08:00
terence tsao
7f964b9ec9 James feedback part 1 2026-01-22 08:28:45 -08:00
terence tsao
db775c6da8 gloas: add new process payload attestation 2026-01-22 08:28:45 -08:00
terence
d440aafacf gloas: add modified proposer slashing processing (#16212)
This PR implements
[process_proposer_slashing](https://github.com/ethereum/consensus-specs/blob/master/specs/gloas/beacon-chain.md#modified-process_proposer_slashing)
alongside spec tests
2026-01-22 15:38:55 +00:00
31 changed files with 1063 additions and 14 deletions

View File

@@ -21,6 +21,7 @@ go_library(
importpath = "github.com/OffchainLabs/prysm/v7/beacon-chain/core/blocks",
visibility = ["//visibility:public"],
deps = [
"//beacon-chain/core/gloas:go_default_library",
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/core/signing:go_default_library",
"//beacon-chain/core/time:go_default_library",

View File

@@ -4,6 +4,7 @@ import (
"context"
"fmt"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/gloas"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/helpers"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/signing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/time"
@@ -11,6 +12,7 @@ import (
"github.com/OffchainLabs/prysm/v7/beacon-chain/state"
"github.com/OffchainLabs/prysm/v7/config/params"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v7/runtime/version"
"github.com/OffchainLabs/prysm/v7/time/slots"
"github.com/pkg/errors"
"google.golang.org/protobuf/proto"
@@ -126,7 +128,16 @@ func processProposerSlashing(
if exitInfo == nil {
return nil, errors.New("exit info is required to process proposer slashing")
}
var err error
// [New in Gloas:EIP7732]: remove the BuilderPendingPayment corresponding to the slashed proposer within 2 epoch window
if beaconState.Version() >= version.Gloas {
err = gloas.RemoveBuilderPendingPayment(beaconState, slashing.Header_1.Header)
if err != nil {
return nil, err
}
}
beaconState, err = validators.SlashValidator(ctx, beaconState, slashing.Header_1.Header.ProposerIndex, exitInfo)
if err != nil {
return nil, errors.Wrapf(err, "could not slash proposer index %d", slashing.Header_1.Header.ProposerIndex)

View File

@@ -4,20 +4,27 @@ go_library(
name = "go_default_library",
srcs = [
"bid.go",
"payload_attestation.go",
"pending_payment.go",
"proposer_slashing.go",
],
importpath = "github.com/OffchainLabs/prysm/v7/beacon-chain/core/gloas",
visibility = ["//visibility:public"],
deps = [
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/core/signing:go_default_library",
"//beacon-chain/core/time:go_default_library",
"//beacon-chain/state:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//consensus-types/primitives:go_default_library",
"//crypto/bls:go_default_library",
"//crypto/bls/common:go_default_library",
"//crypto/hash:go_default_library",
"//encoding/bytesutil:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//time/slots:go_default_library",
"@com_github_pkg_errors//:go_default_library",
@@ -28,7 +35,9 @@ go_test(
name = "go_default_test",
srcs = [
"bid_test.go",
"payload_attestation_test.go",
"pending_payment_test.go",
"proposer_slashing_test.go",
],
embed = [":go_default_library"],
deps = [
@@ -37,6 +46,7 @@ go_test(
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/state-native:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//consensus-types/primitives:go_default_library",
"//crypto/bls:go_default_library",
@@ -47,8 +57,10 @@ go_test(
"//proto/prysm/v1alpha1/validator-client:go_default_library",
"//runtime/version:go_default_library",
"//testing/require:go_default_library",
"//testing/util:go_default_library",
"//time/slots:go_default_library",
"@com_github_prysmaticlabs_fastssz//:go_default_library",
"@com_github_prysmaticlabs_go_bitfield//:go_default_library",
"@org_golang_google_protobuf//proto:go_default_library",
],
)

View File

@@ -0,0 +1,245 @@
package gloas
import (
"bytes"
"context"
"encoding/binary"
"fmt"
"slices"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/helpers"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/signing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/state"
fieldparams "github.com/OffchainLabs/prysm/v7/config/fieldparams"
"github.com/OffchainLabs/prysm/v7/config/params"
consensus_types "github.com/OffchainLabs/prysm/v7/consensus-types"
"github.com/OffchainLabs/prysm/v7/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/crypto/bls"
"github.com/OffchainLabs/prysm/v7/crypto/hash"
"github.com/OffchainLabs/prysm/v7/encoding/bytesutil"
eth "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v7/time/slots"
"github.com/pkg/errors"
)
// ProcessPayloadAttestations validates payload attestations in a block body.
// Spec v1.7.0-alpha.0 (pseudocode):
// process_payload_attestation(state: BeaconState, payload_attestation: PayloadAttestation):
//
// data = payload_attestation.data
// assert data.beacon_block_root == state.latest_block_header.parent_root
// assert data.slot + 1 == state.slot
// indexed = get_indexed_payload_attestation(state, data.slot, payload_attestation)
// assert is_valid_indexed_payload_attestation(state, indexed)
func ProcessPayloadAttestations(ctx context.Context, st state.BeaconState, body interfaces.ReadOnlyBeaconBlockBody) error {
atts, err := body.PayloadAttestations()
if err != nil {
return errors.Wrap(err, "failed to get payload attestations from block body")
}
if len(atts) == 0 {
return nil
}
header := st.LatestBlockHeader()
for i, att := range atts {
data := att.Data
if !bytes.Equal(data.BeaconBlockRoot, header.ParentRoot) {
return fmt.Errorf("payload attestation %d has wrong parent: got %x want %x", i, data.BeaconBlockRoot, header.ParentRoot)
}
if data.Slot+1 != st.Slot() {
return fmt.Errorf("payload attestation %d has wrong slot: got %d want %d", i, data.Slot+1, st.Slot())
}
indexed, err := indexedPayloadAttestation(ctx, st, att)
if err != nil {
return errors.Wrapf(err, "payload attestation %d failed to convert to indexed form", i)
}
if err := validIndexedPayloadAttestation(st, indexed); err != nil {
return errors.Wrapf(err, "payload attestation %d failed to verify indexed form", i)
}
}
return nil
}
// indexedPayloadAttestation converts a payload attestation into its indexed form.
func indexedPayloadAttestation(ctx context.Context, st state.ReadOnlyBeaconState, att *eth.PayloadAttestation) (*consensus_types.IndexedPayloadAttestation, error) {
committee, err := payloadCommittee(ctx, st, att.Data.Slot)
if err != nil {
return nil, err
}
indices := make([]primitives.ValidatorIndex, 0, len(committee))
for i, idx := range committee {
if att.AggregationBits.BitAt(uint64(i)) {
indices = append(indices, idx)
}
}
slices.Sort(indices)
return &consensus_types.IndexedPayloadAttestation{
AttestingIndices: indices,
Data: att.Data,
Signature: att.Signature,
}, nil
}
// payloadCommittee returns the payload timeliness committee for a given slot for the state.
// Spec v1.7.0-alpha.0 (pseudocode):
// get_ptc(state: BeaconState, slot: Slot) -> Vector[ValidatorIndex, PTC_SIZE]:
//
// epoch = compute_epoch_at_slot(slot)
// seed = hash(get_seed(state, epoch, DOMAIN_PTC_ATTESTER) + uint_to_bytes(slot))
// indices = []
// committees_per_slot = get_committee_count_per_slot(state, epoch)
// for i in range(committees_per_slot):
// committee = get_beacon_committee(state, slot, CommitteeIndex(i))
// indices.extend(committee)
// return compute_balance_weighted_selection(state, indices, seed, size=PTC_SIZE, shuffle_indices=False)
func payloadCommittee(ctx context.Context, st state.ReadOnlyBeaconState, slot primitives.Slot) ([]primitives.ValidatorIndex, error) {
epoch := slots.ToEpoch(slot)
seed, err := ptcSeed(st, epoch, slot)
if err != nil {
return nil, err
}
activeCount, err := helpers.ActiveValidatorCount(ctx, st, epoch)
if err != nil {
return nil, err
}
committeesPerSlot := helpers.SlotCommitteeCount(activeCount)
out := make([]primitives.ValidatorIndex, 0, committeesPerSlot)
for i := primitives.CommitteeIndex(0); i < primitives.CommitteeIndex(committeesPerSlot); i++ {
committee, err := helpers.BeaconCommitteeFromState(ctx, st, slot, i)
if err != nil {
return nil, errors.Wrapf(err, "failed to get beacon committee %d", i)
}
out = append(out, committee...)
}
return selectByBalance(st, out, seed, fieldparams.PTCSize)
}
// ptcSeed computes the seed for the payload timeliness committee.
func ptcSeed(st state.ReadOnlyBeaconState, epoch primitives.Epoch, slot primitives.Slot) ([32]byte, error) {
seed, err := helpers.Seed(st, epoch, params.BeaconConfig().DomainPTCAttester)
if err != nil {
return [32]byte{}, err
}
return hash.Hash(append(seed[:], bytesutil.Bytes8(uint64(slot))...)), nil
}
// selectByBalance selects a balance-weighted subset of input candidates.
// Spec v1.7.0-alpha.0 (pseudocode):
// compute_balance_weighted_selection(state, indices, seed, size, shuffle_indices):
// Note: shuffle_indices is false for PTC.
//
// total = len(indices); selected = []; i = 0
// while len(selected) < size:
// next = i % total
// if shuffle_indices: next = compute_shuffled_index(next, total, seed)
// if compute_balance_weighted_acceptance(state, indices[next], seed, i):
// selected.append(indices[next])
// i += 1
func selectByBalance(st state.ReadOnlyBeaconState, candidates []primitives.ValidatorIndex, seed [32]byte, count uint64) ([]primitives.ValidatorIndex, error) {
if len(candidates) == 0 {
return nil, errors.New("no candidates for balance weighted selection")
}
hashFunc := hash.CustomSHA256Hasher()
// Pre-allocate buffer for hash input: seed (32 bytes) + round counter (8 bytes).
var buf [40]byte
copy(buf[:], seed[:])
maxBalance := params.BeaconConfig().MaxEffectiveBalanceElectra
selected := make([]primitives.ValidatorIndex, 0, count)
total := uint64(len(candidates))
for i := uint64(0); uint64(len(selected)) < count; i++ {
idx := candidates[i%total]
ok, err := acceptByBalance(st, idx, buf[:], hashFunc, maxBalance, i)
if err != nil {
return nil, err
}
if ok {
selected = append(selected, idx)
}
}
return selected, nil
}
// acceptByBalance determines if a validator is accepted based on its effective balance.
// Spec v1.7.0-alpha.0 (pseudocode):
// compute_balance_weighted_acceptance(state, index, seed, i):
//
// MAX_RANDOM_VALUE = 2**16 - 1
// random_bytes = hash(seed + uint_to_bytes(i // 16))
// offset = i % 16 * 2
// random_value = bytes_to_uint64(random_bytes[offset:offset+2])
// effective_balance = state.validators[index].effective_balance
// return effective_balance * MAX_RANDOM_VALUE >= MAX_EFFECTIVE_BALANCE_ELECTRA * random_value
func acceptByBalance(st state.ReadOnlyBeaconState, idx primitives.ValidatorIndex, seedBuf []byte, hashFunc func([]byte) [32]byte, maxBalance uint64, round uint64) (bool, error) {
// Reuse the seed buffer by overwriting the last 8 bytes with the round counter.
binary.LittleEndian.PutUint64(seedBuf[len(seedBuf)-8:], round/16)
random := hashFunc(seedBuf)
offset := (round % 16) * 2
randomValue := uint64(binary.LittleEndian.Uint16(random[offset : offset+2])) // 16-bit draw per spec
val, err := st.ValidatorAtIndex(idx)
if err != nil {
return false, errors.Wrapf(err, "validator %d", idx)
}
return val.EffectiveBalance*fieldparams.MaxRandomValueElectra >= maxBalance*randomValue, nil
}
// validIndexedPayloadAttestation verifies the signature of an indexed payload attestation.
// Spec v1.7.0-alpha.0 (pseudocode):
// is_valid_indexed_payload_attestation(state: BeaconState, indexed_payload_attestation: IndexedPayloadAttestation) -> bool:
//
// indices = indexed_payload_attestation.attesting_indices
// return len(indices) > 0 and indices == sorted(indices) and
// bls.FastAggregateVerify(
// [state.validators[i].pubkey for i in indices],
// compute_signing_root(indexed_payload_attestation.data, get_domain(state, DOMAIN_PTC_ATTESTER, compute_epoch_at_slot(attestation.data.slot)),
// indexed_payload_attestation.signature,
// )
func validIndexedPayloadAttestation(st state.ReadOnlyBeaconState, att *consensus_types.IndexedPayloadAttestation) error {
indices := att.AttestingIndices
if len(indices) == 0 || !slices.IsSorted(indices) {
return errors.New("attesting indices empty or unsorted")
}
pubkeys := make([]bls.PublicKey, len(indices))
for i, idx := range indices {
val, err := st.ValidatorAtIndexReadOnly(idx)
if err != nil {
return errors.Wrapf(err, "validator %d", idx)
}
keyBytes := val.PublicKey()
key, err := bls.PublicKeyFromBytes(keyBytes[:])
if err != nil {
return errors.Wrapf(err, "pubkey %d", idx)
}
pubkeys[i] = key
}
domain, err := signing.Domain(st.Fork(), slots.ToEpoch(att.Data.Slot), params.BeaconConfig().DomainPTCAttester, st.GenesisValidatorsRoot())
if err != nil {
return err
}
root, err := signing.ComputeSigningRoot(att.Data, domain)
if err != nil {
return err
}
sig, err := bls.SignatureFromBytes(att.Signature)
if err != nil {
return err
}
if !sig.FastAggregateVerify(pubkeys, root) {
return errors.New("invalid signature")
}
return nil
}

View File

@@ -0,0 +1,280 @@
package gloas_test
import (
"bytes"
"testing"
"github.com/OffchainLabs/go-bitfield"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/gloas"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/helpers"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/signing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/state"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v7/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/crypto/bls"
"github.com/OffchainLabs/prysm/v7/crypto/bls/common"
eth "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v7/testing/require"
testutil "github.com/OffchainLabs/prysm/v7/testing/util"
"github.com/OffchainLabs/prysm/v7/time/slots"
)
func TestProcessPayloadAttestations_WrongParent(t *testing.T) {
setupTestConfig(t)
_, pk := newKey(t)
st := newTestState(t, []*eth.Validator{activeValidator(pk)}, 1)
require.NoError(t, st.SetSlot(2))
parentRoot := bytes.Repeat([]byte{0xaa}, 32)
require.NoError(t, st.SetLatestBlockHeader(&eth.BeaconBlockHeader{ParentRoot: parentRoot}))
att := &eth.PayloadAttestation{
Data: &eth.PayloadAttestationData{
BeaconBlockRoot: bytes.Repeat([]byte{0xbb}, 32),
Slot: 1,
},
AggregationBits: bitfield.NewBitvector512(),
Signature: make([]byte, 96),
}
body := buildBody(t, att)
err := gloas.ProcessPayloadAttestations(t.Context(), st, body)
require.ErrorContains(t, "wrong parent", err)
}
func TestProcessPayloadAttestations_WrongSlot(t *testing.T) {
setupTestConfig(t)
_, pk := newKey(t)
st := newTestState(t, []*eth.Validator{activeValidator(pk)}, 1)
require.NoError(t, st.SetSlot(3))
parentRoot := bytes.Repeat([]byte{0xaa}, 32)
require.NoError(t, st.SetLatestBlockHeader(&eth.BeaconBlockHeader{ParentRoot: parentRoot}))
att := &eth.PayloadAttestation{
Data: &eth.PayloadAttestationData{
BeaconBlockRoot: parentRoot,
Slot: 1,
},
AggregationBits: bitfield.NewBitvector512(),
Signature: make([]byte, 96),
}
body := buildBody(t, att)
err := gloas.ProcessPayloadAttestations(t.Context(), st, body)
require.ErrorContains(t, "wrong slot", err)
}
func TestProcessPayloadAttestations_InvalidSignature(t *testing.T) {
setupTestConfig(t)
_, pk1 := newKey(t)
sk2, pk2 := newKey(t)
vals := []*eth.Validator{activeValidator(pk1), activeValidator(pk2)}
st := newTestState(t, vals, 2)
parentRoot := bytes.Repeat([]byte{0xaa}, 32)
require.NoError(t, st.SetLatestBlockHeader(&eth.BeaconBlockHeader{ParentRoot: parentRoot}))
attData := &eth.PayloadAttestationData{
BeaconBlockRoot: parentRoot,
Slot: 1,
}
att := &eth.PayloadAttestation{
Data: attData,
AggregationBits: setBits(bitfield.NewBitvector512(), 0),
Signature: signAttestation(t, st, attData, []common.SecretKey{sk2}),
}
body := buildBody(t, att)
err := gloas.ProcessPayloadAttestations(t.Context(), st, body)
require.ErrorContains(t, "failed to verify indexed form", err)
require.ErrorContains(t, "invalid signature", err)
}
func TestProcessPayloadAttestations_HappyPath(t *testing.T) {
helpers.ClearCache()
setupTestConfig(t)
sk1, pk1 := newKey(t)
sk2, pk2 := newKey(t)
vals := []*eth.Validator{activeValidator(pk1), activeValidator(pk2)}
st := newTestState(t, vals, 2)
parentRoot := bytes.Repeat([]byte{0xaa}, 32)
require.NoError(t, st.SetLatestBlockHeader(&eth.BeaconBlockHeader{ParentRoot: parentRoot}))
attData := &eth.PayloadAttestationData{
BeaconBlockRoot: parentRoot,
Slot: 1,
}
aggBits := bitfield.NewBitvector512()
aggBits.SetBitAt(0, true)
aggBits.SetBitAt(1, true)
att := &eth.PayloadAttestation{
Data: attData,
AggregationBits: aggBits,
Signature: signAttestation(t, st, attData, []common.SecretKey{sk1, sk2}),
}
body := buildBody(t, att)
err := gloas.ProcessPayloadAttestations(t.Context(), st, body)
require.NoError(t, err)
}
func TestProcessPayloadAttestations_MultipleAttestations(t *testing.T) {
helpers.ClearCache()
setupTestConfig(t)
sk1, pk1 := newKey(t)
sk2, pk2 := newKey(t)
vals := []*eth.Validator{activeValidator(pk1), activeValidator(pk2)}
st := newTestState(t, vals, 2)
parentRoot := bytes.Repeat([]byte{0xaa}, 32)
require.NoError(t, st.SetLatestBlockHeader(&eth.BeaconBlockHeader{ParentRoot: parentRoot}))
attData1 := &eth.PayloadAttestationData{
BeaconBlockRoot: parentRoot,
Slot: 1,
}
attData2 := &eth.PayloadAttestationData{
BeaconBlockRoot: parentRoot,
Slot: 1,
}
att1 := &eth.PayloadAttestation{
Data: attData1,
AggregationBits: setBits(bitfield.NewBitvector512(), 0),
Signature: signAttestation(t, st, attData1, []common.SecretKey{sk1}),
}
att2 := &eth.PayloadAttestation{
Data: attData2,
AggregationBits: setBits(bitfield.NewBitvector512(), 1),
Signature: signAttestation(t, st, attData2, []common.SecretKey{sk2}),
}
body := buildBody(t, att1, att2)
err := gloas.ProcessPayloadAttestations(t.Context(), st, body)
require.NoError(t, err)
}
func TestProcessPayloadAttestations_IndexedVerificationError(t *testing.T) {
setupTestConfig(t)
_, pk := newKey(t)
st := newTestState(t, []*eth.Validator{activeValidator(pk)}, 1)
parentRoot := bytes.Repeat([]byte{0xaa}, 32)
require.NoError(t, st.SetLatestBlockHeader(&eth.BeaconBlockHeader{ParentRoot: parentRoot}))
attData := &eth.PayloadAttestationData{
BeaconBlockRoot: parentRoot,
Slot: 0,
}
att := &eth.PayloadAttestation{
Data: attData,
AggregationBits: setBits(bitfield.NewBitvector512(), 0),
Signature: make([]byte, 96),
}
body := buildBody(t, att)
errState := &validatorLookupErrState{
BeaconState: st,
errIndex: 0,
}
err := gloas.ProcessPayloadAttestations(t.Context(), errState, body)
require.ErrorContains(t, "failed to verify indexed form", err)
require.ErrorContains(t, "validator 0", err)
}
func newTestState(t *testing.T, vals []*eth.Validator, slot primitives.Slot) state.BeaconState {
st, err := testutil.NewBeaconState()
require.NoError(t, err)
for _, v := range vals {
require.NoError(t, st.AppendValidator(v))
require.NoError(t, st.AppendBalance(v.EffectiveBalance))
}
require.NoError(t, st.SetSlot(slot))
require.NoError(t, helpers.UpdateCommitteeCache(t.Context(), st, slots.ToEpoch(slot)))
return st
}
func setupTestConfig(t *testing.T) {
params.SetupTestConfigCleanup(t)
cfg := params.BeaconConfig().Copy()
cfg.SlotsPerEpoch = 1
cfg.MaxEffectiveBalanceElectra = cfg.MaxEffectiveBalance
params.OverrideBeaconConfig(cfg)
}
func buildBody(t *testing.T, atts ...*eth.PayloadAttestation) interfaces.ReadOnlyBeaconBlockBody {
body := &eth.BeaconBlockBodyGloas{
PayloadAttestations: atts,
RandaoReveal: make([]byte, 96),
Eth1Data: &eth.Eth1Data{},
Graffiti: make([]byte, 32),
ProposerSlashings: []*eth.ProposerSlashing{},
AttesterSlashings: []*eth.AttesterSlashingElectra{},
Attestations: []*eth.AttestationElectra{},
Deposits: []*eth.Deposit{},
VoluntaryExits: []*eth.SignedVoluntaryExit{},
SyncAggregate: &eth.SyncAggregate{},
BlsToExecutionChanges: []*eth.SignedBLSToExecutionChange{},
}
wrapped, err := blocks.NewBeaconBlockBody(body)
require.NoError(t, err)
return wrapped
}
func setBits(bits bitfield.Bitvector512, idx uint64) bitfield.Bitvector512 {
bits.SetBitAt(idx, true)
return bits
}
func activeValidator(pub []byte) *eth.Validator {
return &eth.Validator{
PublicKey: pub,
EffectiveBalance: params.BeaconConfig().MaxEffectiveBalance,
WithdrawalCredentials: make([]byte, 32),
ActivationEligibilityEpoch: 0,
ActivationEpoch: 0,
ExitEpoch: params.BeaconConfig().FarFutureEpoch,
WithdrawableEpoch: params.BeaconConfig().FarFutureEpoch,
}
}
func newKey(t *testing.T) (common.SecretKey, []byte) {
sk, err := bls.RandKey()
require.NoError(t, err)
return sk, sk.PublicKey().Marshal()
}
func signAttestation(t *testing.T, st state.ReadOnlyBeaconState, data *eth.PayloadAttestationData, sks []common.SecretKey) []byte {
domain, err := signing.Domain(st.Fork(), slots.ToEpoch(st.Slot()), params.BeaconConfig().DomainPTCAttester, st.GenesisValidatorsRoot())
require.NoError(t, err)
root, err := signing.ComputeSigningRoot(data, domain)
require.NoError(t, err)
sigs := make([]common.Signature, len(sks))
for i, sk := range sks {
sigs[i] = sk.Sign(root[:])
}
agg := bls.AggregateSignatures(sigs)
return agg.Marshal()
}
type validatorLookupErrState struct {
state.BeaconState
errIndex primitives.ValidatorIndex
}
// ValidatorAtIndexReadOnly is overridden to simulate a missing validator lookup.
func (s *validatorLookupErrState) ValidatorAtIndexReadOnly(idx primitives.ValidatorIndex) (state.ReadOnlyValidator, error) {
if idx == s.errIndex {
return nil, state.ErrNilValidatorsInState
}
return s.BeaconState.ValidatorAtIndexReadOnly(idx)
}

View File

@@ -0,0 +1,43 @@
package gloas
import (
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/time"
"github.com/OffchainLabs/prysm/v7/beacon-chain/state"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
eth "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v7/time/slots"
"github.com/pkg/errors"
)
// RemoveBuilderPendingPayment removes the pending builder payment for the proposal slot.
// Spec v1.7.0 (pseudocode):
//
// slot = header_1.slot
// proposal_epoch = compute_epoch_at_slot(slot)
// if proposal_epoch == get_current_epoch(state):
// payment_index = SLOTS_PER_EPOCH + slot % SLOTS_PER_EPOCH
// state.builder_pending_payments[payment_index] = BuilderPendingPayment()
// elif proposal_epoch == get_previous_epoch(state):
// payment_index = slot % SLOTS_PER_EPOCH
// state.builder_pending_payments[payment_index] = BuilderPendingPayment()
func RemoveBuilderPendingPayment(st state.BeaconState, header *eth.BeaconBlockHeader) error {
proposalEpoch := slots.ToEpoch(header.Slot)
currentEpoch := time.CurrentEpoch(st)
slotsPerEpoch := params.BeaconConfig().SlotsPerEpoch
var paymentIndex primitives.Slot
if proposalEpoch == currentEpoch {
paymentIndex = slotsPerEpoch + header.Slot%slotsPerEpoch
} else if proposalEpoch+1 == currentEpoch {
paymentIndex = header.Slot % slotsPerEpoch
} else {
return nil
}
if err := st.ClearBuilderPendingPayment(paymentIndex); err != nil {
return errors.Wrap(err, "could not clear builder pending payment")
}
return nil
}

View File

@@ -0,0 +1,112 @@
package gloas
import (
"bytes"
"testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/state"
state_native "github.com/OffchainLabs/prysm/v7/beacon-chain/state/state-native"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
eth "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v7/testing/require"
)
func TestRemoveBuilderPendingPayment_CurrentEpoch(t *testing.T) {
slotsPerEpoch := params.BeaconConfig().SlotsPerEpoch
stateSlot := slotsPerEpoch*2 + 1
headerSlot := slotsPerEpoch * 2
st := newGloasStateWithPayments(t, stateSlot)
paymentIndex := int(slotsPerEpoch + headerSlot%slotsPerEpoch)
setPendingPayment(t, st, paymentIndex, 123)
err := RemoveBuilderPendingPayment(st, &eth.BeaconBlockHeader{Slot: headerSlot})
require.NoError(t, err)
got := getPendingPayment(t, st, paymentIndex)
require.NotNil(t, got.Withdrawal)
require.DeepEqual(t, make([]byte, 20), got.Withdrawal.FeeRecipient)
require.Equal(t, uint64(0), uint64(got.Withdrawal.Amount))
}
func TestRemoveBuilderPendingPayment_PreviousEpoch(t *testing.T) {
slotsPerEpoch := params.BeaconConfig().SlotsPerEpoch
stateSlot := slotsPerEpoch*2 + 1
headerSlot := slotsPerEpoch + 7
st := newGloasStateWithPayments(t, stateSlot)
paymentIndex := int(headerSlot % slotsPerEpoch)
setPendingPayment(t, st, paymentIndex, 456)
err := RemoveBuilderPendingPayment(st, &eth.BeaconBlockHeader{Slot: headerSlot})
require.NoError(t, err)
got := getPendingPayment(t, st, paymentIndex)
require.NotNil(t, got.Withdrawal)
require.DeepEqual(t, make([]byte, 20), got.Withdrawal.FeeRecipient)
require.Equal(t, uint64(0), uint64(got.Withdrawal.Amount))
}
func TestRemoveBuilderPendingPayment_OlderThanTwoEpoch(t *testing.T) {
slotsPerEpoch := params.BeaconConfig().SlotsPerEpoch
stateSlot := slotsPerEpoch*4 + 1 // current epoch far ahead
headerSlot := slotsPerEpoch * 2 // two epochs behind
st := newGloasStateWithPayments(t, stateSlot)
paymentIndex := int(headerSlot % slotsPerEpoch)
original := getPendingPayment(t, st, paymentIndex)
err := RemoveBuilderPendingPayment(st, &eth.BeaconBlockHeader{Slot: headerSlot})
require.NoError(t, err)
after := getPendingPayment(t, st, paymentIndex)
require.DeepEqual(t, original.Withdrawal.FeeRecipient, after.Withdrawal.FeeRecipient)
require.Equal(t, original.Withdrawal.Amount, after.Withdrawal.Amount)
}
func newGloasStateWithPayments(t *testing.T, slot primitives.Slot) state.BeaconState {
t.Helper()
slotsPerEpoch := params.BeaconConfig().SlotsPerEpoch
paymentCount := int(slotsPerEpoch * 2)
payments := make([]*eth.BuilderPendingPayment, paymentCount)
for i := range payments {
payments[i] = &eth.BuilderPendingPayment{
Withdrawal: &eth.BuilderPendingWithdrawal{
FeeRecipient: bytes.Repeat([]byte{0x01}, 20),
Amount: 1,
},
}
}
st, err := state_native.InitializeFromProtoUnsafeGloas(&eth.BeaconStateGloas{
Slot: slot,
BuilderPendingPayments: payments,
})
require.NoError(t, err)
return st
}
func setPendingPayment(t *testing.T, st state.BeaconState, index int, amount uint64) {
t.Helper()
payment := &eth.BuilderPendingPayment{
Withdrawal: &eth.BuilderPendingWithdrawal{
FeeRecipient: bytes.Repeat([]byte{0x02}, 20),
Amount: primitives.Gwei(amount),
},
}
require.NoError(t, st.SetBuilderPendingPayment(primitives.Slot(index), payment))
}
func getPendingPayment(t *testing.T, st state.BeaconState, index int) *eth.BuilderPendingPayment {
t.Helper()
stateProto := st.ToProtoUnsafe().(*eth.BeaconStateGloas)
return stateProto.BuilderPendingPayments[index]
}

View File

@@ -197,6 +197,9 @@ func TestGetSpec(t *testing.T) {
var dbb [4]byte
copy(dbb[:], []byte{'0', '0', '0', '8'})
config.DomainBeaconBuilder = dbb
var dptc [4]byte
copy(dptc[:], []byte{'0', '0', '0', '8'})
config.DomainPTCAttester = dptc
var dam [4]byte
copy(dam[:], []byte{'1', '0', '0', '0'})
config.DomainApplicationMask = dam
@@ -212,7 +215,7 @@ func TestGetSpec(t *testing.T) {
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), &resp))
data, ok := resp.Data.(map[string]any)
require.Equal(t, true, ok)
assert.Equal(t, 180, len(data))
assert.Equal(t, 181, len(data))
for k, v := range data {
t.Run(k, func(t *testing.T) {
switch k {
@@ -414,6 +417,8 @@ func TestGetSpec(t *testing.T) {
assert.Equal(t, "0x30303036", v)
case "DOMAIN_AGGREGATE_AND_PROOF":
assert.Equal(t, "0x30303037", v)
case "DOMAIN_PTC_ATTESTER":
assert.Equal(t, "0x30303038", v)
case "DOMAIN_APPLICATION_MASK":
assert.Equal(t, "0x31303030", v)
case "DOMAIN_SYNC_COMMITTEE":

View File

@@ -9,6 +9,7 @@ import (
type writeOnlyGloasFields interface {
SetExecutionPayloadBid(h interfaces.ROExecutionPayloadBid) error
SetBuilderPendingPayment(index primitives.Slot, payment *ethpb.BuilderPendingPayment) error
ClearBuilderPendingPayment(index primitives.Slot) error
RotateBuilderPendingPayments() error
AppendBuilderPendingWithdrawals([]*ethpb.BuilderPendingWithdrawal) error
UpdateExecutionPayloadAvailabilityAtIndex(idx uint64, val byte) error

View File

@@ -27,7 +27,7 @@ func (b *BeaconState) RotateBuilderPendingPayments() error {
copy(b.builderPendingPayments[:slotsPerEpoch], b.builderPendingPayments[slotsPerEpoch:2*slotsPerEpoch])
for i := slotsPerEpoch; i < primitives.Slot(len(b.builderPendingPayments)); i++ {
b.builderPendingPayments[i] = emptyPayment()
b.builderPendingPayments[i] = emptyBuilderPendingPayment
}
b.markFieldAsDirty(types.BuilderPendingPayments)
@@ -35,6 +35,13 @@ func (b *BeaconState) RotateBuilderPendingPayments() error {
return nil
}
// emptyBuilderPendingPayment is a shared zero-value payment used to clear entries.
var emptyBuilderPendingPayment = &ethpb.BuilderPendingPayment{
Withdrawal: &ethpb.BuilderPendingWithdrawal{
FeeRecipient: make([]byte, 20),
},
}
// AppendBuilderPendingWithdrawals appends builder pending withdrawals to the beacon state.
// If the withdrawals slice is shared, it copies the slice first to preserve references.
func (b *BeaconState) AppendBuilderPendingWithdrawals(withdrawals []*ethpb.BuilderPendingWithdrawal) error {
@@ -62,17 +69,6 @@ func (b *BeaconState) AppendBuilderPendingWithdrawals(withdrawals []*ethpb.Build
return nil
}
func emptyPayment() *ethpb.BuilderPendingPayment {
return &ethpb.BuilderPendingPayment{
Weight: 0,
Withdrawal: &ethpb.BuilderPendingWithdrawal{
FeeRecipient: make([]byte, 20),
Amount: 0,
BuilderIndex: 0,
},
}
}
// SetExecutionPayloadBid sets the latest execution payload bid in the state.
func (b *BeaconState) SetExecutionPayloadBid(h interfaces.ROExecutionPayloadBid) error {
if b.version < version.Gloas {
@@ -106,6 +102,25 @@ func (b *BeaconState) SetExecutionPayloadBid(h interfaces.ROExecutionPayloadBid)
return nil
}
// ClearBuilderPendingPayment clears a builder pending payment at the specified index.
func (b *BeaconState) ClearBuilderPendingPayment(index primitives.Slot) error {
if b.version < version.Gloas {
return errNotSupported("ClearBuilderPendingPayment", b.version)
}
b.lock.Lock()
defer b.lock.Unlock()
if uint64(index) >= uint64(len(b.builderPendingPayments)) {
return fmt.Errorf("builder pending payments index %d out of range (len=%d)", index, len(b.builderPendingPayments))
}
b.builderPendingPayments[index] = emptyBuilderPendingPayment
b.markFieldAsDirty(types.BuilderPendingPayments)
return nil
}
// SetBuilderPendingPayment sets a builder pending payment at the specified index.
func (b *BeaconState) SetBuilderPendingPayment(index primitives.Slot, payment *ethpb.BuilderPendingPayment) error {
if b.version < version.Gloas {

View File

@@ -141,6 +141,46 @@ func TestSetBuilderPendingPayment(t *testing.T) {
})
}
func TestClearBuilderPendingPayment(t *testing.T) {
t.Run("previous fork returns expected error", func(t *testing.T) {
st := &BeaconState{version: version.Fulu}
err := st.ClearBuilderPendingPayment(0)
require.ErrorContains(t, "is not supported", err)
})
t.Run("clears and marks dirty", func(t *testing.T) {
st := &BeaconState{
version: version.Gloas,
dirtyFields: make(map[types.FieldIndex]bool),
builderPendingPayments: make([]*ethpb.BuilderPendingPayment, 2),
}
st.builderPendingPayments[1] = &ethpb.BuilderPendingPayment{
Weight: 2,
Withdrawal: &ethpb.BuilderPendingWithdrawal{
Amount: 99,
BuilderIndex: 1,
},
}
require.NoError(t, st.ClearBuilderPendingPayment(1))
require.Equal(t, emptyBuilderPendingPayment, st.builderPendingPayments[1])
require.Equal(t, true, st.dirtyFields[types.BuilderPendingPayments])
})
t.Run("returns error on out of range index", func(t *testing.T) {
st := &BeaconState{
version: version.Gloas,
dirtyFields: make(map[types.FieldIndex]bool),
builderPendingPayments: make([]*ethpb.BuilderPendingPayment, 1),
}
err := st.ClearBuilderPendingPayment(2)
require.ErrorContains(t, "out of range", err)
require.Equal(t, false, st.dirtyFields[types.BuilderPendingPayments])
})
}
func TestRotateBuilderPendingPayments(t *testing.T) {
totalPayments := 2 * params.BeaconConfig().SlotsPerEpoch
payments := make([]*ethpb.BuilderPendingPayment, totalPayments)

View File

@@ -0,0 +1,2 @@
### Added
- Implement modified proposer slashing for gloas

View File

@@ -0,0 +1,3 @@
### Added
- Add Gloas process payload attestation

View File

@@ -50,4 +50,7 @@ const (
// Introduced in Fulu network upgrade.
NumberOfColumns = 128 // NumberOfColumns refers to the specified number of data columns that can exist in a network.
CellsPerBlob = 64 // CellsPerBlob refers to the number of cells in a (non-extended) blob.
// Introduced in Gloas network upgrade.
PTCSize = 512 // PTCSize is the size of the payload timeliness committee.
)

View File

@@ -50,4 +50,7 @@ const (
// Introduced in Fulu network upgrade.
NumberOfColumns = 128 // NumberOfColumns refers to the specified number of data columns that can exist in a network.
CellsPerBlob = 64 // CellsPerBlob refers to the number of cells in a (non-extended) blob.
// Introduced in Gloas network upgrade.
PTCSize = 2 // PTCSize is the size of the payload timeliness committee.
)

View File

@@ -142,6 +142,7 @@ type BeaconChainConfig struct {
DomainApplicationBuilder [4]byte `yaml:"DOMAIN_APPLICATION_BUILDER" spec:"true"` // DomainApplicationBuilder defines the BLS signature domain for application builder.
DomainBLSToExecutionChange [4]byte `yaml:"DOMAIN_BLS_TO_EXECUTION_CHANGE" spec:"true"` // DomainBLSToExecutionChange defines the BLS signature domain to change withdrawal addresses to ETH1 prefix
DomainBeaconBuilder [4]byte `yaml:"DOMAIN_BEACON_BUILDER" spec:"true"` // DomainBeaconBuilder defines the BLS signature domain for beacon block builder.
DomainPTCAttester [4]byte `yaml:"DOMAIN_PTC_ATTESTER" spec:"true"` // DomainPTCAttester defines the BLS signature domain for payload transaction committee attester.
// Prysm constants.
GenesisValidatorsRoot [32]byte // GenesisValidatorsRoot is the root hash of the genesis validators.

View File

@@ -185,6 +185,7 @@ var mainnetBeaconConfig = &BeaconChainConfig{
DomainApplicationBuilder: bytesutil.Uint32ToBytes4(0x00000001),
DomainBLSToExecutionChange: bytesutil.Uint32ToBytes4(0x0A000000),
DomainBeaconBuilder: bytesutil.Uint32ToBytes4(0x0B000000),
DomainPTCAttester: bytesutil.Uint32ToBytes4(0x0C000000),
// Prysm constants.
GenesisValidatorsRoot: [32]byte{75, 54, 61, 185, 78, 40, 97, 32, 215, 110, 185, 5, 52, 15, 221, 78, 84, 191, 233, 240, 107, 243, 63, 246, 207, 90, 210, 127, 81, 27, 254, 149},

View File

@@ -2,10 +2,15 @@ load("@prysm//tools/go:def.bzl", "go_library")
go_library(
name = "go_default_library",
srcs = ["types.go"],
srcs = [
"indexed_payload_attestation.go",
"types.go",
],
importpath = "github.com/OffchainLabs/prysm/v7/consensus-types",
visibility = ["//visibility:public"],
deps = [
"//consensus-types/primitives:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//runtime/version:go_default_library",
"@com_github_pkg_errors//:go_default_library",
],

View File

@@ -0,0 +1,36 @@
package consensus_types
import (
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
eth "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
)
type IndexedPayloadAttestation struct {
AttestingIndices []primitives.ValidatorIndex
Data *eth.PayloadAttestationData
Signature []byte
}
// GetAttestingIndices returns the attesting indices or nil when the receiver is nil.
func (x *IndexedPayloadAttestation) GetAttestingIndices() []primitives.ValidatorIndex {
if x == nil {
return nil
}
return x.AttestingIndices
}
// GetData returns the attestation data or nil when the receiver is nil.
func (x *IndexedPayloadAttestation) GetData() *eth.PayloadAttestationData {
if x == nil {
return nil
}
return x.Data
}
// GetSignature returns the signature bytes or nil when the receiver is nil.
func (x *IndexedPayloadAttestation) GetSignature() []byte {
if x == nil {
return nil
}
return x.Signature
}

View File

@@ -202,6 +202,8 @@ go_test(
"fulu__ssz_static__ssz_static_test.go",
"gloas__epoch_processing__process_builder_pending_payments_test.go",
"gloas__operations__execution_payload_header_test.go",
"gloas__operations__payload_attestation_test.go",
"gloas__operations__proposer_slashing_test.go",
"gloas__sanity__slots_test.go",
"gloas__ssz_static__ssz_static_test.go",
"phase0__epoch_processing__effective_balance_updates_test.go",

View File

@@ -0,0 +1,11 @@
package mainnet
import (
"testing"
"github.com/OffchainLabs/prysm/v7/testing/spectest/shared/gloas/operations"
)
func TestMainnet_Gloas_Operations_PayloadAttestation(t *testing.T) {
operations.RunPayloadAttestationTest(t, "mainnet")
}

View File

@@ -0,0 +1,11 @@
package mainnet
import (
"testing"
"github.com/OffchainLabs/prysm/v7/testing/spectest/shared/gloas/operations"
)
func TestMainnet_Gloas_Operations_ProposerSlashing(t *testing.T) {
operations.RunProposerSlashingTest(t, "mainnet")
}

View File

@@ -208,6 +208,8 @@ go_test(
"fulu__ssz_static__ssz_static_test.go",
"gloas__epoch_processing__process_builder_pending_payments_test.go",
"gloas__operations__execution_payload_bid_test.go",
"gloas__operations__payload_attestation_test.go",
"gloas__operations__proposer_slashing_test.go",
"gloas__sanity__slots_test.go",
"gloas__ssz_static__ssz_static_test.go",
"phase0__epoch_processing__effective_balance_updates_test.go",

View File

@@ -0,0 +1,11 @@
package minimal
import (
"testing"
"github.com/OffchainLabs/prysm/v7/testing/spectest/shared/gloas/operations"
)
func TestMinimal_Gloas_Operations_PayloadAttestation(t *testing.T) {
operations.RunPayloadAttestationTest(t, "minimal")
}

View File

@@ -0,0 +1,11 @@
package minimal
import (
"testing"
"github.com/OffchainLabs/prysm/v7/testing/spectest/shared/gloas/operations"
)
func TestMinimal_Gloas_Operations_ProposerSlashing(t *testing.T) {
operations.RunProposerSlashingTest(t, "minimal")
}

View File

@@ -13,6 +13,7 @@ go_library(
"deposit_request.go",
"execution_payload.go",
"execution_payload_bid.go",
"payload_attestation.go",
"proposer_slashing.go",
"slashing.go",
"sync_aggregate.go",

View File

@@ -0,0 +1,122 @@
package operations
import (
"context"
"path"
"testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/gloas"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/helpers"
"github.com/OffchainLabs/prysm/v7/beacon-chain/state"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
eth "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v7/testing/require"
"github.com/OffchainLabs/prysm/v7/testing/spectest/utils"
"github.com/OffchainLabs/prysm/v7/testing/util"
"github.com/golang/snappy"
)
type PayloadAttestationOperation func(ctx context.Context, s state.BeaconState, att *eth.PayloadAttestation) (state.BeaconState, error)
func RunPayloadAttestationTest(t *testing.T, config string, fork string, sszToState SSZToState) {
runPayloadAttestationTest(t, config, fork, "payload_attestation", sszToState, func(ctx context.Context, s state.BeaconState, att *eth.PayloadAttestation) (state.BeaconState, error) {
// Create a mock block body with the payload attestation
body, err := createMockBlockBodyWithPayloadAttestation(att)
if err != nil {
return s, err
}
// Wrap the protobuf body in the interface
wrappedBody, err := blocks.NewBeaconBlockBody(body)
if err != nil {
return s, err
}
err = gloas.ProcessPayloadAttestations(ctx, s, wrappedBody)
return s, err
})
}
func runPayloadAttestationTest(t *testing.T, config string, fork string, objName string, sszToState SSZToState, operationFn PayloadAttestationOperation) {
require.NoError(t, utils.SetConfig(t, config))
testFolders, testsFolderPath := utils.TestFolders(t, config, fork, "operations/"+objName+"/pyspec_tests")
if len(testFolders) == 0 {
t.Fatalf("No test folders found for %s/%s/%s", config, fork, "operations/"+objName+"/pyspec_tests")
}
for _, folder := range testFolders {
t.Run(folder.Name(), func(t *testing.T) {
helpers.ClearCache()
folderPath := path.Join(testsFolderPath, folder.Name())
// Load payload attestation from payload_attestation.ssz_snappy
attestationFile, err := util.BazelFileBytes(folderPath, "payload_attestation.ssz_snappy")
require.NoError(t, err)
attestationSSZ, err := snappy.Decode(nil /* dst */, attestationFile)
require.NoError(t, err, "Failed to decompress payload attestation")
// Unmarshal payload attestation
att := &eth.PayloadAttestation{}
err = att.UnmarshalSSZ(attestationSSZ)
require.NoError(t, err, "Failed to unmarshal payload attestation")
runPayloadAttestationOperationTest(t, folderPath, att, sszToState, operationFn)
})
}
}
// runPayloadAttestationOperationTest runs a single payload attestation operation test
func runPayloadAttestationOperationTest(t *testing.T, folderPath string, att *eth.PayloadAttestation, sszToState SSZToState, operationFn PayloadAttestationOperation) {
preBeaconStateFile, err := util.BazelFileBytes(folderPath, "pre.ssz_snappy")
require.NoError(t, err)
preBeaconStateSSZ, err := snappy.Decode(nil /* dst */, preBeaconStateFile)
require.NoError(t, err, "Failed to decompress")
beaconState, err := sszToState(preBeaconStateSSZ)
require.NoError(t, err)
// Check if post state exists
postStateExists := true
postBeaconStateFile, err := util.BazelFileBytes(folderPath, "post.ssz_snappy")
if err != nil {
postStateExists = false
}
ctx := t.Context()
resultState, err := operationFn(ctx, beaconState, att)
if postStateExists {
// Test should succeed
require.NoError(t, err, "Operation should succeed")
// Compare with expected post state
postBeaconStateSSZ, err := snappy.Decode(nil /* dst */, postBeaconStateFile)
require.NoError(t, err, "Failed to decompress post state")
expectedState, err := sszToState(postBeaconStateSSZ)
require.NoError(t, err)
expectedRoot, err := expectedState.HashTreeRoot(ctx)
require.NoError(t, err)
resultRoot, err := resultState.HashTreeRoot(ctx)
require.NoError(t, err)
require.DeepEqual(t, expectedRoot, resultRoot, "Post state does not match expected")
} else {
// Test should fail (no post.ssz_snappy means the operation should error)
require.NotNil(t, err, "Operation should fail but succeeded")
}
}
// createMockBlockBodyWithPayloadAttestation creates a mock block body containing the payload attestation
func createMockBlockBodyWithPayloadAttestation(att *eth.PayloadAttestation) (*eth.BeaconBlockBodyGloas, error) {
body := &eth.BeaconBlockBodyGloas{
PayloadAttestations: []*eth.PayloadAttestation{att},
// Default values
RandaoReveal: make([]byte, 96),
Eth1Data: &eth.Eth1Data{},
Graffiti: make([]byte, 32),
ProposerSlashings: []*eth.ProposerSlashing{},
AttesterSlashings: []*eth.AttesterSlashingElectra{},
Attestations: []*eth.AttestationElectra{},
Deposits: []*eth.Deposit{},
VoluntaryExits: []*eth.SignedVoluntaryExit{},
SyncAggregate: &eth.SyncAggregate{},
BlsToExecutionChanges: []*eth.SignedBLSToExecutionChange{},
}
return body, nil
}

View File

@@ -6,6 +6,8 @@ go_library(
srcs = [
"execution_payload_bid.go",
"helpers.go",
"payload_attestation.go",
"proposer_slashing.go",
],
importpath = "github.com/OffchainLabs/prysm/v7/testing/spectest/shared/gloas/operations",
visibility = ["//visibility:public"],

View File

@@ -3,6 +3,8 @@ package operations
import (
"github.com/OffchainLabs/prysm/v7/beacon-chain/state"
state_native "github.com/OffchainLabs/prysm/v7/beacon-chain/state/state-native"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v7/consensus-types/interfaces"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
)
@@ -13,3 +15,18 @@ func sszToState(b []byte) (state.BeaconState, error) {
}
return state_native.InitializeFromProtoGloas(base)
}
func sszToBlock(b []byte) (interfaces.SignedBeaconBlock, error) {
base := &ethpb.BeaconBlockGloas{}
if err := base.UnmarshalSSZ(b); err != nil {
return nil, err
}
return blocks.NewSignedBeaconBlock(&ethpb.SignedBeaconBlockGloas{Block: base})
}
func sszToBlockBody(b []byte) (interfaces.ReadOnlyBeaconBlockBody, error) {
base := &ethpb.BeaconBlockBodyGloas{}
if err := base.UnmarshalSSZ(b); err != nil {
return nil, err
}
return blocks.NewBeaconBlockBody(base)
}

View File

@@ -0,0 +1,12 @@
package operations
import (
"testing"
"github.com/OffchainLabs/prysm/v7/runtime/version"
common "github.com/OffchainLabs/prysm/v7/testing/spectest/shared/common/operations"
)
func RunPayloadAttestationTest(t *testing.T, config string) {
common.RunPayloadAttestationTest(t, config, version.String(version.Gloas), sszToState)
}

View File

@@ -0,0 +1,28 @@
package operations
import (
"testing"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v7/consensus-types/interfaces"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v7/runtime/version"
common "github.com/OffchainLabs/prysm/v7/testing/spectest/shared/common/operations"
)
func blockWithProposerSlashing(ssz []byte) (interfaces.SignedBeaconBlock, error) {
ps := &ethpb.ProposerSlashing{}
if err := ps.UnmarshalSSZ(ssz); err != nil {
return nil, err
}
b := &ethpb.SignedBeaconBlockGloas{
Block: &ethpb.BeaconBlockGloas{
Body: &ethpb.BeaconBlockBodyGloas{ProposerSlashings: []*ethpb.ProposerSlashing{ps}},
},
}
return blocks.NewSignedBeaconBlock(b)
}
func RunProposerSlashingTest(t *testing.T, config string) {
common.RunProposerSlashingTest(t, config, version.String(version.Gloas), blockWithProposerSlashing, sszToState)
}