Canonical LC (#15585)

* create lc cache to track branches

* save lc stuff

* remove finalized data from LC cache on finalization

* read lc stuff

* edit tests

* changelog

* linter

* address commments

* address commments 2

* address commments 3

* address commments 4

* lint

* address commments 5 x_x

* set beacon lcStore to mimick registrable services

* clean up the error propagation

* pass the state to saveLCBootstrap since it's not saved in db yet
This commit is contained in:
Bastin
2025-09-16 14:20:07 +02:00
committed by GitHub
parent 238d5c07df
commit 360e89767f
27 changed files with 1454 additions and 283 deletions

View File

@@ -35,7 +35,7 @@ func WithMaxGoroutines(x int) Option {
// WithLCStore for light client store access.
func WithLCStore() Option {
return func(s *Service) error {
s.lcStore = lightclient.NewLightClientStore(s.cfg.BeaconDB, s.cfg.P2P, s.cfg.StateNotifier.StateFeed())
s.lcStore = lightclient.NewLightClientStore(s.cfg.P2P, s.cfg.StateNotifier.StateFeed(), s.cfg.BeaconDB)
return nil
}
}

View File

@@ -3,14 +3,11 @@ package blockchain
import (
"context"
"fmt"
"strings"
"time"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/helpers"
lightclient "github.com/OffchainLabs/prysm/v6/beacon-chain/core/light-client"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed"
statefeed "github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed/state"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/helpers"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/transition"
doublylinkedtree "github.com/OffchainLabs/prysm/v6/beacon-chain/forkchoice/doubly-linked-tree"
forkchoicetypes "github.com/OffchainLabs/prysm/v6/beacon-chain/forkchoice/types"
@@ -133,35 +130,26 @@ func (s *Service) sendStateFeedOnBlock(cfg *postBlockProcessConfig) {
})
}
// processLightClientUpdates saves the light client data in lcStore, when feature flag is enabled.
func (s *Service) processLightClientUpdates(cfg *postBlockProcessConfig) {
if err := s.processLightClientUpdate(cfg); err != nil {
log.WithError(err).Error("Failed to process light client update")
}
if err := s.processLightClientOptimisticUpdate(cfg.ctx, cfg.roblock, cfg.postState); err != nil {
log.WithError(err).Error("Failed to process light client optimistic update")
}
if err := s.processLightClientFinalityUpdate(cfg.ctx, cfg.roblock, cfg.postState); err != nil {
log.WithError(err).Error("Failed to process light client finality update")
}
}
// processLightClientUpdate saves the light client update for this block
// if it's better than the already saved one, when feature flag is enabled.
func (s *Service) processLightClientUpdate(cfg *postBlockProcessConfig) error {
attestedRoot := cfg.roblock.Block().ParentRoot()
attestedBlock, err := s.getBlock(cfg.ctx, attestedRoot)
if err != nil {
return errors.Wrapf(err, "could not get attested block for root %#x", attestedRoot)
log.WithError(err).Error("processLightClientUpdates: Could not get attested block")
return
}
if attestedBlock == nil || attestedBlock.IsNil() {
return errors.New("attested block is nil")
log.Error("processLightClientUpdates: Could not get attested block")
return
}
attestedState, err := s.cfg.StateGen.StateByRoot(cfg.ctx, attestedRoot)
if err != nil {
return errors.Wrapf(err, "could not get attested state for root %#x", attestedRoot)
log.WithError(err).Error("processLightClientUpdates: Could not get attested state")
return
}
if attestedState == nil || attestedState.IsNil() {
return errors.New("attested state is nil")
log.Error("processLightClientUpdates: Could not get attested state")
return
}
finalizedRoot := attestedState.FinalizedCheckpoint().Root
@@ -169,98 +157,17 @@ func (s *Service) processLightClientUpdate(cfg *postBlockProcessConfig) error {
if err != nil {
if errors.Is(err, errBlockNotFoundInCacheOrDB) {
log.Debugf("Skipping saving light client update because finalized block is nil for root %#x", finalizedRoot)
return nil
return
}
return errors.Wrapf(err, "could not get finalized block for root %#x", finalizedRoot)
log.WithError(err).Error("processLightClientUpdates: Could not get finalized block")
return
}
update, err := lightclient.NewLightClientUpdateFromBeaconState(cfg.ctx, cfg.postState, cfg.roblock, attestedState, attestedBlock, finalizedBlock)
err = s.lcStore.SaveLCData(cfg.ctx, cfg.postState, cfg.roblock, attestedState, attestedBlock, finalizedBlock, s.headRoot())
if err != nil {
return errors.Wrapf(err, "could not create light client update")
log.WithError(err).Error("processLightClientUpdates: Could not save light client data")
}
period := slots.SyncCommitteePeriod(slots.ToEpoch(attestedState.Slot()))
return s.lcStore.SaveLightClientUpdate(cfg.ctx, period, update)
}
func (s *Service) processLightClientFinalityUpdate(
ctx context.Context,
signed interfaces.ReadOnlySignedBeaconBlock,
postState state.BeaconState,
) error {
attestedRoot := signed.Block().ParentRoot()
attestedBlock, err := s.cfg.BeaconDB.Block(ctx, attestedRoot)
if err != nil {
return errors.Wrapf(err, "could not get attested block for root %#x", attestedRoot)
}
attestedState, err := s.cfg.StateGen.StateByRoot(ctx, attestedRoot)
if err != nil {
return errors.Wrapf(err, "could not get attested state for root %#x", attestedRoot)
}
finalizedCheckpoint := attestedState.FinalizedCheckpoint()
if finalizedCheckpoint == nil {
return nil
}
finalizedRoot := bytesutil.ToBytes32(finalizedCheckpoint.Root)
finalizedBlock, err := s.cfg.BeaconDB.Block(ctx, finalizedRoot)
if err != nil {
if errors.Is(err, errBlockNotFoundInCacheOrDB) {
log.Debugf("Skipping processing light client finality update: Finalized block is nil for root %#x", finalizedRoot)
return nil
}
return errors.Wrapf(err, "could not get finalized block for root %#x", finalizedRoot)
}
newUpdate, err := lightclient.NewLightClientFinalityUpdateFromBeaconState(ctx, postState, signed, attestedState, attestedBlock, finalizedBlock)
if err != nil {
return errors.Wrap(err, "could not create light client finality update")
}
if !lightclient.IsBetterFinalityUpdate(newUpdate, s.lcStore.LastFinalityUpdate()) {
log.Debug("Skip saving light client finality update: current update is better")
return nil
}
s.lcStore.SetLastFinalityUpdate(newUpdate, true)
return nil
}
func (s *Service) processLightClientOptimisticUpdate(ctx context.Context, signed interfaces.ReadOnlySignedBeaconBlock,
postState state.BeaconState) error {
attestedRoot := signed.Block().ParentRoot()
attestedBlock, err := s.cfg.BeaconDB.Block(ctx, attestedRoot)
if err != nil {
return errors.Wrapf(err, "could not get attested block for root %#x", attestedRoot)
}
attestedState, err := s.cfg.StateGen.StateByRoot(ctx, attestedRoot)
if err != nil {
return errors.Wrapf(err, "could not get attested state for root %#x", attestedRoot)
}
newUpdate, err := lightclient.NewLightClientOptimisticUpdateFromBeaconState(ctx, postState, signed, attestedState, attestedBlock)
if err != nil {
if strings.Contains(err.Error(), lightclient.ErrNotEnoughSyncCommitteeBits) {
log.WithError(err).Debug("Skipping processing light client optimistic update")
return nil
}
return errors.Wrap(err, "could not create light client optimistic update")
}
if !lightclient.IsBetterOptimisticUpdate(newUpdate, s.lcStore.LastOptimisticUpdate()) {
log.Debug("Skip saving light client optimistic update: current update is better")
return nil
}
s.lcStore.SetLastOptimisticUpdate(newUpdate, true)
return nil
log.Debug("Processed light client updates")
}
// updateCachesPostBlockProcessing updates the next slot cache and handles the epoch

View File

@@ -2795,6 +2795,11 @@ func TestProcessLightClientUpdate(t *testing.T) {
s, tr := minimalTestService(t, WithLCStore())
ctx := tr.ctx
headState, err := util.NewBeaconState()
require.NoError(t, err)
require.NoError(t, s.cfg.BeaconDB.SaveState(ctx, headState, [32]byte{1, 2}))
require.NoError(t, s.cfg.BeaconDB.SaveHeadBlockRoot(ctx, [32]byte{1, 2}))
for testVersion := version.Altair; testVersion <= version.Electra; testVersion++ {
t.Run(version.String(testVersion), func(t *testing.T) {
l := util.NewTestLightClient(t, testVersion)
@@ -2817,6 +2822,8 @@ func TestProcessLightClientUpdate(t *testing.T) {
require.NoError(t, err)
err = s.cfg.BeaconDB.SaveState(ctx, l.State, currentBlockRoot)
require.NoError(t, err)
err = s.cfg.BeaconDB.SaveHeadBlockRoot(ctx, currentBlockRoot)
require.NoError(t, err)
err = s.cfg.BeaconDB.SaveBlock(ctx, l.FinalizedBlock)
require.NoError(t, err)
@@ -2831,10 +2838,9 @@ func TestProcessLightClientUpdate(t *testing.T) {
period := slots.SyncCommitteePeriod(slots.ToEpoch(l.AttestedState.Slot()))
t.Run("no old update", func(t *testing.T) {
require.NoError(t, s.processLightClientUpdate(cfg))
s.processLightClientUpdates(cfg)
// Check that the light client update is saved
u, err := s.lcStore.LightClientUpdate(ctx, period)
u, err := s.lcStore.LightClientUpdate(ctx, period, l.Block)
require.NoError(t, err)
require.NotNil(t, u)
attestedStateRoot, err := l.AttestedState.HashTreeRoot(ctx)
@@ -2848,12 +2854,12 @@ func TestProcessLightClientUpdate(t *testing.T) {
oldUpdate, err := lightClient.CreateDefaultLightClientUpdate(l.AttestedBlock)
require.NoError(t, err)
err = s.lcStore.SaveLightClientUpdate(ctx, period, oldUpdate)
err = s.cfg.BeaconDB.SaveLightClientUpdate(ctx, period, oldUpdate)
require.NoError(t, err)
require.NoError(t, s.processLightClientUpdate(cfg))
s.processLightClientUpdates(cfg)
u, err := s.lcStore.LightClientUpdate(ctx, period)
u, err := s.lcStore.LightClientUpdate(ctx, period, l.Block)
require.NoError(t, err)
require.NotNil(t, u)
attestedStateRoot, err := l.AttestedState.HashTreeRoot(ctx)
@@ -2877,12 +2883,12 @@ func TestProcessLightClientUpdate(t *testing.T) {
SyncCommitteeSignature: make([]byte, 96),
})
err = s.lcStore.SaveLightClientUpdate(ctx, period, oldUpdate)
err = s.cfg.BeaconDB.SaveLightClientUpdate(ctx, period, oldUpdate)
require.NoError(t, err)
require.NoError(t, s.processLightClientUpdate(cfg))
s.processLightClientUpdates(cfg)
u, err := s.lcStore.LightClientUpdate(ctx, period)
u, err := s.lcStore.LightClientUpdate(ctx, period, l.Block)
require.NoError(t, err)
require.NotNil(t, u)
require.DeepEqual(t, oldUpdate, u)
@@ -3190,6 +3196,11 @@ func TestProcessLightClientOptimisticUpdate(t *testing.T) {
s.cfg.P2P = &mockp2p.FakeP2P{}
ctx := tr.ctx
headState, err := util.NewBeaconState()
require.NoError(t, err)
require.NoError(t, s.cfg.BeaconDB.SaveState(ctx, headState, [32]byte{1, 2}))
require.NoError(t, s.cfg.BeaconDB.SaveHeadBlockRoot(ctx, [32]byte{1, 2}))
testCases := []struct {
name string
oldOptions []util.LightClientOption
@@ -3205,7 +3216,7 @@ func TestProcessLightClientOptimisticUpdate(t *testing.T) {
{
name: "Same age",
oldOptions: []util.LightClientOption{},
newOptions: []util.LightClientOption{util.WithSupermajority()}, // supermajority does not matter here and is only added to result in two different updates
newOptions: []util.LightClientOption{util.WithSupermajority(0)}, // supermajority does not matter here and is only added to result in two different updates
expectReplace: false,
},
{
@@ -3249,14 +3260,14 @@ func TestProcessLightClientOptimisticUpdate(t *testing.T) {
t.Run(version.String(testVersion)+"_"+tc.name, func(t *testing.T) {
s.genesisTime = time.Unix(time.Now().Unix()-(int64(forkEpoch)*int64(params.BeaconConfig().SlotsPerEpoch)*int64(params.BeaconConfig().SecondsPerSlot)), 0)
s.lcStore = lightClient.NewLightClientStore(s.cfg.BeaconDB, s.cfg.P2P, s.cfg.StateNotifier.StateFeed())
s.lcStore = lightClient.NewLightClientStore(s.cfg.P2P, s.cfg.StateNotifier.StateFeed(), s.cfg.BeaconDB)
var oldActualUpdate interfaces.LightClientOptimisticUpdate
var err error
if tc.oldOptions != nil {
// config for old update
lOld, cfgOld := setupLightClientTestRequirements(ctx, t, s, testVersion, tc.oldOptions...)
require.NoError(t, s.processLightClientOptimisticUpdate(cfgOld.ctx, cfgOld.roblock, cfgOld.postState))
s.processLightClientUpdates(cfgOld)
oldActualUpdate, err = lightClient.NewLightClientOptimisticUpdateFromBeaconState(lOld.Ctx, lOld.State, lOld.Block, lOld.AttestedState, lOld.AttestedBlock)
require.NoError(t, err)
@@ -3270,7 +3281,7 @@ func TestProcessLightClientOptimisticUpdate(t *testing.T) {
// config for new update
lNew, cfgNew := setupLightClientTestRequirements(ctx, t, s, testVersion, tc.newOptions...)
require.NoError(t, s.processLightClientOptimisticUpdate(cfgNew.ctx, cfgNew.roblock, cfgNew.postState))
s.processLightClientUpdates(cfgNew)
newActualUpdate, err := lightClient.NewLightClientOptimisticUpdateFromBeaconState(lNew.Ctx, lNew.State, lNew.Block, lNew.AttestedState, lNew.AttestedBlock)
require.NoError(t, err)
@@ -3311,6 +3322,7 @@ func TestProcessLightClientFinalityUpdate(t *testing.T) {
s, tr := minimalTestService(t)
s.cfg.P2P = &mockp2p.FakeP2P{}
ctx := tr.ctx
s.head = &head{}
testCases := []struct {
name string
@@ -3389,15 +3401,18 @@ func TestProcessLightClientFinalityUpdate(t *testing.T) {
t.Run(version.String(testVersion)+"_"+tc.name, func(t *testing.T) {
s.genesisTime = time.Unix(time.Now().Unix()-(int64(forkEpoch)*int64(params.BeaconConfig().SlotsPerEpoch)*int64(params.BeaconConfig().SecondsPerSlot)), 0)
s.lcStore = lightClient.NewLightClientStore(s.cfg.BeaconDB, s.cfg.P2P, s.cfg.StateNotifier.StateFeed())
s.lcStore = lightClient.NewLightClientStore(s.cfg.P2P, s.cfg.StateNotifier.StateFeed(), s.cfg.BeaconDB)
var actualOldUpdate, actualNewUpdate interfaces.LightClientFinalityUpdate
var err error
if tc.oldOptions != nil {
// config for old update
lOld, cfgOld := setupLightClientTestRequirements(ctx, t, s, testVersion, tc.oldOptions...)
require.NoError(t, s.processLightClientFinalityUpdate(cfgOld.ctx, cfgOld.roblock, cfgOld.postState))
blkRoot, err := lOld.Block.Block().HashTreeRoot()
require.NoError(t, err)
s.head.block = lOld.Block
s.head.root = blkRoot
s.processLightClientUpdates(cfgOld)
// check that the old update is saved
actualOldUpdate, err = lightClient.NewLightClientFinalityUpdateFromBeaconState(ctx, cfgOld.postState, cfgOld.roblock, lOld.AttestedState, lOld.AttestedBlock, lOld.FinalizedBlock)
@@ -3408,7 +3423,11 @@ func TestProcessLightClientFinalityUpdate(t *testing.T) {
// config for new update
lNew, cfgNew := setupLightClientTestRequirements(ctx, t, s, testVersion, tc.newOptions...)
require.NoError(t, s.processLightClientFinalityUpdate(cfgNew.ctx, cfgNew.roblock, cfgNew.postState))
blkRoot, err := lNew.Block.Block().HashTreeRoot()
require.NoError(t, err)
s.head.block = lNew.Block
s.head.root = blkRoot
s.processLightClientUpdates(cfgNew)
// check that the actual old update and the actual new update are different
actualNewUpdate, err = lightClient.NewLightClientFinalityUpdateFromBeaconState(ctx, cfgNew.postState, cfgNew.roblock, lNew.AttestedState, lNew.AttestedBlock, lNew.FinalizedBlock)

View File

@@ -320,13 +320,28 @@ func (s *Service) executePostFinalizationTasks(ctx context.Context, finalizedSta
if features.Get().EnableLightClient {
// Save a light client bootstrap for the finalized checkpoint
go func() {
err := s.lcStore.SaveLightClientBootstrap(s.ctx, finalized.Root)
st, err := s.cfg.StateGen.StateByRoot(ctx, finalized.Root)
if err != nil {
log.WithError(err).Error("Could not retrieve state for finalized root to save light client bootstrap")
return
}
err = s.lcStore.SaveLightClientBootstrap(s.ctx, finalized.Root, st)
if err != nil {
log.WithError(err).Error("Could not save light client bootstrap by block root")
} else {
log.Debugf("Saved light client bootstrap for finalized root %#x", finalized.Root)
}
}()
// Clean up the light client store caches
go func() {
err := s.lcStore.MigrateToCold(s.ctx, finalized.Root)
if err != nil {
log.WithError(err).Error("Could not migrate light client store to cold storage")
} else {
log.Debugf("Migrated light client store to cold storage for finalized root %#x", finalized.Root)
}
}()
}
}

View File

@@ -594,11 +594,7 @@ func TestProcessLightClientBootstrap(t *testing.T) {
require.NoError(t, s.cfg.ForkChoiceStore.UpdateFinalizedCheckpoint(&forkchoicetypes.Checkpoint{Epoch: cp.Epoch, Root: [32]byte(cp.Root)}))
sss, err := s.cfg.BeaconDB.State(ctx, finalizedBlockRoot)
require.NoError(t, err)
require.NotNil(t, sss)
s.executePostFinalizationTasks(s.ctx, l.FinalizedState)
s.executePostFinalizationTasks(s.ctx, l.AttestedState)
// wait for the goroutine to finish processing
time.Sleep(1 * time.Second)

View File

@@ -3,6 +3,7 @@ load("@prysm//tools/go:def.bzl", "go_library", "go_test")
go_library(
name = "go_default_library",
srcs = [
"cache.go",
"helpers.go",
"lightclient.go",
"store.go",
@@ -38,25 +39,30 @@ go_library(
go_test(
name = "go_default_test",
srcs = [
"cache_test.go",
"lightclient_test.go",
"store_test.go",
],
embed = [":go_default_library"],
deps = [
":go_default_library",
"//async/event:go_default_library",
"//beacon-chain/db:go_default_library",
"//beacon-chain/db/testing:go_default_library",
"//beacon-chain/p2p/testing: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/light-client:go_default_library",
"//consensus-types/primitives:go_default_library",
"//encoding/ssz:go_default_library",
"//proto/engine/v1:go_default_library",
"//proto/prysm/v1alpha1: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_pkg_errors//:go_default_library",
],
)

View File

@@ -0,0 +1,26 @@
package light_client
import (
"github.com/OffchainLabs/prysm/v6/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
)
// cache tracks LC data over the non finalized chain for different branches.
type cache struct {
items map[[32]byte]*cacheItem
}
// cacheItem represents the LC data for a block. It tracks the best update and finality update seen in that branch.
type cacheItem struct {
parent *cacheItem // parent item in the cache, can be nil
period uint64 // sync committee period
slot primitives.Slot // slot of the signature block
bestUpdate interfaces.LightClientUpdate
bestFinalityUpdate interfaces.LightClientFinalityUpdate
}
func newLightClientCache() *cache {
return &cache{
items: make(map[[32]byte]*cacheItem),
}
}

View File

@@ -0,0 +1,24 @@
package light_client
import (
"testing"
"github.com/OffchainLabs/prysm/v6/testing/require"
)
func TestLCCache(t *testing.T) {
lcCache := newLightClientCache()
require.NotNil(t, lcCache)
item := &cacheItem{
period: 5,
bestUpdate: nil,
bestFinalityUpdate: nil,
}
blkRoot := [32]byte{4, 5, 6}
lcCache.items[blkRoot] = item
require.Equal(t, item, lcCache.items[blkRoot], "Expected to find the item in the cache")
}

View File

@@ -592,6 +592,10 @@ func HasFinality(update interfaces.LightClientUpdate) (bool, error) {
}
func IsBetterUpdate(newUpdate, oldUpdate interfaces.LightClientUpdate) (bool, error) {
if oldUpdate == nil || oldUpdate.IsNil() {
return true, nil
}
maxActiveParticipants := newUpdate.SyncAggregate().SyncCommitteeBits.Len()
newNumActiveParticipants := newUpdate.SyncAggregate().SyncCommitteeBits.Count()
oldNumActiveParticipants := oldUpdate.SyncAggregate().SyncCommitteeBits.Count()
@@ -778,7 +782,7 @@ func IsFinalityUpdateValidForBroadcast(newUpdate, oldUpdate interfaces.LightClie
// This does not concern broadcasting, but rather the decision of whether to save the new update.
// For broadcasting checks, use IsFinalityUpdateValidForBroadcast.
func IsBetterFinalityUpdate(newUpdate, oldUpdate interfaces.LightClientFinalityUpdate) bool {
if oldUpdate == nil {
if oldUpdate == nil || oldUpdate.IsNil() {
return true
}
@@ -804,7 +808,7 @@ func IsBetterFinalityUpdate(newUpdate, oldUpdate interfaces.LightClientFinalityU
}
func IsBetterOptimisticUpdate(newUpdate, oldUpdate interfaces.LightClientOptimisticUpdate) bool {
if oldUpdate == nil {
if oldUpdate == nil || oldUpdate.IsNil() {
return true
}
// The attested_header.beacon.slot is greater than that of all previously forwarded optimistic updates

View File

@@ -9,7 +9,10 @@ import (
statefeed "github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed/state"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db/iface"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p"
"github.com/OffchainLabs/prysm/v6/beacon-chain/state"
"github.com/OffchainLabs/prysm/v6/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v6/time/slots"
"github.com/pkg/errors"
log "github.com/sirupsen/logrus"
)
@@ -24,16 +27,127 @@ type Store struct {
lastOptimisticUpdate interfaces.LightClientOptimisticUpdate // tracks the best optimistic update seen so far
p2p p2p.Accessor
stateFeed event.SubscriberSender
cache *cache // non finality cache
}
func NewLightClientStore(db iface.HeadAccessDatabase, p p2p.Accessor, e event.SubscriberSender) *Store {
func NewLightClientStore(p p2p.Accessor, e event.SubscriberSender, db iface.HeadAccessDatabase) *Store {
return &Store{
beaconDB: db,
p2p: p,
stateFeed: e,
cache: newLightClientCache(),
}
}
func (s *Store) SaveLCData(ctx context.Context,
state state.BeaconState,
block interfaces.ReadOnlySignedBeaconBlock,
attestedState state.BeaconState,
attestedBlock interfaces.ReadOnlySignedBeaconBlock,
finalizedBlock interfaces.ReadOnlySignedBeaconBlock,
headBlockRoot [32]byte) error {
s.mu.Lock()
defer s.mu.Unlock()
// compute required data
update, err := NewLightClientUpdateFromBeaconState(ctx, state, block, attestedState, attestedBlock, finalizedBlock)
if err != nil {
return errors.Wrapf(err, "failed to create light client update")
}
finalityUpdate, err := NewLightClientFinalityUpdateFromBeaconState(ctx, state, block, attestedState, attestedBlock, finalizedBlock)
if err != nil {
return errors.Wrapf(err, "failed to create light client finality update")
}
optimisticUpdate, err := NewLightClientOptimisticUpdateFromBeaconState(ctx, state, block, attestedState, attestedBlock)
if err != nil {
return errors.Wrapf(err, "failed to create light client optimistic update")
}
period := slots.SyncCommitteePeriod(slots.ToEpoch(update.AttestedHeader().Beacon().Slot))
blockRoot, err := attestedBlock.Block().HashTreeRoot()
if err != nil {
return errors.Wrapf(err, "failed to compute attested block root")
}
parentRoot := [32]byte(update.AttestedHeader().Beacon().ParentRoot)
signatureBlockRoot, err := block.Block().HashTreeRoot()
if err != nil {
return errors.Wrapf(err, "failed to compute signature block root")
}
newBlockIsHead := signatureBlockRoot == headBlockRoot
// create the new cache item
newCacheItem := &cacheItem{
period: period,
slot: attestedBlock.Block().Slot(),
}
// check if parent exists in cache
parentItem, ok := s.cache.items[parentRoot]
if ok {
newCacheItem.parent = parentItem
} else {
// if not, create an item for the parent, but don't need to save it since it's the accumulated best update and is just used for comparison
bestUpdateSoFar, err := s.beaconDB.LightClientUpdate(ctx, period)
if err != nil {
return errors.Wrapf(err, "could not get best light client update for period %d", period)
}
parentItem = &cacheItem{
period: period,
bestUpdate: bestUpdateSoFar,
bestFinalityUpdate: s.lastFinalityUpdate,
}
}
// if at a period boundary, no need to compare data, just save new ones
if parentItem.period != period {
newCacheItem.bestUpdate = update
newCacheItem.bestFinalityUpdate = finalityUpdate
s.cache.items[blockRoot] = newCacheItem
s.setLastOptimisticUpdate(optimisticUpdate, true)
// if the new block is not head, we don't want to change our lastFinalityUpdate
if newBlockIsHead {
s.setLastFinalityUpdate(finalityUpdate, true)
}
return nil
}
// if in the same period, compare updates
isUpdateBetter, err := IsBetterUpdate(update, parentItem.bestUpdate)
if err != nil {
return errors.Wrapf(err, "could not compare light client updates")
}
if isUpdateBetter {
newCacheItem.bestUpdate = update
} else {
newCacheItem.bestUpdate = parentItem.bestUpdate
}
isBetterFinalityUpdate := IsBetterFinalityUpdate(finalityUpdate, parentItem.bestFinalityUpdate)
if isBetterFinalityUpdate {
newCacheItem.bestFinalityUpdate = finalityUpdate
} else {
newCacheItem.bestFinalityUpdate = parentItem.bestFinalityUpdate
}
// save new item in cache
s.cache.items[blockRoot] = newCacheItem
// save lastOptimisticUpdate if better
if isBetterOptimisticUpdate := IsBetterOptimisticUpdate(optimisticUpdate, s.lastOptimisticUpdate); isBetterOptimisticUpdate {
s.setLastOptimisticUpdate(optimisticUpdate, true)
}
// if the new block is considered the head, set the last finality update
if newBlockIsHead {
s.setLastFinalityUpdate(newCacheItem.bestFinalityUpdate, isBetterFinalityUpdate)
}
return nil
}
func (s *Store) LightClientBootstrap(ctx context.Context, blockRoot [32]byte) (interfaces.LightClientBootstrap, error) {
s.mu.RLock()
defer s.mu.RUnlock()
@@ -50,7 +164,7 @@ func (s *Store) LightClientBootstrap(ctx context.Context, blockRoot [32]byte) (i
return bootstrap, nil
}
func (s *Store) SaveLightClientBootstrap(ctx context.Context, blockRoot [32]byte) error {
func (s *Store) SaveLightClientBootstrap(ctx context.Context, blockRoot [32]byte, state state.BeaconState) error {
s.mu.Lock()
defer s.mu.Unlock()
@@ -59,15 +173,7 @@ func (s *Store) SaveLightClientBootstrap(ctx context.Context, blockRoot [32]byte
return errors.Wrapf(err, "failed to fetch block for root %x", blockRoot)
}
if blk == nil {
return errors.Errorf("failed to fetch block for root %x", blockRoot)
}
state, err := s.beaconDB.State(ctx, blockRoot)
if err != nil {
return errors.Wrapf(err, "failed to fetch state for block root %x", blockRoot)
}
if state == nil {
return errors.Errorf("failed to fetch state for block root %x", blockRoot)
return errors.Errorf("nil block for root %x", blockRoot)
}
bootstrap, err := NewLightClientBootstrapFromBeaconState(ctx, state.Slot(), state, blk)
@@ -82,17 +188,27 @@ func (s *Store) SaveLightClientBootstrap(ctx context.Context, blockRoot [32]byte
return nil
}
func (s *Store) LightClientUpdates(ctx context.Context, startPeriod, endPeriod uint64) ([]interfaces.LightClientUpdate, error) {
func (s *Store) LightClientUpdates(ctx context.Context, startPeriod, endPeriod uint64, headBlock interfaces.ReadOnlySignedBeaconBlock) ([]interfaces.LightClientUpdate, error) {
s.mu.RLock()
defer s.mu.RUnlock()
// Fetch the light client updatesMap from the database
updatesMap, err := s.beaconDB.LightClientUpdates(ctx, startPeriod, endPeriod)
if err != nil {
return nil, err
return nil, errors.Wrapf(err, "failed to get updates from the database")
}
cacheUpdatesByPeriod, err := s.getCacheUpdatesByPeriod(headBlock)
if err != nil {
return nil, errors.Wrapf(err, "failed to get updates from cache")
}
for period, update := range cacheUpdatesByPeriod {
updatesMap[period] = update
}
var updates []interfaces.LightClientUpdate
for i := startPeriod; i <= endPeriod; i++ {
update, ok := updatesMap[i]
if !ok {
@@ -105,56 +221,47 @@ func (s *Store) LightClientUpdates(ctx context.Context, startPeriod, endPeriod u
return updates, nil
}
func (s *Store) LightClientUpdate(ctx context.Context, period uint64) (interfaces.LightClientUpdate, error) {
s.mu.RLock()
defer s.mu.RUnlock()
// Fetch the light client update for the given period from the database
update, err := s.beaconDB.LightClientUpdate(ctx, period)
func (s *Store) LightClientUpdate(ctx context.Context, period uint64, headBlock interfaces.ReadOnlySignedBeaconBlock) (interfaces.LightClientUpdate, error) {
// we don't need to lock here because the LightClientUpdates method locks the store
updates, err := s.LightClientUpdates(ctx, period, period, headBlock)
if err != nil {
return nil, err
return nil, errors.Wrapf(err, "failed to get light client update for period %d", period)
}
return update, nil
if len(updates) == 0 {
return nil, nil
}
return updates[0], nil
}
func (s *Store) SaveLightClientUpdate(ctx context.Context, period uint64, update interfaces.LightClientUpdate) error {
s.mu.Lock()
defer s.mu.Unlock()
func (s *Store) getCacheUpdatesByPeriod(headBlock interfaces.ReadOnlySignedBeaconBlock) (map[uint64]interfaces.LightClientUpdate, error) {
updatesByPeriod := make(map[uint64]interfaces.LightClientUpdate)
oldUpdate, err := s.beaconDB.LightClientUpdate(ctx, period)
if err != nil {
return errors.Wrapf(err, "could not get current light client update")
cacheHeadRoot := headBlock.Block().ParentRoot()
cacheHeadItem, ok := s.cache.items[cacheHeadRoot]
if !ok {
log.Debugf("Head root %x not found in light client cache. Returning empty updates map for non finality cache.", cacheHeadRoot)
return updatesByPeriod, nil
}
if oldUpdate == nil {
if err := s.beaconDB.SaveLightClientUpdate(ctx, period, update); err != nil {
return errors.Wrapf(err, "could not save light client update")
for cacheHeadItem != nil {
if _, exists := updatesByPeriod[cacheHeadItem.period]; !exists {
updatesByPeriod[cacheHeadItem.period] = cacheHeadItem.bestUpdate
}
log.WithField("period", period).Debug("Saved new light client update")
return nil
cacheHeadItem = cacheHeadItem.parent
}
isNewUpdateBetter, err := IsBetterUpdate(update, oldUpdate)
if err != nil {
return errors.Wrapf(err, "could not compare light client updates")
}
if isNewUpdateBetter {
if err := s.beaconDB.SaveLightClientUpdate(ctx, period, update); err != nil {
return errors.Wrapf(err, "could not save light client update")
}
log.WithField("period", period).Debug("Saved new light client update")
return nil
}
log.WithField("period", period).Debug("New light client update is not better than the current one, skipping save")
return nil
return updatesByPeriod, nil
}
func (s *Store) SetLastFinalityUpdate(update interfaces.LightClientFinalityUpdate, broadcast bool) {
s.mu.Lock()
defer s.mu.Unlock()
s.setLastFinalityUpdate(update, broadcast)
}
func (s *Store) setLastFinalityUpdate(update interfaces.LightClientFinalityUpdate, broadcast bool) {
if broadcast && IsFinalityUpdateValidForBroadcast(update, s.lastFinalityUpdate) {
if err := s.p2p.BroadcastLightClientFinalityUpdate(context.Background(), update); err != nil {
log.WithError(err).Error("Could not broadcast light client finality update")
@@ -180,6 +287,10 @@ func (s *Store) SetLastOptimisticUpdate(update interfaces.LightClientOptimisticU
s.mu.Lock()
defer s.mu.Unlock()
s.setLastOptimisticUpdate(update, broadcast)
}
func (s *Store) setLastOptimisticUpdate(update interfaces.LightClientOptimisticUpdate, broadcast bool) {
if broadcast {
if err := s.p2p.BroadcastLightClientOptimisticUpdate(context.Background(), update); err != nil {
log.WithError(err).Error("Could not broadcast light client optimistic update")
@@ -200,3 +311,73 @@ func (s *Store) LastOptimisticUpdate() interfaces.LightClientOptimisticUpdate {
defer s.mu.RUnlock()
return s.lastOptimisticUpdate
}
func (s *Store) MigrateToCold(ctx context.Context, finalizedRoot [32]byte) error {
s.mu.Lock()
defer s.mu.Unlock()
// If there cache is empty (some problem in processing data), we can skip migration.
// This is a safety check and should not happen in normal operation.
if len(s.cache.items) == 0 {
log.Debug("Non-finality cache is empty. Skipping migration.")
return nil
}
blk, err := s.beaconDB.Block(ctx, finalizedRoot)
if err != nil {
return errors.Wrapf(err, "failed to fetch block for finalized root %x", finalizedRoot)
}
if blk == nil {
return errors.Errorf("nil block for finalized root %x", finalizedRoot)
}
finalizedSlot := blk.Block().Slot()
finalizedCacheHeadRoot := blk.Block().ParentRoot()
var finalizedCacheHead *cacheItem
var ok bool
finalizedCacheHead, ok = s.cache.items[finalizedCacheHeadRoot]
if !ok {
log.Debugf("Finalized block's parent root %x not found in light client cache. Cleaning the broken part of the cache.", finalizedCacheHeadRoot)
// delete non-finality cache items older than finalized slot
s.cleanCache(finalizedSlot)
return nil
}
updateByPeriod := make(map[uint64]interfaces.LightClientUpdate)
// Traverse the cache from the head item to the tail, collecting updates
for item := finalizedCacheHead; item != nil; item = item.parent {
if _, seen := updateByPeriod[item.period]; seen {
// We already have an update for this period, skip this item
continue
}
updateByPeriod[item.period] = item.bestUpdate
}
// save updates to db
for period, update := range updateByPeriod {
err = s.beaconDB.SaveLightClientUpdate(ctx, period, update)
if err != nil {
log.WithError(err).Errorf("failed to save light client update for period %d. Skipping this period.", period)
}
}
// delete non-finality cache items older than finalized slot
s.cleanCache(finalizedSlot)
return nil
}
func (s *Store) cleanCache(finalizedSlot primitives.Slot) {
// delete non-finality cache items older than finalized slot
for k, v := range s.cache.items {
if v.slot < finalizedSlot {
delete(s.cache.items, k)
}
if v.parent != nil && v.parent.slot < finalizedSlot {
v.parent = nil // remove parent reference
}
}
}

View File

@@ -1,16 +1,22 @@
package light_client_test
package light_client
import (
"context"
"testing"
"github.com/OffchainLabs/prysm/v6/async/event"
lightClient "github.com/OffchainLabs/prysm/v6/beacon-chain/core/light-client"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db"
testDB "github.com/OffchainLabs/prysm/v6/beacon-chain/db/testing"
p2pTesting "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/testing"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v6/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
ethpb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v6/runtime/version"
"github.com/OffchainLabs/prysm/v6/testing/require"
"github.com/OffchainLabs/prysm/v6/testing/util"
"github.com/OffchainLabs/prysm/v6/time/slots"
)
func TestLightClientStore(t *testing.T) {
@@ -24,22 +30,22 @@ func TestLightClientStore(t *testing.T) {
params.OverrideBeaconConfig(cfg)
// Initialize the light client store
lcStore := lightClient.NewLightClientStore(testDB.SetupDB(t), &p2pTesting.FakeP2P{}, new(event.Feed))
lcStore := NewLightClientStore(&p2pTesting.FakeP2P{}, new(event.Feed), testDB.SetupDB(t))
// Create test light client updates for Capella and Deneb
lCapella := util.NewTestLightClient(t, version.Capella)
opUpdateCapella, err := lightClient.NewLightClientOptimisticUpdateFromBeaconState(lCapella.Ctx, lCapella.State, lCapella.Block, lCapella.AttestedState, lCapella.AttestedBlock)
opUpdateCapella, err := NewLightClientOptimisticUpdateFromBeaconState(lCapella.Ctx, lCapella.State, lCapella.Block, lCapella.AttestedState, lCapella.AttestedBlock)
require.NoError(t, err)
require.NotNil(t, opUpdateCapella, "OptimisticUpdateCapella is nil")
finUpdateCapella, err := lightClient.NewLightClientFinalityUpdateFromBeaconState(lCapella.Ctx, lCapella.State, lCapella.Block, lCapella.AttestedState, lCapella.AttestedBlock, lCapella.FinalizedBlock)
finUpdateCapella, err := NewLightClientFinalityUpdateFromBeaconState(lCapella.Ctx, lCapella.State, lCapella.Block, lCapella.AttestedState, lCapella.AttestedBlock, lCapella.FinalizedBlock)
require.NoError(t, err)
require.NotNil(t, finUpdateCapella, "FinalityUpdateCapella is nil")
lDeneb := util.NewTestLightClient(t, version.Deneb)
opUpdateDeneb, err := lightClient.NewLightClientOptimisticUpdateFromBeaconState(lDeneb.Ctx, lDeneb.State, lDeneb.Block, lDeneb.AttestedState, lDeneb.AttestedBlock)
opUpdateDeneb, err := NewLightClientOptimisticUpdateFromBeaconState(lDeneb.Ctx, lDeneb.State, lDeneb.Block, lDeneb.AttestedState, lDeneb.AttestedBlock)
require.NoError(t, err)
require.NotNil(t, opUpdateDeneb, "OptimisticUpdateDeneb is nil")
finUpdateDeneb, err := lightClient.NewLightClientFinalityUpdateFromBeaconState(lDeneb.Ctx, lDeneb.State, lDeneb.Block, lDeneb.AttestedState, lDeneb.AttestedBlock, lDeneb.FinalizedBlock)
finUpdateDeneb, err := NewLightClientFinalityUpdateFromBeaconState(lDeneb.Ctx, lDeneb.State, lDeneb.Block, lDeneb.AttestedState, lDeneb.AttestedBlock, lDeneb.FinalizedBlock)
require.NoError(t, err)
require.NotNil(t, finUpdateDeneb, "FinalityUpdateDeneb is nil")
@@ -66,16 +72,16 @@ func TestLightClientStore(t *testing.T) {
func TestLightClientStore_SetLastFinalityUpdate(t *testing.T) {
p2p := p2pTesting.NewTestP2P(t)
lcStore := lightClient.NewLightClientStore(testDB.SetupDB(t), p2p, new(event.Feed))
lcStore := NewLightClientStore(p2p, new(event.Feed), testDB.SetupDB(t))
// update 0 with basic data and no supermajority following an empty lastFinalityUpdate - should save and broadcast
l0 := util.NewTestLightClient(t, version.Altair)
update0, err := lightClient.NewLightClientFinalityUpdateFromBeaconState(l0.Ctx, l0.State, l0.Block, l0.AttestedState, l0.AttestedBlock, l0.FinalizedBlock)
update0, err := NewLightClientFinalityUpdateFromBeaconState(l0.Ctx, l0.State, l0.Block, l0.AttestedState, l0.AttestedBlock, l0.FinalizedBlock)
require.NoError(t, err, "Failed to create light client finality update")
require.Equal(t, true, lightClient.IsBetterFinalityUpdate(update0, lcStore.LastFinalityUpdate()), "update0 should be better than nil")
require.Equal(t, true, IsBetterFinalityUpdate(update0, lcStore.LastFinalityUpdate()), "update0 should be better than nil")
// update0 should be valid for broadcast - meaning it should be broadcasted
require.Equal(t, true, lightClient.IsFinalityUpdateValidForBroadcast(update0, lcStore.LastFinalityUpdate()), "update0 should be valid for broadcast")
require.Equal(t, true, IsFinalityUpdateValidForBroadcast(update0, lcStore.LastFinalityUpdate()), "update0 should be valid for broadcast")
lcStore.SetLastFinalityUpdate(update0, true)
require.Equal(t, update0, lcStore.LastFinalityUpdate(), "lastFinalityUpdate should match the set value")
@@ -84,12 +90,12 @@ func TestLightClientStore_SetLastFinalityUpdate(t *testing.T) {
// update 1 with same finality slot, increased attested slot, and no supermajority - should save but not broadcast
l1 := util.NewTestLightClient(t, version.Altair, util.WithIncreasedAttestedSlot(1))
update1, err := lightClient.NewLightClientFinalityUpdateFromBeaconState(l1.Ctx, l1.State, l1.Block, l1.AttestedState, l1.AttestedBlock, l1.FinalizedBlock)
update1, err := NewLightClientFinalityUpdateFromBeaconState(l1.Ctx, l1.State, l1.Block, l1.AttestedState, l1.AttestedBlock, l1.FinalizedBlock)
require.NoError(t, err, "Failed to create light client finality update")
require.Equal(t, true, lightClient.IsBetterFinalityUpdate(update1, update0), "update1 should be better than update0")
require.Equal(t, true, IsBetterFinalityUpdate(update1, update0), "update1 should be better than update0")
// update1 should not be valid for broadcast - meaning it should not be broadcasted
require.Equal(t, false, lightClient.IsFinalityUpdateValidForBroadcast(update1, lcStore.LastFinalityUpdate()), "update1 should not be valid for broadcast")
require.Equal(t, false, IsFinalityUpdateValidForBroadcast(update1, lcStore.LastFinalityUpdate()), "update1 should not be valid for broadcast")
lcStore.SetLastFinalityUpdate(update1, true)
require.Equal(t, update1, lcStore.LastFinalityUpdate(), "lastFinalityUpdate should match the set value")
@@ -97,13 +103,13 @@ func TestLightClientStore_SetLastFinalityUpdate(t *testing.T) {
p2p.BroadcastCalled.Store(false) // Reset for next test
// update 2 with same finality slot, increased attested slot, and supermajority - should save and broadcast
l2 := util.NewTestLightClient(t, version.Altair, util.WithIncreasedAttestedSlot(2), util.WithSupermajority())
update2, err := lightClient.NewLightClientFinalityUpdateFromBeaconState(l2.Ctx, l2.State, l2.Block, l2.AttestedState, l2.AttestedBlock, l2.FinalizedBlock)
l2 := util.NewTestLightClient(t, version.Altair, util.WithIncreasedAttestedSlot(2), util.WithSupermajority(0))
update2, err := NewLightClientFinalityUpdateFromBeaconState(l2.Ctx, l2.State, l2.Block, l2.AttestedState, l2.AttestedBlock, l2.FinalizedBlock)
require.NoError(t, err, "Failed to create light client finality update")
require.Equal(t, true, lightClient.IsBetterFinalityUpdate(update2, update1), "update2 should be better than update1")
require.Equal(t, true, IsBetterFinalityUpdate(update2, update1), "update2 should be better than update1")
// update2 should be valid for broadcast - meaning it should be broadcasted
require.Equal(t, true, lightClient.IsFinalityUpdateValidForBroadcast(update2, lcStore.LastFinalityUpdate()), "update2 should be valid for broadcast")
require.Equal(t, true, IsFinalityUpdateValidForBroadcast(update2, lcStore.LastFinalityUpdate()), "update2 should be valid for broadcast")
lcStore.SetLastFinalityUpdate(update2, true)
require.Equal(t, update2, lcStore.LastFinalityUpdate(), "lastFinalityUpdate should match the set value")
@@ -111,26 +117,26 @@ func TestLightClientStore_SetLastFinalityUpdate(t *testing.T) {
p2p.BroadcastCalled.Store(false) // Reset for next test
// update 3 with same finality slot, increased attested slot, and supermajority - should save but not broadcast
l3 := util.NewTestLightClient(t, version.Altair, util.WithIncreasedAttestedSlot(3), util.WithSupermajority())
update3, err := lightClient.NewLightClientFinalityUpdateFromBeaconState(l3.Ctx, l3.State, l3.Block, l3.AttestedState, l3.AttestedBlock, l3.FinalizedBlock)
l3 := util.NewTestLightClient(t, version.Altair, util.WithIncreasedAttestedSlot(3), util.WithSupermajority(0))
update3, err := NewLightClientFinalityUpdateFromBeaconState(l3.Ctx, l3.State, l3.Block, l3.AttestedState, l3.AttestedBlock, l3.FinalizedBlock)
require.NoError(t, err, "Failed to create light client finality update")
require.Equal(t, true, lightClient.IsBetterFinalityUpdate(update3, update2), "update3 should be better than update2")
require.Equal(t, true, IsBetterFinalityUpdate(update3, update2), "update3 should be better than update2")
// update3 should not be valid for broadcast - meaning it should not be broadcasted
require.Equal(t, false, lightClient.IsFinalityUpdateValidForBroadcast(update3, lcStore.LastFinalityUpdate()), "update3 should not be valid for broadcast")
require.Equal(t, false, IsFinalityUpdateValidForBroadcast(update3, lcStore.LastFinalityUpdate()), "update3 should not be valid for broadcast")
lcStore.SetLastFinalityUpdate(update3, true)
require.Equal(t, update3, lcStore.LastFinalityUpdate(), "lastFinalityUpdate should match the set value")
require.Equal(t, false, p2p.BroadcastCalled.Load(), "Broadcast should not have been when previous was already broadcast")
// update 4 with increased finality slot, increased attested slot, and supermajority - should save and broadcast
l4 := util.NewTestLightClient(t, version.Altair, util.WithIncreasedFinalizedSlot(1), util.WithIncreasedAttestedSlot(1), util.WithSupermajority())
update4, err := lightClient.NewLightClientFinalityUpdateFromBeaconState(l4.Ctx, l4.State, l4.Block, l4.AttestedState, l4.AttestedBlock, l4.FinalizedBlock)
l4 := util.NewTestLightClient(t, version.Altair, util.WithIncreasedFinalizedSlot(1), util.WithIncreasedAttestedSlot(1), util.WithSupermajority(0))
update4, err := NewLightClientFinalityUpdateFromBeaconState(l4.Ctx, l4.State, l4.Block, l4.AttestedState, l4.AttestedBlock, l4.FinalizedBlock)
require.NoError(t, err, "Failed to create light client finality update")
require.Equal(t, true, lightClient.IsBetterFinalityUpdate(update4, update3), "update4 should be better than update3")
require.Equal(t, true, IsBetterFinalityUpdate(update4, update3), "update4 should be better than update3")
// update4 should be valid for broadcast - meaning it should be broadcasted
require.Equal(t, true, lightClient.IsFinalityUpdateValidForBroadcast(update4, lcStore.LastFinalityUpdate()), "update4 should be valid for broadcast")
require.Equal(t, true, IsFinalityUpdateValidForBroadcast(update4, lcStore.LastFinalityUpdate()), "update4 should be valid for broadcast")
lcStore.SetLastFinalityUpdate(update4, true)
require.Equal(t, update4, lcStore.LastFinalityUpdate(), "lastFinalityUpdate should match the set value")
@@ -138,13 +144,13 @@ func TestLightClientStore_SetLastFinalityUpdate(t *testing.T) {
p2p.BroadcastCalled.Store(false) // Reset for next test
// update 5 with the same new finality slot, increased attested slot, and supermajority - should save but not broadcast
l5 := util.NewTestLightClient(t, version.Altair, util.WithIncreasedFinalizedSlot(1), util.WithIncreasedAttestedSlot(2), util.WithSupermajority())
update5, err := lightClient.NewLightClientFinalityUpdateFromBeaconState(l5.Ctx, l5.State, l5.Block, l5.AttestedState, l5.AttestedBlock, l5.FinalizedBlock)
l5 := util.NewTestLightClient(t, version.Altair, util.WithIncreasedFinalizedSlot(1), util.WithIncreasedAttestedSlot(2), util.WithSupermajority(0))
update5, err := NewLightClientFinalityUpdateFromBeaconState(l5.Ctx, l5.State, l5.Block, l5.AttestedState, l5.AttestedBlock, l5.FinalizedBlock)
require.NoError(t, err, "Failed to create light client finality update")
require.Equal(t, true, lightClient.IsBetterFinalityUpdate(update5, update4), "update5 should be better than update4")
require.Equal(t, true, IsBetterFinalityUpdate(update5, update4), "update5 should be better than update4")
// update5 should not be valid for broadcast - meaning it should not be broadcasted
require.Equal(t, false, lightClient.IsFinalityUpdateValidForBroadcast(update5, lcStore.LastFinalityUpdate()), "update5 should not be valid for broadcast")
require.Equal(t, false, IsFinalityUpdateValidForBroadcast(update5, lcStore.LastFinalityUpdate()), "update5 should not be valid for broadcast")
lcStore.SetLastFinalityUpdate(update5, true)
require.Equal(t, update5, lcStore.LastFinalityUpdate(), "lastFinalityUpdate should match the set value")
@@ -152,14 +158,802 @@ func TestLightClientStore_SetLastFinalityUpdate(t *testing.T) {
// update 6 with the same new finality slot, increased attested slot, and no supermajority - should save but not broadcast
l6 := util.NewTestLightClient(t, version.Altair, util.WithIncreasedFinalizedSlot(1), util.WithIncreasedAttestedSlot(3))
update6, err := lightClient.NewLightClientFinalityUpdateFromBeaconState(l6.Ctx, l6.State, l6.Block, l6.AttestedState, l6.AttestedBlock, l6.FinalizedBlock)
update6, err := NewLightClientFinalityUpdateFromBeaconState(l6.Ctx, l6.State, l6.Block, l6.AttestedState, l6.AttestedBlock, l6.FinalizedBlock)
require.NoError(t, err, "Failed to create light client finality update")
require.Equal(t, true, lightClient.IsBetterFinalityUpdate(update6, update5), "update6 should be better than update5")
require.Equal(t, true, IsBetterFinalityUpdate(update6, update5), "update6 should be better than update5")
// update6 should not be valid for broadcast - meaning it should not be broadcasted
require.Equal(t, false, lightClient.IsFinalityUpdateValidForBroadcast(update6, lcStore.LastFinalityUpdate()), "update6 should not be valid for broadcast")
require.Equal(t, false, IsFinalityUpdateValidForBroadcast(update6, lcStore.LastFinalityUpdate()), "update6 should not be valid for broadcast")
lcStore.SetLastFinalityUpdate(update6, true)
require.Equal(t, update6, lcStore.LastFinalityUpdate(), "lastFinalityUpdate should match the set value")
require.Equal(t, false, p2p.BroadcastCalled.Load(), "Broadcast should not have been called when previous was already broadcast with supermajority")
}
func TestLightClientStore_SaveLCData(t *testing.T) {
t.Run("no parent in cache or db - new is head", func(t *testing.T) {
db := testDB.SetupDB(t)
s := NewLightClientStore(&p2pTesting.FakeP2P{}, new(event.Feed), db)
require.NotNil(t, s)
l := util.NewTestLightClient(t, version.Altair)
blkRoot, err := l.Block.Block().HashTreeRoot()
require.NoError(t, err)
require.NoError(t, s.SaveLCData(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock, blkRoot), "Failed to save light client data")
update, err := NewLightClientUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
finalityUpdate, err := NewLightClientFinalityUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
optimisticUpdate, err := NewLightClientOptimisticUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock)
require.NoError(t, err)
attstedBlkRoot, err := l.AttestedBlock.Block().HashTreeRoot()
require.NoError(t, err)
require.DeepEqual(t, finalityUpdate, s.lastFinalityUpdate, "Expected to find the last finality update in the store")
require.DeepEqual(t, optimisticUpdate, s.lastOptimisticUpdate, "Expected to find the last optimistic update in the store")
require.DeepEqual(t, update, s.cache.items[attstedBlkRoot].bestUpdate, "Expected to find the update in the non-finality cache")
require.DeepEqual(t, finalityUpdate, s.cache.items[attstedBlkRoot].bestFinalityUpdate, "Expected to find the finality update in the non-finality cache")
})
t.Run("no parent in cache or db - new not head", func(t *testing.T) {
db := testDB.SetupDB(t)
s := NewLightClientStore(&p2pTesting.FakeP2P{}, new(event.Feed), db)
require.NotNil(t, s)
l := util.NewTestLightClient(t, version.Altair)
blkRoot, err := l.FinalizedBlock.Block().HashTreeRoot()
require.NoError(t, err)
require.NoError(t, s.SaveLCData(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock, blkRoot), "Failed to save light client data")
update, err := NewLightClientUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
finalityUpdate, err := NewLightClientFinalityUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
optimisticUpdate, err := NewLightClientOptimisticUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock)
require.NoError(t, err)
attstedBlkRoot, err := l.AttestedBlock.Block().HashTreeRoot()
require.NoError(t, err)
require.IsNil(t, s.lastFinalityUpdate, "Expected to not find the last finality update in the store since the block is not head")
require.DeepEqual(t, optimisticUpdate, s.lastOptimisticUpdate, "Expected to find the last optimistic update in the store")
require.DeepEqual(t, update, s.cache.items[attstedBlkRoot].bestUpdate, "Expected to find the update in the non-finality cache")
require.DeepEqual(t, finalityUpdate, s.cache.items[attstedBlkRoot].bestFinalityUpdate, "Expected to find the finality update in the non-finality cache")
})
t.Run("parent in db", func(t *testing.T) {
db := testDB.SetupDB(t)
s := NewLightClientStore(&p2pTesting.FakeP2P{}, new(event.Feed), db)
require.NotNil(t, s)
l := util.NewTestLightClient(t, version.Altair)
// save an update for this period in db
period := slots.SyncCommitteePeriod(slots.ToEpoch(l.AttestedBlock.Block().Slot()))
update, err := NewLightClientUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
require.NoError(t, db.SaveLightClientUpdate(l.Ctx, period, update), "Failed to save light client update in db")
l2 := util.NewTestLightClient(t, version.Altair, util.WithIncreasedAttestedSlot(1), util.WithSupermajority(0)) // updates from this setup should be all better
blkRoot, err := l2.Block.Block().HashTreeRoot()
require.NoError(t, err)
require.NoError(t, s.SaveLCData(l2.Ctx, l2.State, l2.Block, l2.AttestedState, l2.AttestedBlock, l2.FinalizedBlock, blkRoot), "Failed to save light client data")
update, err = NewLightClientUpdateFromBeaconState(l2.Ctx, l2.State, l2.Block, l2.AttestedState, l2.AttestedBlock, l2.FinalizedBlock)
require.NoError(t, err)
finalityUpdate, err := NewLightClientFinalityUpdateFromBeaconState(l2.Ctx, l2.State, l2.Block, l2.AttestedState, l2.AttestedBlock, l2.FinalizedBlock)
require.NoError(t, err)
optimisticUpdate, err := NewLightClientOptimisticUpdateFromBeaconState(l2.Ctx, l2.State, l2.Block, l2.AttestedState, l2.AttestedBlock)
require.NoError(t, err)
attstedBlkRoot, err := l2.AttestedBlock.Block().HashTreeRoot()
require.NoError(t, err)
require.DeepEqual(t, finalityUpdate, s.lastFinalityUpdate, "Expected to find the last finality update in the store")
require.DeepEqual(t, optimisticUpdate, s.lastOptimisticUpdate, "Expected to find the last optimistic update in the store")
require.DeepEqual(t, update, s.cache.items[attstedBlkRoot].bestUpdate, "Expected to find the update in the non-finality cache")
require.DeepEqual(t, finalityUpdate, s.cache.items[attstedBlkRoot].bestFinalityUpdate, "Expected to find the finality update in the non-finality cache")
})
t.Run("parent in cache", func(t *testing.T) {
db := testDB.SetupDB(t)
s := NewLightClientStore(&p2pTesting.FakeP2P{}, new(event.Feed), db)
require.NotNil(t, s)
l := util.NewTestLightClient(t, version.Altair)
l2 := util.NewTestLightClient(t, version.Altair, util.WithIncreasedAttestedSlot(1), util.WithSupermajority(0)) // updates from this setup should be all better
// save the cache item for this period in cache
period := slots.SyncCommitteePeriod(slots.ToEpoch(l.AttestedBlock.Block().Slot()))
update, err := NewLightClientUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
finalityUpdate, err := NewLightClientFinalityUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
item := &cacheItem{
period: period,
bestUpdate: update,
bestFinalityUpdate: finalityUpdate,
}
attestedBlockRoot := l2.AttestedBlock.Block().ParentRoot() // we want this item to be the parent of the new block
s.cache.items[attestedBlockRoot] = item
blkRoot, err := l2.Block.Block().HashTreeRoot()
require.NoError(t, err)
require.NoError(t, s.SaveLCData(l2.Ctx, l2.State, l2.Block, l2.AttestedState, l2.AttestedBlock, l2.FinalizedBlock, blkRoot), "Failed to save light client data")
update, err = NewLightClientUpdateFromBeaconState(l2.Ctx, l2.State, l2.Block, l2.AttestedState, l2.AttestedBlock, l2.FinalizedBlock)
require.NoError(t, err)
finalityUpdate, err = NewLightClientFinalityUpdateFromBeaconState(l2.Ctx, l2.State, l2.Block, l2.AttestedState, l2.AttestedBlock, l2.FinalizedBlock)
require.NoError(t, err)
optimisticUpdate, err := NewLightClientOptimisticUpdateFromBeaconState(l2.Ctx, l2.State, l2.Block, l2.AttestedState, l2.AttestedBlock)
require.NoError(t, err)
attstedBlkRoot, err := l2.AttestedBlock.Block().HashTreeRoot()
require.NoError(t, err)
require.DeepEqual(t, finalityUpdate, s.lastFinalityUpdate, "Expected to find the last finality update in the store")
require.DeepEqual(t, optimisticUpdate, s.lastOptimisticUpdate, "Expected to find the last optimistic update in the store")
require.DeepEqual(t, update, s.cache.items[attstedBlkRoot].bestUpdate, "Expected to find the update in the non-finality cache")
require.DeepEqual(t, finalityUpdate, s.cache.items[attstedBlkRoot].bestFinalityUpdate, "Expected to find the finality update in the non-finality cache")
})
t.Run("parent in the previous period", func(t *testing.T) {
db := testDB.SetupDB(t)
s := NewLightClientStore(&p2pTesting.FakeP2P{}, new(event.Feed), db)
require.NotNil(t, s)
l := util.NewTestLightClient(t, version.Altair)
l2 := util.NewTestLightClient(t, version.Bellatrix, util.WithIncreasedAttestedSlot(1), util.WithSupermajority(0)) // updates from this setup should be all better
// save the cache item for this period1 in cache
period1 := slots.SyncCommitteePeriod(slots.ToEpoch(l.AttestedBlock.Block().Slot()))
update, err := NewLightClientUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
finalityUpdate, err := NewLightClientFinalityUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
item := &cacheItem{
period: period1,
bestUpdate: update,
bestFinalityUpdate: finalityUpdate,
}
attestedBlockRoot := l2.AttestedBlock.Block().ParentRoot() // we want this item to be the parent of the new block
s.cache.items[attestedBlockRoot] = item
blkRoot, err := l2.Block.Block().HashTreeRoot()
require.NoError(t, err)
require.NoError(t, s.SaveLCData(l2.Ctx, l2.State, l2.Block, l2.AttestedState, l2.AttestedBlock, l2.FinalizedBlock, blkRoot), "Failed to save light client data")
update, err = NewLightClientUpdateFromBeaconState(l2.Ctx, l2.State, l2.Block, l2.AttestedState, l2.AttestedBlock, l2.FinalizedBlock)
require.NoError(t, err)
finalityUpdate, err = NewLightClientFinalityUpdateFromBeaconState(l2.Ctx, l2.State, l2.Block, l2.AttestedState, l2.AttestedBlock, l2.FinalizedBlock)
require.NoError(t, err)
optimisticUpdate, err := NewLightClientOptimisticUpdateFromBeaconState(l2.Ctx, l2.State, l2.Block, l2.AttestedState, l2.AttestedBlock)
require.NoError(t, err)
attstedBlkRoot, err := l2.AttestedBlock.Block().HashTreeRoot()
require.NoError(t, err)
require.DeepEqual(t, finalityUpdate, s.lastFinalityUpdate, "Expected to find the last finality update in the store")
require.DeepEqual(t, optimisticUpdate, s.lastOptimisticUpdate, "Expected to find the last optimistic update in the store")
require.DeepEqual(t, update, s.cache.items[attstedBlkRoot].bestUpdate, "Expected to find the update in the non-finality cache")
require.DeepEqual(t, finalityUpdate, s.cache.items[attstedBlkRoot].bestFinalityUpdate, "Expected to find the finality update in the non-finality cache")
})
}
func TestLightClientStore_MigrateToCold(t *testing.T) {
// This tests the scenario where chain advances but the cache is empty.
// It should see that there is nothing in the cache to migrate and just update the tail to the new finalized root.
t.Run("empty cache", func(t *testing.T) {
beaconDB := testDB.SetupDB(t)
ctx := context.Background()
finalizedBlockRoot, _ := saveInitialFinalizedCheckpointData(t, ctx, beaconDB)
s := NewLightClientStore(&p2pTesting.FakeP2P{}, new(event.Feed), beaconDB)
require.NotNil(t, s)
for i := 0; i < 3; i++ {
newBlock := util.NewBeaconBlock()
newBlock.Block.Slot = primitives.Slot(32 + uint64(i))
newBlock.Block.ParentRoot = finalizedBlockRoot[:]
signedNewBlock, err := blocks.NewSignedBeaconBlock(newBlock)
require.NoError(t, err)
blockRoot, err := signedNewBlock.Block().HashTreeRoot()
require.NoError(t, err)
require.NoError(t, beaconDB.SaveBlock(ctx, signedNewBlock))
finalizedBlockRoot = blockRoot
}
err := s.MigrateToCold(ctx, finalizedBlockRoot)
require.NoError(t, err)
require.Equal(t, 0, len(s.cache.items))
})
// This tests the scenario where chain advances but the CANONICAL cache is empty.
// It should see that there is nothing in the canonical cache to migrate and just update the tail to the new finalized root AND delete anything non-canonical.
t.Run("non canonical cache", func(t *testing.T) {
beaconDB := testDB.SetupDB(t)
ctx := context.Background()
finalizedBlockRoot, _ := saveInitialFinalizedCheckpointData(t, ctx, beaconDB)
s := NewLightClientStore(&p2pTesting.FakeP2P{}, new(event.Feed), beaconDB)
require.NotNil(t, s)
for i := 0; i < 3; i++ {
newBlock := util.NewBeaconBlock()
newBlock.Block.Slot = primitives.Slot(32 + uint64(i))
newBlock.Block.ParentRoot = finalizedBlockRoot[:]
signedNewBlock, err := blocks.NewSignedBeaconBlock(newBlock)
require.NoError(t, err)
blockRoot, err := signedNewBlock.Block().HashTreeRoot()
require.NoError(t, err)
require.NoError(t, beaconDB.SaveBlock(ctx, signedNewBlock))
finalizedBlockRoot = blockRoot
}
// Add a non-canonical item to the cache
cacheItem := &cacheItem{
period: 0,
slot: 33,
}
nonCanonicalBlockRoot := [32]byte{1, 2, 3, 4}
s.cache.items[nonCanonicalBlockRoot] = cacheItem
require.Equal(t, 1, len(s.cache.items))
err := s.MigrateToCold(ctx, finalizedBlockRoot)
require.NoError(t, err)
require.Equal(t, 0, len(s.cache.items), "Expected the non-canonical item in the cache to be deleted")
})
// db has update - cache has both canonical and non-canonical items.
// should update the update in db and delete cache.
t.Run("mixed cache - finality immediately after cache", func(t *testing.T) {
beaconDB := testDB.SetupDB(t)
ctx := context.Background()
finalizedBlockRoot, _ := saveInitialFinalizedCheckpointData(t, ctx, beaconDB)
require.NoError(t, beaconDB.SaveHeadBlockRoot(ctx, finalizedBlockRoot))
s := NewLightClientStore(&p2pTesting.FakeP2P{}, new(event.Feed), beaconDB)
require.NotNil(t, s)
// Save an update for this period in db
l := util.NewTestLightClient(t, version.Altair)
update, err := NewLightClientUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
period := slots.SyncCommitteePeriod(slots.ToEpoch(l.AttestedBlock.Block().Slot()))
require.NoError(t, beaconDB.SaveLightClientUpdate(ctx, period, update))
lastBlockRoot := finalizedBlockRoot
lastAttestedRoot := finalizedBlockRoot
lastUpdate := update
for i := 1; i < 4; i++ {
l = util.NewTestLightClient(t, version.Altair, util.WithIncreasedAttestedSlot(uint64(i)), util.WithSupermajority(uint64(i)), util.WithAttestedParentRoot(lastAttestedRoot))
require.NoError(t, beaconDB.SaveBlock(ctx, l.Block))
require.NoError(t, s.SaveLCData(ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock, [32]byte{1}))
lastBlockRoot, err = l.Block.Block().HashTreeRoot()
require.NoError(t, err)
lastAttestedRoot, err = l.AttestedBlock.Block().HashTreeRoot()
require.NoError(t, err)
update, err = NewLightClientUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
lastUpdate = update
}
require.Equal(t, 3, len(s.cache.items))
// Add a non-canonical item to the cache
cacheItem := &cacheItem{
period: 0,
slot: 33,
}
nonCanonicalBlockRoot := [32]byte{1, 2, 3, 4}
s.cache.items[nonCanonicalBlockRoot] = cacheItem
require.Equal(t, 4, len(s.cache.items))
err = s.MigrateToCold(ctx, lastBlockRoot)
require.NoError(t, err)
require.Equal(t, 0, len(s.cache.items), "Expected the non-canonical item in the cache to be deleted")
u, err := beaconDB.LightClientUpdate(ctx, period)
require.NoError(t, err)
require.NotNil(t, u)
require.DeepEqual(t, lastUpdate, u)
})
// db has update - cache has both canonical and non-canonical items. finalized height is in the middle.
// should update the update in db and delete items in cache before finalized slot.
t.Run("mixed cache - finality middle of cache", func(t *testing.T) {
beaconDB := testDB.SetupDB(t)
ctx := context.Background()
finalizedBlockRoot, _ := saveInitialFinalizedCheckpointData(t, ctx, beaconDB)
require.NoError(t, beaconDB.SaveHeadBlockRoot(ctx, finalizedBlockRoot))
s := NewLightClientStore(&p2pTesting.FakeP2P{}, new(event.Feed), beaconDB)
require.NotNil(t, s)
// Save an update for this period in db
l := util.NewTestLightClient(t, version.Altair)
update, err := NewLightClientUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
period := slots.SyncCommitteePeriod(slots.ToEpoch(l.AttestedBlock.Block().Slot()))
require.NoError(t, beaconDB.SaveLightClientUpdate(ctx, period, update))
lastBlockRoot := finalizedBlockRoot
lastUpdate := update
lastAttestedRoot := [32]byte{}
for i := 1; i < 4; i++ {
l = util.NewTestLightClient(t, version.Altair, util.WithIncreasedAttestedSlot(uint64(i)), util.WithSupermajority(uint64(i)), util.WithAttestedParentRoot(lastAttestedRoot))
require.NoError(t, beaconDB.SaveBlock(ctx, l.Block))
require.NoError(t, s.SaveLCData(ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock, [32]byte{1}))
root, err := l.Block.Block().HashTreeRoot()
require.NoError(t, err)
lastBlockRoot = root
update, err = NewLightClientUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
lastUpdate = update
lastAttestedRoot, err = l.AttestedBlock.Block().HashTreeRoot()
require.NoError(t, err)
}
require.Equal(t, 3, len(s.cache.items))
// Add a non-canonical item to the cache
cacheItem := &cacheItem{
period: 0,
slot: 33,
}
nonCanonicalBlockRoot := [32]byte{1, 2, 3, 4}
s.cache.items[nonCanonicalBlockRoot] = cacheItem
require.Equal(t, 4, len(s.cache.items))
for i := 4; i < 7; i++ {
l = util.NewTestLightClient(t, version.Altair, util.WithIncreasedAttestedSlot(uint64(i)), util.WithSupermajority(0), util.WithAttestedParentRoot(lastAttestedRoot))
require.NoError(t, beaconDB.SaveBlock(ctx, l.Block))
require.NoError(t, s.SaveLCData(ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock, [32]byte{1}))
lastAttestedRoot, err = l.AttestedBlock.Block().HashTreeRoot()
require.NoError(t, err)
}
require.Equal(t, 7, len(s.cache.items))
err = s.MigrateToCold(ctx, lastBlockRoot)
require.NoError(t, err)
require.Equal(t, 3, len(s.cache.items), "Expected the non-canonical item in the cache to be deleted")
u, err := beaconDB.LightClientUpdate(ctx, period)
require.NoError(t, err)
require.NotNil(t, u)
require.DeepEqual(t, lastUpdate, u)
})
// we have multiple periods in the cache before finalization happens. we expect all of them to be saved in db.
t.Run("finality after multiple periods in cache", func(t *testing.T) {
beaconDB := testDB.SetupDB(t)
ctx := context.Background()
cfg := params.BeaconConfig().Copy()
cfg.EpochsPerSyncCommitteePeriod = 1
params.OverrideBeaconConfig(cfg)
finalizedBlockRoot, _ := saveInitialFinalizedCheckpointData(t, ctx, beaconDB)
require.NoError(t, beaconDB.SaveHeadBlockRoot(ctx, finalizedBlockRoot))
s := NewLightClientStore(&p2pTesting.FakeP2P{}, new(event.Feed), beaconDB)
require.NotNil(t, s)
// Save an update for this period1 in db
l := util.NewTestLightClient(t, version.Altair)
update, err := NewLightClientUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
period1 := slots.SyncCommitteePeriod(slots.ToEpoch(l.AttestedBlock.Block().Slot()))
require.NoError(t, beaconDB.SaveLightClientUpdate(ctx, period1, update))
lastBlockRoot := finalizedBlockRoot
lastUpdatePeriod1 := update
lastAttestedRoot := [32]byte{}
for i := 1; i < 4; i++ {
l = util.NewTestLightClient(t, version.Altair, util.WithIncreasedAttestedSlot(uint64(i)), util.WithSupermajority(uint64(i)), util.WithAttestedParentRoot(lastAttestedRoot))
require.NoError(t, beaconDB.SaveBlock(ctx, l.Block))
require.NoError(t, s.SaveLCData(ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock, [32]byte{1}))
root, err := l.Block.Block().HashTreeRoot()
require.NoError(t, err)
lastBlockRoot = root
lastUpdatePeriod1, err = NewLightClientUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
lastAttestedRoot, err = l.AttestedBlock.Block().HashTreeRoot()
require.NoError(t, err)
}
period2 := period1
var lastUpdatePeriod2 interfaces.LightClientUpdate
for i := 1; i < 4; i++ {
l = util.NewTestLightClient(t, version.Altair, util.WithIncreasedAttestedSlot(uint64(i)+33), util.WithSupermajority(uint64(i)), util.WithAttestedParentRoot(lastAttestedRoot))
require.NoError(t, beaconDB.SaveBlock(ctx, l.Block))
require.NoError(t, s.SaveLCData(ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock, [32]byte{1}))
root, err := l.Block.Block().HashTreeRoot()
require.NoError(t, err)
lastBlockRoot = root
lastUpdatePeriod2, err = NewLightClientUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
lastAttestedRoot, err = l.AttestedBlock.Block().HashTreeRoot()
require.NoError(t, err)
period2 = slots.SyncCommitteePeriod(slots.ToEpoch(l.AttestedBlock.Block().Slot()))
}
require.Equal(t, 6, len(s.cache.items))
// Add a non-canonical item to the cache
cacheItem := &cacheItem{
period: 0,
slot: 33,
}
nonCanonicalBlockRoot := [32]byte{1, 2, 3, 4}
s.cache.items[nonCanonicalBlockRoot] = cacheItem
require.Equal(t, 7, len(s.cache.items))
for i := 4; i < 7; i++ {
l = util.NewTestLightClient(t, version.Altair, util.WithIncreasedAttestedSlot(uint64(i)+33), util.WithSupermajority(0), util.WithAttestedParentRoot(lastAttestedRoot))
require.NoError(t, beaconDB.SaveBlock(ctx, l.Block))
require.NoError(t, s.SaveLCData(ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock, [32]byte{1}))
lastAttestedRoot, err = l.AttestedBlock.Block().HashTreeRoot()
require.NoError(t, err)
}
require.Equal(t, 10, len(s.cache.items))
err = s.MigrateToCold(ctx, lastBlockRoot)
require.NoError(t, err)
require.Equal(t, 3, len(s.cache.items), "Expected the non-canonical item in the cache to be deleted")
u, err := beaconDB.LightClientUpdate(ctx, period2)
require.NoError(t, err)
require.NotNil(t, u)
require.DeepEqual(t, lastUpdatePeriod2, u)
u, err = beaconDB.LightClientUpdate(ctx, period1)
require.NoError(t, err)
require.NotNil(t, u)
require.DeepEqual(t, lastUpdatePeriod1, u)
})
}
func saveInitialFinalizedCheckpointData(t *testing.T, ctx context.Context, beaconDB db.Database) ([32]byte, interfaces.SignedBeaconBlock) {
genesis := util.NewBeaconBlock()
genesisRoot, err := genesis.Block.HashTreeRoot()
require.NoError(t, err)
require.NoError(t, beaconDB.SaveGenesisBlockRoot(ctx, genesisRoot))
util.SaveBlock(t, ctx, beaconDB, genesis)
genesisState, err := util.NewBeaconState()
require.NoError(t, err)
require.NoError(t, beaconDB.SaveState(ctx, genesisState, genesisRoot))
finalizedState, err := util.NewBeaconState()
require.NoError(t, err)
finalizedBlock := util.NewBeaconBlock()
finalizedBlock.Block.Slot = 32
finalizedBlock.Block.ParentRoot = genesisRoot[:]
signedFinalizedBlock, err := blocks.NewSignedBeaconBlock(finalizedBlock)
require.NoError(t, err)
finalizedBlockHeader, err := signedFinalizedBlock.Header()
require.NoError(t, err)
require.NoError(t, finalizedState.SetLatestBlockHeader(finalizedBlockHeader.Header))
finalizedStateRoot, err := finalizedState.HashTreeRoot(ctx)
require.NoError(t, err)
finalizedBlock.Block.StateRoot = finalizedStateRoot[:]
signedFinalizedBlock, err = blocks.NewSignedBeaconBlock(finalizedBlock)
require.NoError(t, err)
finalizedBlockRoot, err := signedFinalizedBlock.Block().HashTreeRoot()
require.NoError(t, err)
cp := ethpb.Checkpoint{
Epoch: 1,
Root: finalizedBlockRoot[:],
}
require.NoError(t, beaconDB.SaveBlock(ctx, signedFinalizedBlock))
require.NoError(t, beaconDB.SaveState(ctx, finalizedState, finalizedBlockRoot))
require.NoError(t, beaconDB.SaveFinalizedCheckpoint(ctx, &cp))
return finalizedBlockRoot, signedFinalizedBlock
}
func TestLightClientStore_LightClientUpdatesByRange(t *testing.T) {
t.Run("no updates", func(t *testing.T) {
d := testDB.SetupDB(t)
ctx := context.Background()
_, finalizedBlock := saveInitialFinalizedCheckpointData(t, ctx, d)
s := NewLightClientStore(&p2pTesting.FakeP2P{}, new(event.Feed), d)
require.NotNil(t, s)
updates, err := s.LightClientUpdates(ctx, 2, 5, finalizedBlock)
require.NoError(t, err)
require.Equal(t, 0, len(updates))
})
t.Run("single update from db", func(t *testing.T) {
d := testDB.SetupDB(t)
ctx := context.Background()
_, finalizedBlock := saveInitialFinalizedCheckpointData(t, ctx, d)
s := NewLightClientStore(&p2pTesting.FakeP2P{}, new(event.Feed), d)
require.NotNil(t, s)
l := util.NewTestLightClient(t, version.Altair)
update, err := NewLightClientUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
require.NoError(t, d.SaveLightClientUpdate(ctx, 3, update))
updates, err := s.LightClientUpdates(ctx, 3, 3, finalizedBlock)
require.NoError(t, err)
require.Equal(t, 1, len(updates))
require.DeepEqual(t, update, updates[0], "Expected to find the update in the store")
})
t.Run("multiple updates from db", func(t *testing.T) {
d := testDB.SetupDB(t)
ctx := context.Background()
_, finalizedBlock := saveInitialFinalizedCheckpointData(t, ctx, d)
s := NewLightClientStore(&p2pTesting.FakeP2P{}, new(event.Feed), d)
require.NotNil(t, s)
l := util.NewTestLightClient(t, version.Altair)
update, err := NewLightClientUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
require.NoError(t, d.SaveLightClientUpdate(ctx, 3, update))
require.NoError(t, d.SaveLightClientUpdate(ctx, 4, update))
require.NoError(t, d.SaveLightClientUpdate(ctx, 5, update))
updates, err := s.LightClientUpdates(ctx, 3, 5, finalizedBlock)
require.NoError(t, err)
require.Equal(t, 3, len(updates))
require.DeepEqual(t, update, updates[0], "Expected to find the update in the store")
require.DeepEqual(t, update, updates[1], "Expected to find the update in the store")
require.DeepEqual(t, update, updates[2], "Expected to find the update in the store")
})
t.Run("single update from cache", func(t *testing.T) {
d := testDB.SetupDB(t)
ctx := context.Background()
_, _ = saveInitialFinalizedCheckpointData(t, ctx, d)
s := NewLightClientStore(&p2pTesting.FakeP2P{}, new(event.Feed), d)
require.NotNil(t, s)
l := util.NewTestLightClient(t, version.Altair)
update, err := NewLightClientUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
cacheItem := &cacheItem{
period: 3,
bestUpdate: update,
}
s.cache.items[[32]byte{3}] = cacheItem
_, headBlock := saveStateAndBlockWithParentRoot(t, ctx, d, [32]byte{3})
updates, err := s.LightClientUpdates(ctx, 3, 3, headBlock)
require.NoError(t, err)
require.Equal(t, 1, len(updates))
require.DeepEqual(t, update, updates[0], "Expected to find the update in the store")
})
t.Run("multiple updates from cache", func(t *testing.T) {
d := testDB.SetupDB(t)
ctx := context.Background()
_, _ = saveInitialFinalizedCheckpointData(t, ctx, d)
s := NewLightClientStore(&p2pTesting.FakeP2P{}, new(event.Feed), d)
require.NotNil(t, s)
l := util.NewTestLightClient(t, version.Altair)
update, err := NewLightClientUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
cacheItemP3 := &cacheItem{
period: 3,
bestUpdate: update,
}
s.cache.items[[32]byte{3}] = cacheItemP3
cacheItemP4 := &cacheItem{
period: 4,
bestUpdate: update,
parent: cacheItemP3,
}
s.cache.items[[32]byte{4}] = cacheItemP4
cacheItemP5 := &cacheItem{
period: 5,
bestUpdate: update,
parent: cacheItemP4,
}
s.cache.items[[32]byte{5}] = cacheItemP5
_, headBlock := saveStateAndBlockWithParentRoot(t, ctx, d, [32]byte{5})
updates, err := s.LightClientUpdates(ctx, 3, 5, headBlock)
require.NoError(t, err)
require.Equal(t, 3, len(updates))
require.DeepEqual(t, update, updates[0], "Expected to find the update in the store")
require.DeepEqual(t, update, updates[1], "Expected to find the update in the store")
require.DeepEqual(t, update, updates[2], "Expected to find the update in the store")
})
t.Run("multiple updates from both db and cache - no overlap", func(t *testing.T) {
d := testDB.SetupDB(t)
ctx := context.Background()
_, _ = saveInitialFinalizedCheckpointData(t, ctx, d)
s := NewLightClientStore(&p2pTesting.FakeP2P{}, new(event.Feed), d)
require.NotNil(t, s)
l := util.NewTestLightClient(t, version.Altair)
update, err := NewLightClientUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
require.NoError(t, d.SaveLightClientUpdate(ctx, 1, update))
require.NoError(t, d.SaveLightClientUpdate(ctx, 2, update))
cacheItemP3 := &cacheItem{
period: 3,
bestUpdate: update,
}
s.cache.items[[32]byte{3}] = cacheItemP3
cacheItemP4 := &cacheItem{
period: 4,
bestUpdate: update,
parent: cacheItemP3,
}
s.cache.items[[32]byte{4}] = cacheItemP4
cacheItemP5 := &cacheItem{
period: 5,
bestUpdate: update,
parent: cacheItemP4,
}
s.cache.items[[32]byte{5}] = cacheItemP5
_, headBlock := saveStateAndBlockWithParentRoot(t, ctx, d, [32]byte{5})
updates, err := s.LightClientUpdates(ctx, 1, 5, headBlock)
require.NoError(t, err)
require.Equal(t, 5, len(updates))
for i := 0; i < 5; i++ {
require.DeepEqual(t, update, updates[i], "Expected to find the update in the store")
}
})
t.Run("multiple updates from both db and cache - overlap", func(t *testing.T) {
d := testDB.SetupDB(t)
ctx := context.Background()
_, _ = saveInitialFinalizedCheckpointData(t, ctx, d)
s := NewLightClientStore(&p2pTesting.FakeP2P{}, new(event.Feed), d)
require.NotNil(t, s)
l1 := util.NewTestLightClient(t, version.Altair)
update1, err := NewLightClientUpdateFromBeaconState(l1.Ctx, l1.State, l1.Block, l1.AttestedState, l1.AttestedBlock, l1.FinalizedBlock)
require.NoError(t, err)
l2 := util.NewTestLightClient(t, version.Altair, util.WithIncreasedAttestedSlot(1))
update2, err := NewLightClientUpdateFromBeaconState(l2.Ctx, l2.State, l2.Block, l2.AttestedState, l2.AttestedBlock, l2.FinalizedBlock)
require.NoError(t, err)
require.DeepNotEqual(t, update1, update2)
require.NoError(t, d.SaveLightClientUpdate(ctx, 1, update1))
require.NoError(t, d.SaveLightClientUpdate(ctx, 2, update1))
require.NoError(t, d.SaveLightClientUpdate(ctx, 3, update1))
require.NoError(t, d.SaveLightClientUpdate(ctx, 4, update1))
cacheItemP3 := &cacheItem{
period: 3,
bestUpdate: update2,
}
s.cache.items[[32]byte{3}] = cacheItemP3
cacheItemP4 := &cacheItem{
period: 4,
bestUpdate: update2,
parent: cacheItemP3,
}
s.cache.items[[32]byte{4}] = cacheItemP4
cacheItemP5 := &cacheItem{
period: 5,
bestUpdate: update2,
parent: cacheItemP4,
}
s.cache.items[[32]byte{5}] = cacheItemP5
_, headBlock := saveStateAndBlockWithParentRoot(t, ctx, d, [32]byte{5})
updates, err := s.LightClientUpdates(ctx, 1, 5, headBlock)
require.NoError(t, err)
require.Equal(t, 5, len(updates))
// first two updates should be update1
for i := 0; i < 2; i++ {
require.DeepEqual(t, update1, updates[i], "Expected to find the update in the store")
}
// next three updates should be update2 - as cache overrides db
for i := 2; i < 5; i++ {
require.DeepEqual(t, update2, updates[i], "Expected to find the update in the store")
}
})
t.Run("first continuous range", func(t *testing.T) {
d := testDB.SetupDB(t)
ctx := context.Background()
_, _ = saveInitialFinalizedCheckpointData(t, ctx, d)
s := NewLightClientStore(&p2pTesting.FakeP2P{}, new(event.Feed), d)
require.NotNil(t, s)
l1 := util.NewTestLightClient(t, version.Altair)
update, err := NewLightClientUpdateFromBeaconState(l1.Ctx, l1.State, l1.Block, l1.AttestedState, l1.AttestedBlock, l1.FinalizedBlock)
require.NoError(t, err)
require.NoError(t, d.SaveLightClientUpdate(ctx, 1, update))
require.NoError(t, d.SaveLightClientUpdate(ctx, 2, update))
cacheItemP4 := &cacheItem{
period: 4,
bestUpdate: update,
}
s.cache.items[[32]byte{4}] = cacheItemP4
cacheItemP5 := &cacheItem{
period: 5,
bestUpdate: update,
parent: cacheItemP4,
}
s.cache.items[[32]byte{5}] = cacheItemP5
_, headBlock := saveStateAndBlockWithParentRoot(t, ctx, d, [32]byte{5})
updates, err := s.LightClientUpdates(ctx, 1, 5, headBlock)
require.NoError(t, err)
require.Equal(t, 2, len(updates))
require.DeepEqual(t, update, updates[0], "Expected to find the update in the store")
require.DeepEqual(t, update, updates[1], "Expected to find the update in the store")
})
}
func saveStateAndBlockWithParentRoot(t *testing.T, ctx context.Context, d db.Database, parentRoot [32]byte) ([32]byte, interfaces.SignedBeaconBlock) {
blk := util.NewBeaconBlock()
blk.Block.ParentRoot = parentRoot[:]
blkRoot, err := blk.Block.HashTreeRoot()
require.NoError(t, err)
util.SaveBlock(t, ctx, d, blk)
st, err := util.NewBeaconState()
require.NoError(t, err)
require.NoError(t, d.SaveState(ctx, st, blkRoot))
signedFinalizedBlock, err := blocks.NewSignedBeaconBlock(blk)
require.NoError(t, err)
return blkRoot, signedFinalizedBlock
}

View File

@@ -253,10 +253,6 @@ func New(cliCtx *cli.Context, cancel context.CancelFunc, opts ...Option) (*Beaco
// their initialization.
beacon.finalizedStateAtStartUp = nil
if features.Get().EnableLightClient {
beacon.lcStore = lightclient.NewLightClientStore(beacon.db, beacon.fetchP2P(), beacon.StateFeed())
}
return beacon, nil
}
@@ -349,6 +345,11 @@ func registerServices(cliCtx *cli.Context, beacon *BeaconNode, synchronizer *sta
return errors.Wrap(err, "could not register P2P service")
}
if features.Get().EnableLightClient {
log.Debugln("Registering Light Client Store")
beacon.registerLightClientStore()
}
log.Debugln("Registering Backfill Service")
if err := beacon.RegisterBackfillService(cliCtx, bfs); err != nil {
return errors.Wrap(err, "could not register Back Fill service")
@@ -1139,6 +1140,11 @@ func (b *BeaconNode) RegisterBackfillService(cliCtx *cli.Context, bfs *backfill.
return b.services.RegisterService(bf)
}
func (b *BeaconNode) registerLightClientStore() {
lcs := lightclient.NewLightClientStore(b.fetchP2P(), b.StateFeed(), b.db)
b.lcStore = lcs
}
func hasNetworkFlag(cliCtx *cli.Context) bool {
for _, flag := range features.NetworkFlags {
for _, name := range flag.Names() {

View File

@@ -74,7 +74,9 @@ func TestNodeStart_Ok(t *testing.T) {
set := flag.NewFlagSet("test", 0)
set.String("datadir", tmp, "node data directory")
set.String("suggested-fee-recipient", "0x6e35733c5af9B61374A128e6F85f553aF09ff89A", "fee recipient")
set.Bool("enable-light-client", true, "enable light client")
require.NoError(t, set.Set("suggested-fee-recipient", "0x6e35733c5af9B61374A128e6F85f553aF09ff89A"))
require.NoError(t, set.Set("enable-light-client", "true"))
ctx, cancel := newCliContextWithCancel(&app, set)
@@ -88,6 +90,7 @@ func TestNodeStart_Ok(t *testing.T) {
node, err := New(ctx, cancel, options...)
require.NoError(t, err)
require.NotNil(t, node.lcStore)
node.services = &runtime.ServiceRegistry{}
go func() {
node.Start()

View File

@@ -102,7 +102,7 @@ func (s *Service) endpoints(
endpoints = append(endpoints, s.prysmValidatorEndpoints(stater, coreService)...)
if features.Get().EnableLightClient {
endpoints = append(endpoints, s.lightClientEndpoints(blocker, stater)...)
endpoints = append(endpoints, s.lightClientEndpoints()...)
}
if enableDebug {
@@ -1034,9 +1034,10 @@ func (*Service) configEndpoints() []endpoint {
}
}
func (s *Service) lightClientEndpoints(blocker lookup.Blocker, stater lookup.Stater) []endpoint {
func (s *Service) lightClientEndpoints() []endpoint {
server := &lightclient.Server{
LCStore: s.cfg.LCStore,
LCStore: s.cfg.LCStore,
HeadFetcher: s.cfg.HeadFetcher,
}
const namespace = "lightclient"

View File

@@ -11,6 +11,7 @@ go_library(
deps = [
"//api:go_default_library",
"//api/server/structs:go_default_library",
"//beacon-chain/blockchain:go_default_library",
"//beacon-chain/core/light-client:go_default_library",
"//beacon-chain/rpc/eth/shared:go_default_library",
"//config/params:go_default_library",
@@ -32,8 +33,10 @@ go_test(
deps = [
"//api/server/structs:go_default_library",
"//async/event:go_default_library",
"//beacon-chain/blockchain/testing:go_default_library",
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/core/light-client:go_default_library",
"//beacon-chain/db:go_default_library",
"//beacon-chain/db/testing:go_default_library",
"//beacon-chain/p2p/testing:go_default_library",
"//config/fieldparams:go_default_library",

View File

@@ -90,10 +90,21 @@ func (s *Server) GetLightClientUpdatesByRange(w http.ResponseWriter, req *http.R
return
}
if startPeriod*uint64(config.EpochsPerSyncCommitteePeriod) < uint64(config.AltairForkEpoch) {
httputil.HandleError(w, "Invalid 'start_period': before Altair fork", http.StatusBadRequest)
return
}
endPeriod := startPeriod + count - 1
headBlock, err := s.HeadFetcher.HeadBlock(ctx)
if err != nil {
httputil.HandleError(w, "Could not get head block: "+err.Error(), http.StatusInternalServerError)
return
}
// get updates
updates, err := s.LCStore.LightClientUpdates(ctx, startPeriod, endPeriod)
updates, err := s.LCStore.LightClientUpdates(ctx, startPeriod, endPeriod, headBlock)
if err != nil {
httputil.HandleError(w, "Could not get light client updates from DB: "+err.Error(), http.StatusInternalServerError)
return

View File

@@ -2,6 +2,7 @@ package lightclient
import (
"bytes"
"context"
"encoding/json"
"fmt"
"math/rand"
@@ -12,8 +13,10 @@ import (
"github.com/OffchainLabs/prysm/v6/api/server/structs"
"github.com/OffchainLabs/prysm/v6/async/event"
blockchainTest "github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/helpers"
lightclient "github.com/OffchainLabs/prysm/v6/beacon-chain/core/light-client"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db"
dbtesting "github.com/OffchainLabs/prysm/v6/beacon-chain/db/testing"
p2ptesting "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/testing"
fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams"
@@ -55,7 +58,8 @@ func TestLightClientHandler_GetLightClientBootstrap(t *testing.T) {
require.NoError(t, err)
db := dbtesting.SetupDB(t)
lcStore := lightclient.NewLightClientStore(db, &p2ptesting.FakeP2P{}, new(event.Feed))
lcStore := lightclient.NewLightClientStore(&p2ptesting.FakeP2P{}, new(event.Feed), db)
require.NoError(t, err)
err = db.SaveLightClientBootstrap(l.Ctx, blockRoot[:], bootstrap)
require.NoError(t, err)
@@ -99,7 +103,8 @@ func TestLightClientHandler_GetLightClientBootstrap(t *testing.T) {
require.NoError(t, err)
db := dbtesting.SetupDB(t)
lcStore := lightclient.NewLightClientStore(db, &p2ptesting.FakeP2P{}, new(event.Feed))
lcStore := lightclient.NewLightClientStore(&p2ptesting.FakeP2P{}, new(event.Feed), db)
require.NoError(t, err)
err = db.SaveLightClientBootstrap(l.Ctx, blockRoot[:], bootstrap)
require.NoError(t, err)
@@ -142,8 +147,9 @@ func TestLightClientHandler_GetLightClientBootstrap(t *testing.T) {
}
t.Run("no bootstrap found", func(t *testing.T) {
lcStore := lightclient.NewLightClientStore(&p2ptesting.FakeP2P{}, new(event.Feed), dbtesting.SetupDB(t))
s := &Server{
LCStore: lightclient.NewLightClientStore(dbtesting.SetupDB(t), &p2ptesting.FakeP2P{}, new(event.Feed)),
LCStore: lcStore,
}
request := httptest.NewRequest("GET", "http://foo.com/", nil)
request.SetPathValue("block_root", hexutil.Encode([]byte{0x00, 0x01, 0x02}))
@@ -185,13 +191,24 @@ func TestLightClientHandler_GetLightClientByRange(t *testing.T) {
updates = append(updates, update)
}
lcStore := lightclient.NewLightClientStore(&p2ptesting.FakeP2P{}, new(event.Feed), db)
blk := util.NewBeaconBlock()
signedBlk, err := blocks.NewSignedBeaconBlock(blk)
require.NoError(t, err)
s := &Server{
LCStore: lightclient.NewLightClientStore(db, &p2ptesting.FakeP2P{}, new(event.Feed)),
LCStore: lcStore,
HeadFetcher: &blockchainTest.ChainService{
Block: signedBlk,
},
}
saveHead(t, ctx, db)
updatePeriod := startPeriod
for _, update := range updates {
err := s.LCStore.SaveLightClientUpdate(ctx, updatePeriod, update)
err := db.SaveLightClientUpdate(ctx, updatePeriod, update)
require.NoError(t, err)
updatePeriod++
}
@@ -326,20 +343,30 @@ func TestLightClientHandler_GetLightClientByRange(t *testing.T) {
secondForkSlot := primitives.Slot(params.BeaconConfig().VersionToForkEpochMap()[testVersion+1] * primitives.Epoch(config.SlotsPerEpoch)).Add(1)
db := dbtesting.SetupDB(t)
lcStore := lightclient.NewLightClientStore(&p2ptesting.FakeP2P{}, new(event.Feed), db)
blk := util.NewBeaconBlock()
signedBlk, err := blocks.NewSignedBeaconBlock(blk)
require.NoError(t, err)
s := &Server{
LCStore: lightclient.NewLightClientStore(db, &p2ptesting.FakeP2P{}, new(event.Feed)),
LCStore: lcStore,
HeadFetcher: &blockchainTest.ChainService{
Block: signedBlk,
},
}
saveHead(t, ctx, db)
updates := make([]interfaces.LightClientUpdate, 2)
updatePeriod := firstForkSlot.Div(uint64(config.EpochsPerSyncCommitteePeriod)).Div(uint64(config.SlotsPerEpoch))
startPeriod := updatePeriod
var err error
updates[0], err = createUpdate(t, testVersion)
require.NoError(t, err)
err = s.LCStore.SaveLightClientUpdate(ctx, uint64(updatePeriod), updates[0])
err = db.SaveLightClientUpdate(ctx, uint64(updatePeriod), updates[0])
require.NoError(t, err)
updatePeriod = secondForkSlot.Div(uint64(config.EpochsPerSyncCommitteePeriod)).Div(uint64(config.SlotsPerEpoch))
@@ -347,7 +374,7 @@ func TestLightClientHandler_GetLightClientByRange(t *testing.T) {
updates[1], err = createUpdate(t, testVersion+1)
require.NoError(t, err)
err = s.LCStore.SaveLightClientUpdate(ctx, uint64(updatePeriod), updates[1])
err = db.SaveLightClientUpdate(ctx, uint64(updatePeriod), updates[1])
require.NoError(t, err)
t.Run("json", func(t *testing.T) {
@@ -446,19 +473,30 @@ func TestLightClientHandler_GetLightClientByRange(t *testing.T) {
slot := primitives.Slot(config.AltairForkEpoch * primitives.Epoch(config.SlotsPerEpoch)).Add(1)
db := dbtesting.SetupDB(t)
lcStore := lightclient.NewLightClientStore(&p2ptesting.FakeP2P{}, new(event.Feed), db)
blk := util.NewBeaconBlock()
signedBlk, err := blocks.NewSignedBeaconBlock(blk)
require.NoError(t, err)
s := &Server{
LCStore: lightclient.NewLightClientStore(db, &p2ptesting.FakeP2P{}, new(event.Feed)),
LCStore: lcStore,
HeadFetcher: &blockchainTest.ChainService{
Block: signedBlk,
},
}
saveHead(t, ctx, db)
updates := make([]interfaces.LightClientUpdate, 3)
updatePeriod := slot.Div(uint64(config.EpochsPerSyncCommitteePeriod)).Div(uint64(config.SlotsPerEpoch))
var err error
for i := 0; i < 3; i++ {
updates[i], err = createUpdate(t, version.Altair)
require.NoError(t, err)
err = s.LCStore.SaveLightClientUpdate(ctx, uint64(updatePeriod), updates[i])
err = db.SaveLightClientUpdate(ctx, uint64(updatePeriod), updates[i])
require.NoError(t, err)
updatePeriod++
@@ -493,20 +531,30 @@ func TestLightClientHandler_GetLightClientByRange(t *testing.T) {
slot := primitives.Slot(config.AltairForkEpoch * primitives.Epoch(config.SlotsPerEpoch)).Add(1)
db := dbtesting.SetupDB(t)
lcStore := lightclient.NewLightClientStore(&p2ptesting.FakeP2P{}, new(event.Feed), db)
blk := util.NewBeaconBlock()
signedBlk, err := blocks.NewSignedBeaconBlock(blk)
require.NoError(t, err)
s := &Server{
LCStore: lightclient.NewLightClientStore(db, &p2ptesting.FakeP2P{}, new(event.Feed)),
LCStore: lcStore,
HeadFetcher: &blockchainTest.ChainService{
Block: signedBlk,
},
}
saveHead(t, ctx, db)
updates := make([]interfaces.LightClientUpdate, 3)
updatePeriod := slot.Div(uint64(config.EpochsPerSyncCommitteePeriod)).Div(uint64(config.SlotsPerEpoch))
var err error
for i := 0; i < 3; i++ {
updates[i], err = createUpdate(t, version.Altair)
require.NoError(t, err)
err = s.LCStore.SaveLightClientUpdate(ctx, uint64(updatePeriod), updates[i])
err = db.SaveLightClientUpdate(ctx, uint64(updatePeriod), updates[i])
require.NoError(t, err)
updatePeriod++
@@ -536,10 +584,16 @@ func TestLightClientHandler_GetLightClientByRange(t *testing.T) {
})
t.Run("start period before altair", func(t *testing.T) {
config.AltairForkEpoch = 1
params.OverrideBeaconConfig(config)
db := dbtesting.SetupDB(t)
lcStore := lightclient.NewLightClientStore(&p2ptesting.FakeP2P{}, new(event.Feed), db)
s := &Server{
LCStore: lightclient.NewLightClientStore(db, &p2ptesting.FakeP2P{}, new(event.Feed)),
LCStore: lcStore,
}
startPeriod := 0
url := fmt.Sprintf("http://foo.com/?count=128&start_period=%d", startPeriod)
request := httptest.NewRequest("GET", url, nil)
@@ -548,11 +602,10 @@ func TestLightClientHandler_GetLightClientByRange(t *testing.T) {
s.GetLightClientUpdatesByRange(writer, request)
require.Equal(t, http.StatusOK, writer.Code)
var resp structs.LightClientUpdatesByRangeResponse
err := json.Unmarshal(writer.Body.Bytes(), &resp.Updates)
require.NoError(t, err)
require.Equal(t, 0, len(resp.Updates))
require.Equal(t, http.StatusBadRequest, writer.Code)
config.AltairForkEpoch = 0
params.OverrideBeaconConfig(config)
})
t.Run("missing updates", func(t *testing.T) {
@@ -560,15 +613,25 @@ func TestLightClientHandler_GetLightClientByRange(t *testing.T) {
t.Run("missing update in the middle", func(t *testing.T) {
db := dbtesting.SetupDB(t)
lcStore := lightclient.NewLightClientStore(&p2ptesting.FakeP2P{}, new(event.Feed), db)
blk := util.NewBeaconBlock()
signedBlk, err := blocks.NewSignedBeaconBlock(blk)
require.NoError(t, err)
s := &Server{
LCStore: lightclient.NewLightClientStore(db, &p2ptesting.FakeP2P{}, new(event.Feed)),
LCStore: lcStore,
HeadFetcher: &blockchainTest.ChainService{
Block: signedBlk,
},
}
saveHead(t, ctx, db)
updates := make([]interfaces.LightClientUpdate, 3)
updatePeriod := slot.Div(uint64(config.EpochsPerSyncCommitteePeriod)).Div(uint64(config.SlotsPerEpoch))
var err error
for i := 0; i < 3; i++ {
if i == 1 { // skip this update
updatePeriod++
@@ -577,7 +640,7 @@ func TestLightClientHandler_GetLightClientByRange(t *testing.T) {
updates[i], err = createUpdate(t, version.Altair)
require.NoError(t, err)
err = s.LCStore.SaveLightClientUpdate(ctx, uint64(updatePeriod), updates[i])
err = db.SaveLightClientUpdate(ctx, uint64(updatePeriod), updates[i])
require.NoError(t, err)
updatePeriod++
@@ -604,15 +667,25 @@ func TestLightClientHandler_GetLightClientByRange(t *testing.T) {
t.Run("missing update at the beginning", func(t *testing.T) {
db := dbtesting.SetupDB(t)
lcStore := lightclient.NewLightClientStore(&p2ptesting.FakeP2P{}, new(event.Feed), db)
blk := util.NewBeaconBlock()
signedBlk, err := blocks.NewSignedBeaconBlock(blk)
require.NoError(t, err)
s := &Server{
LCStore: lightclient.NewLightClientStore(db, &p2ptesting.FakeP2P{}, new(event.Feed)),
LCStore: lcStore,
HeadFetcher: &blockchainTest.ChainService{
Block: signedBlk,
},
}
saveHead(t, ctx, db)
updates := make([]interfaces.LightClientUpdate, 3)
updatePeriod := slot.Div(uint64(config.EpochsPerSyncCommitteePeriod)).Div(uint64(config.SlotsPerEpoch))
var err error
for i := 0; i < 3; i++ {
if i == 0 { // skip this update
updatePeriod++
@@ -622,7 +695,7 @@ func TestLightClientHandler_GetLightClientByRange(t *testing.T) {
updates[i], err = createUpdate(t, version.Altair)
require.NoError(t, err)
err = s.LCStore.SaveLightClientUpdate(ctx, uint64(updatePeriod), updates[i])
err = db.SaveLightClientUpdate(ctx, uint64(updatePeriod), updates[i])
require.NoError(t, err)
updatePeriod++
@@ -665,7 +738,12 @@ func TestLightClientHandler_GetLightClientFinalityUpdate(t *testing.T) {
update, err := lightclient.NewLightClientFinalityUpdateFromBeaconState(ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
s := &Server{LCStore: lightclient.NewLightClientStore(dbtesting.SetupDB(t), &p2ptesting.FakeP2P{}, new(event.Feed))}
lcStore := lightclient.NewLightClientStore(&p2ptesting.FakeP2P{}, new(event.Feed), dbtesting.SetupDB(t))
require.NoError(t, err)
s := &Server{
LCStore: lcStore,
}
s.LCStore.SetLastFinalityUpdate(update, false)
request := httptest.NewRequest("GET", "http://foo.com", nil)
@@ -690,7 +768,12 @@ func TestLightClientHandler_GetLightClientFinalityUpdate(t *testing.T) {
update, err := lightclient.NewLightClientFinalityUpdateFromBeaconState(ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
s := &Server{LCStore: lightclient.NewLightClientStore(dbtesting.SetupDB(t), &p2ptesting.FakeP2P{}, new(event.Feed))}
lcStore := lightclient.NewLightClientStore(&p2ptesting.FakeP2P{}, new(event.Feed), dbtesting.SetupDB(t))
require.NoError(t, err)
s := &Server{
LCStore: lcStore,
}
s.LCStore.SetLastFinalityUpdate(update, false)
request := httptest.NewRequest("GET", "http://foo.com", nil)
@@ -729,7 +812,11 @@ func TestLightClientHandler_GetLightClientOptimisticUpdate(t *testing.T) {
helpers.ClearCache()
t.Run("no update", func(t *testing.T) {
s := &Server{LCStore: lightclient.NewLightClientStore(dbtesting.SetupDB(t), &p2ptesting.FakeP2P{}, new(event.Feed))}
lcStore := lightclient.NewLightClientStore(&p2ptesting.FakeP2P{}, new(event.Feed), dbtesting.SetupDB(t))
s := &Server{
LCStore: lcStore,
}
request := httptest.NewRequest("GET", "http://foo.com", nil)
writer := httptest.NewRecorder()
@@ -745,7 +832,12 @@ func TestLightClientHandler_GetLightClientOptimisticUpdate(t *testing.T) {
update, err := lightclient.NewLightClientOptimisticUpdateFromBeaconState(ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock)
require.NoError(t, err)
s := &Server{LCStore: lightclient.NewLightClientStore(dbtesting.SetupDB(t), &p2ptesting.FakeP2P{}, new(event.Feed))}
lcStore := lightclient.NewLightClientStore(&p2ptesting.FakeP2P{}, new(event.Feed), dbtesting.SetupDB(t))
require.NoError(t, err)
s := &Server{
LCStore: lcStore,
}
s.LCStore.SetLastOptimisticUpdate(update, false)
request := httptest.NewRequest("GET", "http://foo.com", nil)
@@ -769,7 +861,12 @@ func TestLightClientHandler_GetLightClientOptimisticUpdate(t *testing.T) {
update, err := lightclient.NewLightClientOptimisticUpdateFromBeaconState(ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock)
require.NoError(t, err)
s := &Server{LCStore: lightclient.NewLightClientStore(dbtesting.SetupDB(t), &p2ptesting.FakeP2P{}, new(event.Feed))}
lcStore := lightclient.NewLightClientStore(&p2ptesting.FakeP2P{}, new(event.Feed), dbtesting.SetupDB(t))
require.NoError(t, err)
s := &Server{
LCStore: lcStore,
}
s.LCStore.SetLastOptimisticUpdate(update, false)
request := httptest.NewRequest("GET", "http://foo.com", nil)
@@ -984,3 +1081,14 @@ func createUpdate(t *testing.T, v int) (interfaces.LightClientUpdate, error) {
return update, nil
}
func saveHead(t *testing.T, ctx context.Context, d db.Database) {
blk := util.NewBeaconBlock()
blkRoot, err := blk.Block.HashTreeRoot()
require.NoError(t, err)
util.SaveBlock(t, ctx, d, blk)
st, err := util.NewBeaconState()
require.NoError(t, err)
require.NoError(t, d.SaveState(ctx, st, blkRoot))
require.NoError(t, d.SaveHeadBlockRoot(ctx, blkRoot))
}

View File

@@ -1,9 +1,11 @@
package lightclient
import (
"github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain"
lightClient "github.com/OffchainLabs/prysm/v6/beacon-chain/core/light-client"
)
type Server struct {
LCStore *lightClient.Store
LCStore *lightClient.Store
HeadFetcher blockchain.HeadFetcher
}

View File

@@ -112,7 +112,15 @@ func (s *Service) lightClientUpdatesByRangeRPCHandler(ctx context.Context, msg i
logger.Infof("LC: requesting updates by range (StartPeriod: %d, EndPeriod: %d)", r.StartPeriod, r.StartPeriod+r.Count-1)
updates, err := s.lcStore.LightClientUpdates(ctx, r.StartPeriod, endPeriod)
headBlock, err := s.cfg.chain.HeadBlock(ctx)
if err != nil {
s.writeErrorResponseToStream(responseCodeServerError, types.ErrGeneric.Error(), stream)
tracing.AnnotateError(span, err)
logger.WithError(err).Error("Cannot retrieve head block")
return err
}
updates, err := s.lcStore.LightClientUpdates(ctx, r.StartPeriod, endPeriod, headBlock)
if err != nil {
s.writeErrorResponseToStream(responseCodeServerError, types.ErrGeneric.Error(), stream)
tracing.AnnotateError(span, err)

View File

@@ -16,6 +16,7 @@ import (
mockSync "github.com/OffchainLabs/prysm/v6/beacon-chain/sync/initial-sync/testing"
"github.com/OffchainLabs/prysm/v6/config/features"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/blocks"
leakybucket "github.com/OffchainLabs/prysm/v6/container/leaky-bucket"
pb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v6/runtime/version"
@@ -43,6 +44,8 @@ func TestRPC_LightClientBootstrap(t *testing.T) {
Genesis: time.Unix(time.Now().Unix(), 0),
}
d := db.SetupDB(t)
lcStore := lightClient.NewLightClientStore(&p2ptest.FakeP2P{}, new(event.Feed), d)
r := Service{
ctx: ctx,
cfg: &config{
@@ -54,7 +57,7 @@ func TestRPC_LightClientBootstrap(t *testing.T) {
stateNotifier: &mockChain.MockStateNotifier{},
},
chainStarted: abool.New(),
lcStore: lightClient.NewLightClientStore(d, &p2ptest.FakeP2P{}, new(event.Feed)),
lcStore: lcStore,
subHandler: newSubTopicHandler(),
rateLimiter: newRateLimiter(p1),
}
@@ -159,6 +162,8 @@ func TestRPC_LightClientOptimisticUpdate(t *testing.T) {
Genesis: time.Unix(time.Now().Unix(), 0),
}
d := db.SetupDB(t)
lcStore := lightClient.NewLightClientStore(&p2ptest.FakeP2P{}, new(event.Feed), d)
r := Service{
ctx: ctx,
cfg: &config{
@@ -170,7 +175,7 @@ func TestRPC_LightClientOptimisticUpdate(t *testing.T) {
stateNotifier: &mockChain.MockStateNotifier{},
},
chainStarted: abool.New(),
lcStore: lightClient.NewLightClientStore(d, &p2ptest.FakeP2P{}, new(event.Feed)),
lcStore: lcStore,
subHandler: newSubTopicHandler(),
rateLimiter: newRateLimiter(p1),
}
@@ -274,6 +279,8 @@ func TestRPC_LightClientFinalityUpdate(t *testing.T) {
Genesis: time.Unix(time.Now().Unix(), 0),
}
d := db.SetupDB(t)
lcStore := lightClient.NewLightClientStore(&p2ptest.FakeP2P{}, new(event.Feed), d)
r := Service{
ctx: ctx,
cfg: &config{
@@ -285,7 +292,7 @@ func TestRPC_LightClientFinalityUpdate(t *testing.T) {
stateNotifier: &mockChain.MockStateNotifier{},
},
chainStarted: abool.New(),
lcStore: lightClient.NewLightClientStore(d, &p2ptest.FakeP2P{}, new(event.Feed)),
lcStore: lcStore,
subHandler: newSubTopicHandler(),
rateLimiter: newRateLimiter(p1),
}
@@ -384,11 +391,19 @@ func TestRPC_LightClientUpdatesByRange(t *testing.T) {
p1.Connect(p2)
require.Equal(t, 1, len(p1.BHost.Network().Peers()), "Expected peers to be connected")
blk := util.NewBeaconBlock()
signedBlk, err := blocks.NewSignedBeaconBlock(blk)
require.NoError(t, err)
chainService := &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Unix(time.Now().Unix(), 0),
Block: signedBlk,
}
d := db.SetupDB(t)
lcStore := lightClient.NewLightClientStore(&p2ptest.FakeP2P{}, new(event.Feed), d)
require.NoError(t, err)
r := Service{
ctx: ctx,
cfg: &config{
@@ -400,7 +415,7 @@ func TestRPC_LightClientUpdatesByRange(t *testing.T) {
stateNotifier: &mockChain.MockStateNotifier{},
},
chainStarted: abool.New(),
lcStore: lightClient.NewLightClientStore(d, &p2ptest.FakeP2P{}, new(event.Feed)),
lcStore: lcStore,
subHandler: newSubTopicHandler(),
rateLimiter: newRateLimiter(p1),
}
@@ -472,7 +487,7 @@ func TestRPC_LightClientUpdatesByRange(t *testing.T) {
t.Fatalf("unsupported version %d", i)
}
updates, err := r.lcStore.LightClientUpdates(ctx, 0, 4)
updates, err := r.lcStore.LightClientUpdates(ctx, 0, 4, signedBlk)
require.NoError(t, err)
updateSSZ, err := updates[uint64(responseCounter)].MarshalSSZ()
require.NoError(t, err)

View File

@@ -702,6 +702,8 @@ func TestSubscribe_ReceivesLCOptimisticUpdate(t *testing.T) {
Genesis: time.Unix(time.Now().Unix()-int64(genesisDrift), 0),
}
d := db.SetupDB(t)
lcStore := lightClient.NewLightClientStore(&p2ptest.FakeP2P{}, new(event.Feed), d)
r := Service{
ctx: ctx,
cfg: &config{
@@ -713,7 +715,7 @@ func TestSubscribe_ReceivesLCOptimisticUpdate(t *testing.T) {
stateNotifier: &mockChain.MockStateNotifier{},
},
chainStarted: abool.New(),
lcStore: lightClient.NewLightClientStore(d, &p2ptest.FakeP2P{}, new(event.Feed)),
lcStore: lcStore,
subHandler: newSubTopicHandler(),
}
markInitSyncComplete(t, &r)
@@ -732,7 +734,7 @@ func TestSubscribe_ReceivesLCOptimisticUpdate(t *testing.T) {
r.markForChainStart()
l := util.NewTestLightClient(t, version.Altair, util.WithSupermajority())
l := util.NewTestLightClient(t, version.Altair, util.WithSupermajority(0))
update, err := lightClient.NewLightClientOptimisticUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock)
require.NoError(t, err, "Error generating light client optimistic update")
@@ -770,6 +772,8 @@ func TestSubscribe_ReceivesLCFinalityUpdate(t *testing.T) {
Genesis: time.Unix(time.Now().Unix()-int64(genesisDrift), 0),
}
d := db.SetupDB(t)
lcStore := lightClient.NewLightClientStore(&p2ptest.FakeP2P{}, new(event.Feed), d)
r := Service{
ctx: ctx,
cfg: &config{
@@ -781,7 +785,7 @@ func TestSubscribe_ReceivesLCFinalityUpdate(t *testing.T) {
stateNotifier: &mockChain.MockStateNotifier{},
},
chainStarted: abool.New(),
lcStore: lightClient.NewLightClientStore(d, &p2ptest.FakeP2P{}, new(event.Feed)),
lcStore: lcStore,
subHandler: newSubTopicHandler(),
}
markInitSyncComplete(t, &r)
@@ -800,7 +804,7 @@ func TestSubscribe_ReceivesLCFinalityUpdate(t *testing.T) {
r.markForChainStart()
l := util.NewTestLightClient(t, version.Altair, util.WithSupermajority())
l := util.NewTestLightClient(t, version.Altair, util.WithSupermajority(0))
update, err := lightClient.NewLightClientFinalityUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err, "Error generating light client finality update")

View File

@@ -8,6 +8,7 @@ import (
"github.com/OffchainLabs/prysm/v6/async/event"
mock "github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/testing"
lightClient "github.com/OffchainLabs/prysm/v6/beacon-chain/core/light-client"
testDB "github.com/OffchainLabs/prysm/v6/beacon-chain/db/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p"
p2ptest "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/startup"
@@ -103,12 +104,13 @@ func TestValidateLightClientOptimisticUpdate(t *testing.T) {
// drift back appropriate number of epochs based on fork + 2 slots for signature slot + time for gossip propagation + any extra drift
genesisDrift := v*slotsPerEpoch*secondsPerSlot + 2*secondsPerSlot + secondsPerSlot/slotIntervals + test.genesisDrift
chainService := &mock.ChainService{Genesis: time.Unix(time.Now().Unix()-int64(genesisDrift), 0)}
s := &Service{cfg: &config{p2p: p, initialSync: &mockSync.Sync{}, clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot)}, lcStore: lightClient.NewLightClientStore(nil, &p2ptest.FakeP2P{}, new(event.Feed))}
lcStore := lightClient.NewLightClientStore(&p2ptest.FakeP2P{}, new(event.Feed), testDB.SetupDB(t))
s := &Service{cfg: &config{p2p: p, initialSync: &mockSync.Sync{}, clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot)}, lcStore: lcStore}
var oldUpdate interfaces.LightClientOptimisticUpdate
var err error
if test.oldUpdateOptions != nil {
l := util.NewTestLightClient(t, v, test.oldUpdateOptions...)
var err error
oldUpdate, err = lightClient.NewLightClientOptimisticUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock)
require.NoError(t, err)
@@ -218,12 +220,12 @@ func TestValidateLightClientFinalityUpdate(t *testing.T) {
{
name: "new update is better - supermajority",
oldUpdateOptions: []util.LightClientOption{},
newUpdateOptions: []util.LightClientOption{util.WithSupermajority()},
newUpdateOptions: []util.LightClientOption{util.WithSupermajority(0)},
expectedResult: pubsub.ValidationAccept,
},
{
name: "old update is better - supermajority",
oldUpdateOptions: []util.LightClientOption{util.WithSupermajority()},
oldUpdateOptions: []util.LightClientOption{util.WithSupermajority(0)},
newUpdateOptions: []util.LightClientOption{},
expectedResult: pubsub.ValidationIgnore,
},
@@ -243,12 +245,13 @@ func TestValidateLightClientFinalityUpdate(t *testing.T) {
// drift back appropriate number of epochs based on fork + 2 slots for signature slot + time for gossip propagation + any extra drift
genesisDrift := v*slotsPerEpoch*secondsPerSlot + 2*secondsPerSlot + secondsPerSlot/slotIntervals + test.genesisDrift
chainService := &mock.ChainService{Genesis: time.Unix(time.Now().Unix()-int64(genesisDrift), 0)}
s := &Service{cfg: &config{p2p: p, initialSync: &mockSync.Sync{}, clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot)}, lcStore: lightClient.NewLightClientStore(nil, &p2ptest.FakeP2P{}, new(event.Feed))}
lcStore := lightClient.NewLightClientStore(&p2ptest.FakeP2P{}, new(event.Feed), testDB.SetupDB(t))
s := &Service{cfg: &config{p2p: p, initialSync: &mockSync.Sync{}, clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot)}, lcStore: lcStore}
var oldUpdate interfaces.LightClientFinalityUpdate
var err error
if test.oldUpdateOptions != nil {
l := util.NewTestLightClient(t, v, test.oldUpdateOptions...)
var err error
oldUpdate, err = lightClient.NewLightClientFinalityUpdateFromBeaconState(l.Ctx, l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)

View File

@@ -0,0 +1,3 @@
### Changed
- Restrict best LC update collection to canonical blocks.

View File

@@ -56,6 +56,7 @@ type LightClientUpdate interface {
SetSyncAggregate(sa *pb.SyncAggregate)
SignatureSlot() primitives.Slot
SetSignatureSlot(slot primitives.Slot)
IsNil() bool
}
type LightClientFinalityUpdate interface {

View File

@@ -89,6 +89,10 @@ func NewWrappedUpdateAltair(p *pb.LightClientUpdateAltair) (interfaces.LightClie
}, nil
}
func (u *updateAltair) IsNil() bool {
return u == nil || u.p == nil
}
func (u *updateAltair) MarshalSSZTo(dst []byte) ([]byte, error) {
return u.p.MarshalSSZTo(dst)
}
@@ -256,6 +260,10 @@ func NewWrappedUpdateCapella(p *pb.LightClientUpdateCapella) (interfaces.LightCl
}, nil
}
func (u *updateCapella) IsNil() bool {
return u == nil || u.p == nil
}
func (u *updateCapella) MarshalSSZTo(dst []byte) ([]byte, error) {
return u.p.MarshalSSZTo(dst)
}
@@ -423,6 +431,10 @@ func NewWrappedUpdateDeneb(p *pb.LightClientUpdateDeneb) (interfaces.LightClient
}, nil
}
func (u *updateDeneb) IsNil() bool {
return u == nil || u.p == nil
}
func (u *updateDeneb) MarshalSSZTo(dst []byte) ([]byte, error) {
return u.p.MarshalSSZTo(dst)
}
@@ -591,6 +603,10 @@ func NewWrappedUpdateElectra(p *pb.LightClientUpdateElectra) (interfaces.LightCl
}, nil
}
func (u *updateElectra) IsNil() bool {
return u == nil || u.p == nil
}
func (u *updateElectra) MarshalSSZTo(dst []byte) ([]byte, error) {
return u.p.MarshalSSZTo(dst)
}

View File

@@ -30,6 +30,8 @@ type TestLightClient struct {
increaseAttestedSlotBy uint64
increaseFinalizedSlotBy uint64
increaseSignatureSlotBy uint64
increaseActiveParticipantsBy uint64
attestedParentRoot [32]byte
T *testing.T
Ctx context.Context
@@ -66,6 +68,13 @@ func NewTestLightClient(t *testing.T, forkVersion int, options ...LightClientOpt
}
}
// WithAttestedParentRoot sets the parent root of the attested block.
func WithAttestedParentRoot(parentRoot [32]byte) LightClientOption {
return func(l *TestLightClient) {
l.attestedParentRoot = parentRoot
}
}
// WithBlinded specifies whether the signature block is blinded or not
func WithBlinded() LightClientOption {
return func(l *TestLightClient) {
@@ -94,9 +103,10 @@ func WithFinalizedCheckpointInPrevFork() LightClientOption {
}
// WithSupermajority specifies whether the sync committee bits have supermajority or not
func WithSupermajority() LightClientOption {
func WithSupermajority(increaseActiveParticipantsBy uint64) LightClientOption {
return func(l *TestLightClient) {
l.supermajority = true
l.increaseActiveParticipantsBy = increaseActiveParticipantsBy
}
}
@@ -180,6 +190,7 @@ func (l *TestLightClient) setupTestAltair() *TestLightClient {
// Attested Block
attestedBlock := NewBeaconBlockAltair()
attestedBlock.Block.Slot = attestedSlot
attestedBlock.Block.ParentRoot = l.attestedParentRoot[:]
signedAttestedBlock, err := blocks.NewSignedBeaconBlock(attestedBlock)
require.NoError(l.T, err)
attestedBlockHeader, err := signedAttestedBlock.Header()
@@ -204,7 +215,7 @@ func (l *TestLightClient) setupTestAltair() *TestLightClient {
var trueBitNum uint64
if l.supermajority {
trueBitNum = uint64((float64(params.BeaconConfig().SyncCommitteeSize) * 2.0 / 3.0) + 1)
trueBitNum = uint64((float64(params.BeaconConfig().SyncCommitteeSize) * 2.0 / 3.0) + 1 + float64(l.increaseActiveParticipantsBy))
} else {
trueBitNum = params.BeaconConfig().MinSyncCommitteeParticipants
}
@@ -318,6 +329,7 @@ func (l *TestLightClient) setupTestBellatrix() *TestLightClient {
attestedBlock := NewBeaconBlockBellatrix()
attestedBlock.Block.Slot = attestedSlot
attestedBlock.Block.ParentRoot = l.attestedParentRoot[:]
signedAttestedBlock, err := blocks.NewSignedBeaconBlock(attestedBlock)
require.NoError(l.T, err)
attestedBlockHeader, err := signedAttestedBlock.Header()
@@ -344,7 +356,7 @@ func (l *TestLightClient) setupTestBellatrix() *TestLightClient {
var trueBitNum uint64
if l.supermajority {
trueBitNum = uint64((float64(params.BeaconConfig().SyncCommitteeSize) * 2.0 / 3.0) + 1)
trueBitNum = uint64((float64(params.BeaconConfig().SyncCommitteeSize) * 2.0 / 3.0) + 1 + float64(l.increaseActiveParticipantsBy))
} else {
trueBitNum = params.BeaconConfig().MinSyncCommitteeParticipants
}
@@ -494,6 +506,7 @@ func (l *TestLightClient) setupTestCapella() *TestLightClient {
// Attested Block
attestedBlock := NewBeaconBlockCapella()
attestedBlock.Block.Slot = attestedSlot
attestedBlock.Block.ParentRoot = l.attestedParentRoot[:]
signedAttestedBlock, err := blocks.NewSignedBeaconBlock(attestedBlock)
require.NoError(l.T, err)
attestedBlockHeader, err := signedAttestedBlock.Header()
@@ -520,7 +533,7 @@ func (l *TestLightClient) setupTestCapella() *TestLightClient {
var trueBitNum uint64
if l.supermajority {
trueBitNum = uint64((float64(params.BeaconConfig().SyncCommitteeSize) * 2.0 / 3.0) + 1)
trueBitNum = uint64((float64(params.BeaconConfig().SyncCommitteeSize) * 2.0 / 3.0) + 1 + float64(l.increaseActiveParticipantsBy))
} else {
trueBitNum = params.BeaconConfig().MinSyncCommitteeParticipants
}
@@ -551,7 +564,7 @@ func (l *TestLightClient) setupTestCapella() *TestLightClient {
var trueBitNum uint64
if l.supermajority {
trueBitNum = uint64((float64(params.BeaconConfig().SyncCommitteeSize) * 2.0 / 3.0) + 1)
trueBitNum = uint64((float64(params.BeaconConfig().SyncCommitteeSize) * 2.0 / 3.0) + 1 + float64(l.increaseActiveParticipantsBy))
} else {
trueBitNum = params.BeaconConfig().MinSyncCommitteeParticipants
}
@@ -671,6 +684,7 @@ func (l *TestLightClient) setupTestDeneb() *TestLightClient {
// Attested Block
attestedBlock := NewBeaconBlockDeneb()
attestedBlock.Block.Slot = attestedSlot
attestedBlock.Block.ParentRoot = l.attestedParentRoot[:]
signedAttestedBlock, err := blocks.NewSignedBeaconBlock(attestedBlock)
require.NoError(l.T, err)
attestedBlockHeader, err := signedAttestedBlock.Header()
@@ -697,7 +711,7 @@ func (l *TestLightClient) setupTestDeneb() *TestLightClient {
var trueBitNum uint64
if l.supermajority {
trueBitNum = uint64((float64(params.BeaconConfig().SyncCommitteeSize) * 2.0 / 3.0) + 1)
trueBitNum = uint64((float64(params.BeaconConfig().SyncCommitteeSize) * 2.0 / 3.0) + 1 + float64(l.increaseActiveParticipantsBy))
} else {
trueBitNum = params.BeaconConfig().MinSyncCommitteeParticipants
}
@@ -728,7 +742,7 @@ func (l *TestLightClient) setupTestDeneb() *TestLightClient {
var trueBitNum uint64
if l.supermajority {
trueBitNum = uint64((float64(params.BeaconConfig().SyncCommitteeSize) * 2.0 / 3.0) + 1)
trueBitNum = uint64((float64(params.BeaconConfig().SyncCommitteeSize) * 2.0 / 3.0) + 1 + float64(l.increaseActiveParticipantsBy))
} else {
trueBitNum = params.BeaconConfig().MinSyncCommitteeParticipants
}
@@ -848,6 +862,7 @@ func (l *TestLightClient) setupTestElectra() *TestLightClient {
// Attested Block
attestedBlock := NewBeaconBlockElectra()
attestedBlock.Block.Slot = attestedSlot
attestedBlock.Block.ParentRoot = l.attestedParentRoot[:]
signedAttestedBlock, err := blocks.NewSignedBeaconBlock(attestedBlock)
require.NoError(l.T, err)
attestedBlockHeader, err := signedAttestedBlock.Header()
@@ -874,7 +889,7 @@ func (l *TestLightClient) setupTestElectra() *TestLightClient {
var trueBitNum uint64
if l.supermajority {
trueBitNum = uint64((float64(params.BeaconConfig().SyncCommitteeSize) * 2.0 / 3.0) + 1)
trueBitNum = uint64((float64(params.BeaconConfig().SyncCommitteeSize) * 2.0 / 3.0) + 1 + float64(l.increaseActiveParticipantsBy))
} else {
trueBitNum = params.BeaconConfig().MinSyncCommitteeParticipants
}
@@ -905,7 +920,7 @@ func (l *TestLightClient) setupTestElectra() *TestLightClient {
var trueBitNum uint64
if l.supermajority {
trueBitNum = uint64((float64(params.BeaconConfig().SyncCommitteeSize) * 2.0 / 3.0) + 1)
trueBitNum = uint64((float64(params.BeaconConfig().SyncCommitteeSize) * 2.0 / 3.0) + 1 + float64(l.increaseActiveParticipantsBy))
} else {
trueBitNum = params.BeaconConfig().MinSyncCommitteeParticipants
}

View File

@@ -53,7 +53,7 @@ func TestLightClientUtils(t *testing.T) {
for i := 1; i < 6; i++ {
t.Run(version.String(i), func(t *testing.T) {
l1 := util.NewTestLightClient(t, i)
l2 := util.NewTestLightClient(t, i, util.WithSupermajority())
l2 := util.NewTestLightClient(t, i, util.WithSupermajority(0))
l1SyncAgg, err := l1.Block.Block().Body().SyncAggregate()
require.NoError(t, err)
l1Bits := l1SyncAgg.SyncCommitteeBits.Count()