Compare commits

..

13 Commits

Author SHA1 Message Date
potuz
bd630da3e6 don't migrate the checkpoint 2026-01-20 12:41:46 -03:00
potuz
1f118763d6 Merge branch 'develop' into hdiff_start_database 2026-01-20 12:22:43 -03:00
potuz
8378b161ce changelog 2026-01-19 13:47:46 -03:00
potuz
e2d39c3420 reinitialize the state diff cache 2026-01-19 13:33:11 -03:00
Bastin
f34596964b Merge branch 'develop' into hdiff_start_database 2026-01-14 11:32:42 +01:00
Potuz
56d03b4d5c Return ErrStateDiffIncompatible when state-diff enabled on incompatible database
When the state-diff feature is enabled but the database was created
without state-diff support, NewKVStore now returns the store along
with ErrStateDiffIncompatible. The caller (openDB) checks for this
error, logs a warning, disables the feature, and continues using
the returned store.

This prevents migrateToColdHdiff from being called on a database
that lacks the required state-diff infrastructure.

Co-Authored-By: Claude Opus 4.5 <noreply@anthropic.com>
2026-01-12 09:11:42 -03:00
Potuz
68ff47db10 fix test 2026-01-02 14:54:58 -03:00
Potuz
403d88b136 Add e2e test for state-diff feature
Register EnableStateDiff and StateDiffExponents flags with the
beacon-chain CLI, and add an e2e test that runs with the state-diff
feature enabled.

🤖 Generated with [Claude Code](https://claude.com/claude-code)

Co-Authored-By: Claude Opus 4.5 <noreply@anthropic.com>
2025-12-31 13:32:30 -03:00
Potuz
b8a82885a7 Initialize state-diff database on new node startup
When starting a node with --enable-state-diff feature flag, the database
is now initialized appropriately based on the startup scenario:

- New database + checkpoint sync: Set offset to checkpoint state slot,
  save checkpoint state as full snapshot
- New database + genesis sync: Set offset to 0, save genesis state as
  full snapshot
- Existing database without state-diff: Log warning that feature is
  ignored due to old schema
- Existing database with state-diff: Return error (restarts will be
  handled in a future PR)

🤖 Generated with [Claude Code](https://claude.com/claude-code)

Co-Authored-By: Claude Opus 4.5 <noreply@anthropic.com>
2025-12-31 12:19:55 -03:00
Potuz
432e4dfbbf fix bazel tags 2025-12-30 16:36:43 -03:00
Potuz
a71990bc43 gazelle 2025-12-30 16:25:07 -03:00
Potuz
cb1d98b7dc Add unit tests for migrateToColdHdiff
Add tests for the state diff migration path in MigrateToCold:
- TestMigrateToColdHdiff_CanUpdateFinalizedInfo: verifies finalized info updates
- TestMigrateToColdHdiff_SkipsSlotsNotInDiffTree: verifies non-diff-tree slots are skipped
- TestMigrateToColdHdiff_NoOpWhenFinalizedSlotNotAdvanced: verifies no-op behavior

The test helper InitStateDiffCacheForTesting is in a separate file with
//go:build testing tag, ensuring it's excluded from production builds but
available for tests via gotags = ["testing"] in Bazel.

🤖 Generated with [Claude Code](https://claude.com/claude-code)

Co-Authored-By: Claude Opus 4.5 <noreply@anthropic.com>
2025-12-30 13:47:34 -03:00
Potuz
8e5c982edb Migrate to cold with state diffs
This PR adds the logic to migrate to cold when the database has the
hdiff feature. The main difference is that the boundary states have to
have the right slot therefore they need to be advanced and aren't
necessarily the post-state of a given beacon block root.
2025-12-30 08:48:12 -03:00
40 changed files with 147 additions and 392 deletions

View File

@@ -71,7 +71,6 @@ go_test(
"state_test.go",
"trailing_slot_state_cache_test.go",
"transition_fuzz_test.go",
"transition_gloas_test.go",
"transition_no_verify_sig_test.go",
"transition_test.go",
],
@@ -107,7 +106,6 @@ go_test(
"//time/slots:go_default_library",
"@com_github_google_gofuzz//:go_default_library",
"@com_github_prysmaticlabs_go_bitfield//:go_default_library",
"@com_github_stretchr_testify//require:go_default_library",
"@org_golang_google_protobuf//proto:go_default_library",
],
)

View File

@@ -142,18 +142,6 @@ func ProcessSlot(ctx context.Context, state state.BeaconState) (state.BeaconStat
); err != nil {
return nil, err
}
// Spec v1.6.1 (pseudocode):
// # [New in Gloas:EIP7732]
// # Unset the next payload availability
// state.execution_payload_availability[(state.slot + 1) % SLOTS_PER_HISTORICAL_ROOT] = 0b0
if state.Version() >= version.Gloas {
index := uint64((state.Slot() + 1) % params.BeaconConfig().SlotsPerHistoricalRoot)
if err := state.UpdateExecutionPayloadAvailabilityAtIndex(index, 0x0); err != nil {
return nil, err
}
}
return state, nil
}

View File

@@ -1,141 +0,0 @@
package transition
import (
"bytes"
"context"
"fmt"
"testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/state"
state_native "github.com/OffchainLabs/prysm/v7/beacon-chain/state/state-native"
fieldparams "github.com/OffchainLabs/prysm/v7/config/fieldparams"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v7/runtime/version"
"github.com/stretchr/testify/require"
)
func TestProcessSlot_GloasClearsNextPayloadAvailability(t *testing.T) {
slot := primitives.Slot(10)
cfg := params.BeaconConfig()
nextIdx := uint64((slot + 1) % cfg.SlotsPerHistoricalRoot)
byteIdx := nextIdx / 8
bitMask := byte(1 << (nextIdx % 8))
availability := bytes.Repeat([]byte{0xFF}, int(cfg.SlotsPerHistoricalRoot/8))
st := newGloasState(t, slot, availability)
_, err := ProcessSlot(context.Background(), st)
require.NoError(t, err)
post := st.ToProto().(*ethpb.BeaconStateGloas)
require.Equal(t, byte(0xFF)&^bitMask, post.ExecutionPayloadAvailability[byteIdx])
}
func TestProcessSlot_GloasClearsNextPayloadAvailability_Wrap(t *testing.T) {
cfg := params.BeaconConfig()
slot := primitives.Slot(cfg.SlotsPerHistoricalRoot - 1)
availability := bytes.Repeat([]byte{0xFF}, int(cfg.SlotsPerHistoricalRoot/8))
st := newGloasState(t, slot, availability)
_, err := ProcessSlot(context.Background(), st)
require.NoError(t, err)
post := st.ToProto().(*ethpb.BeaconStateGloas)
require.Equal(t, byte(0xFE), post.ExecutionPayloadAvailability[0])
}
func TestProcessSlot_GloasAvailabilityUpdateError(t *testing.T) {
slot := primitives.Slot(7)
availability := make([]byte, 1)
st := newGloasState(t, slot, availability)
_, err := ProcessSlot(context.Background(), st)
cfg := params.BeaconConfig()
idx := uint64((slot + 1) % cfg.SlotsPerHistoricalRoot)
byteIdx := idx / 8
require.EqualError(t, err, fmt.Sprintf(
"bit index %d (byte index %d) out of range for execution payload availability length %d",
idx, byteIdx, len(availability),
))
}
func newGloasState(t *testing.T, slot primitives.Slot, availability []byte) state.BeaconState {
t.Helper()
cfg := params.BeaconConfig()
protoState := &ethpb.BeaconStateGloas{
Slot: slot,
LatestBlockHeader: testBeaconBlockHeader(),
BlockRoots: make([][]byte, cfg.SlotsPerHistoricalRoot),
StateRoots: make([][]byte, cfg.SlotsPerHistoricalRoot),
RandaoMixes: make([][]byte, fieldparams.RandaoMixesLength),
ExecutionPayloadAvailability: availability,
BuilderPendingPayments: make([]*ethpb.BuilderPendingPayment, int(cfg.SlotsPerEpoch*2)),
LatestExecutionPayloadBid: &ethpb.ExecutionPayloadBid{
ParentBlockHash: make([]byte, 32),
ParentBlockRoot: make([]byte, 32),
BlockHash: make([]byte, 32),
PrevRandao: make([]byte, 32),
FeeRecipient: make([]byte, 20),
BlobKzgCommitmentsRoot: make([]byte, 32),
},
Eth1Data: &ethpb.Eth1Data{
DepositRoot: make([]byte, 32),
BlockHash: make([]byte, 32),
},
PreviousEpochParticipation: []byte{},
CurrentEpochParticipation: []byte{},
JustificationBits: []byte{0},
PreviousJustifiedCheckpoint: &ethpb.Checkpoint{Root: make([]byte, 32)},
CurrentJustifiedCheckpoint: &ethpb.Checkpoint{Root: make([]byte, 32)},
FinalizedCheckpoint: &ethpb.Checkpoint{Root: make([]byte, 32)},
CurrentSyncCommittee: &ethpb.SyncCommittee{},
NextSyncCommittee: &ethpb.SyncCommittee{},
}
for i := range protoState.BlockRoots {
protoState.BlockRoots[i] = make([]byte, 32)
}
for i := range protoState.StateRoots {
protoState.StateRoots[i] = make([]byte, 32)
}
for i := range protoState.RandaoMixes {
protoState.RandaoMixes[i] = make([]byte, 32)
}
for i := range protoState.BuilderPendingPayments {
protoState.BuilderPendingPayments[i] = &ethpb.BuilderPendingPayment{
Withdrawal: &ethpb.BuilderPendingWithdrawal{
FeeRecipient: make([]byte, 20),
},
}
}
pubkeys := make([][]byte, cfg.SyncCommitteeSize)
for i := range pubkeys {
pubkeys[i] = make([]byte, fieldparams.BLSPubkeyLength)
}
aggPubkey := make([]byte, fieldparams.BLSPubkeyLength)
protoState.CurrentSyncCommittee = &ethpb.SyncCommittee{
Pubkeys: pubkeys,
AggregatePubkey: aggPubkey,
}
protoState.NextSyncCommittee = &ethpb.SyncCommittee{
Pubkeys: pubkeys,
AggregatePubkey: aggPubkey,
}
st, err := state_native.InitializeFromProtoGloas(protoState)
require.NoError(t, err)
require.Equal(t, version.Gloas, st.Version())
return st
}
func testBeaconBlockHeader() *ethpb.BeaconBlockHeader {
return &ethpb.BeaconBlockHeader{
ParentRoot: make([]byte, 32),
StateRoot: make([]byte, 32),
BodyRoot: make([]byte, 32),
}
}

View File

@@ -22,6 +22,10 @@ var ErrNotFoundFeeRecipient = errors.Wrap(ErrNotFound, "fee recipient")
// ErrNotFoundMetadataSeqNum is a not found error specifically for the metadata sequence number getter
var ErrNotFoundMetadataSeqNum = errors.Wrap(ErrNotFound, "metadata sequence number")
// ErrStateDiffIncompatible is returned when state-diff feature is enabled
// but the database was created without state-diff support.
var ErrStateDiffIncompatible = errors.New("state-diff feature enabled but database was created without state-diff support")
var errEmptyBlockSlice = errors.New("[]blocks.ROBlock is empty")
var errIncorrectBlockParent = errors.New("unexpected missing or forked blocks in a []ROBlock")
var errFinalizedChildNotFound = errors.New("unable to find finalized root descending from backfill batch")

View File

@@ -7,6 +7,7 @@ import (
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/blocks"
dbIface "github.com/OffchainLabs/prysm/v7/beacon-chain/db/iface"
"github.com/OffchainLabs/prysm/v7/beacon-chain/state"
"github.com/OffchainLabs/prysm/v7/config/features"
"github.com/OffchainLabs/prysm/v7/encoding/ssz/detect"
"github.com/OffchainLabs/prysm/v7/genesis"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
@@ -42,6 +43,15 @@ func (s *Store) SaveGenesisData(ctx context.Context, genesisState state.BeaconSt
if err := s.SaveGenesisBlockRoot(ctx, genesisBlkRoot); err != nil {
return errors.Wrap(err, "could not save genesis block root")
}
// Initialize state-diff if enabled and not yet initialized.
if features.Get().EnableStateDiff && s.stateDiffCache == nil {
if err := s.initializeStateDiff(0, genesisState); err != nil {
return errors.Wrap(err, "failed to initialize state diff for genesis")
}
log.Info("Initialized state-diff with genesis state")
}
return nil
}

View File

@@ -204,11 +204,31 @@ func NewKVStore(ctx context.Context, dirPath string, opts ...KVStoreOption) (*St
}
if features.Get().EnableStateDiff {
sdCache, err := newStateDiffCache(kv)
// Check if offset already exists (existing state-diff database).
hasOffset, err := kv.hasStateDiffOffset()
if err != nil {
return nil, err
}
kv.stateDiffCache = sdCache
if hasOffset {
// Existing state-diff database - restarts not yet supported.
return nil, errors.New("restarting with existing state-diff database not yet supported")
}
// Check if this is a new database (no head block).
headBlock, err := kv.HeadBlock(ctx)
if err != nil {
return nil, errors.Wrap(err, "could not get head block")
}
if headBlock == nil {
// New database - will be initialized later during checkpoint/genesis sync.
// stateDiffCache stays nil until SaveOrigin or SaveGenesisData initializes it.
log.Info("State-diff enabled: will be initialized during checkpoint or genesis sync")
} else {
// Existing database without state-diff - return store with error for caller to handle.
return kv, ErrStateDiffIncompatible
}
}
return kv, nil

View File

@@ -14,6 +14,7 @@ import (
"github.com/OffchainLabs/prysm/v7/math"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v7/runtime/version"
pkgerrors "github.com/pkg/errors"
"go.etcd.io/bbolt"
)
@@ -122,6 +123,54 @@ func (s *Store) getOffset() uint64 {
return s.stateDiffCache.getOffset()
}
// hasStateDiffOffset checks if the state-diff offset has been set in the database.
// This is used to detect if an existing database has state-diff enabled.
func (s *Store) hasStateDiffOffset() (bool, error) {
var hasOffset bool
err := s.db.View(func(tx *bbolt.Tx) error {
bucket := tx.Bucket(stateDiffBucket)
if bucket == nil {
return nil
}
hasOffset = bucket.Get(offsetKey) != nil
return nil
})
return hasOffset, err
}
// initializeStateDiff sets up the state-diff schema for a new database.
// This should be called during checkpoint sync or genesis sync.
func (s *Store) initializeStateDiff(slot primitives.Slot, initialState state.ReadOnlyBeaconState) error {
// Write offset directly to the database (without using cache which doesn't exist yet).
err := s.db.Update(func(tx *bbolt.Tx) error {
bucket := tx.Bucket(stateDiffBucket)
if bucket == nil {
return bbolt.ErrBucketNotFound
}
offsetBytes := make([]byte, 8)
binary.LittleEndian.PutUint64(offsetBytes, uint64(slot))
return bucket.Put(offsetKey, offsetBytes)
})
if err != nil {
return pkgerrors.Wrap(err, "failed to set offset")
}
// Create the state diff cache (this will read the offset from the database).
sdCache, err := newStateDiffCache(s)
if err != nil {
return pkgerrors.Wrap(err, "failed to create state diff cache")
}
s.stateDiffCache = sdCache
// Save the initial state as a full snapshot.
if err := s.saveFullSnapshot(initialState); err != nil {
return pkgerrors.Wrap(err, "failed to save initial snapshot")
}
return nil
}
func keyForSnapshot(v int) ([]byte, error) {
switch v {
case version.Fulu:

View File

@@ -4,6 +4,7 @@ import (
"context"
"fmt"
"github.com/OffchainLabs/prysm/v7/config/features"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/encoding/ssz/detect"
@@ -111,5 +112,12 @@ func (s *Store) SaveOrigin(ctx context.Context, serState, serBlock []byte) error
return errors.Wrap(err, "save finalized checkpoint")
}
if features.Get().EnableStateDiff {
if err := s.initializeStateDiff(state.Slot(), state); err != nil {
return errors.Wrap(err, "failed to initialize state diff")
}
log.WithField("slot", state.Slot()).Info("Initialized state-diff with checkpoint state")
}
return nil
}

View File

@@ -540,7 +540,12 @@ func openDB(ctx context.Context, dbPath string, clearer *dbClearer) (*kv.Store,
log.WithField("databasePath", dbPath).Info("Checking DB")
d, err := kv.NewKVStore(ctx, dbPath)
if err != nil {
if errors.Is(err, kv.ErrStateDiffIncompatible) {
log.WithError(err).Warn("Disabling state-diff feature")
cfg := features.Get()
cfg.EnableStateDiff = false
features.Init(cfg)
} else if err != nil {
return nil, errors.Wrapf(err, "could not create database at %s", dbPath)
}

View File

@@ -132,7 +132,7 @@ func TestGetSpec(t *testing.T) {
config.MinSyncCommitteeParticipants = 71
config.ProposerReorgCutoffBPS = primitives.BP(121)
config.AttestationDueBPS = primitives.BP(122)
config.AggregateDueBPS = primitives.BP(123)
config.AggregrateDueBPS = primitives.BP(123)
config.ContributionDueBPS = primitives.BP(124)
config.TerminalBlockHash = common.HexToHash("TerminalBlockHash")
config.TerminalBlockHashActivationEpoch = 72
@@ -470,7 +470,7 @@ func TestGetSpec(t *testing.T) {
assert.Equal(t, "121", v)
case "ATTESTATION_DUE_BPS":
assert.Equal(t, "122", v)
case "AGGREGATE_DUE_BPS":
case "AGGREGRATE_DUE_BPS":
assert.Equal(t, "123", v)
case "CONTRIBUTION_DUE_BPS":
assert.Equal(t, "124", v)

View File

@@ -11,7 +11,6 @@ type writeOnlyGloasFields interface {
SetBuilderPendingPayment(index primitives.Slot, payment *ethpb.BuilderPendingPayment) error
RotateBuilderPendingPayments() error
AppendBuilderPendingWithdrawals([]*ethpb.BuilderPendingWithdrawal) error
UpdateExecutionPayloadAvailabilityAtIndex(idx uint64, val byte) error
}
type readOnlyGloasFields interface {

View File

@@ -124,25 +124,3 @@ func (b *BeaconState) SetBuilderPendingPayment(index primitives.Slot, payment *e
b.markFieldAsDirty(types.BuilderPendingPayments)
return nil
}
// UpdateExecutionPayloadAvailabilityAtIndex updates the execution payload availability bit at a specific index.
func (b *BeaconState) UpdateExecutionPayloadAvailabilityAtIndex(idx uint64, val byte) error {
b.lock.Lock()
defer b.lock.Unlock()
byteIndex := idx / 8
bitIndex := idx % 8
if byteIndex >= uint64(len(b.executionPayloadAvailability)) {
return fmt.Errorf("bit index %d (byte index %d) out of range for execution payload availability length %d", idx, byteIndex, len(b.executionPayloadAvailability))
}
if val != 0 {
b.executionPayloadAvailability[byteIndex] |= (1 << bitIndex)
} else {
b.executionPayloadAvailability[byteIndex] &^= (1 << bitIndex)
}
b.markFieldAsDirty(types.ExecutionPayloadAvailability)
return nil
}

View File

@@ -247,44 +247,3 @@ func TestAppendBuilderPendingWithdrawals_UnsupportedVersion(t *testing.T) {
err := st.AppendBuilderPendingWithdrawals([]*ethpb.BuilderPendingWithdrawal{{}})
require.ErrorContains(t, "AppendBuilderPendingWithdrawals", err)
}
func TestUpdateExecutionPayloadAvailabilityAtIndex_SetAndClear(t *testing.T) {
st := newGloasStateWithAvailability(t, make([]byte, 1024))
otherIdx := uint64(8) // byte 1, bit 0
idx := uint64(9) // byte 1, bit 1
require.NoError(t, st.UpdateExecutionPayloadAvailabilityAtIndex(otherIdx, 1))
require.Equal(t, byte(0x01), st.executionPayloadAvailability[1])
require.NoError(t, st.UpdateExecutionPayloadAvailabilityAtIndex(idx, 1))
require.Equal(t, byte(0x03), st.executionPayloadAvailability[1])
require.NoError(t, st.UpdateExecutionPayloadAvailabilityAtIndex(idx, 0))
require.Equal(t, byte(0x01), st.executionPayloadAvailability[1])
}
func TestUpdateExecutionPayloadAvailabilityAtIndex_OutOfRange(t *testing.T) {
st := newGloasStateWithAvailability(t, make([]byte, 1024))
idx := uint64(len(st.executionPayloadAvailability)) * 8
err := st.UpdateExecutionPayloadAvailabilityAtIndex(idx, 1)
require.ErrorContains(t, "out of range", err)
for _, b := range st.executionPayloadAvailability {
if b != 0 {
t.Fatalf("execution payload availability mutated on error")
}
}
}
func newGloasStateWithAvailability(t *testing.T, availability []byte) *BeaconState {
t.Helper()
st, err := InitializeFromProtoUnsafeGloas(&ethpb.BeaconStateGloas{
ExecutionPayloadAvailability: availability,
})
require.NoError(t, err)
return st.(*BeaconState)
}

View File

@@ -137,7 +137,7 @@ func (s *State) migrateToColdHdiff(ctx context.Context, fRoot [32]byte) error {
if ctx.Err() != nil {
return ctx.Err()
}
_, lvl, err := s.beaconDB.SlotInDiffTree(slot)
offset, lvl, err := s.beaconDB.SlotInDiffTree(slot)
if err != nil {
log.WithError(err).Errorf("could not determine if slot %d is in diff tree", slot)
continue
@@ -145,6 +145,9 @@ func (s *State) migrateToColdHdiff(ctx context.Context, fRoot [32]byte) error {
if lvl == -1 {
continue
}
if uint64(slot) == offset {
continue
}
// The state needs to be saved.
// Try the epoch boundary cache first.
cached, exists, err := s.epochBoundaryStateCache.getBySlot(slot)

View File

@@ -1,2 +0,0 @@
### Fixed
- Fixed a typo: AggregrateDueBPS -> AggregateDueBPS.

View File

@@ -1,3 +0,0 @@
### Ignored
- delayed head evaluator check to mid epoch for e2e.

View File

@@ -0,0 +1,3 @@
### Added
- Initialize db with state-diff feature flag.

View File

@@ -1,3 +0,0 @@
### Added
- Add slot processing with execution payload availability updates

View File

@@ -158,6 +158,7 @@ var appFlags = []cli.Flag{
dasFlags.BackfillOldestSlot,
dasFlags.BlobRetentionEpochFlag,
flags.BatchVerifierLimit,
flags.StateDiffExponents,
flags.DisableEphemeralLogFile,
}

View File

@@ -74,6 +74,7 @@ var appHelpFlagGroups = []flagGroup{
flags.RPCHost,
flags.RPCPort,
flags.BatchVerifierLimit,
flags.StateDiffExponents,
},
},
{

View File

@@ -280,6 +280,7 @@ var BeaconChainFlags = combinedFlags([]cli.Flag{
DisableQUIC,
EnableDiscoveryReboot,
enableExperimentalAttestationPool,
EnableStateDiff,
forceHeadFlag,
blacklistRoots,
enableHashtree,

View File

@@ -88,7 +88,7 @@ type BeaconChainConfig struct {
IntervalsPerSlot uint64 `yaml:"INTERVALS_PER_SLOT"` // IntervalsPerSlot defines the number of fork choice intervals in a slot defined in the fork choice spec.
ProposerReorgCutoffBPS primitives.BP `yaml:"PROPOSER_REORG_CUTOFF_BPS" spec:"true"` // ProposerReorgCutoffBPS defines the proposer reorg deadline in basis points of the slot.
AttestationDueBPS primitives.BP `yaml:"ATTESTATION_DUE_BPS" spec:"true"` // AttestationDueBPS defines the attestation due time in basis points of the slot.
AggregateDueBPS primitives.BP `yaml:"AGGREGATE_DUE_BPS" spec:"true"` // AggregateDueBPS defines the aggregate due time in basis points of the slot.
AggregrateDueBPS primitives.BP `yaml:"AGGREGRATE_DUE_BPS" spec:"true"` // AggregrateDueBPS defines the aggregate due time in basis points of the slot.
SyncMessageDueBPS primitives.BP `yaml:"SYNC_MESSAGE_DUE_BPS" spec:"true"` // SyncMessageDueBPS defines the sync message due time in basis points of the slot.
ContributionDueBPS primitives.BP `yaml:"CONTRIBUTION_DUE_BPS" spec:"true"` // ContributionDueBPS defines the contribution due time in basis points of the slot.

View File

@@ -243,7 +243,7 @@ func ConfigToYaml(cfg *BeaconChainConfig) []byte {
fmt.Sprintf("MAX_BLOBS_PER_BLOCK: %d", cfg.DeprecatedMaxBlobsPerBlock),
fmt.Sprintf("PROPOSER_REORG_CUTOFF_BPS: %d", cfg.ProposerReorgCutoffBPS),
fmt.Sprintf("ATTESTATION_DUE_BPS: %d", cfg.AttestationDueBPS),
fmt.Sprintf("AGGREGATE_DUE_BPS: %d", cfg.AggregateDueBPS),
fmt.Sprintf("AGGREGRATE_DUE_BPS: %d", cfg.AggregrateDueBPS),
fmt.Sprintf("SYNC_MESSAGE_DUE_BPS: %d", cfg.SyncMessageDueBPS),
fmt.Sprintf("CONTRIBUTION_DUE_BPS: %d", cfg.ContributionDueBPS),
}

View File

@@ -24,6 +24,7 @@ import (
// These are variables that we don't use in Prysm. (i.e. future hardfork, light client... etc)
// IMPORTANT: Use one field per line and sort these alphabetically to reduce conflicts.
var placeholderFields = []string{
"AGGREGATE_DUE_BPS",
"AGGREGATE_DUE_BPS_GLOAS",
"ATTESTATION_DEADLINE",
"ATTESTATION_DUE_BPS_GLOAS",
@@ -98,7 +99,7 @@ func assertEqualConfigs(t *testing.T, name string, fields []string, expected, ac
assert.Equal(t, expected.HysteresisDownwardMultiplier, actual.HysteresisDownwardMultiplier, "%s: HysteresisDownwardMultiplier", name)
assert.Equal(t, expected.HysteresisUpwardMultiplier, actual.HysteresisUpwardMultiplier, "%s: HysteresisUpwardMultiplier", name)
assert.Equal(t, expected.AttestationDueBPS, actual.AttestationDueBPS, "%s: AttestationDueBPS", name)
assert.Equal(t, expected.AggregateDueBPS, actual.AggregateDueBPS, "%s: AggregateDueBPS", name)
assert.Equal(t, expected.AggregrateDueBPS, actual.AggregrateDueBPS, "%s: AggregrateDueBPS", name)
assert.Equal(t, expected.ContributionDueBPS, actual.ContributionDueBPS, "%s: ContributionDueBPS", name)
assert.Equal(t, expected.ProposerReorgCutoffBPS, actual.ProposerReorgCutoffBPS, "%s: ProposerReorgCutoffBPS", name)
assert.Equal(t, expected.SyncMessageDueBPS, actual.SyncMessageDueBPS, "%s: SyncMessageDueBPS", name)

View File

@@ -123,7 +123,7 @@ var mainnetBeaconConfig = &BeaconChainConfig{
// Time-based protocol parameters.
ProposerReorgCutoffBPS: primitives.BP(1667),
AttestationDueBPS: primitives.BP(3333),
AggregateDueBPS: primitives.BP(6667),
AggregrateDueBPS: primitives.BP(6667),
SyncMessageDueBPS: primitives.BP(3333),
ContributionDueBPS: primitives.BP(6667),

View File

@@ -97,6 +97,7 @@ go_test(
"endtoend_setup_test.go",
"endtoend_test.go",
"minimal_e2e_test.go",
"minimal_hdiff_e2e_test.go",
"minimal_slashing_e2e_test.go",
"slasher_simulator_e2e_test.go",
],

View File

@@ -296,7 +296,7 @@ func (r *testRunner) waitForMatchingHead(ctx context.Context, timeout time.Durat
}
func (r *testRunner) testCheckpointSync(ctx context.Context, g *errgroup.Group, i int, conns []*grpc.ClientConn, bnAPI, enr, minerEnr string) error {
matchTimeout := 5 * time.Minute
matchTimeout := 3 * time.Minute
ethNode := eth1.NewNode(i, minerEnr)
g.Go(func() error {
return ethNode.Start(ctx)

View File

@@ -10,7 +10,6 @@ import (
"net/http"
"time"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
eth "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
e2e "github.com/OffchainLabs/prysm/v7/testing/endtoend/params"
@@ -129,42 +128,8 @@ func finishedSyncing(_ *e2etypes.EvaluationContext, conns ...*grpc.ClientConn) e
return nil
}
// waitForMidEpoch waits until we're at least halfway into the current epoch
// and 3/4 into the current slot. This prevents race conditions at epoch
// boundaries and slot boundaries where different nodes may report different heads.
func waitForMidEpoch(conn *grpc.ClientConn) error {
beaconClient := eth.NewBeaconChainClient(conn)
slotsPerEpoch := params.BeaconConfig().SlotsPerEpoch
secondsPerSlot := params.BeaconConfig().SecondsPerSlot
midEpochSlot := slotsPerEpoch / 2
for {
chainHead, err := beaconClient.GetChainHead(context.Background(), &emptypb.Empty{})
if err != nil {
return err
}
slotInEpoch := chainHead.HeadSlot % slotsPerEpoch
// If we're at least halfway into the epoch, we're safe
if slotInEpoch >= midEpochSlot {
// Wait 3/4 into the slot to ensure block propagation
time.Sleep(time.Duration(secondsPerSlot) * time.Second * 3 / 4)
return nil
}
// Wait for the remaining slots until mid-epoch
slotsToWait := midEpochSlot - slotInEpoch
time.Sleep(time.Duration(slotsToWait) * time.Duration(secondsPerSlot) * time.Second)
}
}
func allNodesHaveSameHead(_ *e2etypes.EvaluationContext, conns ...*grpc.ClientConn) error {
// Wait until we're at least halfway into the epoch to avoid race conditions
// at epoch boundaries where nodes may report different epochs.
if err := waitForMidEpoch(conns[0]); err != nil {
return errors.Wrap(err, "failed waiting for mid-epoch")
}
headEpochs := make([]primitives.Epoch, len(conns))
headBlockRoots := make([][]byte, len(conns))
justifiedRoots := make([][]byte, len(conns))
prevJustifiedRoots := make([][]byte, len(conns))
finalizedRoots := make([][]byte, len(conns))
@@ -181,7 +146,6 @@ func allNodesHaveSameHead(_ *e2etypes.EvaluationContext, conns ...*grpc.ClientCo
return errors.Wrapf(err, "connection number=%d", conIdx)
}
headEpochs[conIdx] = chainHead.HeadEpoch
headBlockRoots[conIdx] = chainHead.HeadBlockRoot
justifiedRoots[conIdx] = chainHead.JustifiedBlockRoot
prevJustifiedRoots[conIdx] = chainHead.PreviousJustifiedBlockRoot
finalizedRoots[conIdx] = chainHead.FinalizedBlockRoot
@@ -202,14 +166,6 @@ func allNodesHaveSameHead(_ *e2etypes.EvaluationContext, conns ...*grpc.ClientCo
headEpochs[i],
)
}
if !bytes.Equal(headBlockRoots[0], headBlockRoots[i]) {
return fmt.Errorf(
"received conflicting head block roots on node %d, expected %#x, received %#x",
i,
headBlockRoots[0],
headBlockRoots[i],
)
}
if !bytes.Equal(justifiedRoots[0], justifiedRoots[i]) {
return fmt.Errorf(
"received conflicting justified block roots on node %d, expected %#x, received %#x: %s and %s",

View File

@@ -0,0 +1,16 @@
package endtoend
import (
"testing"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/runtime/version"
"github.com/OffchainLabs/prysm/v7/testing/endtoend/types"
)
func TestEndToEnd_MinimalConfig_WithStateDiff(t *testing.T) {
r := e2eMinimal(t, types.InitForkCfg(version.Bellatrix, version.Electra, params.E2ETestConfig()),
types.WithStateDiff(),
)
r.run()
}

View File

@@ -76,6 +76,15 @@ func WithSSZOnly() E2EConfigOpt {
}
}
func WithStateDiff() E2EConfigOpt {
return func(cfg *E2EConfig) {
cfg.BeaconFlags = append(cfg.BeaconFlags,
"--enable-state-diff",
"--state-diff-exponents=6,5", // Small exponents for quick testing
)
}
}
// WithExitEpoch sets a custom epoch for voluntary exit submission.
// This affects ProposeVoluntaryExit, ValidatorsHaveExited, SubmitWithdrawal, and ValidatorsHaveWithdrawn evaluators.
func WithExitEpoch(e primitives.Epoch) E2EConfigOpt {

View File

@@ -202,7 +202,6 @@ 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__sanity__slots_test.go",
"gloas__ssz_static__ssz_static_test.go",
"phase0__epoch_processing__effective_balance_updates_test.go",
"phase0__epoch_processing__epoch_processing_test.go",
@@ -283,7 +282,6 @@ go_test(
"//testing/spectest/shared/fulu/ssz_static:go_default_library",
"//testing/spectest/shared/gloas/epoch_processing:go_default_library",
"//testing/spectest/shared/gloas/operations:go_default_library",
"//testing/spectest/shared/gloas/sanity:go_default_library",
"//testing/spectest/shared/gloas/ssz_static:go_default_library",
"//testing/spectest/shared/phase0/epoch_processing:go_default_library",
"//testing/spectest/shared/phase0/finality:go_default_library",

View File

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

View File

@@ -208,7 +208,6 @@ 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__sanity__slots_test.go",
"gloas__ssz_static__ssz_static_test.go",
"phase0__epoch_processing__effective_balance_updates_test.go",
"phase0__epoch_processing__epoch_processing_test.go",
@@ -293,7 +292,6 @@ go_test(
"//testing/spectest/shared/fulu/ssz_static:go_default_library",
"//testing/spectest/shared/gloas/epoch_processing:go_default_library",
"//testing/spectest/shared/gloas/operations:go_default_library",
"//testing/spectest/shared/gloas/sanity:go_default_library",
"//testing/spectest/shared/gloas/ssz_static:go_default_library",
"//testing/spectest/shared/phase0/epoch_processing:go_default_library",
"//testing/spectest/shared/phase0/finality:go_default_library",

View File

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

View File

@@ -1,19 +0,0 @@
load("@prysm//tools/go:def.bzl", "go_library")
go_library(
name = "go_default_library",
testonly = True,
srcs = ["slot_processing.go"],
importpath = "github.com/OffchainLabs/prysm/v7/testing/spectest/shared/gloas/sanity",
visibility = ["//testing/spectest:__subpackages__"],
deps = [
"//beacon-chain/core/transition:go_default_library",
"//beacon-chain/state/state-native:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//testing/require:go_default_library",
"//testing/spectest/utils:go_default_library",
"//testing/util:go_default_library",
"@com_github_golang_snappy//:go_default_library",
"@org_golang_google_protobuf//proto:go_default_library",
],
)

View File

@@ -1,61 +0,0 @@
package sanity
import (
"context"
"strconv"
"testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/transition"
state_native "github.com/OffchainLabs/prysm/v7/beacon-chain/state/state-native"
ethpb "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"
"google.golang.org/protobuf/proto"
)
func init() {
transition.SkipSlotCache.Disable()
}
// RunSlotProcessingTests executes "sanity/slots" tests.
func RunSlotProcessingTests(t *testing.T, config string) {
require.NoError(t, utils.SetConfig(t, config))
testFolders, testsFolderPath := utils.TestFolders(t, config, "gloas", "sanity/slots/pyspec_tests")
for _, folder := range testFolders {
t.Run(folder.Name(), func(t *testing.T) {
preBeaconStateFile, err := util.BazelFileBytes(testsFolderPath, folder.Name(), "pre.ssz_snappy")
require.NoError(t, err)
preBeaconStateSSZ, err := snappy.Decode(nil /* dst */, preBeaconStateFile)
require.NoError(t, err, "Failed to decompress")
base := &ethpb.BeaconStateGloas{}
require.NoError(t, base.UnmarshalSSZ(preBeaconStateSSZ), "Failed to unmarshal")
beaconState, err := state_native.InitializeFromProtoGloas(base)
require.NoError(t, err)
file, err := util.BazelFileBytes(testsFolderPath, folder.Name(), "slots.yaml")
require.NoError(t, err)
fileStr := string(file)
slotsCount, err := strconv.ParseUint(fileStr[:len(fileStr)-5], 10, 64)
require.NoError(t, err)
postBeaconStateFile, err := util.BazelFileBytes(testsFolderPath, folder.Name(), "post.ssz_snappy")
require.NoError(t, err)
postBeaconStateSSZ, err := snappy.Decode(nil /* dst */, postBeaconStateFile)
require.NoError(t, err, "Failed to decompress")
postBeaconState := &ethpb.BeaconStateGloas{}
require.NoError(t, postBeaconState.UnmarshalSSZ(postBeaconStateSSZ), "Failed to unmarshal")
postState, err := transition.ProcessSlots(context.Background(), beaconState, beaconState.Slot().Add(slotsCount))
require.NoError(t, err)
pbState, err := state_native.ProtobufBeaconStateGloas(postState.ToProto())
require.NoError(t, err)
if !proto.Equal(pbState, postBeaconState) {
t.Fatal("Did not receive expected post state")
}
})
}
}

View File

@@ -207,7 +207,7 @@ func (v *validator) signSlotWithSelectionProof(ctx context.Context, pubKey [fiel
// such that any attestations from this slot have time to reach the beacon node
// before creating the aggregated attestation.
func (v *validator) waitToSlotTwoThirds(ctx context.Context, slot primitives.Slot) {
v.waitUntilSlotComponent(ctx, slot, params.BeaconConfig().AggregateDueBPS)
v.waitUntilSlotComponent(ctx, slot, params.BeaconConfig().AggregrateDueBPS)
}
// This returns the signature of validator signing over aggregate and

View File

@@ -257,7 +257,7 @@ func TestWaitForSlotTwoThird_WaitCorrectly(t *testing.T) {
numOfSlots := primitives.Slot(4)
slotDuration := params.BeaconConfig().SlotDuration()
validator.genesisTime = currentTime.Add(-slotDuration * time.Duration(numOfSlots))
timeToSleep := params.BeaconConfig().SlotComponentDuration(params.BeaconConfig().AggregateDueBPS)
timeToSleep := params.BeaconConfig().SlotComponentDuration(params.BeaconConfig().AggregrateDueBPS)
twoThirdTime := currentTime.Add(timeToSleep)
validator.waitToSlotTwoThirds(t.Context(), numOfSlots)

View File

@@ -51,7 +51,7 @@ func (v *validator) slotComponentSpanName(component primitives.BP) string {
switch component {
case cfg.AttestationDueBPS:
return "validator.waitAttestationWindow"
case cfg.AggregateDueBPS:
case cfg.AggregrateDueBPS:
return "validator.waitAggregateWindow"
case cfg.SyncMessageDueBPS:
return "validator.waitSyncMessageWindow"

View File

@@ -47,7 +47,7 @@ func TestSlotComponentSpanName(t *testing.T) {
},
{
name: "aggregate",
component: cfg.AggregateDueBPS,
component: cfg.AggregrateDueBPS,
expected: "validator.waitAggregateWindow",
},
{