mirror of
https://gitlab.com/pulsechaincom/prysm-pulse.git
synced 2024-12-22 19:40:37 +00:00
Add pending blobs queue for missing parent block (#13005)
* Add pending blobs queue for missing parent block * Prune sidecars older than previous slot * Prune sidecar based on time * Tests * Fix state notifier * Wait for chain to start * Remove logs * Remove bad logs * James feedback * Fix conflict * Rm outdated check * Potuz's feedback * Kasey's feedback * Use 11s mark * Use secs * Add pending blobs queue for missing parent block * Prune sidecars older than previous slot * Prune sidecar based on time * Tests * Fix state notifier * Wait for chain to start * Remove logs * Remove bad logs * James feedback * Fix conflict * Rm outdated check * Potuz's feedback * Kasey's feedback * Use 11s mark * Use secs * Add test case for duplicates * Radek's feedback * Fix test
This commit is contained in:
parent
cc81444e13
commit
cb5ce74a23
@ -72,6 +72,7 @@ type ChainService struct {
|
||||
OptimisticRoots map[[32]byte]bool
|
||||
BlockSlot primitives.Slot
|
||||
SyncingRoot [32]byte
|
||||
Blobs []*ethpb.BlobSidecar
|
||||
}
|
||||
|
||||
func (s *ChainService) Ancestor(ctx context.Context, root []byte, slot primitives.Slot) ([]byte, error) {
|
||||
@ -612,6 +613,7 @@ func (c *ChainService) BlockBeingSynced(root [32]byte) bool {
|
||||
}
|
||||
|
||||
// ReceiveBlob implements the same method in the chain service
|
||||
func (*ChainService) ReceiveBlob(_ context.Context, _ *ethpb.BlobSidecar) error {
|
||||
func (c *ChainService) ReceiveBlob(_ context.Context, b *ethpb.BlobSidecar) error {
|
||||
c.Blobs = append(c.Blobs, b)
|
||||
return nil
|
||||
}
|
||||
|
@ -716,6 +716,7 @@ func (b *BeaconNode) registerSyncService(initialSyncComplete chan struct{}) erro
|
||||
regularsync.WithExecutionPayloadReconstructor(web3Service),
|
||||
regularsync.WithClockWaiter(b.clockWaiter),
|
||||
regularsync.WithInitialSyncComplete(initialSyncComplete),
|
||||
regularsync.WithStateNotifier(b),
|
||||
)
|
||||
return b.services.RegisterService(rs)
|
||||
}
|
||||
|
@ -17,6 +17,7 @@ go_library(
|
||||
"metrics.go",
|
||||
"options.go",
|
||||
"pending_attestations_queue.go",
|
||||
"pending_blobs_queue.go",
|
||||
"pending_blocks_queue.go",
|
||||
"rate_limiter.go",
|
||||
"rpc.go",
|
||||
@ -69,6 +70,7 @@ go_library(
|
||||
"//beacon-chain/core/feed:go_default_library",
|
||||
"//beacon-chain/core/feed/block:go_default_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/signing:go_default_library",
|
||||
"//beacon-chain/core/transition:go_default_library",
|
||||
@ -148,6 +150,7 @@ go_test(
|
||||
"error_test.go",
|
||||
"fork_watcher_test.go",
|
||||
"pending_attestations_queue_test.go",
|
||||
"pending_blobs_queue_test.go",
|
||||
"pending_blocks_queue_test.go",
|
||||
"rate_limiter_test.go",
|
||||
"rpc_beacon_blocks_by_range_test.go",
|
||||
|
@ -4,6 +4,7 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/v4/async/event"
|
||||
blockfeed "github.com/prysmaticlabs/prysm/v4/beacon-chain/core/feed/block"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/core/feed/operation"
|
||||
statefeed "github.com/prysmaticlabs/prysm/v4/beacon-chain/core/feed/state"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/execution"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/operations/attestations"
|
||||
@ -143,3 +144,11 @@ func WithInitialSyncComplete(c chan struct{}) Option {
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
// WithStateNotifier to notify an event feed of state processing.
|
||||
func WithStateNotifier(n statefeed.Notifier) Option {
|
||||
return func(s *Service) error {
|
||||
s.cfg.stateNotifier = n
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
154
beacon-chain/sync/pending_blobs_queue.go
Normal file
154
beacon-chain/sync/pending_blobs_queue.go
Normal file
@ -0,0 +1,154 @@
|
||||
package sync
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
pubsub "github.com/libp2p/go-libp2p-pubsub"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/core/feed"
|
||||
statefeed "github.com/prysmaticlabs/prysm/v4/beacon-chain/core/feed/state"
|
||||
"github.com/prysmaticlabs/prysm/v4/config/params"
|
||||
"github.com/prysmaticlabs/prysm/v4/encoding/bytesutil"
|
||||
eth "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
|
||||
"github.com/prysmaticlabs/prysm/v4/time/slots"
|
||||
)
|
||||
|
||||
// processPendingBlobs listens for state changes and handles pending blobs.
|
||||
func (s *Service) processPendingBlobs() {
|
||||
s.waitForChainStart()
|
||||
|
||||
eventFeed := make(chan *feed.Event, 1)
|
||||
sub := s.cfg.stateNotifier.StateFeed().Subscribe(eventFeed)
|
||||
defer sub.Unsubscribe()
|
||||
|
||||
// Initialize the cleanup ticker at 11s mark. The node is less busy that time.
|
||||
cleanupTicker := slots.NewSlotTickerWithIntervals(s.cfg.clock.GenesisTime(), []time.Duration{time.Duration(params.BeaconConfig().SecondsPerSlot-1) * time.Second} /* 11s */)
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-s.ctx.Done():
|
||||
log.Debug("Stopping pending blobs queue")
|
||||
return
|
||||
case <-sub.Err():
|
||||
log.Debugf("Stopping pending blobs queue due to state feed error: %v", sub.Err())
|
||||
return
|
||||
case e := <-eventFeed:
|
||||
s.handleEvent(s.ctx, e)
|
||||
case <-cleanupTicker.C():
|
||||
if s.pendingBlobSidecars == nil {
|
||||
return
|
||||
}
|
||||
s.pendingBlobSidecars.cleanup()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// handleEvent processes incoming events.
|
||||
func (s *Service) handleEvent(ctx context.Context, e *feed.Event) {
|
||||
if e.Type == statefeed.BlockProcessed {
|
||||
s.handleNewBlockEvent(ctx, e)
|
||||
}
|
||||
}
|
||||
|
||||
// handleNewBlockEvent processes blobs when a parent block is processed.
|
||||
func (s *Service) handleNewBlockEvent(ctx context.Context, e *feed.Event) {
|
||||
data, ok := e.Data.(*statefeed.BlockProcessedData)
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
if data == nil || data.SignedBlock.IsNil() {
|
||||
return
|
||||
}
|
||||
s.processBlobsFromSidecars(ctx, data.SignedBlock.Block().ParentRoot())
|
||||
}
|
||||
|
||||
// processBlobsFromSidecars processes blobs for a given parent root.
|
||||
func (s *Service) processBlobsFromSidecars(ctx context.Context, parentRoot [32]byte) {
|
||||
blobs := s.pendingBlobSidecars.pop(parentRoot)
|
||||
for _, blob := range blobs {
|
||||
if err := s.receiveBlob(ctx, blob); err != nil {
|
||||
log.WithError(err).Error("Failed to validate blob in pending queue")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// receiveBlob validates and processes a blob.
|
||||
func (s *Service) receiveBlob(ctx context.Context, blob *eth.SignedBlobSidecar) error {
|
||||
result, err := s.validateBlobPostSeenParent(ctx, blob)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if result != pubsub.ValidationAccept {
|
||||
return fmt.Errorf("unexpected pubsub result: %d", result)
|
||||
}
|
||||
return s.cfg.chain.ReceiveBlob(ctx, blob.Message)
|
||||
}
|
||||
|
||||
// blobWithExpiration holds blobs with an expiration time.
|
||||
type blobWithExpiration struct {
|
||||
blob []*eth.SignedBlobSidecar
|
||||
expiresAt time.Time
|
||||
}
|
||||
|
||||
// pendingBlobSidecars holds pending blobs with expiration.
|
||||
type pendingBlobSidecars struct {
|
||||
sync.RWMutex
|
||||
blobSidecars map[[32]byte]*blobWithExpiration // Key is the block root.
|
||||
}
|
||||
|
||||
// newPendingBlobSidecars initializes a new cache of pending blobs.
|
||||
func newPendingBlobSidecars() *pendingBlobSidecars {
|
||||
return &pendingBlobSidecars{
|
||||
blobSidecars: make(map[[32]byte]*blobWithExpiration),
|
||||
}
|
||||
}
|
||||
|
||||
// add adds a new blob to the cache.
|
||||
func (p *pendingBlobSidecars) add(blob *eth.SignedBlobSidecar) {
|
||||
p.Lock()
|
||||
defer p.Unlock()
|
||||
parentRoot := bytesutil.ToBytes32(blob.Message.BlockParentRoot)
|
||||
expirationTime := time.Now().Add(time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Second)
|
||||
|
||||
if existing, exists := p.blobSidecars[parentRoot]; exists {
|
||||
for _, sidecar := range existing.blob {
|
||||
if sidecar.Message.Index == blob.Message.Index {
|
||||
return // Ignore duplicate blob index
|
||||
}
|
||||
}
|
||||
existing.blob = append(existing.blob, blob)
|
||||
} else {
|
||||
p.blobSidecars[parentRoot] = &blobWithExpiration{
|
||||
blob: []*eth.SignedBlobSidecar{blob},
|
||||
expiresAt: expirationTime,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// pop removes and returns blobs for a given parent root.
|
||||
func (p *pendingBlobSidecars) pop(parentRoot [32]byte) []*eth.SignedBlobSidecar {
|
||||
p.Lock()
|
||||
defer p.Unlock()
|
||||
blobs, exists := p.blobSidecars[parentRoot]
|
||||
if exists {
|
||||
delete(p.blobSidecars, parentRoot)
|
||||
}
|
||||
if blobs != nil {
|
||||
return blobs.blob
|
||||
}
|
||||
return nil // Return nil if blobs does not exist
|
||||
}
|
||||
|
||||
// cleanup removes expired blobs from the cache.
|
||||
func (p *pendingBlobSidecars) cleanup() {
|
||||
p.Lock()
|
||||
defer p.Unlock()
|
||||
now := time.Now()
|
||||
for root, blobInfo := range p.blobSidecars {
|
||||
if blobInfo.expiresAt.Before(now) {
|
||||
delete(p.blobSidecars, root)
|
||||
}
|
||||
}
|
||||
}
|
96
beacon-chain/sync/pending_blobs_queue_test.go
Normal file
96
beacon-chain/sync/pending_blobs_queue_test.go
Normal file
@ -0,0 +1,96 @@
|
||||
package sync
|
||||
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
mock "github.com/prysmaticlabs/prysm/v4/beacon-chain/blockchain/testing"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/core/signing"
|
||||
dbtest "github.com/prysmaticlabs/prysm/v4/beacon-chain/db/testing"
|
||||
doublylinkedtree "github.com/prysmaticlabs/prysm/v4/beacon-chain/forkchoice/doubly-linked-tree"
|
||||
p2ptest "github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p/testing"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/startup"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/state/stategen"
|
||||
mockSync "github.com/prysmaticlabs/prysm/v4/beacon-chain/sync/initial-sync/testing"
|
||||
lruwrpr "github.com/prysmaticlabs/prysm/v4/cache/lru"
|
||||
"github.com/prysmaticlabs/prysm/v4/config/params"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
|
||||
eth "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
|
||||
"github.com/prysmaticlabs/prysm/v4/testing/require"
|
||||
"github.com/prysmaticlabs/prysm/v4/testing/util"
|
||||
)
|
||||
|
||||
func TestProcessBlobsFromSidecars(t *testing.T) {
|
||||
db := dbtest.SetupDB(t)
|
||||
ctx := context.Background()
|
||||
p := p2ptest.NewTestP2P(t)
|
||||
chainService := &mock.ChainService{Genesis: time.Now(), FinalizedCheckPoint: ð.Checkpoint{}, DB: db}
|
||||
stateGen := stategen.New(db, doublylinkedtree.New())
|
||||
s := &Service{
|
||||
pendingBlobSidecars: newPendingBlobSidecars(),
|
||||
seenBlobCache: lruwrpr.New(10),
|
||||
cfg: &config{
|
||||
p2p: p,
|
||||
initialSync: &mockSync.Sync{},
|
||||
chain: chainService,
|
||||
stateGen: stateGen,
|
||||
clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot)}}
|
||||
|
||||
b := util.NewBlobsidecar()
|
||||
b.Message.Slot = chainService.CurrentSlot() + 1
|
||||
beaconState, privKeys := util.DeterministicGenesisState(t, 100)
|
||||
|
||||
bb := util.NewBeaconBlock()
|
||||
signedBb, err := blocks.NewSignedBeaconBlock(bb)
|
||||
require.NoError(t, err)
|
||||
require.NoError(t, db.SaveBlock(ctx, signedBb))
|
||||
r, err := signedBb.Block().HashTreeRoot()
|
||||
require.NoError(t, err)
|
||||
require.NoError(t, db.SaveState(ctx, beaconState, r))
|
||||
|
||||
b.Message.BlockParentRoot = r[:]
|
||||
b.Message.ProposerIndex = 21
|
||||
b.Signature, err = signing.ComputeDomainAndSign(beaconState, 0, b.Message, params.BeaconConfig().DomainBlobSidecar, privKeys[21])
|
||||
require.NoError(t, err)
|
||||
|
||||
s.pendingBlobSidecars.add(b)
|
||||
require.Equal(t, 1, len(s.pendingBlobSidecars.blobSidecars))
|
||||
s.processBlobsFromSidecars(ctx, r)
|
||||
require.Equal(t, 0, len(s.pendingBlobSidecars.blobSidecars))
|
||||
|
||||
// Make sure chain service has the blob.
|
||||
require.DeepEqual(t, chainService.Blobs[0], b.Message)
|
||||
}
|
||||
|
||||
func TestPendingBlobSidecars(t *testing.T) {
|
||||
// Test Initialization
|
||||
cache := newPendingBlobSidecars()
|
||||
require.Equal(t, 0, len(cache.blobSidecars))
|
||||
|
||||
// Test Add
|
||||
parentRoot := [32]byte{1, 2, 3}
|
||||
blob := ð.SignedBlobSidecar{Message: ð.BlobSidecar{}}
|
||||
blob.Message.BlockParentRoot = parentRoot[:]
|
||||
cache.add(blob)
|
||||
_, exists := cache.blobSidecars[parentRoot]
|
||||
require.Equal(t, true, exists)
|
||||
require.Equal(t, 1, len(cache.blobSidecars))
|
||||
|
||||
// Test Add duplicates
|
||||
cache.add(blob)
|
||||
require.Equal(t, 1, len(cache.blobSidecars))
|
||||
|
||||
// Test Pop
|
||||
poppedBlob := cache.pop(parentRoot)
|
||||
require.Equal(t, 0, len(cache.blobSidecars))
|
||||
require.DeepEqual(t, poppedBlob[0].Message.BlockParentRoot, parentRoot[:])
|
||||
|
||||
// Test Cleanup
|
||||
// For this, we can manually set an expired time to simulate an expired blob
|
||||
cache.add(blob)
|
||||
cache.blobSidecars[parentRoot].expiresAt = time.Now().Add(-time.Second)
|
||||
cache.cleanup()
|
||||
_, exists = cache.blobSidecars[parentRoot]
|
||||
require.Equal(t, false, exists)
|
||||
}
|
@ -308,10 +308,11 @@ func TestHandshakeHandlers_Roundtrip(t *testing.T) {
|
||||
r := &Service{
|
||||
ctx: ctx,
|
||||
cfg: &config{
|
||||
p2p: p1,
|
||||
chain: chain,
|
||||
clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot),
|
||||
beaconDB: db,
|
||||
p2p: p1,
|
||||
chain: chain,
|
||||
clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot),
|
||||
beaconDB: db,
|
||||
stateNotifier: chain.StateNotifier(),
|
||||
},
|
||||
rateLimiter: newRateLimiter(p1),
|
||||
clockWaiter: cw,
|
||||
@ -326,9 +327,10 @@ func TestHandshakeHandlers_Roundtrip(t *testing.T) {
|
||||
r2 := &Service{
|
||||
ctx: ctx,
|
||||
cfg: &config{
|
||||
chain: chain2,
|
||||
clock: startup.NewClock(chain2.Genesis, chain2.ValidatorsRoot),
|
||||
p2p: p2,
|
||||
chain: chain2,
|
||||
clock: startup.NewClock(chain2.Genesis, chain2.ValidatorsRoot),
|
||||
p2p: p2,
|
||||
stateNotifier: chain.StateNotifier(),
|
||||
},
|
||||
rateLimiter: newRateLimiter(p2),
|
||||
}
|
||||
@ -536,9 +538,10 @@ func TestStatusRPCRequest_FinalizedBlockExists(t *testing.T) {
|
||||
}
|
||||
r := &Service{
|
||||
cfg: &config{
|
||||
p2p: p1,
|
||||
chain: chain,
|
||||
clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot),
|
||||
p2p: p1,
|
||||
chain: chain,
|
||||
clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot),
|
||||
stateNotifier: chain.StateNotifier(),
|
||||
},
|
||||
ctx: context.Background(),
|
||||
rateLimiter: newRateLimiter(p1),
|
||||
@ -559,10 +562,11 @@ func TestStatusRPCRequest_FinalizedBlockExists(t *testing.T) {
|
||||
}
|
||||
r2 := &Service{
|
||||
cfg: &config{
|
||||
p2p: p1,
|
||||
chain: chain2,
|
||||
clock: startup.NewClock(chain2.Genesis, chain2.ValidatorsRoot),
|
||||
beaconDB: db,
|
||||
p2p: p1,
|
||||
chain: chain2,
|
||||
clock: startup.NewClock(chain2.Genesis, chain2.ValidatorsRoot),
|
||||
beaconDB: db,
|
||||
stateNotifier: chain.StateNotifier(),
|
||||
},
|
||||
ctx: context.Background(),
|
||||
rateLimiter: newRateLimiter(p1),
|
||||
@ -722,9 +726,10 @@ func TestStatusRPCRequest_FinalizedBlockSkippedSlots(t *testing.T) {
|
||||
}
|
||||
r := &Service{
|
||||
cfg: &config{
|
||||
p2p: p1,
|
||||
chain: chain,
|
||||
clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot),
|
||||
p2p: p1,
|
||||
chain: chain,
|
||||
clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot),
|
||||
stateNotifier: chain.StateNotifier(),
|
||||
},
|
||||
ctx: context.Background(),
|
||||
rateLimiter: newRateLimiter(p1),
|
||||
@ -746,10 +751,11 @@ func TestStatusRPCRequest_FinalizedBlockSkippedSlots(t *testing.T) {
|
||||
}
|
||||
r2 := &Service{
|
||||
cfg: &config{
|
||||
p2p: p2,
|
||||
chain: chain2,
|
||||
clock: startup.NewClock(chain2.Genesis, chain2.ValidatorsRoot),
|
||||
beaconDB: db,
|
||||
p2p: p2,
|
||||
chain: chain2,
|
||||
clock: startup.NewClock(chain2.Genesis, chain2.ValidatorsRoot),
|
||||
beaconDB: db,
|
||||
stateNotifier: chain.StateNotifier(),
|
||||
},
|
||||
|
||||
ctx: context.Background(),
|
||||
@ -821,8 +827,9 @@ func TestStatusRPCRequest_BadPeerHandshake(t *testing.T) {
|
||||
|
||||
r := &Service{
|
||||
cfg: &config{
|
||||
p2p: p1,
|
||||
chain: chain,
|
||||
p2p: p1,
|
||||
chain: chain,
|
||||
stateNotifier: chain.StateNotifier(),
|
||||
},
|
||||
|
||||
ctx: ctx,
|
||||
@ -906,8 +913,9 @@ func TestStatusRPC_ValidGenesisMessage(t *testing.T) {
|
||||
}
|
||||
r := &Service{
|
||||
cfg: &config{
|
||||
chain: chain,
|
||||
clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot),
|
||||
chain: chain,
|
||||
clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot),
|
||||
stateNotifier: chain.StateNotifier(),
|
||||
},
|
||||
ctx: context.Background(),
|
||||
}
|
||||
@ -983,9 +991,10 @@ func TestShouldResync(t *testing.T) {
|
||||
}
|
||||
r := &Service{
|
||||
cfg: &config{
|
||||
chain: chain,
|
||||
clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot),
|
||||
initialSync: &mockSync.Sync{IsSyncing: tt.args.syncing},
|
||||
chain: chain,
|
||||
clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot),
|
||||
initialSync: &mockSync.Sync{IsSyncing: tt.args.syncing},
|
||||
stateNotifier: chain.StateNotifier(),
|
||||
},
|
||||
ctx: context.Background(),
|
||||
}
|
||||
|
@ -22,6 +22,7 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/blockchain"
|
||||
blockfeed "github.com/prysmaticlabs/prysm/v4/beacon-chain/core/feed/block"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/core/feed/operation"
|
||||
statefeed "github.com/prysmaticlabs/prysm/v4/beacon-chain/core/feed/state"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/execution"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/operations/attestations"
|
||||
@ -88,6 +89,7 @@ type config struct {
|
||||
slasherAttestationsFeed *event.Feed
|
||||
slasherBlockHeadersFeed *event.Feed
|
||||
clock *startup.Clock
|
||||
stateNotifier statefeed.Notifier
|
||||
}
|
||||
|
||||
// This defines the interface for interacting with block chain service
|
||||
@ -146,6 +148,7 @@ type Service struct {
|
||||
signatureChan chan *signatureVerifier
|
||||
clockWaiter startup.ClockWaiter
|
||||
initialSyncComplete chan struct{}
|
||||
pendingBlobSidecars *pendingBlobSidecars
|
||||
}
|
||||
|
||||
// NewService initializes new regular sync service.
|
||||
@ -161,6 +164,7 @@ func NewService(ctx context.Context, opts ...Option) *Service {
|
||||
seenPendingBlocks: make(map[[32]byte]bool),
|
||||
blkRootToPendingAtts: make(map[[32]byte][]*ethpb.SignedAggregateAttestationAndProof),
|
||||
signatureChan: make(chan *signatureVerifier, verifierLimit),
|
||||
pendingBlobSidecars: newPendingBlobSidecars(),
|
||||
}
|
||||
for _, opt := range opts {
|
||||
if err := opt(r); err != nil {
|
||||
@ -187,6 +191,7 @@ func (s *Service) Start() {
|
||||
s.cfg.p2p.AddPingMethod(s.sendPingRequest)
|
||||
s.processPendingBlocksQueue()
|
||||
s.processPendingAttsQueue()
|
||||
s.processPendingBlobs()
|
||||
s.maintainPeerStatuses()
|
||||
s.resyncIfBehind()
|
||||
|
||||
|
@ -14,6 +14,7 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/state"
|
||||
"github.com/prysmaticlabs/prysm/v4/config/params"
|
||||
"github.com/prysmaticlabs/prysm/v4/crypto/bls"
|
||||
"github.com/prysmaticlabs/prysm/v4/crypto/rand"
|
||||
"github.com/prysmaticlabs/prysm/v4/encoding/bytesutil"
|
||||
"github.com/prysmaticlabs/prysm/v4/network/forks"
|
||||
eth "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
|
||||
@ -91,7 +92,13 @@ func (s *Service) validateBlob(ctx context.Context, pid peer.ID, msg *pubsub.Mes
|
||||
parentRoot := bytesutil.ToBytes32(blob.BlockParentRoot)
|
||||
switch parentStatus := s.handleBlobParentStatus(ctx, parentRoot); parentStatus {
|
||||
case pubsub.ValidationIgnore:
|
||||
log.WithFields(blobFields(blob)).Debug("Ignored blob: parent block not found")
|
||||
log.WithFields(blobFields(blob)).Debug("Parent block not found - saving blob to cache")
|
||||
go func() {
|
||||
if err := s.sendBatchRootRequest(ctx, [][32]byte{parentRoot}, rand.NewGenerator()); err != nil {
|
||||
log.WithError(err).WithFields(blobFields(blob)).Debug("Failed to send batch root request")
|
||||
}
|
||||
}()
|
||||
s.pendingBlobSidecars.add(sBlob)
|
||||
return pubsub.ValidationIgnore, nil
|
||||
case pubsub.ValidationReject:
|
||||
log.WithFields(blobFields(blob)).Warning("Rejected blob: parent block is invalid")
|
||||
@ -99,6 +106,35 @@ func (s *Service) validateBlob(ctx context.Context, pid peer.ID, msg *pubsub.Mes
|
||||
default:
|
||||
}
|
||||
|
||||
pubsubResult, err := s.validateBlobPostSeenParent(ctx, sBlob)
|
||||
if err != nil {
|
||||
return pubsubResult, err
|
||||
}
|
||||
if pubsubResult != pubsub.ValidationAccept {
|
||||
return pubsubResult, nil
|
||||
}
|
||||
|
||||
startTime, err := slots.ToTime(genesisTime, blob.Slot)
|
||||
if err != nil {
|
||||
return pubsub.ValidationIgnore, err
|
||||
}
|
||||
fields := blobFields(blob)
|
||||
sinceSlotStartTime := receivedTime.Sub(startTime)
|
||||
fields["sinceSlotStartTime"] = sinceSlotStartTime
|
||||
fields["validationTime"] = s.cfg.clock.Now().Sub(receivedTime)
|
||||
log.WithFields(fields).Debug("Received blob sidecar gossip")
|
||||
|
||||
blobSidecarArrivalGossipSummary.Observe(float64(sinceSlotStartTime.Milliseconds()))
|
||||
|
||||
msg.ValidatorData = sBlob
|
||||
|
||||
return pubsub.ValidationAccept, nil
|
||||
}
|
||||
|
||||
func (s *Service) validateBlobPostSeenParent(ctx context.Context, sBlob *eth.SignedBlobSidecar) (pubsub.ValidationResult, error) {
|
||||
blob := sBlob.Message
|
||||
parentRoot := bytesutil.ToBytes32(blob.BlockParentRoot)
|
||||
|
||||
// [REJECT] The sidecar is from a higher slot than the sidecar's block's parent (defined by sidecar.block_parent_root).
|
||||
parentSlot, err := s.cfg.chain.RecentBlockSlot(parentRoot)
|
||||
if err != nil {
|
||||
@ -140,21 +176,6 @@ func (s *Service) validateBlob(ctx context.Context, pid peer.ID, msg *pubsub.Mes
|
||||
log.WithFields(blobFields(blob)).Debug(err)
|
||||
return pubsub.ValidationReject, err
|
||||
}
|
||||
|
||||
startTime, err := slots.ToTime(genesisTime, blob.Slot)
|
||||
if err != nil {
|
||||
return pubsub.ValidationIgnore, err
|
||||
}
|
||||
fields := blobFields(blob)
|
||||
sinceSlotStartTime := receivedTime.Sub(startTime)
|
||||
fields["sinceSlotStartTime"] = sinceSlotStartTime
|
||||
fields["validationTime"] = prysmTime.Now().Sub(receivedTime)
|
||||
log.WithFields(fields).Debug("Received blob sidecar gossip")
|
||||
|
||||
blobSidecarArrivalGossipSummary.Observe(float64(sinceSlotStartTime.Milliseconds()))
|
||||
|
||||
msg.ValidatorData = sBlob
|
||||
|
||||
return pubsub.ValidationAccept, nil
|
||||
}
|
||||
|
||||
|
@ -59,10 +59,12 @@ func TestService_ValidateBlsToExecutionChange(t *testing.T) {
|
||||
WithInitialSync(&mockSync.Sync{IsSyncing: true}),
|
||||
WithChainService(chainService),
|
||||
WithOperationNotifier(chainService.OperationNotifier()),
|
||||
WithStateNotifier(chainService.StateNotifier()),
|
||||
},
|
||||
setupSvc: func(s *Service, msg *ethpb.SignedBLSToExecutionChange, topic string) (*Service, string) {
|
||||
s.cfg.stateGen = stategen.New(beaconDB, doublylinkedtree.New())
|
||||
s.cfg.beaconDB = beaconDB
|
||||
s.cfg.clock = startup.NewClock(time.Now(), [32]byte{'A'})
|
||||
s.initCaches()
|
||||
return s, topic
|
||||
},
|
||||
@ -86,10 +88,12 @@ func TestService_ValidateBlsToExecutionChange(t *testing.T) {
|
||||
WithInitialSync(&mockSync.Sync{IsSyncing: false}),
|
||||
WithChainService(chainService),
|
||||
WithOperationNotifier(chainService.OperationNotifier()),
|
||||
WithStateNotifier(chainService.StateNotifier()),
|
||||
},
|
||||
setupSvc: func(s *Service, msg *ethpb.SignedBLSToExecutionChange, topic string) (*Service, string) {
|
||||
s.cfg.stateGen = stategen.New(beaconDB, doublylinkedtree.New())
|
||||
s.cfg.beaconDB = beaconDB
|
||||
s.cfg.clock = startup.NewClock(time.Now(), [32]byte{'A'})
|
||||
s.initCaches()
|
||||
return s, topic
|
||||
},
|
||||
@ -114,10 +118,12 @@ func TestService_ValidateBlsToExecutionChange(t *testing.T) {
|
||||
WithChainService(chainService),
|
||||
WithOperationNotifier(chainService.OperationNotifier()),
|
||||
WithBlsToExecPool(blstoexec.NewPool()),
|
||||
WithStateNotifier(chainService.StateNotifier()),
|
||||
},
|
||||
setupSvc: func(s *Service, msg *ethpb.SignedBLSToExecutionChange, topic string) (*Service, string) {
|
||||
s.cfg.stateGen = stategen.New(beaconDB, doublylinkedtree.New())
|
||||
s.cfg.beaconDB = beaconDB
|
||||
s.cfg.clock = startup.NewClock(time.Now(), [32]byte{'A'})
|
||||
s.initCaches()
|
||||
s.cfg.blsToExecPool.InsertBLSToExecChange(ðpb.SignedBLSToExecutionChange{
|
||||
Message: ðpb.BLSToExecutionChange{
|
||||
@ -151,15 +157,18 @@ func TestService_ValidateBlsToExecutionChange(t *testing.T) {
|
||||
WithChainService(chainService),
|
||||
WithOperationNotifier(chainService.OperationNotifier()),
|
||||
WithBlsToExecPool(blstoexec.NewPool()),
|
||||
WithStateNotifier(chainService.StateNotifier()),
|
||||
},
|
||||
clock: startup.NewClock(time.Now().Add(-time.Second*time.Duration(params.BeaconConfig().SecondsPerSlot*10)), [32]byte{'A'}),
|
||||
setupSvc: func(s *Service, msg *ethpb.SignedBLSToExecutionChange, topic string) (*Service, string) {
|
||||
s.cfg.stateGen = stategen.New(beaconDB, doublylinkedtree.New())
|
||||
s.cfg.beaconDB = beaconDB
|
||||
s.cfg.clock = startup.NewClock(time.Now(), [32]byte{'A'})
|
||||
s.initCaches()
|
||||
st, keys := util.DeterministicGenesisStateBellatrix(t, 128)
|
||||
s.cfg.chain = &mockChain.ChainService{
|
||||
State: st,
|
||||
State: st,
|
||||
Genesis: time.Now(),
|
||||
}
|
||||
|
||||
msg.Message.ValidatorIndex = 50
|
||||
@ -195,15 +204,18 @@ func TestService_ValidateBlsToExecutionChange(t *testing.T) {
|
||||
WithChainService(chainService),
|
||||
WithOperationNotifier(chainService.OperationNotifier()),
|
||||
WithBlsToExecPool(blstoexec.NewPool()),
|
||||
WithStateNotifier(chainService.StateNotifier()),
|
||||
},
|
||||
clock: startup.NewClock(time.Now().Add(-time.Second*time.Duration(params.BeaconConfig().SecondsPerSlot)*time.Duration(10)), [32]byte{'A'}),
|
||||
setupSvc: func(s *Service, msg *ethpb.SignedBLSToExecutionChange, topic string) (*Service, string) {
|
||||
s.cfg.stateGen = stategen.New(beaconDB, doublylinkedtree.New())
|
||||
s.cfg.beaconDB = beaconDB
|
||||
s.cfg.clock = startup.NewClock(time.Now(), [32]byte{'A'})
|
||||
s.initCaches()
|
||||
st, _ := util.DeterministicGenesisStateCapella(t, 128)
|
||||
s.cfg.chain = &mockChain.ChainService{
|
||||
State: st,
|
||||
State: st,
|
||||
Genesis: time.Now(),
|
||||
}
|
||||
|
||||
msg.Message.ValidatorIndex = 130
|
||||
@ -230,15 +242,18 @@ func TestService_ValidateBlsToExecutionChange(t *testing.T) {
|
||||
WithChainService(chainService),
|
||||
WithOperationNotifier(chainService.OperationNotifier()),
|
||||
WithBlsToExecPool(blstoexec.NewPool()),
|
||||
WithStateNotifier(chainService.StateNotifier()),
|
||||
},
|
||||
clock: startup.NewClock(time.Now().Add(-time.Second*time.Duration(params.BeaconConfig().SecondsPerSlot)*time.Duration(10)), [32]byte{'A'}),
|
||||
setupSvc: func(s *Service, msg *ethpb.SignedBLSToExecutionChange, topic string) (*Service, string) {
|
||||
s.cfg.stateGen = stategen.New(beaconDB, doublylinkedtree.New())
|
||||
s.cfg.beaconDB = beaconDB
|
||||
s.cfg.clock = startup.NewClock(time.Now(), [32]byte{'A'})
|
||||
s.initCaches()
|
||||
st, keys := util.DeterministicGenesisStateCapella(t, 128)
|
||||
s.cfg.chain = &mockChain.ChainService{
|
||||
State: st,
|
||||
State: st,
|
||||
Genesis: time.Now(),
|
||||
}
|
||||
|
||||
msg.Message.ValidatorIndex = 50
|
||||
@ -268,11 +283,13 @@ func TestService_ValidateBlsToExecutionChange(t *testing.T) {
|
||||
WithChainService(chainService),
|
||||
WithOperationNotifier(chainService.OperationNotifier()),
|
||||
WithBlsToExecPool(blstoexec.NewPool()),
|
||||
WithStateNotifier(chainService.StateNotifier()),
|
||||
},
|
||||
clock: startup.NewClock(time.Now().Add(-time.Second*time.Duration(params.BeaconConfig().SecondsPerSlot)*time.Duration(10)), [32]byte{'A'}),
|
||||
setupSvc: func(s *Service, msg *ethpb.SignedBLSToExecutionChange, topic string) (*Service, string) {
|
||||
s.cfg.stateGen = stategen.New(beaconDB, doublylinkedtree.New())
|
||||
s.cfg.beaconDB = beaconDB
|
||||
s.cfg.clock = startup.NewClock(time.Now(), [32]byte{'A'})
|
||||
s.initCaches()
|
||||
st, keys := util.DeterministicGenesisStateCapella(t, 128)
|
||||
assert.NoError(t, st.ApplyToEveryValidator(func(idx int, val *ethpb.Validator) (bool, *ethpb.Validator, error) {
|
||||
@ -283,7 +300,8 @@ func TestService_ValidateBlsToExecutionChange(t *testing.T) {
|
||||
return true, val, nil
|
||||
}))
|
||||
s.cfg.chain = &mockChain.ChainService{
|
||||
State: st,
|
||||
State: st,
|
||||
Genesis: time.Now(),
|
||||
}
|
||||
|
||||
msg.Message.ValidatorIndex = 50
|
||||
@ -313,15 +331,18 @@ func TestService_ValidateBlsToExecutionChange(t *testing.T) {
|
||||
WithChainService(chainService),
|
||||
WithOperationNotifier(chainService.OperationNotifier()),
|
||||
WithBlsToExecPool(blstoexec.NewPool()),
|
||||
WithStateNotifier(chainService.StateNotifier()),
|
||||
},
|
||||
clock: startup.NewClock(time.Now().Add(-time.Second*time.Duration(params.BeaconConfig().SecondsPerSlot)*time.Duration(10)), [32]byte{'A'}),
|
||||
setupSvc: func(s *Service, msg *ethpb.SignedBLSToExecutionChange, topic string) (*Service, string) {
|
||||
s.cfg.stateGen = stategen.New(beaconDB, doublylinkedtree.New())
|
||||
s.cfg.beaconDB = beaconDB
|
||||
s.cfg.clock = startup.NewClock(time.Now(), [32]byte{'A'})
|
||||
s.initCaches()
|
||||
st, keys := util.DeterministicGenesisStateCapella(t, 128)
|
||||
s.cfg.chain = &mockChain.ChainService{
|
||||
State: st,
|
||||
State: st,
|
||||
Genesis: time.Now(),
|
||||
}
|
||||
|
||||
msg.Message.ValidatorIndex = 50
|
||||
@ -354,15 +375,18 @@ func TestService_ValidateBlsToExecutionChange(t *testing.T) {
|
||||
WithChainService(chainService),
|
||||
WithOperationNotifier(chainService.OperationNotifier()),
|
||||
WithBlsToExecPool(blstoexec.NewPool()),
|
||||
WithStateNotifier(chainService.StateNotifier()),
|
||||
},
|
||||
clock: startup.NewClock(time.Now().Add(-time.Second*time.Duration(params.BeaconConfig().SecondsPerSlot)*time.Duration(10)), [32]byte{'A'}),
|
||||
setupSvc: func(s *Service, msg *ethpb.SignedBLSToExecutionChange, topic string) (*Service, string) {
|
||||
s.cfg.stateGen = stategen.New(beaconDB, doublylinkedtree.New())
|
||||
s.cfg.beaconDB = beaconDB
|
||||
s.cfg.clock = startup.NewClock(time.Now(), [32]byte{'A'})
|
||||
s.initCaches()
|
||||
st, keys := util.DeterministicGenesisStateCapella(t, 128)
|
||||
s.cfg.chain = &mockChain.ChainService{
|
||||
State: st,
|
||||
State: st,
|
||||
Genesis: time.Now(),
|
||||
}
|
||||
|
||||
msg.Message.ValidatorIndex = 50
|
||||
|
@ -175,7 +175,7 @@ func TestService_ValidateSyncCommitteeMessage(t *testing.T) {
|
||||
s.cfg.stateGen = stategen.New(beaconDB, doublylinkedtree.New())
|
||||
s.cfg.beaconDB = beaconDB
|
||||
s.initCaches()
|
||||
s.cfg.chain = &mockChain.ChainService{}
|
||||
s.cfg.chain = &mockChain.ChainService{Genesis: time.Now()}
|
||||
incorrectRoot := [32]byte{0xBB}
|
||||
msg.BlockRoot = incorrectRoot[:]
|
||||
|
||||
@ -210,6 +210,7 @@ func TestService_ValidateSyncCommitteeMessage(t *testing.T) {
|
||||
assert.NoError(t, err)
|
||||
s.cfg.chain = &mockChain.ChainService{
|
||||
SyncCommitteeIndices: []primitives.CommitteeIndex{0},
|
||||
Genesis: time.Now(),
|
||||
}
|
||||
numOfVals := hState.NumValidators()
|
||||
|
||||
@ -254,7 +255,9 @@ func TestService_ValidateSyncCommitteeMessage(t *testing.T) {
|
||||
msg.BlockRoot = headRoot[:]
|
||||
hState, err := beaconDB.State(context.Background(), headRoot)
|
||||
assert.NoError(t, err)
|
||||
s.cfg.chain = &mockChain.ChainService{}
|
||||
s.cfg.chain = &mockChain.ChainService{
|
||||
Genesis: time.Now(),
|
||||
}
|
||||
|
||||
numOfVals := hState.NumValidators()
|
||||
|
||||
@ -314,6 +317,7 @@ func TestService_ValidateSyncCommitteeMessage(t *testing.T) {
|
||||
SyncCommitteeIndices: []primitives.CommitteeIndex{primitives.CommitteeIndex(subCommitteeSize)},
|
||||
SyncCommitteeDomain: d,
|
||||
PublicKey: bytesutil.ToBytes48(keys[chosenVal].PublicKey().Marshal()),
|
||||
Genesis: time.Now(),
|
||||
}
|
||||
|
||||
// Set Topic and Subnet
|
||||
@ -366,6 +370,7 @@ func TestService_ValidateSyncCommitteeMessage(t *testing.T) {
|
||||
SyncCommitteeIndices: []primitives.CommitteeIndex{primitives.CommitteeIndex(subCommitteeSize)},
|
||||
SyncCommitteeDomain: d,
|
||||
PublicKey: bytesutil.ToBytes48(keys[chosenVal].PublicKey().Marshal()),
|
||||
Genesis: time.Now(),
|
||||
}
|
||||
|
||||
msg.Signature = keys[chosenVal].Sign(sigRoot[:]).Marshal()
|
||||
@ -401,7 +406,7 @@ func TestService_ValidateSyncCommitteeMessage(t *testing.T) {
|
||||
defer cancel()
|
||||
|
||||
cw := startup.NewClockSynchronizer()
|
||||
opts := []Option{WithClockWaiter(cw)}
|
||||
opts := []Option{WithClockWaiter(cw), WithStateNotifier(chainService.StateNotifier())}
|
||||
svc := NewService(ctx, append(opts, tt.svcopts...)...)
|
||||
var clock *startup.Clock
|
||||
svc, tt.args.topic, clock = tt.setupSvc(svc, tt.args.msg, tt.args.topic)
|
||||
|
@ -177,7 +177,9 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
|
||||
s.cfg.stateGen = stategen.New(database, doublylinkedtree.New())
|
||||
s.cfg.beaconDB = database
|
||||
s.initCaches()
|
||||
s.cfg.chain = &mockChain.ChainService{}
|
||||
s.cfg.chain = &mockChain.ChainService{
|
||||
Genesis: time.Now(),
|
||||
}
|
||||
msg.Message.Contribution.BlockRoot = headRoot[:]
|
||||
msg.Message.Contribution.AggregationBits.SetBitAt(1, true)
|
||||
|
||||
@ -216,7 +218,9 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
|
||||
s.cfg.stateGen = stategen.New(database, doublylinkedtree.New())
|
||||
s.cfg.beaconDB = database
|
||||
s.initCaches()
|
||||
s.cfg.chain = &mockChain.ChainService{}
|
||||
s.cfg.chain = &mockChain.ChainService{
|
||||
Genesis: time.Now(),
|
||||
}
|
||||
msg.Message.Contribution.BlockRoot = headRoot[:]
|
||||
msg.Message.Contribution.AggregationBits.SetBitAt(1, true)
|
||||
msg.Message.Contribution.SubcommitteeIndex = 20
|
||||
@ -255,7 +259,9 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
|
||||
s.cfg.stateGen = stategen.New(database, doublylinkedtree.New())
|
||||
s.cfg.beaconDB = database
|
||||
s.initCaches()
|
||||
s.cfg.chain = &mockChain.ChainService{}
|
||||
s.cfg.chain = &mockChain.ChainService{
|
||||
Genesis: time.Now(),
|
||||
}
|
||||
msg.Message.Contribution.BlockRoot = headRoot[:]
|
||||
incorrectProof := [96]byte{0xBB}
|
||||
msg.Message.SelectionProof = incorrectProof[:]
|
||||
@ -295,7 +301,9 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
|
||||
s.cfg.stateGen = stategen.New(database, doublylinkedtree.New())
|
||||
s.cfg.beaconDB = database
|
||||
s.initCaches()
|
||||
s.cfg.chain = &mockChain.ChainService{}
|
||||
s.cfg.chain = &mockChain.ChainService{
|
||||
Genesis: time.Now(),
|
||||
}
|
||||
msg.Message.Contribution.BlockRoot = headRoot[:]
|
||||
hState, err := database.State(context.Background(), headRoot)
|
||||
assert.NoError(t, err)
|
||||
@ -376,6 +384,7 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
|
||||
}
|
||||
subCommitteeSize := params.BeaconConfig().SyncCommitteeSize / params.BeaconConfig().SyncCommitteeSubnetCount
|
||||
s.cfg.chain = &mockChain.ChainService{
|
||||
Genesis: time.Now(),
|
||||
SyncCommitteeIndices: []primitives.CommitteeIndex{primitives.CommitteeIndex(msg.Message.Contribution.SubcommitteeIndex * subCommitteeSize)},
|
||||
}
|
||||
msg.Message.Contribution.AggregationBits.SetBitAt(1, true)
|
||||
@ -455,6 +464,7 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
|
||||
SyncCommitteeIndices: []primitives.CommitteeIndex{primitives.CommitteeIndex(msg.Message.Contribution.SubcommitteeIndex * subCommitteeSize)},
|
||||
PublicKey: bytesutil.ToBytes48(pubkey),
|
||||
SyncSelectionProofDomain: d,
|
||||
Genesis: time.Now(),
|
||||
}
|
||||
|
||||
s.initCaches()
|
||||
@ -534,6 +544,7 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
|
||||
}
|
||||
s.cfg.chain = &mockChain.ChainService{
|
||||
SyncCommitteeIndices: []primitives.CommitteeIndex{1},
|
||||
Genesis: time.Now(),
|
||||
}
|
||||
|
||||
s.initCaches()
|
||||
@ -616,6 +627,7 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
|
||||
SyncSelectionProofDomain: d,
|
||||
SyncContributionProofDomain: cd,
|
||||
SyncCommitteeDomain: make([]byte, 32),
|
||||
Genesis: time.Now(),
|
||||
}
|
||||
s.initCaches()
|
||||
gt := time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot))
|
||||
@ -709,6 +721,7 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
|
||||
SyncContributionProofDomain: cd,
|
||||
SyncCommitteeDomain: d,
|
||||
SyncCommitteePubkeys: pubkeys,
|
||||
Genesis: time.Now(),
|
||||
}
|
||||
s.initCaches()
|
||||
gt := time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot))
|
||||
@ -804,6 +817,7 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
|
||||
SyncContributionProofDomain: cd,
|
||||
SyncCommitteeDomain: d,
|
||||
SyncCommitteePubkeys: pubkeys,
|
||||
Genesis: time.Now(),
|
||||
}
|
||||
gt := time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot))
|
||||
|
||||
@ -836,7 +850,7 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(ctx)
|
||||
defer cancel()
|
||||
cw := startup.NewClockSynchronizer()
|
||||
svc := NewService(ctx, append([]Option{WithClockWaiter(cw)}, tt.svcopts...)...)
|
||||
svc := NewService(ctx, append([]Option{WithClockWaiter(cw), WithStateNotifier(chainService.StateNotifier())}, tt.svcopts...)...)
|
||||
var clock *startup.Clock
|
||||
svc, clock = tt.setupSvc(svc, tt.args.msg)
|
||||
require.NoError(t, cw.SetClock(clock))
|
||||
|
Loading…
Reference in New Issue
Block a user