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:
terencechain 2023-10-17 07:42:15 -07:00 committed by GitHub
parent cc81444e13
commit cb5ce74a23
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 402 additions and 59 deletions

View File

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

View File

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

View File

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

View File

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

View 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)
}
}
}

View 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: &eth.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 := &eth.SignedBlobSidecar{Message: &eth.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)
}

View File

@ -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(),
}

View File

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

View File

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

View File

@ -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(&ethpb.SignedBLSToExecutionChange{
Message: &ethpb.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

View File

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

View File

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