Compare commits

...

7 Commits

Author SHA1 Message Date
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
18 changed files with 455 additions and 22 deletions

View File

@@ -89,6 +89,7 @@ type NoHeadAccessDatabase interface {
SaveBlocks(ctx context.Context, blocks []interfaces.ReadOnlySignedBeaconBlock) error
SaveROBlocks(ctx context.Context, blks []blocks.ROBlock, cache bool) error
SaveGenesisBlockRoot(ctx context.Context, blockRoot [32]byte) error
SlotByBlockRoot(context.Context, [32]byte) (primitives.Slot, error)
// State related methods.
SaveState(ctx context.Context, state state.ReadOnlyBeaconState, blockRoot [32]byte) error
SaveStates(ctx context.Context, states []state.ReadOnlyBeaconState, blockRoots [][32]byte) error
@@ -96,6 +97,7 @@ type NoHeadAccessDatabase interface {
DeleteStates(ctx context.Context, blockRoots [][32]byte) error
SaveStateSummary(ctx context.Context, summary *ethpb.StateSummary) error
SaveStateSummaries(ctx context.Context, summaries []*ethpb.StateSummary) error
SlotInDiffTree(primitives.Slot) (uint64, int, error)
// Checkpoint operations.
SaveJustifiedCheckpoint(ctx context.Context, checkpoint *ethpb.Checkpoint) error
SaveFinalizedCheckpoint(ctx context.Context, checkpoint *ethpb.Checkpoint) error

View File

@@ -32,6 +32,7 @@ go_library(
"state_diff_helpers.go",
"state_summary.go",
"state_summary_cache.go",
"testing_helpers.go",
"utils.go",
"validated_checkpoint.go",
"wss.go",

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 - warn and disable feature.
log.Warn("State-diff feature ignored: database was created without state-diff support")
}
}
return kv, nil

View File

@@ -23,6 +23,16 @@ const (
The data at level 0 is saved every 2**exponent[0] slots and always contains a full state snapshot that is used as a base for the delta saved at other levels.
*/
// SlotInDiffTree returns whether the given slot is a saving point in the diff tree.
// It it is, it also returns the offset and level in the tree.
func (s *Store) SlotInDiffTree(slot primitives.Slot) (uint64, int, error) {
offset := s.getOffset()
if uint64(slot) < offset {
return 0, -1, ErrSlotBeforeOffset
}
return offset, computeLevel(offset, slot), nil
}
// saveStateByDiff takes a state and decides between saving a full state snapshot or a diff.
func (s *Store) saveStateByDiff(ctx context.Context, st state.ReadOnlyBeaconState) error {
_, span := trace.StartSpan(ctx, "BeaconDB.saveStateByDiff")
@@ -33,13 +43,10 @@ func (s *Store) saveStateByDiff(ctx context.Context, st state.ReadOnlyBeaconStat
}
slot := st.Slot()
offset := s.getOffset()
if uint64(slot) < offset {
return ErrSlotBeforeOffset
offset, lvl, err := s.SlotInDiffTree(slot)
if err != nil {
return errors.Wrap(err, "could not determine if slot is in diff tree")
}
// Find the level to save the state.
lvl := computeLevel(offset, slot)
if lvl == -1 {
return 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"
)
@@ -119,6 +120,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

@@ -0,0 +1,35 @@
package kv
import (
"encoding/binary"
"go.etcd.io/bbolt"
)
// InitStateDiffCacheForTesting initializes the state diff cache with the given offset.
// This is intended for testing purposes when setting up state diff after database creation.
// This file is only compiled when the "testing" build tag is set.
func (s *Store) InitStateDiffCacheForTesting(offset uint64) error {
// First, set the offset in the database.
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, offset)
return bucket.Put([]byte("offset"), offsetBytes)
})
if err != nil {
return err
}
// Then create the state diff cache.
sdCache, err := newStateDiffCache(s)
if err != nil {
return err
}
s.stateDiffCache = sdCache
return nil
}

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,13 @@ func (s *Store) SaveOrigin(ctx context.Context, serState, serBlock []byte) error
return errors.Wrap(err, "save finalized checkpoint")
}
// Initialize state-diff if enabled and not yet initialized.
if features.Get().EnableStateDiff && s.stateDiffCache == nil {
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

@@ -29,6 +29,7 @@ go_library(
"//beacon-chain/state:go_default_library",
"//beacon-chain/sync/backfill/coverage:go_default_library",
"//cache/lru:go_default_library",
"//config/features:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/interfaces:go_default_library",
@@ -68,11 +69,14 @@ go_test(
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/core/transition:go_default_library",
"//beacon-chain/db:go_default_library",
"//beacon-chain/db/kv:go_default_library",
"//beacon-chain/db/testing:go_default_library",
"//beacon-chain/forkchoice/doubly-linked-tree:go_default_library",
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/state-native:go_default_library",
"//beacon-chain/state/testing:go_default_library",
"//cmd/beacon-chain/flags:go_default_library",
"//config/features:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/blocks/testing:go_default_library",

View File

@@ -5,10 +5,13 @@ import (
"encoding/hex"
"fmt"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/transition"
"github.com/OffchainLabs/prysm/v7/beacon-chain/state"
"github.com/OffchainLabs/prysm/v7/config/features"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/encoding/bytesutil"
"github.com/OffchainLabs/prysm/v7/monitoring/tracing/trace"
"github.com/pkg/errors"
"github.com/sirupsen/logrus"
)
@@ -25,6 +28,10 @@ func (s *State) MigrateToCold(ctx context.Context, fRoot [32]byte) error {
s.migrationLock.Lock()
defer s.migrationLock.Unlock()
if features.Get().EnableStateDiff {
return s.migrateToColdHdiff(ctx, fRoot)
}
s.finalizedInfo.lock.RLock()
oldFSlot := s.finalizedInfo.slot
s.finalizedInfo.lock.RUnlock()
@@ -90,21 +97,8 @@ func (s *State) MigrateToCold(ctx context.Context, fRoot [32]byte) error {
}
}
}
if s.beaconDB.HasState(ctx, aRoot) {
// If you are migrating a state and its already part of the hot state cache saved to the db,
// you can just remove it from the hot state cache as it becomes redundant.
s.saveHotStateDB.lock.Lock()
roots := s.saveHotStateDB.blockRootsOfSavedStates
for i := range roots {
if aRoot == roots[i] {
s.saveHotStateDB.blockRootsOfSavedStates = append(roots[:i], roots[i+1:]...)
// There shouldn't be duplicated roots in `blockRootsOfSavedStates`.
// Break here is ok.
break
}
}
s.saveHotStateDB.lock.Unlock()
s.migrateHotToCold(aRoot)
continue
}
@@ -129,3 +123,103 @@ func (s *State) MigrateToCold(ctx context.Context, fRoot [32]byte) error {
return nil
}
// migrateToColdHdiff saves the state-diffs for slots that are in the state diff tree after finalization
func (s *State) migrateToColdHdiff(ctx context.Context, fRoot [32]byte) error {
s.finalizedInfo.lock.RLock()
oldFSlot := s.finalizedInfo.slot
s.finalizedInfo.lock.RUnlock()
fSlot, err := s.beaconDB.SlotByBlockRoot(ctx, fRoot)
if err != nil {
return errors.Wrap(err, "could not get slot by block root")
}
for slot := oldFSlot; slot < fSlot; slot++ {
if ctx.Err() != nil {
return ctx.Err()
}
_, 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
}
if lvl == -1 {
continue
}
// The state needs to be saved.
// Try the epoch boundary cache first.
cached, exists, err := s.epochBoundaryStateCache.getBySlot(slot)
if err != nil {
log.WithError(err).Errorf("could not get epoch boundary state for slot %d", slot)
cached = nil
exists = false
}
var aRoot [32]byte
var aState state.BeaconState
if exists {
aRoot = cached.root
aState = cached.state
} else {
_, roots, err := s.beaconDB.HighestRootsBelowSlot(ctx, slot)
if err != nil {
return err
}
// Given the block has been finalized, the db should not have more than one block in a given slot.
// We should error out when this happens.
if len(roots) != 1 {
return errUnknownBlock
}
aRoot = roots[0]
// Different than the legacy MigrateToCold, we need to always get the state even if
// the state exists in DB as part of the hot state db, because we need to process slots
// to the state diff tree slots.
aState, err = s.StateByRoot(ctx, aRoot)
if err != nil {
return err
}
}
if s.beaconDB.HasState(ctx, aRoot) {
s.migrateHotToCold(aRoot)
continue
}
// advance slots to the target slot
if aState.Slot() < slot {
aState, err = transition.ProcessSlots(ctx, aState, slot)
if err != nil {
return errors.Wrapf(err, "could not process slots to slot %d", slot)
}
}
if err := s.beaconDB.SaveState(ctx, aState, aRoot); err != nil {
return err
}
log.WithFields(
logrus.Fields{
"slot": aState.Slot(),
"root": fmt.Sprintf("%#x", aRoot),
}).Info("Saved state in DB")
}
// Update finalized info in memory.
fInfo, ok, err := s.epochBoundaryStateCache.getByBlockRoot(fRoot)
if err != nil {
return err
}
if ok {
s.SaveFinalizedState(fSlot, fRoot, fInfo.state)
}
return nil
}
func (s *State) migrateHotToCold(aRoot [32]byte) {
// If you are migrating a state and its already part of the hot state cache saved to the db,
// you can just remove it from the hot state cache as it becomes redundant.
s.saveHotStateDB.lock.Lock()
roots := s.saveHotStateDB.blockRootsOfSavedStates
for i := range roots {
if aRoot == roots[i] {
s.saveHotStateDB.blockRootsOfSavedStates = append(roots[:i], roots[i+1:]...)
// There shouldn't be duplicated roots in `blockRootsOfSavedStates`.
// Break here is ok.
break
}
}
s.saveHotStateDB.lock.Unlock()
}

View File

@@ -4,8 +4,11 @@ import (
"testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/blocks"
"github.com/OffchainLabs/prysm/v7/beacon-chain/db/kv"
testDB "github.com/OffchainLabs/prysm/v7/beacon-chain/db/testing"
doublylinkedtree "github.com/OffchainLabs/prysm/v7/beacon-chain/forkchoice/doubly-linked-tree"
"github.com/OffchainLabs/prysm/v7/cmd/beacon-chain/flags"
"github.com/OffchainLabs/prysm/v7/config/features"
consensusblocks "github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
@@ -224,3 +227,170 @@ func TestMigrateToCold_ParallelCalls(t *testing.T) {
assert.DeepEqual(t, [][32]byte{r7}, service.saveHotStateDB.blockRootsOfSavedStates, "Did not remove all saved hot state roots")
require.LogsContain(t, hook, "Saved state in DB")
}
// =========================================================================
// Tests for migrateToColdHdiff (state diff migration)
// =========================================================================
// setStateDiffExponents sets state diff exponents for testing.
// Uses exponents [6, 5] which means:
// - Level 0: Every 2^6 = 64 slots (full snapshot)
// - Level 1: Every 2^5 = 32 slots (diff)
func setStateDiffExponents() {
globalFlags := flags.GlobalFlags{
StateDiffExponents: []int{6, 5},
}
flags.Init(&globalFlags)
}
// TestMigrateToColdHdiff_CanUpdateFinalizedInfo verifies that the migration
// correctly updates finalized info when migrating to slots not in the diff tree.
func TestMigrateToColdHdiff_CanUpdateFinalizedInfo(t *testing.T) {
ctx := t.Context()
// Set exponents and create DB first (without EnableStateDiff flag).
setStateDiffExponents()
beaconDB := testDB.SetupDB(t)
// Initialize the state diff cache via the method on *kv.Store (not in interface).
require.NoError(t, beaconDB.(*kv.Store).InitStateDiffCacheForTesting(0))
// Now enable the feature flag.
resetCfg := features.InitWithReset(&features.Flags{EnableStateDiff: true})
defer resetCfg()
service := New(beaconDB, doublylinkedtree.New())
beaconState, _ := util.DeterministicGenesisState(t, 32)
genesisStateRoot, err := beaconState.HashTreeRoot(ctx)
require.NoError(t, err)
genesis := blocks.NewGenesisBlock(genesisStateRoot[:])
util.SaveBlock(t, ctx, beaconDB, genesis)
gRoot, err := genesis.Block.HashTreeRoot()
require.NoError(t, err)
require.NoError(t, beaconDB.SaveGenesisBlockRoot(ctx, gRoot))
// Put genesis state in epoch boundary cache so migrateToColdHdiff doesn't need to retrieve from DB.
require.NoError(t, service.epochBoundaryStateCache.put(gRoot, beaconState))
// Set initial finalized info at genesis.
service.finalizedInfo = &finalizedInfo{
slot: 0,
root: gRoot,
state: beaconState,
}
// Create finalized block at slot 10 (not in diff tree, so no intermediate states saved).
finalizedState := beaconState.Copy()
require.NoError(t, finalizedState.SetSlot(10))
b := util.NewBeaconBlock()
b.Block.Slot = 10
fRoot, err := b.Block.HashTreeRoot()
require.NoError(t, err)
util.SaveBlock(t, ctx, beaconDB, b)
require.NoError(t, service.epochBoundaryStateCache.put(fRoot, finalizedState))
require.NoError(t, service.MigrateToCold(ctx, fRoot))
// Verify finalized info is updated.
assert.Equal(t, primitives.Slot(10), service.finalizedInfo.slot)
assert.DeepEqual(t, fRoot, service.finalizedInfo.root)
expectedHTR, err := finalizedState.HashTreeRoot(ctx)
require.NoError(t, err)
actualHTR, err := service.finalizedInfo.state.HashTreeRoot(ctx)
require.NoError(t, err)
assert.DeepEqual(t, expectedHTR, actualHTR)
}
// TestMigrateToColdHdiff_SkipsSlotsNotInDiffTree verifies that the migration
// skips slots that are not in the diff tree.
func TestMigrateToColdHdiff_SkipsSlotsNotInDiffTree(t *testing.T) {
hook := logTest.NewGlobal()
ctx := t.Context()
// Set exponents and create DB first (without EnableStateDiff flag).
setStateDiffExponents()
beaconDB := testDB.SetupDB(t)
// Initialize the state diff cache via the method on *kv.Store (not in interface).
require.NoError(t, beaconDB.(*kv.Store).InitStateDiffCacheForTesting(0))
// Now enable the feature flag.
resetCfg := features.InitWithReset(&features.Flags{EnableStateDiff: true})
defer resetCfg()
service := New(beaconDB, doublylinkedtree.New())
beaconState, pks := util.DeterministicGenesisState(t, 32)
genesisStateRoot, err := beaconState.HashTreeRoot(ctx)
require.NoError(t, err)
genesis := blocks.NewGenesisBlock(genesisStateRoot[:])
util.SaveBlock(t, ctx, beaconDB, genesis)
gRoot, err := genesis.Block.HashTreeRoot()
require.NoError(t, err)
require.NoError(t, beaconDB.SaveGenesisBlockRoot(ctx, gRoot))
// Start from slot 1 to avoid slot 0 which is in the diff tree.
service.finalizedInfo = &finalizedInfo{
slot: 1,
root: gRoot,
state: beaconState,
}
// Reset the log hook to ignore setup logs.
hook.Reset()
// Create a block at slot 20 (NOT in diff tree with exponents [6,5]).
b20, err := util.GenerateFullBlock(beaconState, pks, util.DefaultBlockGenConfig(), 20)
require.NoError(t, err)
r20, err := b20.Block.HashTreeRoot()
require.NoError(t, err)
util.SaveBlock(t, ctx, beaconDB, b20)
require.NoError(t, beaconDB.SaveStateSummary(ctx, &ethpb.StateSummary{Slot: 20, Root: r20[:]}))
// Put finalized state in cache.
finalizedState := beaconState.Copy()
require.NoError(t, finalizedState.SetSlot(20))
require.NoError(t, service.epochBoundaryStateCache.put(r20, finalizedState))
require.NoError(t, service.MigrateToCold(ctx, r20))
// Verify NO states were saved during migration (slots 1-19 are not in diff tree).
assert.LogsDoNotContain(t, hook, "Saved state in DB")
}
// TestMigrateToColdHdiff_NoOpWhenFinalizedSlotNotAdvanced verifies that
// migration is a no-op when the finalized slot has not advanced.
func TestMigrateToColdHdiff_NoOpWhenFinalizedSlotNotAdvanced(t *testing.T) {
ctx := t.Context()
// Set exponents and create DB first (without EnableStateDiff flag).
setStateDiffExponents()
beaconDB := testDB.SetupDB(t)
// Initialize the state diff cache via the method on *kv.Store (not in interface).
require.NoError(t, beaconDB.(*kv.Store).InitStateDiffCacheForTesting(0))
// Now enable the feature flag.
resetCfg := features.InitWithReset(&features.Flags{EnableStateDiff: true})
defer resetCfg()
service := New(beaconDB, doublylinkedtree.New())
beaconState, _ := util.DeterministicGenesisState(t, 32)
genesisStateRoot, err := beaconState.HashTreeRoot(ctx)
require.NoError(t, err)
genesis := blocks.NewGenesisBlock(genesisStateRoot[:])
util.SaveBlock(t, ctx, beaconDB, genesis)
gRoot, err := genesis.Block.HashTreeRoot()
require.NoError(t, err)
require.NoError(t, beaconDB.SaveGenesisBlockRoot(ctx, gRoot))
// Set finalized info already at slot 50.
finalizedState := beaconState.Copy()
require.NoError(t, finalizedState.SetSlot(50))
service.finalizedInfo = &finalizedInfo{
slot: 50,
root: gRoot,
state: finalizedState,
}
// Create block at same slot 50.
b := util.NewBeaconBlock()
b.Block.Slot = 50
fRoot, err := b.Block.HashTreeRoot()
require.NoError(t, err)
util.SaveBlock(t, ctx, beaconDB, b)
require.NoError(t, service.epochBoundaryStateCache.put(fRoot, finalizedState))
// Migration should be a no-op (finalized slot not advancing).
require.NoError(t, service.MigrateToCold(ctx, fRoot))
}

View File

@@ -0,0 +1,3 @@
### Added
- Migrate to cold with the hdiff feature.

View File

@@ -156,6 +156,7 @@ var appFlags = []cli.Flag{
dasFlags.BackfillOldestSlot,
dasFlags.BlobRetentionEpochFlag,
flags.BatchVerifierLimit,
flags.StateDiffExponents,
}
func init() {

View File

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

View File

@@ -270,6 +270,7 @@ var BeaconChainFlags = combinedFlags([]cli.Flag{
DisableQUIC,
EnableDiscoveryReboot,
enableExperimentalAttestationPool,
EnableStateDiff,
forceHeadFlag,
blacklistRoots,
}, deprecatedBeaconFlags, deprecatedFlags, upcomingDeprecation)

View File

@@ -95,6 +95,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

@@ -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

@@ -67,6 +67,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
)
}
}
// E2EConfig defines the struct for all configurations needed for E2E testing.
type E2EConfig struct {
TestCheckpointSync bool