mirror of
https://github.com/OffchainLabs/prysm.git
synced 2026-01-09 21:38:05 -05:00
Compare commits
7 Commits
fulu-e2e
...
hdiff_star
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
68ff47db10 | ||
|
|
403d88b136 | ||
|
|
b8a82885a7 | ||
|
|
432e4dfbbf | ||
|
|
a71990bc43 | ||
|
|
cb1d98b7dc | ||
|
|
8e5c982edb |
@@ -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
|
||||
|
||||
@@ -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",
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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:
|
||||
|
||||
35
beacon-chain/db/kv/testing_helpers.go
Normal file
35
beacon-chain/db/kv/testing_helpers.go
Normal 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
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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",
|
||||
|
||||
@@ -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()
|
||||
}
|
||||
|
||||
@@ -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, ðpb.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))
|
||||
}
|
||||
|
||||
3
changelog/potuz_hdiff_migrate_to_cold.md
Normal file
3
changelog/potuz_hdiff_migrate_to_cold.md
Normal file
@@ -0,0 +1,3 @@
|
||||
### Added
|
||||
|
||||
- Migrate to cold with the hdiff feature.
|
||||
@@ -156,6 +156,7 @@ var appFlags = []cli.Flag{
|
||||
dasFlags.BackfillOldestSlot,
|
||||
dasFlags.BlobRetentionEpochFlag,
|
||||
flags.BatchVerifierLimit,
|
||||
flags.StateDiffExponents,
|
||||
}
|
||||
|
||||
func init() {
|
||||
|
||||
@@ -74,6 +74,7 @@ var appHelpFlagGroups = []flagGroup{
|
||||
flags.RPCHost,
|
||||
flags.RPCPort,
|
||||
flags.BatchVerifierLimit,
|
||||
flags.StateDiffExponents,
|
||||
},
|
||||
},
|
||||
{
|
||||
|
||||
@@ -270,6 +270,7 @@ var BeaconChainFlags = combinedFlags([]cli.Flag{
|
||||
DisableQUIC,
|
||||
EnableDiscoveryReboot,
|
||||
enableExperimentalAttestationPool,
|
||||
EnableStateDiff,
|
||||
forceHeadFlag,
|
||||
blacklistRoots,
|
||||
}, deprecatedBeaconFlags, deprecatedFlags, upcomingDeprecation)
|
||||
|
||||
@@ -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",
|
||||
],
|
||||
|
||||
16
testing/endtoend/minimal_hdiff_e2e_test.go
Normal file
16
testing/endtoend/minimal_hdiff_e2e_test.go
Normal 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()
|
||||
}
|
||||
@@ -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
|
||||
|
||||
Reference in New Issue
Block a user