Add LC p2p validators and subscribers (#15214)

* add and pass lcstore to sync service

* validator for optimistic updates

* validator for finality updates

* subscribers

* gossip scorings

* tmp - add validation test

* optimistic update validation tests

* finality update validation tests

* tests for subscribers

* deps

* changelog entry

* play around with config cleanup

* better logs

* better logs 2

* better logs 3

* address comments

* add comments
This commit is contained in:
Bastin
2025-04-29 16:07:17 +02:00
committed by GitHub
parent 1298dc3a46
commit b69c71d65a
17 changed files with 845 additions and 0 deletions

View File

@@ -865,6 +865,7 @@ func (b *BeaconNode) registerSyncService(initialSyncComplete chan struct{}, bFil
regularsync.WithVerifierWaiter(b.verifyInitWaiter),
regularsync.WithAvailableBlocker(bFillStore),
regularsync.WithSlasherEnabled(b.slasherEnabled),
regularsync.WithLightClientStore(b.lcStore),
)
return b.services.RegisterService(rs)
}

View File

@@ -44,6 +44,12 @@ const (
// blsToExecutionChangeWeight specifies the scoring weight that we apply to
// our bls to execution topic.
blsToExecutionChangeWeight = 0.05
// lightClientOptimisticUpdateWeight specifies the scoring weight that we apply to
// our light client optimistic update topic.
lightClientOptimisticUpdateWeight = 0.05
// lightClientFinalityUpdateWeight specifies the scoring weight that we apply to
// our light client finality update topic.
lightClientFinalityUpdateWeight = 0.05
// maxInMeshScore describes the max score a peer can attain from being in the mesh.
maxInMeshScore = 10
@@ -124,6 +130,10 @@ func (s *Service) topicScoreParams(topic string) (*pubsub.TopicScoreParams, erro
case strings.Contains(topic, GossipBlobSidecarMessage):
// TODO(Deneb): Using the default block scoring. But this should be updated.
return defaultBlockTopicParams(), nil
case strings.Contains(topic, GossipLightClientOptimisticUpdateMessage):
return defaultLightClientOptimisticUpdateTopicParams(), nil
case strings.Contains(topic, GossipLightClientFinalityUpdateMessage):
return defaultLightClientFinalityUpdateTopicParams(), nil
default:
return nil, errors.Errorf("unrecognized topic provided for parameter registration: %s", topic)
}
@@ -503,6 +513,50 @@ func defaultBlsToExecutionChangeTopicParams() *pubsub.TopicScoreParams {
}
}
func defaultLightClientOptimisticUpdateTopicParams() *pubsub.TopicScoreParams {
return &pubsub.TopicScoreParams{
TopicWeight: lightClientOptimisticUpdateWeight,
TimeInMeshWeight: maxInMeshScore / inMeshCap(),
TimeInMeshQuantum: inMeshTime(),
TimeInMeshCap: inMeshCap(),
FirstMessageDeliveriesWeight: 2,
FirstMessageDeliveriesDecay: scoreDecay(oneHundredEpochs),
FirstMessageDeliveriesCap: 5,
MeshMessageDeliveriesWeight: 0,
MeshMessageDeliveriesDecay: 0,
MeshMessageDeliveriesCap: 0,
MeshMessageDeliveriesThreshold: 0,
MeshMessageDeliveriesWindow: 0,
MeshMessageDeliveriesActivation: 0,
MeshFailurePenaltyWeight: 0,
MeshFailurePenaltyDecay: 0,
InvalidMessageDeliveriesWeight: -2000,
InvalidMessageDeliveriesDecay: scoreDecay(invalidDecayPeriod),
}
}
func defaultLightClientFinalityUpdateTopicParams() *pubsub.TopicScoreParams {
return &pubsub.TopicScoreParams{
TopicWeight: lightClientFinalityUpdateWeight,
TimeInMeshWeight: maxInMeshScore / inMeshCap(),
TimeInMeshQuantum: inMeshTime(),
TimeInMeshCap: inMeshCap(),
FirstMessageDeliveriesWeight: 2,
FirstMessageDeliveriesDecay: scoreDecay(oneHundredEpochs),
FirstMessageDeliveriesCap: 5,
MeshMessageDeliveriesWeight: 0,
MeshMessageDeliveriesDecay: 0,
MeshMessageDeliveriesCap: 0,
MeshMessageDeliveriesThreshold: 0,
MeshMessageDeliveriesWindow: 0,
MeshMessageDeliveriesActivation: 0,
MeshFailurePenaltyWeight: 0,
MeshFailurePenaltyDecay: 0,
InvalidMessageDeliveriesWeight: -2000,
InvalidMessageDeliveriesDecay: scoreDecay(invalidDecayPeriod),
}
}
func oneSlotDuration() time.Duration {
return time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Second
}

View File

@@ -75,4 +75,6 @@ func TestLoggingParameters(_ *testing.T) {
logGossipParameters("testing", defaultAttesterSlashingTopicParams())
logGossipParameters("testing", defaultProposerSlashingTopicParams())
logGossipParameters("testing", defaultVoluntaryExitTopicParams())
logGossipParameters("testing", defaultLightClientOptimisticUpdateTopicParams())
logGossipParameters("testing", defaultLightClientFinalityUpdateTopicParams())
}

View File

@@ -73,6 +73,12 @@ func TestGossipTopicMappings_CorrectType(t *testing.T) {
pMessage = GossipTopicMappings(AggregateAndProofSubnetTopicFormat, altairForkEpoch)
_, ok = pMessage.(*ethpb.SignedAggregateAttestationAndProof)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(LightClientOptimisticUpdateTopicFormat, altairForkEpoch)
_, ok = pMessage.(*ethpb.LightClientOptimisticUpdateAltair)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(LightClientFinalityUpdateTopicFormat, altairForkEpoch)
_, ok = pMessage.(*ethpb.LightClientFinalityUpdateAltair)
assert.Equal(t, true, ok)
// Bellatrix Fork
pMessage = GossipTopicMappings(BlockSubnetTopicFormat, bellatrixForkEpoch)
@@ -87,6 +93,12 @@ func TestGossipTopicMappings_CorrectType(t *testing.T) {
pMessage = GossipTopicMappings(AggregateAndProofSubnetTopicFormat, bellatrixForkEpoch)
_, ok = pMessage.(*ethpb.SignedAggregateAttestationAndProof)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(LightClientOptimisticUpdateTopicFormat, bellatrixForkEpoch)
_, ok = pMessage.(*ethpb.LightClientOptimisticUpdateAltair)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(LightClientFinalityUpdateTopicFormat, bellatrixForkEpoch)
_, ok = pMessage.(*ethpb.LightClientFinalityUpdateAltair)
assert.Equal(t, true, ok)
// Capella Fork
pMessage = GossipTopicMappings(BlockSubnetTopicFormat, capellaForkEpoch)
@@ -101,6 +113,12 @@ func TestGossipTopicMappings_CorrectType(t *testing.T) {
pMessage = GossipTopicMappings(AggregateAndProofSubnetTopicFormat, capellaForkEpoch)
_, ok = pMessage.(*ethpb.SignedAggregateAttestationAndProof)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(LightClientOptimisticUpdateTopicFormat, capellaForkEpoch)
_, ok = pMessage.(*ethpb.LightClientOptimisticUpdateCapella)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(LightClientFinalityUpdateTopicFormat, capellaForkEpoch)
_, ok = pMessage.(*ethpb.LightClientFinalityUpdateCapella)
assert.Equal(t, true, ok)
// Deneb Fork
pMessage = GossipTopicMappings(BlockSubnetTopicFormat, denebForkEpoch)
@@ -115,6 +133,12 @@ func TestGossipTopicMappings_CorrectType(t *testing.T) {
pMessage = GossipTopicMappings(AggregateAndProofSubnetTopicFormat, denebForkEpoch)
_, ok = pMessage.(*ethpb.SignedAggregateAttestationAndProof)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(LightClientOptimisticUpdateTopicFormat, denebForkEpoch)
_, ok = pMessage.(*ethpb.LightClientOptimisticUpdateDeneb)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(LightClientFinalityUpdateTopicFormat, denebForkEpoch)
_, ok = pMessage.(*ethpb.LightClientFinalityUpdateDeneb)
assert.Equal(t, true, ok)
// Electra Fork
pMessage = GossipTopicMappings(BlockSubnetTopicFormat, electraForkEpoch)
@@ -129,4 +153,10 @@ func TestGossipTopicMappings_CorrectType(t *testing.T) {
pMessage = GossipTopicMappings(AggregateAndProofSubnetTopicFormat, electraForkEpoch)
_, ok = pMessage.(*ethpb.SignedAggregateAttestationAndProofElectra)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(LightClientOptimisticUpdateTopicFormat, electraForkEpoch)
_, ok = pMessage.(*ethpb.LightClientOptimisticUpdateDeneb)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(LightClientFinalityUpdateTopicFormat, electraForkEpoch)
_, ok = pMessage.(*ethpb.LightClientFinalityUpdateElectra)
assert.Equal(t, true, ok)
}

View File

@@ -21,6 +21,7 @@ go_library(
"//config/params: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",
"//consensus-types/wrapper:go_default_library",
"//encoding/bytesutil:go_default_library",

View File

@@ -4,6 +4,7 @@ import (
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v6/consensus-types/interfaces"
lightclientConsensusTypes "github.com/OffchainLabs/prysm/v6/consensus-types/light-client"
"github.com/OffchainLabs/prysm/v6/consensus-types/wrapper"
"github.com/OffchainLabs/prysm/v6/encoding/bytesutil"
enginev1 "github.com/OffchainLabs/prysm/v6/proto/engine/v1"
@@ -36,6 +37,12 @@ var (
// AttesterSlashingMap maps the fork-version to the underlying data type for that particular
// fork period.
AttesterSlashingMap map[[4]byte]func() (ethpb.AttSlashing, error)
// LightClientOptimisticUpdateMap maps the fork-version to the underlying data type for that
// particular fork period.
LightClientOptimisticUpdateMap map[[4]byte]func() (interfaces.LightClientOptimisticUpdate, error)
// LightClientFinalityUpdateMap maps the fork-version to the underlying data type for that
// particular fork period.
LightClientFinalityUpdateMap map[[4]byte]func() (interfaces.LightClientFinalityUpdate, error)
)
// InitializeDataMaps initializes all the relevant object maps. This function is called to
@@ -179,4 +186,42 @@ func InitializeDataMaps() {
return &ethpb.AttesterSlashingElectra{}, nil
},
}
// Reset our light client optimistic update map.
LightClientOptimisticUpdateMap = map[[4]byte]func() (interfaces.LightClientOptimisticUpdate, error){
bytesutil.ToBytes4(params.BeaconConfig().AltairForkVersion): func() (interfaces.LightClientOptimisticUpdate, error) {
return lightclientConsensusTypes.NewEmptyOptimisticUpdateAltair(), nil
},
bytesutil.ToBytes4(params.BeaconConfig().BellatrixForkVersion): func() (interfaces.LightClientOptimisticUpdate, error) {
return lightclientConsensusTypes.NewEmptyOptimisticUpdateAltair(), nil
},
bytesutil.ToBytes4(params.BeaconConfig().CapellaForkVersion): func() (interfaces.LightClientOptimisticUpdate, error) {
return lightclientConsensusTypes.NewEmptyOptimisticUpdateCapella(), nil
},
bytesutil.ToBytes4(params.BeaconConfig().DenebForkVersion): func() (interfaces.LightClientOptimisticUpdate, error) {
return lightclientConsensusTypes.NewEmptyOptimisticUpdateDeneb(), nil
},
bytesutil.ToBytes4(params.BeaconConfig().ElectraForkVersion): func() (interfaces.LightClientOptimisticUpdate, error) {
return lightclientConsensusTypes.NewEmptyOptimisticUpdateDeneb(), nil
},
}
// Reset our light client finality update map.
LightClientFinalityUpdateMap = map[[4]byte]func() (interfaces.LightClientFinalityUpdate, error){
bytesutil.ToBytes4(params.BeaconConfig().AltairForkVersion): func() (interfaces.LightClientFinalityUpdate, error) {
return lightclientConsensusTypes.NewEmptyFinalityUpdateAltair(), nil
},
bytesutil.ToBytes4(params.BeaconConfig().BellatrixForkVersion): func() (interfaces.LightClientFinalityUpdate, error) {
return lightclientConsensusTypes.NewEmptyFinalityUpdateAltair(), nil
},
bytesutil.ToBytes4(params.BeaconConfig().CapellaForkVersion): func() (interfaces.LightClientFinalityUpdate, error) {
return lightclientConsensusTypes.NewEmptyFinalityUpdateCapella(), nil
},
bytesutil.ToBytes4(params.BeaconConfig().DenebForkVersion): func() (interfaces.LightClientFinalityUpdate, error) {
return lightclientConsensusTypes.NewEmptyFinalityUpdateDeneb(), nil
},
bytesutil.ToBytes4(params.BeaconConfig().ElectraForkVersion): func() (interfaces.LightClientFinalityUpdate, error) {
return lightclientConsensusTypes.NewEmptyFinalityUpdateElectra(), nil
},
}
}

View File

@@ -38,6 +38,7 @@ go_library(
"subscriber_blob_sidecar.go",
"subscriber_bls_to_execution_change.go",
"subscriber_handlers.go",
"subscriber_light_client.go",
"subscriber_sync_committee_message.go",
"subscriber_sync_contribution_proof.go",
"subscription_topic_handler.go",
@@ -47,6 +48,7 @@ go_library(
"validate_beacon_blocks.go",
"validate_blob.go",
"validate_bls_to_execution_change.go",
"validate_light_client.go",
"validate_proposer_slashing.go",
"validate_sync_committee_message.go",
"validate_sync_contribution_proof.go",
@@ -71,6 +73,7 @@ go_library(
"//beacon-chain/core/feed/operation:go_default_library",
"//beacon-chain/core/feed/state:go_default_library",
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/core/light-client:go_default_library",
"//beacon-chain/core/signing:go_default_library",
"//beacon-chain/core/transition:go_default_library",
"//beacon-chain/core/transition/interop:go_default_library",
@@ -101,6 +104,7 @@ go_library(
"//config/params: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",
"//consensus-types/wrapper:go_default_library",
"//container/leaky-bucket:go_default_library",
@@ -182,6 +186,7 @@ go_test(
"validate_beacon_blocks_test.go",
"validate_blob_test.go",
"validate_bls_to_execution_change_test.go",
"validate_light_client_test.go",
"validate_proposer_slashing_test.go",
"validate_sync_committee_message_test.go",
"validate_sync_contribution_proof_test.go",
@@ -198,6 +203,7 @@ go_test(
"//beacon-chain/core/feed:go_default_library",
"//beacon-chain/core/feed/operation:go_default_library",
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/core/light-client:go_default_library",
"//beacon-chain/core/signing:go_default_library",
"//beacon-chain/core/time:go_default_library",
"//beacon-chain/core/transition:go_default_library",

View File

@@ -89,6 +89,10 @@ func extractValidDataTypeFromTopic(topic string, digest []byte, clock *startup.C
return extractDataTypeFromTypeMap(types.AggregateAttestationMap, digest, clock)
case p2p.AttesterSlashingSubnetTopicFormat:
return extractDataTypeFromTypeMap(types.AttesterSlashingMap, digest, clock)
case p2p.LightClientOptimisticUpdateTopicFormat:
return extractDataTypeFromTypeMap(types.LightClientOptimisticUpdateMap, digest, clock)
case p2p.LightClientFinalityUpdateTopicFormat:
return extractDataTypeFromTypeMap(types.LightClientFinalityUpdateMap, digest, clock)
}
return nil, nil
}

View File

@@ -6,6 +6,7 @@ import (
blockfeed "github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed/block"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed/operation"
statefeed "github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed/state"
lightClient "github.com/OffchainLabs/prysm/v6/beacon-chain/core/light-client"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db/filesystem"
"github.com/OffchainLabs/prysm/v6/beacon-chain/execution"
@@ -196,3 +197,11 @@ func WithSlasherEnabled(enabled bool) Option {
return nil
}
}
// WithLightClientStore allows the sync package to access light client data.
func WithLightClientStore(lcs *lightClient.Store) Option {
return func(s *Service) error {
s.lcStore = lcs
return nil
}
}

View File

@@ -9,6 +9,7 @@ import (
"sync"
"time"
lightClient "github.com/OffchainLabs/prysm/v6/beacon-chain/core/light-client"
lru "github.com/hashicorp/golang-lru"
pubsub "github.com/libp2p/go-libp2p-pubsub"
libp2pcore "github.com/libp2p/go-libp2p/core"
@@ -165,6 +166,7 @@ type Service struct {
availableBlocker coverage.AvailableBlocker
ctxMap ContextByteVersions
slasherEnabled bool
lcStore *lightClient.Store
}
// NewService initializes new regular sync service.

View File

@@ -0,0 +1,66 @@
package sync
import (
"context"
"fmt"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed"
statefeed "github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed/state"
lightclientTypes "github.com/OffchainLabs/prysm/v6/consensus-types/light-client"
"github.com/sirupsen/logrus"
"google.golang.org/protobuf/proto"
)
func (s *Service) lightClientOptimisticUpdateSubscriber(_ context.Context, msg proto.Message) error {
update, err := lightclientTypes.NewWrappedOptimisticUpdate(msg)
if err != nil {
return err
}
attestedHeaderRoot, err := update.AttestedHeader().Beacon().HashTreeRoot()
if err != nil {
return err
}
log.WithFields(logrus.Fields{
"attestedSlot": fmt.Sprintf("%d", update.AttestedHeader().Beacon().Slot),
"signatureSlot": fmt.Sprintf("%d", update.SignatureSlot()),
"attestedHeaderRoot": fmt.Sprintf("%x", attestedHeaderRoot),
}).Debug("Saving newly received light client optimistic update.")
s.lcStore.SetLastOptimisticUpdate(update)
s.cfg.stateNotifier.StateFeed().Send(&feed.Event{
Type: statefeed.LightClientOptimisticUpdate,
Data: update,
})
return nil
}
func (s *Service) lightClientFinalityUpdateSubscriber(_ context.Context, msg proto.Message) error {
update, err := lightclientTypes.NewWrappedFinalityUpdate(msg)
if err != nil {
return err
}
attestedHeaderRoot, err := update.AttestedHeader().Beacon().HashTreeRoot()
if err != nil {
return err
}
log.WithFields(logrus.Fields{
"attestedSlot": fmt.Sprintf("%d", update.AttestedHeader().Beacon().Slot),
"signatureSlot": fmt.Sprintf("%d", update.SignatureSlot()),
"attestedHeaderRoot": fmt.Sprintf("%x", attestedHeaderRoot),
}).Debug("Saving newly received light client finality update.")
s.lcStore.SetLastFinalityUpdate(update)
s.cfg.stateNotifier.StateFeed().Send(&feed.Event{
Type: statefeed.LightClientFinalityUpdate,
Data: update,
})
return nil
}

View File

@@ -11,6 +11,7 @@ import (
"github.com/OffchainLabs/prysm/v6/async/abool"
mockChain "github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/cache"
lightClient "github.com/OffchainLabs/prysm/v6/beacon-chain/core/light-client"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/signing"
db "github.com/OffchainLabs/prysm/v6/beacon-chain/db/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/operations/slashings"
@@ -27,6 +28,7 @@ import (
"github.com/OffchainLabs/prysm/v6/encoding/bytesutil"
"github.com/OffchainLabs/prysm/v6/network/forks"
pb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v6/runtime/version"
"github.com/OffchainLabs/prysm/v6/testing/assert"
"github.com/OffchainLabs/prysm/v6/testing/require"
"github.com/OffchainLabs/prysm/v6/testing/util"
@@ -638,3 +640,137 @@ func createPeer(t *testing.T, topics ...string) *p2ptest.TestP2P {
}
return p
}
func TestSubscribe_ReceivesLCOptimisticUpdate(t *testing.T) {
origNC := params.BeaconConfig()
// restore network config after test completes
defer func() {
params.OverrideBeaconConfig(origNC)
}()
params.SetupTestConfigCleanup(t)
p2pService := p2ptest.NewTestP2P(t)
ctx := context.Background()
cfg := params.BeaconConfig().Copy()
cfg.AltairForkEpoch = 1
cfg.ForkVersionSchedule[[4]byte{1, 0, 0, 0}] = 1
params.OverrideBeaconConfig(cfg)
secondsPerSlot := int(params.BeaconConfig().SecondsPerSlot)
slotIntervals := int(params.BeaconConfig().IntervalsPerSlot)
slotsPerEpoch := int(params.BeaconConfig().SlotsPerEpoch)
genesisDrift := slotsPerEpoch*secondsPerSlot + 2*secondsPerSlot + secondsPerSlot/slotIntervals
chainService := &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Unix(time.Now().Unix()-int64(genesisDrift), 0),
}
d := db.SetupDB(t)
r := Service{
ctx: ctx,
cfg: &config{
p2p: p2pService,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: chainService,
beaconDB: d,
clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot),
stateNotifier: &mockChain.MockStateNotifier{},
},
chainStarted: abool.New(),
lcStore: &lightClient.Store{},
subHandler: newSubTopicHandler(),
}
topic := p2p.LightClientOptimisticUpdateTopicFormat
var wg sync.WaitGroup
wg.Add(1)
var err error
p2pService.Digest, err = r.currentForkDigest()
require.NoError(t, err)
r.subscribe(topic, r.validateLightClientOptimisticUpdate, func(ctx context.Context, msg proto.Message) error {
require.NoError(t, r.lightClientOptimisticUpdateSubscriber(ctx, msg))
wg.Done()
return nil
}, p2pService.Digest)
r.markForChainStart()
l := util.NewTestLightClient(t, version.Altair, util.WithSupermajority())
update, err := lightClient.NewLightClientOptimisticUpdateFromBeaconState(l.Ctx, l.State.Slot(), l.State, l.Block, l.AttestedState, l.AttestedBlock)
require.NoError(t, err, "Error generating light client optimistic update")
p2pService.ReceivePubSub(topic, update.Proto())
if util.WaitTimeout(&wg, time.Second) {
t.Fatal("Did not receive PubSub in 1 second")
}
u := r.lcStore.LastOptimisticUpdate()
assert.DeepEqual(t, update.Proto(), u.Proto())
}
func TestSubscribe_ReceivesLCFinalityUpdate(t *testing.T) {
origNC := params.BeaconConfig()
// restore network config after test completes
defer func() {
params.OverrideBeaconConfig(origNC)
}()
params.SetupTestConfigCleanup(t)
p2pService := p2ptest.NewTestP2P(t)
ctx := context.Background()
cfg := params.BeaconConfig().Copy()
cfg.AltairForkEpoch = 1
cfg.ForkVersionSchedule[[4]byte{1, 0, 0, 0}] = 1
params.OverrideBeaconConfig(cfg)
secondsPerSlot := int(params.BeaconConfig().SecondsPerSlot)
slotIntervals := int(params.BeaconConfig().IntervalsPerSlot)
slotsPerEpoch := int(params.BeaconConfig().SlotsPerEpoch)
genesisDrift := slotsPerEpoch*secondsPerSlot + 2*secondsPerSlot + secondsPerSlot/slotIntervals
chainService := &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Unix(time.Now().Unix()-int64(genesisDrift), 0),
}
d := db.SetupDB(t)
r := Service{
ctx: ctx,
cfg: &config{
p2p: p2pService,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: chainService,
beaconDB: d,
clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot),
stateNotifier: &mockChain.MockStateNotifier{},
},
chainStarted: abool.New(),
lcStore: &lightClient.Store{},
subHandler: newSubTopicHandler(),
}
topic := p2p.LightClientFinalityUpdateTopicFormat
var wg sync.WaitGroup
wg.Add(1)
var err error
p2pService.Digest, err = r.currentForkDigest()
require.NoError(t, err)
r.subscribe(topic, r.validateLightClientFinalityUpdate, func(ctx context.Context, msg proto.Message) error {
require.NoError(t, r.lightClientFinalityUpdateSubscriber(ctx, msg))
wg.Done()
return nil
}, p2pService.Digest)
r.markForChainStart()
l := util.NewTestLightClient(t, version.Altair, util.WithSupermajority())
update, err := lightClient.NewLightClientFinalityUpdateFromBeaconState(l.Ctx, l.State.Slot(), l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err, "Error generating light client finality update")
p2pService.ReceivePubSub(topic, update.Proto())
if util.WaitTimeout(&wg, time.Second) {
t.Fatal("Did not receive PubSub in 1 second")
}
u := r.lcStore.LastFinalityUpdate()
assert.DeepEqual(t, update.Proto(), u.Proto())
}

View File

@@ -0,0 +1,160 @@
package sync
import (
"context"
"fmt"
"time"
lightClient "github.com/OffchainLabs/prysm/v6/beacon-chain/core/light-client"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v6/monitoring/tracing"
"github.com/OffchainLabs/prysm/v6/monitoring/tracing/trace"
"github.com/OffchainLabs/prysm/v6/time/slots"
pubsub "github.com/libp2p/go-libp2p-pubsub"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/sirupsen/logrus"
)
func (s *Service) validateLightClientOptimisticUpdate(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
// Validation runs on publish (not just subscriptions), so we should approve any message from
// ourselves.
if pid == s.cfg.p2p.PeerID() {
return pubsub.ValidationAccept, nil
}
// Ignore updates while syncing
if s.cfg.initialSync.Syncing() {
return pubsub.ValidationIgnore, nil
}
_, span := trace.StartSpan(ctx, "sync.validateLightClientOptimisticUpdate")
defer span.End()
m, err := s.decodePubsubMessage(msg)
if err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationReject, err
}
newUpdate, ok := m.(interfaces.LightClientOptimisticUpdate)
if !ok {
return pubsub.ValidationReject, errWrongMessage
}
attestedHeaderRoot, err := newUpdate.AttestedHeader().Beacon().HashTreeRoot()
if err != nil {
return pubsub.ValidationIgnore, err
}
// [IGNORE] The optimistic_update is received after the block at signature_slot was given enough time
// to propagate through the network -- i.e. validate that one-third of optimistic_update.signature_slot
// has transpired (SECONDS_PER_SLOT / INTERVALS_PER_SLOT seconds after the start of the slot,
// with a MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance)
earliestValidTime := slots.StartTime(uint64(s.cfg.clock.GenesisTime().Unix()), newUpdate.SignatureSlot()).
Add(time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot/params.BeaconConfig().IntervalsPerSlot)).
Add(-params.BeaconConfig().MaximumGossipClockDisparityDuration())
if s.cfg.clock.Now().Before(earliestValidTime) {
log.Debug("Newly received light client optimistic update ignored. not enough time passed for block to propagate")
return pubsub.ValidationIgnore, nil
}
lastStoredUpdate := s.lcStore.LastOptimisticUpdate()
if lastStoredUpdate != nil {
lastUpdateSlot := lastStoredUpdate.AttestedHeader().Beacon().Slot
newUpdateSlot := newUpdate.AttestedHeader().Beacon().Slot
// [IGNORE] The attested_header.beacon.slot is greater than that of all previously forwarded optimistic_updates
if newUpdateSlot <= lastUpdateSlot {
log.Debug("Newly received light client optimistic update ignored. new update is older than stored update")
return pubsub.ValidationIgnore, nil
}
}
log.WithFields(logrus.Fields{
"attestedSlot": fmt.Sprintf("%d", newUpdate.AttestedHeader().Beacon().Slot),
"signatureSlot": fmt.Sprintf("%d", newUpdate.SignatureSlot()),
"attestedHeaderRoot": fmt.Sprintf("%x", attestedHeaderRoot),
}).Debug("New gossiped light client optimistic update validated.")
msg.ValidatorData = newUpdate.Proto()
return pubsub.ValidationAccept, nil
}
func (s *Service) validateLightClientFinalityUpdate(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
// Validation runs on publish (not just subscriptions), so we should approve any message from
// ourselves.
if pid == s.cfg.p2p.PeerID() {
return pubsub.ValidationAccept, nil
}
// Ignore updates while syncing
if s.cfg.initialSync.Syncing() {
return pubsub.ValidationIgnore, nil
}
_, span := trace.StartSpan(ctx, "sync.validateLightClientFinalityUpdate")
defer span.End()
m, err := s.decodePubsubMessage(msg)
if err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationReject, err
}
newUpdate, ok := m.(interfaces.LightClientFinalityUpdate)
if !ok {
return pubsub.ValidationReject, errWrongMessage
}
attestedHeaderRoot, err := newUpdate.AttestedHeader().Beacon().HashTreeRoot()
if err != nil {
return pubsub.ValidationIgnore, err
}
// [IGNORE] The optimistic_update is received after the block at signature_slot was given enough time
// to propagate through the network -- i.e. validate that one-third of optimistic_update.signature_slot
// has transpired (SECONDS_PER_SLOT / INTERVALS_PER_SLOT seconds after the start of the slot,
// with a MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance)
earliestValidTime := slots.StartTime(uint64(s.cfg.clock.GenesisTime().Unix()), newUpdate.SignatureSlot()).
Add(time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot/params.BeaconConfig().IntervalsPerSlot)).
Add(-params.BeaconConfig().MaximumGossipClockDisparityDuration())
if s.cfg.clock.Now().Before(earliestValidTime) {
log.Debug("Newly received light client finality update ignored. not enough time passed for block to propagate")
return pubsub.ValidationIgnore, nil
}
lastStoredUpdate := s.lcStore.LastFinalityUpdate()
if lastStoredUpdate != nil {
lastUpdateSlot := lastStoredUpdate.FinalizedHeader().Beacon().Slot
newUpdateSlot := newUpdate.FinalizedHeader().Beacon().Slot
// [IGNORE] The finalized_header.beacon.slot is greater than that of all previously forwarded finality_updates,
// or it matches the highest previously forwarded slot and also has a sync_aggregate indicating supermajority (> 2/3)
// sync committee participation while the previously forwarded finality_update for that slot did not indicate supermajority
lastUpdateHasSupermajority := lightClient.UpdateHasSupermajority(lastStoredUpdate.SyncAggregate())
newUpdateHasSupermajority := lightClient.UpdateHasSupermajority(newUpdate.SyncAggregate())
if newUpdateSlot < lastUpdateSlot {
log.Debug("Newly received light client finality update ignored. new update is older than stored update")
return pubsub.ValidationIgnore, nil
}
if newUpdateSlot == lastUpdateSlot && (lastUpdateHasSupermajority || !newUpdateHasSupermajority) {
log.WithFields(logrus.Fields{
"attestedSlot": fmt.Sprintf("%d", newUpdate.AttestedHeader().Beacon().Slot),
"signatureSlot": fmt.Sprintf("%d", newUpdate.SignatureSlot()),
"attestedHeaderRoot": fmt.Sprintf("%x", attestedHeaderRoot),
}).Debug("Newly received light client finality update ignored. no supermajority advantage.")
return pubsub.ValidationIgnore, nil
}
}
log.WithFields(logrus.Fields{
"attestedSlot": fmt.Sprintf("%d", newUpdate.AttestedHeader().Beacon().Slot),
"signatureSlot": fmt.Sprintf("%d", newUpdate.SignatureSlot()),
"attestedHeaderRoot": fmt.Sprintf("%x", attestedHeaderRoot),
}).Debug("New gossiped light client finality update validated.")
msg.ValidatorData = newUpdate.Proto()
return pubsub.ValidationAccept, nil
}

View File

@@ -0,0 +1,284 @@
package sync
import (
"bytes"
"context"
"testing"
"time"
mock "github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/testing"
lightClient "github.com/OffchainLabs/prysm/v6/beacon-chain/core/light-client"
"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"
mockSync "github.com/OffchainLabs/prysm/v6/beacon-chain/sync/initial-sync/testing"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v6/runtime/version"
"github.com/OffchainLabs/prysm/v6/testing/require"
"github.com/OffchainLabs/prysm/v6/testing/util"
pubsub "github.com/libp2p/go-libp2p-pubsub"
pb "github.com/libp2p/go-libp2p-pubsub/pb"
)
func TestValidateLightClientOptimisticUpdate_NilMessageOrTopic(t *testing.T) {
params.SetupTestConfigCleanup(t)
ctx := context.Background()
p := p2ptest.NewTestP2P(t)
s := &Service{cfg: &config{p2p: p, initialSync: &mockSync.Sync{}}}
_, err := s.validateLightClientOptimisticUpdate(ctx, "", nil)
require.ErrorIs(t, err, errNilPubsubMessage)
_, err = s.validateLightClientOptimisticUpdate(ctx, "", &pubsub.Message{Message: &pb.Message{}})
require.ErrorIs(t, err, errNilPubsubMessage)
emptyTopic := ""
_, err = s.validateLightClientOptimisticUpdate(ctx, "", &pubsub.Message{Message: &pb.Message{
Topic: &emptyTopic,
}})
require.ErrorIs(t, err, errNilPubsubMessage)
}
func TestValidateLightClientOptimisticUpdate(t *testing.T) {
params.SetupTestConfigCleanup(t)
cfg := params.BeaconConfig().Copy()
cfg.AltairForkEpoch = 1
cfg.BellatrixForkEpoch = 2
cfg.CapellaForkEpoch = 3
cfg.DenebForkEpoch = 4
cfg.ElectraForkEpoch = 5
cfg.ForkVersionSchedule[[4]byte{1, 0, 0, 0}] = 1
cfg.ForkVersionSchedule[[4]byte{2, 0, 0, 0}] = 2
cfg.ForkVersionSchedule[[4]byte{3, 0, 0, 0}] = 3
cfg.ForkVersionSchedule[[4]byte{4, 0, 0, 0}] = 4
cfg.ForkVersionSchedule[[4]byte{5, 0, 0, 0}] = 5
params.OverrideBeaconConfig(cfg)
secondsPerSlot := int(params.BeaconConfig().SecondsPerSlot)
slotIntervals := int(params.BeaconConfig().IntervalsPerSlot)
slotsPerEpoch := int(params.BeaconConfig().SlotsPerEpoch)
tests := []struct {
name string
genesisDrift int
oldUpdateOptions []util.LightClientOption
newUpdateOptions []util.LightClientOption
expectedResult pubsub.ValidationResult
expectedErr error
}{
{
name: "no previous update",
oldUpdateOptions: nil,
newUpdateOptions: []util.LightClientOption{},
expectedResult: pubsub.ValidationAccept,
},
{
name: "not enough time passed",
genesisDrift: -secondsPerSlot / slotIntervals,
oldUpdateOptions: nil,
newUpdateOptions: []util.LightClientOption{},
expectedResult: pubsub.ValidationIgnore,
},
{
name: "new update has no age advantage",
oldUpdateOptions: []util.LightClientOption{},
newUpdateOptions: []util.LightClientOption{},
expectedResult: pubsub.ValidationIgnore,
},
{
name: "new update is better - younger",
genesisDrift: secondsPerSlot,
oldUpdateOptions: []util.LightClientOption{},
newUpdateOptions: []util.LightClientOption{util.WithIncreasedAttestedSlot(1)},
expectedResult: pubsub.ValidationAccept,
},
}
for _, test := range tests {
for v := 1; v < 6; v++ {
t.Run(test.name+"_"+version.String(v), func(t *testing.T) {
ctx := context.Background()
p := p2ptest.NewTestP2P(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.Store{}}
var oldUpdate interfaces.LightClientOptimisticUpdate
var err error
if test.oldUpdateOptions != nil {
l := util.NewTestLightClient(t, v, test.oldUpdateOptions...)
oldUpdate, err = lightClient.NewLightClientOptimisticUpdateFromBeaconState(l.Ctx, l.State.Slot(), l.State, l.Block, l.AttestedState, l.AttestedBlock)
require.NoError(t, err)
s.lcStore.SetLastOptimisticUpdate(oldUpdate)
}
l := util.NewTestLightClient(t, v, test.newUpdateOptions...)
newUpdate, err := lightClient.NewLightClientOptimisticUpdateFromBeaconState(l.Ctx, l.State.Slot(), l.State, l.Block, l.AttestedState, l.AttestedBlock)
require.NoError(t, err)
buf := new(bytes.Buffer)
_, err = p.Encoding().EncodeGossip(buf, newUpdate)
require.NoError(t, err)
topic := p2p.LightClientOptimisticUpdateTopicFormat
digest, err := s.currentForkDigest()
require.NoError(t, err)
topic = s.addDigestToTopic(topic, digest)
r, err := s.validateLightClientOptimisticUpdate(ctx, "", &pubsub.Message{
Message: &pb.Message{
Data: buf.Bytes(),
Topic: &topic,
}})
if test.expectedErr != nil {
require.ErrorIs(t, err, test.expectedErr)
} else {
require.NoError(t, err)
require.Equal(t, test.expectedResult, r)
}
})
}
}
}
func TestValidateLightClientFinalityUpdate_NilMessageOrTopic(t *testing.T) {
params.SetupTestConfigCleanup(t)
ctx := context.Background()
p := p2ptest.NewTestP2P(t)
s := &Service{cfg: &config{p2p: p, initialSync: &mockSync.Sync{}}}
_, err := s.validateLightClientFinalityUpdate(ctx, "", nil)
require.ErrorIs(t, err, errNilPubsubMessage)
_, err = s.validateLightClientFinalityUpdate(ctx, "", &pubsub.Message{Message: &pb.Message{}})
require.ErrorIs(t, err, errNilPubsubMessage)
emptyTopic := ""
_, err = s.validateLightClientFinalityUpdate(ctx, "", &pubsub.Message{Message: &pb.Message{
Topic: &emptyTopic,
}})
require.ErrorIs(t, err, errNilPubsubMessage)
}
func TestValidateLightClientFinalityUpdate(t *testing.T) {
params.SetupTestConfigCleanup(t)
cfg := params.BeaconConfig().Copy()
cfg.AltairForkEpoch = 1
cfg.BellatrixForkEpoch = 2
cfg.CapellaForkEpoch = 3
cfg.DenebForkEpoch = 4
cfg.ElectraForkEpoch = 5
cfg.ForkVersionSchedule[[4]byte{1, 0, 0, 0}] = 1
cfg.ForkVersionSchedule[[4]byte{2, 0, 0, 0}] = 2
cfg.ForkVersionSchedule[[4]byte{3, 0, 0, 0}] = 3
cfg.ForkVersionSchedule[[4]byte{4, 0, 0, 0}] = 4
cfg.ForkVersionSchedule[[4]byte{5, 0, 0, 0}] = 5
params.OverrideBeaconConfig(cfg)
secondsPerSlot := int(params.BeaconConfig().SecondsPerSlot)
slotIntervals := int(params.BeaconConfig().IntervalsPerSlot)
slotsPerEpoch := int(params.BeaconConfig().SlotsPerEpoch)
tests := []struct {
name string
genesisDrift int
oldUpdateOptions []util.LightClientOption
newUpdateOptions []util.LightClientOption
expectedResult pubsub.ValidationResult
expectedErr error
}{
{
name: "no previous update",
oldUpdateOptions: nil,
newUpdateOptions: []util.LightClientOption{},
expectedResult: pubsub.ValidationAccept,
},
{
name: "not enough time passed",
genesisDrift: -secondsPerSlot / slotIntervals,
oldUpdateOptions: nil,
newUpdateOptions: []util.LightClientOption{},
expectedResult: pubsub.ValidationIgnore,
},
{
name: "new update has no advantage",
oldUpdateOptions: []util.LightClientOption{},
newUpdateOptions: []util.LightClientOption{},
expectedResult: pubsub.ValidationIgnore,
},
{
name: "new update is better - age",
genesisDrift: secondsPerSlot,
oldUpdateOptions: []util.LightClientOption{},
newUpdateOptions: []util.LightClientOption{util.WithIncreasedFinalizedSlot(1)},
expectedResult: pubsub.ValidationAccept,
},
{
name: "new update is better - supermajority",
oldUpdateOptions: []util.LightClientOption{},
newUpdateOptions: []util.LightClientOption{util.WithSupermajority()},
expectedResult: pubsub.ValidationAccept,
},
{
name: "old update is better - supermajority",
oldUpdateOptions: []util.LightClientOption{util.WithSupermajority()},
newUpdateOptions: []util.LightClientOption{},
expectedResult: pubsub.ValidationIgnore,
},
{
name: "old update is better - age",
oldUpdateOptions: []util.LightClientOption{util.WithIncreasedAttestedSlot(1)},
newUpdateOptions: []util.LightClientOption{},
expectedResult: pubsub.ValidationIgnore,
},
}
for _, test := range tests {
for v := 1; v < 6; v++ {
t.Run(test.name+"_"+version.String(v), func(t *testing.T) {
ctx := context.Background()
p := p2ptest.NewTestP2P(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.Store{}}
var oldUpdate interfaces.LightClientFinalityUpdate
var err error
if test.oldUpdateOptions != nil {
l := util.NewTestLightClient(t, v, test.oldUpdateOptions...)
oldUpdate, err = lightClient.NewLightClientFinalityUpdateFromBeaconState(l.Ctx, l.State.Slot(), l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
s.lcStore.SetLastFinalityUpdate(oldUpdate)
}
l := util.NewTestLightClient(t, v, test.newUpdateOptions...)
newUpdate, err := lightClient.NewLightClientFinalityUpdateFromBeaconState(l.Ctx, l.State.Slot(), l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
buf := new(bytes.Buffer)
_, err = p.Encoding().EncodeGossip(buf, newUpdate)
require.NoError(t, err)
topic := p2p.LightClientFinalityUpdateTopicFormat
digest, err := s.currentForkDigest()
require.NoError(t, err)
topic = s.addDigestToTopic(topic, digest)
r, err := s.validateLightClientFinalityUpdate(ctx, "", &pubsub.Message{
Message: &pb.Message{
Data: buf.Bytes(),
Topic: &topic,
}})
if test.expectedErr != nil {
require.ErrorIs(t, err, test.expectedErr)
} else {
require.NoError(t, err)
require.Equal(t, test.expectedResult, r)
}
})
}
}
}

View File

@@ -0,0 +1,3 @@
### Added
- Add light client p2p validator and subscriber functions.

View File

@@ -99,6 +99,12 @@ type finalityUpdateAltair struct {
finalityBranch interfaces.LightClientFinalityBranch
}
// NewEmptyFinalityUpdateAltair normally should never be called and NewFinalityUpdateFromUpdate should be used instead.
// This function exists only for scenarios where an empty struct is required.
func NewEmptyFinalityUpdateAltair() interfaces.LightClientFinalityUpdate {
return &finalityUpdateAltair{}
}
func (u *finalityUpdateAltair) IsNil() bool {
return u == nil || u.p == nil
}
@@ -205,6 +211,12 @@ type finalityUpdateCapella struct {
finalityBranch interfaces.LightClientFinalityBranch
}
// NewEmptyFinalityUpdateCapella normally should never be called and NewFinalityUpdateFromUpdate should be used instead.
// This function exists only for scenarios where an empty struct is required.
func NewEmptyFinalityUpdateCapella() interfaces.LightClientFinalityUpdate {
return &finalityUpdateCapella{}
}
func (u *finalityUpdateCapella) IsNil() bool {
return u == nil || u.p == nil
}
@@ -311,6 +323,12 @@ type finalityUpdateDeneb struct {
finalityBranch interfaces.LightClientFinalityBranch
}
// NewEmptyFinalityUpdateDeneb normally should never be called and NewFinalityUpdateFromUpdate should be used instead.
// This function exists only for scenarios where an empty struct is required.
func NewEmptyFinalityUpdateDeneb() interfaces.LightClientFinalityUpdate {
return &finalityUpdateDeneb{}
}
func (u *finalityUpdateDeneb) IsNil() bool {
return u == nil || u.p == nil
}
@@ -417,6 +435,12 @@ type finalityUpdateElectra struct {
finalityBranch interfaces.LightClientFinalityBranchElectra
}
// NewEmptyFinalityUpdateElectra normally should never be called and NewFinalityUpdateFromUpdate should be used instead.
// This function exists only for scenarios where an empty struct is required.
func NewEmptyFinalityUpdateElectra() interfaces.LightClientFinalityUpdate {
return &finalityUpdateElectra{}
}
func (u *finalityUpdateElectra) IsNil() bool {
return u == nil || u.p == nil
}

View File

@@ -78,6 +78,12 @@ type optimisticUpdateAltair struct {
attestedHeader interfaces.LightClientHeader
}
// NewEmptyOptimisticUpdateAltair normally should never be called and NewOptimisticUpdateFromUpdate should be used instead.
// This function exists only for scenarios where an empty struct is required.
func NewEmptyOptimisticUpdateAltair() interfaces.LightClientOptimisticUpdate {
return &optimisticUpdateAltair{}
}
func (u *optimisticUpdateAltair) IsNil() bool {
return u == nil || u.p == nil
}
@@ -156,6 +162,12 @@ type optimisticUpdateCapella struct {
attestedHeader interfaces.LightClientHeader
}
// NewEmptyOptimisticUpdateCapella normally should never be called and NewOptimisticUpdateFromUpdate should be used instead.
// This function exists only for scenarios where an empty struct is required.
func NewEmptyOptimisticUpdateCapella() interfaces.LightClientOptimisticUpdate {
return &optimisticUpdateCapella{}
}
func (u *optimisticUpdateCapella) IsNil() bool {
return u == nil || u.p == nil
}
@@ -234,6 +246,12 @@ type optimisticUpdateDeneb struct {
attestedHeader interfaces.LightClientHeader
}
// NewEmptyOptimisticUpdateDeneb normally should never be called and NewOptimisticUpdateFromUpdate should be used instead.
// This function exists only for scenarios where an empty struct is required.
func NewEmptyOptimisticUpdateDeneb() interfaces.LightClientOptimisticUpdate {
return &optimisticUpdateDeneb{}
}
func (u *optimisticUpdateDeneb) IsNil() bool {
return u == nil || u.p == nil
}