[Service Config Revamp] - Sync Service With Functional Options (#9859)

* sync config refactor

* rem

* rem

* testing

* gaz

* next

* fuzz

* build

* fuzz

* rev

* log

* cfg
This commit is contained in:
Raul Jordan
2021-11-05 15:08:58 -04:00
committed by GitHub
parent 2b0e132201
commit 233171d17c
61 changed files with 1290 additions and 1167 deletions

View File

@@ -564,24 +564,24 @@ func (b *BeaconNode) registerSyncService() error {
return err
}
rs := regularsync.NewService(b.ctx, &regularsync.Config{
DB: b.db,
P2P: b.fetchP2P(),
Chain: chainService,
InitialSync: initSync,
StateNotifier: b,
BlockNotifier: b,
AttestationNotifier: b,
OperationNotifier: b,
AttPool: b.attestationPool,
ExitPool: b.exitPool,
SlashingPool: b.slashingsPool,
SyncCommsPool: b.syncCommitteePool,
StateGen: b.stateGen,
SlasherAttestationsFeed: b.slasherAttestationsFeed,
SlasherBlockHeadersFeed: b.slasherBlockHeadersFeed,
})
rs := regularsync.NewService(
b.ctx,
regularsync.WithDatabase(b.db),
regularsync.WithP2P(b.fetchP2P()),
regularsync.WithChainService(chainService),
regularsync.WithInitialSync(initSync),
regularsync.WithStateNotifier(b),
regularsync.WithBlockNotifier(b),
regularsync.WithAttestationNotifier(b),
regularsync.WithOperationNotifier(b),
regularsync.WithAttestationPool(b.attestationPool),
regularsync.WithExitPool(b.exitPool),
regularsync.WithSlashingPool(b.slashingsPool),
regularsync.WithSyncCommsPool(b.syncCommitteePool),
regularsync.WithStateGen(b.stateGen),
regularsync.WithSlasherAttestationsFeed(b.slasherAttestationsFeed),
regularsync.WithSlasherBlockHeadersFeed(b.slasherBlockHeadersFeed),
)
return b.services.RegisterService(rs)
}

View File

@@ -13,6 +13,7 @@ go_library(
"fuzz_exports.go", # keep
"log.go",
"metrics.go",
"options.go",
"pending_attestations_queue.go",
"pending_blocks_queue.go",
"rate_limiter.go",

View File

@@ -24,7 +24,7 @@ func (s *Service) decodePubsubMessage(msg *pubsub.Message) (ssz.Unmarshaler, err
if err != nil {
return nil, errors.Wrapf(err, "extraction failed for topic: %s", topic)
}
topic = strings.TrimSuffix(topic, s.cfg.P2P.Encoding().ProtocolSuffix())
topic = strings.TrimSuffix(topic, s.cfg.p2p.Encoding().ProtocolSuffix())
topic, err = s.replaceForkDigest(topic)
if err != nil {
return nil, err
@@ -49,12 +49,12 @@ func (s *Service) decodePubsubMessage(msg *pubsub.Message) (ssz.Unmarshaler, err
}
// Handle different message types across forks.
if topic == p2p.BlockSubnetTopicFormat {
m, err = extractBlockDataType(fDigest[:], s.cfg.Chain)
m, err = extractBlockDataType(fDigest[:], s.cfg.chain)
if err != nil {
return nil, err
}
}
if err := s.cfg.P2P.Encoding().DecodeGossip(msg.Data, m); err != nil {
if err := s.cfg.p2p.Encoding().DecodeGossip(msg.Data, m); err != nil {
return nil, err
}
return m, nil

View File

@@ -77,7 +77,7 @@ func TestService_decodePubsubMessage(t *testing.T) {
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
s := &Service{
cfg: &Config{P2P: p2ptesting.NewTestP2P(t), Chain: &mock.ChainService{ValidatorsRoot: [32]byte{}, Genesis: time.Now()}},
cfg: &config{p2p: p2ptesting.NewTestP2P(t), chain: &mock.ChainService{ValidatorsRoot: [32]byte{}, Genesis: time.Now()}},
}
if tt.topic != "" {
if tt.input == nil {

View File

@@ -19,7 +19,7 @@ var responseCodeInvalidRequest = byte(0x01)
var responseCodeServerError = byte(0x02)
func (s *Service) generateErrorResponse(code byte, reason string) ([]byte, error) {
return createErrorResponse(code, reason, s.cfg.P2P)
return createErrorResponse(code, reason, s.cfg.p2p)
}
// ReadStatusCode response from a RPC stream.

View File

@@ -12,7 +12,7 @@ import (
func TestRegularSync_generateErrorResponse(t *testing.T) {
r := &Service{
cfg: &Config{P2P: p2ptest.NewTestP2P(t)},
cfg: &config{p2p: p2ptest.NewTestP2P(t)},
}
data, err := r.generateErrorResponse(responseCodeServerError, "something bad happened")
require.NoError(t, err)
@@ -23,6 +23,6 @@ func TestRegularSync_generateErrorResponse(t *testing.T) {
require.NoError(t, err)
assert.Equal(t, responseCodeServerError, b[0], "The first byte was not the status code")
msg := &types.ErrorMessage{}
require.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(buf, msg))
require.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(buf, msg))
assert.Equal(t, "something bad happened", string(*msg), "Received the wrong message")
}

View File

@@ -12,7 +12,7 @@ import (
// Is a background routine that observes for new incoming forks. Depending on the epoch
// it will be in charge of subscribing/unsubscribing the relevant topics at the fork boundaries.
func (s *Service) forkWatcher() {
slotTicker := slots.NewSlotTicker(s.cfg.Chain.GenesisTime(), params.BeaconConfig().SecondsPerSlot)
slotTicker := slots.NewSlotTicker(s.cfg.chain.GenesisTime(), params.BeaconConfig().SecondsPerSlot)
for {
select {
// In the event of a node restart, we will still end up subscribing to the correct
@@ -39,8 +39,8 @@ func (s *Service) forkWatcher() {
// Checks if there is a fork in the next epoch and if there is
// it registers the appropriate gossip and rpc topics.
func (s *Service) registerForUpcomingFork(currEpoch types.Epoch) error {
genRoot := s.cfg.Chain.GenesisValidatorRoot()
isNextForkEpoch, err := forks.IsForkNextEpoch(s.cfg.Chain.GenesisTime(), genRoot[:])
genRoot := s.cfg.chain.GenesisValidatorRoot()
isNextForkEpoch, err := forks.IsForkNextEpoch(s.cfg.chain.GenesisTime(), genRoot[:])
if err != nil {
return errors.Wrap(err, "Could not retrieve next fork epoch")
}
@@ -67,7 +67,7 @@ func (s *Service) registerForUpcomingFork(currEpoch types.Epoch) error {
// Checks if there was a fork in the previous epoch, and if there
// was then we deregister the topics from that particular fork.
func (s *Service) deregisterFromPastFork(currEpoch types.Epoch) error {
genRoot := s.cfg.Chain.GenesisValidatorRoot()
genRoot := s.cfg.chain.GenesisValidatorRoot()
// This method takes care of the de-registration of
// old gossip pubsub handlers. Once we are at the epoch
// after the fork, we de-register from all the outdated topics.

View File

@@ -37,11 +37,11 @@ func TestService_CheckForNextEpochFork(t *testing.T) {
r := &Service{
ctx: ctx,
cancel: cancel,
cfg: &Config{
P2P: p2p,
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
InitialSync: &mockSync.Sync{IsSyncing: false},
cfg: &config{
p2p: p2p,
chain: chainService,
stateNotifier: chainService.StateNotifier(),
initialSync: &mockSync.Sync{IsSyncing: false},
},
chainStarted: abool.New(),
subHandler: newSubTopicHandler(),
@@ -70,11 +70,11 @@ func TestService_CheckForNextEpochFork(t *testing.T) {
r := &Service{
ctx: ctx,
cancel: cancel,
cfg: &Config{
P2P: p2p,
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
InitialSync: &mockSync.Sync{IsSyncing: false},
cfg: &config{
p2p: p2p,
chain: chainService,
stateNotifier: chainService.StateNotifier(),
initialSync: &mockSync.Sync{IsSyncing: false},
},
chainStarted: abool.New(),
subHandler: newSubTopicHandler(),
@@ -84,17 +84,17 @@ func TestService_CheckForNextEpochFork(t *testing.T) {
currEpoch: 4,
wantErr: false,
postSvcCheck: func(t *testing.T, s *Service) {
genRoot := s.cfg.Chain.GenesisValidatorRoot()
genRoot := s.cfg.chain.GenesisValidatorRoot()
digest, err := forks.ForkDigestFromEpoch(5, genRoot[:])
assert.NoError(t, err)
assert.Equal(t, true, s.subHandler.digestExists(digest))
rpcMap := make(map[string]bool)
for _, p := range s.cfg.P2P.Host().Mux().Protocols() {
for _, p := range s.cfg.p2p.Host().Mux().Protocols() {
rpcMap[p] = true
}
assert.Equal(t, true, rpcMap[p2p.RPCBlocksByRangeTopicV2+s.cfg.P2P.Encoding().ProtocolSuffix()], "topic doesn't exist")
assert.Equal(t, true, rpcMap[p2p.RPCBlocksByRootTopicV2+s.cfg.P2P.Encoding().ProtocolSuffix()], "topic doesn't exist")
assert.Equal(t, true, rpcMap[p2p.RPCMetaDataTopicV2+s.cfg.P2P.Encoding().ProtocolSuffix()], "topic doesn't exist")
assert.Equal(t, true, rpcMap[p2p.RPCBlocksByRangeTopicV2+s.cfg.p2p.Encoding().ProtocolSuffix()], "topic doesn't exist")
assert.Equal(t, true, rpcMap[p2p.RPCBlocksByRootTopicV2+s.cfg.p2p.Encoding().ProtocolSuffix()], "topic doesn't exist")
assert.Equal(t, true, rpcMap[p2p.RPCMetaDataTopicV2+s.cfg.p2p.Encoding().ProtocolSuffix()], "topic doesn't exist")
},
},
}
@@ -130,11 +130,11 @@ func TestService_CheckForPreviousEpochFork(t *testing.T) {
r := &Service{
ctx: ctx,
cancel: cancel,
cfg: &Config{
P2P: p2p,
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
InitialSync: &mockSync.Sync{IsSyncing: false},
cfg: &config{
p2p: p2p,
chain: chainService,
stateNotifier: chainService.StateNotifier(),
initialSync: &mockSync.Sync{IsSyncing: false},
},
chainStarted: abool.New(),
subHandler: newSubTopicHandler(),
@@ -145,17 +145,17 @@ func TestService_CheckForPreviousEpochFork(t *testing.T) {
currEpoch: 10,
wantErr: false,
postSvcCheck: func(t *testing.T, s *Service) {
ptcls := s.cfg.P2P.Host().Mux().Protocols()
ptcls := s.cfg.p2p.Host().Mux().Protocols()
pMap := make(map[string]bool)
for _, p := range ptcls {
pMap[p] = true
}
assert.Equal(t, true, pMap[p2p.RPCGoodByeTopicV1+s.cfg.P2P.Encoding().ProtocolSuffix()])
assert.Equal(t, true, pMap[p2p.RPCStatusTopicV1+s.cfg.P2P.Encoding().ProtocolSuffix()])
assert.Equal(t, true, pMap[p2p.RPCPingTopicV1+s.cfg.P2P.Encoding().ProtocolSuffix()])
assert.Equal(t, true, pMap[p2p.RPCMetaDataTopicV1+s.cfg.P2P.Encoding().ProtocolSuffix()])
assert.Equal(t, true, pMap[p2p.RPCBlocksByRangeTopicV1+s.cfg.P2P.Encoding().ProtocolSuffix()])
assert.Equal(t, true, pMap[p2p.RPCBlocksByRootTopicV1+s.cfg.P2P.Encoding().ProtocolSuffix()])
assert.Equal(t, true, pMap[p2p.RPCGoodByeTopicV1+s.cfg.p2p.Encoding().ProtocolSuffix()])
assert.Equal(t, true, pMap[p2p.RPCStatusTopicV1+s.cfg.p2p.Encoding().ProtocolSuffix()])
assert.Equal(t, true, pMap[p2p.RPCPingTopicV1+s.cfg.p2p.Encoding().ProtocolSuffix()])
assert.Equal(t, true, pMap[p2p.RPCMetaDataTopicV1+s.cfg.p2p.Encoding().ProtocolSuffix()])
assert.Equal(t, true, pMap[p2p.RPCBlocksByRangeTopicV1+s.cfg.p2p.Encoding().ProtocolSuffix()])
assert.Equal(t, true, pMap[p2p.RPCBlocksByRootTopicV1+s.cfg.p2p.Encoding().ProtocolSuffix()])
},
},
{
@@ -174,11 +174,11 @@ func TestService_CheckForPreviousEpochFork(t *testing.T) {
r := &Service{
ctx: ctx,
cancel: cancel,
cfg: &Config{
P2P: p2p,
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
InitialSync: &mockSync.Sync{IsSyncing: false},
cfg: &config{
p2p: p2p,
chain: chainService,
stateNotifier: chainService.StateNotifier(),
initialSync: &mockSync.Sync{IsSyncing: false},
},
chainStarted: abool.New(),
subHandler: newSubTopicHandler(),
@@ -191,7 +191,7 @@ func TestService_CheckForPreviousEpochFork(t *testing.T) {
chainService.Genesis = prevGenesis
r.registerRPCHandlersAltair()
genRoot := r.cfg.Chain.GenesisValidatorRoot()
genRoot := r.cfg.chain.GenesisValidatorRoot()
digest, err := forks.ForkDigestFromEpoch(0, genRoot[:])
assert.NoError(t, err)
r.registerSubscribers(0, digest)
@@ -207,7 +207,7 @@ func TestService_CheckForPreviousEpochFork(t *testing.T) {
currEpoch: 4,
wantErr: false,
postSvcCheck: func(t *testing.T, s *Service) {
genRoot := s.cfg.Chain.GenesisValidatorRoot()
genRoot := s.cfg.chain.GenesisValidatorRoot()
digest, err := forks.ForkDigestFromEpoch(0, genRoot[:])
assert.NoError(t, err)
assert.Equal(t, false, s.subHandler.digestExists(digest))
@@ -215,21 +215,21 @@ func TestService_CheckForPreviousEpochFork(t *testing.T) {
assert.NoError(t, err)
assert.Equal(t, true, s.subHandler.digestExists(digest))
ptcls := s.cfg.P2P.Host().Mux().Protocols()
ptcls := s.cfg.p2p.Host().Mux().Protocols()
pMap := make(map[string]bool)
for _, p := range ptcls {
pMap[p] = true
}
assert.Equal(t, true, pMap[p2p.RPCGoodByeTopicV1+s.cfg.P2P.Encoding().ProtocolSuffix()])
assert.Equal(t, true, pMap[p2p.RPCStatusTopicV1+s.cfg.P2P.Encoding().ProtocolSuffix()])
assert.Equal(t, true, pMap[p2p.RPCPingTopicV1+s.cfg.P2P.Encoding().ProtocolSuffix()])
assert.Equal(t, true, pMap[p2p.RPCMetaDataTopicV2+s.cfg.P2P.Encoding().ProtocolSuffix()])
assert.Equal(t, true, pMap[p2p.RPCBlocksByRangeTopicV2+s.cfg.P2P.Encoding().ProtocolSuffix()])
assert.Equal(t, true, pMap[p2p.RPCBlocksByRootTopicV2+s.cfg.P2P.Encoding().ProtocolSuffix()])
assert.Equal(t, true, pMap[p2p.RPCGoodByeTopicV1+s.cfg.p2p.Encoding().ProtocolSuffix()])
assert.Equal(t, true, pMap[p2p.RPCStatusTopicV1+s.cfg.p2p.Encoding().ProtocolSuffix()])
assert.Equal(t, true, pMap[p2p.RPCPingTopicV1+s.cfg.p2p.Encoding().ProtocolSuffix()])
assert.Equal(t, true, pMap[p2p.RPCMetaDataTopicV2+s.cfg.p2p.Encoding().ProtocolSuffix()])
assert.Equal(t, true, pMap[p2p.RPCBlocksByRangeTopicV2+s.cfg.p2p.Encoding().ProtocolSuffix()])
assert.Equal(t, true, pMap[p2p.RPCBlocksByRootTopicV2+s.cfg.p2p.Encoding().ProtocolSuffix()])
assert.Equal(t, false, pMap[p2p.RPCMetaDataTopicV1+s.cfg.P2P.Encoding().ProtocolSuffix()])
assert.Equal(t, false, pMap[p2p.RPCBlocksByRangeTopicV1+s.cfg.P2P.Encoding().ProtocolSuffix()])
assert.Equal(t, false, pMap[p2p.RPCBlocksByRootTopicV1+s.cfg.P2P.Encoding().ProtocolSuffix()])
assert.Equal(t, false, pMap[p2p.RPCMetaDataTopicV1+s.cfg.p2p.Encoding().ProtocolSuffix()])
assert.Equal(t, false, pMap[p2p.RPCBlocksByRangeTopicV1+s.cfg.p2p.Encoding().ProtocolSuffix()])
assert.Equal(t, false, pMap[p2p.RPCBlocksByRootTopicV1+s.cfg.p2p.Encoding().ProtocolSuffix()])
},
},
}

View File

@@ -1,3 +1,4 @@
//go:build libfuzzer
// +build libfuzzer
package sync
@@ -14,18 +15,17 @@ import (
)
// NewRegularSyncFuzz service without registering handlers.
func NewRegularSyncFuzz(cfg *Config) *Service {
rLimiter := newRateLimiter(cfg.P2P)
func NewRegularSyncFuzz(opts ...Option) *Service {
ctx, cancel := context.WithCancel(context.Background())
r := &Service{
cfg: cfg,
cfg: &config{},
ctx: ctx,
cancel: cancel,
slotToPendingBlocks: gcache.New(time.Second, 2*time.Second),
seenPendingBlocks: make(map[[32]byte]bool),
blkRootToPendingAtts: make(map[[32]byte][]*ethpb.SignedAggregateAttestationAndProof),
rateLimiter: rLimiter,
}
r.rateLimiter = newRateLimiter(r.cfg.p2p)
return r
}

View File

@@ -75,7 +75,7 @@ var (
func (s *Service) updateMetrics() {
// do not update metrics if genesis time
// has not been initialized
if s.cfg.Chain.GenesisTime().IsZero() {
if s.cfg.chain.GenesisTime().IsZero() {
return
}
// We update the dynamic subnet topics.
@@ -83,12 +83,12 @@ func (s *Service) updateMetrics() {
if err != nil {
log.WithError(err).Debugf("Could not compute fork digest")
}
indices := s.aggregatorSubnetIndices(s.cfg.Chain.CurrentSlot())
syncIndices := cache.SyncSubnetIDs.GetAllSubnets(slots.ToEpoch(s.cfg.Chain.CurrentSlot()))
indices := s.aggregatorSubnetIndices(s.cfg.chain.CurrentSlot())
syncIndices := cache.SyncSubnetIDs.GetAllSubnets(slots.ToEpoch(s.cfg.chain.CurrentSlot()))
attTopic := p2p.GossipTypeMapping[reflect.TypeOf(&pb.Attestation{})]
syncTopic := p2p.GossipTypeMapping[reflect.TypeOf(&pb.SyncCommitteeMessage{})]
attTopic += s.cfg.P2P.Encoding().ProtocolSuffix()
syncTopic += s.cfg.P2P.Encoding().ProtocolSuffix()
attTopic += s.cfg.p2p.Encoding().ProtocolSuffix()
syncTopic += s.cfg.p2p.Encoding().ProtocolSuffix()
if flags.Get().SubscribeToAllSubnets {
for i := uint64(0); i < params.BeaconNetworkConfig().AttestationSubnetCount; i++ {
s.collectMetricForSubnet(attTopic, digest, i)
@@ -111,21 +111,21 @@ func (s *Service) updateMetrics() {
if strings.Contains(topic, p2p.GossipAttestationMessage) || strings.Contains(topic, p2p.GossipSyncCommitteeMessage) {
continue
}
topic += s.cfg.P2P.Encoding().ProtocolSuffix()
topic += s.cfg.p2p.Encoding().ProtocolSuffix()
if !strings.Contains(topic, "%x") {
topicPeerCount.WithLabelValues(topic).Set(float64(len(s.cfg.P2P.PubSub().ListPeers(topic))))
topicPeerCount.WithLabelValues(topic).Set(float64(len(s.cfg.p2p.PubSub().ListPeers(topic))))
continue
}
formattedTopic := fmt.Sprintf(topic, digest)
topicPeerCount.WithLabelValues(formattedTopic).Set(float64(len(s.cfg.P2P.PubSub().ListPeers(formattedTopic))))
topicPeerCount.WithLabelValues(formattedTopic).Set(float64(len(s.cfg.p2p.PubSub().ListPeers(formattedTopic))))
}
for _, topic := range s.cfg.P2P.PubSub().GetTopics() {
subscribedTopicPeerCount.WithLabelValues(topic).Set(float64(len(s.cfg.P2P.PubSub().ListPeers(topic))))
for _, topic := range s.cfg.p2p.PubSub().GetTopics() {
subscribedTopicPeerCount.WithLabelValues(topic).Set(float64(len(s.cfg.p2p.PubSub().ListPeers(topic))))
}
}
func (s *Service) collectMetricForSubnet(topic string, digest [4]byte, index uint64) {
formattedTopic := fmt.Sprintf(topic, digest, index)
topicPeerCount.WithLabelValues(formattedTopic).Set(float64(len(s.cfg.P2P.PubSub().ListPeers(formattedTopic))))
topicPeerCount.WithLabelValues(formattedTopic).Set(float64(len(s.cfg.p2p.PubSub().ListPeers(formattedTopic))))
}

View File

@@ -0,0 +1,122 @@
package sync
import (
"github.com/prysmaticlabs/prysm/async/event"
blockfeed "github.com/prysmaticlabs/prysm/beacon-chain/core/feed/block"
"github.com/prysmaticlabs/prysm/beacon-chain/core/feed/operation"
statefeed "github.com/prysmaticlabs/prysm/beacon-chain/core/feed/state"
"github.com/prysmaticlabs/prysm/beacon-chain/db"
"github.com/prysmaticlabs/prysm/beacon-chain/operations/attestations"
"github.com/prysmaticlabs/prysm/beacon-chain/operations/slashings"
"github.com/prysmaticlabs/prysm/beacon-chain/operations/synccommittee"
"github.com/prysmaticlabs/prysm/beacon-chain/operations/voluntaryexits"
"github.com/prysmaticlabs/prysm/beacon-chain/p2p"
"github.com/prysmaticlabs/prysm/beacon-chain/state/stategen"
)
type Option func(s *Service) error
func WithAttestationNotifier(notifier operation.Notifier) Option {
return func(s *Service) error {
s.cfg.attestationNotifier = notifier
return nil
}
}
func WithP2P(p2p p2p.P2P) Option {
return func(s *Service) error {
s.cfg.p2p = p2p
return nil
}
}
func WithDatabase(db db.NoHeadAccessDatabase) Option {
return func(s *Service) error {
s.cfg.beaconDB = db
return nil
}
}
func WithAttestationPool(attPool attestations.Pool) Option {
return func(s *Service) error {
s.cfg.attPool = attPool
return nil
}
}
func WithExitPool(exitPool voluntaryexits.PoolManager) Option {
return func(s *Service) error {
s.cfg.exitPool = exitPool
return nil
}
}
func WithSlashingPool(slashingPool slashings.PoolManager) Option {
return func(s *Service) error {
s.cfg.slashingPool = slashingPool
return nil
}
}
func WithSyncCommsPool(syncCommsPool synccommittee.Pool) Option {
return func(s *Service) error {
s.cfg.syncCommsPool = syncCommsPool
return nil
}
}
func WithChainService(chain blockchainService) Option {
return func(s *Service) error {
s.cfg.chain = chain
return nil
}
}
func WithInitialSync(initialSync Checker) Option {
return func(s *Service) error {
s.cfg.initialSync = initialSync
return nil
}
}
func WithStateNotifier(stateNotifier statefeed.Notifier) Option {
return func(s *Service) error {
s.cfg.stateNotifier = stateNotifier
return nil
}
}
func WithBlockNotifier(blockNotifier blockfeed.Notifier) Option {
return func(s *Service) error {
s.cfg.blockNotifier = blockNotifier
return nil
}
}
func WithOperationNotifier(operationNotifier operation.Notifier) Option {
return func(s *Service) error {
s.cfg.operationNotifier = operationNotifier
return nil
}
}
func WithStateGen(stateGen *stategen.State) Option {
return func(s *Service) error {
s.cfg.stateGen = stateGen
return nil
}
}
func WithSlasherAttestationsFeed(slasherAttestationsFeed *event.Feed) Option {
return func(s *Service) error {
s.cfg.slasherAttestationsFeed = slasherAttestationsFeed
return nil
}
}
func WithSlasherBlockHeadersFeed(slasherBlockHeadersFeed *event.Feed) Option {
return func(s *Service) error {
s.cfg.slasherBlockHeadersFeed = slasherBlockHeadersFeed
return nil
}
}

View File

@@ -45,7 +45,7 @@ func (s *Service) processPendingAtts(ctx context.Context) error {
// Before a node processes pending attestations queue, it verifies
// the attestations in the queue are still valid. Attestations will
// be deleted from the queue if invalid (ie. getting staled from falling too many slots behind).
s.validatePendingAtts(ctx, s.cfg.Chain.CurrentSlot())
s.validatePendingAtts(ctx, s.cfg.chain.CurrentSlot())
s.pendingAttsLock.RLock()
roots := make([][32]byte, 0, len(s.blkRootToPendingAtts))
@@ -61,7 +61,7 @@ func (s *Service) processPendingAtts(ctx context.Context) error {
attestations := s.blkRootToPendingAtts[bRoot]
s.pendingAttsLock.RUnlock()
// has the pending attestation's missing block arrived and the node processed block yet?
if s.cfg.DB.HasBlock(ctx, bRoot) && (s.cfg.DB.HasState(ctx, bRoot) || s.cfg.DB.HasStateSummary(ctx, bRoot)) {
if s.cfg.beaconDB.HasBlock(ctx, bRoot) && (s.cfg.beaconDB.HasState(ctx, bRoot) || s.cfg.beaconDB.HasStateSummary(ctx, bRoot)) {
for _, signedAtt := range attestations {
att := signedAtt.Message
// The pending attestations can arrive in both aggregated and unaggregated forms,
@@ -75,14 +75,14 @@ func (s *Service) processPendingAtts(ctx context.Context) error {
}
aggValid := pubsub.ValidationAccept == valRes
if s.validateBlockInAttestation(ctx, signedAtt) && aggValid {
if err := s.cfg.AttPool.SaveAggregatedAttestation(att.Aggregate); err != nil {
if err := s.cfg.attPool.SaveAggregatedAttestation(att.Aggregate); err != nil {
log.WithError(err).Debug("Could not save aggregate attestation")
continue
}
s.setAggregatorIndexEpochSeen(att.Aggregate.Data.Target.Epoch, att.AggregatorIndex)
// Broadcasting the signed attestation again once a node is able to process it.
if err := s.cfg.P2P.Broadcast(ctx, signedAtt); err != nil {
if err := s.cfg.p2p.Broadcast(ctx, signedAtt); err != nil {
log.WithError(err).Debug("Could not broadcast")
}
}
@@ -90,15 +90,15 @@ func (s *Service) processPendingAtts(ctx context.Context) error {
// This is an important validation before retrieving attestation pre state to defend against
// attestation's target intentionally reference checkpoint that's long ago.
// Verify current finalized checkpoint is an ancestor of the block defined by the attestation's beacon block root.
if err := s.cfg.Chain.VerifyFinalizedConsistency(ctx, att.Aggregate.Data.BeaconBlockRoot); err != nil {
if err := s.cfg.chain.VerifyFinalizedConsistency(ctx, att.Aggregate.Data.BeaconBlockRoot); err != nil {
log.WithError(err).Debug("Could not verify finalized consistency")
continue
}
if err := s.cfg.Chain.VerifyLmdFfgConsistency(ctx, att.Aggregate); err != nil {
if err := s.cfg.chain.VerifyLmdFfgConsistency(ctx, att.Aggregate); err != nil {
log.WithError(err).Debug("Could not verify FFG consistency")
continue
}
preState, err := s.cfg.Chain.AttestationTargetState(ctx, att.Aggregate.Data.Target)
preState, err := s.cfg.chain.AttestationTargetState(ctx, att.Aggregate.Data.Target)
if err != nil {
log.WithError(err).Debug("Could not retrieve attestation prestate")
continue
@@ -110,7 +110,7 @@ func (s *Service) processPendingAtts(ctx context.Context) error {
continue
}
if valid == pubsub.ValidationAccept {
if err := s.cfg.AttPool.SaveUnaggregatedAttestation(att.Aggregate); err != nil {
if err := s.cfg.attPool.SaveUnaggregatedAttestation(att.Aggregate); err != nil {
log.WithError(err).Debug("Could not save unaggregated attestation")
continue
}
@@ -122,7 +122,7 @@ func (s *Service) processPendingAtts(ctx context.Context) error {
continue
}
// Broadcasting the signed attestation again once a node is able to process it.
if err := s.cfg.P2P.BroadcastAttestation(ctx, helpers.ComputeSubnetForAttestation(valCount, signedAtt.Message.Aggregate), signedAtt.Message.Aggregate); err != nil {
if err := s.cfg.p2p.BroadcastAttestation(ctx, helpers.ComputeSubnetForAttestation(valCount, signedAtt.Message.Aggregate), signedAtt.Message.Aggregate); err != nil {
log.WithError(err).Debug("Could not broadcast")
}
}
@@ -140,7 +140,7 @@ func (s *Service) processPendingAtts(ctx context.Context) error {
} else {
// Pending attestation's missing block has not arrived yet.
log.WithFields(logrus.Fields{
"currentSlot": s.cfg.Chain.CurrentSlot(),
"currentSlot": s.cfg.chain.CurrentSlot(),
"attSlot": attestations[0].Message.Aggregate.Data.Slot,
"attCount": len(attestations),
"blockRoot": hex.EncodeToString(bytesutil.Trunc(bRoot[:])),

View File

@@ -44,7 +44,7 @@ func TestProcessPendingAtts_NoBlockRequestBlock(t *testing.T) {
p1.Peers().SetChainState(p2.PeerID(), &pb.Status{})
r := &Service{
cfg: &Config{P2P: p1, DB: db, Chain: &mock.ChainService{Genesis: prysmTime.Now(), FinalizedCheckPoint: &ethpb.Checkpoint{}}},
cfg: &config{p2p: p1, beaconDB: db, chain: &mock.ChainService{Genesis: prysmTime.Now(), FinalizedCheckPoint: &ethpb.Checkpoint{}}},
blkRootToPendingAtts: make(map[[32]byte][]*ethpb.SignedAggregateAttestationAndProof),
chainStarted: abool.New(),
}
@@ -107,16 +107,16 @@ func TestProcessPendingAtts_HasBlockSaveUnAggregatedAtt(t *testing.T) {
require.NoError(t, beaconState.SetGenesisTime(uint64(time.Now().Unix())))
r := &Service{
cfg: &Config{
P2P: p1,
DB: db,
Chain: &mock.ChainService{Genesis: time.Now(),
cfg: &config{
p2p: p1,
beaconDB: db,
chain: &mock.ChainService{Genesis: time.Now(),
State: beaconState,
FinalizedCheckPoint: &ethpb.Checkpoint{
Root: aggregateAndProof.Aggregate.Data.BeaconBlockRoot,
Epoch: 0,
}},
AttPool: attestations.NewPool(),
attPool: attestations.NewPool(),
},
blkRootToPendingAtts: make(map[[32]byte][]*ethpb.SignedAggregateAttestationAndProof),
seenUnAggregatedAttestationCache: lruwrpr.New(10),
@@ -124,16 +124,16 @@ func TestProcessPendingAtts_HasBlockSaveUnAggregatedAtt(t *testing.T) {
s, err := util.NewBeaconState()
require.NoError(t, err)
require.NoError(t, r.cfg.DB.SaveState(context.Background(), s, root))
require.NoError(t, r.cfg.beaconDB.SaveState(context.Background(), s, root))
r.blkRootToPendingAtts[root] = []*ethpb.SignedAggregateAttestationAndProof{{Message: aggregateAndProof, Signature: aggreSig}}
require.NoError(t, r.processPendingAtts(context.Background()))
atts, err := r.cfg.AttPool.UnaggregatedAttestations()
atts, err := r.cfg.attPool.UnaggregatedAttestations()
require.NoError(t, err)
assert.Equal(t, 1, len(atts), "Did not save unaggregated att")
assert.DeepEqual(t, att, atts[0], "Incorrect saved att")
assert.Equal(t, 0, len(r.cfg.AttPool.AggregatedAttestations()), "Did save aggregated att")
assert.Equal(t, 0, len(r.cfg.attPool.AggregatedAttestations()), "Did save aggregated att")
require.LogsContain(t, hook, "Verified and saved pending attestations to pool")
}
@@ -143,11 +143,11 @@ func TestProcessPendingAtts_NoBroadcastWithBadSignature(t *testing.T) {
s, _ := util.DeterministicGenesisState(t, 256)
r := &Service{
cfg: &Config{
P2P: p1,
DB: db,
Chain: &mock.ChainService{State: s, Genesis: prysmTime.Now(), FinalizedCheckPoint: &ethpb.Checkpoint{Root: make([]byte, 32)}},
AttPool: attestations.NewPool(),
cfg: &config{
p2p: p1,
beaconDB: db,
chain: &mock.ChainService{State: s, Genesis: prysmTime.Now(), FinalizedCheckPoint: &ethpb.Checkpoint{Root: make([]byte, 32)}},
attPool: attestations.NewPool(),
},
blkRootToPendingAtts: make(map[[32]byte][]*ethpb.SignedAggregateAttestationAndProof),
}
@@ -166,15 +166,15 @@ func TestProcessPendingAtts_NoBroadcastWithBadSignature(t *testing.T) {
b := util.NewBeaconBlock()
r32, err := b.Block.HashTreeRoot()
require.NoError(t, err)
require.NoError(t, r.cfg.DB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b)))
require.NoError(t, r.cfg.DB.SaveState(context.Background(), s, r32))
require.NoError(t, r.cfg.beaconDB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b)))
require.NoError(t, r.cfg.beaconDB.SaveState(context.Background(), s, r32))
r.blkRootToPendingAtts[r32] = []*ethpb.SignedAggregateAttestationAndProof{{Message: a, Signature: make([]byte, params.BeaconConfig().BLSSignatureLength)}}
require.NoError(t, r.processPendingAtts(context.Background()))
assert.Equal(t, false, p1.BroadcastCalled, "Broadcasted bad aggregate")
// Clear pool.
err = r.cfg.AttPool.DeleteUnaggregatedAttestation(a.Aggregate)
err = r.cfg.attPool.DeleteUnaggregatedAttestation(a.Aggregate)
require.NoError(t, err)
validators := uint64(256)
@@ -217,16 +217,16 @@ func TestProcessPendingAtts_NoBroadcastWithBadSignature(t *testing.T) {
require.NoError(t, s.SetGenesisTime(uint64(time.Now().Unix())))
r = &Service{
cfg: &Config{
P2P: p1,
DB: db,
Chain: &mock.ChainService{Genesis: time.Now(),
cfg: &config{
p2p: p1,
beaconDB: db,
chain: &mock.ChainService{Genesis: time.Now(),
State: s,
FinalizedCheckPoint: &ethpb.Checkpoint{
Root: aggregateAndProof.Aggregate.Data.BeaconBlockRoot,
Epoch: 0,
}},
AttPool: attestations.NewPool(),
attPool: attestations.NewPool(),
},
blkRootToPendingAtts: make(map[[32]byte][]*ethpb.SignedAggregateAttestationAndProof),
seenUnAggregatedAttestationCache: lruwrpr.New(10),
@@ -294,16 +294,16 @@ func TestProcessPendingAtts_HasBlockSaveAggregatedAtt(t *testing.T) {
require.NoError(t, beaconState.SetGenesisTime(uint64(time.Now().Unix())))
r := &Service{
cfg: &Config{
P2P: p1,
DB: db,
Chain: &mock.ChainService{Genesis: time.Now(),
cfg: &config{
p2p: p1,
beaconDB: db,
chain: &mock.ChainService{Genesis: time.Now(),
State: beaconState,
FinalizedCheckPoint: &ethpb.Checkpoint{
Root: aggregateAndProof.Aggregate.Data.BeaconBlockRoot,
Epoch: 0,
}},
AttPool: attestations.NewPool(),
attPool: attestations.NewPool(),
},
blkRootToPendingAtts: make(map[[32]byte][]*ethpb.SignedAggregateAttestationAndProof),
seenAggregatedAttestationCache: lruwrpr.New(10),
@@ -311,14 +311,14 @@ func TestProcessPendingAtts_HasBlockSaveAggregatedAtt(t *testing.T) {
s, err := util.NewBeaconState()
require.NoError(t, err)
require.NoError(t, r.cfg.DB.SaveState(context.Background(), s, root))
require.NoError(t, r.cfg.beaconDB.SaveState(context.Background(), s, root))
r.blkRootToPendingAtts[root] = []*ethpb.SignedAggregateAttestationAndProof{{Message: aggregateAndProof, Signature: aggreSig}}
require.NoError(t, r.processPendingAtts(context.Background()))
assert.Equal(t, 1, len(r.cfg.AttPool.AggregatedAttestations()), "Did not save aggregated att")
assert.DeepEqual(t, att, r.cfg.AttPool.AggregatedAttestations()[0], "Incorrect saved att")
atts, err := r.cfg.AttPool.UnaggregatedAttestations()
assert.Equal(t, 1, len(r.cfg.attPool.AggregatedAttestations()), "Did not save aggregated att")
assert.DeepEqual(t, att, r.cfg.attPool.AggregatedAttestations()[0], "Incorrect saved att")
atts, err := r.cfg.attPool.UnaggregatedAttestations()
require.NoError(t, err)
assert.Equal(t, 0, len(atts), "Did save aggregated att")
require.LogsContain(t, hook, "Verified and saved pending attestations to pool")

View File

@@ -48,7 +48,7 @@ func (s *Service) processPendingBlocks(ctx context.Context) error {
ctx, span := trace.StartSpan(ctx, "processPendingBlocks")
defer span.End()
pids := s.cfg.P2P.Peers().Connected()
pids := s.cfg.p2p.Peers().Connected()
if err := s.validatePendingSlots(); err != nil {
return errors.Wrap(err, "could not validate pending slots")
}
@@ -64,7 +64,7 @@ func (s *Service) processPendingBlocks(ctx context.Context) error {
for _, slot := range slots {
// process the blocks during their respective slot.
// otherwise wait for the right slot to process the block.
if slot > s.cfg.Chain.CurrentSlot() {
if slot > s.cfg.chain.CurrentSlot() {
continue
}
@@ -117,10 +117,10 @@ func (s *Service) processPendingBlocks(ctx context.Context) error {
continue
}
inDB := s.cfg.DB.HasBlock(ctx, bytesutil.ToBytes32(b.Block().ParentRoot()))
inDB := s.cfg.beaconDB.HasBlock(ctx, bytesutil.ToBytes32(b.Block().ParentRoot()))
hasPeer := len(pids) != 0
// Only request for missing parent block if it's not in DB, not in pending cache
// Only request for missing parent block if it's not in beaconDB, not in pending cache
// and has peer in the peer list.
if !inPendingQueue && !inDB && hasPeer {
log.WithFields(logrus.Fields{
@@ -148,7 +148,7 @@ func (s *Service) processPendingBlocks(ctx context.Context) error {
continue
}
if err := s.cfg.Chain.ReceiveBlock(ctx, b, blkRoot); err != nil {
if err := s.cfg.chain.ReceiveBlock(ctx, b, blkRoot); err != nil {
log.Debugf("Could not process block from slot %d: %v", b.Block().Slot(), err)
s.setBadBlock(ctx, blkRoot)
tracing.AnnotateError(span, err)
@@ -161,7 +161,7 @@ func (s *Service) processPendingBlocks(ctx context.Context) error {
s.setSeenBlockIndexSlot(b.Block().Slot(), b.Block().ProposerIndex())
// Broadcasting the block again once a node is able to process it.
if err := s.cfg.P2P.Broadcast(ctx, b.Proto()); err != nil {
if err := s.cfg.p2p.Broadcast(ctx, b.Proto()); err != nil {
log.WithError(err).Debug("Could not broadcast block")
}
@@ -191,7 +191,7 @@ func (s *Service) sendBatchRootRequest(ctx context.Context, roots [][32]byte, ra
return nil
}
_, bestPeers := s.cfg.P2P.Peers().BestFinalized(maxPeerRequest, s.cfg.Chain.FinalizedCheckpt().Epoch)
_, bestPeers := s.cfg.p2p.Peers().BestFinalized(maxPeerRequest, s.cfg.chain.FinalizedCheckpt().Epoch)
if len(bestPeers) == 0 {
return nil
}
@@ -252,7 +252,7 @@ func (s *Service) validatePendingSlots() error {
defer s.pendingQueueLock.Unlock()
oldBlockRoots := make(map[[32]byte]bool)
finalizedEpoch := s.cfg.Chain.FinalizedCheckpt().Epoch
finalizedEpoch := s.cfg.chain.FinalizedCheckpt().Epoch
if s.slotToPendingBlocks == nil {
return errors.New("slotToPendingBlocks cache can't be nil")
}

View File

@@ -38,15 +38,15 @@ func TestRegularSyncBeaconBlockSubscriber_ProcessPendingBlocks1(t *testing.T) {
p1 := p2ptest.NewTestP2P(t)
r := &Service{
cfg: &Config{
P2P: p1,
DB: db,
Chain: &mock.ChainService{
cfg: &config{
p2p: p1,
beaconDB: db,
chain: &mock.ChainService{
FinalizedCheckPoint: &ethpb.Checkpoint{
Epoch: 0,
},
},
StateGen: stategen.New(db),
stateGen: stategen.New(db),
},
slotToPendingBlocks: gcache.New(time.Second, 2*time.Second),
seenPendingBlocks: make(map[[32]byte]bool),
@@ -54,13 +54,13 @@ func TestRegularSyncBeaconBlockSubscriber_ProcessPendingBlocks1(t *testing.T) {
r.initCaches()
b0 := util.NewBeaconBlock()
require.NoError(t, r.cfg.DB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b0)))
require.NoError(t, r.cfg.beaconDB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b0)))
b0Root, err := b0.Block.HashTreeRoot()
require.NoError(t, err)
b3 := util.NewBeaconBlock()
b3.Block.Slot = 3
b3.Block.ParentRoot = b0Root[:]
require.NoError(t, r.cfg.DB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b3)))
require.NoError(t, r.cfg.beaconDB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b3)))
// Incomplete block link
b1 := util.NewBeaconBlock()
b1.Block.Slot = 1
@@ -82,7 +82,7 @@ func TestRegularSyncBeaconBlockSubscriber_ProcessPendingBlocks1(t *testing.T) {
// Add b1 to the cache
require.NoError(t, r.insertBlockToPendingQueue(b1.Block.Slot, wrapper.WrappedPhase0SignedBeaconBlock(b1), b1Root))
require.NoError(t, r.cfg.DB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b1)))
require.NoError(t, r.cfg.beaconDB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b1)))
// Insert bad b1 in the cache to verify the good one doesn't get replaced.
require.NoError(t, r.insertBlockToPendingQueue(b1.Block.Slot, wrapper.WrappedPhase0SignedBeaconBlock(util.NewBeaconBlock()), [32]byte{}))
@@ -99,10 +99,10 @@ func TestRegularSync_InsertDuplicateBlocks(t *testing.T) {
p1 := p2ptest.NewTestP2P(t)
r := &Service{
cfg: &Config{
P2P: p1,
DB: db,
Chain: &mock.ChainService{
cfg: &config{
p2p: p1,
beaconDB: db,
chain: &mock.ChainService{
FinalizedCheckPoint: &ethpb.Checkpoint{
Epoch: 0,
Root: make([]byte, 32),
@@ -116,7 +116,7 @@ func TestRegularSync_InsertDuplicateBlocks(t *testing.T) {
b0 := util.NewBeaconBlock()
b0r := [32]byte{'a'}
require.NoError(t, r.cfg.DB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b0)))
require.NoError(t, r.cfg.beaconDB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b0)))
b0Root, err := b0.Block.HashTreeRoot()
require.NoError(t, err)
b1 := util.NewBeaconBlock()
@@ -167,16 +167,16 @@ func TestRegularSyncBeaconBlockSubscriber_ProcessPendingBlocks_2Chains(t *testin
})
r := &Service{
cfg: &Config{
P2P: p1,
DB: db,
Chain: &mock.ChainService{
cfg: &config{
p2p: p1,
beaconDB: db,
chain: &mock.ChainService{
FinalizedCheckPoint: &ethpb.Checkpoint{
Epoch: 0,
Root: make([]byte, 32),
},
},
StateGen: stategen.New(db),
stateGen: stategen.New(db),
},
slotToPendingBlocks: gcache.New(time.Second, 2*time.Second),
seenPendingBlocks: make(map[[32]byte]bool),
@@ -188,13 +188,13 @@ func TestRegularSyncBeaconBlockSubscriber_ProcessPendingBlocks_2Chains(t *testin
p1.Peers().SetChainState(p2.PeerID(), &ethpb.Status{})
b0 := util.NewBeaconBlock()
require.NoError(t, r.cfg.DB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b0)))
require.NoError(t, r.cfg.beaconDB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b0)))
b0Root, err := b0.Block.HashTreeRoot()
require.NoError(t, err)
b1 := util.NewBeaconBlock()
b1.Block.Slot = 1
b1.Block.ParentRoot = b0Root[:]
require.NoError(t, r.cfg.DB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b1)))
require.NoError(t, r.cfg.beaconDB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b1)))
b1Root, err := b1.Block.HashTreeRoot()
require.NoError(t, err)
@@ -231,7 +231,7 @@ func TestRegularSyncBeaconBlockSubscriber_ProcessPendingBlocks_2Chains(t *testin
// Add b3 to the cache
require.NoError(t, r.insertBlockToPendingQueue(b3.Block.Slot, wrapper.WrappedPhase0SignedBeaconBlock(b3), b3Root))
require.NoError(t, r.cfg.DB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b3)))
require.NoError(t, r.cfg.beaconDB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b3)))
require.NoError(t, r.processPendingBlocks(context.Background())) // Marks a block as bad
require.NoError(t, r.processPendingBlocks(context.Background())) // Bad block removed on second run
@@ -242,7 +242,7 @@ func TestRegularSyncBeaconBlockSubscriber_ProcessPendingBlocks_2Chains(t *testin
// Add b2 to the cache
require.NoError(t, r.insertBlockToPendingQueue(b2.Block.Slot, wrapper.WrappedPhase0SignedBeaconBlock(b2), b2Root))
require.NoError(t, r.cfg.DB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b2)))
require.NoError(t, r.cfg.beaconDB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b2)))
require.NoError(t, r.processPendingBlocks(context.Background())) // Marks a block as bad
require.NoError(t, r.processPendingBlocks(context.Background())) // Bad block removed on second run
@@ -259,10 +259,10 @@ func TestRegularSyncBeaconBlockSubscriber_PruneOldPendingBlocks(t *testing.T) {
assert.Equal(t, 1, len(p1.BHost.Network().Peers()), "Expected peers to be connected")
r := &Service{
cfg: &Config{
P2P: p1,
DB: db,
Chain: &mock.ChainService{
cfg: &config{
p2p: p1,
beaconDB: db,
chain: &mock.ChainService{
FinalizedCheckPoint: &ethpb.Checkpoint{
Epoch: 1,
Root: make([]byte, 32),
@@ -279,13 +279,13 @@ func TestRegularSyncBeaconBlockSubscriber_PruneOldPendingBlocks(t *testing.T) {
p1.Peers().SetChainState(p1.PeerID(), &ethpb.Status{})
b0 := util.NewBeaconBlock()
require.NoError(t, r.cfg.DB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b0)))
require.NoError(t, r.cfg.beaconDB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b0)))
b0Root, err := b0.Block.HashTreeRoot()
require.NoError(t, err)
b1 := util.NewBeaconBlock()
b1.Block.Slot = 1
b1.Block.ParentRoot = b0Root[:]
require.NoError(t, r.cfg.DB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b1)))
require.NoError(t, r.cfg.beaconDB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b1)))
b1Root, err := b1.Block.HashTreeRoot()
require.NoError(t, err)
@@ -345,10 +345,10 @@ func TestService_BatchRootRequest(t *testing.T) {
assert.Equal(t, 1, len(p1.BHost.Network().Peers()), "Expected peers to be connected")
r := &Service{
cfg: &Config{
P2P: p1,
DB: db,
Chain: &mock.ChainService{
cfg: &config{
p2p: p1,
beaconDB: db,
chain: &mock.ChainService{
FinalizedCheckPoint: &ethpb.Checkpoint{
Epoch: 1,
Root: make([]byte, 32),
@@ -367,13 +367,13 @@ func TestService_BatchRootRequest(t *testing.T) {
p1.Peers().SetChainState(p2.PeerID(), &ethpb.Status{FinalizedEpoch: 2})
b0 := util.NewBeaconBlock()
require.NoError(t, r.cfg.DB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b0)))
require.NoError(t, r.cfg.beaconDB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b0)))
b0Root, err := b0.Block.HashTreeRoot()
require.NoError(t, err)
b1 := util.NewBeaconBlock()
b1.Block.Slot = 1
b1.Block.ParentRoot = b0Root[:]
require.NoError(t, r.cfg.DB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b1)))
require.NoError(t, r.cfg.beaconDB.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(b1)))
b1Root, err := b1.Block.HashTreeRoot()
require.NoError(t, err)
@@ -460,11 +460,11 @@ func TestService_ProcessPendingBlockOnCorrectSlot(t *testing.T) {
Epoch: 0,
}}
r := &Service{
cfg: &Config{
P2P: p1,
DB: db,
Chain: &mockChain,
StateGen: stategen.New(db),
cfg: &config{
p2p: p1,
beaconDB: db,
chain: &mockChain,
stateGen: stategen.New(db),
},
slotToPendingBlocks: gcache.New(time.Second, 2*time.Second),
seenPendingBlocks: make(map[[32]byte]bool),
@@ -532,11 +532,11 @@ func TestService_ProcessBadPendingBlocks(t *testing.T) {
Epoch: 0,
}}
r := &Service{
cfg: &Config{
P2P: p1,
DB: db,
Chain: &mockChain,
StateGen: stategen.New(db),
cfg: &config{
p2p: p1,
beaconDB: db,
chain: &mockChain,
stateGen: stategen.New(db),
},
slotToPendingBlocks: gcache.New(time.Second, 2*time.Second),
seenPendingBlocks: make(map[[32]byte]bool),

View File

@@ -34,7 +34,7 @@ type rpcHandler func(context.Context, interface{}, libp2pcore.Stream) error
// registerRPCHandlers for p2p RPC.
func (s *Service) registerRPCHandlers() {
currEpoch := slots.ToEpoch(s.cfg.Chain.CurrentSlot())
currEpoch := slots.ToEpoch(s.cfg.chain.CurrentSlot())
// Register V2 handlers if we are past altair fork epoch.
if currEpoch >= params.BeaconConfig().AltairForkEpoch {
s.registerRPC(
@@ -97,20 +97,20 @@ func (s *Service) registerRPCHandlersAltair() {
// Remove all v1 Stream handlers that are no longer supported
// from altair onwards.
func (s *Service) unregisterPhase0Handlers() {
fullBlockRangeTopic := p2p.RPCBlocksByRangeTopicV1 + s.cfg.P2P.Encoding().ProtocolSuffix()
fullBlockRootTopic := p2p.RPCBlocksByRootTopicV1 + s.cfg.P2P.Encoding().ProtocolSuffix()
fullMetadataTopic := p2p.RPCMetaDataTopicV1 + s.cfg.P2P.Encoding().ProtocolSuffix()
fullBlockRangeTopic := p2p.RPCBlocksByRangeTopicV1 + s.cfg.p2p.Encoding().ProtocolSuffix()
fullBlockRootTopic := p2p.RPCBlocksByRootTopicV1 + s.cfg.p2p.Encoding().ProtocolSuffix()
fullMetadataTopic := p2p.RPCMetaDataTopicV1 + s.cfg.p2p.Encoding().ProtocolSuffix()
s.cfg.P2P.Host().RemoveStreamHandler(protocol.ID(fullBlockRangeTopic))
s.cfg.P2P.Host().RemoveStreamHandler(protocol.ID(fullBlockRootTopic))
s.cfg.P2P.Host().RemoveStreamHandler(protocol.ID(fullMetadataTopic))
s.cfg.p2p.Host().RemoveStreamHandler(protocol.ID(fullBlockRangeTopic))
s.cfg.p2p.Host().RemoveStreamHandler(protocol.ID(fullBlockRootTopic))
s.cfg.p2p.Host().RemoveStreamHandler(protocol.ID(fullMetadataTopic))
}
// registerRPC for a given topic with an expected protobuf message type.
func (s *Service) registerRPC(baseTopic string, handle rpcHandler) {
topic := baseTopic + s.cfg.P2P.Encoding().ProtocolSuffix()
topic := baseTopic + s.cfg.p2p.Encoding().ProtocolSuffix()
log := log.WithField("topic", topic)
s.cfg.P2P.SetStreamHandler(topic, func(stream network.Stream) {
s.cfg.p2p.SetStreamHandler(topic, func(stream network.Stream) {
defer func() {
if r := recover(); r != nil {
log.WithField("error", r).Error("Panic occurred")
@@ -137,7 +137,7 @@ func (s *Service) registerRPC(baseTopic string, handle rpcHandler) {
log := log.WithField("peer", stream.Conn().RemotePeer().Pretty()).WithField("topic", string(stream.Protocol()))
// Check before hand that peer is valid.
if s.cfg.P2P.Peers().IsBad(stream.Conn().RemotePeer()) {
if s.cfg.p2p.Peers().IsBad(stream.Conn().RemotePeer()) {
if err := s.sendGoodByeAndDisconnect(ctx, p2ptypes.GoodbyeCodeBanned, stream.Conn().RemotePeer()); err != nil {
log.Debugf("Could not disconnect from peer: %v", err)
}
@@ -189,7 +189,7 @@ func (s *Service) registerRPC(baseTopic string, handle rpcHandler) {
log.Errorf("message of %T does not support marshaller interface", msg)
return
}
if err := s.cfg.P2P.Encoding().DecodeWithMaxLength(stream, msg); err != nil {
if err := s.cfg.p2p.Encoding().DecodeWithMaxLength(stream, msg); err != nil {
// Debug logs for goodbye/status errors
if strings.Contains(topic, p2p.RPCGoodByeTopicV1) || strings.Contains(topic, p2p.RPCStatusTopicV1) {
log.WithError(err).Debug("Could not decode goodbye stream message")
@@ -214,7 +214,7 @@ func (s *Service) registerRPC(baseTopic string, handle rpcHandler) {
log.Errorf("message of %T does not support marshaller interface", msg)
return
}
if err := s.cfg.P2P.Encoding().DecodeWithMaxLength(stream, msg); err != nil {
if err := s.cfg.p2p.Encoding().DecodeWithMaxLength(stream, msg); err != nil {
log.WithError(err).Debug("Could not decode stream message")
tracing.AnnotateError(span, err)
return

View File

@@ -36,7 +36,7 @@ func (s *Service) beaconBlocksByRangeRPCHandler(ctx context.Context, msg interfa
}
if err := s.validateRangeRequest(m); err != nil {
s.writeErrorResponseToStream(responseCodeInvalidRequest, err.Error(), stream)
s.cfg.P2P.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer())
s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer())
tracing.AnnotateError(span, err)
return err
}
@@ -123,7 +123,7 @@ func (s *Service) writeBlockRangeToStream(ctx context.Context, startSlot, endSlo
defer span.End()
filter := filters.NewFilter().SetStartSlot(startSlot).SetEndSlot(endSlot).SetSlotStep(step)
blks, roots, err := s.cfg.DB.Blocks(ctx, filter)
blks, roots, err := s.cfg.beaconDB.Blocks(ctx, filter)
if err != nil {
log.WithError(err).Debug("Could not retrieve blocks")
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)
@@ -183,7 +183,7 @@ func (s *Service) validateRangeRequest(r *pb.BeaconBlocksByRangeRequest) error {
// Add a buffer for possible large range requests from nodes syncing close to the
// head of the chain.
buffer := rangeLimit * 2
highestExpectedSlot := s.cfg.Chain.CurrentSlot().Add(uint64(buffer))
highestExpectedSlot := s.cfg.chain.CurrentSlot().Add(uint64(buffer))
// Ensure all request params are within appropriate bounds
if count == 0 || count > maxRequestBlocks {
@@ -215,7 +215,7 @@ func (s *Service) filterBlocks(ctx context.Context, blks []block.SignedBeaconBlo
newBlks := make([]block.SignedBeaconBlock, 0, len(blks))
for i, b := range blks {
isCanonical, err := s.cfg.Chain.IsCanonical(ctx, roots[i])
isCanonical, err := s.cfg.chain.IsCanonical(ctx, roots[i])
if err != nil {
return nil, err
}
@@ -247,11 +247,11 @@ func (s *Service) filterBlocks(ctx context.Context, blks []block.SignedBeaconBlo
}
func (s *Service) writeErrorResponseToStream(responseCode byte, reason string, stream libp2pcore.Stream) {
writeErrorResponseToStream(responseCode, reason, stream, s.cfg.P2P)
writeErrorResponseToStream(responseCode, reason, stream, s.cfg.p2p)
}
func (s *Service) retrieveGenesisBlock(ctx context.Context) (block.SignedBeaconBlock, [32]byte, error) {
genBlock, err := s.cfg.DB.GenesisBlock(ctx)
genBlock, err := s.cfg.beaconDB.GenesisBlock(ctx)
if err != nil {
return nil, [32]byte{}, err
}

View File

@@ -52,7 +52,7 @@ func TestRPCBeaconBlocksByRange_RPCHandlerReturnsBlocks(t *testing.T) {
}
// Start service with 160 as allowed blocks capacity (and almost zero capacity recovery).
r := &Service{cfg: &Config{P2P: p1, DB: d, Chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
r := &Service{cfg: &config{p2p: p1, beaconDB: d, chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
pcl := protocol.ID(p2p.RPCBlocksByRangeTopicV1)
topic := string(pcl)
r.rateLimiter.limiterMap[topic] = leakybucket.NewCollector(0.000001, int64(req.Count*10), false)
@@ -63,7 +63,7 @@ func TestRPCBeaconBlocksByRange_RPCHandlerReturnsBlocks(t *testing.T) {
for i := req.StartSlot; i < req.StartSlot.Add(req.Count*req.Step); i += types.Slot(req.Step) {
expectSuccess(t, stream)
res := util.NewBeaconBlock()
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, res))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, res))
if res.Block.Slot.SubSlot(req.StartSlot).Mod(req.Step) != 0 {
t.Errorf("Received unexpected block slot %d", res.Block.Slot)
}
@@ -114,7 +114,7 @@ func TestRPCBeaconBlocksByRange_ReturnCorrectNumberBack(t *testing.T) {
require.NoError(t, d.SaveGenesisBlockRoot(context.Background(), genRoot))
// Start service with 160 as allowed blocks capacity (and almost zero capacity recovery).
r := &Service{cfg: &Config{P2P: p1, DB: d, Chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
r := &Service{cfg: &config{p2p: p1, beaconDB: d, chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
pcl := protocol.ID(p2p.RPCBlocksByRangeTopicV1)
topic := string(pcl)
r.rateLimiter.limiterMap[topic] = leakybucket.NewCollector(0.000001, int64(req.Count*10), false)
@@ -129,7 +129,7 @@ func TestRPCBeaconBlocksByRange_ReturnCorrectNumberBack(t *testing.T) {
for i := newReq.StartSlot; i < newReq.StartSlot.Add(newReq.Count*newReq.Step); i += types.Slot(newReq.Step) {
expectSuccess(t, stream)
res := util.NewBeaconBlock()
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, res))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, res))
if res.Block.Slot.SubSlot(newReq.StartSlot).Mod(newReq.Step) != 0 {
t.Errorf("Received unexpected block slot %d", res.Block.Slot)
}
@@ -178,7 +178,7 @@ func TestRPCBeaconBlocksByRange_RPCHandlerReturnsSortedBlocks(t *testing.T) {
}
// Start service with 160 as allowed blocks capacity (and almost zero capacity recovery).
r := &Service{cfg: &Config{P2P: p1, DB: d, Chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
r := &Service{cfg: &config{p2p: p1, beaconDB: d, chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
pcl := protocol.ID(p2p.RPCBlocksByRangeTopicV1)
topic := string(pcl)
r.rateLimiter.limiterMap[topic] = leakybucket.NewCollector(0.000001, int64(req.Count*10), false)
@@ -192,7 +192,7 @@ func TestRPCBeaconBlocksByRange_RPCHandlerReturnsSortedBlocks(t *testing.T) {
for i, j := req.StartSlot, 0; i < req.StartSlot.Add(req.Count*req.Step); i += types.Slot(req.Step) {
expectSuccess(t, stream)
res := &ethpb.SignedBeaconBlock{}
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, res))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, res))
if res.Block.Slot < prevSlot {
t.Errorf("Received block is unsorted with slot %d lower than previous slot %d", res.Block.Slot, prevSlot)
}
@@ -243,7 +243,7 @@ func TestRPCBeaconBlocksByRange_ReturnsGenesisBlock(t *testing.T) {
prevRoot = rt
}
r := &Service{cfg: &Config{P2P: p1, DB: d, Chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
r := &Service{cfg: &config{p2p: p1, beaconDB: d, chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
pcl := protocol.ID(p2p.RPCBlocksByRangeTopicV1)
topic := string(pcl)
r.rateLimiter.limiterMap[topic] = leakybucket.NewCollector(10000, 10000, false)
@@ -255,12 +255,12 @@ func TestRPCBeaconBlocksByRange_ReturnsGenesisBlock(t *testing.T) {
// check for genesis block
expectSuccess(t, stream)
res := &ethpb.SignedBeaconBlock{}
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, res))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, res))
assert.Equal(t, types.Slot(0), res.Block.Slot, "genesis block was not returned")
for i := req.StartSlot.Add(req.Step); i < types.Slot(req.Count*req.Step); i += types.Slot(req.Step) {
expectSuccess(t, stream)
res := &ethpb.SignedBeaconBlock{}
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, res))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, res))
}
})
@@ -306,7 +306,7 @@ func TestRPCBeaconBlocksByRange_RPCHandlerRateLimitOverflow(t *testing.T) {
}
expectSuccess(t, stream)
res := util.NewBeaconBlock()
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, res))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, res))
if res.Block.Slot.SubSlot(req.StartSlot).Mod(req.Step) != 0 {
t.Errorf("Received unexpected block slot %d", res.Block.Slot)
}
@@ -330,7 +330,7 @@ func TestRPCBeaconBlocksByRange_RPCHandlerRateLimitOverflow(t *testing.T) {
assert.Equal(t, 1, len(p1.BHost.Network().Peers()), "Expected peers to be connected")
capacity := int64(flags.Get().BlockBatchLimit * 3)
r := &Service{cfg: &Config{P2P: p1, DB: d, Chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
r := &Service{cfg: &config{p2p: p1, beaconDB: d, chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
pcl := protocol.ID(p2p.RPCBlocksByRangeTopicV1)
topic := string(pcl)
@@ -356,7 +356,7 @@ func TestRPCBeaconBlocksByRange_RPCHandlerRateLimitOverflow(t *testing.T) {
assert.Equal(t, 1, len(p1.BHost.Network().Peers()), "Expected peers to be connected")
capacity := int64(flags.Get().BlockBatchLimit * 3)
r := &Service{cfg: &Config{P2P: p1, DB: d, Chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
r := &Service{cfg: &config{p2p: p1, beaconDB: d, chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
pcl := protocol.ID(p2p.RPCBlocksByRangeTopicV1)
topic := string(pcl)
@@ -386,7 +386,7 @@ func TestRPCBeaconBlocksByRange_RPCHandlerRateLimitOverflow(t *testing.T) {
assert.Equal(t, 1, len(p1.BHost.Network().Peers()), "Expected peers to be connected")
capacity := int64(flags.Get().BlockBatchLimit * flags.Get().BlockBatchLimitBurstFactor)
r := &Service{cfg: &Config{P2P: p1, DB: d, Chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
r := &Service{cfg: &config{p2p: p1, beaconDB: d, chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
pcl := protocol.ID(p2p.RPCBlocksByRangeTopicV1)
topic := string(pcl)
r.rateLimiter.limiterMap[topic] = leakybucket.NewCollector(0.000001, capacity, false)
@@ -418,8 +418,8 @@ func TestRPCBeaconBlocksByRange_validateRangeRequest(t *testing.T) {
slotsSinceGenesis := types.Slot(1000)
offset := int64(slotsSinceGenesis.Mul(params.BeaconConfig().SecondsPerSlot))
r := &Service{
cfg: &Config{
Chain: &chainMock.ChainService{
cfg: &config{
chain: &chainMock.ChainService{
Genesis: time.Now().Add(time.Second * time.Duration(-1*offset)),
},
},
@@ -560,7 +560,7 @@ func TestRPCBeaconBlocksByRange_EnforceResponseInvariants(t *testing.T) {
for i := req.StartSlot; i < req.StartSlot.Add(req.Count*req.Step); i += types.Slot(req.Step) {
expectSuccess(t, stream)
blk := util.NewBeaconBlock()
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, blk))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, blk))
if blk.Block.Slot.SubSlot(req.StartSlot).Mod(req.Step) != 0 {
t.Errorf("Received unexpected block slot %d", blk.Block.Slot)
}
@@ -585,7 +585,7 @@ func TestRPCBeaconBlocksByRange_EnforceResponseInvariants(t *testing.T) {
p1.Connect(p2)
assert.Equal(t, 1, len(p1.BHost.Network().Peers()), "Expected peers to be connected")
r := &Service{cfg: &Config{P2P: p1, DB: d, Chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
r := &Service{cfg: &config{p2p: p1, beaconDB: d, chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
r.rateLimiter.limiterMap[string(pcl)] = leakybucket.NewCollector(0.000001, 640, false)
req := &pb.BeaconBlocksByRangeRequest{
StartSlot: 448,
@@ -722,7 +722,7 @@ func TestRPCBeaconBlocksByRange_FilterBlocks(t *testing.T) {
break
}
blk := util.NewBeaconBlock()
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, blk))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, blk))
if blk.Block.Slot.SubSlot(req.StartSlot).Mod(req.Step) != 0 {
t.Errorf("Received unexpected block slot %d", blk.Block.Slot)
}
@@ -749,14 +749,14 @@ func TestRPCBeaconBlocksByRange_FilterBlocks(t *testing.T) {
p1.Connect(p2)
assert.Equal(t, 1, len(p1.BHost.Network().Peers()), "Expected peers to be connected")
r := &Service{cfg: &Config{P2P: p1, DB: d, Chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
r := &Service{cfg: &config{p2p: p1, beaconDB: d, chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
r.rateLimiter.limiterMap[string(pcl)] = leakybucket.NewCollector(0.000001, 640, false)
req := &pb.BeaconBlocksByRangeRequest{
StartSlot: 1,
Step: 1,
Count: 64,
}
saveBlocks(d, r.cfg.Chain.(*chainMock.ChainService), req, true)
saveBlocks(d, r.cfg.chain.(*chainMock.ChainService), req, true)
hook.Reset()
err := sendRequest(p1, p2, r, req, func(blocks []*ethpb.SignedBeaconBlock) {
@@ -780,14 +780,14 @@ func TestRPCBeaconBlocksByRange_FilterBlocks(t *testing.T) {
p1.Connect(p2)
assert.Equal(t, 1, len(p1.BHost.Network().Peers()), "Expected peers to be connected")
r := &Service{cfg: &Config{P2P: p1, DB: d, Chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
r := &Service{cfg: &config{p2p: p1, beaconDB: d, chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
r.rateLimiter.limiterMap[string(pcl)] = leakybucket.NewCollector(0.000001, 640, false)
req := &pb.BeaconBlocksByRangeRequest{
StartSlot: 1,
Step: 1,
Count: 64,
}
saveBadBlocks(d, r.cfg.Chain.(*chainMock.ChainService), req, 2, true)
saveBadBlocks(d, r.cfg.chain.(*chainMock.ChainService), req, 2, true)
hook.Reset()
err := sendRequest(p1, p2, r, req, func(blocks []*ethpb.SignedBeaconBlock) {
@@ -815,14 +815,14 @@ func TestRPCBeaconBlocksByRange_FilterBlocks(t *testing.T) {
p1.Connect(p2)
assert.Equal(t, 1, len(p1.BHost.Network().Peers()), "Expected peers to be connected")
r := &Service{cfg: &Config{P2P: p1, DB: d, Chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
r := &Service{cfg: &config{p2p: p1, beaconDB: d, chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
r.rateLimiter.limiterMap[string(pcl)] = leakybucket.NewCollector(0.000001, 640, false)
req := &pb.BeaconBlocksByRangeRequest{
StartSlot: 1,
Step: 1,
Count: 128,
}
saveBadBlocks(d, r.cfg.Chain.(*chainMock.ChainService), req, 65, true)
saveBadBlocks(d, r.cfg.chain.(*chainMock.ChainService), req, 65, true)
hook.Reset()
err := sendRequest(p1, p2, r, req, func(blocks []*ethpb.SignedBeaconBlock) {
@@ -850,19 +850,19 @@ func TestRPCBeaconBlocksByRange_FilterBlocks(t *testing.T) {
p1.Connect(p2)
assert.Equal(t, 1, len(p1.BHost.Network().Peers()), "Expected peers to be connected")
r := &Service{cfg: &Config{P2P: p1, DB: d, Chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
r := &Service{cfg: &config{p2p: p1, beaconDB: d, chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
r.rateLimiter.limiterMap[string(pcl)] = leakybucket.NewCollector(0.000001, 640, false)
req := &pb.BeaconBlocksByRangeRequest{
StartSlot: 1,
Step: 1,
Count: 64,
}
saveBlocks(d, r.cfg.Chain.(*chainMock.ChainService), req, true)
saveBlocks(d, r.cfg.chain.(*chainMock.ChainService), req, true)
req.StartSlot = 65
req.Step = 1
req.Count = 128
// Save unfinalized chain.
saveBlocks(d, r.cfg.Chain.(*chainMock.ChainService), req, false)
saveBlocks(d, r.cfg.chain.(*chainMock.ChainService), req, false)
req.StartSlot = 1
hook.Reset()
@@ -890,21 +890,21 @@ func TestRPCBeaconBlocksByRange_FilterBlocks(t *testing.T) {
p1.Connect(p2)
assert.Equal(t, 1, len(p1.BHost.Network().Peers()), "Expected peers to be connected")
r := &Service{cfg: &Config{P2P: p1, DB: d, Chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
r := &Service{cfg: &config{p2p: p1, beaconDB: d, chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
r.rateLimiter.limiterMap[string(pcl)] = leakybucket.NewCollector(0.000001, 640, false)
req := &pb.BeaconBlocksByRangeRequest{
StartSlot: 1,
Step: 1,
Count: 64,
}
saveBlocks(d, r.cfg.Chain.(*chainMock.ChainService), req, true)
saveBlocks(d, r.cfg.chain.(*chainMock.ChainService), req, true)
// Create a duplicate set of unfinalized blocks.
req.StartSlot = 1
req.Step = 1
req.Count = 300
// Save unfinalized chain.
saveBlocks(d, r.cfg.Chain.(*chainMock.ChainService), req, false)
saveBlocks(d, r.cfg.chain.(*chainMock.ChainService), req, false)
req.Count = 64
hook.Reset()

View File

@@ -17,7 +17,7 @@ func (s *Service) sendRecentBeaconBlocksRequest(ctx context.Context, blockRoots
ctx, cancel := context.WithTimeout(ctx, respTimeout)
defer cancel()
_, err := SendBeaconBlocksByRootRequest(ctx, s.cfg.Chain, s.cfg.P2P, id, blockRoots, func(blk block.SignedBeaconBlock) error {
_, err := SendBeaconBlocksByRootRequest(ctx, s.cfg.chain, s.cfg.p2p, id, blockRoots, func(blk block.SignedBeaconBlock) error {
blkRoot, err := blk.Block().HashTreeRoot()
if err != nil {
return err
@@ -62,7 +62,7 @@ func (s *Service) beaconBlocksRootRPCHandler(ctx context.Context, msg interface{
s.rateLimiter.add(stream, int64(len(blockRoots)))
for _, root := range blockRoots {
blk, err := s.cfg.DB.Block(ctx, root)
blk, err := s.cfg.beaconDB.Block(ctx, root)
if err != nil {
log.WithError(err).Debug("Could not fetch block")
s.writeErrorResponseToStream(responseCodeServerError, types.ErrGeneric.Error(), stream)

View File

@@ -43,8 +43,8 @@ func TestRecentBeaconBlocksRPCHandler_ReturnsBlocks(t *testing.T) {
blkRoots = append(blkRoots, root)
}
r := &Service{cfg: &Config{P2P: p1, DB: d}, rateLimiter: newRateLimiter(p1)}
r.cfg.Chain = &mock.ChainService{ValidatorsRoot: [32]byte{}}
r := &Service{cfg: &config{p2p: p1, beaconDB: d}, rateLimiter: newRateLimiter(p1)}
r.cfg.chain = &mock.ChainService{ValidatorsRoot: [32]byte{}}
pcl := protocol.ID(p2p.RPCBlocksByRootTopicV1)
topic := string(pcl)
r.rateLimiter.limiterMap[topic] = leakybucket.NewCollector(10000, 10000, false)
@@ -56,7 +56,7 @@ func TestRecentBeaconBlocksRPCHandler_ReturnsBlocks(t *testing.T) {
for i := range blkRoots {
expectSuccess(t, stream)
res := util.NewBeaconBlock()
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, res))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, res))
if uint64(res.Block.Slot) != uint64(i+1) {
t.Errorf("Received unexpected block slot %d but wanted %d", res.Block.Slot, i+1)
}
@@ -99,9 +99,9 @@ func TestRecentBeaconBlocks_RPCRequestSent(t *testing.T) {
expectedRoots := p2pTypes.BeaconBlockByRootsReq{blockBRoot, blockARoot}
r := &Service{
cfg: &Config{
P2P: p1,
Chain: &mock.ChainService{
cfg: &config{
p2p: p1,
chain: &mock.ChainService{
State: genesisState,
FinalizedCheckPoint: finalizedCheckpt,
Root: blockARoot[:],
@@ -152,7 +152,7 @@ func TestRecentBeaconBlocksRPCHandler_HandleZeroBlocks(t *testing.T) {
assert.Equal(t, 1, len(p1.BHost.Network().Peers()), "Expected peers to be connected")
d := db.SetupDB(t)
r := &Service{cfg: &Config{P2P: p1, DB: d}, rateLimiter: newRateLimiter(p1)}
r := &Service{cfg: &config{p2p: p1, beaconDB: d}, rateLimiter: newRateLimiter(p1)}
pcl := protocol.ID(p2p.RPCBlocksByRootTopicV1)
topic := string(pcl)
r.rateLimiter.limiterMap[topic] = leakybucket.NewCollector(1, 1, false)

View File

@@ -20,7 +20,7 @@ import (
// response_chunk ::= <result> | <context-bytes> | <encoding-dependent-header> | <encoded-payload>
func (s *Service) chunkBlockWriter(stream libp2pcore.Stream, blk block.SignedBeaconBlock) error {
SetStreamWriteDeadline(stream, defaultWriteDuration)
return WriteBlockChunk(stream, s.cfg.Chain, s.cfg.P2P.Encoding(), blk)
return WriteBlockChunk(stream, s.cfg.chain, s.cfg.p2p.Encoding(), blk)
}
// WriteBlockChunk writes block chunk object to stream.

View File

@@ -47,18 +47,18 @@ func (s *Service) goodbyeRPCHandler(_ context.Context, msg interface{}, stream l
s.rateLimiter.add(stream, 1)
log := log.WithField("Reason", goodbyeMessage(*m))
log.WithField("peer", stream.Conn().RemotePeer()).Debug("Peer has sent a goodbye message")
s.cfg.P2P.Peers().SetNextValidTime(stream.Conn().RemotePeer(), goodByeBackoff(*m))
s.cfg.p2p.Peers().SetNextValidTime(stream.Conn().RemotePeer(), goodByeBackoff(*m))
// closes all streams with the peer
return s.cfg.P2P.Disconnect(stream.Conn().RemotePeer())
return s.cfg.p2p.Disconnect(stream.Conn().RemotePeer())
}
// disconnectBadPeer checks whether peer is considered bad by some scorer, and tries to disconnect
// the peer, if that is the case. Additionally, disconnection reason is obtained from scorer.
func (s *Service) disconnectBadPeer(ctx context.Context, id peer.ID) {
if !s.cfg.P2P.Peers().IsBad(id) {
if !s.cfg.p2p.Peers().IsBad(id) {
return
}
err := s.cfg.P2P.Peers().Scorers().ValidationError(id)
err := s.cfg.p2p.Peers().Scorers().ValidationError(id)
goodbyeCode := p2ptypes.ErrToGoodbyeCode(err)
if err == nil {
goodbyeCode = p2ptypes.GoodbyeCodeBanned
@@ -80,7 +80,7 @@ func (s *Service) sendGoodByeAndDisconnect(ctx context.Context, code p2ptypes.RP
defer lock.Unlock()
// In the event we are already disconnected, exit early from the
// goodbye method to prevent redundant streams from being created.
if s.cfg.P2P.Host().Network().Connectedness(id) == network.NotConnected {
if s.cfg.p2p.Host().Network().Connectedness(id) == network.NotConnected {
return nil
}
if err := s.sendGoodByeMessage(ctx, code, id); err != nil {
@@ -89,18 +89,18 @@ func (s *Service) sendGoodByeAndDisconnect(ctx context.Context, code p2ptypes.RP
"peer": id,
}).Debug("Could not send goodbye message to peer")
}
return s.cfg.P2P.Disconnect(id)
return s.cfg.p2p.Disconnect(id)
}
func (s *Service) sendGoodByeMessage(ctx context.Context, code p2ptypes.RPCGoodbyeCode, id peer.ID) error {
ctx, cancel := context.WithTimeout(ctx, respTimeout)
defer cancel()
topic, err := p2p.TopicFromMessage(p2p.GoodbyeMessageName, slots.ToEpoch(s.cfg.Chain.CurrentSlot()))
topic, err := p2p.TopicFromMessage(p2p.GoodbyeMessageName, slots.ToEpoch(s.cfg.chain.CurrentSlot()))
if err != nil {
return err
}
stream, err := s.cfg.P2P.Send(ctx, &code, topic, id)
stream, err := s.cfg.p2p.Send(ctx, &code, topic, id)
if err != nil {
return err
}

View File

@@ -28,9 +28,9 @@ func TestGoodByeRPCHandler_Disconnects_With_Peer(t *testing.T) {
// Set up a head state in the database with data we expect.
d := db.SetupDB(t)
r := &Service{
cfg: &Config{
DB: d,
P2P: p1,
cfg: &config{
beaconDB: d,
p2p: p1,
},
rateLimiter: newRateLimiter(p1),
}
@@ -73,9 +73,9 @@ func TestGoodByeRPCHandler_BackOffPeer(t *testing.T) {
// Set up a head state in the database with data we expect.
d := db.SetupDB(t)
r := &Service{
cfg: &Config{
DB: d,
P2P: p1,
cfg: &config{
beaconDB: d,
p2p: p1,
},
rateLimiter: newRateLimiter(p1),
}
@@ -148,10 +148,10 @@ func TestSendGoodbye_SendsMessage(t *testing.T) {
// Set up a head state in the database with data we expect.
d := db.SetupDB(t)
r := &Service{
cfg: &Config{
DB: d,
P2P: p1,
Chain: &mock.ChainService{ValidatorsRoot: [32]byte{}, Genesis: time.Now()},
cfg: &config{
beaconDB: d,
p2p: p1,
chain: &mock.ChainService{ValidatorsRoot: [32]byte{}, Genesis: time.Now()},
},
rateLimiter: newRateLimiter(p1),
}
@@ -166,7 +166,7 @@ func TestSendGoodbye_SendsMessage(t *testing.T) {
p2.BHost.SetStreamHandler(pcl, func(stream network.Stream) {
defer wg.Done()
out := new(types.SSZUint64)
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, out))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, out))
assert.Equal(t, failureCode, *out)
assert.NoError(t, stream.Close())
})
@@ -193,10 +193,10 @@ func TestSendGoodbye_DisconnectWithPeer(t *testing.T) {
// Set up a head state in the database with data we expect.
d := db.SetupDB(t)
r := &Service{
cfg: &Config{
DB: d,
P2P: p1,
Chain: &mock.ChainService{Genesis: time.Now(), ValidatorsRoot: [32]byte{}},
cfg: &config{
beaconDB: d,
p2p: p1,
chain: &mock.ChainService{Genesis: time.Now(), ValidatorsRoot: [32]byte{}},
},
rateLimiter: newRateLimiter(p1),
}
@@ -211,7 +211,7 @@ func TestSendGoodbye_DisconnectWithPeer(t *testing.T) {
p2.BHost.SetStreamHandler(pcl, func(stream network.Stream) {
defer wg.Done()
out := new(types.SSZUint64)
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, out))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, out))
assert.Equal(t, failureCode, *out)
assert.NoError(t, stream.Close())
})

View File

@@ -30,7 +30,7 @@ func (s *Service) metaDataHandler(_ context.Context, _ interface{}, stream libp2
}
s.rateLimiter.add(stream, 1)
if s.cfg.P2P.Metadata() == nil || s.cfg.P2P.Metadata().IsNil() {
if s.cfg.p2p.Metadata() == nil || s.cfg.p2p.Metadata().IsNil() {
nilErr := errors.New("nil metadata stored for host")
resp, err := s.generateErrorResponse(responseCodeServerError, types.ErrGeneric.Error())
if err != nil {
@@ -50,7 +50,7 @@ func (s *Service) metaDataHandler(_ context.Context, _ interface{}, stream libp2
}
return err
}
currMd := s.cfg.P2P.Metadata()
currMd := s.cfg.p2p.Metadata()
switch streamVersion {
case p2p.SchemaVersionV1:
// We have a v1 metadata object saved locally, so we
@@ -77,7 +77,7 @@ func (s *Service) metaDataHandler(_ context.Context, _ interface{}, stream libp2
if _, err := stream.Write([]byte{responseCodeSuccess}); err != nil {
return err
}
_, err = s.cfg.P2P.Encoding().EncodeWithMaxLength(stream, currMd)
_, err = s.cfg.p2p.Encoding().EncodeWithMaxLength(stream, currMd)
if err != nil {
return err
}
@@ -89,29 +89,29 @@ func (s *Service) sendMetaDataRequest(ctx context.Context, id peer.ID) (metadata
ctx, cancel := context.WithTimeout(ctx, respTimeout)
defer cancel()
topic, err := p2p.TopicFromMessage(p2p.MetadataMessageName, slots.ToEpoch(s.cfg.Chain.CurrentSlot()))
topic, err := p2p.TopicFromMessage(p2p.MetadataMessageName, slots.ToEpoch(s.cfg.chain.CurrentSlot()))
if err != nil {
return nil, err
}
stream, err := s.cfg.P2P.Send(ctx, new(interface{}), topic, id)
stream, err := s.cfg.p2p.Send(ctx, new(interface{}), topic, id)
if err != nil {
return nil, err
}
defer closeStream(stream, log)
code, errMsg, err := ReadStatusCode(stream, s.cfg.P2P.Encoding())
code, errMsg, err := ReadStatusCode(stream, s.cfg.p2p.Encoding())
if err != nil {
return nil, err
}
if code != 0 {
s.cfg.P2P.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer())
s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer())
return nil, errors.New(errMsg)
}
valRoot := s.cfg.Chain.GenesisValidatorRoot()
rpcCtx, err := forks.ForkDigestFromEpoch(slots.ToEpoch(s.cfg.Chain.CurrentSlot()), valRoot[:])
valRoot := s.cfg.chain.GenesisValidatorRoot()
rpcCtx, err := forks.ForkDigestFromEpoch(slots.ToEpoch(s.cfg.chain.CurrentSlot()), valRoot[:])
if err != nil {
return nil, err
}
msg, err := extractMetaDataType(rpcCtx[:], s.cfg.Chain)
msg, err := extractMetaDataType(rpcCtx[:], s.cfg.chain)
if err != nil {
return nil, err
}
@@ -126,7 +126,7 @@ func (s *Service) sendMetaDataRequest(ctx context.Context, id peer.ID) (metadata
if err := validateVersion(topicVersion, stream); err != nil {
return nil, err
}
if err := s.cfg.P2P.Encoding().DecodeWithMaxLength(stream, msg); err != nil {
if err := s.cfg.p2p.Encoding().DecodeWithMaxLength(stream, msg); err != nil {
return nil, err
}
return msg, nil

View File

@@ -40,10 +40,10 @@ func TestMetaDataRPCHandler_ReceivesMetadata(t *testing.T) {
// Set up a head state in the database with data we expect.
d := db.SetupDB(t)
r := &Service{
cfg: &Config{
DB: d,
P2P: p1,
Chain: &mock.ChainService{
cfg: &config{
beaconDB: d,
p2p: p1,
chain: &mock.ChainService{
ValidatorsRoot: [32]byte{},
},
},
@@ -60,7 +60,7 @@ func TestMetaDataRPCHandler_ReceivesMetadata(t *testing.T) {
defer wg.Done()
expectSuccess(t, stream)
out := new(pb.MetaDataV0)
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, out))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, out))
assert.DeepEqual(t, p1.LocalMetadata.InnerObject(), out, "MetadataV0 unequal")
})
stream1, err := p1.BHost.NewStream(context.Background(), p2.BHost.ID(), pcl)
@@ -92,25 +92,25 @@ func TestMetadataRPCHandler_SendsMetadata(t *testing.T) {
// Set up a head state in the database with data we expect.
d := db.SetupDB(t)
r := &Service{
cfg: &Config{
DB: d,
P2P: p1,
Chain: &mock.ChainService{Genesis: time.Now(), ValidatorsRoot: [32]byte{}},
cfg: &config{
beaconDB: d,
p2p: p1,
chain: &mock.ChainService{Genesis: time.Now(), ValidatorsRoot: [32]byte{}},
},
rateLimiter: newRateLimiter(p1),
}
r2 := &Service{
cfg: &Config{
DB: d,
P2P: p2,
Chain: &mock.ChainService{Genesis: time.Now(), ValidatorsRoot: [32]byte{}},
cfg: &config{
beaconDB: d,
p2p: p2,
chain: &mock.ChainService{Genesis: time.Now(), ValidatorsRoot: [32]byte{}},
},
rateLimiter: newRateLimiter(p2),
}
// Setup streams
pcl := protocol.ID(p2p.RPCMetaDataTopicV1 + r.cfg.P2P.Encoding().ProtocolSuffix())
pcl := protocol.ID(p2p.RPCMetaDataTopicV1 + r.cfg.p2p.Encoding().ProtocolSuffix())
topic := string(pcl)
r.rateLimiter.limiterMap[topic] = leakybucket.NewCollector(1, 1, false)
r2.rateLimiter.limiterMap[topic] = leakybucket.NewCollector(1, 1, false)
@@ -159,25 +159,25 @@ func TestMetadataRPCHandler_SendsMetadataAltair(t *testing.T) {
// Set up a head state in the database with data we expect.
d := db.SetupDB(t)
r := &Service{
cfg: &Config{
DB: d,
P2P: p1,
Chain: &mock.ChainService{Genesis: time.Now().Add(-5 * oneEpoch()), ValidatorsRoot: [32]byte{}},
cfg: &config{
beaconDB: d,
p2p: p1,
chain: &mock.ChainService{Genesis: time.Now().Add(-5 * oneEpoch()), ValidatorsRoot: [32]byte{}},
},
rateLimiter: newRateLimiter(p1),
}
r2 := &Service{
cfg: &Config{
DB: d,
P2P: p2,
Chain: &mock.ChainService{Genesis: time.Now().Add(-5 * oneEpoch()), ValidatorsRoot: [32]byte{}},
cfg: &config{
beaconDB: d,
p2p: p2,
chain: &mock.ChainService{Genesis: time.Now().Add(-5 * oneEpoch()), ValidatorsRoot: [32]byte{}},
},
rateLimiter: newRateLimiter(p2),
}
// Setup streams
pcl := protocol.ID(p2p.RPCMetaDataTopicV2 + r.cfg.P2P.Encoding().ProtocolSuffix())
pcl := protocol.ID(p2p.RPCMetaDataTopicV2 + r.cfg.p2p.Encoding().ProtocolSuffix())
topic := string(pcl)
r.rateLimiter.limiterMap[topic] = leakybucket.NewCollector(2, 2, false)
r2.rateLimiter.limiterMap[topic] = leakybucket.NewCollector(2, 2, false)

View File

@@ -31,7 +31,7 @@ func (s *Service) pingHandler(_ context.Context, msg interface{}, stream libp2pc
if err != nil {
// Descore peer for giving us a bad sequence number.
if errors.Is(err, p2ptypes.ErrInvalidSequenceNum) {
s.cfg.P2P.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer())
s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer())
s.writeErrorResponseToStream(responseCodeInvalidRequest, p2ptypes.ErrInvalidSequenceNum.Error(), stream)
}
return err
@@ -39,8 +39,8 @@ func (s *Service) pingHandler(_ context.Context, msg interface{}, stream libp2pc
if _, err := stream.Write([]byte{responseCodeSuccess}); err != nil {
return err
}
sq := types.SSZUint64(s.cfg.P2P.MetadataSeq())
if _, err := s.cfg.P2P.Encoding().EncodeWithMaxLength(stream, &sq); err != nil {
sq := types.SSZUint64(s.cfg.p2p.MetadataSeq())
if _, err := s.cfg.p2p.Encoding().EncodeWithMaxLength(stream, &sq); err != nil {
return err
}
@@ -67,7 +67,7 @@ func (s *Service) pingHandler(_ context.Context, msg interface{}, stream libp2pc
return
}
// update metadata if there is no error
s.cfg.P2P.Peers().SetMetadata(stream.Conn().RemotePeer(), md)
s.cfg.p2p.Peers().SetMetadata(stream.Conn().RemotePeer(), md)
}()
return nil
@@ -77,38 +77,38 @@ func (s *Service) sendPingRequest(ctx context.Context, id peer.ID) error {
ctx, cancel := context.WithTimeout(ctx, respTimeout)
defer cancel()
metadataSeq := types.SSZUint64(s.cfg.P2P.MetadataSeq())
topic, err := p2p.TopicFromMessage(p2p.PingMessageName, slots.ToEpoch(s.cfg.Chain.CurrentSlot()))
metadataSeq := types.SSZUint64(s.cfg.p2p.MetadataSeq())
topic, err := p2p.TopicFromMessage(p2p.PingMessageName, slots.ToEpoch(s.cfg.chain.CurrentSlot()))
if err != nil {
return err
}
stream, err := s.cfg.P2P.Send(ctx, &metadataSeq, topic, id)
stream, err := s.cfg.p2p.Send(ctx, &metadataSeq, topic, id)
if err != nil {
return err
}
currentTime := time.Now()
defer closeStream(stream, log)
code, errMsg, err := ReadStatusCode(stream, s.cfg.P2P.Encoding())
code, errMsg, err := ReadStatusCode(stream, s.cfg.p2p.Encoding())
if err != nil {
return err
}
// Records the latency of the ping request for that peer.
s.cfg.P2P.Host().Peerstore().RecordLatency(id, time.Now().Sub(currentTime))
s.cfg.p2p.Host().Peerstore().RecordLatency(id, time.Now().Sub(currentTime))
if code != 0 {
s.cfg.P2P.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer())
s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer())
return errors.New(errMsg)
}
msg := new(types.SSZUint64)
if err := s.cfg.P2P.Encoding().DecodeWithMaxLength(stream, msg); err != nil {
if err := s.cfg.p2p.Encoding().DecodeWithMaxLength(stream, msg); err != nil {
return err
}
valid, err := s.validateSequenceNum(*msg, stream.Conn().RemotePeer())
if err != nil {
// Descore peer for giving us a bad sequence number.
if errors.Is(err, p2ptypes.ErrInvalidSequenceNum) {
s.cfg.P2P.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer())
s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer())
}
return err
}
@@ -121,13 +121,13 @@ func (s *Service) sendPingRequest(ctx context.Context, id peer.ID) error {
// already done in the request method.
return err
}
s.cfg.P2P.Peers().SetMetadata(stream.Conn().RemotePeer(), md)
s.cfg.p2p.Peers().SetMetadata(stream.Conn().RemotePeer(), md)
return nil
}
// validates the peer's sequence number.
func (s *Service) validateSequenceNum(seq types.SSZUint64, id peer.ID) (bool, error) {
md, err := s.cfg.P2P.Peers().Metadata(id)
md, err := s.cfg.p2p.Peers().Metadata(id)
if err != nil {
return false, err
}

View File

@@ -41,9 +41,9 @@ func TestPingRPCHandler_ReceivesPing(t *testing.T) {
// Set up a head state in the database with data we expect.
d := db.SetupDB(t)
r := &Service{
cfg: &Config{
DB: d,
P2P: p1,
cfg: &config{
beaconDB: d,
p2p: p1,
},
rateLimiter: newRateLimiter(p1),
}
@@ -61,7 +61,7 @@ func TestPingRPCHandler_ReceivesPing(t *testing.T) {
defer wg.Done()
expectSuccess(t, stream)
out := new(types.SSZUint64)
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, out))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, out))
assert.Equal(t, uint64(2), uint64(*out))
})
stream1, err := p1.BHost.NewStream(context.Background(), p2.BHost.ID(), pcl)
@@ -98,10 +98,10 @@ func TestPingRPCHandler_SendsPing(t *testing.T) {
// Set up a head state in the database with data we expect.
d := db.SetupDB(t)
r := &Service{
cfg: &Config{
DB: d,
P2P: p1,
Chain: &mock.ChainService{ValidatorsRoot: [32]byte{}, Genesis: time.Now()},
cfg: &config{
beaconDB: d,
p2p: p1,
chain: &mock.ChainService{ValidatorsRoot: [32]byte{}, Genesis: time.Now()},
},
rateLimiter: newRateLimiter(p1),
}
@@ -113,10 +113,10 @@ func TestPingRPCHandler_SendsPing(t *testing.T) {
p2.Peers().SetMetadata(p1.BHost.ID(), p1.LocalMetadata)
r2 := &Service{
cfg: &Config{
DB: d,
P2P: p2,
Chain: &mock.ChainService{ValidatorsRoot: [32]byte{}, Genesis: time.Now()},
cfg: &config{
beaconDB: d,
p2p: p2,
chain: &mock.ChainService{ValidatorsRoot: [32]byte{}, Genesis: time.Now()},
},
rateLimiter: newRateLimiter(p2),
}
@@ -130,7 +130,7 @@ func TestPingRPCHandler_SendsPing(t *testing.T) {
p2.BHost.SetStreamHandler(pcl, func(stream network.Stream) {
defer wg.Done()
out := new(types.SSZUint64)
assert.NoError(t, r2.cfg.P2P.Encoding().DecodeWithMaxLength(stream, out))
assert.NoError(t, r2.cfg.p2p.Encoding().DecodeWithMaxLength(stream, out))
assert.Equal(t, uint64(2), uint64(*out))
assert.NoError(t, r2.pingHandler(context.Background(), out, stream))
})
@@ -165,9 +165,9 @@ func TestPingRPCHandler_BadSequenceNumber(t *testing.T) {
// Set up a head state in the database with data we expect.
d := db.SetupDB(t)
r := &Service{
cfg: &Config{
DB: d,
P2P: p1,
cfg: &config{
beaconDB: d,
p2p: p1,
},
rateLimiter: newRateLimiter(p1),
}

View File

@@ -30,27 +30,27 @@ func (s *Service) maintainPeerStatuses() {
interval := time.Duration(params.BeaconConfig().SlotsPerEpoch.Div(2).Mul(params.BeaconConfig().SecondsPerSlot)) * time.Second
async.RunEvery(s.ctx, interval, func() {
wg := new(sync.WaitGroup)
for _, pid := range s.cfg.P2P.Peers().Connected() {
for _, pid := range s.cfg.p2p.Peers().Connected() {
wg.Add(1)
go func(id peer.ID) {
defer wg.Done()
// If our peer status has not been updated correctly we disconnect over here
// and set the connection state over here instead.
if s.cfg.P2P.Host().Network().Connectedness(id) != network.Connected {
s.cfg.P2P.Peers().SetConnectionState(id, peers.PeerDisconnecting)
if err := s.cfg.P2P.Disconnect(id); err != nil {
if s.cfg.p2p.Host().Network().Connectedness(id) != network.Connected {
s.cfg.p2p.Peers().SetConnectionState(id, peers.PeerDisconnecting)
if err := s.cfg.p2p.Disconnect(id); err != nil {
log.Debugf("Error when disconnecting with peer: %v", err)
}
s.cfg.P2P.Peers().SetConnectionState(id, peers.PeerDisconnected)
s.cfg.p2p.Peers().SetConnectionState(id, peers.PeerDisconnected)
return
}
// Disconnect from peers that are considered bad by any of the registered scorers.
if s.cfg.P2P.Peers().IsBad(id) {
if s.cfg.p2p.Peers().IsBad(id) {
s.disconnectBadPeer(s.ctx, id)
return
}
// If the status hasn't been updated in the recent interval time.
lastUpdated, err := s.cfg.P2P.Peers().ChainStateLastUpdated(id)
lastUpdated, err := s.cfg.p2p.Peers().ChainStateLastUpdated(id)
if err != nil {
// Peer has vanished; nothing to do.
return
@@ -58,7 +58,7 @@ func (s *Service) maintainPeerStatuses() {
if prysmTime.Now().After(lastUpdated.Add(interval)) {
if err := s.reValidatePeer(s.ctx, id); err != nil {
log.WithField("peer", id).WithError(err).Debug("Could not revalidate peer")
s.cfg.P2P.Peers().Scorers().BadResponsesScorer().Increment(id)
s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(id)
}
}
}(pid)
@@ -66,7 +66,7 @@ func (s *Service) maintainPeerStatuses() {
// Wait for all status checks to finish and then proceed onwards to
// pruning excess peers.
wg.Wait()
peerIds := s.cfg.P2P.Peers().PeersToPrune()
peerIds := s.cfg.p2p.Peers().PeersToPrune()
peerIds = s.filterNeededPeers(peerIds)
for _, id := range peerIds {
if err := s.sendGoodByeAndDisconnect(s.ctx, p2ptypes.GoodbyeCodeTooManyPeers, id); err != nil {
@@ -84,21 +84,21 @@ func (s *Service) resyncIfBehind() {
interval := time.Duration(millisecondsPerEpoch/16) * time.Millisecond
async.RunEvery(s.ctx, interval, func() {
if s.shouldReSync() {
syncedEpoch := slots.ToEpoch(s.cfg.Chain.HeadSlot())
syncedEpoch := slots.ToEpoch(s.cfg.chain.HeadSlot())
// Factor number of expected minimum sync peers, to make sure that enough peers are
// available to resync (some peers may go away between checking non-finalized peers and
// actual resyncing).
highestEpoch, _ := s.cfg.P2P.Peers().BestNonFinalized(flags.Get().MinimumSyncPeers*2, syncedEpoch)
highestEpoch, _ := s.cfg.p2p.Peers().BestNonFinalized(flags.Get().MinimumSyncPeers*2, syncedEpoch)
// Check if the current node is more than 1 epoch behind.
if highestEpoch > (syncedEpoch + 1) {
log.WithFields(logrus.Fields{
"currentEpoch": slots.ToEpoch(s.cfg.Chain.CurrentSlot()),
"currentEpoch": slots.ToEpoch(s.cfg.chain.CurrentSlot()),
"syncedEpoch": syncedEpoch,
"peersEpoch": highestEpoch,
}).Info("Fallen behind peers; reverting to initial sync to catch up")
numberOfTimesResyncedCounter.Inc()
s.clearPendingSlots()
if err := s.cfg.InitialSync.Resync(); err != nil {
if err := s.cfg.initialSync.Resync(); err != nil {
log.Errorf("Could not resync chain: %v", err)
}
}
@@ -108,13 +108,13 @@ func (s *Service) resyncIfBehind() {
// shouldReSync returns true if the node is not syncing and falls behind two epochs.
func (s *Service) shouldReSync() bool {
syncedEpoch := slots.ToEpoch(s.cfg.Chain.HeadSlot())
currentEpoch := slots.ToEpoch(s.cfg.Chain.CurrentSlot())
syncedEpoch := slots.ToEpoch(s.cfg.chain.HeadSlot())
currentEpoch := slots.ToEpoch(s.cfg.chain.CurrentSlot())
prevEpoch := types.Epoch(0)
if currentEpoch > 1 {
prevEpoch = currentEpoch - 1
}
return s.cfg.InitialSync != nil && !s.cfg.InitialSync.Syncing() && syncedEpoch < prevEpoch
return s.cfg.initialSync != nil && !s.cfg.initialSync.Syncing() && syncedEpoch < prevEpoch
}
// sendRPCStatusRequest for a given topic with an expected protobuf message type.
@@ -122,7 +122,7 @@ func (s *Service) sendRPCStatusRequest(ctx context.Context, id peer.ID) error {
ctx, cancel := context.WithTimeout(ctx, respTimeout)
defer cancel()
headRoot, err := s.cfg.Chain.HeadRoot(ctx)
headRoot, err := s.cfg.chain.HeadRoot(ctx)
if err != nil {
return err
}
@@ -133,46 +133,46 @@ func (s *Service) sendRPCStatusRequest(ctx context.Context, id peer.ID) error {
}
resp := &pb.Status{
ForkDigest: forkDigest[:],
FinalizedRoot: s.cfg.Chain.FinalizedCheckpt().Root,
FinalizedEpoch: s.cfg.Chain.FinalizedCheckpt().Epoch,
FinalizedRoot: s.cfg.chain.FinalizedCheckpt().Root,
FinalizedEpoch: s.cfg.chain.FinalizedCheckpt().Epoch,
HeadRoot: headRoot,
HeadSlot: s.cfg.Chain.HeadSlot(),
HeadSlot: s.cfg.chain.HeadSlot(),
}
topic, err := p2p.TopicFromMessage(p2p.StatusMessageName, slots.ToEpoch(s.cfg.Chain.CurrentSlot()))
topic, err := p2p.TopicFromMessage(p2p.StatusMessageName, slots.ToEpoch(s.cfg.chain.CurrentSlot()))
if err != nil {
return err
}
stream, err := s.cfg.P2P.Send(ctx, resp, topic, id)
stream, err := s.cfg.p2p.Send(ctx, resp, topic, id)
if err != nil {
return err
}
defer closeStream(stream, log)
code, errMsg, err := ReadStatusCode(stream, s.cfg.P2P.Encoding())
code, errMsg, err := ReadStatusCode(stream, s.cfg.p2p.Encoding())
if err != nil {
return err
}
if code != 0 {
s.cfg.P2P.Peers().Scorers().BadResponsesScorer().Increment(id)
s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(id)
return errors.New(errMsg)
}
msg := &pb.Status{}
if err := s.cfg.P2P.Encoding().DecodeWithMaxLength(stream, msg); err != nil {
if err := s.cfg.p2p.Encoding().DecodeWithMaxLength(stream, msg); err != nil {
return err
}
// If validation fails, validation error is logged, and peer status scorer will mark peer as bad.
err = s.validateStatusMessage(ctx, msg)
s.cfg.P2P.Peers().Scorers().PeerStatusScorer().SetPeerStatus(id, msg, err)
if s.cfg.P2P.Peers().IsBad(id) {
s.cfg.p2p.Peers().Scorers().PeerStatusScorer().SetPeerStatus(id, msg, err)
if s.cfg.p2p.Peers().IsBad(id) {
s.disconnectBadPeer(s.ctx, id)
}
return err
}
func (s *Service) reValidatePeer(ctx context.Context, id peer.ID) error {
s.cfg.P2P.Peers().Scorers().PeerStatusScorer().SetHeadSlot(s.cfg.Chain.HeadSlot())
s.cfg.p2p.Peers().Scorers().PeerStatusScorer().SetHeadSlot(s.cfg.chain.HeadSlot())
if err := s.sendRPCStatusRequest(ctx, id); err != nil {
return err
}
@@ -212,7 +212,7 @@ func (s *Service) statusRPCHandler(ctx context.Context, msg interface{}, stream
respCode = responseCodeServerError
case p2ptypes.ErrWrongForkDigestVersion:
// Respond with our status and disconnect with the peer.
s.cfg.P2P.Peers().SetChainState(remotePeer, m)
s.cfg.p2p.Peers().SetChainState(remotePeer, m)
if err := s.respondWithStatus(ctx, stream); err != nil {
return err
}
@@ -224,7 +224,7 @@ func (s *Service) statusRPCHandler(ctx context.Context, msg interface{}, stream
return nil
default:
respCode = responseCodeInvalidRequest
s.cfg.P2P.Peers().Scorers().BadResponsesScorer().Increment(remotePeer)
s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(remotePeer)
}
originalErr := err
@@ -241,7 +241,7 @@ func (s *Service) statusRPCHandler(ctx context.Context, msg interface{}, stream
}
return originalErr
}
s.cfg.P2P.Peers().SetChainState(remotePeer, m)
s.cfg.p2p.Peers().SetChainState(remotePeer, m)
if err := s.respondWithStatus(ctx, stream); err != nil {
return err
@@ -251,7 +251,7 @@ func (s *Service) statusRPCHandler(ctx context.Context, msg interface{}, stream
}
func (s *Service) respondWithStatus(ctx context.Context, stream network.Stream) error {
headRoot, err := s.cfg.Chain.HeadRoot(ctx)
headRoot, err := s.cfg.chain.HeadRoot(ctx)
if err != nil {
return err
}
@@ -262,16 +262,16 @@ func (s *Service) respondWithStatus(ctx context.Context, stream network.Stream)
}
resp := &pb.Status{
ForkDigest: forkDigest[:],
FinalizedRoot: s.cfg.Chain.FinalizedCheckpt().Root,
FinalizedEpoch: s.cfg.Chain.FinalizedCheckpt().Epoch,
FinalizedRoot: s.cfg.chain.FinalizedCheckpt().Root,
FinalizedEpoch: s.cfg.chain.FinalizedCheckpt().Epoch,
HeadRoot: headRoot,
HeadSlot: s.cfg.Chain.HeadSlot(),
HeadSlot: s.cfg.chain.HeadSlot(),
}
if _, err := stream.Write([]byte{responseCodeSuccess}); err != nil {
log.WithError(err).Debug("Could not write to stream")
}
_, err = s.cfg.P2P.Encoding().EncodeWithMaxLength(stream, resp)
_, err = s.cfg.p2p.Encoding().EncodeWithMaxLength(stream, resp)
return err
}
@@ -283,8 +283,8 @@ func (s *Service) validateStatusMessage(ctx context.Context, msg *pb.Status) err
if !bytes.Equal(forkDigest[:], msg.ForkDigest) {
return p2ptypes.ErrWrongForkDigestVersion
}
genesis := s.cfg.Chain.GenesisTime()
finalizedEpoch := s.cfg.Chain.FinalizedCheckpt().Epoch
genesis := s.cfg.chain.GenesisTime()
finalizedEpoch := s.cfg.chain.FinalizedCheckpt().Epoch
maxEpoch := slots.EpochsSinceGenesis(genesis)
// It would take a minimum of 2 epochs to finalize a
// previous epoch
@@ -306,10 +306,10 @@ func (s *Service) validateStatusMessage(ctx context.Context, msg *pb.Status) err
if finalizedAtGenesis && rootIsEqual {
return nil
}
if !s.cfg.DB.IsFinalizedBlock(ctx, bytesutil.ToBytes32(msg.FinalizedRoot)) {
if !s.cfg.beaconDB.IsFinalizedBlock(ctx, bytesutil.ToBytes32(msg.FinalizedRoot)) {
return p2ptypes.ErrInvalidFinalizedRoot
}
blk, err := s.cfg.DB.Block(ctx, bytesutil.ToBytes32(msg.FinalizedRoot))
blk, err := s.cfg.beaconDB.Block(ctx, bytesutil.ToBytes32(msg.FinalizedRoot))
if err != nil {
return p2ptypes.ErrGeneric
}
@@ -325,7 +325,7 @@ func (s *Service) validateStatusMessage(ctx context.Context, msg *pb.Status) err
return p2ptypes.ErrGeneric
}
if startSlot > blk.Block().Slot() {
childBlock, err := s.cfg.DB.FinalizedChildBlock(ctx, bytesutil.ToBytes32(msg.FinalizedRoot))
childBlock, err := s.cfg.beaconDB.FinalizedChildBlock(ctx, bytesutil.ToBytes32(msg.FinalizedRoot))
if err != nil {
return p2ptypes.ErrGeneric
}

View File

@@ -43,9 +43,9 @@ func TestStatusRPCHandler_Disconnects_OnForkVersionMismatch(t *testing.T) {
root := [32]byte{'C'}
r := &Service{
cfg: &Config{
P2P: p1,
Chain: &mock.ChainService{
cfg: &config{
p2p: p1,
chain: &mock.ChainService{
Fork: &ethpb.Fork{
PreviousVersion: params.BeaconConfig().GenesisForkVersion,
CurrentVersion: params.BeaconConfig().GenesisForkVersion,
@@ -71,7 +71,7 @@ func TestStatusRPCHandler_Disconnects_OnForkVersionMismatch(t *testing.T) {
defer wg.Done()
expectSuccess(t, stream)
out := &pb.Status{}
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, out))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, out))
assert.DeepEqual(t, root[:], out.FinalizedRoot)
assert.NoError(t, stream.Close())
})
@@ -84,7 +84,7 @@ func TestStatusRPCHandler_Disconnects_OnForkVersionMismatch(t *testing.T) {
p2.BHost.SetStreamHandler(pcl2, func(stream network.Stream) {
defer wg2.Done()
msg := new(types.SSZUint64)
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, msg))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, msg))
assert.Equal(t, p2ptypes.GoodbyeCodeWrongNetwork, *msg)
assert.NoError(t, stream.Close())
})
@@ -111,9 +111,9 @@ func TestStatusRPCHandler_ConnectsOnGenesis(t *testing.T) {
root := [32]byte{}
r := &Service{
cfg: &Config{
P2P: p1,
Chain: &mock.ChainService{
cfg: &config{
p2p: p1,
chain: &mock.ChainService{
Fork: &ethpb.Fork{
PreviousVersion: params.BeaconConfig().GenesisForkVersion,
CurrentVersion: params.BeaconConfig().GenesisForkVersion,
@@ -139,7 +139,7 @@ func TestStatusRPCHandler_ConnectsOnGenesis(t *testing.T) {
defer wg.Done()
expectSuccess(t, stream)
out := &pb.Status{}
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, out))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, out))
assert.DeepEqual(t, root[:], out.FinalizedRoot)
})
@@ -189,9 +189,9 @@ func TestStatusRPCHandler_ReturnsHelloMessage(t *testing.T) {
genTime := time.Now().Unix() - totalSec
r := &Service{
cfg: &Config{
P2P: p1,
Chain: &mock.ChainService{
cfg: &config{
p2p: p1,
chain: &mock.ChainService{
State: genesisState,
FinalizedCheckPoint: finalizedCheckpt,
Root: headRoot[:],
@@ -202,7 +202,7 @@ func TestStatusRPCHandler_ReturnsHelloMessage(t *testing.T) {
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Unix(genTime, 0),
},
DB: db,
beaconDB: db,
},
rateLimiter: newRateLimiter(p1),
}
@@ -219,7 +219,7 @@ func TestStatusRPCHandler_ReturnsHelloMessage(t *testing.T) {
defer wg.Done()
expectSuccess(t, stream)
out := &pb.Status{}
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, out))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, out))
expected := &pb.Status{
ForkDigest: digest[:],
HeadSlot: genesisState.Slot(),
@@ -274,9 +274,9 @@ func TestHandshakeHandlers_Roundtrip(t *testing.T) {
require.NoError(t, err)
require.NoError(t, db.SaveGenesisBlockRoot(context.Background(), finalizedRoot))
r := &Service{
cfg: &Config{
P2P: p1,
Chain: &mock.ChainService{
cfg: &config{
p2p: p1,
chain: &mock.ChainService{
State: st,
FinalizedCheckPoint: &ethpb.Checkpoint{Epoch: 0, Root: finalizedRoot[:]},
Fork: &ethpb.Fork{
@@ -287,7 +287,7 @@ func TestHandshakeHandlers_Roundtrip(t *testing.T) {
ValidatorsRoot: [32]byte{'A'},
Root: make([]byte, 32),
},
DB: db,
beaconDB: db,
},
ctx: context.Background(),
rateLimiter: newRateLimiter(p1),
@@ -296,11 +296,11 @@ func TestHandshakeHandlers_Roundtrip(t *testing.T) {
require.NoError(t, err)
r2 := &Service{
cfg: &Config{
Chain: &mock.ChainService{
cfg: &config{
chain: &mock.ChainService{
FinalizedCheckPoint: &ethpb.Checkpoint{Epoch: 0, Root: finalizedRoot[:]},
},
P2P: p2,
p2p: p2,
},
rateLimiter: newRateLimiter(p2),
}
@@ -318,13 +318,13 @@ func TestHandshakeHandlers_Roundtrip(t *testing.T) {
p2.BHost.SetStreamHandler(pcl, func(stream network.Stream) {
defer wg.Done()
out := &pb.Status{}
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, out))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, out))
log.WithField("status", out).Warn("received status")
resp := &pb.Status{HeadSlot: 100, HeadRoot: make([]byte, 32), ForkDigest: p2.Digest[:],
FinalizedRoot: finalizedRoot[:], FinalizedEpoch: 0}
_, err := stream.Write([]byte{responseCodeSuccess})
assert.NoError(t, err)
_, err = r.cfg.P2P.Encoding().EncodeWithMaxLength(stream, resp)
_, err = r.cfg.p2p.Encoding().EncodeWithMaxLength(stream, resp)
assert.NoError(t, err)
log.WithField("status", out).Warn("sending status")
if err := stream.Close(); err != nil {
@@ -340,7 +340,7 @@ func TestHandshakeHandlers_Roundtrip(t *testing.T) {
p2.BHost.SetStreamHandler(pcl, func(stream network.Stream) {
defer wg2.Done()
out := new(types.SSZUint64)
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, out))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, out))
assert.Equal(t, uint64(2), uint64(*out))
assert.NoError(t, r2.pingHandler(context.Background(), out, stream))
assert.NoError(t, stream.Close())
@@ -408,9 +408,9 @@ func TestStatusRPCRequest_RequestSent(t *testing.T) {
}
r := &Service{
cfg: &Config{
P2P: p1,
Chain: &mock.ChainService{
cfg: &config{
p2p: p1,
chain: &mock.ChainService{
State: genesisState,
FinalizedCheckPoint: finalizedCheckpt,
Root: headRoot[:],
@@ -435,7 +435,7 @@ func TestStatusRPCRequest_RequestSent(t *testing.T) {
p2.BHost.SetStreamHandler(pcl, func(stream network.Stream) {
defer wg.Done()
out := &pb.Status{}
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, out))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, out))
digest, err := r.currentForkDigest()
assert.NoError(t, err)
expected := &pb.Status{
@@ -490,9 +490,9 @@ func TestStatusRPCRequest_FinalizedBlockExists(t *testing.T) {
totalSec := int64(params.BeaconConfig().SlotsPerEpoch.Mul(5 * params.BeaconConfig().SecondsPerSlot))
genTime := time.Now().Unix() - totalSec
r := &Service{
cfg: &Config{
P2P: p1,
Chain: &mock.ChainService{
cfg: &config{
p2p: p1,
chain: &mock.ChainService{
State: genesisState,
FinalizedCheckPoint: finalizedCheckpt,
Root: headRoot[:],
@@ -509,9 +509,9 @@ func TestStatusRPCRequest_FinalizedBlockExists(t *testing.T) {
}
r2 := &Service{
cfg: &Config{
P2P: p1,
Chain: &mock.ChainService{
cfg: &config{
p2p: p1,
chain: &mock.ChainService{
State: genesisState,
FinalizedCheckPoint: finalizedCheckpt,
Root: headRoot[:],
@@ -522,7 +522,7 @@ func TestStatusRPCRequest_FinalizedBlockExists(t *testing.T) {
Genesis: time.Unix(genTime, 0),
ValidatorsRoot: [32]byte{'A'},
},
DB: db,
beaconDB: db,
},
ctx: context.Background(),
rateLimiter: newRateLimiter(p1),
@@ -537,7 +537,7 @@ func TestStatusRPCRequest_FinalizedBlockExists(t *testing.T) {
p2.BHost.SetStreamHandler(pcl, func(stream network.Stream) {
defer wg.Done()
out := &pb.Status{}
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, out))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, out))
assert.NoError(t, r2.validateStatusMessage(context.Background(), out))
})
@@ -663,9 +663,9 @@ func TestStatusRPCRequest_FinalizedBlockSkippedSlots(t *testing.T) {
totalSec := uint64(params.BeaconConfig().SlotsPerEpoch.Mul(uint64(epoch) * params.BeaconConfig().SecondsPerSlot))
genTime := time.Now().Unix() - int64(totalSec)
r := &Service{
cfg: &Config{
P2P: p1,
Chain: &mock.ChainService{
cfg: &config{
p2p: p1,
chain: &mock.ChainService{
State: nState,
FinalizedCheckPoint: remoteFinalizedChkpt,
Root: rHeadRoot[:],
@@ -682,9 +682,9 @@ func TestStatusRPCRequest_FinalizedBlockSkippedSlots(t *testing.T) {
}
r2 := &Service{
cfg: &Config{
P2P: p2,
Chain: &mock.ChainService{
cfg: &config{
p2p: p2,
chain: &mock.ChainService{
State: nState,
FinalizedCheckPoint: finalizedCheckpt,
Root: headRoot[:],
@@ -695,7 +695,7 @@ func TestStatusRPCRequest_FinalizedBlockSkippedSlots(t *testing.T) {
Genesis: time.Unix(genTime, 0),
ValidatorsRoot: [32]byte{'A'},
},
DB: db,
beaconDB: db,
},
ctx: context.Background(),
@@ -711,7 +711,7 @@ func TestStatusRPCRequest_FinalizedBlockSkippedSlots(t *testing.T) {
p2.BHost.SetStreamHandler(pcl, func(stream network.Stream) {
defer wg.Done()
out := &pb.Status{}
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, out))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, out))
assert.Equal(t, tt.expectError, r2.validateStatusMessage(context.Background(), out) != nil)
})
@@ -750,9 +750,9 @@ func TestStatusRPCRequest_BadPeerHandshake(t *testing.T) {
}
r := &Service{
cfg: &Config{
P2P: p1,
Chain: &mock.ChainService{
cfg: &config{
p2p: p1,
chain: &mock.ChainService{
State: genesisState,
FinalizedCheckPoint: finalizedCheckpt,
Root: headRoot[:],
@@ -780,7 +780,7 @@ func TestStatusRPCRequest_BadPeerHandshake(t *testing.T) {
p2.BHost.SetStreamHandler(pcl, func(stream network.Stream) {
defer wg.Done()
out := &pb.Status{}
assert.NoError(t, r.cfg.P2P.Encoding().DecodeWithMaxLength(stream, out))
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, out))
expected := &pb.Status{
ForkDigest: []byte{1, 1, 1, 1},
HeadSlot: genesisState.Slot(),
@@ -791,7 +791,7 @@ func TestStatusRPCRequest_BadPeerHandshake(t *testing.T) {
if _, err := stream.Write([]byte{responseCodeSuccess}); err != nil {
log.WithError(err).Debug("Could not write to stream")
}
_, err := r.cfg.P2P.Encoding().EncodeWithMaxLength(stream, expected)
_, err := r.cfg.p2p.Encoding().EncodeWithMaxLength(stream, expected)
assert.NoError(t, err)
})
@@ -830,8 +830,8 @@ func TestStatusRPC_ValidGenesisMessage(t *testing.T) {
Root: finalizedRoot[:],
}
r := &Service{
cfg: &Config{
Chain: &mock.ChainService{
cfg: &config{
chain: &mock.ChainService{
State: genesisState,
FinalizedCheckPoint: finalizedCheckpt,
Root: headRoot[:],
@@ -912,12 +912,12 @@ func TestShouldResync(t *testing.T) {
require.NoError(t, err)
require.NoError(t, headState.SetSlot(tt.args.headSlot))
r := &Service{
cfg: &Config{
Chain: &mock.ChainService{
cfg: &config{
chain: &mock.ChainService{
State: headState,
Genesis: tt.args.genesis,
},
InitialSync: &mockSync.Sync{IsSyncing: tt.args.syncing},
initialSync: &mockSync.Sync{IsSyncing: tt.args.syncing},
},
ctx: context.Background(),
}

View File

@@ -50,7 +50,7 @@ func TestRegisterRPC_ReceivesValidMessage(t *testing.T) {
p2p := p2ptest.NewTestP2P(t)
r := &Service{
ctx: context.Background(),
cfg: &Config{P2P: p2p},
cfg: &config{p2p: p2p},
rateLimiter: newRateLimiter(p2p),
}

View File

@@ -66,23 +66,23 @@ var (
// Common type for functional p2p validation options.
type validationFn func(ctx context.Context) (pubsub.ValidationResult, error)
// Config to set up the regular sync service.
type Config struct {
AttestationNotifier operation.Notifier
P2P p2p.P2P
DB db.NoHeadAccessDatabase
AttPool attestations.Pool
ExitPool voluntaryexits.PoolManager
SlashingPool slashings.PoolManager
SyncCommsPool synccommittee.Pool
Chain blockchainService
InitialSync Checker
StateNotifier statefeed.Notifier
BlockNotifier blockfeed.Notifier
OperationNotifier operation.Notifier
StateGen *stategen.State
SlasherAttestationsFeed *event.Feed
SlasherBlockHeadersFeed *event.Feed
// config to hold dependencies for the sync service.
type config struct {
attestationNotifier operation.Notifier
p2p p2p.P2P
beaconDB db.NoHeadAccessDatabase
attPool attestations.Pool
exitPool voluntaryexits.PoolManager
slashingPool slashings.PoolManager
syncCommsPool synccommittee.Pool
chain blockchainService
initialSync Checker
stateNotifier statefeed.Notifier
blockNotifier blockfeed.Notifier
operationNotifier operation.Notifier
stateGen *stategen.State
slasherAttestationsFeed *event.Feed
slasherBlockHeadersFeed *event.Feed
}
// This defines the interface for interacting with block chain service
@@ -100,7 +100,7 @@ type blockchainService interface {
// Service is responsible for handling all run time p2p related operations as the
// main entry point for network messages.
type Service struct {
cfg *Config
cfg *config
ctx context.Context
cancel context.CancelFunc
slotToPendingBlocks *gcache.Cache
@@ -134,23 +134,26 @@ type Service struct {
}
// NewService initializes new regular sync service.
func NewService(ctx context.Context, cfg *Config) *Service {
func NewService(ctx context.Context, opts ...Option) *Service {
c := gcache.New(pendingBlockExpTime /* exp time */, 2*pendingBlockExpTime /* prune time */)
rLimiter := newRateLimiter(cfg.P2P)
ctx, cancel := context.WithCancel(ctx)
r := &Service{
cfg: cfg,
ctx: ctx,
cancel: cancel,
chainStarted: abool.New(),
cfg: &config{},
slotToPendingBlocks: c,
seenPendingBlocks: make(map[[32]byte]bool),
blkRootToPendingAtts: make(map[[32]byte][]*ethpb.SignedAggregateAttestationAndProof),
subHandler: newSubTopicHandler(),
rateLimiter: rLimiter,
signatureChan: make(chan *signatureVerifier, verifierLimit),
}
for _, opt := range opts {
if err := opt(r); err != nil {
return nil
}
}
r.subHandler = newSubTopicHandler()
r.rateLimiter = newRateLimiter(r.cfg.p2p)
go r.registerHandlers()
go r.verifierRoutine()
@@ -162,12 +165,12 @@ func NewService(ctx context.Context, cfg *Config) *Service {
func (s *Service) Start() {
s.initCaches()
s.cfg.P2P.AddConnectionHandler(s.reValidatePeer, s.sendGoodbye)
s.cfg.P2P.AddDisconnectionHandler(func(_ context.Context, _ peer.ID) error {
s.cfg.p2p.AddConnectionHandler(s.reValidatePeer, s.sendGoodbye)
s.cfg.p2p.AddDisconnectionHandler(func(_ context.Context, _ peer.ID) error {
// no-op
return nil
})
s.cfg.P2P.AddPingMethod(s.sendPingRequest)
s.cfg.p2p.AddPingMethod(s.sendPingRequest)
s.processPendingBlocksQueue()
s.processPendingAttsQueue()
s.maintainPeerStatuses()
@@ -187,11 +190,11 @@ func (s *Service) Stop() error {
}
}()
// Removing RPC Stream handlers.
for _, p := range s.cfg.P2P.Host().Mux().Protocols() {
s.cfg.P2P.Host().RemoveStreamHandler(protocol.ID(p))
for _, p := range s.cfg.p2p.Host().Mux().Protocols() {
s.cfg.p2p.Host().RemoveStreamHandler(protocol.ID(p))
}
// Deregister Topic Subscribers.
for _, t := range s.cfg.P2P.PubSub().GetTopics() {
for _, t := range s.cfg.p2p.PubSub().GetTopics() {
s.unSubscribeFromTopic(t)
}
defer s.cancel()
@@ -202,8 +205,8 @@ func (s *Service) Stop() error {
func (s *Service) Status() error {
// If our head slot is on a previous epoch and our peers are reporting their head block are
// in the most recent epoch, then we might be out of sync.
if headEpoch := slots.ToEpoch(s.cfg.Chain.HeadSlot()); headEpoch+1 < slots.ToEpoch(s.cfg.Chain.CurrentSlot()) &&
headEpoch+1 < s.cfg.P2P.Peers().HighestEpoch() {
if headEpoch := slots.ToEpoch(s.cfg.chain.HeadSlot()); headEpoch+1 < slots.ToEpoch(s.cfg.chain.CurrentSlot()) &&
headEpoch+1 < s.cfg.p2p.Peers().HighestEpoch() {
return errors.New("out of sync")
}
return nil
@@ -226,7 +229,7 @@ func (s *Service) initCaches() {
func (s *Service) registerHandlers() {
// Wait until chain start.
stateChannel := make(chan *feed.Event, 1)
stateSub := s.cfg.StateNotifier.StateFeed().Subscribe(stateChannel)
stateSub := s.cfg.stateNotifier.StateFeed().Subscribe(stateChannel)
defer stateSub.Unsubscribe()
for {
select {
@@ -263,7 +266,7 @@ func (s *Service) registerHandlers() {
log.WithError(err).Error("Could not retrieve current fork digest")
return
}
currentEpoch := slots.ToEpoch(slots.CurrentSlot(uint64(s.cfg.Chain.GenesisTime().Unix())))
currentEpoch := slots.ToEpoch(slots.CurrentSlot(uint64(s.cfg.chain.GenesisTime().Unix())))
s.registerSubscribers(currentEpoch, digest)
go s.forkWatcher()
return

View File

@@ -26,10 +26,10 @@ func TestService_StatusZeroEpoch(t *testing.T) {
bState, err := v1.InitializeFromProto(&ethpb.BeaconState{Slot: 0})
require.NoError(t, err)
r := &Service{
cfg: &Config{
P2P: p2ptest.NewTestP2P(t),
InitialSync: new(mockSync.Sync),
Chain: &mockChain.ChainService{
cfg: &config{
p2p: p2ptest.NewTestP2P(t),
initialSync: new(mockSync.Sync),
chain: &mockChain.ChainService{
Genesis: time.Now(),
State: bState,
},
@@ -49,11 +49,11 @@ func TestSyncHandlers_WaitToSync(t *testing.T) {
}
r := Service{
ctx: context.Background(),
cfg: &Config{
P2P: p2p,
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
InitialSync: &mockSync.Sync{IsSyncing: false},
cfg: &config{
p2p: p2p,
chain: chainService,
stateNotifier: chainService.StateNotifier(),
initialSync: &mockSync.Sync{IsSyncing: false},
},
chainStarted: abool.New(),
}
@@ -61,7 +61,7 @@ func TestSyncHandlers_WaitToSync(t *testing.T) {
topic := "/eth2/%x/beacon_block"
go r.registerHandlers()
time.Sleep(100 * time.Millisecond)
i := r.cfg.StateNotifier.StateFeed().Send(&feed.Event{
i := r.cfg.stateNotifier.StateFeed().Send(&feed.Event{
Type: statefeed.Initialized,
Data: &statefeed.InitializedData{
StartTime: time.Now(),
@@ -92,11 +92,11 @@ func TestSyncHandlers_WaitForChainStart(t *testing.T) {
}
r := Service{
ctx: context.Background(),
cfg: &Config{
P2P: p2p,
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
InitialSync: &mockSync.Sync{IsSyncing: false},
cfg: &config{
p2p: p2p,
chain: chainService,
stateNotifier: chainService.StateNotifier(),
initialSync: &mockSync.Sync{IsSyncing: false},
},
chainStarted: abool.New(),
slotToPendingBlocks: gcache.New(time.Second, 2*time.Second),
@@ -104,7 +104,7 @@ func TestSyncHandlers_WaitForChainStart(t *testing.T) {
go r.registerHandlers()
time.Sleep(100 * time.Millisecond)
i := r.cfg.StateNotifier.StateFeed().Send(&feed.Event{
i := r.cfg.stateNotifier.StateFeed().Send(&feed.Event{
Type: statefeed.Initialized,
Data: &statefeed.InitializedData{
StartTime: time.Now().Add(2 * time.Second),
@@ -128,11 +128,11 @@ func TestSyncHandlers_WaitTillSynced(t *testing.T) {
}
r := Service{
ctx: context.Background(),
cfg: &Config{
P2P: p2p,
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
InitialSync: &mockSync.Sync{IsSyncing: false},
cfg: &config{
p2p: p2p,
chain: chainService,
stateNotifier: chainService.StateNotifier(),
initialSync: &mockSync.Sync{IsSyncing: false},
},
chainStarted: abool.New(),
subHandler: newSubTopicHandler(),
@@ -141,7 +141,7 @@ func TestSyncHandlers_WaitTillSynced(t *testing.T) {
topic := "/eth2/%x/beacon_block"
go r.registerHandlers()
time.Sleep(100 * time.Millisecond)
i := r.cfg.StateNotifier.StateFeed().Send(&feed.Event{
i := r.cfg.stateNotifier.StateFeed().Send(&feed.Event{
Type: statefeed.Initialized,
Data: &statefeed.InitializedData{
StartTime: time.Now(),
@@ -159,9 +159,9 @@ func TestSyncHandlers_WaitTillSynced(t *testing.T) {
msg.Block.ParentRoot = util.Random32Bytes(t)
msg.Signature = sk.Sign([]byte("data")).Marshal()
p2p.Digest, err = r.currentForkDigest()
r.cfg.BlockNotifier = chainService.BlockNotifier()
r.cfg.blockNotifier = chainService.BlockNotifier()
blockChan := make(chan feed.Event, 1)
sub := r.cfg.BlockNotifier.BlockFeed().Subscribe(blockChan)
sub := r.cfg.blockNotifier.BlockFeed().Subscribe(blockChan)
require.NoError(t, err)
p2p.ReceivePubSub(topic, msg)
@@ -172,7 +172,7 @@ func TestSyncHandlers_WaitTillSynced(t *testing.T) {
assert.Equal(t, 0, len(blockChan), "block was received by sync service despite not being fully synced")
i = r.cfg.StateNotifier.StateFeed().Send(&feed.Event{
i = r.cfg.stateNotifier.StateFeed().Send(&feed.Event{
Type: statefeed.Synced,
Data: &statefeed.SyncedData{
StartTime: time.Now(),
@@ -207,11 +207,11 @@ func TestSyncService_StopCleanly(t *testing.T) {
r := Service{
ctx: ctx,
cancel: cancel,
cfg: &Config{
P2P: p2p,
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
InitialSync: &mockSync.Sync{IsSyncing: false},
cfg: &config{
p2p: p2p,
chain: chainService,
stateNotifier: chainService.StateNotifier(),
initialSync: &mockSync.Sync{IsSyncing: false},
},
chainStarted: abool.New(),
subHandler: newSubTopicHandler(),
@@ -219,7 +219,7 @@ func TestSyncService_StopCleanly(t *testing.T) {
go r.registerHandlers()
time.Sleep(100 * time.Millisecond)
i := r.cfg.StateNotifier.StateFeed().Send(&feed.Event{
i := r.cfg.stateNotifier.StateFeed().Send(&feed.Event{
Type: statefeed.Initialized,
Data: &statefeed.InitializedData{
StartTime: time.Now(),
@@ -237,7 +237,7 @@ func TestSyncService_StopCleanly(t *testing.T) {
time.Sleep(2 * time.Second)
require.Equal(t, true, r.chainStarted.IsSet(), "Did not receive chain start event.")
i = r.cfg.StateNotifier.StateFeed().Send(&feed.Event{
i = r.cfg.stateNotifier.StateFeed().Send(&feed.Event{
Type: statefeed.Synced,
Data: &statefeed.SyncedData{
StartTime: time.Now(),
@@ -249,14 +249,14 @@ func TestSyncService_StopCleanly(t *testing.T) {
time.Sleep(1 * time.Second)
require.NotEqual(t, 0, len(r.cfg.P2P.PubSub().GetTopics()))
require.NotEqual(t, 0, len(r.cfg.P2P.Host().Mux().Protocols()))
require.NotEqual(t, 0, len(r.cfg.p2p.PubSub().GetTopics()))
require.NotEqual(t, 0, len(r.cfg.p2p.Host().Mux().Protocols()))
// Both pubsub and rpc topcis should be unsubscribed.
require.NoError(t, r.Stop())
// Sleep to allow pubsub topics to be deregistered.
time.Sleep(1 * time.Second)
require.Equal(t, 0, len(r.cfg.P2P.PubSub().GetTopics()))
require.Equal(t, 0, len(r.cfg.P2P.Host().Mux().Protocols()))
require.Equal(t, 0, len(r.cfg.p2p.PubSub().GetTopics()))
require.Equal(t, 0, len(r.cfg.p2p.Host().Mux().Protocols()))
}

View File

@@ -124,7 +124,7 @@ func (s *Service) registerSubscribers(epoch types.Epoch, digest [4]byte) {
// subscribe to a given topic with a given validator and subscription handler.
// The base protobuf message is used to initialize new messages for decoding.
func (s *Service) subscribe(topic string, validator wrappedVal, handle subHandler, digest [4]byte) *pubsub.Subscription {
genRoot := s.cfg.Chain.GenesisValidatorRoot()
genRoot := s.cfg.chain.GenesisValidatorRoot()
_, e, err := forks.RetrieveForkDataFromDigest(digest, genRoot[:])
if err != nil {
// Impossible condition as it would mean digest does not exist.
@@ -139,7 +139,7 @@ func (s *Service) subscribe(topic string, validator wrappedVal, handle subHandle
}
func (s *Service) subscribeWithBase(topic string, validator wrappedVal, handle subHandler) *pubsub.Subscription {
topic += s.cfg.P2P.Encoding().ProtocolSuffix()
topic += s.cfg.p2p.Encoding().ProtocolSuffix()
log := log.WithField("topic", topic)
// Do not resubscribe already seen subscriptions.
@@ -149,12 +149,12 @@ func (s *Service) subscribeWithBase(topic string, validator wrappedVal, handle s
return nil
}
if err := s.cfg.P2P.PubSub().RegisterTopicValidator(s.wrapAndReportValidation(topic, validator)); err != nil {
if err := s.cfg.p2p.PubSub().RegisterTopicValidator(s.wrapAndReportValidation(topic, validator)); err != nil {
log.WithError(err).Error("Could not register validator for topic")
return nil
}
sub, err := s.cfg.P2P.SubscribeToTopic(topic)
sub, err := s.cfg.p2p.SubscribeToTopic(topic)
if err != nil {
// Any error subscribing to a PubSub topic would be the result of a misconfiguration of
// libp2p PubSub library or a subscription request to a topic that fails to match the topic
@@ -211,7 +211,7 @@ func (s *Service) subscribeWithBase(topic string, validator wrappedVal, handle s
return
}
if msg.ReceivedFrom == s.cfg.P2P.PeerID() {
if msg.ReceivedFrom == s.cfg.p2p.PeerID() {
continue
}
@@ -260,10 +260,10 @@ func (s *Service) wrapAndReportValidation(topic string, v wrappedVal) (string, p
if b == pubsub.ValidationReject {
log.WithError(err).WithFields(logrus.Fields{
"topic": topic,
"multiaddress": multiAddr(pid, s.cfg.P2P.Peers()),
"multiaddress": multiAddr(pid, s.cfg.p2p.Peers()),
"peer id": pid.String(),
"agent": agentString(pid, s.cfg.P2P.Host()),
"gossip score": s.cfg.P2P.Peers().Scorers().GossipScorer().Score(pid),
"agent": agentString(pid, s.cfg.p2p.Host()),
"gossip score": s.cfg.p2p.Peers().Scorers().GossipScorer().Score(pid),
}).Debugf("Gossip message was rejected")
messageFailedValidationCounter.WithLabelValues(topic).Inc()
}
@@ -271,10 +271,10 @@ func (s *Service) wrapAndReportValidation(topic string, v wrappedVal) (string, p
if err != nil {
log.WithError(err).WithFields(logrus.Fields{
"topic": topic,
"multiaddress": multiAddr(pid, s.cfg.P2P.Peers()),
"multiaddress": multiAddr(pid, s.cfg.p2p.Peers()),
"peer id": pid.String(),
"agent": agentString(pid, s.cfg.P2P.Host()),
"gossip score": s.cfg.P2P.Peers().Scorers().GossipScorer().Score(pid),
"agent": agentString(pid, s.cfg.p2p.Host()),
"gossip score": s.cfg.p2p.Peers().Scorers().GossipScorer().Score(pid),
}).Debugf("Gossip message was ignored")
}
messageIgnoredValidationCounter.WithLabelValues(topic).Inc()
@@ -286,7 +286,7 @@ func (s *Service) wrapAndReportValidation(topic string, v wrappedVal) (string, p
// subscribe to a static subnet with the given topic and index.A given validator and subscription handler is
// used to handle messages from the subnet. The base protobuf message is used to initialize new messages for decoding.
func (s *Service) subscribeStaticWithSubnets(topic string, validator wrappedVal, handle subHandler, digest [4]byte) {
genRoot := s.cfg.Chain.GenesisValidatorRoot()
genRoot := s.cfg.chain.GenesisValidatorRoot()
_, e, err := forks.RetrieveForkDataFromDigest(digest, genRoot[:])
if err != nil {
// Impossible condition as it would mean digest does not exist.
@@ -300,7 +300,7 @@ func (s *Service) subscribeStaticWithSubnets(topic string, validator wrappedVal,
for i := uint64(0); i < params.BeaconNetworkConfig().AttestationSubnetCount; i++ {
s.subscribeWithBase(s.addDigestAndIndexToTopic(topic, digest, i), validator, handle)
}
genesis := s.cfg.Chain.GenesisTime()
genesis := s.cfg.chain.GenesisTime()
ticker := slots.NewSlotTicker(genesis, params.BeaconConfig().SecondsPerSlot)
go func() {
@@ -310,7 +310,7 @@ func (s *Service) subscribeStaticWithSubnets(topic string, validator wrappedVal,
ticker.Done()
return
case <-ticker.C():
if s.chainStarted.IsSet() && s.cfg.InitialSync.Syncing() {
if s.chainStarted.IsSet() && s.cfg.initialSync.Syncing() {
continue
}
valid, err := isDigestValid(digest, genesis, genRoot)
@@ -322,7 +322,7 @@ func (s *Service) subscribeStaticWithSubnets(topic string, validator wrappedVal,
log.Warnf("Attestation subnets with digest %#x are no longer valid, unsubscribing from all of them.", digest)
// Unsubscribes from all our current subnets.
for i := uint64(0); i < params.BeaconNetworkConfig().AttestationSubnetCount; i++ {
fullTopic := fmt.Sprintf(topic, digest, i) + s.cfg.P2P.Encoding().ProtocolSuffix()
fullTopic := fmt.Sprintf(topic, digest, i) + s.cfg.p2p.Encoding().ProtocolSuffix()
s.unSubscribeFromTopic(fullTopic)
}
ticker.Done()
@@ -333,7 +333,7 @@ func (s *Service) subscribeStaticWithSubnets(topic string, validator wrappedVal,
if !s.validPeersExist(s.addDigestAndIndexToTopic(topic, digest, i)) {
log.Debugf("No peers found subscribed to attestation gossip subnet with "+
"committee index %d. Searching network for peers subscribed to the subnet.", i)
_, err := s.cfg.P2P.FindPeersWithSubnet(
_, err := s.cfg.p2p.FindPeersWithSubnet(
s.ctx,
s.addDigestAndIndexToTopic(topic, digest, i),
i,
@@ -359,7 +359,7 @@ func (s *Service) subscribeDynamicWithSubnets(
handle subHandler,
digest [4]byte,
) {
genRoot := s.cfg.Chain.GenesisValidatorRoot()
genRoot := s.cfg.chain.GenesisValidatorRoot()
_, e, err := forks.RetrieveForkDataFromDigest(digest, genRoot[:])
if err != nil {
// Impossible condition as it would mean digest does not exist.
@@ -370,7 +370,7 @@ func (s *Service) subscribeDynamicWithSubnets(
panic(fmt.Sprintf("%s is not mapped to any message in GossipTopicMappings", topicFormat))
}
subscriptions := make(map[uint64]*pubsub.Subscription, params.BeaconConfig().MaxCommitteesPerSlot)
genesis := s.cfg.Chain.GenesisTime()
genesis := s.cfg.chain.GenesisTime()
ticker := slots.NewSlotTicker(genesis, params.BeaconConfig().SecondsPerSlot)
go func() {
@@ -380,7 +380,7 @@ func (s *Service) subscribeDynamicWithSubnets(
ticker.Done()
return
case currentSlot := <-ticker.C():
if s.chainStarted.IsSet() && s.cfg.InitialSync.Syncing() {
if s.chainStarted.IsSet() && s.cfg.initialSync.Syncing() {
continue
}
valid, err := isDigestValid(digest, genesis, genRoot)
@@ -426,7 +426,7 @@ func (s *Service) reValidateSubscriptions(subscriptions map[uint64]*pubsub.Subsc
}
if !wanted && v != nil {
v.Cancel()
fullTopic := fmt.Sprintf(topicFormat, digest, k) + s.cfg.P2P.Encoding().ProtocolSuffix()
fullTopic := fmt.Sprintf(topicFormat, digest, k) + s.cfg.p2p.Encoding().ProtocolSuffix()
s.unSubscribeFromTopic(fullTopic)
delete(subscriptions, k)
}
@@ -452,7 +452,7 @@ func (s *Service) subscribeAggregatorSubnet(
if !s.validPeersExist(subnetTopic) {
log.Debugf("No peers found subscribed to attestation gossip subnet with "+
"committee index %d. Searching network for peers subscribed to the subnet.", idx)
_, err := s.cfg.P2P.FindPeersWithSubnet(s.ctx, subnetTopic, idx, flags.Get().MinimumPeersPerSubnet)
_, err := s.cfg.p2p.FindPeersWithSubnet(s.ctx, subnetTopic, idx, flags.Get().MinimumPeersPerSubnet)
if err != nil {
log.WithError(err).Debug("Could not search for peers")
}
@@ -478,7 +478,7 @@ func (s *Service) subscribeSyncSubnet(
if !s.validPeersExist(subnetTopic) {
log.Debugf("No peers found subscribed to sync gossip subnet with "+
"committee index %d. Searching network for peers subscribed to the subnet.", idx)
_, err := s.cfg.P2P.FindPeersWithSubnet(s.ctx, subnetTopic, idx, flags.Get().MinimumPeersPerSubnet)
_, err := s.cfg.p2p.FindPeersWithSubnet(s.ctx, subnetTopic, idx, flags.Get().MinimumPeersPerSubnet)
if err != nil {
log.WithError(err).Debug("Could not search for peers")
}
@@ -488,7 +488,7 @@ func (s *Service) subscribeSyncSubnet(
// subscribe to a static subnet with the given topic and index. A given validator and subscription handler is
// used to handle messages from the subnet. The base protobuf message is used to initialize new messages for decoding.
func (s *Service) subscribeStaticWithSyncSubnets(topic string, validator wrappedVal, handle subHandler, digest [4]byte) {
genRoot := s.cfg.Chain.GenesisValidatorRoot()
genRoot := s.cfg.chain.GenesisValidatorRoot()
_, e, err := forks.RetrieveForkDataFromDigest(digest, genRoot[:])
if err != nil {
panic(err)
@@ -500,7 +500,7 @@ func (s *Service) subscribeStaticWithSyncSubnets(topic string, validator wrapped
for i := uint64(0); i < params.BeaconConfig().SyncCommitteeSubnetCount; i++ {
s.subscribeWithBase(s.addDigestAndIndexToTopic(topic, digest, i), validator, handle)
}
genesis := s.cfg.Chain.GenesisTime()
genesis := s.cfg.chain.GenesisTime()
ticker := slots.NewSlotTicker(genesis, params.BeaconConfig().SecondsPerSlot)
go func() {
@@ -510,7 +510,7 @@ func (s *Service) subscribeStaticWithSyncSubnets(topic string, validator wrapped
ticker.Done()
return
case <-ticker.C():
if s.chainStarted.IsSet() && s.cfg.InitialSync.Syncing() {
if s.chainStarted.IsSet() && s.cfg.initialSync.Syncing() {
continue
}
valid, err := isDigestValid(digest, genesis, genRoot)
@@ -522,7 +522,7 @@ func (s *Service) subscribeStaticWithSyncSubnets(topic string, validator wrapped
log.Warnf("Sync subnets with digest %#x are no longer valid, unsubscribing from all of them.", digest)
// Unsubscribes from all our current subnets.
for i := uint64(0); i < params.BeaconConfig().SyncCommitteeSubnetCount; i++ {
fullTopic := fmt.Sprintf(topic, digest, i) + s.cfg.P2P.Encoding().ProtocolSuffix()
fullTopic := fmt.Sprintf(topic, digest, i) + s.cfg.p2p.Encoding().ProtocolSuffix()
s.unSubscribeFromTopic(fullTopic)
}
ticker.Done()
@@ -533,7 +533,7 @@ func (s *Service) subscribeStaticWithSyncSubnets(topic string, validator wrapped
if !s.validPeersExist(s.addDigestAndIndexToTopic(topic, digest, i)) {
log.Debugf("No peers found subscribed to sync gossip subnet with "+
"committee index %d. Searching network for peers subscribed to the subnet.", i)
_, err := s.cfg.P2P.FindPeersWithSubnet(
_, err := s.cfg.p2p.FindPeersWithSubnet(
s.ctx,
s.addDigestAndIndexToTopic(topic, digest, i),
i,
@@ -559,7 +559,7 @@ func (s *Service) subscribeDynamicWithSyncSubnets(
handle subHandler,
digest [4]byte,
) {
genRoot := s.cfg.Chain.GenesisValidatorRoot()
genRoot := s.cfg.chain.GenesisValidatorRoot()
_, e, err := forks.RetrieveForkDataFromDigest(digest, genRoot[:])
if err != nil {
panic(err)
@@ -569,7 +569,7 @@ func (s *Service) subscribeDynamicWithSyncSubnets(
panic(fmt.Sprintf("%s is not mapped to any message in GossipTopicMappings", topicFormat))
}
subscriptions := make(map[uint64]*pubsub.Subscription, params.BeaconConfig().SyncCommitteeSubnetCount)
genesis := s.cfg.Chain.GenesisTime()
genesis := s.cfg.chain.GenesisTime()
ticker := slots.NewSlotTicker(genesis, params.BeaconConfig().SecondsPerSlot)
go func() {
@@ -579,7 +579,7 @@ func (s *Service) subscribeDynamicWithSyncSubnets(
ticker.Done()
return
case currentSlot := <-ticker.C():
if s.chainStarted.IsSet() && s.cfg.InitialSync.Syncing() {
if s.chainStarted.IsSet() && s.cfg.initialSync.Syncing() {
continue
}
valid, err := isDigestValid(digest, genesis, genRoot)
@@ -616,7 +616,7 @@ func (s *Service) lookupAttesterSubnets(digest [4]byte, idx uint64) {
log.Debugf("No peers found subscribed to attestation gossip subnet with "+
"committee index %d. Searching network for peers subscribed to the subnet.", idx)
// perform a search for peers with the desired committee index.
_, err := s.cfg.P2P.FindPeersWithSubnet(s.ctx, subnetTopic, idx, flags.Get().MinimumPeersPerSubnet)
_, err := s.cfg.p2p.FindPeersWithSubnet(s.ctx, subnetTopic, idx, flags.Get().MinimumPeersPerSubnet)
if err != nil {
log.WithError(err).Debug("Could not search for peers")
}
@@ -625,7 +625,7 @@ func (s *Service) lookupAttesterSubnets(digest [4]byte, idx uint64) {
func (s *Service) unSubscribeFromTopic(topic string) {
log.WithField("topic", topic).Debug("Unsubscribing from topic")
if err := s.cfg.P2P.PubSub().UnregisterTopicValidator(topic); err != nil {
if err := s.cfg.p2p.PubSub().UnregisterTopicValidator(topic); err != nil {
log.WithError(err).Error("Could not unregister topic validator")
}
sub := s.subHandler.subForTopic(topic)
@@ -633,14 +633,14 @@ func (s *Service) unSubscribeFromTopic(topic string) {
sub.Cancel()
}
s.subHandler.removeTopic(topic)
if err := s.cfg.P2P.LeaveTopic(topic); err != nil {
if err := s.cfg.p2p.LeaveTopic(topic); err != nil {
log.WithError(err).Error("Unable to leave topic")
}
}
// find if we have peers who are subscribed to the same subnet
func (s *Service) validPeersExist(subnetTopic string) bool {
numOfPeers := s.cfg.P2P.PubSub().ListPeers(subnetTopic + s.cfg.P2P.Encoding().ProtocolSuffix())
numOfPeers := s.cfg.p2p.PubSub().ListPeers(subnetTopic + s.cfg.p2p.Encoding().ProtocolSuffix())
return len(numOfPeers) >= flags.Get().MinimumPeersPerSubnet
}
@@ -671,7 +671,7 @@ func (s *Service) filterNeededPeers(pids []peer.ID) []peer.ID {
log.WithError(err).Error("Could not compute fork digest")
return pids
}
currSlot := s.cfg.Chain.CurrentSlot()
currSlot := s.cfg.chain.CurrentSlot()
wantedSubs := s.retrievePersistentSubs(currSlot)
wantedSubs = slice.SetUint64(append(wantedSubs, s.attesterSubnetIndices(currSlot)...))
topic := p2p.GossipTypeMapping[reflect.TypeOf(&pb.Attestation{})]
@@ -680,8 +680,8 @@ func (s *Service) filterNeededPeers(pids []peer.ID) []peer.ID {
peerMap := make(map[peer.ID]bool)
for _, sub := range wantedSubs {
subnetTopic := fmt.Sprintf(topic, digest, sub) + s.cfg.P2P.Encoding().ProtocolSuffix()
peers := s.cfg.P2P.PubSub().ListPeers(subnetTopic)
subnetTopic := fmt.Sprintf(topic, digest, sub) + s.cfg.p2p.Encoding().ProtocolSuffix()
peers := s.cfg.p2p.PubSub().ListPeers(subnetTopic)
if len(peers) > flags.Get().MinimumPeersPerSubnet {
// In the event we have more than the minimum, we can
// mark the remaining as viable for pruning.
@@ -725,8 +725,8 @@ func (s *Service) addDigestAndIndexToTopic(topic string, digest [4]byte, idx uin
}
func (s *Service) currentForkDigest() ([4]byte, error) {
genRoot := s.cfg.Chain.GenesisValidatorRoot()
return forks.CreateForkDigest(s.cfg.Chain.GenesisTime(), genRoot[:])
genRoot := s.cfg.chain.GenesisValidatorRoot()
return forks.CreateForkDigest(s.cfg.chain.GenesisTime(), genRoot[:])
}
// Checks if the provided digest matches up with the current supposed digest.

View File

@@ -24,8 +24,8 @@ func (s *Service) beaconAggregateProofSubscriber(_ context.Context, msg proto.Me
// An unaggregated attestation can make it here. Its valid, the aggregator it just itself, although it means poor performance for the subnet.
if !helpers.IsAggregated(a.Message.Aggregate) {
return s.cfg.AttPool.SaveUnaggregatedAttestation(a.Message.Aggregate)
return s.cfg.attPool.SaveUnaggregatedAttestation(a.Message.Aggregate)
}
return s.cfg.AttPool.SaveAggregatedAttestation(a.Message.Aggregate)
return s.cfg.attPool.SaveAggregatedAttestation(a.Message.Aggregate)
}

View File

@@ -17,9 +17,9 @@ import (
func TestBeaconAggregateProofSubscriber_CanSaveAggregatedAttestation(t *testing.T) {
r := &Service{
cfg: &Config{
AttPool: attestations.NewPool(),
AttestationNotifier: (&mock.ChainService{}).OperationNotifier(),
cfg: &config{
attPool: attestations.NewPool(),
attestationNotifier: (&mock.ChainService{}).OperationNotifier(),
},
seenUnAggregatedAttestationCache: lruwrpr.New(10),
}
@@ -34,14 +34,14 @@ func TestBeaconAggregateProofSubscriber_CanSaveAggregatedAttestation(t *testing.
Signature: make([]byte, params.BeaconConfig().BLSSignatureLength),
}
require.NoError(t, r.beaconAggregateProofSubscriber(context.Background(), a))
assert.DeepSSZEqual(t, []*ethpb.Attestation{a.Message.Aggregate}, r.cfg.AttPool.AggregatedAttestations(), "Did not save aggregated attestation")
assert.DeepSSZEqual(t, []*ethpb.Attestation{a.Message.Aggregate}, r.cfg.attPool.AggregatedAttestations(), "Did not save aggregated attestation")
}
func TestBeaconAggregateProofSubscriber_CanSaveUnaggregatedAttestation(t *testing.T) {
r := &Service{
cfg: &Config{
AttPool: attestations.NewPool(),
AttestationNotifier: (&mock.ChainService{}).OperationNotifier(),
cfg: &config{
attPool: attestations.NewPool(),
attestationNotifier: (&mock.ChainService{}).OperationNotifier(),
},
seenUnAggregatedAttestationCache: lruwrpr.New(10),
}
@@ -57,7 +57,7 @@ func TestBeaconAggregateProofSubscriber_CanSaveUnaggregatedAttestation(t *testin
}
require.NoError(t, r.beaconAggregateProofSubscriber(context.Background(), a))
atts, err := r.cfg.AttPool.UnaggregatedAttestations()
atts, err := r.cfg.attPool.UnaggregatedAttestations()
require.NoError(t, err)
assert.DeepEqual(t, []*ethpb.Attestation{a.Message.Aggregate}, atts, "Did not save unaggregated attestation")
}

View File

@@ -25,7 +25,7 @@ func (s *Service) committeeIndexBeaconAttestationSubscriber(_ context.Context, m
}
s.setSeenCommitteeIndicesSlot(a.Data.Slot, a.Data.CommitteeIndex, a.AggregationBits)
exists, err := s.cfg.AttPool.HasAggregatedAttestation(a)
exists, err := s.cfg.attPool.HasAggregatedAttestation(a)
if err != nil {
return errors.Wrap(err, "Could not determine if attestation pool has this atttestation")
}
@@ -33,7 +33,7 @@ func (s *Service) committeeIndexBeaconAttestationSubscriber(_ context.Context, m
return nil
}
return s.cfg.AttPool.SaveUnaggregatedAttestation(a)
return s.cfg.attPool.SaveUnaggregatedAttestation(a)
}
func (s *Service) persistentSubnetIndices() []uint64 {

View File

@@ -32,7 +32,7 @@ func (s *Service) beaconBlockSubscriber(ctx context.Context, msg proto.Message)
return err
}
if err := s.cfg.Chain.ReceiveBlock(ctx, signed, root); err != nil {
if err := s.cfg.chain.ReceiveBlock(ctx, signed, root); err != nil {
interop.WriteBlockToDisk(signed, true /*failed*/)
s.setBadBlock(ctx, root)
return err
@@ -53,12 +53,12 @@ func (s *Service) beaconBlockSubscriber(ctx context.Context, msg proto.Message)
func (s *Service) deleteAttsInPool(atts []*ethpb.Attestation) error {
for _, att := range atts {
if helpers.IsAggregated(att) {
if err := s.cfg.AttPool.DeleteAggregatedAttestation(att); err != nil {
if err := s.cfg.attPool.DeleteAggregatedAttestation(att); err != nil {
return err
}
} else {
// Ideally there's shouldn't be any unaggregated attestation in the block.
if err := s.cfg.AttPool.DeleteUnaggregatedAttestation(att); err != nil {
if err := s.cfg.attPool.DeleteUnaggregatedAttestation(att); err != nil {
return err
}
}

View File

@@ -21,23 +21,23 @@ import (
func TestDeleteAttsInPool(t *testing.T) {
r := &Service{
cfg: &Config{AttPool: attestations.NewPool()},
cfg: &config{attPool: attestations.NewPool()},
}
att1 := util.HydrateAttestation(&ethpb.Attestation{AggregationBits: bitfield.Bitlist{0b1101}})
att2 := util.HydrateAttestation(&ethpb.Attestation{AggregationBits: bitfield.Bitlist{0b1110}})
att3 := util.HydrateAttestation(&ethpb.Attestation{AggregationBits: bitfield.Bitlist{0b1011}})
att4 := util.HydrateAttestation(&ethpb.Attestation{AggregationBits: bitfield.Bitlist{0b1001}})
require.NoError(t, r.cfg.AttPool.SaveAggregatedAttestation(att1))
require.NoError(t, r.cfg.AttPool.SaveAggregatedAttestation(att2))
require.NoError(t, r.cfg.AttPool.SaveAggregatedAttestation(att3))
require.NoError(t, r.cfg.AttPool.SaveUnaggregatedAttestation(att4))
require.NoError(t, r.cfg.attPool.SaveAggregatedAttestation(att1))
require.NoError(t, r.cfg.attPool.SaveAggregatedAttestation(att2))
require.NoError(t, r.cfg.attPool.SaveAggregatedAttestation(att3))
require.NoError(t, r.cfg.attPool.SaveUnaggregatedAttestation(att4))
// Seen 1, 3 and 4 in block.
require.NoError(t, r.deleteAttsInPool([]*ethpb.Attestation{att1, att3, att4}))
// Only 2 should remain.
assert.DeepEqual(t, []*ethpb.Attestation{att2}, r.cfg.AttPool.AggregatedAttestations(), "Did not get wanted attestations")
assert.DeepEqual(t, []*ethpb.Attestation{att2}, r.cfg.attPool.AggregatedAttestations(), "Did not get wanted attestations")
}
func TestService_beaconBlockSubscriber(t *testing.T) {
@@ -68,10 +68,10 @@ func TestService_beaconBlockSubscriber(t *testing.T) {
},
wantedErr: "nil inner state",
check: func(t *testing.T, s *Service) {
if s.cfg.AttPool.AggregatedAttestationCount() == 0 {
if s.cfg.attPool.AggregatedAttestationCount() == 0 {
t.Error("Expected at least 1 aggregated attestation in the pool")
}
if s.cfg.AttPool.UnaggregatedAttestationCount() == 0 {
if s.cfg.attPool.UnaggregatedAttestationCount() == 0 {
t.Error("Expected at least 1 unaggregated attestation in the pool")
}
},
@@ -81,21 +81,21 @@ func TestService_beaconBlockSubscriber(t *testing.T) {
t.Run(tt.name, func(t *testing.T) {
db := dbtest.SetupDB(t)
s := &Service{
cfg: &Config{
Chain: &chainMock.ChainService{
cfg: &config{
chain: &chainMock.ChainService{
DB: db,
Root: make([]byte, 32),
},
AttPool: attestations.NewPool(),
attPool: attestations.NewPool(),
},
}
s.initCaches()
// Set up attestation pool.
for _, att := range pooledAttestations {
if helpers.IsAggregated(att) {
assert.NoError(t, s.cfg.AttPool.SaveAggregatedAttestation(att))
assert.NoError(t, s.cfg.attPool.SaveAggregatedAttestation(att))
} else {
assert.NoError(t, s.cfg.AttPool.SaveUnaggregatedAttestation(att))
assert.NoError(t, s.cfg.attPool.SaveUnaggregatedAttestation(att))
}
}
// Perform method under test call.

View File

@@ -20,11 +20,11 @@ func (s *Service) voluntaryExitSubscriber(ctx context.Context, msg proto.Message
}
s.setExitIndexSeen(ve.Exit.ValidatorIndex)
headState, err := s.cfg.Chain.HeadState(ctx)
headState, err := s.cfg.chain.HeadState(ctx)
if err != nil {
return err
}
s.cfg.ExitPool.InsertVoluntaryExit(ctx, headState, ve)
s.cfg.exitPool.InsertVoluntaryExit(ctx, headState, ve)
return nil
}
@@ -37,11 +37,11 @@ func (s *Service) attesterSlashingSubscriber(ctx context.Context, msg proto.Mess
aSlashing1IsNil := aSlashing == nil || aSlashing.Attestation_1 == nil || aSlashing.Attestation_1.AttestingIndices == nil
aSlashing2IsNil := aSlashing == nil || aSlashing.Attestation_2 == nil || aSlashing.Attestation_2.AttestingIndices == nil
if !aSlashing1IsNil && !aSlashing2IsNil {
headState, err := s.cfg.Chain.HeadState(ctx)
headState, err := s.cfg.chain.HeadState(ctx)
if err != nil {
return err
}
if err := s.cfg.SlashingPool.InsertAttesterSlashing(ctx, headState, aSlashing); err != nil {
if err := s.cfg.slashingPool.InsertAttesterSlashing(ctx, headState, aSlashing); err != nil {
return errors.Wrap(err, "could not insert attester slashing into pool")
}
s.setAttesterSlashingIndicesSeen(aSlashing.Attestation_1.AttestingIndices, aSlashing.Attestation_2.AttestingIndices)
@@ -58,11 +58,11 @@ func (s *Service) proposerSlashingSubscriber(ctx context.Context, msg proto.Mess
header1IsNil := pSlashing == nil || pSlashing.Header_1 == nil || pSlashing.Header_1.Header == nil
header2IsNil := pSlashing == nil || pSlashing.Header_2 == nil || pSlashing.Header_2.Header == nil
if !header1IsNil && !header2IsNil {
headState, err := s.cfg.Chain.HeadState(ctx)
headState, err := s.cfg.chain.HeadState(ctx)
if err != nil {
return err
}
if err := s.cfg.SlashingPool.InsertProposerSlashing(ctx, headState, pSlashing); err != nil {
if err := s.cfg.slashingPool.InsertProposerSlashing(ctx, headState, pSlashing); err != nil {
return errors.Wrap(err, "could not insert proposer slashing into pool")
}
s.setProposerSlashingIndexSeen(pSlashing.Header_1.Header.ProposerIndex)

View File

@@ -20,5 +20,5 @@ func (s *Service) syncCommitteeMessageSubscriber(_ context.Context, msg proto.Me
return errors.New("nil sync committee message")
}
return s.cfg.SyncCommsPool.SaveSyncCommitteeMessage(m)
return s.cfg.syncCommsPool.SaveSyncCommitteeMessage(m)
}

View File

@@ -22,5 +22,5 @@ func (s *Service) syncContributionAndProofSubscriber(_ context.Context, msg prot
return errors.New("nil contribution")
}
return s.cfg.SyncCommsPool.SaveSyncCommitteeContribution(sContr.Message.Contribution)
return s.cfg.syncCommsPool.SaveSyncCommitteeContribution(sContr.Message.Contribution)
}

View File

@@ -40,10 +40,10 @@ func TestSubscribe_ReceivesValidMessage(t *testing.T) {
p2pService := p2ptest.NewTestP2P(t)
r := Service{
ctx: context.Background(),
cfg: &Config{
P2P: p2pService,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: &mockChain.ChainService{
cfg: &config{
p2p: p2pService,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now(),
},
@@ -80,10 +80,10 @@ func TestSubscribe_UnsubscribeTopic(t *testing.T) {
p2pService := p2ptest.NewTestP2P(t)
r := Service{
ctx: context.Background(),
cfg: &Config{
P2P: p2pService,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: &mockChain.ChainService{
cfg: &config{
p2p: p2pService,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now(),
},
@@ -123,12 +123,12 @@ func TestSubscribe_ReceivesAttesterSlashing(t *testing.T) {
}
r := Service{
ctx: ctx,
cfg: &Config{
P2P: p2pService,
InitialSync: &mockSync.Sync{IsSyncing: false},
SlashingPool: slashings.NewPool(),
Chain: chainService,
DB: d,
cfg: &config{
p2p: p2pService,
initialSync: &mockSync.Sync{IsSyncing: false},
slashingPool: slashings.NewPool(),
chain: chainService,
beaconDB: d,
},
seenAttesterSlashingCache: make(map[uint64]bool),
chainStarted: abool.New(),
@@ -156,14 +156,14 @@ func TestSubscribe_ReceivesAttesterSlashing(t *testing.T) {
1, /* validator index */
)
require.NoError(t, err, "Error generating attester slashing")
err = r.cfg.DB.SaveState(ctx, beaconState, bytesutil.ToBytes32(attesterSlashing.Attestation_1.Data.BeaconBlockRoot))
err = r.cfg.beaconDB.SaveState(ctx, beaconState, bytesutil.ToBytes32(attesterSlashing.Attestation_1.Data.BeaconBlockRoot))
require.NoError(t, err)
p2pService.ReceivePubSub(topic, attesterSlashing)
if util.WaitTimeout(&wg, time.Second) {
t.Fatal("Did not receive PubSub in 1 second")
}
as := r.cfg.SlashingPool.PendingAttesterSlashings(ctx, beaconState, false /*noLimit*/)
as := r.cfg.slashingPool.PendingAttesterSlashings(ctx, beaconState, false /*noLimit*/)
assert.Equal(t, 1, len(as), "Expected attester slashing")
}
@@ -177,12 +177,12 @@ func TestSubscribe_ReceivesProposerSlashing(t *testing.T) {
d := db.SetupDB(t)
r := Service{
ctx: ctx,
cfg: &Config{
P2P: p2pService,
InitialSync: &mockSync.Sync{IsSyncing: false},
SlashingPool: slashings.NewPool(),
Chain: chainService,
DB: d,
cfg: &config{
p2p: p2pService,
initialSync: &mockSync.Sync{IsSyncing: false},
slashingPool: slashings.NewPool(),
chain: chainService,
beaconDB: d,
},
seenProposerSlashingCache: lruwrpr.New(10),
chainStarted: abool.New(),
@@ -216,7 +216,7 @@ func TestSubscribe_ReceivesProposerSlashing(t *testing.T) {
if util.WaitTimeout(&wg, time.Second) {
t.Fatal("Did not receive PubSub in 1 second")
}
ps := r.cfg.SlashingPool.PendingProposerSlashings(ctx, beaconState, false /*noLimit*/)
ps := r.cfg.slashingPool.PendingProposerSlashings(ctx, beaconState, false /*noLimit*/)
assert.Equal(t, 1, len(ps), "Expected proposer slashing")
}
@@ -224,12 +224,12 @@ func TestSubscribe_HandlesPanic(t *testing.T) {
p := p2ptest.NewTestP2P(t)
r := Service{
ctx: context.Background(),
cfg: &Config{
Chain: &mockChain.ChainService{
cfg: &config{
chain: &mockChain.ChainService{
Genesis: time.Now(),
ValidatorsRoot: [32]byte{'A'},
},
P2P: p,
p2p: p,
},
subHandler: newSubTopicHandler(),
chainStarted: abool.New(),
@@ -259,12 +259,12 @@ func TestRevalidateSubscription_CorrectlyFormatsTopic(t *testing.T) {
hook := logTest.NewGlobal()
r := Service{
ctx: context.Background(),
cfg: &Config{
Chain: &mockChain.ChainService{
cfg: &config{
chain: &mockChain.ChainService{
Genesis: time.Now(),
ValidatorsRoot: [32]byte{'A'},
},
P2P: p,
p2p: p,
},
chainStarted: abool.New(),
subHandler: newSubTopicHandler(),
@@ -275,18 +275,18 @@ func TestRevalidateSubscription_CorrectlyFormatsTopic(t *testing.T) {
defaultTopic := "/eth2/testing/%#x/committee%d"
// committee index 1
fullTopic := fmt.Sprintf(defaultTopic, digest, 1) + r.cfg.P2P.Encoding().ProtocolSuffix()
fullTopic := fmt.Sprintf(defaultTopic, digest, 1) + r.cfg.p2p.Encoding().ProtocolSuffix()
_, topVal := r.wrapAndReportValidation(fullTopic, r.noopValidator)
require.NoError(t, r.cfg.P2P.PubSub().RegisterTopicValidator(fullTopic, topVal))
subscriptions[1], err = r.cfg.P2P.SubscribeToTopic(fullTopic)
require.NoError(t, r.cfg.p2p.PubSub().RegisterTopicValidator(fullTopic, topVal))
subscriptions[1], err = r.cfg.p2p.SubscribeToTopic(fullTopic)
require.NoError(t, err)
// committee index 2
fullTopic = fmt.Sprintf(defaultTopic, digest, 2) + r.cfg.P2P.Encoding().ProtocolSuffix()
fullTopic = fmt.Sprintf(defaultTopic, digest, 2) + r.cfg.p2p.Encoding().ProtocolSuffix()
_, topVal = r.wrapAndReportValidation(fullTopic, r.noopValidator)
err = r.cfg.P2P.PubSub().RegisterTopicValidator(fullTopic, topVal)
err = r.cfg.p2p.PubSub().RegisterTopicValidator(fullTopic, topVal)
require.NoError(t, err)
subscriptions[2], err = r.cfg.P2P.SubscribeToTopic(fullTopic)
subscriptions[2], err = r.cfg.p2p.SubscribeToTopic(fullTopic)
require.NoError(t, err)
r.reValidateSubscriptions(subscriptions, []uint64{2}, defaultTopic, digest)
@@ -298,12 +298,12 @@ func TestStaticSubnets(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
r := Service{
ctx: ctx,
cfg: &Config{
Chain: &mockChain.ChainService{
cfg: &config{
chain: &mockChain.ChainService{
Genesis: time.Now(),
ValidatorsRoot: [32]byte{'A'},
},
P2P: p,
p2p: p,
},
chainStarted: abool.New(),
subHandler: newSubTopicHandler(),
@@ -315,7 +315,7 @@ func TestStaticSubnets(t *testing.T) {
// no-op
return nil
}, d)
topics := r.cfg.P2P.PubSub().GetTopics()
topics := r.cfg.p2p.PubSub().GetTopics()
if uint64(len(topics)) != params.BeaconNetworkConfig().AttestationSubnetCount {
t.Errorf("Wanted the number of subnet topics registered to be %d but got %d", params.BeaconNetworkConfig().AttestationSubnetCount, len(topics))
}
@@ -421,8 +421,8 @@ func Test_wrapAndReportValidation(t *testing.T) {
chainStarted.SetTo(tt.args.chainstarted)
s := &Service{
chainStarted: chainStarted,
cfg: &Config{
Chain: mChain,
cfg: &config{
chain: mChain,
},
subHandler: newSubTopicHandler(),
}
@@ -446,13 +446,13 @@ func TestFilterSubnetPeers(t *testing.T) {
currSlot := types.Slot(100)
r := Service{
ctx: ctx,
cfg: &Config{
Chain: &mockChain.ChainService{
cfg: &config{
chain: &mockChain.ChainService{
Genesis: time.Now(),
ValidatorsRoot: [32]byte{'A'},
Slot: &currSlot,
},
P2P: p,
p2p: p,
},
chainStarted: abool.New(),
subHandler: newSubTopicHandler(),
@@ -461,7 +461,7 @@ func TestFilterSubnetPeers(t *testing.T) {
defer cache.SubnetIDs.EmptyAllCaches()
digest, err := r.currentForkDigest()
assert.NoError(t, err)
defaultTopic := "/eth2/%x/beacon_attestation_%d" + r.cfg.P2P.Encoding().ProtocolSuffix()
defaultTopic := "/eth2/%x/beacon_attestation_%d" + r.cfg.p2p.Encoding().ProtocolSuffix()
subnet10 := r.addDigestAndIndexToTopic(defaultTopic, digest, 10)
cache.SubnetIDs.AddAggregatorSubnetID(currSlot, 10)
@@ -508,13 +508,13 @@ func TestSubscribeWithSyncSubnets_StaticOK(t *testing.T) {
currSlot := types.Slot(100)
r := Service{
ctx: ctx,
cfg: &Config{
Chain: &mockChain.ChainService{
cfg: &config{
chain: &mockChain.ChainService{
Genesis: time.Now(),
ValidatorsRoot: [32]byte{'A'},
Slot: &currSlot,
},
P2P: p,
p2p: p,
},
chainStarted: abool.New(),
subHandler: newSubTopicHandler(),
@@ -524,7 +524,7 @@ func TestSubscribeWithSyncSubnets_StaticOK(t *testing.T) {
digest, err := r.currentForkDigest()
assert.NoError(t, err)
r.subscribeStaticWithSyncSubnets(p2p.SyncCommitteeSubnetTopicFormat, nil, nil, digest)
assert.Equal(t, int(params.BeaconConfig().SyncCommitteeSubnetCount), len(r.cfg.P2P.PubSub().GetTopics()))
assert.Equal(t, int(params.BeaconConfig().SyncCommitteeSubnetCount), len(r.cfg.p2p.PubSub().GetTopics()))
cancel()
}
@@ -538,35 +538,35 @@ func TestSubscribeWithSyncSubnets_DynamicOK(t *testing.T) {
currSlot := types.Slot(100)
r := Service{
ctx: ctx,
cfg: &Config{
Chain: &mockChain.ChainService{
cfg: &config{
chain: &mockChain.ChainService{
Genesis: time.Now(),
ValidatorsRoot: [32]byte{'A'},
Slot: &currSlot,
},
P2P: p,
p2p: p,
},
chainStarted: abool.New(),
subHandler: newSubTopicHandler(),
}
// Empty cache at the end of the test.
defer cache.SyncSubnetIDs.EmptyAllCaches()
slot := r.cfg.Chain.CurrentSlot()
slot := r.cfg.chain.CurrentSlot()
currEpoch := slots.ToEpoch(slot)
cache.SyncSubnetIDs.AddSyncCommitteeSubnets([]byte("pubkey"), currEpoch, []uint64{0, 1}, 10*time.Second)
digest, err := r.currentForkDigest()
assert.NoError(t, err)
r.subscribeDynamicWithSyncSubnets(p2p.SyncCommitteeSubnetTopicFormat, nil, nil, digest)
time.Sleep(2 * time.Second)
assert.Equal(t, 2, len(r.cfg.P2P.PubSub().GetTopics()))
assert.Equal(t, 2, len(r.cfg.p2p.PubSub().GetTopics()))
topicMap := map[string]bool{}
for _, t := range r.cfg.P2P.PubSub().GetTopics() {
for _, t := range r.cfg.p2p.PubSub().GetTopics() {
topicMap[t] = true
}
firstSub := fmt.Sprintf(p2p.SyncCommitteeSubnetTopicFormat, digest, 0) + r.cfg.P2P.Encoding().ProtocolSuffix()
firstSub := fmt.Sprintf(p2p.SyncCommitteeSubnetTopicFormat, digest, 0) + r.cfg.p2p.Encoding().ProtocolSuffix()
assert.Equal(t, true, topicMap[firstSub])
secondSub := fmt.Sprintf(p2p.SyncCommitteeSubnetTopicFormat, digest, 1) + r.cfg.P2P.Encoding().ProtocolSuffix()
secondSub := fmt.Sprintf(p2p.SyncCommitteeSubnetTopicFormat, digest, 1) + r.cfg.p2p.Encoding().ProtocolSuffix()
assert.Equal(t, true, topicMap[secondSub])
cancel()
}
@@ -583,28 +583,28 @@ func TestSubscribeWithSyncSubnets_StaticSwitchFork(t *testing.T) {
currSlot := types.Slot(100)
r := Service{
ctx: ctx,
cfg: &Config{
Chain: &mockChain.ChainService{
cfg: &config{
chain: &mockChain.ChainService{
Genesis: time.Now().Add(-time.Duration(uint64(params.BeaconConfig().SlotsPerEpoch)*params.BeaconConfig().SecondsPerSlot) * time.Second),
ValidatorsRoot: [32]byte{'A'},
Slot: &currSlot,
},
P2P: p,
p2p: p,
},
chainStarted: abool.New(),
subHandler: newSubTopicHandler(),
}
// Empty cache at the end of the test.
defer cache.SyncSubnetIDs.EmptyAllCaches()
genRoot := r.cfg.Chain.GenesisValidatorRoot()
genRoot := r.cfg.chain.GenesisValidatorRoot()
digest, err := signing.ComputeForkDigest(params.BeaconConfig().GenesisForkVersion, genRoot[:])
assert.NoError(t, err)
r.subscribeStaticWithSyncSubnets(p2p.SyncCommitteeSubnetTopicFormat, nil, nil, digest)
assert.Equal(t, int(params.BeaconConfig().SyncCommitteeSubnetCount), len(r.cfg.P2P.PubSub().GetTopics()))
assert.Equal(t, int(params.BeaconConfig().SyncCommitteeSubnetCount), len(r.cfg.p2p.PubSub().GetTopics()))
// Expect that all old topics will be unsubscribed.
time.Sleep(2 * time.Second)
assert.Equal(t, 0, len(r.cfg.P2P.PubSub().GetTopics()))
assert.Equal(t, 0, len(r.cfg.p2p.PubSub().GetTopics()))
cancel()
}
@@ -622,13 +622,13 @@ func TestSubscribeWithSyncSubnets_DynamicSwitchFork(t *testing.T) {
currSlot := types.Slot(100)
r := Service{
ctx: ctx,
cfg: &Config{
Chain: &mockChain.ChainService{
cfg: &config{
chain: &mockChain.ChainService{
Genesis: time.Now().Add(-time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Second),
ValidatorsRoot: [32]byte{'A'},
Slot: &currSlot,
},
P2P: p,
p2p: p,
},
chainStarted: abool.New(),
subHandler: newSubTopicHandler(),
@@ -636,26 +636,26 @@ func TestSubscribeWithSyncSubnets_DynamicSwitchFork(t *testing.T) {
// Empty cache at the end of the test.
defer cache.SyncSubnetIDs.EmptyAllCaches()
cache.SyncSubnetIDs.AddSyncCommitteeSubnets([]byte("pubkey"), 0, []uint64{0, 1}, 10*time.Second)
genRoot := r.cfg.Chain.GenesisValidatorRoot()
genRoot := r.cfg.chain.GenesisValidatorRoot()
digest, err := signing.ComputeForkDigest(params.BeaconConfig().GenesisForkVersion, genRoot[:])
assert.NoError(t, err)
r.subscribeDynamicWithSyncSubnets(p2p.SyncCommitteeSubnetTopicFormat, nil, nil, digest)
time.Sleep(2 * time.Second)
assert.Equal(t, 2, len(r.cfg.P2P.PubSub().GetTopics()))
assert.Equal(t, 2, len(r.cfg.p2p.PubSub().GetTopics()))
topicMap := map[string]bool{}
for _, t := range r.cfg.P2P.PubSub().GetTopics() {
for _, t := range r.cfg.p2p.PubSub().GetTopics() {
topicMap[t] = true
}
firstSub := fmt.Sprintf(p2p.SyncCommitteeSubnetTopicFormat, digest, 0) + r.cfg.P2P.Encoding().ProtocolSuffix()
firstSub := fmt.Sprintf(p2p.SyncCommitteeSubnetTopicFormat, digest, 0) + r.cfg.p2p.Encoding().ProtocolSuffix()
assert.Equal(t, true, topicMap[firstSub])
secondSub := fmt.Sprintf(p2p.SyncCommitteeSubnetTopicFormat, digest, 1) + r.cfg.P2P.Encoding().ProtocolSuffix()
secondSub := fmt.Sprintf(p2p.SyncCommitteeSubnetTopicFormat, digest, 1) + r.cfg.p2p.Encoding().ProtocolSuffix()
assert.Equal(t, true, topicMap[secondSub])
// Expect that all old topics will be unsubscribed.
time.Sleep(2 * time.Second)
assert.Equal(t, 0, len(r.cfg.P2P.PubSub().GetTopics()))
assert.Equal(t, 0, len(r.cfg.p2p.PubSub().GetTopics()))
cancel()
}

View File

@@ -28,7 +28,7 @@ import (
// validateAggregateAndProof verifies the aggregated signature and the selection proof is valid before forwarding to the
// network and downstream services.
func (s *Service) validateAggregateAndProof(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
if pid == s.cfg.P2P.PeerID() {
if pid == s.cfg.p2p.PeerID() {
return pubsub.ValidationAccept, nil
}
@@ -37,7 +37,7 @@ func (s *Service) validateAggregateAndProof(ctx context.Context, pid peer.ID, ms
// To process the following it requires the recent blocks to be present in the database, so we'll skip
// validating or processing aggregated attestations until fully synced.
if s.cfg.InitialSync.Syncing() {
if s.cfg.initialSync.Syncing() {
return pubsub.ValidationIgnore, nil
}
@@ -63,7 +63,7 @@ func (s *Service) validateAggregateAndProof(ctx context.Context, pid peer.ID, ms
// Broadcast the aggregated attestation on a feed to notify other services in the beacon node
// of a received aggregated attestation.
s.cfg.AttestationNotifier.OperationFeed().Send(&feed.Event{
s.cfg.attestationNotifier.OperationFeed().Send(&feed.Event{
Type: operation.AggregatedAttReceived,
Data: &operation.AggregatedAttReceivedData{
Attestation: m.Message,
@@ -76,7 +76,7 @@ func (s *Service) validateAggregateAndProof(ctx context.Context, pid peer.ID, ms
// Attestation's slot is within ATTESTATION_PROPAGATION_SLOT_RANGE and early attestation
// processing tolerance.
if err := helpers.ValidateAttestationTime(m.Message.Aggregate.Data.Slot, s.cfg.Chain.GenesisTime(),
if err := helpers.ValidateAttestationTime(m.Message.Aggregate.Data.Slot, s.cfg.chain.GenesisTime(),
earlyAttestationProcessingTolerance); err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationIgnore, err
@@ -117,18 +117,18 @@ func (s *Service) validateAggregatedAtt(ctx context.Context, signed *ethpb.Signe
// This verification is not in the spec, however we guard against it as it opens us up
// to weird edge cases during verification. The attestation technically could be used to add value to a block,
// but it's invalid in the spirit of the protocol. Here we choose safety over profit.
if err := s.cfg.Chain.VerifyLmdFfgConsistency(ctx, signed.Message.Aggregate); err != nil {
if err := s.cfg.chain.VerifyLmdFfgConsistency(ctx, signed.Message.Aggregate); err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationReject, err
}
// Verify current finalized checkpoint is an ancestor of the block defined by the attestation's beacon block root.
if err := s.cfg.Chain.VerifyFinalizedConsistency(ctx, signed.Message.Aggregate.Data.BeaconBlockRoot); err != nil {
if err := s.cfg.chain.VerifyFinalizedConsistency(ctx, signed.Message.Aggregate.Data.BeaconBlockRoot); err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationReject, err
}
bs, err := s.cfg.Chain.AttestationTargetState(ctx, signed.Message.Aggregate.Data.Target)
bs, err := s.cfg.chain.AttestationTargetState(ctx, signed.Message.Aggregate.Data.Target)
if err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationIgnore, err
@@ -198,7 +198,7 @@ func (s *Service) validateAggregatedAtt(ctx context.Context, signed *ethpb.Signe
func (s *Service) validateBlockInAttestation(ctx context.Context, satt *ethpb.SignedAggregateAttestationAndProof) bool {
a := satt.Message
// Verify the block being voted and the processed state is in DB. The block should have passed validation if it's in the DB.
// Verify the block being voted and the processed state is in beaconDB. The block should have passed validation if it's in the beaconDB.
blockRoot := bytesutil.ToBytes32(a.Aggregate.Data.BeaconBlockRoot)
if !s.hasBlockAndState(ctx, blockRoot) {
// A node doesn't have the block, it'll request from peer while saving the pending attestation to a queue.

View File

@@ -114,12 +114,12 @@ func TestValidateAggregateAndProof_NoBlock(t *testing.T) {
c := lruwrpr.New(10)
r := &Service{
cfg: &Config{
P2P: p,
DB: db,
InitialSync: &mockSync.Sync{IsSyncing: false},
AttPool: attestations.NewPool(),
Chain: &mock.ChainService{},
cfg: &config{
p2p: p,
beaconDB: db,
initialSync: &mockSync.Sync{IsSyncing: false},
attPool: attestations.NewPool(),
chain: &mock.ChainService{},
},
blkRootToPendingAtts: make(map[[32]byte][]*ethpb.SignedAggregateAttestationAndProof),
seenAggregatedAttestationCache: c,
@@ -181,16 +181,16 @@ func TestValidateAggregateAndProof_NotWithinSlotRange(t *testing.T) {
require.NoError(t, beaconState.SetGenesisTime(uint64(time.Now().Unix())))
r := &Service{
cfg: &Config{
P2P: p,
DB: db,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: &mock.ChainService{
cfg: &config{
p2p: p,
beaconDB: db,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: &mock.ChainService{
Genesis: time.Now(),
State: beaconState,
},
AttPool: attestations.NewPool(),
AttestationNotifier: (&mock.ChainService{}).OperationNotifier(),
attPool: attestations.NewPool(),
attestationNotifier: (&mock.ChainService{}).OperationNotifier(),
},
seenAggregatedAttestationCache: lruwrpr.New(10),
}
@@ -264,14 +264,14 @@ func TestValidateAggregateAndProof_ExistedInPool(t *testing.T) {
require.NoError(t, beaconState.SetGenesisTime(uint64(time.Now().Unix())))
r := &Service{
cfg: &Config{
AttPool: attestations.NewPool(),
P2P: p,
DB: db,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: &mock.ChainService{Genesis: time.Now(),
cfg: &config{
attPool: attestations.NewPool(),
p2p: p,
beaconDB: db,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: &mock.ChainService{Genesis: time.Now(),
State: beaconState},
AttestationNotifier: (&mock.ChainService{}).OperationNotifier(),
attestationNotifier: (&mock.ChainService{}).OperationNotifier(),
},
seenAggregatedAttestationCache: lruwrpr.New(10),
blkRootToPendingAtts: make(map[[32]byte][]*ethpb.SignedAggregateAttestationAndProof),
@@ -290,7 +290,7 @@ func TestValidateAggregateAndProof_ExistedInPool(t *testing.T) {
},
}
require.NoError(t, r.cfg.AttPool.SaveBlockAttestation(att))
require.NoError(t, r.cfg.attPool.SaveBlockAttestation(att))
if res, err := r.validateAggregateAndProof(context.Background(), "", msg); res == pubsub.ValidationAccept {
_ = err
t.Error("Expected validate to fail")
@@ -354,19 +354,19 @@ func TestValidateAggregateAndProof_CanValidate(t *testing.T) {
require.NoError(t, beaconState.SetGenesisTime(uint64(time.Now().Unix())))
r := &Service{
cfg: &Config{
P2P: p,
DB: db,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: &mock.ChainService{Genesis: time.Now().Add(-oneEpoch()),
cfg: &config{
p2p: p,
beaconDB: db,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: &mock.ChainService{Genesis: time.Now().Add(-oneEpoch()),
State: beaconState,
ValidAttestation: true,
FinalizedCheckPoint: &ethpb.Checkpoint{
Epoch: 0,
Root: att.Data.BeaconBlockRoot,
}},
AttPool: attestations.NewPool(),
AttestationNotifier: (&mock.ChainService{}).OperationNotifier(),
attPool: attestations.NewPool(),
attestationNotifier: (&mock.ChainService{}).OperationNotifier(),
},
seenAggregatedAttestationCache: lruwrpr.New(10),
}
@@ -448,11 +448,11 @@ func TestVerifyIndexInCommittee_SeenAggregatorEpoch(t *testing.T) {
require.NoError(t, beaconState.SetGenesisTime(uint64(time.Now().Unix())))
r := &Service{
cfg: &Config{
P2P: p,
DB: db,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: &mock.ChainService{Genesis: time.Now().Add(-oneEpoch()),
cfg: &config{
p2p: p,
beaconDB: db,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: &mock.ChainService{Genesis: time.Now().Add(-oneEpoch()),
ValidatorsRoot: [32]byte{'A'},
State: beaconState,
ValidAttestation: true,
@@ -461,8 +461,8 @@ func TestVerifyIndexInCommittee_SeenAggregatorEpoch(t *testing.T) {
Root: signedAggregateAndProof.Message.Aggregate.Data.BeaconBlockRoot,
}},
AttPool: attestations.NewPool(),
AttestationNotifier: (&mock.ChainService{}).OperationNotifier(),
attPool: attestations.NewPool(),
attestationNotifier: (&mock.ChainService{}).OperationNotifier(),
},
seenAggregatedAttestationCache: lruwrpr.New(10),
}
@@ -562,18 +562,18 @@ func TestValidateAggregateAndProof_BadBlock(t *testing.T) {
require.NoError(t, beaconState.SetGenesisTime(uint64(time.Now().Unix())))
r := &Service{
cfg: &Config{
P2P: p,
DB: db,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: &mock.ChainService{Genesis: time.Now(),
cfg: &config{
p2p: p,
beaconDB: db,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: &mock.ChainService{Genesis: time.Now(),
State: beaconState,
ValidAttestation: true,
FinalizedCheckPoint: &ethpb.Checkpoint{
Epoch: 0,
}},
AttPool: attestations.NewPool(),
AttestationNotifier: (&mock.ChainService{}).OperationNotifier(),
attPool: attestations.NewPool(),
attestationNotifier: (&mock.ChainService{}).OperationNotifier(),
},
seenAggregatedAttestationCache: lruwrpr.New(10),
}
@@ -652,19 +652,19 @@ func TestValidateAggregateAndProof_RejectWhenAttEpochDoesntEqualTargetEpoch(t *t
require.NoError(t, beaconState.SetGenesisTime(uint64(time.Now().Unix())))
r := &Service{
cfg: &Config{
P2P: p,
DB: db,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: &mock.ChainService{Genesis: time.Now(),
cfg: &config{
p2p: p,
beaconDB: db,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: &mock.ChainService{Genesis: time.Now(),
State: beaconState,
ValidAttestation: true,
FinalizedCheckPoint: &ethpb.Checkpoint{
Epoch: 0,
Root: att.Data.BeaconBlockRoot,
}},
AttPool: attestations.NewPool(),
AttestationNotifier: (&mock.ChainService{}).OperationNotifier(),
attPool: attestations.NewPool(),
attestationNotifier: (&mock.ChainService{}).OperationNotifier(),
},
seenAggregatedAttestationCache: lruwrpr.New(10),
}

View File

@@ -17,12 +17,12 @@ import (
func (s *Service) validateAttesterSlashing(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() {
if pid == s.cfg.p2p.PeerID() {
return pubsub.ValidationAccept, nil
}
// The head state will be too far away to validate any slashing.
if s.cfg.InitialSync.Syncing() {
if s.cfg.initialSync.Syncing() {
return pubsub.ValidationIgnore, nil
}
@@ -46,7 +46,7 @@ func (s *Service) validateAttesterSlashing(ctx context.Context, pid peer.ID, msg
return pubsub.ValidationIgnore, nil
}
headState, err := s.cfg.Chain.HeadState(ctx)
headState, err := s.cfg.chain.HeadState(ctx)
if err != nil {
return pubsub.ValidationIgnore, err
}

View File

@@ -80,10 +80,10 @@ func TestValidateAttesterSlashing_ValidSlashing(t *testing.T) {
slashing, s := setupValidAttesterSlashing(t)
r := &Service{
cfg: &Config{
P2P: p,
Chain: &mock.ChainService{State: s, Genesis: time.Now()},
InitialSync: &mockSync.Sync{IsSyncing: false},
cfg: &config{
p2p: p,
chain: &mock.ChainService{State: s, Genesis: time.Now()},
initialSync: &mockSync.Sync{IsSyncing: false},
},
seenAttesterSlashingCache: make(map[uint64]bool),
subHandler: newSubTopicHandler(),
@@ -116,10 +116,10 @@ func TestValidateAttesterSlashing_CanFilter(t *testing.T) {
ctx := context.Background()
r := &Service{
cfg: &Config{
P2P: p,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: &mock.ChainService{Genesis: time.Now()},
cfg: &config{
p2p: p,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: &mock.ChainService{Genesis: time.Now()},
},
seenAttesterSlashingCache: make(map[uint64]bool),
subHandler: newSubTopicHandler(),
@@ -185,10 +185,10 @@ func TestValidateAttesterSlashing_ContextTimeout(t *testing.T) {
defer cancel()
r := &Service{
cfg: &Config{
P2P: p,
Chain: &mock.ChainService{State: state},
InitialSync: &mockSync.Sync{IsSyncing: false},
cfg: &config{
p2p: p,
chain: &mock.ChainService{State: state},
initialSync: &mockSync.Sync{IsSyncing: false},
},
seenAttesterSlashingCache: make(map[uint64]bool),
}
@@ -217,10 +217,10 @@ func TestValidateAttesterSlashing_Syncing(t *testing.T) {
slashing, s := setupValidAttesterSlashing(t)
r := &Service{
cfg: &Config{
P2P: p,
Chain: &mock.ChainService{State: s},
InitialSync: &mockSync.Sync{IsSyncing: true},
cfg: &config{
p2p: p,
chain: &mock.ChainService{State: s},
initialSync: &mockSync.Sync{IsSyncing: true},
},
}

View File

@@ -32,12 +32,12 @@ import (
// - attestation.data.slot is within the last ATTESTATION_PROPAGATION_SLOT_RANGE slots (attestation.data.slot + ATTESTATION_PROPAGATION_SLOT_RANGE >= current_slot >= attestation.data.slot).
// - The signature of attestation is valid.
func (s *Service) validateCommitteeIndexBeaconAttestation(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
if pid == s.cfg.P2P.PeerID() {
if pid == s.cfg.p2p.PeerID() {
return pubsub.ValidationAccept, nil
}
// Attestation processing requires the target block to be present in the database, so we'll skip
// validating or processing attestations until fully synced.
if s.cfg.InitialSync.Syncing() {
if s.cfg.initialSync.Syncing() {
return pubsub.ValidationIgnore, nil
}
ctx, span := trace.StartSpan(ctx, "sync.validateCommitteeIndexBeaconAttestation")
@@ -67,7 +67,7 @@ func (s *Service) validateCommitteeIndexBeaconAttestation(ctx context.Context, p
}
// Broadcast the unaggregated attestation on a feed to notify other services in the beacon node
// of a received unaggregated attestation.
s.cfg.AttestationNotifier.OperationFeed().Send(&feed.Event{
s.cfg.attestationNotifier.OperationFeed().Send(&feed.Event{
Type: operation.UnaggregatedAttReceived,
Data: &operation.UnAggregatedAttReceivedData{
Attestation: att,
@@ -76,7 +76,7 @@ func (s *Service) validateCommitteeIndexBeaconAttestation(ctx context.Context, p
// Attestation's slot is within ATTESTATION_PROPAGATION_SLOT_RANGE and early attestation
// processing tolerance.
if err := helpers.ValidateAttestationTime(att.Data.Slot, s.cfg.Chain.GenesisTime(),
if err := helpers.ValidateAttestationTime(att.Data.Slot, s.cfg.chain.GenesisTime(),
earlyAttestationProcessingTolerance); err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationIgnore, err
@@ -92,7 +92,7 @@ func (s *Service) validateCommitteeIndexBeaconAttestation(ctx context.Context, p
// Using a different context to prevent timeouts as this operation can be expensive
// and we want to avoid affecting the critical code path.
ctx := context.TODO()
preState, err := s.cfg.Chain.AttestationTargetState(ctx, att.Data.Target)
preState, err := s.cfg.chain.AttestationTargetState(ctx, att.Data.Target)
if err != nil {
log.WithError(err).Error("Could not retrieve pre state")
tracing.AnnotateError(span, err)
@@ -110,7 +110,7 @@ func (s *Service) validateCommitteeIndexBeaconAttestation(ctx context.Context, p
tracing.AnnotateError(span, err)
return
}
s.cfg.SlasherAttestationsFeed.Send(indexedAtt)
s.cfg.slasherAttestationsFeed.Send(indexedAtt)
}()
}
@@ -126,7 +126,7 @@ func (s *Service) validateCommitteeIndexBeaconAttestation(ctx context.Context, p
return pubsub.ValidationReject, errors.New("attestation data references bad block root")
}
// Verify the block being voted and the processed state is in DB and the block has passed validation if it's in the DB.
// Verify the block being voted and the processed state is in beaconDB and the block has passed validation if it's in the beaconDB.
blockRoot := bytesutil.ToBytes32(att.Data.BeaconBlockRoot)
if !s.hasBlockAndState(ctx, blockRoot) {
// A node doesn't have the block, it'll request from peer while saving the pending attestation to a queue.
@@ -134,16 +134,16 @@ func (s *Service) validateCommitteeIndexBeaconAttestation(ctx context.Context, p
return pubsub.ValidationIgnore, nil
}
if err := s.cfg.Chain.VerifyFinalizedConsistency(ctx, att.Data.BeaconBlockRoot); err != nil {
if err := s.cfg.chain.VerifyFinalizedConsistency(ctx, att.Data.BeaconBlockRoot); err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationReject, err
}
if err := s.cfg.Chain.VerifyLmdFfgConsistency(ctx, att); err != nil {
if err := s.cfg.chain.VerifyLmdFfgConsistency(ctx, att); err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationReject, err
}
preState, err := s.cfg.Chain.AttestationTargetState(ctx, att.Data.Target)
preState, err := s.cfg.chain.AttestationTargetState(ctx, att.Data.Target)
if err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationIgnore, err
@@ -212,7 +212,7 @@ func (s *Service) validateUnaggregatedAttWithState(ctx context.Context, a *eth.A
}
// Attestation must be unaggregated and the bit index must exist in the range of committee indices.
// Note: The Ethereum Beacon Chain spec suggests (len(get_attesting_indices(state, attestation.data, attestation.aggregation_bits)) == 1)
// Note: The Ethereum Beacon chain spec suggests (len(get_attesting_indices(state, attestation.data, attestation.aggregation_bits)) == 1)
// however this validation can be achieved without use of get_attesting_indices which is an O(n) lookup.
if a.AggregationBits.Count() != 1 || a.AggregationBits.BitIndices()[0] >= len(committee) {
return pubsub.ValidationReject, errors.New("attestation bitfield is invalid")
@@ -255,8 +255,8 @@ func (s *Service) setSeenCommitteeIndicesSlot(slot types.Slot, committeeID types
// hasBlockAndState returns true if the beacon node knows about a block and associated state in the
// database or cache.
func (s *Service) hasBlockAndState(ctx context.Context, blockRoot [32]byte) bool {
hasStateSummary := s.cfg.DB.HasStateSummary(ctx, blockRoot)
hasState := hasStateSummary || s.cfg.DB.HasState(ctx, blockRoot)
hasBlock := s.cfg.Chain.HasInitSyncBlock(blockRoot) || s.cfg.DB.HasBlock(ctx, blockRoot)
hasStateSummary := s.cfg.beaconDB.HasStateSummary(ctx, blockRoot)
hasState := hasStateSummary || s.cfg.beaconDB.HasState(ctx, blockRoot)
hasBlock := s.cfg.chain.HasInitSyncBlock(blockRoot) || s.cfg.beaconDB.HasBlock(ctx, blockRoot)
return hasState && hasBlock
}

View File

@@ -37,12 +37,12 @@ func TestService_validateCommitteeIndexBeaconAttestation(t *testing.T) {
}
s := &Service{
cfg: &Config{
InitialSync: &mockSync.Sync{IsSyncing: false},
P2P: p,
DB: db,
Chain: chain,
AttestationNotifier: (&mockChain.ChainService{}).OperationNotifier(),
cfg: &config{
initialSync: &mockSync.Sync{IsSyncing: false},
p2p: p,
beaconDB: db,
chain: chain,
attestationNotifier: (&mockChain.ChainService{}).OperationNotifier(),
},
blkRootToPendingAtts: make(map[[32]byte][]*ethpb.SignedAggregateAttestationAndProof),
seenUnAggregatedAttestationCache: lruwrpr.New(10),

View File

@@ -32,12 +32,12 @@ func (s *Service) validateBeaconBlockPubSub(ctx context.Context, pid peer.ID, ms
receivedTime := prysmTime.Now()
// Validation runs on publish (not just subscriptions), so we should approve any message from
// ourselves.
if pid == s.cfg.P2P.PeerID() {
if pid == s.cfg.p2p.PeerID() {
return pubsub.ValidationAccept, nil
}
// We should not attempt to process blocks until fully synced, but propagation is OK.
if s.cfg.InitialSync.Syncing() {
if s.cfg.initialSync.Syncing() {
return pubsub.ValidationIgnore, nil
}
@@ -64,7 +64,7 @@ func (s *Service) validateBeaconBlockPubSub(ctx context.Context, pid peer.ID, ms
// Broadcast the block on a feed to notify other services in the beacon node
// of a received block (even if it does not process correctly through a state transition).
s.cfg.BlockNotifier.BlockFeed().Send(&feed.Event{
s.cfg.blockNotifier.BlockFeed().Send(&feed.Event{
Type: blockfeed.ReceivedBlock,
Data: &blockfeed.ReceivedBlockData{
SignedBlock: blk,
@@ -79,7 +79,7 @@ func (s *Service) validateBeaconBlockPubSub(ctx context.Context, pid peer.ID, ms
if err != nil {
log.WithError(err).WithField("blockSlot", blk.Block().Slot()).Warn("Could not extract block header")
}
s.cfg.SlasherBlockHeadersFeed.Send(blockHeader)
s.cfg.slasherBlockHeadersFeed.Send(blockHeader)
}()
}
@@ -93,7 +93,7 @@ func (s *Service) validateBeaconBlockPubSub(ctx context.Context, pid peer.ID, ms
log.WithError(err).WithField("blockSlot", blk.Block().Slot()).Debug("Ignored block")
return pubsub.ValidationIgnore, nil
}
if s.cfg.DB.HasBlock(ctx, blockRoot) {
if s.cfg.beaconDB.HasBlock(ctx, blockRoot) {
return pubsub.ValidationIgnore, nil
}
// Check if parent is a bad block and then reject the block.
@@ -113,7 +113,7 @@ func (s *Service) validateBeaconBlockPubSub(ctx context.Context, pid peer.ID, ms
// Be lenient in handling early blocks. Instead of discarding blocks arriving later than
// MAXIMUM_GOSSIP_CLOCK_DISPARITY in future, we tolerate blocks arriving at max two slots
// earlier (SECONDS_PER_SLOT * 2 seconds). Queue such blocks and process them at the right slot.
genesisTime := uint64(s.cfg.Chain.GenesisTime().Unix())
genesisTime := uint64(s.cfg.chain.GenesisTime().Unix())
if err := slots.VerifyTime(genesisTime, blk.Block().Slot(), earlyBlockProcessingTolerance); err != nil {
log.WithError(err).WithField("blockSlot", blk.Block().Slot()).Debug("Ignored block")
return pubsub.ValidationIgnore, nil
@@ -125,7 +125,7 @@ func (s *Service) validateBeaconBlockPubSub(ctx context.Context, pid peer.ID, ms
return pubsub.ValidationIgnore, nil
}
startSlot, err := slots.EpochStart(s.cfg.Chain.FinalizedCheckpt().Epoch)
startSlot, err := slots.EpochStart(s.cfg.chain.FinalizedCheckpt().Epoch)
if err != nil {
log.WithError(err).WithField("blockSlot", blk.Block().Slot()).Debug("Ignored block")
return pubsub.ValidationIgnore, nil
@@ -144,12 +144,12 @@ func (s *Service) validateBeaconBlockPubSub(ctx context.Context, pid peer.ID, ms
return pubsub.ValidationIgnore, err
}
s.pendingQueueLock.Unlock()
e := fmt.Errorf("early block, with current slot %d < block slot %d", s.cfg.Chain.CurrentSlot(), blk.Block().Slot())
e := fmt.Errorf("early block, with current slot %d < block slot %d", s.cfg.chain.CurrentSlot(), blk.Block().Slot())
return pubsub.ValidationIgnore, e
}
// Handle block when the parent is unknown.
if !s.cfg.DB.HasBlock(ctx, bytesutil.ToBytes32(blk.Block().ParentRoot())) && !s.cfg.Chain.HasInitSyncBlock(bytesutil.ToBytes32(blk.Block().ParentRoot())) {
if !s.cfg.beaconDB.HasBlock(ctx, bytesutil.ToBytes32(blk.Block().ParentRoot())) && !s.cfg.chain.HasInitSyncBlock(bytesutil.ToBytes32(blk.Block().ParentRoot())) {
s.pendingQueueLock.Lock()
if err := s.insertBlockToPendingQueue(blk.Block().Slot(), blk, blockRoot); err != nil {
s.pendingQueueLock.Unlock()
@@ -183,19 +183,19 @@ func (s *Service) validateBeaconBlock(ctx context.Context, blk block.SignedBeaco
ctx, span := trace.StartSpan(ctx, "sync.validateBeaconBlock")
defer span.End()
if err := s.cfg.Chain.VerifyBlkDescendant(ctx, bytesutil.ToBytes32(blk.Block().ParentRoot())); err != nil {
if err := s.cfg.chain.VerifyBlkDescendant(ctx, bytesutil.ToBytes32(blk.Block().ParentRoot())); err != nil {
s.setBadBlock(ctx, blockRoot)
return err
}
hasStateSummaryDB := s.cfg.DB.HasStateSummary(ctx, bytesutil.ToBytes32(blk.Block().ParentRoot()))
hasStateSummaryDB := s.cfg.beaconDB.HasStateSummary(ctx, bytesutil.ToBytes32(blk.Block().ParentRoot()))
if !hasStateSummaryDB {
_, err := s.cfg.StateGen.RecoverStateSummary(ctx, bytesutil.ToBytes32(blk.Block().ParentRoot()))
_, err := s.cfg.stateGen.RecoverStateSummary(ctx, bytesutil.ToBytes32(blk.Block().ParentRoot()))
if err != nil {
return err
}
}
parentState, err := s.cfg.StateGen.StateByRoot(ctx, bytesutil.ToBytes32(blk.Block().ParentRoot()))
parentState, err := s.cfg.stateGen.StateByRoot(ctx, bytesutil.ToBytes32(blk.Block().ParentRoot()))
if err != nil {
return err
}

View File

@@ -69,13 +69,13 @@ func TestValidateBeaconBlockPubSub_InvalidSignature(t *testing.T) {
Root: make([]byte, 32),
}}
r := &Service{
cfg: &Config{
DB: db,
P2P: p,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
BlockNotifier: chainService.BlockNotifier(),
StateGen: stateGen,
cfg: &config{
beaconDB: db,
p2p: p,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: chainService,
blockNotifier: chainService.BlockNotifier(),
stateGen: stateGen,
},
seenBlockCache: lruwrpr.New(10),
badBlockCache: lruwrpr.New(10),
@@ -109,12 +109,12 @@ func TestValidateBeaconBlockPubSub_BlockAlreadyPresentInDB(t *testing.T) {
chainService := &mock.ChainService{Genesis: time.Now()}
r := &Service{
cfg: &Config{
DB: db,
P2P: p,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
BlockNotifier: chainService.BlockNotifier(),
cfg: &config{
beaconDB: db,
p2p: p,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: chainService,
blockNotifier: chainService.BlockNotifier(),
},
seenBlockCache: lruwrpr.New(10),
badBlockCache: lruwrpr.New(10),
@@ -167,13 +167,13 @@ func TestValidateBeaconBlockPubSub_CanRecoverStateSummary(t *testing.T) {
},
}
r := &Service{
cfg: &Config{
DB: db,
P2P: p,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
BlockNotifier: chainService.BlockNotifier(),
StateGen: stateGen,
cfg: &config{
beaconDB: db,
p2p: p,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: chainService,
blockNotifier: chainService.BlockNotifier(),
stateGen: stateGen,
},
seenBlockCache: lruwrpr.New(10),
badBlockCache: lruwrpr.New(10),
@@ -231,13 +231,13 @@ func TestValidateBeaconBlockPubSub_IsInCache(t *testing.T) {
InitSyncBlockRoots: map[[32]byte]bool{bRoot: true},
}
r := &Service{
cfg: &Config{
DB: db,
P2P: p,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
BlockNotifier: chainService.BlockNotifier(),
StateGen: stateGen,
cfg: &config{
beaconDB: db,
p2p: p,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: chainService,
blockNotifier: chainService.BlockNotifier(),
stateGen: stateGen,
},
seenBlockCache: lruwrpr.New(10),
badBlockCache: lruwrpr.New(10),
@@ -295,13 +295,13 @@ func TestValidateBeaconBlockPubSub_ValidProposerSignature(t *testing.T) {
},
}
r := &Service{
cfg: &Config{
DB: db,
P2P: p,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
BlockNotifier: chainService.BlockNotifier(),
StateGen: stateGen,
cfg: &config{
beaconDB: db,
p2p: p,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: chainService,
blockNotifier: chainService.BlockNotifier(),
stateGen: stateGen,
},
seenBlockCache: lruwrpr.New(10),
badBlockCache: lruwrpr.New(10),
@@ -361,13 +361,13 @@ func TestValidateBeaconBlockPubSub_WithLookahead(t *testing.T) {
Epoch: 0,
}}
r := &Service{
cfg: &Config{
DB: db,
P2P: p,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
BlockNotifier: chainService.BlockNotifier(),
StateGen: stateGen,
cfg: &config{
beaconDB: db,
p2p: p,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: chainService,
blockNotifier: chainService.BlockNotifier(),
stateGen: stateGen,
},
seenBlockCache: lruwrpr.New(10),
badBlockCache: lruwrpr.New(10),
@@ -428,13 +428,13 @@ func TestValidateBeaconBlockPubSub_AdvanceEpochsForState(t *testing.T) {
Epoch: 0,
}}
r := &Service{
cfg: &Config{
DB: db,
P2P: p,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
BlockNotifier: chainService.BlockNotifier(),
StateGen: stateGen,
cfg: &config{
beaconDB: db,
p2p: p,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: chainService,
blockNotifier: chainService.BlockNotifier(),
stateGen: stateGen,
},
seenBlockCache: lruwrpr.New(10),
badBlockCache: lruwrpr.New(10),
@@ -478,12 +478,12 @@ func TestValidateBeaconBlockPubSub_Syncing(t *testing.T) {
Epoch: 0,
}}
r := &Service{
cfg: &Config{
DB: db,
P2P: p,
InitialSync: &mockSync.Sync{IsSyncing: true},
Chain: chainService,
BlockNotifier: chainService.BlockNotifier(),
cfg: &config{
beaconDB: db,
p2p: p,
initialSync: &mockSync.Sync{IsSyncing: true},
chain: chainService,
blockNotifier: chainService.BlockNotifier(),
},
}
@@ -534,13 +534,13 @@ func TestValidateBeaconBlockPubSub_AcceptBlocksFromNearFuture(t *testing.T) {
Root: make([]byte, 32),
}}
r := &Service{
cfg: &Config{
P2P: p,
DB: db,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
BlockNotifier: chainService.BlockNotifier(),
StateGen: stateGen,
cfg: &config{
p2p: p,
beaconDB: db,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: chainService,
blockNotifier: chainService.BlockNotifier(),
stateGen: stateGen,
},
chainStarted: abool.New(),
seenBlockCache: lruwrpr.New(10),
@@ -586,12 +586,12 @@ func TestValidateBeaconBlockPubSub_RejectBlocksFromFuture(t *testing.T) {
chainService := &mock.ChainService{Genesis: time.Now()}
r := &Service{
cfg: &Config{
P2P: p,
DB: db,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
BlockNotifier: chainService.BlockNotifier(),
cfg: &config{
p2p: p,
beaconDB: db,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: chainService,
blockNotifier: chainService.BlockNotifier(),
},
chainStarted: abool.New(),
seenBlockCache: lruwrpr.New(10),
@@ -637,12 +637,12 @@ func TestValidateBeaconBlockPubSub_RejectBlocksFromThePast(t *testing.T) {
},
}
r := &Service{
cfg: &Config{
DB: db,
P2P: p,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
BlockNotifier: chainService.BlockNotifier(),
cfg: &config{
beaconDB: db,
p2p: p,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: chainService,
blockNotifier: chainService.BlockNotifier(),
},
seenBlockCache: lruwrpr.New(10),
badBlockCache: lruwrpr.New(10),
@@ -692,12 +692,12 @@ func TestValidateBeaconBlockPubSub_SeenProposerSlot(t *testing.T) {
},
}
r := &Service{
cfg: &Config{
DB: db,
P2P: p,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
BlockNotifier: chainService.BlockNotifier(),
cfg: &config{
beaconDB: db,
p2p: p,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: chainService,
blockNotifier: chainService.BlockNotifier(),
},
seenBlockCache: lruwrpr.New(10),
badBlockCache: lruwrpr.New(10),
@@ -740,13 +740,13 @@ func TestValidateBeaconBlockPubSub_FilterByFinalizedEpoch(t *testing.T) {
}
r := &Service{
cfg: &Config{
DB: db,
P2P: p,
Chain: chain,
BlockNotifier: chain.BlockNotifier(),
AttPool: attestations.NewPool(),
InitialSync: &mockSync.Sync{IsSyncing: false},
cfg: &config{
beaconDB: db,
p2p: p,
chain: chain,
blockNotifier: chain.BlockNotifier(),
attPool: attestations.NewPool(),
initialSync: &mockSync.Sync{IsSyncing: false},
},
seenBlockCache: lruwrpr.New(10),
badBlockCache: lruwrpr.New(10),
@@ -821,13 +821,13 @@ func TestValidateBeaconBlockPubSub_ParentNotFinalizedDescendant(t *testing.T) {
VerifyBlkDescendantErr: errors.New("not part of finalized chain"),
}
r := &Service{
cfg: &Config{
DB: db,
P2P: p,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
BlockNotifier: chainService.BlockNotifier(),
StateGen: stateGen,
cfg: &config{
beaconDB: db,
p2p: p,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: chainService,
blockNotifier: chainService.BlockNotifier(),
stateGen: stateGen,
},
seenBlockCache: lruwrpr.New(10),
badBlockCache: lruwrpr.New(10),
@@ -886,13 +886,13 @@ func TestValidateBeaconBlockPubSub_InvalidParentBlock(t *testing.T) {
Epoch: 0,
}}
r := &Service{
cfg: &Config{
DB: db,
P2P: p,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
BlockNotifier: chainService.BlockNotifier(),
StateGen: stateGen,
cfg: &config{
beaconDB: db,
p2p: p,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: chainService,
blockNotifier: chainService.BlockNotifier(),
stateGen: stateGen,
},
seenBlockCache: lruwrpr.New(10),
badBlockCache: lruwrpr.New(10),
@@ -987,13 +987,13 @@ func TestValidateBeaconBlockPubSub_RejectEvilBlocksFromFuture(t *testing.T) {
},
}
r := &Service{
cfg: &Config{
DB: db,
P2P: p,
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
BlockNotifier: chainService.BlockNotifier(),
StateGen: stateGen,
cfg: &config{
beaconDB: db,
p2p: p,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: chainService,
blockNotifier: chainService.BlockNotifier(),
stateGen: stateGen,
},
seenBlockCache: lruwrpr.New(10),
badBlockCache: lruwrpr.New(10),

View File

@@ -17,12 +17,12 @@ import (
func (s *Service) validateProposerSlashing(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() {
if pid == s.cfg.p2p.PeerID() {
return pubsub.ValidationAccept, nil
}
// The head state will be too far away to validate any slashing.
if s.cfg.InitialSync.Syncing() {
if s.cfg.initialSync.Syncing() {
return pubsub.ValidationIgnore, nil
}
@@ -47,7 +47,7 @@ func (s *Service) validateProposerSlashing(ctx context.Context, pid peer.ID, msg
return pubsub.ValidationIgnore, nil
}
headState, err := s.cfg.Chain.HeadState(ctx)
headState, err := s.cfg.chain.HeadState(ctx)
if err != nil {
return pubsub.ValidationIgnore, err
}

View File

@@ -114,10 +114,10 @@ func TestValidateProposerSlashing_ValidSlashing(t *testing.T) {
slashing, s := setupValidProposerSlashing(t)
r := &Service{
cfg: &Config{
P2P: p,
Chain: &mock.ChainService{State: s, Genesis: time.Now()},
InitialSync: &mockSync.Sync{IsSyncing: false},
cfg: &config{
p2p: p,
chain: &mock.ChainService{State: s, Genesis: time.Now()},
initialSync: &mockSync.Sync{IsSyncing: false},
},
seenProposerSlashingCache: lruwrpr.New(10),
}
@@ -156,10 +156,10 @@ func TestValidateProposerSlashing_ContextTimeout(t *testing.T) {
defer cancel()
r := &Service{
cfg: &Config{
P2P: p,
Chain: &mock.ChainService{State: state},
InitialSync: &mockSync.Sync{IsSyncing: false},
cfg: &config{
p2p: p,
chain: &mock.ChainService{State: state},
initialSync: &mockSync.Sync{IsSyncing: false},
},
seenProposerSlashingCache: lruwrpr.New(10),
}
@@ -187,10 +187,10 @@ func TestValidateProposerSlashing_Syncing(t *testing.T) {
slashing, s := setupValidProposerSlashing(t)
r := &Service{
cfg: &Config{
P2P: p,
Chain: &mock.ChainService{State: s},
InitialSync: &mockSync.Sync{IsSyncing: true},
cfg: &config{
p2p: p,
chain: &mock.ChainService{State: s},
initialSync: &mockSync.Sync{IsSyncing: true},
},
}

View File

@@ -49,12 +49,12 @@ func (s *Service) validateSyncCommitteeMessage(
ctx, span := trace.StartSpan(ctx, "sync.validateSyncCommitteeMessage")
defer span.End()
if pid == s.cfg.P2P.PeerID() {
if pid == s.cfg.p2p.PeerID() {
return pubsub.ValidationAccept, nil
}
// Basic validations before proceeding.
if s.cfg.InitialSync.Syncing() {
if s.cfg.initialSync.Syncing() {
return pubsub.ValidationIgnore, nil
}
if msg.Topic == nil {
@@ -72,14 +72,14 @@ func (s *Service) validateSyncCommitteeMessage(
// The message's `slot` is for the current slot (with a MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance).
if err := altair.ValidateSyncMessageTime(
m.Slot,
s.cfg.Chain.GenesisTime(),
s.cfg.chain.GenesisTime(),
params.BeaconNetworkConfig().MaximumGossipClockDisparity,
); err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationIgnore, err
}
committeeIndices, err := s.cfg.Chain.HeadSyncCommitteeIndices(ctx, m.ValidatorIndex, m.Slot)
committeeIndices, err := s.cfg.chain.HeadSyncCommitteeIndices(ctx, m.ValidatorIndex, m.Slot)
if err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationIgnore, err
@@ -212,7 +212,7 @@ func (s *Service) rejectInvalidSyncCommitteeSignature(m *ethpb.SyncCommitteeMess
// Ignore the message if it is not possible to retrieve the signing root.
// For internal errors, the correct behaviour is to ignore rather than reject outright,
// since the failure is locally derived.
d, err := s.cfg.Chain.HeadSyncCommitteeDomain(ctx, m.Slot)
d, err := s.cfg.chain.HeadSyncCommitteeDomain(ctx, m.Slot)
if err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationIgnore, err
@@ -226,7 +226,7 @@ func (s *Service) rejectInvalidSyncCommitteeSignature(m *ethpb.SyncCommitteeMess
// Reject for a validator index that is not found, as we should not remain peered with a node
// that is on such a different fork than our chain.
pubKey, err := s.cfg.Chain.HeadValidatorIndexToPublicKey(ctx, m.ValidatorIndex)
pubKey, err := s.cfg.chain.HeadValidatorIndexToPublicKey(ctx, m.ValidatorIndex)
if err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationReject, err

View File

@@ -55,17 +55,17 @@ func TestService_ValidateSyncCommitteeMessage(t *testing.T) {
}{
{
name: "Is syncing",
svc: NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: true},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
}),
svc: NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: true}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
),
setupSvc: func(s *Service, msg *ethpb.SyncCommitteeMessage, topic string) (*Service, string) {
s.cfg.StateGen = stategen.New(beaconDB)
s.cfg.stateGen = stategen.New(beaconDB)
msg.BlockRoot = headRoot[:]
s.cfg.DB = beaconDB
s.cfg.beaconDB = beaconDB
s.initCaches()
return s, topic
},
@@ -83,17 +83,17 @@ func TestService_ValidateSyncCommitteeMessage(t *testing.T) {
},
{
name: "Bad Topic",
svc: NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
}),
svc: NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: false}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
),
setupSvc: func(s *Service, msg *ethpb.SyncCommitteeMessage, topic string) (*Service, string) {
s.cfg.StateGen = stategen.New(beaconDB)
s.cfg.stateGen = stategen.New(beaconDB)
msg.BlockRoot = headRoot[:]
s.cfg.DB = beaconDB
s.cfg.beaconDB = beaconDB
s.initCaches()
return s, topic
},
@@ -111,16 +111,16 @@ func TestService_ValidateSyncCommitteeMessage(t *testing.T) {
},
{
name: "Future Slot Message",
svc: NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
}),
svc: NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: false}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
),
setupSvc: func(s *Service, msg *ethpb.SyncCommitteeMessage, topic string) (*Service, string) {
s.cfg.StateGen = stategen.New(beaconDB)
s.cfg.DB = beaconDB
s.cfg.stateGen = stategen.New(beaconDB)
s.cfg.beaconDB = beaconDB
s.initCaches()
return s, topic
},
@@ -138,16 +138,16 @@ func TestService_ValidateSyncCommitteeMessage(t *testing.T) {
},
{
name: "Already Seen Message",
svc: NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
}),
svc: NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: false}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
),
setupSvc: func(s *Service, msg *ethpb.SyncCommitteeMessage, topic string) (*Service, string) {
s.cfg.StateGen = stategen.New(beaconDB)
s.cfg.DB = beaconDB
s.cfg.stateGen = stategen.New(beaconDB)
s.cfg.beaconDB = beaconDB
s.initCaches()
s.setSeenSyncMessageIndexSlot(1, 1, 0)
@@ -167,18 +167,18 @@ func TestService_ValidateSyncCommitteeMessage(t *testing.T) {
},
{
name: "Non-existent block root",
svc: NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
}),
svc: NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: false}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
),
setupSvc: func(s *Service, msg *ethpb.SyncCommitteeMessage, topic string) (*Service, string) {
s.cfg.StateGen = stategen.New(beaconDB)
s.cfg.DB = beaconDB
s.cfg.stateGen = stategen.New(beaconDB)
s.cfg.beaconDB = beaconDB
s.initCaches()
s.cfg.Chain = &mockChain.ChainService{
s.cfg.chain = &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(10)),
}
@@ -201,21 +201,21 @@ func TestService_ValidateSyncCommitteeMessage(t *testing.T) {
},
{
name: "Subnet is non-existent",
svc: NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
}),
svc: NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: false}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
),
setupSvc: func(s *Service, msg *ethpb.SyncCommitteeMessage, topic string) (*Service, string) {
s.cfg.StateGen = stategen.New(beaconDB)
s.cfg.DB = beaconDB
s.cfg.stateGen = stategen.New(beaconDB)
s.cfg.beaconDB = beaconDB
s.initCaches()
msg.BlockRoot = headRoot[:]
hState, err := beaconDB.State(context.Background(), headRoot)
assert.NoError(t, err)
s.cfg.Chain = &mockChain.ChainService{
s.cfg.chain = &mockChain.ChainService{
SyncCommitteeIndices: []types.CommitteeIndex{0},
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(hState.Slot()-1)),
@@ -249,21 +249,21 @@ func TestService_ValidateSyncCommitteeMessage(t *testing.T) {
},
{
name: "Validator is non-existent",
svc: NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
}),
svc: NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: false}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
),
setupSvc: func(s *Service, msg *ethpb.SyncCommitteeMessage, topic string) (*Service, string) {
s.cfg.StateGen = stategen.New(beaconDB)
s.cfg.DB = beaconDB
s.cfg.stateGen = stategen.New(beaconDB)
s.cfg.beaconDB = beaconDB
s.initCaches()
msg.BlockRoot = headRoot[:]
hState, err := beaconDB.State(context.Background(), headRoot)
assert.NoError(t, err)
s.cfg.Chain = &mockChain.ChainService{
s.cfg.chain = &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(hState.Slot()-1)),
}
@@ -296,16 +296,16 @@ func TestService_ValidateSyncCommitteeMessage(t *testing.T) {
},
{
name: "Invalid Sync Committee Signature",
svc: NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
}),
svc: NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: false}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
),
setupSvc: func(s *Service, msg *ethpb.SyncCommitteeMessage, topic string) (*Service, string) {
s.cfg.StateGen = stategen.New(beaconDB)
s.cfg.DB = beaconDB
s.cfg.stateGen = stategen.New(beaconDB)
s.cfg.beaconDB = beaconDB
s.initCaches()
msg.BlockRoot = headRoot[:]
hState, err := beaconDB.State(context.Background(), headRoot)
@@ -322,7 +322,7 @@ func TestService_ValidateSyncCommitteeMessage(t *testing.T) {
d, err := signing.Domain(hState.Fork(), slots.ToEpoch(hState.Slot()), params.BeaconConfig().DomainSyncCommittee, hState.GenesisValidatorRoot())
assert.NoError(t, err)
subCommitteeSize := params.BeaconConfig().SyncCommitteeSize / params.BeaconConfig().SyncCommitteeSubnetCount
s.cfg.Chain = &mockChain.ChainService{
s.cfg.chain = &mockChain.ChainService{
SyncCommitteeIndices: []types.CommitteeIndex{types.CommitteeIndex(subCommitteeSize)},
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(hState.Slot()-1)),
@@ -351,16 +351,16 @@ func TestService_ValidateSyncCommitteeMessage(t *testing.T) {
},
{
name: "Valid Sync Committee Signature",
svc: NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
}),
svc: NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: false}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
),
setupSvc: func(s *Service, msg *ethpb.SyncCommitteeMessage, topic string) (*Service, string) {
s.cfg.StateGen = stategen.New(beaconDB)
s.cfg.DB = beaconDB
s.cfg.stateGen = stategen.New(beaconDB)
s.cfg.beaconDB = beaconDB
s.initCaches()
msg.BlockRoot = headRoot[:]
hState, err := beaconDB.State(context.Background(), headRoot)
@@ -376,7 +376,7 @@ func TestService_ValidateSyncCommitteeMessage(t *testing.T) {
sigRoot, err := signing.ComputeSigningRoot(&rawBytes, d)
assert.NoError(t, err)
s.cfg.Chain = &mockChain.ChainService{
s.cfg.chain = &mockChain.ChainService{
SyncCommitteeIndices: []types.CommitteeIndex{types.CommitteeIndex(subCommitteeSize)},
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(hState.Slot()-1)),
@@ -477,15 +477,15 @@ func TestService_ignoreHasSeenSyncMsg(t *testing.T) {
func TestService_rejectIncorrectSyncCommittee(t *testing.T) {
tests := []struct {
name string
cfg *Config
cfg *config
setupTopic func(s *Service) string
committeeIndices []types.CommitteeIndex
want pubsub.ValidationResult
}{
{
name: "invalid",
cfg: &Config{
Chain: &mockChain.ChainService{
cfg: &config{
chain: &mockChain.ChainService{
Genesis: time.Now(),
ValidatorsRoot: [32]byte{1},
},
@@ -498,8 +498,8 @@ func TestService_rejectIncorrectSyncCommittee(t *testing.T) {
},
{
name: "valid",
cfg: &Config{
Chain: &mockChain.ChainService{
cfg: &config{
chain: &mockChain.ChainService{
Genesis: time.Now(),
ValidatorsRoot: [32]byte{1},
},

View File

@@ -44,12 +44,12 @@ func (s *Service) validateSyncContributionAndProof(ctx context.Context, pid peer
defer span.End()
// Accept the sync committee contribution if the contribution came from itself.
if pid == s.cfg.P2P.PeerID() {
if pid == s.cfg.p2p.PeerID() {
return pubsub.ValidationAccept, nil
}
// Ignore the sync committee contribution if the beacon node is syncing.
if s.cfg.InitialSync.Syncing() {
if s.cfg.initialSync.Syncing() {
return pubsub.ValidationIgnore, nil
}
m, err := s.readSyncContributionMessage(msg)
@@ -59,7 +59,7 @@ func (s *Service) validateSyncContributionAndProof(ctx context.Context, pid peer
}
// The contribution's slot is for the current slot (with a `MAXIMUM_GOSSIP_CLOCK_DISPARITY` allowance).
if err := altair.ValidateSyncMessageTime(m.Message.Contribution.Slot, s.cfg.Chain.GenesisTime(), params.BeaconNetworkConfig().MaximumGossipClockDisparity); err != nil {
if err := altair.ValidateSyncMessageTime(m.Message.Contribution.Slot, s.cfg.chain.GenesisTime(), params.BeaconNetworkConfig().MaximumGossipClockDisparity); err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationIgnore, err
}
@@ -84,7 +84,7 @@ func (s *Service) validateSyncContributionAndProof(ctx context.Context, pid peer
// Broadcast the contribution on a feed to notify other services in the beacon node
// of a received contribution.
s.cfg.OperationNotifier.OperationFeed().Send(&feed.Event{
s.cfg.operationNotifier.OperationFeed().Send(&feed.Event{
Type: opfeed.SyncCommitteeContributionReceived,
Data: &opfeed.SyncCommitteeContributionReceivedData{
Contribution: m,
@@ -162,7 +162,7 @@ func (s *Service) rejectInvalidIndexInSubCommittee(m *ethpb.SignedContributionAn
_, span := trace.StartSpan(ctx, "sync.rejectInvalidIndexInSubCommittee")
defer span.End()
// The aggregator's validator index is in the declared subcommittee of the current sync committee.
committeeIndices, err := s.cfg.Chain.HeadSyncCommitteeIndices(ctx, m.Message.AggregatorIndex, m.Message.Contribution.Slot)
committeeIndices, err := s.cfg.chain.HeadSyncCommitteeIndices(ctx, m.Message.AggregatorIndex, m.Message.Contribution.Slot)
if err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationIgnore, err
@@ -204,12 +204,12 @@ func (s *Service) rejectInvalidContributionSignature(m *ethpb.SignedContribution
_, span := trace.StartSpan(ctx, "sync.rejectInvalidContributionSignature")
defer span.End()
// The aggregator signature, `signed_contribution_and_proof.signature`, is valid.
d, err := s.cfg.Chain.HeadSyncContributionProofDomain(ctx, m.Message.Contribution.Slot)
d, err := s.cfg.chain.HeadSyncContributionProofDomain(ctx, m.Message.Contribution.Slot)
if err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationIgnore, err
}
pubkey, err := s.cfg.Chain.HeadValidatorIndexToPublicKey(ctx, m.Message.AggregatorIndex)
pubkey, err := s.cfg.chain.HeadValidatorIndexToPublicKey(ctx, m.Message.AggregatorIndex)
if err != nil {
return pubsub.ValidationIgnore, err
}
@@ -248,7 +248,7 @@ func (s *Service) rejectInvalidSyncAggregateSignature(m *ethpb.SignedContributio
// derived from the participation info in `aggregation_bits` for the subcommittee specified by the `contribution.subcommittee_index`.
var activePubkeys []bls.PublicKey
var activeRawPubkeys [][]byte
syncPubkeys, err := s.cfg.Chain.HeadSyncCommitteePubKeys(ctx, m.Message.Contribution.Slot, types.CommitteeIndex(m.Message.Contribution.SubcommitteeIndex))
syncPubkeys, err := s.cfg.chain.HeadSyncCommitteePubKeys(ctx, m.Message.Contribution.Slot, types.CommitteeIndex(m.Message.Contribution.SubcommitteeIndex))
if err != nil {
return pubsub.ValidationIgnore, err
}
@@ -269,7 +269,7 @@ func (s *Service) rejectInvalidSyncAggregateSignature(m *ethpb.SignedContributio
activeRawPubkeys = append(activeRawPubkeys, pk)
}
}
d, err := s.cfg.Chain.HeadSyncCommitteeDomain(ctx, m.Message.Contribution.Slot)
d, err := s.cfg.chain.HeadSyncCommitteeDomain(ctx, m.Message.Contribution.Slot)
if err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationIgnore, err
@@ -332,11 +332,11 @@ func (s *Service) setSyncContributionIndexSlotSeen(slot types.Slot, aggregatorIn
// selection proof.
func (s *Service) verifySyncSelectionData(ctx context.Context, m *ethpb.ContributionAndProof) error {
selectionData := &ethpb.SyncAggregatorSelectionData{Slot: m.Contribution.Slot, SubcommitteeIndex: m.Contribution.SubcommitteeIndex}
domain, err := s.cfg.Chain.HeadSyncSelectionProofDomain(ctx, m.Contribution.Slot)
domain, err := s.cfg.chain.HeadSyncSelectionProofDomain(ctx, m.Contribution.Slot)
if err != nil {
return err
}
pubkey, err := s.cfg.Chain.HeadValidatorIndexToPublicKey(ctx, m.AggregatorIndex)
pubkey, err := s.cfg.chain.HeadValidatorIndexToPublicKey(ctx, m.AggregatorIndex)
if err != nil {
return err
}

View File

@@ -64,17 +64,17 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
}{
{
name: "Is syncing",
svc: NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: true},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
}),
svc: NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: true}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
),
setupSvc: func(s *Service, msg *ethpb.SignedContributionAndProof) *Service {
s.cfg.StateGen = stategen.New(db)
s.cfg.stateGen = stategen.New(db)
msg.Message.Contribution.BlockRoot = headRoot[:]
s.cfg.DB = db
s.cfg.beaconDB = db
s.initCaches()
return s
},
@@ -100,17 +100,17 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
},
{
name: "Bad Topic",
svc: NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
}),
svc: NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: false}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
),
setupSvc: func(s *Service, msg *ethpb.SignedContributionAndProof) *Service {
s.cfg.StateGen = stategen.New(db)
s.cfg.stateGen = stategen.New(db)
msg.Message.Contribution.BlockRoot = headRoot[:]
s.cfg.DB = db
s.cfg.beaconDB = db
s.initCaches()
return s
},
@@ -136,16 +136,16 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
},
{
name: "Future Slot Message",
svc: NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
}),
svc: NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: false}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
),
setupSvc: func(s *Service, msg *ethpb.SignedContributionAndProof) *Service {
s.cfg.StateGen = stategen.New(db)
s.cfg.DB = db
s.cfg.stateGen = stategen.New(db)
s.cfg.beaconDB = db
s.initCaches()
return s
},
@@ -171,18 +171,18 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
},
{
name: "Already Seen Message",
svc: NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
}),
svc: NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: false}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
),
setupSvc: func(s *Service, msg *ethpb.SignedContributionAndProof) *Service {
s.cfg.StateGen = stategen.New(db)
s.cfg.DB = db
s.cfg.stateGen = stategen.New(db)
s.cfg.beaconDB = db
s.initCaches()
s.cfg.Chain = &mockChain.ChainService{
s.cfg.chain = &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot)),
}
@@ -214,18 +214,18 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
},
{
name: "Invalid Subcommittee Index",
svc: NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
}),
svc: NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: false}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
),
setupSvc: func(s *Service, msg *ethpb.SignedContributionAndProof) *Service {
s.cfg.StateGen = stategen.New(db)
s.cfg.DB = db
s.cfg.stateGen = stategen.New(db)
s.cfg.beaconDB = db
s.initCaches()
s.cfg.Chain = &mockChain.ChainService{
s.cfg.chain = &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot)),
}
@@ -257,18 +257,18 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
},
{
name: "Invalid Selection Proof",
svc: NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
}),
svc: NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: false}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
),
setupSvc: func(s *Service, msg *ethpb.SignedContributionAndProof) *Service {
s.cfg.StateGen = stategen.New(db)
s.cfg.DB = db
s.cfg.stateGen = stategen.New(db)
s.cfg.beaconDB = db
s.initCaches()
s.cfg.Chain = &mockChain.ChainService{
s.cfg.chain = &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot)),
}
@@ -301,18 +301,18 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
},
{
name: "Invalid Aggregator",
svc: NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
}),
svc: NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: false}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
),
setupSvc: func(s *Service, msg *ethpb.SignedContributionAndProof) *Service {
s.cfg.StateGen = stategen.New(db)
s.cfg.DB = db
s.cfg.stateGen = stategen.New(db)
s.cfg.beaconDB = db
s.initCaches()
s.cfg.Chain = &mockChain.ChainService{
s.cfg.chain = &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot)),
}
@@ -363,16 +363,16 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
},
{
name: "Failed Selection Proof Verification ",
svc: NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
}),
svc: NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: false}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
),
setupSvc: func(s *Service, msg *ethpb.SignedContributionAndProof) *Service {
s.cfg.StateGen = stategen.New(db)
s.cfg.DB = db
s.cfg.stateGen = stategen.New(db)
s.cfg.beaconDB = db
msg.Message.Contribution.BlockRoot = headRoot[:]
hState, err := db.State(context.Background(), headRoot)
assert.NoError(t, err)
@@ -396,7 +396,7 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
}
}
subCommitteeSize := params.BeaconConfig().SyncCommitteeSize / params.BeaconConfig().SyncCommitteeSubnetCount
s.cfg.Chain = &mockChain.ChainService{
s.cfg.chain = &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot)),
SyncCommitteeIndices: []types.CommitteeIndex{types.CommitteeIndex(msg.Message.Contribution.SubcommitteeIndex * subCommitteeSize)},
@@ -428,17 +428,17 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
},
{
name: "Invalid Proof Signature",
svc: NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
}),
svc: NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: false}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
),
setupSvc: func(s *Service, msg *ethpb.SignedContributionAndProof) *Service {
s.cfg.StateGen = stategen.New(db)
s.cfg.DB = db
s.cfg.Chain = chainService
s.cfg.stateGen = stategen.New(db)
s.cfg.beaconDB = db
s.cfg.chain = chainService
msg.Message.Contribution.BlockRoot = headRoot[:]
hState, err := db.State(context.Background(), headRoot)
assert.NoError(t, err)
@@ -475,7 +475,7 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
d, err := signing.Domain(hState.Fork(), slots.ToEpoch(slots.PrevSlot(hState.Slot())), params.BeaconConfig().DomainSyncCommitteeSelectionProof, hState.GenesisValidatorRoot())
require.NoError(t, err)
subCommitteeSize := params.BeaconConfig().SyncCommitteeSize / params.BeaconConfig().SyncCommitteeSubnetCount
s.cfg.Chain = &mockChain.ChainService{
s.cfg.chain = &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot)),
SyncCommitteeIndices: []types.CommitteeIndex{types.CommitteeIndex(msg.Message.Contribution.SubcommitteeIndex * subCommitteeSize)},
@@ -508,16 +508,16 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
},
{
name: "Invalid Sync Aggregate",
svc: NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
}),
svc: NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: false}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
),
setupSvc: func(s *Service, msg *ethpb.SignedContributionAndProof) *Service {
s.cfg.StateGen = stategen.New(db)
s.cfg.DB = db
s.cfg.stateGen = stategen.New(db)
s.cfg.beaconDB = db
msg.Message.Contribution.BlockRoot = headRoot[:]
hState, err := db.State(context.Background(), headRoot)
assert.NoError(t, err)
@@ -559,7 +559,7 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
}
}
}
s.cfg.Chain = &mockChain.ChainService{
s.cfg.chain = &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot)),
SyncCommitteeIndices: []types.CommitteeIndex{1},
@@ -590,17 +590,17 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
},
{
name: "Invalid Signed Sync Contribution And Proof - Zero Bits Set",
svc: NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
}),
svc: NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: false}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
),
setupSvc: func(s *Service, msg *ethpb.SignedContributionAndProof) *Service {
s.cfg.StateGen = stategen.New(db)
s.cfg.stateGen = stategen.New(db)
msg.Message.Contribution.BlockRoot = headRoot[:]
s.cfg.DB = db
s.cfg.beaconDB = db
hState, err := db.State(context.Background(), headRoot)
assert.NoError(t, err)
sc, err := hState.CurrentSyncCommittee()
@@ -640,7 +640,7 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
d, err := signing.Domain(hState.Fork(), slots.ToEpoch(slots.PrevSlot(hState.Slot())), params.BeaconConfig().DomainSyncCommitteeSelectionProof, hState.GenesisValidatorRoot())
require.NoError(t, err)
subCommitteeSize := params.BeaconConfig().SyncCommitteeSize / params.BeaconConfig().SyncCommitteeSubnetCount
s.cfg.Chain = &mockChain.ChainService{
s.cfg.chain = &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot)),
SyncCommitteeIndices: []types.CommitteeIndex{types.CommitteeIndex(msg.Message.Contribution.SubcommitteeIndex * subCommitteeSize)},
@@ -674,17 +674,17 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
},
{
name: "Valid Signed Sync Contribution And Proof - Single Bit Set",
svc: NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
}),
svc: NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: false}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
),
setupSvc: func(s *Service, msg *ethpb.SignedContributionAndProof) *Service {
s.cfg.StateGen = stategen.New(db)
s.cfg.stateGen = stategen.New(db)
msg.Message.Contribution.BlockRoot = headRoot[:]
s.cfg.DB = db
s.cfg.beaconDB = db
hState, err := db.State(context.Background(), headRoot)
assert.NoError(t, err)
sc, err := hState.CurrentSyncCommittee()
@@ -735,7 +735,7 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
pd, err := signing.Domain(hState.Fork(), slots.ToEpoch(slots.PrevSlot(hState.Slot())), params.BeaconConfig().DomainSyncCommitteeSelectionProof, hState.GenesisValidatorRoot())
require.NoError(t, err)
subCommitteeSize := params.BeaconConfig().SyncCommitteeSize / params.BeaconConfig().SyncCommitteeSubnetCount
s.cfg.Chain = &mockChain.ChainService{
s.cfg.chain = &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot)),
SyncCommitteeIndices: []types.CommitteeIndex{types.CommitteeIndex(msg.Message.Contribution.SubcommitteeIndex * subCommitteeSize)},
@@ -770,17 +770,17 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
},
{
name: "Valid Signed Sync Contribution And Proof with Multiple Signatures",
svc: NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
}),
svc: NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: false}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
),
setupSvc: func(s *Service, msg *ethpb.SignedContributionAndProof) *Service {
s.cfg.StateGen = stategen.New(db)
s.cfg.stateGen = stategen.New(db)
msg.Message.Contribution.BlockRoot = headRoot[:]
s.cfg.DB = db
s.cfg.beaconDB = db
hState, err := db.State(context.Background(), headRoot)
assert.NoError(t, err)
sc, err := hState.CurrentSyncCommittee()
@@ -833,7 +833,7 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
pd, err := signing.Domain(hState.Fork(), slots.ToEpoch(slots.PrevSlot(hState.Slot())), params.BeaconConfig().DomainSyncCommitteeSelectionProof, hState.GenesisValidatorRoot())
require.NoError(t, err)
subCommitteeSize := params.BeaconConfig().SyncCommitteeSize / params.BeaconConfig().SyncCommitteeSubnetCount
s.cfg.Chain = &mockChain.ChainService{
s.cfg.chain = &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot)),
SyncCommitteeIndices: []types.CommitteeIndex{types.CommitteeIndex(msg.Message.Contribution.SubcommitteeIndex * subCommitteeSize)},
@@ -917,16 +917,16 @@ func TestService_ValidateSyncContributionAndProof_Broadcast(t *testing.T) {
Genesis: time.Now(),
ValidatorsRoot: [32]byte{'A'},
}
s := NewService(context.Background(), &Config{
P2P: mockp2p.NewTestP2P(t),
InitialSync: &mockSync.Sync{IsSyncing: false},
Chain: chainService,
StateNotifier: chainService.StateNotifier(),
OperationNotifier: chainService.OperationNotifier(),
})
s.cfg.StateGen = stategen.New(db)
s := NewService(context.Background(),
WithP2P(mockp2p.NewTestP2P(t)),
WithInitialSync(&mockSync.Sync{IsSyncing: false}),
WithChainService(chainService),
WithStateNotifier(chainService.StateNotifier()),
WithOperationNotifier(chainService.OperationNotifier()),
)
s.cfg.stateGen = stategen.New(db)
msg.Message.Contribution.BlockRoot = headRoot[:]
s.cfg.DB = db
s.cfg.beaconDB = db
hState, err := db.State(context.Background(), headRoot)
assert.NoError(t, err)
sc, err := hState.CurrentSyncCommittee()
@@ -977,7 +977,7 @@ func TestService_ValidateSyncContributionAndProof_Broadcast(t *testing.T) {
pd, err := signing.Domain(hState.Fork(), slots.ToEpoch(slots.PrevSlot(hState.Slot())), params.BeaconConfig().DomainSyncCommitteeSelectionProof, hState.GenesisValidatorRoot())
require.NoError(t, err)
subCommitteeSize := params.BeaconConfig().SyncCommitteeSize / params.BeaconConfig().SyncCommitteeSubnetCount
s.cfg.Chain = &mockChain.ChainService{
s.cfg.chain = &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot)),
SyncCommitteeIndices: []types.CommitteeIndex{types.CommitteeIndex(msg.Message.Contribution.SubcommitteeIndex * subCommitteeSize)},
@@ -1003,7 +1003,7 @@ func TestService_ValidateSyncContributionAndProof_Broadcast(t *testing.T) {
// Subscribe to operation notifications.
opChannel := make(chan *feed.Event, 1)
opSub := s.cfg.OperationNotifier.OperationFeed().Subscribe(opChannel)
opSub := s.cfg.operationNotifier.OperationFeed().Subscribe(opChannel)
defer opSub.Unsubscribe()
_, err = s.validateSyncContributionAndProof(ctx, pid, pubsubMsg)

View File

@@ -20,12 +20,12 @@ import (
func (s *Service) validateVoluntaryExit(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() {
if pid == s.cfg.p2p.PeerID() {
return pubsub.ValidationAccept, nil
}
// The head state will be too far away to validate any voluntary exit.
if s.cfg.InitialSync.Syncing() {
if s.cfg.initialSync.Syncing() {
return pubsub.ValidationIgnore, nil
}
@@ -50,7 +50,7 @@ func (s *Service) validateVoluntaryExit(ctx context.Context, pid peer.ID, msg *p
return pubsub.ValidationIgnore, nil
}
headState, err := s.cfg.Chain.HeadState(ctx)
headState, err := s.cfg.chain.HeadState(ctx)
if err != nil {
return pubsub.ValidationIgnore, err
}
@@ -70,7 +70,7 @@ func (s *Service) validateVoluntaryExit(ctx context.Context, pid peer.ID, msg *p
// Broadcast the voluntary exit on a feed to notify other services in the beacon node
// of a received voluntary exit.
s.cfg.OperationNotifier.OperationFeed().Send(&feed.Event{
s.cfg.operationNotifier.OperationFeed().Send(&feed.Event{
Type: opfeed.ExitReceived,
Data: &opfeed.ExitReceivedData{
Exit: exit,

View File

@@ -77,14 +77,14 @@ func TestValidateVoluntaryExit_ValidExit(t *testing.T) {
exit, s := setupValidExit(t)
r := &Service{
cfg: &Config{
P2P: p,
Chain: &mock.ChainService{
cfg: &config{
p2p: p,
chain: &mock.ChainService{
State: s,
Genesis: time.Now(),
},
InitialSync: &mockSync.Sync{IsSyncing: false},
OperationNotifier: (&mock.ChainService{}).OperationNotifier(),
initialSync: &mockSync.Sync{IsSyncing: false},
operationNotifier: (&mock.ChainService{}).OperationNotifier(),
},
seenExitCache: lruwrpr.New(10),
}
@@ -105,7 +105,7 @@ func TestValidateVoluntaryExit_ValidExit(t *testing.T) {
// Subscribe to operation notifications.
opChannel := make(chan *feed.Event, 1)
opSub := r.cfg.OperationNotifier.OperationFeed().Subscribe(opChannel)
opSub := r.cfg.operationNotifier.OperationFeed().Subscribe(opChannel)
defer opSub.Unsubscribe()
res, err := r.validateVoluntaryExit(ctx, "", m)
@@ -139,12 +139,12 @@ func TestValidateVoluntaryExit_InvalidExitSlot(t *testing.T) {
// Set state slot to 1 to cause exit object fail to verify.
require.NoError(t, s.SetSlot(1))
r := &Service{
cfg: &Config{
P2P: p,
Chain: &mock.ChainService{
cfg: &config{
p2p: p,
chain: &mock.ChainService{
State: s,
},
InitialSync: &mockSync.Sync{IsSyncing: false},
initialSync: &mockSync.Sync{IsSyncing: false},
},
seenExitCache: lruwrpr.New(10),
}
@@ -172,12 +172,12 @@ func TestValidateVoluntaryExit_ValidExit_Syncing(t *testing.T) {
exit, s := setupValidExit(t)
r := &Service{
cfg: &Config{
P2P: p,
Chain: &mock.ChainService{
cfg: &config{
p2p: p,
chain: &mock.ChainService{
State: s,
},
InitialSync: &mockSync.Sync{IsSyncing: true},
initialSync: &mockSync.Sync{IsSyncing: true},
},
}
buf := new(bytes.Buffer)

View File

@@ -110,10 +110,10 @@ var (
Usage: "Starts the beacon node with the previously saved head state instead of finalized state.",
}
// SlotsPerArchivedPoint specifies the number of slots between the archived points, to save beacon state in the cold
// section of DB.
// section of beaconDB.
SlotsPerArchivedPoint = &cli.IntFlag{
Name: "slots-per-archive-point",
Usage: "The slot durations of when an archived state gets saved in the DB.",
Usage: "The slot durations of when an archived state gets saved in the beaconDB.",
Value: 2048,
}
// DisableDiscv5 disables running discv5.

View File

@@ -1,3 +1,4 @@
//go:build libfuzzer
// +build libfuzzer
package fuzz
@@ -155,19 +156,19 @@ func BeaconFuzzBlock(b []byte) {
}
chain.Start()
s := sync.NewRegularSyncFuzz(&sync.Config{
DB: db1,
P2P: p2p,
Chain: chain,
InitialSync: fakeChecker{},
StateNotifier: sn,
BlockNotifier: bn,
OperationNotifier: an,
AttPool: ap,
ExitPool: ep,
SlashingPool: sp,
StateGen: sgen,
})
s := sync.NewRegularSyncFuzz(
sync.WithDatabase(db1),
sync.WithP2P(p2p),
sync.WithChainService(chain),
sync.WithInitialSync(fakeChecker{}),
sync.WithStateNotifier(sn),
sync.WithBlockNotifier(bn),
sync.WithOperationNotifier(an),
sync.WithAttestationPool(ap),
sync.WithExitPool(ep),
sync.WithSlashingPool(sp),
sync.WithStateGen(sgen),
)
s.InitCaches()

View File

@@ -43,22 +43,18 @@ func init() {
if err := p.Connect(info); err != nil {
panic(errors.Wrap(err, "could not connect to peer"))
}
regularsync.NewService(context.Background(), &regularsync.Config{
P2P: p,
DB: nil,
AttPool: nil,
ExitPool: nil,
SlashingPool: nil,
Chain: &mock.ChainService{
Root: bytesutil.PadTo([]byte("root"), 32),
FinalizedCheckPoint: &ethpb.Checkpoint{Epoch: 4, Root: make([]byte, 32)},
Fork: &ethpb.Fork{CurrentVersion: []byte("foo")},
},
StateNotifier: (&mock.ChainService{}).StateNotifier(),
OperationNotifier: (&mock.ChainService{}).OperationNotifier(),
InitialSync: &mockSync.Sync{IsSyncing: false},
BlockNotifier: nil,
})
regularsync.NewService(context.Background(),
regularsync.WithP2P(p),
regularsync.WithChainService(
&mock.ChainService{
Root: bytesutil.PadTo([]byte("root"), 32),
FinalizedCheckPoint: &ethpb.Checkpoint{Epoch: 4, Root: make([]byte, 32)},
Fork: &ethpb.Fork{CurrentVersion: []byte("foo")},
}),
regularsync.WithStateNotifier((&mock.ChainService{}).StateNotifier()),
regularsync.WithOperationNotifier((&mock.ChainService{}).OperationNotifier()),
regularsync.WithInitialSync(&mockSync.Sync{IsSyncing: false}),
)
}
// FuzzP2PRPCStatus wraps BeaconFuzzP2PRPCStatus in a go-fuzz compatible interface