From 8cfeda14739e59f2cdfd4de0d8eb21f6e1591a46 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Mon, 3 Nov 2025 19:02:41 +0400 Subject: [PATCH 01/39] WIP gossipsub controller --- beacon-chain/p2p/interfaces.go | 8 + beacon-chain/p2p/service.go | 12 + beacon-chain/sync/BUILD.bazel | 7 + beacon-chain/sync/fork_watcher.go | 35 +- beacon-chain/sync/gossipsub_controller.go | 140 ++++++++ beacon-chain/sync/gossipsub_topic_family.go | 148 ++++++++ beacon-chain/sync/service.go | 11 +- beacon-chain/sync/subscriber.go | 153 +------- .../sync/topic_families_dynamic_subnets.go | 215 ++++++++++++ .../sync/topic_families_static_subnets.go | 47 +++ .../sync/topic_families_without_subnets.go | 330 ++++++++++++++++++ 11 files changed, 926 insertions(+), 180 deletions(-) create mode 100644 beacon-chain/sync/gossipsub_controller.go create mode 100644 beacon-chain/sync/gossipsub_topic_family.go create mode 100644 beacon-chain/sync/topic_families_dynamic_subnets.go create mode 100644 beacon-chain/sync/topic_families_static_subnets.go create mode 100644 beacon-chain/sync/topic_families_without_subnets.go diff --git a/beacon-chain/p2p/interfaces.go b/beacon-chain/p2p/interfaces.go index 2d2d0993d342..c04f211d9d90 100644 --- a/beacon-chain/p2p/interfaces.go +++ b/beacon-chain/p2p/interfaces.go @@ -35,6 +35,7 @@ type ( PeersProvider MetadataProvider CustodyManager + SubnetFilter } // Accessor provides access to the Broadcaster, PeerManager and CustodyManager interfaces. @@ -129,4 +130,11 @@ type ( UpdateEarliestAvailableSlot(earliestAvailableSlot primitives.Slot) error CustodyGroupCountFromPeer(peer.ID) uint64 } + + // SubnetFilter provides methods for extracting subnet information from a peer's ENR records. + SubnetFilter interface { + AttestationSubnets(nodeID enode.ID, node *enode.Node, record *enr.Record) (map[uint64]bool, error) + SyncSubnets(nodeID enode.ID, node *enode.Node, record *enr.Record) (map[uint64]bool, error) + DataColumnSubnets(nodeID enode.ID, node *enode.Node, record *enr.Record) (map[uint64]bool, error) + } ) diff --git a/beacon-chain/p2p/service.go b/beacon-chain/p2p/service.go index 40cff9e35f0a..85c81427f830 100644 --- a/beacon-chain/p2p/service.go +++ b/beacon-chain/p2p/service.go @@ -557,3 +557,15 @@ func (s *Service) downscorePeer(peerID peer.ID, reason string) { newScore := s.Peers().Scorers().BadResponsesScorer().Increment(peerID) log.WithFields(logrus.Fields{"peerID": peerID, "reason": reason, "newScore": newScore}).Debug("Downscore peer") } + +func (s *Service) AttestationSubnets(nodeID enode.ID, node *enode.Node, record *enr.Record) (map[uint64]bool, error) { + return attestationSubnets(record) +} + +func (s *Service) SyncSubnets(nodeID enode.ID, node *enode.Node, record *enr.Record) (map[uint64]bool, error) { + return syncSubnets(record) +} + +func (s *Service) DataColumnSubnets(nodeID enode.ID, node *enode.Node, record *enr.Record) (map[uint64]bool, error) { + return dataColumnSubnets(nodeID, record) +} diff --git a/beacon-chain/sync/BUILD.bazel b/beacon-chain/sync/BUILD.bazel index d151af3d3eb0..f01e8d37c460 100644 --- a/beacon-chain/sync/BUILD.bazel +++ b/beacon-chain/sync/BUILD.bazel @@ -15,6 +15,8 @@ go_library( "error.go", "fork_watcher.go", "fuzz_exports.go", # keep + "gossipsub_controller.go", + "gossipsub_topic_family.go", "log.go", "metrics.go", "once.go", @@ -49,6 +51,9 @@ go_library( "subscriber_sync_committee_message.go", "subscriber_sync_contribution_proof.go", "subscription_topic_handler.go", + "topic_families_dynamic_subnets.go", + "topic_families_static_subnets.go", + "topic_families_without_subnets.go", "validate_aggregate_proof.go", "validate_attester_slashing.go", "validate_beacon_attestation.go", @@ -135,6 +140,8 @@ go_library( "//time:go_default_library", "//time/slots:go_default_library", "@com_github_ethereum_go_ethereum//common/hexutil:go_default_library", + "@com_github_ethereum_go_ethereum//p2p/enode:go_default_library", + "@com_github_ethereum_go_ethereum//p2p/enr:go_default_library", "@com_github_hashicorp_golang_lru//:go_default_library", "@com_github_libp2p_go_libp2p//core:go_default_library", "@com_github_libp2p_go_libp2p//core/host:go_default_library", diff --git a/beacon-chain/sync/fork_watcher.go b/beacon-chain/sync/fork_watcher.go index a4791f3c40b1..def3dd64badf 100644 --- a/beacon-chain/sync/fork_watcher.go +++ b/beacon-chain/sync/fork_watcher.go @@ -1,7 +1,6 @@ package sync import ( - "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p" "github.com/OffchainLabs/prysm/v6/config/params" "github.com/OffchainLabs/prysm/v6/consensus-types/primitives" "github.com/OffchainLabs/prysm/v6/time/slots" @@ -13,24 +12,17 @@ import ( // - We are subscribed to the correct gossipsub topics (for the current and upcoming epoch). // - We have registered the correct RPC stream handlers (for the current and upcoming epoch). // - We have cleaned up gossipsub topics and RPC stream handlers that are no longer needed. -func (s *Service) p2pHandlerControlLoop() { - // At startup, launch registration and peer discovery loops, and register rpc stream handlers. - startEntry := params.GetNetworkScheduleEntry(s.cfg.clock.CurrentEpoch()) - s.registerSubscribers(startEntry) - +func (s *Service) p2pRPCHandlerControlLoop() { slotTicker := slots.NewSlotTicker(s.cfg.clock.GenesisTime(), params.BeaconConfig().SecondsPerSlot) for { select { - // In the event of a node restart, we will still end up subscribing to the correct - // topics during/after the fork epoch. This routine is to ensure correct - // subscriptions for nodes running before a fork epoch. case <-slotTicker.C(): current := s.cfg.clock.CurrentEpoch() - if err := s.ensureRegistrationsForEpoch(current); err != nil { + if err := s.ensureRPCRegistrationsForEpoch(current); err != nil { log.WithError(err).Error("Unable to check for fork in the next epoch") continue } - if err := s.ensureDeregistrationForEpoch(current); err != nil { + if err := s.ensureRPCDeregistrationForEpoch(current); err != nil { log.WithError(err).Error("Unable to check for fork in the previous epoch") continue } @@ -44,9 +36,8 @@ func (s *Service) p2pHandlerControlLoop() { // ensureRegistrationsForEpoch ensures that gossip topic and RPC stream handler // registrations are in place for the current and subsequent epoch. -func (s *Service) ensureRegistrationsForEpoch(epoch primitives.Epoch) error { +func (s *Service) ensureRPCRegistrationsForEpoch(epoch primitives.Epoch) error { current := params.GetNetworkScheduleEntry(epoch) - s.registerSubscribers(current) currentHandler, err := s.rpcHandlerByTopicFromFork(current.VersionEnum) if err != nil { @@ -62,7 +53,6 @@ func (s *Service) ensureRegistrationsForEpoch(epoch primitives.Epoch) error { if current.Epoch == next.Epoch { return nil // no fork in the next epoch } - s.registerSubscribers(next) if s.digestActionDone(next.ForkDigest, registerRpcOnce) { return nil @@ -84,7 +74,7 @@ func (s *Service) ensureRegistrationsForEpoch(epoch primitives.Epoch) error { } // ensureDeregistrationForEpoch deregisters appropriate gossip and RPC topic if there is a fork in the current epoch. -func (s *Service) ensureDeregistrationForEpoch(currentEpoch primitives.Epoch) error { +func (s *Service) ensureRPCDeregistrationForEpoch(currentEpoch primitives.Epoch) error { current := params.GetNetworkScheduleEntry(currentEpoch) // If we are still in our genesis fork version then exit early. @@ -115,20 +105,5 @@ func (s *Service) ensureDeregistrationForEpoch(currentEpoch primitives.Epoch) er } } - // Unsubscribe from all gossip topics with the previous fork digest. - if s.digestActionDone(previous.ForkDigest, unregisterGossipOnce) { - return nil - } - for _, t := range s.subHandler.allTopics() { - retDigest, err := p2p.ExtractGossipDigest(t) - if err != nil { - log.WithError(err).Error("Could not retrieve digest") - continue - } - if retDigest == previous.ForkDigest { - s.unSubscribeFromTopic(t) - } - } - return nil } diff --git a/beacon-chain/sync/gossipsub_controller.go b/beacon-chain/sync/gossipsub_controller.go new file mode 100644 index 000000000000..8edfaf74461b --- /dev/null +++ b/beacon-chain/sync/gossipsub_controller.go @@ -0,0 +1,140 @@ +package sync + +import ( + "context" + "fmt" + "sync" + + "github.com/OffchainLabs/prysm/v6/config/params" + "github.com/OffchainLabs/prysm/v6/consensus-types/primitives" + "github.com/OffchainLabs/prysm/v6/time/slots" + "github.com/sirupsen/logrus" +) + +type topicFamilyKey struct { + topicName string + forkDigest [4]byte +} + +func topicFamilyKeyFrom(tf GossipsubTopicFamily) topicFamilyKey { + return topicFamilyKey{topicName: fmt.Sprintf("%T", tf), forkDigest: tf.NetworkScheduleEntry().ForkDigest} +} + +type GossipsubController struct { + ctx context.Context + cancel context.CancelFunc + + syncService *Service + + wg sync.WaitGroup + + mu sync.RWMutex + activeTopicFamilies map[topicFamilyKey]GossipsubTopicFamily +} + +func NewGossipsubController(ctx context.Context, s *Service) *GossipsubController { + ctx, cancel := context.WithCancel(ctx) + return &GossipsubController{ + ctx: ctx, + cancel: cancel, + syncService: s, + activeTopicFamilies: make(map[topicFamilyKey]GossipsubTopicFamily), + } +} + +func (g *GossipsubController) Start() { + currentEpoch := g.syncService.cfg.clock.CurrentEpoch() + + g.updateActiveTopicFamilies(currentEpoch) + g.wg.Go(func() { g.controlLoop() }) + + log.Info("GossipsubController started") +} + +func (g *GossipsubController) controlLoop() { + slotTicker := slots.NewSlotTicker(g.syncService.cfg.clock.GenesisTime(), params.BeaconConfig().SecondsPerSlot) + defer slotTicker.Done() + + for { + select { + case <-slotTicker.C(): + currentEpoch := g.syncService.cfg.clock.CurrentEpoch() + g.updateActiveTopicFamilies(currentEpoch) + + case <-g.ctx.Done(): + return + } + } +} + +func (g *GossipsubController) updateActiveTopicFamilies(currentEpoch primitives.Epoch) { + currentNSE := params.GetNetworkScheduleEntry(currentEpoch) + + families := TopicFamiliesForEpoch(currentEpoch, g.syncService, currentNSE) + isForkBoundary, nextNSE := isNextEpochForkBoundary(currentEpoch) + if isForkBoundary { + families = append(families, TopicFamiliesForEpoch(nextNSE.Epoch, g.syncService, nextNSE)...) + } + g.mu.Lock() + defer g.mu.Unlock() + + // register topic families for the current NSE -> this is idempotent + for _, family := range families { + key := topicFamilyKeyFrom(family) + if _, ok := g.activeTopicFamilies[key]; ok { + continue + } + g.activeTopicFamilies[key] = family + + family.Subscribe() + + log.WithFields(logrus.Fields{ + "topicName": key.topicName, + "forkDigest": fmt.Sprintf("%#x", key.forkDigest), + "epoch": currentEpoch, + }).Info("Registered topic family") + } + + // remove topic families for the previous NSE -> this is idempotent + if beyond, previous := isOneEpochBeyondForkBoundary(currentEpoch); beyond { + for key, family := range g.activeTopicFamilies { + if key.forkDigest == previous.ForkDigest { + + family.Unsubscribe() + + delete(g.activeTopicFamilies, key) + log.WithFields(logrus.Fields{ + "topicName": key.topicName, + "forkDigest": fmt.Sprintf("%#x", key.forkDigest), + }).Info("Removed topic family") + } + } + } +} + +func (g *GossipsubController) Stop() { + g.cancel() + g.wg.Wait() +} + +func isNextEpochForkBoundary(currentEpoch primitives.Epoch) (bool, params.NetworkScheduleEntry) { + current := params.GetNetworkScheduleEntry(currentEpoch) + next := params.GetNetworkScheduleEntry(currentEpoch + 1) + if current.Epoch == next.Epoch { + return false, next // no fork in the next epoch + } + return true, next // there is a fork in the next epoch +} + +func isOneEpochBeyondForkBoundary(currentEpoch primitives.Epoch) (bool, params.NetworkScheduleEntry) { + current := params.GetNetworkScheduleEntry(currentEpoch) + previous := params.GetNetworkScheduleEntry(current.Epoch - 1) + + if current.Epoch == params.BeaconConfig().GenesisEpoch { + return false, previous + } + if currentEpoch < current.Epoch+1 { + return false, previous + } + return true, previous +} diff --git a/beacon-chain/sync/gossipsub_topic_family.go b/beacon-chain/sync/gossipsub_topic_family.go new file mode 100644 index 000000000000..f15ea862e962 --- /dev/null +++ b/beacon-chain/sync/gossipsub_topic_family.go @@ -0,0 +1,148 @@ +package sync + +import ( + "context" + + "github.com/OffchainLabs/prysm/v6/config/features" + "github.com/OffchainLabs/prysm/v6/config/params" + "github.com/OffchainLabs/prysm/v6/consensus-types/primitives" + "github.com/ethereum/go-ethereum/p2p/enode" + pubsub "github.com/libp2p/go-libp2p-pubsub" + "github.com/libp2p/go-libp2p/core/peer" + "google.golang.org/protobuf/proto" +) + +// wrappedVal represents a gossip validator which also returns an error along with the result. +type wrappedVal func(context.Context, peer.ID, *pubsub.Message) (pubsub.ValidationResult, error) + +// subHandler represents handler for a given subscription. +type subHandler func(context.Context, proto.Message) error + +// noopHandler is used for subscriptions that do not require anything to be done. +var noopHandler subHandler = func(ctx context.Context, msg proto.Message) error { + return nil +} + +type baseGossipsubTopicFamily struct { + syncService *Service + protocolSuffix string + nse params.NetworkScheduleEntry +} + +func (b *baseGossipsubTopicFamily) NetworkScheduleEntry() params.NetworkScheduleEntry { + return b.nse +} + +type GossipsubTopicFamily interface { + Validator() wrappedVal + Handler() subHandler + NetworkScheduleEntry() params.NetworkScheduleEntry + Subscribe() + Unsubscribe() +} + +type GossipsubTopicFamilyWithoutDynamicSubnets interface { + GossipsubTopicFamily + GetFullTopicString() string +} + +type GossipsubTopicFamilyWithDynamicSubnets interface { + GossipsubTopicFamily + GetFullTopicString(subnet uint64) string + GetSubnetsToJoin(slot primitives.Slot) map[uint64]bool + GetSubnetsForBroadcast(slot primitives.Slot) map[uint64]bool + GetTopicsForNode(node *enode.Node) ([]string, error) +} + +type topicFamilyEntry struct { + activationEpoch primitives.Epoch + deactivationEpoch *primitives.Epoch // optional; inactive at >= deactivationEpoch + factory func(s *Service, nse params.NetworkScheduleEntry) []GossipsubTopicFamily +} + +var topicFamilySchedule = []topicFamilyEntry{ + // Genesis topic families + { + activationEpoch: params.BeaconConfig().GenesisEpoch, + factory: func(s *Service, nse params.NetworkScheduleEntry) []GossipsubTopicFamily { + return []GossipsubTopicFamily{ + NewBlockTopicFamily(s, nse), + NewAggregateAndProofTopicFamily(s, nse), + NewVoluntaryExitTopicFamily(s, nse), + NewProposerSlashingTopicFamily(s, nse), + NewAttesterSlashingTopicFamily(s, nse), + NewAttestationTopicFamily(s, nse), + } + }, + }, + // Altair topic families + { + activationEpoch: params.BeaconConfig().AltairForkEpoch, + factory: func(s *Service, nse params.NetworkScheduleEntry) []GossipsubTopicFamily { + families := []GossipsubTopicFamily{ + NewSyncContributionAndProofTopicFamily(s, nse), + NewSyncCommitteeTopicFamily(s, nse), + } + if features.Get().EnableLightClient { + families = append(families, + NewLightClientOptimisticUpdateTopicFamily(s, nse), + NewLightClientFinalityUpdateTopicFamily(s, nse), + ) + } + return families + }, + }, + // Capella topic families + { + activationEpoch: params.BeaconConfig().CapellaForkEpoch, + factory: func(s *Service, nse params.NetworkScheduleEntry) []GossipsubTopicFamily { + return []GossipsubTopicFamily{NewBlsToExecutionChangeTopicFamily(s, nse)} + }, + }, + // Blob topic families (static per-subnet) in Deneb and Electra forks (removed in Fulu) + { + activationEpoch: params.BeaconConfig().DenebForkEpoch, + deactivationEpoch: func() *primitives.Epoch { e := params.BeaconConfig().ElectraForkEpoch; return &e }(), + factory: func(s *Service, nse params.NetworkScheduleEntry) []GossipsubTopicFamily { + count := params.BeaconConfig().BlobsidecarSubnetCount + families := make([]GossipsubTopicFamily, 0, count) + for i := uint64(0); i < count; i++ { + families = append(families, NewBlobTopicFamily(s, nse, i)) + } + return families + }, + }, + { + activationEpoch: params.BeaconConfig().ElectraForkEpoch, + deactivationEpoch: func() *primitives.Epoch { e := params.BeaconConfig().FuluForkEpoch; return &e }(), + factory: func(s *Service, nse params.NetworkScheduleEntry) []GossipsubTopicFamily { + count := params.BeaconConfig().BlobsidecarSubnetCountElectra + families := make([]GossipsubTopicFamily, 0, count) + for i := uint64(0); i < count; i++ { + families = append(families, NewBlobTopicFamily(s, nse, i)) + } + return families + }, + }, + // Fulu data column topic family + { + activationEpoch: params.BeaconConfig().FuluForkEpoch, + factory: func(s *Service, nse params.NetworkScheduleEntry) []GossipsubTopicFamily { + return []GossipsubTopicFamily{NewDataColumnTopicFamily(s, nse)} + }, + }, +} + +func TopicFamiliesForEpoch(epoch primitives.Epoch, s *Service, nse params.NetworkScheduleEntry) []GossipsubTopicFamily { + var activeFamilies []GossipsubTopicFamily + for _, entry := range topicFamilySchedule { + if epoch < entry.activationEpoch { + continue + } + if entry.deactivationEpoch != nil && epoch >= *entry.deactivationEpoch { + continue + } + activeFamilies = append(activeFamilies, entry.factory(s, nse)...) + } + return activeFamilies +} diff --git a/beacon-chain/sync/service.go b/beacon-chain/sync/service.go index 02ccffd9f386..b8456fc4caf4 100644 --- a/beacon-chain/sync/service.go +++ b/beacon-chain/sync/service.go @@ -182,6 +182,7 @@ type Service struct { dataColumnLogCh chan dataColumnLogEntry digestActions perDigestSet subscriptionSpawner func(func()) // see Service.spawn for details + gossipsubController *GossipsubController } // NewService initializes new regular sync service. @@ -198,6 +199,7 @@ func NewService(ctx context.Context, opts ...Option) *Service { dataColumnLogCh: make(chan dataColumnLogEntry, 1000), reconstructionRandGen: rand.NewGenerator(), } + r.gossipsubController = NewGossipsubController(ctx, r) for _, opt := range opts { if err := opt(r); err != nil { @@ -326,6 +328,10 @@ func (s *Service) Stop() error { for _, t := range s.cfg.p2p.PubSub().GetTopics() { s.unSubscribeFromTopic(t) } + + // Stop the gossipsub controller. + s.gossipsubController.Stop() + return nil } @@ -405,7 +411,10 @@ func (s *Service) startDiscoveryAndSubscriptions() { } // Start the fork watcher. - go s.p2pHandlerControlLoop() + go s.p2pRPCHandlerControlLoop() + + // Start the gossipsub controller. + go s.gossipsubController.Start() } func (s *Service) writeErrorResponseToStream(responseCode byte, reason string, stream libp2pcore.Stream) { diff --git a/beacon-chain/sync/subscriber.go b/beacon-chain/sync/subscriber.go index babe32eb9fae..1ed6fc208578 100644 --- a/beacon-chain/sync/subscriber.go +++ b/beacon-chain/sync/subscriber.go @@ -38,17 +38,6 @@ const pubsubMessageTimeout = 30 * time.Second var errInvalidDigest = errors.New("invalid digest") -// wrappedVal represents a gossip validator which also returns an error along with the result. -type wrappedVal func(context.Context, peer.ID, *pubsub.Message) (pubsub.ValidationResult, error) - -// subHandler represents handler for a given subscription. -type subHandler func(context.Context, proto.Message) error - -// noopHandler is used for subscriptions that do not require anything to be done. -var noopHandler subHandler = func(ctx context.Context, msg proto.Message) error { - return nil -} - // subscribeParameters holds the parameters that are needed to construct a set of subscriptions topics for a given // set of gossipsub subnets. type subscribeParameters struct { @@ -203,136 +192,6 @@ func (s *Service) spawn(f func()) { } } -// Register PubSub subscribers -func (s *Service) registerSubscribers(nse params.NetworkScheduleEntry) bool { - // If we have already registered for this fork digest, exit early. - if s.digestActionDone(nse.ForkDigest, registerGossipOnce) { - return false - } - s.spawn(func() { - s.subscribe(p2p.BlockSubnetTopicFormat, s.validateBeaconBlockPubSub, s.beaconBlockSubscriber, nse) - }) - s.spawn(func() { - s.subscribe(p2p.AggregateAndProofSubnetTopicFormat, s.validateAggregateAndProof, s.beaconAggregateProofSubscriber, nse) - }) - s.spawn(func() { - s.subscribe(p2p.ExitSubnetTopicFormat, s.validateVoluntaryExit, s.voluntaryExitSubscriber, nse) - }) - s.spawn(func() { - s.subscribe(p2p.ProposerSlashingSubnetTopicFormat, s.validateProposerSlashing, s.proposerSlashingSubscriber, nse) - }) - s.spawn(func() { - s.subscribe(p2p.AttesterSlashingSubnetTopicFormat, s.validateAttesterSlashing, s.attesterSlashingSubscriber, nse) - }) - s.spawn(func() { - s.subscribeWithParameters(subscribeParameters{ - topicFormat: p2p.AttestationSubnetTopicFormat, - validate: s.validateCommitteeIndexBeaconAttestation, - handle: s.committeeIndexBeaconAttestationSubscriber, - getSubnetsToJoin: s.persistentAndAggregatorSubnetIndices, - getSubnetsRequiringPeers: attesterSubnetIndices, - nse: nse, - }) - }) - - // New gossip topic in Altair - if params.BeaconConfig().AltairForkEpoch <= nse.Epoch { - s.spawn(func() { - s.subscribe( - p2p.SyncContributionAndProofSubnetTopicFormat, - s.validateSyncContributionAndProof, - s.syncContributionAndProofSubscriber, - nse, - ) - }) - s.spawn(func() { - s.subscribeWithParameters(subscribeParameters{ - topicFormat: p2p.SyncCommitteeSubnetTopicFormat, - validate: s.validateSyncCommitteeMessage, - handle: s.syncCommitteeMessageSubscriber, - getSubnetsToJoin: s.activeSyncSubnetIndices, - nse: nse, - }) - }) - - if features.Get().EnableLightClient { - s.spawn(func() { - s.subscribe( - p2p.LightClientOptimisticUpdateTopicFormat, - s.validateLightClientOptimisticUpdate, - noopHandler, - nse, - ) - }) - s.spawn(func() { - s.subscribe( - p2p.LightClientFinalityUpdateTopicFormat, - s.validateLightClientFinalityUpdate, - noopHandler, - nse, - ) - }) - } - } - - // New gossip topic in Capella - if params.BeaconConfig().CapellaForkEpoch <= nse.Epoch { - s.spawn(func() { - s.subscribe( - p2p.BlsToExecutionChangeSubnetTopicFormat, - s.validateBlsToExecutionChange, - s.blsToExecutionChangeSubscriber, - nse, - ) - }) - } - - // New gossip topic in Deneb, removed in Electra - if params.BeaconConfig().DenebForkEpoch <= nse.Epoch && nse.Epoch < params.BeaconConfig().ElectraForkEpoch { - s.spawn(func() { - s.subscribeWithParameters(subscribeParameters{ - topicFormat: p2p.BlobSubnetTopicFormat, - validate: s.validateBlob, - handle: s.blobSubscriber, - nse: nse, - getSubnetsToJoin: func(primitives.Slot) map[uint64]bool { - return mapFromCount(params.BeaconConfig().BlobsidecarSubnetCount) - }, - }) - }) - } - - // New gossip topic in Electra, removed in Fulu - if params.BeaconConfig().ElectraForkEpoch <= nse.Epoch && nse.Epoch < params.BeaconConfig().FuluForkEpoch { - s.spawn(func() { - s.subscribeWithParameters(subscribeParameters{ - topicFormat: p2p.BlobSubnetTopicFormat, - validate: s.validateBlob, - handle: s.blobSubscriber, - nse: nse, - getSubnetsToJoin: func(currentSlot primitives.Slot) map[uint64]bool { - return mapFromCount(params.BeaconConfig().BlobsidecarSubnetCountElectra) - }, - }) - }) - } - - // New gossip topic in Fulu. - if params.BeaconConfig().FuluForkEpoch <= nse.Epoch { - s.spawn(func() { - s.subscribeWithParameters(subscribeParameters{ - topicFormat: p2p.DataColumnSubnetTopicFormat, - validate: s.validateDataColumn, - handle: s.dataColumnSubscriber, - nse: nse, - getSubnetsToJoin: s.dataColumnSubnetIndices, - getSubnetsRequiringPeers: s.allDataColumnSubnets, - }) - }) - } - return true -} - func (s *Service) subscriptionRequestExpired(nse params.NetworkScheduleEntry) bool { next := params.NextNetworkScheduleEntry(nse.Epoch) return next.Epoch != nse.Epoch && s.cfg.clock.CurrentEpoch() > next.Epoch @@ -349,7 +208,9 @@ func (s *Service) subscribeLogFields(topic string, nse params.NetworkScheduleEnt // 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, nse params.NetworkScheduleEntry) { +func (s *Service) subscribe(tf GossipsubTopicFamilyWithoutDynamicSubnets) { + topic := tf.GetFullTopicString() + nse := tf.NetworkScheduleEntry() if err := s.waitForInitialSync(s.ctx); err != nil { log.WithFields(s.subscribeLogFields(topic, nse)).WithError(err).Debug("Context cancelled while waiting for initial sync, not subscribing to topic") return @@ -360,16 +221,10 @@ func (s *Service) subscribe(topic string, validator wrappedVal, handle subHandle log.WithFields(s.subscribeLogFields(topic, nse)).Debug("Not subscribing to topic as we are already past the next fork epoch") return } - base := p2p.GossipTopicMappings(topic, nse.Epoch) - if base == nil { - // Impossible condition as it would mean topic does not exist. - panic(fmt.Sprintf("%s is not mapped to any message in GossipTopicMappings", topic)) // lint:nopanic -- Impossible condition. - } - s.subscribeWithBase(s.addDigestToTopic(topic, nse.ForkDigest), validator, handle) + s.subscribeWithBase(topic, tf.Validator(), tf.Handler()) } func (s *Service) subscribeWithBase(topic string, validator wrappedVal, handle subHandler) *pubsub.Subscription { - topic += s.cfg.p2p.Encoding().ProtocolSuffix() log := log.WithField("topic", topic) // Do not resubscribe already seen subscriptions. diff --git a/beacon-chain/sync/topic_families_dynamic_subnets.go b/beacon-chain/sync/topic_families_dynamic_subnets.go new file mode 100644 index 000000000000..12030bcf0663 --- /dev/null +++ b/beacon-chain/sync/topic_families_dynamic_subnets.go @@ -0,0 +1,215 @@ +package sync + +import ( + "fmt" + + "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p" + "github.com/OffchainLabs/prysm/v6/config/params" + "github.com/OffchainLabs/prysm/v6/consensus-types/primitives" + "github.com/ethereum/go-ethereum/p2p/enode" + "github.com/ethereum/go-ethereum/p2p/enr" + "github.com/pkg/errors" +) + +// AttestationTopicFamily +var _ GossipsubTopicFamilyWithDynamicSubnets = (*AttestationTopicFamily)(nil) + +type AttestationTopicFamily struct { + baseGossipsubTopicFamily +} + +// NewAttestationTopicFamily creates a new AttestationTopicFamily. +func NewAttestationTopicFamily(s *Service, nse params.NetworkScheduleEntry) *AttestationTopicFamily { + return &AttestationTopicFamily{ + baseGossipsubTopicFamily{ + syncService: s, + nse: nse, + protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix(), + }, + } +} + +// Validator returns the validator function for attestation subnets. +func (a *AttestationTopicFamily) Validator() wrappedVal { + return a.syncService.validateCommitteeIndexBeaconAttestation +} + +// Handler returns the message handler for attestation subnets. +func (a *AttestationTopicFamily) Handler() subHandler { + return a.syncService.committeeIndexBeaconAttestationSubscriber +} + +// GetFullTopicString builds the full topic string for an attestation subnet. +func (a *AttestationTopicFamily) GetFullTopicString(subnet uint64) string { + return fmt.Sprintf(p2p.AttestationSubnetTopicFormat, a.nse.ForkDigest, subnet) + a.protocolSuffix +} + +// GetSubnetsToJoin returns persistent and aggregator subnets. +func (a *AttestationTopicFamily) GetSubnetsToJoin(slot primitives.Slot) map[uint64]bool { + return a.syncService.persistentAndAggregatorSubnetIndices(slot) +} + +// GetSubnetsForBroadcast returns subnets needed for attestation duties. +func (a *AttestationTopicFamily) GetSubnetsForBroadcast(slot primitives.Slot) map[uint64]bool { + return attesterSubnetIndices(slot) +} + +// GetTopicsForNode returns all topics for the given node that are relevant to this topic family. +func (a *AttestationTopicFamily) GetTopicsForNode(node *enode.Node) ([]string, error) { + return getTopicsForNode(a.syncService, a, node, a.syncService.cfg.p2p.AttestationSubnets) +} + +// TODO +func (a *AttestationTopicFamily) Subscribe() { + +} + +func (a *AttestationTopicFamily) Unsubscribe() { + +} + +// SyncCommitteeTopicFamily +var _ GossipsubTopicFamilyWithDynamicSubnets = (*SyncCommitteeTopicFamily)(nil) + +type SyncCommitteeTopicFamily struct { + baseGossipsubTopicFamily +} + +// NewSyncCommitteeTopicFamily creates a new SyncCommitteeTopicFamily. +func NewSyncCommitteeTopicFamily(s *Service, nse params.NetworkScheduleEntry) *SyncCommitteeTopicFamily { + return &SyncCommitteeTopicFamily{ + baseGossipsubTopicFamily{ + syncService: s, + nse: nse, + protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix(), + }, + } +} + +// Validator returns the validator function for sync committee subnets. +func (s *SyncCommitteeTopicFamily) Validator() wrappedVal { + return s.syncService.validateSyncCommitteeMessage +} + +// Handler returns the message handler for sync committee subnets. +func (s *SyncCommitteeTopicFamily) Handler() subHandler { + return s.syncService.syncCommitteeMessageSubscriber +} + +// GetFullTopicString builds the full topic string for a sync committee subnet. +func (s *SyncCommitteeTopicFamily) GetFullTopicString(subnet uint64) string { + return fmt.Sprintf(p2p.SyncCommitteeSubnetTopicFormat, s.nse.ForkDigest, subnet) + s.protocolSuffix +} + +// GetSubnetsToJoin returns active sync committee subnets. +func (s *SyncCommitteeTopicFamily) GetSubnetsToJoin(slot primitives.Slot) map[uint64]bool { + return s.syncService.activeSyncSubnetIndices(slot) +} + +// GetSubnetsForBroadcast returns nil as there are no separate peer requirements. +func (s *SyncCommitteeTopicFamily) GetSubnetsForBroadcast(slot primitives.Slot) map[uint64]bool { + return nil +} + +// GetTopicsForNode returns all topics for the given node that are relevant to this topic family. +func (s *SyncCommitteeTopicFamily) GetTopicsForNode(node *enode.Node) ([]string, error) { + return getTopicsForNode(s.syncService, s, node, s.syncService.cfg.p2p.SyncSubnets) +} + +// TODO +func (s *SyncCommitteeTopicFamily) Subscribe() { + +} + +func (s *SyncCommitteeTopicFamily) Unsubscribe() { + +} + +// DataColumnTopicFamily +var _ GossipsubTopicFamilyWithDynamicSubnets = (*DataColumnTopicFamily)(nil) + +type DataColumnTopicFamily struct { + baseGossipsubTopicFamily +} + +// NewDataColumnTopicFamily creates a new DataColumnTopicFamily. +func NewDataColumnTopicFamily(s *Service, nse params.NetworkScheduleEntry) *DataColumnTopicFamily { + return &DataColumnTopicFamily{ + baseGossipsubTopicFamily{ + syncService: s, + nse: nse, + protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix(), + }, + } +} + +// Validator returns the validator function for data column subnets. +func (d *DataColumnTopicFamily) Validator() wrappedVal { + return d.syncService.validateDataColumn +} + +// Handler returns the message handler for data column subnets. +func (d *DataColumnTopicFamily) Handler() subHandler { + return d.syncService.dataColumnSubscriber +} + +// GetFullTopicString builds the full topic string for a data column subnet. +func (d *DataColumnTopicFamily) GetFullTopicString(subnet uint64) string { + return fmt.Sprintf(p2p.DataColumnSubnetTopicFormat, d.nse.ForkDigest, subnet) + d.protocolSuffix +} + +// GetSubnetsToJoin returns data column subnets. +func (d *DataColumnTopicFamily) GetSubnetsToJoin(slot primitives.Slot) map[uint64]bool { + return d.syncService.dataColumnSubnetIndices(slot) +} + +// GetSubnetsForBroadcast returns all data column subnets. +func (d *DataColumnTopicFamily) GetSubnetsForBroadcast(slot primitives.Slot) map[uint64]bool { + return d.syncService.allDataColumnSubnets(slot) +} + +// GetTopicsForNode returns all topics for the given node that are relevant to this topic family. +func (d *DataColumnTopicFamily) GetTopicsForNode(node *enode.Node) ([]string, error) { + return getTopicsForNode(d.syncService, d, node, d.syncService.cfg.p2p.DataColumnSubnets) +} + +// TODO +func (d *DataColumnTopicFamily) Subscribe() { + +} + +func (d *DataColumnTopicFamily) Unsubscribe() { + +} + +type nodeSubnetExtractor func(id enode.ID, n *enode.Node, r *enr.Record) (map[uint64]bool, error) + +func getTopicsForNode( + s *Service, + tf GossipsubTopicFamilyWithDynamicSubnets, + node *enode.Node, + extractor nodeSubnetExtractor, +) ([]string, error) { + if node == nil { + return nil, errors.New("enode is nil") + } + currentSlot := s.cfg.clock.CurrentSlot() + neededSubnets := computeAllNeededSubnets( + currentSlot, + tf.GetSubnetsToJoin, + tf.GetSubnetsForBroadcast, + ) + + nodeSubnets, err := extractor(node.ID(), node, node.Record()) + if err != nil { + return nil, err + } + + var topics []string + for subnet := range neededSubnets { + if nodeSubnets[subnet] { + topics = append(topics, tf.GetFullTopicString(subnet)) + } + } + return topics, nil +} diff --git a/beacon-chain/sync/topic_families_static_subnets.go b/beacon-chain/sync/topic_families_static_subnets.go new file mode 100644 index 000000000000..0497c3252022 --- /dev/null +++ b/beacon-chain/sync/topic_families_static_subnets.go @@ -0,0 +1,47 @@ +package sync + +import ( + "fmt" + + "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p" + "github.com/OffchainLabs/prysm/v6/config/params" +) + +var _ GossipsubTopicFamilyWithoutDynamicSubnets = (*BlobTopicFamily)(nil) + +// BlobTopicFamily represents a static-subnet family instance for a specific blob subnet index. +type BlobTopicFamily struct { + baseGossipsubTopicFamily + subnetIndex uint64 +} + +func NewBlobTopicFamily(s *Service, nse params.NetworkScheduleEntry, subnetIndex uint64) *BlobTopicFamily { + return &BlobTopicFamily{ + baseGossipsubTopicFamily{ + syncService: s, + nse: nse, + protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix(), + }, + subnetIndex, + } +} + +func (b *BlobTopicFamily) Validator() wrappedVal { + return b.syncService.validateBlob +} + +func (b *BlobTopicFamily) Handler() subHandler { + return b.syncService.blobSubscriber +} + +func (b *BlobTopicFamily) GetFullTopicString() string { + return fmt.Sprintf(p2p.BlobSubnetTopicFormat, b.nse.ForkDigest, b.subnetIndex) + b.protocolSuffix +} + +func (b *BlobTopicFamily) Subscribe() { + b.syncService.subscribe(b) +} + +func (b *BlobTopicFamily) Unsubscribe() { + b.syncService.unSubscribeFromTopic(b.GetFullTopicString()) +} diff --git a/beacon-chain/sync/topic_families_without_subnets.go b/beacon-chain/sync/topic_families_without_subnets.go new file mode 100644 index 000000000000..bd5903ab58ae --- /dev/null +++ b/beacon-chain/sync/topic_families_without_subnets.go @@ -0,0 +1,330 @@ +package sync + +import ( + "fmt" + + "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p" + "github.com/OffchainLabs/prysm/v6/config/params" +) + +// Blocks +var _ GossipsubTopicFamilyWithoutDynamicSubnets = (*BlockTopicFamily)(nil) + +type BlockTopicFamily struct { + baseGossipsubTopicFamily +} + +func NewBlockTopicFamily(s *Service, nse params.NetworkScheduleEntry) *BlockTopicFamily { + return &BlockTopicFamily{ + baseGossipsubTopicFamily{ + syncService: s, + nse: nse, + protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix()}, + } +} + +func (b *BlockTopicFamily) Validator() wrappedVal { + return b.syncService.validateBeaconBlockPubSub +} + +func (b *BlockTopicFamily) Handler() subHandler { + return b.syncService.beaconBlockSubscriber +} + +func (b *BlockTopicFamily) GetFullTopicString() string { + return fmt.Sprintf(p2p.BlockSubnetTopicFormat, b.nse.ForkDigest) + b.protocolSuffix +} + +func (b *BlockTopicFamily) Subscribe() { + b.syncService.subscribe(b) +} + +func (b *BlockTopicFamily) Unsubscribe() { + b.syncService.unSubscribeFromTopic(b.GetFullTopicString()) +} + +// Aggregate and Proof +var _ GossipsubTopicFamilyWithoutDynamicSubnets = (*AggregateAndProofTopicFamily)(nil) + +type AggregateAndProofTopicFamily struct { + baseGossipsubTopicFamily +} + +func NewAggregateAndProofTopicFamily(s *Service, nse params.NetworkScheduleEntry) *AggregateAndProofTopicFamily { + return &AggregateAndProofTopicFamily{ + baseGossipsubTopicFamily{ + syncService: s, + nse: nse, + protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix()}, + } +} + +func (a *AggregateAndProofTopicFamily) Validator() wrappedVal { + return a.syncService.validateAggregateAndProof +} + +func (a *AggregateAndProofTopicFamily) Handler() subHandler { + return a.syncService.beaconAggregateProofSubscriber +} + +func (a *AggregateAndProofTopicFamily) GetFullTopicString() string { + return fmt.Sprintf(p2p.AggregateAndProofSubnetTopicFormat, a.nse.ForkDigest) + a.protocolSuffix +} + +func (a *AggregateAndProofTopicFamily) Subscribe() { + a.syncService.subscribe(a) +} + +func (a *AggregateAndProofTopicFamily) Unsubscribe() { + a.syncService.unSubscribeFromTopic(a.GetFullTopicString()) +} + +// Voluntary Exit +var _ GossipsubTopicFamilyWithoutDynamicSubnets = (*VoluntaryExitTopicFamily)(nil) + +type VoluntaryExitTopicFamily struct { + baseGossipsubTopicFamily +} + +func NewVoluntaryExitTopicFamily(s *Service, nse params.NetworkScheduleEntry) *VoluntaryExitTopicFamily { + return &VoluntaryExitTopicFamily{ + baseGossipsubTopicFamily{ + syncService: s, + nse: nse, + protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix()}, + } +} + +func (v *VoluntaryExitTopicFamily) Validator() wrappedVal { + return v.syncService.validateVoluntaryExit +} + +func (v *VoluntaryExitTopicFamily) Handler() subHandler { + return v.syncService.voluntaryExitSubscriber +} + +func (v *VoluntaryExitTopicFamily) GetFullTopicString() string { + return fmt.Sprintf(p2p.ExitSubnetTopicFormat, v.nse.ForkDigest) + v.protocolSuffix +} + +func (v *VoluntaryExitTopicFamily) Subscribe() { + v.syncService.subscribe(v) +} + +func (v *VoluntaryExitTopicFamily) Unsubscribe() { + v.syncService.unSubscribeFromTopic(v.GetFullTopicString()) +} + +// Proposer Slashing +var _ GossipsubTopicFamilyWithoutDynamicSubnets = (*ProposerSlashingTopicFamily)(nil) + +type ProposerSlashingTopicFamily struct { + baseGossipsubTopicFamily +} + +func NewProposerSlashingTopicFamily(s *Service, nse params.NetworkScheduleEntry) *ProposerSlashingTopicFamily { + return &ProposerSlashingTopicFamily{ + baseGossipsubTopicFamily{ + syncService: s, + nse: nse, + protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix()}, + } +} + +func (p *ProposerSlashingTopicFamily) Validator() wrappedVal { + return p.syncService.validateProposerSlashing +} + +func (p *ProposerSlashingTopicFamily) Handler() subHandler { + return p.syncService.proposerSlashingSubscriber +} + +func (p *ProposerSlashingTopicFamily) GetFullTopicString() string { + return fmt.Sprintf(p2p.ProposerSlashingSubnetTopicFormat, p.nse.ForkDigest) + p.protocolSuffix +} + +func (p *ProposerSlashingTopicFamily) Subscribe() { + p.syncService.subscribe(p) +} + +func (p *ProposerSlashingTopicFamily) Unsubscribe() { + p.syncService.unSubscribeFromTopic(p.GetFullTopicString()) +} + +// Attester Slashing +var _ GossipsubTopicFamilyWithoutDynamicSubnets = (*AttesterSlashingTopicFamily)(nil) + +type AttesterSlashingTopicFamily struct { + baseGossipsubTopicFamily +} + +func NewAttesterSlashingTopicFamily(s *Service, nse params.NetworkScheduleEntry) *AttesterSlashingTopicFamily { + return &AttesterSlashingTopicFamily{ + baseGossipsubTopicFamily{ + syncService: s, + nse: nse, + protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix()}, + } +} + +func (a *AttesterSlashingTopicFamily) Validator() wrappedVal { + return a.syncService.validateAttesterSlashing +} + +func (a *AttesterSlashingTopicFamily) Handler() subHandler { + return a.syncService.attesterSlashingSubscriber +} + +func (a *AttesterSlashingTopicFamily) GetFullTopicString() string { + return fmt.Sprintf(p2p.AttesterSlashingSubnetTopicFormat, a.nse.ForkDigest) + a.protocolSuffix +} + +// TODO: Do we really need to spawn go-routines here ? +func (a *AttesterSlashingTopicFamily) Subscribe() { + a.syncService.subscribe(a) +} + +func (a *AttesterSlashingTopicFamily) Unsubscribe() { + a.syncService.unSubscribeFromTopic(a.GetFullTopicString()) +} + +// Sync Contribution and Proof (Altair+) +var _ GossipsubTopicFamilyWithoutDynamicSubnets = (*SyncContributionAndProofTopicFamily)(nil) + +type SyncContributionAndProofTopicFamily struct{ baseGossipsubTopicFamily } + +func NewSyncContributionAndProofTopicFamily(s *Service, nse params.NetworkScheduleEntry) *SyncContributionAndProofTopicFamily { + return &SyncContributionAndProofTopicFamily{ + baseGossipsubTopicFamily{ + syncService: s, + nse: nse, + protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix()}, + } +} + +func (sc *SyncContributionAndProofTopicFamily) Validator() wrappedVal { + return sc.syncService.validateSyncContributionAndProof +} + +func (sc *SyncContributionAndProofTopicFamily) Handler() subHandler { + return sc.syncService.syncContributionAndProofSubscriber +} + +func (sc *SyncContributionAndProofTopicFamily) GetFullTopicString() string { + return fmt.Sprintf(p2p.SyncContributionAndProofSubnetTopicFormat, sc.nse.ForkDigest) + sc.protocolSuffix +} + +func (sc *SyncContributionAndProofTopicFamily) Subscribe() { + sc.syncService.subscribe(sc) +} + +func (sc *SyncContributionAndProofTopicFamily) Unsubscribe() { + sc.syncService.unSubscribeFromTopic(sc.GetFullTopicString()) +} + +// Light Client Optimistic Update (Altair+) +var _ GossipsubTopicFamilyWithoutDynamicSubnets = (*LightClientOptimisticUpdateTopicFamily)(nil) + +type LightClientOptimisticUpdateTopicFamily struct { + baseGossipsubTopicFamily +} + +func NewLightClientOptimisticUpdateTopicFamily(s *Service, nse params.NetworkScheduleEntry) *LightClientOptimisticUpdateTopicFamily { + return &LightClientOptimisticUpdateTopicFamily{ + baseGossipsubTopicFamily{ + syncService: s, + nse: nse, + protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix()}, + } +} + +func (l *LightClientOptimisticUpdateTopicFamily) Validator() wrappedVal { + return l.syncService.validateLightClientOptimisticUpdate +} + +func (l *LightClientOptimisticUpdateTopicFamily) Handler() subHandler { + return noopHandler +} + +func (l *LightClientOptimisticUpdateTopicFamily) GetFullTopicString() string { + return fmt.Sprintf(p2p.LightClientOptimisticUpdateTopicFormat, l.nse.ForkDigest) + l.protocolSuffix +} + +func (l *LightClientOptimisticUpdateTopicFamily) Subscribe() { + l.syncService.subscribe(l) +} + +func (l *LightClientOptimisticUpdateTopicFamily) Unsubscribe() { + l.syncService.unSubscribeFromTopic(l.GetFullTopicString()) +} + +// Light Client Finality Update (Altair+) +var _ GossipsubTopicFamilyWithoutDynamicSubnets = (*LightClientFinalityUpdateTopicFamily)(nil) + +type LightClientFinalityUpdateTopicFamily struct { + baseGossipsubTopicFamily +} + +func NewLightClientFinalityUpdateTopicFamily(s *Service, nse params.NetworkScheduleEntry) *LightClientFinalityUpdateTopicFamily { + return &LightClientFinalityUpdateTopicFamily{ + baseGossipsubTopicFamily{ + syncService: s, + nse: nse, + protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix()}, + } +} + +func (l *LightClientFinalityUpdateTopicFamily) Validator() wrappedVal { + return l.syncService.validateLightClientFinalityUpdate +} + +func (l *LightClientFinalityUpdateTopicFamily) Handler() subHandler { + return noopHandler +} + +func (l *LightClientFinalityUpdateTopicFamily) GetFullTopicString() string { + return fmt.Sprintf(p2p.LightClientFinalityUpdateTopicFormat, l.nse.ForkDigest) + l.protocolSuffix +} + +func (l *LightClientFinalityUpdateTopicFamily) Subscribe() { + l.syncService.subscribe(l) +} +func (l *LightClientFinalityUpdateTopicFamily) Unsubscribe() { + l.syncService.unSubscribeFromTopic(l.GetFullTopicString()) +} + +// BLS to Execution Change (Capella+) +var _ GossipsubTopicFamilyWithoutDynamicSubnets = (*BlsToExecutionChangeTopicFamily)(nil) + +type BlsToExecutionChangeTopicFamily struct { + baseGossipsubTopicFamily +} + +func NewBlsToExecutionChangeTopicFamily(s *Service, nse params.NetworkScheduleEntry) *BlsToExecutionChangeTopicFamily { + return &BlsToExecutionChangeTopicFamily{ + baseGossipsubTopicFamily{ + syncService: s, + nse: nse, + protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix()}, + } +} + +func (b *BlsToExecutionChangeTopicFamily) Validator() wrappedVal { + return b.syncService.validateBlsToExecutionChange +} + +func (b *BlsToExecutionChangeTopicFamily) Handler() subHandler { + return b.syncService.blsToExecutionChangeSubscriber +} + +func (b *BlsToExecutionChangeTopicFamily) GetFullTopicString() string { + return fmt.Sprintf(p2p.BlsToExecutionChangeSubnetTopicFormat, b.nse.ForkDigest) + b.protocolSuffix +} + +func (b *BlsToExecutionChangeTopicFamily) Subscribe() { + b.syncService.subscribe(b) +} + +func (b *BlsToExecutionChangeTopicFamily) Unsubscribe() { + b.syncService.unSubscribeFromTopic(b.GetFullTopicString()) +} From 7853cb9db0a511f1ad31eb3cdb52862bfe9942f0 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Mon, 3 Nov 2025 19:14:46 +0400 Subject: [PATCH 02/39] changelog fragment --- changelog/aarshkshah1992-gossipsub-control-pane.md | 4 ++++ 1 file changed, 4 insertions(+) create mode 100644 changelog/aarshkshah1992-gossipsub-control-pane.md diff --git a/changelog/aarshkshah1992-gossipsub-control-pane.md b/changelog/aarshkshah1992-gossipsub-control-pane.md new file mode 100644 index 000000000000..a369fb02d1eb --- /dev/null +++ b/changelog/aarshkshah1992-gossipsub-control-pane.md @@ -0,0 +1,4 @@ +### Added + +- A Gossipsub controller that uses well defined inetrfaces and implementations for ALL topics (subnetted and otherwise) +that Prysm subscribes to to the hide the gnarly details of gossipsub subscription management. \ No newline at end of file From ce3660d2e7079fd207d05c0799089c07631ba7fe Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Tue, 4 Nov 2025 20:08:11 +0400 Subject: [PATCH 03/39] finish draft --- beacon-chain/p2p/broadcaster.go | 19 +- beacon-chain/p2p/interfaces.go | 11 +- beacon-chain/p2p/service.go | 6 +- beacon-chain/p2p/subnets.go | 32 ++-- beacon-chain/p2p/subnets_test.go | 27 ++- beacon-chain/p2p/testing/fuzz_p2p.go | 3 +- beacon-chain/p2p/testing/mock_peermanager.go | 3 +- beacon-chain/p2p/testing/p2p.go | 9 +- beacon-chain/sync/fork_watcher_test.go | 26 +-- beacon-chain/sync/gossipsub_controller.go | 4 + beacon-chain/sync/service_test.go | 23 ++- beacon-chain/sync/subscriber.go | 153 +++++----------- beacon-chain/sync/subscriber_test.go | 170 ++++++++++++++---- .../sync/topic_families_dynamic_subnets.go | 94 ++++++---- .../sync/topic_families_static_subnets.go | 2 +- .../sync/topic_families_without_subnets.go | 18 +- 16 files changed, 341 insertions(+), 259 deletions(-) diff --git a/beacon-chain/p2p/broadcaster.go b/beacon-chain/p2p/broadcaster.go index 03b38e29acb0..ec09343ce788 100644 --- a/beacon-chain/p2p/broadcaster.go +++ b/beacon-chain/p2p/broadcaster.go @@ -131,7 +131,8 @@ func (s *Service) internalBroadcastAttestation(ctx context.Context, subnet uint6 s.subnetLocker(subnet).Lock() defer s.subnetLocker(subnet).Unlock() - if err := s.FindAndDialPeersWithSubnets(ctx, AttestationSubnetTopicFormat, forkDigest, minimumPeersPerSubnetForBroadcast, map[uint64]bool{subnet: true}); err != nil { + builder := func(idx uint64) string { return attestationToTopic(idx, forkDigest) + s.Encoding().ProtocolSuffix() } + if err := s.FindAndDialPeersWithSubnets(ctx, builder, minimumPeersPerSubnetForBroadcast, map[uint64]bool{subnet: true}); err != nil { return errors.Wrap(err, "find peers with subnets") } @@ -187,7 +188,8 @@ func (s *Service) broadcastSyncCommittee(ctx context.Context, subnet uint64, sMs if err := func() error { s.subnetLocker(wrappedSubIdx).Lock() defer s.subnetLocker(wrappedSubIdx).Unlock() - if err := s.FindAndDialPeersWithSubnets(ctx, SyncCommitteeSubnetTopicFormat, forkDigest, minimumPeersPerSubnetForBroadcast, map[uint64]bool{subnet: true}); err != nil { + builder := func(idx uint64) string { return syncCommitteeToTopic(idx, forkDigest) + s.Encoding().ProtocolSuffix() } + if err := s.FindAndDialPeersWithSubnets(ctx, builder, minimumPeersPerSubnetForBroadcast, map[uint64]bool{subnet: true}); err != nil { return errors.Wrap(err, "find peers with subnets") } @@ -252,7 +254,8 @@ func (s *Service) internalBroadcastBlob(ctx context.Context, subnet uint64, blob s.subnetLocker(wrappedSubIdx).Lock() defer s.subnetLocker(wrappedSubIdx).Unlock() - if err := s.FindAndDialPeersWithSubnets(ctx, BlobSubnetTopicFormat, forkDigest, minimumPeersPerSubnetForBroadcast, map[uint64]bool{subnet: true}); err != nil { + builder := func(idx uint64) string { return blobSubnetToTopic(idx, forkDigest) + s.Encoding().ProtocolSuffix() } + if err := s.FindAndDialPeersWithSubnets(ctx, builder, minimumPeersPerSubnetForBroadcast, map[uint64]bool{subnet: true}); err != nil { return errors.Wrap(err, "find peers with subnets") } @@ -392,7 +395,10 @@ func (s *Service) broadcastDataColumnSidecars(ctx context.Context, forkDigest [f wrappedSubIdx := subnet + dataColumnSubnetVal // Find peers if needed. - if err := s.findPeersIfNeeded(ctx, wrappedSubIdx, DataColumnSubnetTopicFormat, forkDigest, subnet); err != nil { + builder := func(idx uint64) string { + return dataColumnSubnetToTopic(idx, forkDigest) + s.Encoding().ProtocolSuffix() + } + if err := s.findPeersIfNeeded(ctx, wrappedSubIdx, builder, subnet); err != nil { tracing.AnnotateError(span, err) log.WithError(err).Error("Cannot find peers if needed") return @@ -487,8 +493,7 @@ func (s *Service) broadcastDataColumnSidecars(ctx context.Context, forkDigest [f func (s *Service) findPeersIfNeeded( ctx context.Context, wrappedSubIdx uint64, - topicFormat string, - forkDigest [fieldparams.VersionLength]byte, + fullTopicForSubnet func(uint64) string, subnet uint64, ) error { // Sending a data column sidecar to only one peer is not ideal, @@ -497,7 +502,7 @@ func (s *Service) findPeersIfNeeded( defer s.subnetLocker(wrappedSubIdx).Unlock() // No peers found, attempt to find peers with this subnet. - if err := s.FindAndDialPeersWithSubnets(ctx, topicFormat, forkDigest, minimumPeersPerSubnetForBroadcast, map[uint64]bool{subnet: true}); err != nil { + if err := s.FindAndDialPeersWithSubnets(ctx, fullTopicForSubnet, minimumPeersPerSubnetForBroadcast, map[uint64]bool{subnet: true}); err != nil { return errors.Wrap(err, "find peers with subnet") } diff --git a/beacon-chain/p2p/interfaces.go b/beacon-chain/p2p/interfaces.go index c04f211d9d90..82e79b3f987c 100644 --- a/beacon-chain/p2p/interfaces.go +++ b/beacon-chain/p2p/interfaces.go @@ -5,7 +5,6 @@ import ( "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/encoder" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers" - fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams" "github.com/OffchainLabs/prysm/v6/consensus-types/blocks" "github.com/OffchainLabs/prysm/v6/consensus-types/interfaces" "github.com/OffchainLabs/prysm/v6/consensus-types/primitives" @@ -35,7 +34,6 @@ type ( PeersProvider MetadataProvider CustodyManager - SubnetFilter } // Accessor provides access to the Broadcaster, PeerManager and CustodyManager interfaces. @@ -102,7 +100,7 @@ type ( NodeID() enode.ID DiscoveryAddresses() ([]multiaddr.Multiaddr, error) RefreshPersistentSubnets() - FindAndDialPeersWithSubnets(ctx context.Context, topicFormat string, digest [fieldparams.VersionLength]byte, minimumPeersPerSubnet int, subnets map[uint64]bool) error + FindAndDialPeersWithSubnets(ctx context.Context, fullTopicForSubnet func(uint64) string, minimumPeersPerSubnet int, subnets map[uint64]bool) error AddPingMethod(reqFunc func(ctx context.Context, id peer.ID) error) } @@ -130,11 +128,4 @@ type ( UpdateEarliestAvailableSlot(earliestAvailableSlot primitives.Slot) error CustodyGroupCountFromPeer(peer.ID) uint64 } - - // SubnetFilter provides methods for extracting subnet information from a peer's ENR records. - SubnetFilter interface { - AttestationSubnets(nodeID enode.ID, node *enode.Node, record *enr.Record) (map[uint64]bool, error) - SyncSubnets(nodeID enode.ID, node *enode.Node, record *enr.Record) (map[uint64]bool, error) - DataColumnSubnets(nodeID enode.ID, node *enode.Node, record *enr.Record) (map[uint64]bool, error) - } ) diff --git a/beacon-chain/p2p/service.go b/beacon-chain/p2p/service.go index 85c81427f830..c9e3a916b257 100644 --- a/beacon-chain/p2p/service.go +++ b/beacon-chain/p2p/service.go @@ -558,14 +558,14 @@ func (s *Service) downscorePeer(peerID peer.ID, reason string) { log.WithFields(logrus.Fields{"peerID": peerID, "reason": reason, "newScore": newScore}).Debug("Downscore peer") } -func (s *Service) AttestationSubnets(nodeID enode.ID, node *enode.Node, record *enr.Record) (map[uint64]bool, error) { +func AttestationSubnets(nodeID enode.ID, node *enode.Node, record *enr.Record) (map[uint64]bool, error) { return attestationSubnets(record) } -func (s *Service) SyncSubnets(nodeID enode.ID, node *enode.Node, record *enr.Record) (map[uint64]bool, error) { +func SyncSubnets(nodeID enode.ID, node *enode.Node, record *enr.Record) (map[uint64]bool, error) { return syncSubnets(record) } -func (s *Service) DataColumnSubnets(nodeID enode.ID, node *enode.Node, record *enr.Record) (map[uint64]bool, error) { +func DataColumnSubnets(nodeID enode.ID, node *enode.Node, record *enr.Record) (map[uint64]bool, error) { return dataColumnSubnets(nodeID, record) } diff --git a/beacon-chain/p2p/subnets.go b/beacon-chain/p2p/subnets.go index 13638dd3bd38..a398d5d1ed2f 100644 --- a/beacon-chain/p2p/subnets.go +++ b/beacon-chain/p2p/subnets.go @@ -12,7 +12,6 @@ import ( "github.com/OffchainLabs/prysm/v6/beacon-chain/core/helpers" "github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas" "github.com/OffchainLabs/prysm/v6/cmd/beacon-chain/flags" - fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams" "github.com/OffchainLabs/prysm/v6/config/params" "github.com/OffchainLabs/prysm/v6/consensus-types/primitives" "github.com/OffchainLabs/prysm/v6/consensus-types/wrapper" @@ -84,8 +83,7 @@ func (s *Service) nodeFilter(topic string, indices map[uint64]int) (func(node *e // In this case, the function returns an error. func (s *Service) FindAndDialPeersWithSubnets( ctx context.Context, - topicFormat string, - digest [fieldparams.VersionLength]byte, + fullTopicForSubnet func(uint64) string, minimumPeersPerSubnet int, subnets map[uint64]bool, ) error { @@ -103,7 +101,7 @@ func (s *Service) FindAndDialPeersWithSubnets( maxConcurrentDials = flags.Get().MaxConcurrentDials } - defectiveSubnets := s.defectiveSubnets(topicFormat, digest, minimumPeersPerSubnet, subnets) + defectiveSubnets := s.defectiveSubnets(fullTopicForSubnet, minimumPeersPerSubnet, subnets) for len(defectiveSubnets) > 0 { // Stop the search/dialing loop if the context is canceled. if err := ctx.Err(); err != nil { @@ -114,7 +112,7 @@ func (s *Service) FindAndDialPeersWithSubnets( ctx, cancel := context.WithTimeout(ctx, batchPeriod) defer cancel() - peersToDial, err := s.findPeersWithSubnets(ctx, topicFormat, digest, minimumPeersPerSubnet, defectiveSubnets) + peersToDial, err := s.findPeersWithSubnets(ctx, fullTopicForSubnet, minimumPeersPerSubnet, defectiveSubnets) if err != nil && !errors.Is(err, context.DeadlineExceeded) { return nil, errors.Wrap(err, "find peers with subnets") } @@ -129,7 +127,7 @@ func (s *Service) FindAndDialPeersWithSubnets( // Dial new peers in batches. s.dialPeers(s.ctx, maxConcurrentDials, peersToDial) - defectiveSubnets = s.defectiveSubnets(topicFormat, digest, minimumPeersPerSubnet, subnets) + defectiveSubnets = s.defectiveSubnets(fullTopicForSubnet, minimumPeersPerSubnet, subnets) } return nil @@ -158,8 +156,7 @@ func updateDefectiveSubnets( // It returns new peers found during the search. func (s *Service) findPeersWithSubnets( ctx context.Context, - topicFormat string, - digest [fieldparams.VersionLength]byte, + fullTopicForSubnet func(uint64) string, minimumPeersPerSubnet int, defectiveSubnetsOrigin map[uint64]int, ) ([]*enode.Node, error) { @@ -181,7 +178,13 @@ func (s *Service) findPeersWithSubnets( }() // Retrieve the filter function that will be used to filter nodes based on the defective subnets. - filter, err := s.nodeFilter(topicFormat, defectiveSubnets) + // Use any subnet's full topic to infer the family type from the topic string. + var sampleTopic string + for k := range defectiveSubnets { + sampleTopic = fullTopicForSubnet(k) + break + } + filter, err := s.nodeFilter(sampleTopic, defectiveSubnets) if err != nil { return nil, errors.Wrap(err, "node filter") } @@ -225,8 +228,8 @@ func (s *Service) findPeersWithSubnets( nodeSubnets, err := filter(node) if err != nil { log.WithError(err).WithFields(logrus.Fields{ - "nodeID": node.ID(), - "topicFormat": topicFormat, + "nodeID": node.ID(), + "topic": sampleTopic, }).Debug("Could not get needed subnets from peer") continue @@ -241,7 +244,7 @@ func (s *Service) findPeersWithSubnets( nodeByNodeID[node.ID()] = node updateDefectiveSubnets(nodeSubnets, defectiveSubnets) - filter, err = s.nodeFilter(topicFormat, defectiveSubnets) + filter, err = s.nodeFilter(sampleTopic, defectiveSubnets) if err != nil { return nil, errors.Wrap(err, "node filter") } @@ -258,14 +261,13 @@ func (s *Service) findPeersWithSubnets( // defectiveSubnets returns a map of subnets that have fewer than the minimum peer count. func (s *Service) defectiveSubnets( - topicFormat string, - digest [fieldparams.VersionLength]byte, + fullTopicForSubnet func(uint64) string, minimumPeersPerSubnet int, subnets map[uint64]bool, ) map[uint64]int { missingCountPerSubnet := make(map[uint64]int, len(subnets)) for subnet := range subnets { - topic := fmt.Sprintf(topicFormat, digest, subnet) + s.Encoding().ProtocolSuffix() + topic := fullTopicForSubnet(subnet) peers := s.pubsub.ListPeers(topic) peerCount := len(peers) if peerCount < minimumPeersPerSubnet { diff --git a/beacon-chain/p2p/subnets_test.go b/beacon-chain/p2p/subnets_test.go index ad2855af6b02..ca8371aebd4b 100644 --- a/beacon-chain/p2p/subnets_test.go +++ b/beacon-chain/p2p/subnets_test.go @@ -168,16 +168,19 @@ func TestStartDiscV5_FindAndDialPeersWithSubnet(t *testing.T) { }() subnets := map[uint64]bool{1: true, 2: true, 3: true} - defectiveSubnets := service.defectiveSubnets(AttestationSubnetTopicFormat, bootNodeForkDigest, minimumPeersPerSubnet, subnets) + builder := func(idx uint64) string { + return fmt.Sprintf(AttestationSubnetTopicFormat, bootNodeForkDigest, idx) + service.Encoding().ProtocolSuffix() + } + defectiveSubnets := service.defectiveSubnets(builder, minimumPeersPerSubnet, subnets) require.Equal(t, subnetCount, len(defectiveSubnets)) ctxWithTimeOut, cancel := context.WithTimeout(ctx, 5*time.Second) defer cancel() - err = service.FindAndDialPeersWithSubnets(ctxWithTimeOut, AttestationSubnetTopicFormat, bootNodeForkDigest, minimumPeersPerSubnet, subnets) + err = service.FindAndDialPeersWithSubnets(ctxWithTimeOut, builder, minimumPeersPerSubnet, subnets) require.NoError(t, err) - defectiveSubnets = service.defectiveSubnets(AttestationSubnetTopicFormat, bootNodeForkDigest, minimumPeersPerSubnet, subnets) + defectiveSubnets = service.defectiveSubnets(builder, minimumPeersPerSubnet, subnets) require.Equal(t, 0, len(defectiveSubnets)) } @@ -762,10 +765,12 @@ func TestFindPeersWithSubnets_NodeDeduplication(t *testing.T) { ctxWithTimeout, cancel := context.WithTimeout(ctx, 100*time.Millisecond) defer cancel() + builder := func(idx uint64) string { + return fmt.Sprintf(AttestationSubnetTopicFormat, digest, idx) + s.Encoding().ProtocolSuffix() + } result, err := s.findPeersWithSubnets( ctxWithTimeout, - AttestationSubnetTopicFormat, - digest, + builder, 1, tt.defectiveSubnets, ) @@ -982,10 +987,12 @@ func TestFindPeersWithSubnets_FilterPeerRemoval(t *testing.T) { ctxWithTimeout, cancel := context.WithTimeout(ctx, 100*time.Millisecond) defer cancel() + builder := func(idx uint64) string { + return fmt.Sprintf(AttestationSubnetTopicFormat, digest, idx) + s.Encoding().ProtocolSuffix() + } result, err := s.findPeersWithSubnets( ctxWithTimeout, - AttestationSubnetTopicFormat, - digest, + builder, 1, tt.defectiveSubnets, ) @@ -1105,10 +1112,12 @@ func TestFindPeersWithSubnets_received_bad_existing_node(t *testing.T) { ctxWithTimeout, cancel := context.WithTimeout(ctx, 1*time.Second) defer cancel() + builder := func(idx uint64) string { + return fmt.Sprintf(AttestationSubnetTopicFormat, digest, idx) + service.Encoding().ProtocolSuffix() + } result, err := service.findPeersWithSubnets( ctxWithTimeout, - AttestationSubnetTopicFormat, - digest, + builder, 1, map[uint64]int{1: 2}, // Need 2 peers for subnet 1 ) diff --git a/beacon-chain/p2p/testing/fuzz_p2p.go b/beacon-chain/p2p/testing/fuzz_p2p.go index b9dbb71e413b..4920d4921808 100644 --- a/beacon-chain/p2p/testing/fuzz_p2p.go +++ b/beacon-chain/p2p/testing/fuzz_p2p.go @@ -5,7 +5,6 @@ import ( "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/encoder" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers" - fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams" "github.com/OffchainLabs/prysm/v6/consensus-types/blocks" "github.com/OffchainLabs/prysm/v6/consensus-types/interfaces" "github.com/OffchainLabs/prysm/v6/consensus-types/primitives" @@ -71,7 +70,7 @@ func (*FakeP2P) DiscoveryAddresses() ([]multiaddr.Multiaddr, error) { } // FindAndDialPeersWithSubnets mocks the p2p func. -func (*FakeP2P) FindAndDialPeersWithSubnets(ctx context.Context, topicFormat string, digest [fieldparams.VersionLength]byte, minimumPeersPerSubnet int, subnets map[uint64]bool) error { +func (*FakeP2P) FindAndDialPeersWithSubnets(ctx context.Context, fullTopicForSubnet func(uint64) string, minimumPeersPerSubnet int, subnets map[uint64]bool) error { return nil } diff --git a/beacon-chain/p2p/testing/mock_peermanager.go b/beacon-chain/p2p/testing/mock_peermanager.go index fed8926c67aa..83bb656a637f 100644 --- a/beacon-chain/p2p/testing/mock_peermanager.go +++ b/beacon-chain/p2p/testing/mock_peermanager.go @@ -4,7 +4,6 @@ import ( "context" "errors" - fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams" "github.com/ethereum/go-ethereum/p2p/enode" "github.com/ethereum/go-ethereum/p2p/enr" "github.com/libp2p/go-libp2p/core/host" @@ -58,7 +57,7 @@ func (m *MockPeerManager) DiscoveryAddresses() ([]multiaddr.Multiaddr, error) { func (*MockPeerManager) RefreshPersistentSubnets() {} // FindAndDialPeersWithSubnet . -func (*MockPeerManager) FindAndDialPeersWithSubnets(ctx context.Context, topicFormat string, digest [fieldparams.VersionLength]byte, minimumPeersPerSubnet int, subnets map[uint64]bool) error { +func (*MockPeerManager) FindAndDialPeersWithSubnets(ctx context.Context, fullTopicForSubnet func(uint64) string, minimumPeersPerSubnet int, subnets map[uint64]bool) error { return nil } diff --git a/beacon-chain/p2p/testing/p2p.go b/beacon-chain/p2p/testing/p2p.go index 641172ea07af..d3be2285f0b8 100644 --- a/beacon-chain/p2p/testing/p2p.go +++ b/beacon-chain/p2p/testing/p2p.go @@ -15,7 +15,6 @@ import ( "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/encoder" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers/scorers" - fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams" "github.com/OffchainLabs/prysm/v6/config/params" "github.com/OffchainLabs/prysm/v6/consensus-types/blocks" "github.com/OffchainLabs/prysm/v6/consensus-types/interfaces" @@ -183,11 +182,7 @@ func (p *TestP2P) ReceivePubSub(topic string, msg proto.Message) { if _, err := p.Encoding().EncodeGossip(buf, castedMsg); err != nil { p.t.Fatalf("Failed to encode message: %v", err) } - digest, err := p.ForkDigest() - if err != nil { - p.t.Fatal(err) - } - topicHandle, err := ps.Join(fmt.Sprintf(topic, digest) + p.Encoding().ProtocolSuffix()) + topicHandle, err := ps.Join(topic) if err != nil { p.t.Fatal(err) } @@ -420,7 +415,7 @@ func (p *TestP2P) Peers() *peers.Status { } // FindAndDialPeersWithSubnets mocks the p2p func. -func (*TestP2P) FindAndDialPeersWithSubnets(ctx context.Context, topicFormat string, digest [fieldparams.VersionLength]byte, minimumPeersPerSubnet int, subnets map[uint64]bool) error { +func (*TestP2P) FindAndDialPeersWithSubnets(ctx context.Context, fullTopicForSubnet func(uint64) string, minimumPeersPerSubnet int, subnets map[uint64]bool) error { return nil } diff --git a/beacon-chain/sync/fork_watcher_test.go b/beacon-chain/sync/fork_watcher_test.go index e6466a763e75..45e2ae17828b 100644 --- a/beacon-chain/sync/fork_watcher_test.go +++ b/beacon-chain/sync/fork_watcher_test.go @@ -2,22 +2,18 @@ package sync import ( "context" - "fmt" "sync" "testing" "time" "github.com/OffchainLabs/prysm/v6/async/abool" mockChain "github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/testing" - "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p" p2ptest "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/testing" "github.com/OffchainLabs/prysm/v6/beacon-chain/startup" mockSync "github.com/OffchainLabs/prysm/v6/beacon-chain/sync/initial-sync/testing" "github.com/OffchainLabs/prysm/v6/config/params" "github.com/OffchainLabs/prysm/v6/consensus-types/primitives" "github.com/OffchainLabs/prysm/v6/genesis" - "github.com/OffchainLabs/prysm/v6/testing/assert" - "github.com/OffchainLabs/prysm/v6/testing/require" ) func defaultClockWithTimeAtEpoch(epoch primitives.Epoch) *startup.Clock { @@ -50,6 +46,8 @@ func testForkWatcherService(t *testing.T, current primitives.Epoch) *Service { return r } +// TODO: Move to gossip controller test +/* func TestRegisterSubscriptions_Idempotent(t *testing.T) { params.SetupTestConfigCleanup(t) genesis.StoreEmbeddedDuringTest(t, params.BeaconConfig().ConfigName) @@ -71,9 +69,11 @@ func TestRegisterSubscriptions_Idempotent(t *testing.T) { } // the goal of this callback is just to assert that spawn is never called. s.subscriptionSpawner = func(func()) { t.Error("registration routines spawned twice for the same digest") } - require.NoError(t, s.ensureRegistrationsForEpoch(fulu)) -} + require.NoError(t, s.ensureRPCRegistrationsForEpoch(fulu)) +}*/ +// TODO: Move to gossip controller test +/* func TestService_CheckForNextEpochFork(t *testing.T) { closedChan := make(chan struct{}) close(closedChan) @@ -121,8 +121,8 @@ func TestService_CheckForNextEpochFork(t *testing.T) { expected := topic + s.cfg.p2p.Encoding().ProtocolSuffix() assert.Equal(t, true, s.subHandler.topicExists(expected), fmt.Sprintf("subnet topic %s doesn't exist", expected)) } - */ - }, +*/ +/*}, }, { name: "capella fork in the next epoch", @@ -195,7 +195,7 @@ func TestService_CheckForNextEpochFork(t *testing.T) { current := tt.epochAtRegistration(tt.forkEpoch) s := testForkWatcherService(t, current) wg := attachSpawner(s) - require.NoError(t, s.ensureRegistrationsForEpoch(s.cfg.clock.CurrentEpoch())) + require.NoError(t, s.ensureRPCRegistrationsForEpoch(s.cfg.clock.CurrentEpoch())) wg.Wait() tt.checkRegistration(t, s) @@ -217,18 +217,18 @@ func TestService_CheckForNextEpochFork(t *testing.T) { // Move the clock to just before the next fork epoch and ensure deregistration is correct wg = attachSpawner(s) s.cfg.clock = defaultClockWithTimeAtEpoch(tt.nextForkEpoch - 1) - require.NoError(t, s.ensureRegistrationsForEpoch(s.cfg.clock.CurrentEpoch())) + require.NoError(t, s.ensureRPCRegistrationsForEpoch(s.cfg.clock.CurrentEpoch())) wg.Wait() - require.NoError(t, s.ensureDeregistrationForEpoch(tt.nextForkEpoch)) + require.NoError(t, s.ensureRPCDeregistrationForEpoch(tt.nextForkEpoch)) assert.Equal(t, true, s.subHandler.digestExists(digest)) // deregister as if it is the epoch after the next fork epoch - require.NoError(t, s.ensureDeregistrationForEpoch(tt.nextForkEpoch+1)) + require.NoError(t, s.ensureRPCDeregistrationForEpoch(tt.nextForkEpoch+1)) assert.Equal(t, false, s.subHandler.digestExists(digest)) assert.Equal(t, true, s.subHandler.digestExists(nextDigest)) }) } -} +}*/ func attachSpawner(s *Service) *sync.WaitGroup { wg := new(sync.WaitGroup) diff --git a/beacon-chain/sync/gossipsub_controller.go b/beacon-chain/sync/gossipsub_controller.go index 8edfaf74461b..21e8dbdc7147 100644 --- a/beacon-chain/sync/gossipsub_controller.go +++ b/beacon-chain/sync/gossipsub_controller.go @@ -44,6 +44,10 @@ func NewGossipsubController(ctx context.Context, s *Service) *GossipsubControlle func (g *GossipsubController) Start() { currentEpoch := g.syncService.cfg.clock.CurrentEpoch() + if err := g.syncService.waitForInitialSync(g.ctx); err != nil { + log.WithError(err).Debug("Context cancelled while waiting for initial sync, not starting GossipsubController") + return + } g.updateActiveTopicFamilies(currentEpoch) g.wg.Go(func() { g.controlLoop() }) diff --git a/beacon-chain/sync/service_test.go b/beacon-chain/sync/service_test.go index b75103ebdc6c..572e18a64a71 100644 --- a/beacon-chain/sync/service_test.go +++ b/beacon-chain/sync/service_test.go @@ -2,6 +2,7 @@ package sync import ( "context" + "fmt" "sync" "testing" "time" @@ -16,6 +17,7 @@ import ( "github.com/OffchainLabs/prysm/v6/beacon-chain/startup" state_native "github.com/OffchainLabs/prysm/v6/beacon-chain/state/state-native" mockSync "github.com/OffchainLabs/prysm/v6/beacon-chain/sync/initial-sync/testing" + "github.com/OffchainLabs/prysm/v6/config/params" "github.com/OffchainLabs/prysm/v6/consensus-types/primitives" leakybucket "github.com/OffchainLabs/prysm/v6/container/leaky-bucket" "github.com/OffchainLabs/prysm/v6/crypto/bls" @@ -67,8 +69,9 @@ func TestSyncHandlers_WaitToSync(t *testing.T) { chainStarted: abool.New(), clockWaiter: gs, } + r.gossipsubController = NewGossipsubController(context.Background(), &r) - topic := "/eth2/%x/beacon_block" + topicFmt := "/eth2/%x/beacon_block" go r.startDiscoveryAndSubscriptions() time.Sleep(100 * time.Millisecond) @@ -82,7 +85,10 @@ func TestSyncHandlers_WaitToSync(t *testing.T) { msg := util.NewBeaconBlock() msg.Block.ParentRoot = util.Random32Bytes(t) msg.Signature = sk.Sign([]byte("data")).Marshal() - p2p.ReceivePubSub(topic, msg) + // Build full topic using current fork digest + nse := params.GetNetworkScheduleEntry(r.cfg.clock.CurrentEpoch()) + fullTopic := fmt.Sprintf(topicFmt, nse.ForkDigest) + p2p.Encoding().ProtocolSuffix() + p2p.ReceivePubSub(fullTopic, msg) // wait for chainstart to be sent time.Sleep(400 * time.Millisecond) require.Equal(t, true, r.chainStarted.IsSet(), "Did not receive chain start event.") @@ -137,6 +143,7 @@ func TestSyncHandlers_WaitTillSynced(t *testing.T) { clockWaiter: gs, initialSyncComplete: make(chan struct{}), } + r.gossipsubController = NewGossipsubController(context.Background(), &r) r.initCaches() var vr [32]byte @@ -169,14 +176,16 @@ func TestSyncHandlers_WaitTillSynced(t *testing.T) { // Save block into DB so that validateBeaconBlockPubSub() process gets short cut. util.SaveBlock(t, ctx, r.cfg.beaconDB, msg) - topic := "/eth2/%x/beacon_block" - p2p.ReceivePubSub(topic, msg) + topicFmt := "/eth2/%x/beacon_block" + nse := params.GetNetworkScheduleEntry(r.cfg.clock.CurrentEpoch()) + fullTopic := fmt.Sprintf(topicFmt, nse.ForkDigest) + p2p.Encoding().ProtocolSuffix() + p2p.ReceivePubSub(fullTopic, msg) assert.Equal(t, 0, len(blockChan), "block was received by sync service despite not being fully synced") close(r.initialSyncComplete) <-syncCompleteCh - p2p.ReceivePubSub(topic, msg) + p2p.ReceivePubSub(fullTopic, msg) select { case <-blockChan: @@ -206,6 +215,7 @@ func TestSyncService_StopCleanly(t *testing.T) { clockWaiter: gs, initialSyncComplete: make(chan struct{}), } + r.gossipsubController = NewGossipsubController(context.Background(), &r) markInitSyncComplete(t, &r) go r.startDiscoveryAndSubscriptions() @@ -265,6 +275,7 @@ func TestService_Stop_SendsGoodbyeMessages(t *testing.T) { cancel: cancel, rateLimiter: newRateLimiter(p1), } + r.gossipsubController = NewGossipsubController(ctx, r) // Initialize context map for RPC ctxMap, err := ContextByteVersionsForValRoot(chain.ValidatorsRoot) @@ -343,6 +354,7 @@ func TestService_Stop_TimeoutHandling(t *testing.T) { cancel: cancel, rateLimiter: newRateLimiter(p1), } + r.gossipsubController = NewGossipsubController(ctx, r) // Initialize context map for RPC ctxMap, err := ContextByteVersionsForValRoot(chain.ValidatorsRoot) @@ -404,6 +416,7 @@ func TestService_Stop_ConcurrentGoodbyeMessages(t *testing.T) { cancel: cancel, rateLimiter: newRateLimiter(p1), } + r.gossipsubController = NewGossipsubController(ctx, r) // Initialize context map for RPC ctxMap, err := ContextByteVersionsForValRoot(chain.ValidatorsRoot) diff --git a/beacon-chain/sync/subscriber.go b/beacon-chain/sync/subscriber.go index 1ed6fc208578..e6d91e715f57 100644 --- a/beacon-chain/sync/subscriber.go +++ b/beacon-chain/sync/subscriber.go @@ -38,53 +38,27 @@ const pubsubMessageTimeout = 30 * time.Second var errInvalidDigest = errors.New("invalid digest") -// subscribeParameters holds the parameters that are needed to construct a set of subscriptions topics for a given -// set of gossipsub subnets. -type subscribeParameters struct { - topicFormat string - validate wrappedVal - handle subHandler - nse params.NetworkScheduleEntry - // getSubnetsToJoin is a function that returns all subnets the node should join. - getSubnetsToJoin func(currentSlot primitives.Slot) map[uint64]bool - // getSubnetsRequiringPeers is a function that returns all subnets that require peers to be found - // but for which no subscriptions are needed. - getSubnetsRequiringPeers func(currentSlot primitives.Slot) map[uint64]bool -} - -// shortTopic is a less verbose version of topic strings used for logging. -func (p subscribeParameters) shortTopic() string { - short := p.topicFormat - fmtLen := len(short) - if fmtLen >= 3 && short[fmtLen-3:] == "_%d" { - short = short[:fmtLen-3] - } - return fmt.Sprintf(short, p.nse.ForkDigest) -} - -func (p subscribeParameters) logFields() logrus.Fields { +func familyLogFields(tf GossipsubTopicFamilyWithDynamicSubnets) logrus.Fields { + nse := tf.NetworkScheduleEntry() return logrus.Fields{ - "topic": p.shortTopic(), + "topicFamily": fmt.Sprintf("%T", tf), + "digest": nse.ForkDigest, + "forkEpoch": nse.Epoch, } } -// fullTopic is the fully qualified topic string, given to gossipsub. -func (p subscribeParameters) fullTopic(subnet uint64, suffix string) string { - return fmt.Sprintf(p.topicFormat, p.nse.ForkDigest, subnet) + suffix -} - -// subnetTracker keeps track of which subnets we are subscribed to, out of the set of -// possible subnets described by a `subscribeParameters`. +// subnetTracker keeps track of which subnets we are subscribed to for a given +// dynamic topic family (attestations, sync-committee, data-column, etc.). type subnetTracker struct { - subscribeParameters + family GossipsubTopicFamilyWithDynamicSubnets mu sync.RWMutex subscriptions map[uint64]*pubsub.Subscription } -func newSubnetTracker(p subscribeParameters) *subnetTracker { +func newSubnetTracker(tf GossipsubTopicFamilyWithDynamicSubnets) *subnetTracker { return &subnetTracker{ - subscribeParameters: p, - subscriptions: make(map[uint64]*pubsub.Subscription), + family: tf, + subscriptions: make(map[uint64]*pubsub.Subscription), } } @@ -181,50 +155,12 @@ func (s *Service) activeSyncSubnetIndices(currentSlot primitives.Slot) map[uint6 return mapFromSlice(subscriptions) } -// spawn allows the Service to use a custom function for launching goroutines. -// This is useful in tests where we can set spawner to a sync.WaitGroup and -// wait for the spawned goroutines to finish. -func (s *Service) spawn(f func()) { - if s.subscriptionSpawner != nil { - s.subscriptionSpawner(f) - } else { - go f() - } -} - func (s *Service) subscriptionRequestExpired(nse params.NetworkScheduleEntry) bool { next := params.NextNetworkScheduleEntry(nse.Epoch) return next.Epoch != nse.Epoch && s.cfg.clock.CurrentEpoch() > next.Epoch } -func (s *Service) subscribeLogFields(topic string, nse params.NetworkScheduleEntry) logrus.Fields { - return logrus.Fields{ - "topic": topic, - "digest": nse.ForkDigest, - "forkEpoch": nse.Epoch, - "currentEpoch": s.cfg.clock.CurrentEpoch(), - } -} - -// 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(tf GossipsubTopicFamilyWithoutDynamicSubnets) { - topic := tf.GetFullTopicString() - nse := tf.NetworkScheduleEntry() - if err := s.waitForInitialSync(s.ctx); err != nil { - log.WithFields(s.subscribeLogFields(topic, nse)).WithError(err).Debug("Context cancelled while waiting for initial sync, not subscribing to topic") - return - } - // Check if this subscribe request is still valid - we may have crossed another fork epoch while waiting for initial sync. - if s.subscriptionRequestExpired(nse) { - // If we are already past the next fork epoch, do not subscribe to this topic. - log.WithFields(s.subscribeLogFields(topic, nse)).Debug("Not subscribing to topic as we are already past the next fork epoch") - return - } - s.subscribeWithBase(topic, tf.Validator(), tf.Handler()) -} - -func (s *Service) subscribeWithBase(topic string, validator wrappedVal, handle subHandler) *pubsub.Subscription { +func (s *Service) subscribe(topic string, validator wrappedVal, handle subHandler) *pubsub.Subscription { log := log.WithField("topic", topic) // Do not resubscribe already seen subscriptions. @@ -387,36 +323,38 @@ func (s *Service) wrapAndReportValidation(topic string, v wrappedVal) (string, p func (s *Service) pruneNotWanted(t *subnetTracker, wantedSubnets map[uint64]bool) { for _, subnet := range t.unwanted(wantedSubnets) { t.cancelSubscription(subnet) - s.unSubscribeFromTopic(t.fullTopic(subnet, s.cfg.p2p.Encoding().ProtocolSuffix())) + s.unSubscribeFromTopic(t.family.GetFullTopicString(subnet)) } } -// subscribeWithParameters subscribes to a list of subnets. -func (s *Service) subscribeWithParameters(p subscribeParameters) { +// subscribeToDynamicSubnetFamily subscribes to a list of subnets. +func (s *Service) subscribeToDynamicSubnetFamily(tf GossipsubTopicFamilyWithDynamicSubnets) *subnetTracker { + tracker := newSubnetTracker(tf) + go s.subscribeToSubnets(tf, tracker) + return tracker +} + +func (s *Service) subscribeToSubnets(tf GossipsubTopicFamilyWithDynamicSubnets, tracker *subnetTracker) { ctx, cancel := context.WithCancel(s.ctx) defer cancel() - tracker := newSubnetTracker(p) go s.ensurePeers(ctx, tracker) - go s.logMinimumPeersPerSubnet(ctx, p) + go s.logMinimumPeersPerSubnet(ctx, tf) - if err := s.waitForInitialSync(ctx); err != nil { - log.WithFields(p.logFields()).WithError(err).Debug("Could not subscribe to subnets as initial sync failed") - return - } s.trySubscribeSubnets(tracker) + slotTicker := slots.NewSlotTicker(s.cfg.clock.GenesisTime(), params.BeaconConfig().SecondsPerSlot) defer slotTicker.Done() for { select { case <-slotTicker.C(): // Check if this subscribe request is still valid - we may have crossed another fork epoch while waiting for initial sync. - if s.subscriptionRequestExpired(p.nse) { + if s.subscriptionRequestExpired(tf.NetworkScheduleEntry()) { // If we are already past the next fork epoch, do not subscribe to this topic. log.WithFields(logrus.Fields{ - "topic": p.shortTopic(), - "digest": p.nse.ForkDigest, - "epoch": p.nse.Epoch, + "topicFamily": fmt.Sprintf("%T", tf), + "digest": tf.NetworkScheduleEntry().ForkDigest, + "epoch": tf.NetworkScheduleEntry().Epoch, "currentEpoch": s.cfg.clock.CurrentEpoch(), }).Debug("Exiting topic subnet subscription loop") return @@ -431,12 +369,11 @@ func (s *Service) subscribeWithParameters(p subscribeParameters) { // trySubscribeSubnets attempts to subscribe to any missing subnets that we should be subscribed to. // Only if initial sync is complete. func (s *Service) trySubscribeSubnets(t *subnetTracker) { - subnetsToJoin := t.getSubnetsToJoin(s.cfg.clock.CurrentSlot()) + subnetsToJoin := t.family.GetSubnetsToJoin(s.cfg.clock.CurrentSlot()) s.pruneNotWanted(t, subnetsToJoin) for _, subnet := range t.missing(subnetsToJoin) { - // TODO: subscribeWithBase appends the protocol suffix, other methods don't. Make this consistent. - topic := t.fullTopic(subnet, "") - t.track(subnet, s.subscribeWithBase(topic, t.validate, t.handle)) + topic := t.family.GetFullTopicString(subnet) + t.track(subnet, s.subscribe(topic, t.family.Validator(), t.family.Handler())) } } @@ -459,17 +396,18 @@ func (s *Service) ensurePeers(ctx context.Context, tracker *subnetTracker) { func (s *Service) tryEnsurePeers(ctx context.Context, tracker *subnetTracker) { timeout := (time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Second) - 100*time.Millisecond minPeers := flags.Get().MinimumPeersPerSubnet - neededSubnets := computeAllNeededSubnets(s.cfg.clock.CurrentSlot(), tracker.getSubnetsToJoin, tracker.getSubnetsRequiringPeers) + neededSubnets := computeAllNeededSubnets(s.cfg.clock.CurrentSlot(), tracker.family) ctx, cancel := context.WithTimeout(ctx, timeout) defer cancel() - err := s.cfg.p2p.FindAndDialPeersWithSubnets(ctx, tracker.topicFormat, tracker.nse.ForkDigest, minPeers, neededSubnets) + builder := func(idx uint64) string { return tracker.family.GetFullTopicString(idx) } + err := s.cfg.p2p.FindAndDialPeersWithSubnets(ctx, builder, minPeers, neededSubnets) if err != nil && !errors.Is(err, context.DeadlineExceeded) { - log.WithFields(tracker.logFields()).WithError(err).Debug("Could not find peers with subnets") + log.WithFields(familyLogFields(tracker.family)).WithError(err).Debug("Could not find peers with subnets") } } -func (s *Service) logMinimumPeersPerSubnet(ctx context.Context, p subscribeParameters) { - logFields := p.logFields() +func (s *Service) logMinimumPeersPerSubnet(ctx context.Context, tf GossipsubTopicFamilyWithDynamicSubnets) { + logFields := familyLogFields(tf) minimumPeersPerSubnet := flags.Get().MinimumPeersPerSubnet // Warn the user if we are not subscribed to enough peers in the subnets. log := log.WithField("minimum", minimumPeersPerSubnet) @@ -480,12 +418,12 @@ func (s *Service) logMinimumPeersPerSubnet(ctx context.Context, p subscribeParam select { case <-logTicker.C: currentSlot := s.cfg.clock.CurrentSlot() - subnetsToFindPeersIndex := computeAllNeededSubnets(currentSlot, p.getSubnetsToJoin, p.getSubnetsRequiringPeers) + subnetsToFindPeersIndex := computeAllNeededSubnets(currentSlot, tf) isSubnetWithMissingPeers := false // Find new peers for wanted subnets if needed. for index := range subnetsToFindPeersIndex { - topic := fmt.Sprintf(p.topicFormat, p.nse.ForkDigest, index) + topic := tf.GetFullTopicString(index) // Check if we have enough peers in the subnet. Skip if we do. if count := s.connectedPeersCount(topic); count < minimumPeersPerSubnet { @@ -521,9 +459,8 @@ func (s *Service) unSubscribeFromTopic(topic string) { } // connectedPeersCount counts how many peer for a given topic are connected to the node. -func (s *Service) connectedPeersCount(subnetTopic string) int { - topic := subnetTopic + s.cfg.p2p.Encoding().ProtocolSuffix() - peersWithSubnet := s.cfg.p2p.PubSub().ListPeers(topic) +func (s *Service) connectedPeersCount(fullTopic string) int { + peersWithSubnet := s.cfg.p2p.PubSub().ListPeers(fullTopic) return len(peersWithSubnet) } @@ -673,17 +610,13 @@ func isDigestValid(digest [4]byte, clock *startup.Clock) (bool, error) { // and the subnets for which we want to find peers. func computeAllNeededSubnets( currentSlot primitives.Slot, - getSubnetsToJoin func(currentSlot primitives.Slot) map[uint64]bool, - getSubnetsRequiringPeers func(currentSlot primitives.Slot) map[uint64]bool, + dtf GossipsubTopicFamilyWithDynamicSubnets, ) map[uint64]bool { // Retrieve the subnets we want to join. - subnetsToJoin := getSubnetsToJoin(currentSlot) + subnetsToJoin := dtf.GetSubnetsToJoin(currentSlot) // Retrieve the subnets we want to find peers into. - subnetsRequiringPeers := make(map[uint64]bool) - if getSubnetsRequiringPeers != nil { - subnetsRequiringPeers = getSubnetsRequiringPeers(currentSlot) - } + subnetsRequiringPeers := dtf.GetSubnetsForBroadcast(currentSlot) // Combine the two maps to get all needed subnets. neededSubnets := make(map[uint64]bool, len(subnetsToJoin)+len(subnetsRequiringPeers)) diff --git a/beacon-chain/sync/subscriber_test.go b/beacon-chain/sync/subscriber_test.go index fd6ac72e85b2..ac67ffd177bc 100644 --- a/beacon-chain/sync/subscriber_test.go +++ b/beacon-chain/sync/subscriber_test.go @@ -30,6 +30,7 @@ import ( "github.com/OffchainLabs/prysm/v6/testing/require" "github.com/OffchainLabs/prysm/v6/testing/util" "github.com/OffchainLabs/prysm/v6/time/slots" + "github.com/ethereum/go-ethereum/p2p/enode" pubsub "github.com/libp2p/go-libp2p-pubsub" pubsubpb "github.com/libp2p/go-libp2p-pubsub/pb" "github.com/libp2p/go-libp2p/core/peer" @@ -37,6 +38,111 @@ import ( "google.golang.org/protobuf/proto" ) +// testStaticFamily implements a minimal static topic family for tests. +type testStaticFamily struct { + nse params.NetworkScheduleEntry + topicFmt string + protocolSuffix string + validator wrappedVal + handler subHandler +} + +func (t testStaticFamily) Validator() wrappedVal { + return t.validator +} + +func (t testStaticFamily) Handler() subHandler { + return t.handler +} + +func (t testStaticFamily) NetworkScheduleEntry() params.NetworkScheduleEntry { + return t.nse +} + +func (t testStaticFamily) GetFullTopicString() string { + return fmt.Sprintf(t.topicFmt, t.nse.ForkDigest) + t.protocolSuffix +} + +func (t testStaticFamily) Subscribe() {} + +func (t testStaticFamily) Unsubscribe() {} + +func makeTestFamily(nse params.NetworkScheduleEntry, topicFmt, suffix string, validator wrappedVal, handler subHandler) testStaticFamily { + return testStaticFamily{ + nse: nse, + topicFmt: topicFmt, + protocolSuffix: suffix, + validator: validator, + handler: handler, + } +} + +func makeFullTopic(topicFmt string, nse params.NetworkScheduleEntry, suffix string) string { + return fmt.Sprintf(topicFmt, nse.ForkDigest) + suffix +} + +// testDynamicFamily implements a minimal dynamic topic family for tests. +type testDynamicFamily struct { + nse params.NetworkScheduleEntry + topicFmt string + protocolSuffix string + validator wrappedVal + handler subHandler + subnetsToJoin func(primitives.Slot) map[uint64]bool + subnetsForCast func(primitives.Slot) map[uint64]bool +} + +func (t *testDynamicFamily) Validator() wrappedVal { + return t.validator +} + +func (t *testDynamicFamily) Handler() subHandler { + return t.handler +} + +func (t *testDynamicFamily) NetworkScheduleEntry() params.NetworkScheduleEntry { + return t.nse +} + +func (t *testDynamicFamily) GetFullTopicString(subnet uint64) string { + return fmt.Sprintf(t.topicFmt, t.nse.ForkDigest, subnet) + t.protocolSuffix +} + +func (t *testDynamicFamily) GetSubnetsToJoin(slot primitives.Slot) map[uint64]bool { + if t.subnetsToJoin != nil { + return t.subnetsToJoin(slot) + } + return nil +} + +func (t *testDynamicFamily) GetSubnetsForBroadcast(slot primitives.Slot) map[uint64]bool { + if t.subnetsForCast != nil { + return t.subnetsForCast(slot) + } + return nil +} + +func (t *testDynamicFamily) Subscribe() {} + +func (t *testDynamicFamily) Unsubscribe() {} + +func (t *testDynamicFamily) GetTopicsForNode(_ *enode.Node) ([]string, error) { + return nil, nil +} + +func makeTestDynamicFamily(nse params.NetworkScheduleEntry, topicFmt, suffix string, validator wrappedVal, handler subHandler, + getJoin func(primitives.Slot) map[uint64]bool, getCast func(primitives.Slot) map[uint64]bool) *testDynamicFamily { + return &testDynamicFamily{ + nse: nse, + topicFmt: topicFmt, + protocolSuffix: suffix, + validator: validator, + handler: handler, + subnetsToJoin: getJoin, + subnetsForCast: getCast, + } +} + func TestSubscribe_ReceivesValidMessage(t *testing.T) { p2pService := p2ptest.NewTestP2P(t) gt := time.Now() @@ -64,7 +170,7 @@ func TestSubscribe_ReceivesValidMessage(t *testing.T) { var wg sync.WaitGroup wg.Add(1) - r.subscribe(topic, r.noopValidator, func(_ context.Context, msg proto.Message) error { + tf := makeTestFamily(nse, topic, r.cfg.p2p.Encoding().ProtocolSuffix(), r.noopValidator, func(_ context.Context, msg proto.Message) error { m, ok := msg.(*pb.SignedVoluntaryExit) assert.Equal(t, true, ok, "Object is not of type *pb.SignedVoluntaryExit") if m.Exit == nil || m.Exit.Epoch != 55 { @@ -72,10 +178,10 @@ func TestSubscribe_ReceivesValidMessage(t *testing.T) { } wg.Done() return nil - }, nse) + }) + r.subscribe(tf.GetFullTopicString(), tf.Validator(), tf.Handler()) r.markForChainStart() - - p2pService.ReceivePubSub(topic, &pb.SignedVoluntaryExit{Exit: &pb.VoluntaryExit{Epoch: 55}, Signature: make([]byte, fieldparams.BLSSignatureLength)}) + p2pService.ReceivePubSub(tf.GetFullTopicString(), &pb.SignedVoluntaryExit{Exit: &pb.VoluntaryExit{Epoch: 55}, Signature: make([]byte, fieldparams.BLSSignatureLength)}) if util.WaitTimeout(&wg, time.Second) { t.Fatal("Did not receive PubSub in 1 second") @@ -110,12 +216,10 @@ func TestSubscribe_UnsubscribeTopic(t *testing.T) { p2pService.Digest = nse.ForkDigest topic := "/eth2/%x/voluntary_exit" - r.subscribe(topic, r.noopValidator, func(_ context.Context, msg proto.Message) error { - return nil - }, nse) + tf := makeTestFamily(nse, topic, r.cfg.p2p.Encoding().ProtocolSuffix(), r.noopValidator, func(_ context.Context, msg proto.Message) error { return nil }) + r.subscribe(tf.GetFullTopicString(), tf.Validator(), tf.Handler()) r.markForChainStart() - - fullTopic := fmt.Sprintf(topic, p2pService.Digest) + p2pService.Encoding().ProtocolSuffix() + fullTopic := tf.GetFullTopicString() assert.Equal(t, true, r.subHandler.topicExists(fullTopic)) topics := p2pService.PubSub().GetTopics() assert.Equal(t, fullTopic, topics[0]) @@ -162,11 +266,12 @@ func TestSubscribe_ReceivesAttesterSlashing(t *testing.T) { wg.Add(1) nse := params.GetNetworkScheduleEntry(r.cfg.clock.CurrentEpoch()) p2pService.Digest = nse.ForkDigest - r.subscribe(topic, r.noopValidator, func(ctx context.Context, msg proto.Message) error { + tf := makeTestFamily(nse, topic, r.cfg.p2p.Encoding().ProtocolSuffix(), r.noopValidator, func(ctx context.Context, msg proto.Message) error { require.NoError(t, r.attesterSlashingSubscriber(ctx, msg)) wg.Done() return nil - }, nse) + }) + r.subscribe(tf.GetFullTopicString(), tf.Validator(), tf.Handler()) beaconState, privKeys := util.DeterministicGenesisState(t, 64) chainService.State = beaconState r.markForChainStart() @@ -178,7 +283,7 @@ func TestSubscribe_ReceivesAttesterSlashing(t *testing.T) { require.NoError(t, err, "Error generating attester slashing") err = r.cfg.beaconDB.SaveState(ctx, beaconState, bytesutil.ToBytes32(attesterSlashing.FirstAttestation().GetData().BeaconBlockRoot)) require.NoError(t, err) - p2pService.ReceivePubSub(topic, attesterSlashing) + p2pService.ReceivePubSub(tf.GetFullTopicString(), attesterSlashing) if util.WaitTimeout(&wg, time.Second) { t.Fatal("Did not receive PubSub in 1 second") @@ -217,11 +322,12 @@ func TestSubscribe_ReceivesProposerSlashing(t *testing.T) { params.OverrideBeaconConfig(params.MainnetConfig()) nse := params.GetNetworkScheduleEntry(r.cfg.clock.CurrentEpoch()) p2pService.Digest = nse.ForkDigest - r.subscribe(topic, r.noopValidator, func(ctx context.Context, msg proto.Message) error { + tf := makeTestFamily(nse, topic, r.cfg.p2p.Encoding().ProtocolSuffix(), r.noopValidator, func(ctx context.Context, msg proto.Message) error { require.NoError(t, r.proposerSlashingSubscriber(ctx, msg)) wg.Done() return nil - }, nse) + }) + r.subscribe(tf.GetFullTopicString(), tf.Validator(), tf.Handler()) beaconState, privKeys := util.DeterministicGenesisState(t, 64) chainService.State = beaconState r.markForChainStart() @@ -232,7 +338,7 @@ func TestSubscribe_ReceivesProposerSlashing(t *testing.T) { ) require.NoError(t, err, "Error generating proposer slashing") - p2pService.ReceivePubSub(topic, proposerSlashing) + p2pService.ReceivePubSub(tf.GetFullTopicString(), proposerSlashing) if util.WaitTimeout(&wg, time.Second) { t.Fatal("Did not receive PubSub in 1 second") @@ -266,12 +372,13 @@ func TestSubscribe_HandlesPanic(t *testing.T) { var wg sync.WaitGroup wg.Add(1) - r.subscribe(topic, r.noopValidator, func(_ context.Context, msg proto.Message) error { + tf := makeTestFamily(nse, topic, p.Encoding().ProtocolSuffix(), r.noopValidator, func(_ context.Context, msg proto.Message) error { defer wg.Done() panic("bad") - }, nse) + }) + r.subscribe(tf.GetFullTopicString(), tf.Validator(), tf.Handler()) r.markForChainStart() - p.ReceivePubSub(topic, &pb.SignedVoluntaryExit{Exit: &pb.VoluntaryExit{Epoch: 55}, Signature: make([]byte, fieldparams.BLSSignatureLength)}) + p.ReceivePubSub(tf.GetFullTopicString(), &pb.SignedVoluntaryExit{Exit: &pb.VoluntaryExit{Epoch: 55}, Signature: make([]byte, fieldparams.BLSSignatureLength)}) if util.WaitTimeout(&wg, time.Second) { t.Fatal("Did not receive PubSub in 1 second") @@ -297,15 +404,12 @@ func TestRevalidateSubscription_CorrectlyFormatsTopic(t *testing.T) { } nse := params.GetNetworkScheduleEntry(r.cfg.clock.CurrentEpoch()) - params := subscribeParameters{ - topicFormat: "/eth2/testing/%#x/committee%d", - nse: nse, - } - tracker := newSubnetTracker(params) + tfDyn := makeTestDynamicFamily(nse, "/eth2/testing/%#x/committee%d", r.cfg.p2p.Encoding().ProtocolSuffix(), r.noopValidator, nil, nil, nil) + tracker := newSubnetTracker(tfDyn) // committee index 1 c1 := uint64(1) - fullTopic := params.fullTopic(c1, r.cfg.p2p.Encoding().ProtocolSuffix()) + fullTopic := tfDyn.GetFullTopicString(c1) _, topVal := r.wrapAndReportValidation(fullTopic, r.noopValidator) require.NoError(t, r.cfg.p2p.PubSub().RegisterTopicValidator(fullTopic, topVal)) sub1, err := r.cfg.p2p.SubscribeToTopic(fullTopic) @@ -314,7 +418,7 @@ func TestRevalidateSubscription_CorrectlyFormatsTopic(t *testing.T) { // committee index 2 c2 := uint64(2) - fullTopic = params.fullTopic(c2, r.cfg.p2p.Encoding().ProtocolSuffix()) + fullTopic = tfDyn.GetFullTopicString(c2) _, topVal = r.wrapAndReportValidation(fullTopic, r.noopValidator) err = r.cfg.p2p.PubSub().RegisterTopicValidator(fullTopic, topVal) require.NoError(t, err) @@ -552,11 +656,8 @@ func TestSubscribeWithSyncSubnets_DynamicOK(t *testing.T) { currEpoch := slots.ToEpoch(slot) cache.SyncSubnetIDs.AddSyncCommitteeSubnets([]byte("pubkey"), currEpoch, []uint64{0, 1}, 10*time.Second) nse := params.GetNetworkScheduleEntry(r.cfg.clock.CurrentEpoch()) - go r.subscribeWithParameters(subscribeParameters{ - topicFormat: p2p.SyncCommitteeSubnetTopicFormat, - nse: nse, - getSubnetsToJoin: r.activeSyncSubnetIndices, - }) + tfDyn := makeTestDynamicFamily(nse, p2p.SyncCommitteeSubnetTopicFormat, r.cfg.p2p.Encoding().ProtocolSuffix(), r.noopValidator, nil, r.activeSyncSubnetIndices, nil) + go r.subscribeToDynamicSubnetFamily(tfDyn) time.Sleep(2 * time.Second) assert.Equal(t, 2, len(r.cfg.p2p.PubSub().GetTopics())) topicMap := map[string]bool{} @@ -601,11 +702,8 @@ func TestSubscribeWithSyncSubnets_DynamicSwitchFork(t *testing.T) { require.Equal(t, [4]byte(params.BeaconConfig().DenebForkVersion), nse.ForkVersion) require.Equal(t, params.BeaconConfig().DenebForkEpoch, nse.Epoch) - sp := newSubnetTracker(subscribeParameters{ - topicFormat: p2p.SyncCommitteeSubnetTopicFormat, - nse: nse, - getSubnetsToJoin: r.activeSyncSubnetIndices, - }) + tfDyn2 := makeTestDynamicFamily(nse, p2p.SyncCommitteeSubnetTopicFormat, r.cfg.p2p.Encoding().ProtocolSuffix(), r.noopValidator, nil, r.activeSyncSubnetIndices, nil) + sp := newSubnetTracker(tfDyn2) r.trySubscribeSubnets(sp) assert.Equal(t, 2, len(r.cfg.p2p.PubSub().GetTopics())) topicMap := map[string]bool{} @@ -625,7 +723,7 @@ func TestSubscribeWithSyncSubnets_DynamicSwitchFork(t *testing.T) { require.Equal(t, [4]byte(params.BeaconConfig().ElectraForkVersion), nse.ForkVersion) require.Equal(t, params.BeaconConfig().ElectraForkEpoch, nse.Epoch) - sp.nse = nse + tfDyn2.nse = nse // clear the cache and re-subscribe to subnets. // this should result in the subscriptions being removed cache.SyncSubnetIDs.EmptyAllCaches() diff --git a/beacon-chain/sync/topic_families_dynamic_subnets.go b/beacon-chain/sync/topic_families_dynamic_subnets.go index 12030bcf0663..59d61750e390 100644 --- a/beacon-chain/sync/topic_families_dynamic_subnets.go +++ b/beacon-chain/sync/topic_families_dynamic_subnets.go @@ -2,6 +2,7 @@ package sync import ( "fmt" + "sync" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p" "github.com/OffchainLabs/prysm/v6/config/params" @@ -9,24 +10,57 @@ import ( "github.com/ethereum/go-ethereum/p2p/enode" "github.com/ethereum/go-ethereum/p2p/enr" "github.com/pkg/errors" + "github.com/sirupsen/logrus" ) // AttestationTopicFamily var _ GossipsubTopicFamilyWithDynamicSubnets = (*AttestationTopicFamily)(nil) -type AttestationTopicFamily struct { +type baseGossipsubTopicFamilyWithDynamicSubnets struct { baseGossipsubTopicFamily + + mu sync.Mutex + tracker *subnetTracker + unsubscribed bool +} + +func (b *baseGossipsubTopicFamilyWithDynamicSubnets) Subscribe(tf GossipsubTopicFamilyWithDynamicSubnets) { + b.mu.Lock() + defer b.mu.Unlock() + if b.unsubscribed { + log.WithFields(logrus.Fields{ + "topicFamily": fmt.Sprintf("%T", tf), + "digest": b.nse.ForkDigest, + "epoch": b.nse.Epoch, + }).Error("Cannot subscribe after unsubscribing") + return + } + b.tracker = b.syncService.subscribeToDynamicSubnetFamily(tf) +} + +func (b *baseGossipsubTopicFamilyWithDynamicSubnets) Unsubscribe() { + b.mu.Lock() + defer b.mu.Unlock() + b.unsubscribed = true + b.syncService.pruneNotWanted(b.tracker, nil) // unsubscribe from all subnets +} + +type AttestationTopicFamily struct { + baseGossipsubTopicFamilyWithDynamicSubnets } // NewAttestationTopicFamily creates a new AttestationTopicFamily. func NewAttestationTopicFamily(s *Service, nse params.NetworkScheduleEntry) *AttestationTopicFamily { - return &AttestationTopicFamily{ - baseGossipsubTopicFamily{ - syncService: s, - nse: nse, - protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix(), + attestationTopicFamily := &AttestationTopicFamily{ + baseGossipsubTopicFamilyWithDynamicSubnets: baseGossipsubTopicFamilyWithDynamicSubnets{ + baseGossipsubTopicFamily: baseGossipsubTopicFamily{ + syncService: s, + nse: nse, + protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix(), + }, }, } + return attestationTopicFamily } // Validator returns the validator function for attestation subnets. @@ -56,32 +90,33 @@ func (a *AttestationTopicFamily) GetSubnetsForBroadcast(slot primitives.Slot) ma // GetTopicsForNode returns all topics for the given node that are relevant to this topic family. func (a *AttestationTopicFamily) GetTopicsForNode(node *enode.Node) ([]string, error) { - return getTopicsForNode(a.syncService, a, node, a.syncService.cfg.p2p.AttestationSubnets) + return getTopicsForNode(a.syncService, a, node, p2p.AttestationSubnets) } -// TODO func (a *AttestationTopicFamily) Subscribe() { - + a.baseGossipsubTopicFamilyWithDynamicSubnets.Subscribe(a) } func (a *AttestationTopicFamily) Unsubscribe() { - + a.baseGossipsubTopicFamilyWithDynamicSubnets.Unsubscribe() } // SyncCommitteeTopicFamily var _ GossipsubTopicFamilyWithDynamicSubnets = (*SyncCommitteeTopicFamily)(nil) type SyncCommitteeTopicFamily struct { - baseGossipsubTopicFamily + baseGossipsubTopicFamilyWithDynamicSubnets } // NewSyncCommitteeTopicFamily creates a new SyncCommitteeTopicFamily. func NewSyncCommitteeTopicFamily(s *Service, nse params.NetworkScheduleEntry) *SyncCommitteeTopicFamily { return &SyncCommitteeTopicFamily{ - baseGossipsubTopicFamily{ - syncService: s, - nse: nse, - protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix(), + baseGossipsubTopicFamilyWithDynamicSubnets: baseGossipsubTopicFamilyWithDynamicSubnets{ + baseGossipsubTopicFamily: baseGossipsubTopicFamily{ + syncService: s, + nse: nse, + protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix(), + }, }, } } @@ -113,32 +148,33 @@ func (s *SyncCommitteeTopicFamily) GetSubnetsForBroadcast(slot primitives.Slot) // GetTopicsForNode returns all topics for the given node that are relevant to this topic family. func (s *SyncCommitteeTopicFamily) GetTopicsForNode(node *enode.Node) ([]string, error) { - return getTopicsForNode(s.syncService, s, node, s.syncService.cfg.p2p.SyncSubnets) + return getTopicsForNode(s.syncService, s, node, p2p.SyncSubnets) } -// TODO func (s *SyncCommitteeTopicFamily) Subscribe() { - + s.baseGossipsubTopicFamilyWithDynamicSubnets.Subscribe(s) } func (s *SyncCommitteeTopicFamily) Unsubscribe() { - + s.baseGossipsubTopicFamilyWithDynamicSubnets.Unsubscribe() } // DataColumnTopicFamily var _ GossipsubTopicFamilyWithDynamicSubnets = (*DataColumnTopicFamily)(nil) type DataColumnTopicFamily struct { - baseGossipsubTopicFamily + baseGossipsubTopicFamilyWithDynamicSubnets } // NewDataColumnTopicFamily creates a new DataColumnTopicFamily. func NewDataColumnTopicFamily(s *Service, nse params.NetworkScheduleEntry) *DataColumnTopicFamily { return &DataColumnTopicFamily{ - baseGossipsubTopicFamily{ - syncService: s, - nse: nse, - protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix(), + baseGossipsubTopicFamilyWithDynamicSubnets: baseGossipsubTopicFamilyWithDynamicSubnets{ + baseGossipsubTopicFamily: baseGossipsubTopicFamily{ + syncService: s, + nse: nse, + protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix(), + }, }, } } @@ -170,16 +206,15 @@ func (d *DataColumnTopicFamily) GetSubnetsForBroadcast(slot primitives.Slot) map // GetTopicsForNode returns all topics for the given node that are relevant to this topic family. func (d *DataColumnTopicFamily) GetTopicsForNode(node *enode.Node) ([]string, error) { - return getTopicsForNode(d.syncService, d, node, d.syncService.cfg.p2p.DataColumnSubnets) + return getTopicsForNode(d.syncService, d, node, p2p.DataColumnSubnets) } -// TODO func (d *DataColumnTopicFamily) Subscribe() { - + d.baseGossipsubTopicFamilyWithDynamicSubnets.Subscribe(d) } func (d *DataColumnTopicFamily) Unsubscribe() { - + d.baseGossipsubTopicFamilyWithDynamicSubnets.Unsubscribe() } type nodeSubnetExtractor func(id enode.ID, n *enode.Node, r *enr.Record) (map[uint64]bool, error) @@ -196,8 +231,7 @@ func getTopicsForNode( currentSlot := s.cfg.clock.CurrentSlot() neededSubnets := computeAllNeededSubnets( currentSlot, - tf.GetSubnetsToJoin, - tf.GetSubnetsForBroadcast, + tf, ) nodeSubnets, err := extractor(node.ID(), node, node.Record()) diff --git a/beacon-chain/sync/topic_families_static_subnets.go b/beacon-chain/sync/topic_families_static_subnets.go index 0497c3252022..d6224f979c0e 100644 --- a/beacon-chain/sync/topic_families_static_subnets.go +++ b/beacon-chain/sync/topic_families_static_subnets.go @@ -39,7 +39,7 @@ func (b *BlobTopicFamily) GetFullTopicString() string { } func (b *BlobTopicFamily) Subscribe() { - b.syncService.subscribe(b) + b.syncService.subscribe(b.GetFullTopicString(), b.Validator(), b.Handler()) } func (b *BlobTopicFamily) Unsubscribe() { diff --git a/beacon-chain/sync/topic_families_without_subnets.go b/beacon-chain/sync/topic_families_without_subnets.go index bd5903ab58ae..e5358fcc8953 100644 --- a/beacon-chain/sync/topic_families_without_subnets.go +++ b/beacon-chain/sync/topic_families_without_subnets.go @@ -36,7 +36,7 @@ func (b *BlockTopicFamily) GetFullTopicString() string { } func (b *BlockTopicFamily) Subscribe() { - b.syncService.subscribe(b) + b.syncService.subscribe(b.GetFullTopicString(), b.Validator(), b.Handler()) } func (b *BlockTopicFamily) Unsubscribe() { @@ -72,7 +72,7 @@ func (a *AggregateAndProofTopicFamily) GetFullTopicString() string { } func (a *AggregateAndProofTopicFamily) Subscribe() { - a.syncService.subscribe(a) + a.syncService.subscribe(a.GetFullTopicString(), a.Validator(), a.Handler()) } func (a *AggregateAndProofTopicFamily) Unsubscribe() { @@ -108,7 +108,7 @@ func (v *VoluntaryExitTopicFamily) GetFullTopicString() string { } func (v *VoluntaryExitTopicFamily) Subscribe() { - v.syncService.subscribe(v) + v.syncService.subscribe(v.GetFullTopicString(), v.Validator(), v.Handler()) } func (v *VoluntaryExitTopicFamily) Unsubscribe() { @@ -144,7 +144,7 @@ func (p *ProposerSlashingTopicFamily) GetFullTopicString() string { } func (p *ProposerSlashingTopicFamily) Subscribe() { - p.syncService.subscribe(p) + p.syncService.subscribe(p.GetFullTopicString(), p.Validator(), p.Handler()) } func (p *ProposerSlashingTopicFamily) Unsubscribe() { @@ -181,7 +181,7 @@ func (a *AttesterSlashingTopicFamily) GetFullTopicString() string { // TODO: Do we really need to spawn go-routines here ? func (a *AttesterSlashingTopicFamily) Subscribe() { - a.syncService.subscribe(a) + a.syncService.subscribe(a.GetFullTopicString(), a.Validator(), a.Handler()) } func (a *AttesterSlashingTopicFamily) Unsubscribe() { @@ -215,7 +215,7 @@ func (sc *SyncContributionAndProofTopicFamily) GetFullTopicString() string { } func (sc *SyncContributionAndProofTopicFamily) Subscribe() { - sc.syncService.subscribe(sc) + sc.syncService.subscribe(sc.GetFullTopicString(), sc.Validator(), sc.Handler()) } func (sc *SyncContributionAndProofTopicFamily) Unsubscribe() { @@ -251,7 +251,7 @@ func (l *LightClientOptimisticUpdateTopicFamily) GetFullTopicString() string { } func (l *LightClientOptimisticUpdateTopicFamily) Subscribe() { - l.syncService.subscribe(l) + l.syncService.subscribe(l.GetFullTopicString(), l.Validator(), l.Handler()) } func (l *LightClientOptimisticUpdateTopicFamily) Unsubscribe() { @@ -287,7 +287,7 @@ func (l *LightClientFinalityUpdateTopicFamily) GetFullTopicString() string { } func (l *LightClientFinalityUpdateTopicFamily) Subscribe() { - l.syncService.subscribe(l) + l.syncService.subscribe(l.GetFullTopicString(), l.Validator(), l.Handler()) } func (l *LightClientFinalityUpdateTopicFamily) Unsubscribe() { l.syncService.unSubscribeFromTopic(l.GetFullTopicString()) @@ -322,7 +322,7 @@ func (b *BlsToExecutionChangeTopicFamily) GetFullTopicString() string { } func (b *BlsToExecutionChangeTopicFamily) Subscribe() { - b.syncService.subscribe(b) + b.syncService.subscribe(b.GetFullTopicString(), b.Validator(), b.Handler()) } func (b *BlsToExecutionChangeTopicFamily) Unsubscribe() { From e14590636f5f86438e082ff39fd0b181d91a257b Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Tue, 4 Nov 2025 20:08:41 +0400 Subject: [PATCH 04/39] bazel gazelle --- beacon-chain/p2p/testing/BUILD.bazel | 1 - beacon-chain/sync/BUILD.bazel | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon-chain/p2p/testing/BUILD.bazel b/beacon-chain/p2p/testing/BUILD.bazel index 2728164369a9..6206705016e9 100644 --- a/beacon-chain/p2p/testing/BUILD.bazel +++ b/beacon-chain/p2p/testing/BUILD.bazel @@ -23,7 +23,6 @@ go_library( "//beacon-chain/p2p/encoder:go_default_library", "//beacon-chain/p2p/peers:go_default_library", "//beacon-chain/p2p/peers/scorers:go_default_library", - "//config/fieldparams:go_default_library", "//config/params:go_default_library", "//consensus-types/blocks:go_default_library", "//consensus-types/interfaces:go_default_library", diff --git a/beacon-chain/sync/BUILD.bazel b/beacon-chain/sync/BUILD.bazel index f01e8d37c460..47cdedfe35dc 100644 --- a/beacon-chain/sync/BUILD.bazel +++ b/beacon-chain/sync/BUILD.bazel @@ -290,6 +290,7 @@ go_test( "@com_github_d4l3k_messagediff//:go_default_library", "@com_github_ethereum_go_ethereum//common:go_default_library", "@com_github_ethereum_go_ethereum//core/types:go_default_library", + "@com_github_ethereum_go_ethereum//p2p/enode:go_default_library", "@com_github_ethereum_go_ethereum//p2p/enr:go_default_library", "@com_github_golang_snappy//:go_default_library", "@com_github_libp2p_go_libp2p//:go_default_library", From ab2e836d3f0f7b9f03e260c8fa796e8eb7a2fabb Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Tue, 4 Nov 2025 21:11:45 +0400 Subject: [PATCH 05/39] fix test --- beacon-chain/sync/rpc_status_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/beacon-chain/sync/rpc_status_test.go b/beacon-chain/sync/rpc_status_test.go index c19a70a4a5a2..d064626b5acd 100644 --- a/beacon-chain/sync/rpc_status_test.go +++ b/beacon-chain/sync/rpc_status_test.go @@ -329,6 +329,8 @@ func TestHandshakeHandlers_Roundtrip(t *testing.T) { chainStarted: abool.New(), subHandler: newSubTopicHandler(), } + r.gossipsubController = NewGossipsubController(ctx, r) + markInitSyncComplete(t, r) clock := startup.NewClockSynchronizer() require.NoError(t, clock.SetClock(startup.NewClock(time.Now(), [32]byte{}))) From 3e985377cec27fa2979512ad2109d4b1af5f80ae Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Wed, 5 Nov 2025 09:05:57 +0400 Subject: [PATCH 06/39] fix test --- beacon-chain/sync/subscriber_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/beacon-chain/sync/subscriber_test.go b/beacon-chain/sync/subscriber_test.go index ac67ffd177bc..a29d330abf24 100644 --- a/beacon-chain/sync/subscriber_test.go +++ b/beacon-chain/sync/subscriber_test.go @@ -583,6 +583,7 @@ func TestFilterSubnetPeers(t *testing.T) { chainStarted: abool.New(), subHandler: newSubTopicHandler(), } + r.gossipsubController = NewGossipsubController(ctx, &r) markInitSyncComplete(t, &r) // Empty cache at the end of the test. defer cache.SubnetIDs.EmptyAllCaches() From 452d42bd10d11d83a187164dcf624a54f2bb4a97 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Wed, 5 Nov 2025 09:21:34 +0400 Subject: [PATCH 07/39] fix test in sync --- beacon-chain/sync/rpc_status_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/beacon-chain/sync/rpc_status_test.go b/beacon-chain/sync/rpc_status_test.go index d064626b5acd..3874c2ac6030 100644 --- a/beacon-chain/sync/rpc_status_test.go +++ b/beacon-chain/sync/rpc_status_test.go @@ -947,6 +947,8 @@ func TestStatusRPCRequest_BadPeerHandshake(t *testing.T) { chainStarted: abool.New(), subHandler: newSubTopicHandler(), } + r.gossipsubController = NewGossipsubController(ctx, r) + markInitSyncComplete(t, r) clock := startup.NewClockSynchronizer() require.NoError(t, clock.SetClock(startup.NewClock(time.Now(), [32]byte{}))) From af2522e5f037d556eb099f989fcc71f83ea5ed1c Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Wed, 5 Nov 2025 09:49:27 +0400 Subject: [PATCH 08/39] fix schedule --- beacon-chain/sync/gossipsub_topic_family.go | 135 ++++++++++---------- 1 file changed, 69 insertions(+), 66 deletions(-) diff --git a/beacon-chain/sync/gossipsub_topic_family.go b/beacon-chain/sync/gossipsub_topic_family.go index f15ea862e962..b42f27ca74b4 100644 --- a/beacon-chain/sync/gossipsub_topic_family.go +++ b/beacon-chain/sync/gossipsub_topic_family.go @@ -60,82 +60,85 @@ type topicFamilyEntry struct { factory func(s *Service, nse params.NetworkScheduleEntry) []GossipsubTopicFamily } -var topicFamilySchedule = []topicFamilyEntry{ - // Genesis topic families - { - activationEpoch: params.BeaconConfig().GenesisEpoch, - factory: func(s *Service, nse params.NetworkScheduleEntry) []GossipsubTopicFamily { - return []GossipsubTopicFamily{ - NewBlockTopicFamily(s, nse), - NewAggregateAndProofTopicFamily(s, nse), - NewVoluntaryExitTopicFamily(s, nse), - NewProposerSlashingTopicFamily(s, nse), - NewAttesterSlashingTopicFamily(s, nse), - NewAttestationTopicFamily(s, nse), - } +func topicFamilySchedule() []topicFamilyEntry { + cfg := params.BeaconConfig() + return []topicFamilyEntry{ + // Genesis topic families + { + activationEpoch: cfg.GenesisEpoch, + factory: func(s *Service, nse params.NetworkScheduleEntry) []GossipsubTopicFamily { + return []GossipsubTopicFamily{ + NewBlockTopicFamily(s, nse), + NewAggregateAndProofTopicFamily(s, nse), + NewVoluntaryExitTopicFamily(s, nse), + NewProposerSlashingTopicFamily(s, nse), + NewAttesterSlashingTopicFamily(s, nse), + NewAttestationTopicFamily(s, nse), + } + }, }, - }, - // Altair topic families - { - activationEpoch: params.BeaconConfig().AltairForkEpoch, - factory: func(s *Service, nse params.NetworkScheduleEntry) []GossipsubTopicFamily { - families := []GossipsubTopicFamily{ - NewSyncContributionAndProofTopicFamily(s, nse), - NewSyncCommitteeTopicFamily(s, nse), - } - if features.Get().EnableLightClient { - families = append(families, - NewLightClientOptimisticUpdateTopicFamily(s, nse), - NewLightClientFinalityUpdateTopicFamily(s, nse), - ) - } - return families + // Altair topic families + { + activationEpoch: cfg.AltairForkEpoch, + factory: func(s *Service, nse params.NetworkScheduleEntry) []GossipsubTopicFamily { + families := []GossipsubTopicFamily{ + NewSyncContributionAndProofTopicFamily(s, nse), + NewSyncCommitteeTopicFamily(s, nse), + } + if features.Get().EnableLightClient { + families = append(families, + NewLightClientOptimisticUpdateTopicFamily(s, nse), + NewLightClientFinalityUpdateTopicFamily(s, nse), + ) + } + return families + }, }, - }, - // Capella topic families - { - activationEpoch: params.BeaconConfig().CapellaForkEpoch, - factory: func(s *Service, nse params.NetworkScheduleEntry) []GossipsubTopicFamily { - return []GossipsubTopicFamily{NewBlsToExecutionChangeTopicFamily(s, nse)} + // Capella topic families + { + activationEpoch: cfg.CapellaForkEpoch, + factory: func(s *Service, nse params.NetworkScheduleEntry) []GossipsubTopicFamily { + return []GossipsubTopicFamily{NewBlsToExecutionChangeTopicFamily(s, nse)} + }, }, - }, - // Blob topic families (static per-subnet) in Deneb and Electra forks (removed in Fulu) - { - activationEpoch: params.BeaconConfig().DenebForkEpoch, - deactivationEpoch: func() *primitives.Epoch { e := params.BeaconConfig().ElectraForkEpoch; return &e }(), - factory: func(s *Service, nse params.NetworkScheduleEntry) []GossipsubTopicFamily { - count := params.BeaconConfig().BlobsidecarSubnetCount - families := make([]GossipsubTopicFamily, 0, count) - for i := uint64(0); i < count; i++ { - families = append(families, NewBlobTopicFamily(s, nse, i)) - } - return families + // Blob topic families (static per-subnet) in Deneb and Electra forks (removed in Fulu) + { + activationEpoch: cfg.DenebForkEpoch, + deactivationEpoch: func() *primitives.Epoch { e := cfg.ElectraForkEpoch; return &e }(), + factory: func(s *Service, nse params.NetworkScheduleEntry) []GossipsubTopicFamily { + count := cfg.BlobsidecarSubnetCount + families := make([]GossipsubTopicFamily, 0, count) + for i := uint64(0); i < count; i++ { + families = append(families, NewBlobTopicFamily(s, nse, i)) + } + return families + }, }, - }, - { - activationEpoch: params.BeaconConfig().ElectraForkEpoch, - deactivationEpoch: func() *primitives.Epoch { e := params.BeaconConfig().FuluForkEpoch; return &e }(), - factory: func(s *Service, nse params.NetworkScheduleEntry) []GossipsubTopicFamily { - count := params.BeaconConfig().BlobsidecarSubnetCountElectra - families := make([]GossipsubTopicFamily, 0, count) - for i := uint64(0); i < count; i++ { - families = append(families, NewBlobTopicFamily(s, nse, i)) - } - return families + { + activationEpoch: cfg.ElectraForkEpoch, + deactivationEpoch: func() *primitives.Epoch { e := cfg.FuluForkEpoch; return &e }(), + factory: func(s *Service, nse params.NetworkScheduleEntry) []GossipsubTopicFamily { + count := cfg.BlobsidecarSubnetCountElectra + families := make([]GossipsubTopicFamily, 0, count) + for i := uint64(0); i < count; i++ { + families = append(families, NewBlobTopicFamily(s, nse, i)) + } + return families + }, }, - }, - // Fulu data column topic family - { - activationEpoch: params.BeaconConfig().FuluForkEpoch, - factory: func(s *Service, nse params.NetworkScheduleEntry) []GossipsubTopicFamily { - return []GossipsubTopicFamily{NewDataColumnTopicFamily(s, nse)} + // Fulu data column topic family + { + activationEpoch: cfg.FuluForkEpoch, + factory: func(s *Service, nse params.NetworkScheduleEntry) []GossipsubTopicFamily { + return []GossipsubTopicFamily{NewDataColumnTopicFamily(s, nse)} + }, }, - }, + } } func TopicFamiliesForEpoch(epoch primitives.Epoch, s *Service, nse params.NetworkScheduleEntry) []GossipsubTopicFamily { var activeFamilies []GossipsubTopicFamily - for _, entry := range topicFamilySchedule { + for _, entry := range topicFamilySchedule() { if epoch < entry.activationEpoch { continue } From 8dfbabc69137a6f7414b838873db647b534384ec Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Wed, 5 Nov 2025 14:45:35 +0400 Subject: [PATCH 09/39] fork watcher test works --- beacon-chain/sync/fork_watcher_test.go | 73 +++----------------------- 1 file changed, 8 insertions(+), 65 deletions(-) diff --git a/beacon-chain/sync/fork_watcher_test.go b/beacon-chain/sync/fork_watcher_test.go index 45e2ae17828b..97832f60c4de 100644 --- a/beacon-chain/sync/fork_watcher_test.go +++ b/beacon-chain/sync/fork_watcher_test.go @@ -8,12 +8,15 @@ import ( "github.com/OffchainLabs/prysm/v6/async/abool" mockChain "github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/testing" + "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p" p2ptest "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/testing" "github.com/OffchainLabs/prysm/v6/beacon-chain/startup" mockSync "github.com/OffchainLabs/prysm/v6/beacon-chain/sync/initial-sync/testing" "github.com/OffchainLabs/prysm/v6/config/params" "github.com/OffchainLabs/prysm/v6/consensus-types/primitives" "github.com/OffchainLabs/prysm/v6/genesis" + "github.com/OffchainLabs/prysm/v6/testing/assert" + "github.com/OffchainLabs/prysm/v6/testing/require" ) func defaultClockWithTimeAtEpoch(epoch primitives.Epoch) *startup.Clock { @@ -46,34 +49,6 @@ func testForkWatcherService(t *testing.T, current primitives.Epoch) *Service { return r } -// TODO: Move to gossip controller test -/* -func TestRegisterSubscriptions_Idempotent(t *testing.T) { - params.SetupTestConfigCleanup(t) - genesis.StoreEmbeddedDuringTest(t, params.BeaconConfig().ConfigName) - fulu := params.BeaconConfig().ElectraForkEpoch + 4096*2 - params.BeaconConfig().FuluForkEpoch = fulu - params.BeaconConfig().InitializeForkSchedule() - - current := fulu - 1 - s := testForkWatcherService(t, current) - next := params.GetNetworkScheduleEntry(fulu) - wg := attachSpawner(s) - require.Equal(t, true, s.registerSubscribers(next)) - done := make(chan struct{}) - go func() { wg.Wait(); close(done) }() - select { - case <-time.After(5 * time.Second): - t.Fatal("timed out waiting for subscriptions to be registered") - case <-done: - } - // the goal of this callback is just to assert that spawn is never called. - s.subscriptionSpawner = func(func()) { t.Error("registration routines spawned twice for the same digest") } - require.NoError(t, s.ensureRPCRegistrationsForEpoch(fulu)) -}*/ - -// TODO: Move to gossip controller test -/* func TestService_CheckForNextEpochFork(t *testing.T) { closedChan := make(chan struct{}) close(closedChan) @@ -103,7 +78,6 @@ func TestService_CheckForNextEpochFork(t *testing.T) { epochAtRegistration: func(e primitives.Epoch) primitives.Epoch { return e - 1 }, nextForkEpoch: params.BeaconConfig().BellatrixForkEpoch, checkRegistration: func(t *testing.T, s *Service) { - digest := params.ForkDigest(params.BeaconConfig().AltairForkEpoch) rpcMap := make(map[string]bool) for _, p := range s.cfg.p2p.Host().Mux().Protocols() { rpcMap[string(p)] = true @@ -111,8 +85,6 @@ func TestService_CheckForNextEpochFork(t *testing.T) { 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") - expected := fmt.Sprintf(p2p.SyncContributionAndProofSubnetTopicFormat+s.cfg.p2p.Encoding().ProtocolSuffix(), digest) - assert.Equal(t, true, s.subHandler.topicExists(expected), "subnet topic doesn't exist") // TODO: we should check subcommittee indices here but we need to work with the committee cache to do it properly /* subIndices := mapFromCount(params.BeaconConfig().SyncCommitteeSubnetCount) @@ -121,20 +93,16 @@ func TestService_CheckForNextEpochFork(t *testing.T) { expected := topic + s.cfg.p2p.Encoding().ProtocolSuffix() assert.Equal(t, true, s.subHandler.topicExists(expected), fmt.Sprintf("subnet topic %s doesn't exist", expected)) } -*/ -/*}, + */ + }, }, { name: "capella fork in the next epoch", checkRegistration: func(t *testing.T, s *Service) { - digest := params.ForkDigest(params.BeaconConfig().CapellaForkEpoch) rpcMap := make(map[string]bool) for _, p := range s.cfg.p2p.Host().Mux().Protocols() { rpcMap[string(p)] = true } - - expected := fmt.Sprintf(p2p.BlsToExecutionChangeSubnetTopicFormat+s.cfg.p2p.Encoding().ProtocolSuffix(), digest) - assert.Equal(t, true, s.subHandler.topicExists(expected), "subnet topic doesn't exist") }, forkEpoch: params.BeaconConfig().CapellaForkEpoch, nextForkEpoch: params.BeaconConfig().DenebForkEpoch, @@ -143,17 +111,10 @@ func TestService_CheckForNextEpochFork(t *testing.T) { { name: "deneb fork in the next epoch", checkRegistration: func(t *testing.T, s *Service) { - digest := params.ForkDigest(params.BeaconConfig().DenebForkEpoch) rpcMap := make(map[string]bool) for _, p := range s.cfg.p2p.Host().Mux().Protocols() { rpcMap[string(p)] = true } - subIndices := mapFromCount(params.BeaconConfig().BlobsidecarSubnetCount) - for idx := range subIndices { - topic := fmt.Sprintf(p2p.BlobSubnetTopicFormat, digest, idx) - expected := topic + s.cfg.p2p.Encoding().ProtocolSuffix() - assert.Equal(t, true, s.subHandler.topicExists(expected), fmt.Sprintf("subnet topic %s doesn't exist", expected)) - } assert.Equal(t, true, rpcMap[p2p.RPCBlobSidecarsByRangeTopicV1+s.cfg.p2p.Encoding().ProtocolSuffix()], "topic doesn't exist") assert.Equal(t, true, rpcMap[p2p.RPCBlobSidecarsByRootTopicV1+s.cfg.p2p.Encoding().ProtocolSuffix()], "topic doesn't exist") }, @@ -162,16 +123,8 @@ func TestService_CheckForNextEpochFork(t *testing.T) { epochAtRegistration: func(e primitives.Epoch) primitives.Epoch { return e - 1 }, }, { - name: "electra fork in the next epoch", - checkRegistration: func(t *testing.T, s *Service) { - digest := params.ForkDigest(params.BeaconConfig().ElectraForkEpoch) - subIndices := mapFromCount(params.BeaconConfig().BlobsidecarSubnetCountElectra) - for idx := range subIndices { - topic := fmt.Sprintf(p2p.BlobSubnetTopicFormat, digest, idx) - expected := topic + s.cfg.p2p.Encoding().ProtocolSuffix() - assert.Equal(t, true, s.subHandler.topicExists(expected), fmt.Sprintf("subnet topic %s doesn't exist", expected)) - } - }, + name: "electra fork in the next epoch", + checkRegistration: func(t *testing.T, s *Service) {}, forkEpoch: params.BeaconConfig().ElectraForkEpoch, nextForkEpoch: params.BeaconConfig().FuluForkEpoch, epochAtRegistration: func(e primitives.Epoch) primitives.Epoch { return e - 1 }, @@ -203,17 +156,12 @@ func TestService_CheckForNextEpochFork(t *testing.T) { return } - // Ensure the topics were registered for the upcoming fork - digest := params.ForkDigest(tt.forkEpoch) - assert.Equal(t, true, s.subHandler.digestExists(digest)) - // After this point we are checking deregistration, which doesn't apply if there isn't a higher // nextForkEpoch. if tt.forkEpoch >= tt.nextForkEpoch { return } - nextDigest := params.ForkDigest(tt.nextForkEpoch) // Move the clock to just before the next fork epoch and ensure deregistration is correct wg = attachSpawner(s) s.cfg.clock = defaultClockWithTimeAtEpoch(tt.nextForkEpoch - 1) @@ -221,14 +169,9 @@ func TestService_CheckForNextEpochFork(t *testing.T) { wg.Wait() require.NoError(t, s.ensureRPCDeregistrationForEpoch(tt.nextForkEpoch)) - assert.Equal(t, true, s.subHandler.digestExists(digest)) - // deregister as if it is the epoch after the next fork epoch - require.NoError(t, s.ensureRPCDeregistrationForEpoch(tt.nextForkEpoch+1)) - assert.Equal(t, false, s.subHandler.digestExists(digest)) - assert.Equal(t, true, s.subHandler.digestExists(nextDigest)) }) } -}*/ +} func attachSpawner(s *Service) *sync.WaitGroup { wg := new(sync.WaitGroup) From 93271050bfa6d762edb7f11febfb49eaa423a6b3 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Wed, 5 Nov 2025 21:17:03 +0400 Subject: [PATCH 10/39] more tests --- beacon-chain/sync/BUILD.bazel | 3 + beacon-chain/sync/gossipsub_topic_family.go | 1 + .../sync/gossipsub_topic_family_test.go | 311 ++++++++++++++++++ beacon-chain/sync/subscriber_test.go | 4 + .../sync/topic_families_dynamic_subnets.go | 12 + .../sync/topic_families_static_subnets.go | 4 + .../sync/topic_families_without_subnets.go | 36 ++ 7 files changed, 371 insertions(+) create mode 100644 beacon-chain/sync/gossipsub_topic_family_test.go diff --git a/beacon-chain/sync/BUILD.bazel b/beacon-chain/sync/BUILD.bazel index 47cdedfe35dc..c03e0abf5f05 100644 --- a/beacon-chain/sync/BUILD.bazel +++ b/beacon-chain/sync/BUILD.bazel @@ -179,6 +179,7 @@ go_test( "decode_pubsub_test.go", "error_test.go", "fork_watcher_test.go", + "gossipsub_topic_family_test.go", "kzg_batch_verifier_test.go", "once_test.go", "pending_attestations_queue_bucket_test.go", @@ -306,6 +307,8 @@ go_test( "@com_github_pkg_errors//:go_default_library", "@com_github_prysmaticlabs_fastssz//:go_default_library", "@com_github_prysmaticlabs_go_bitfield//:go_default_library", + "@com_github_prysmaticlabs_prysm_v4//config/params:go_default_library", + "@com_github_prysmaticlabs_prysm_v4//consensus-types/primitives:go_default_library", "@com_github_sirupsen_logrus//:go_default_library", "@com_github_sirupsen_logrus//hooks/test:go_default_library", "@com_github_stretchr_testify//require:go_default_library", diff --git a/beacon-chain/sync/gossipsub_topic_family.go b/beacon-chain/sync/gossipsub_topic_family.go index b42f27ca74b4..b6dd538814de 100644 --- a/beacon-chain/sync/gossipsub_topic_family.go +++ b/beacon-chain/sync/gossipsub_topic_family.go @@ -34,6 +34,7 @@ func (b *baseGossipsubTopicFamily) NetworkScheduleEntry() params.NetworkSchedule } type GossipsubTopicFamily interface { + Name() string Validator() wrappedVal Handler() subHandler NetworkScheduleEntry() params.NetworkScheduleEntry diff --git a/beacon-chain/sync/gossipsub_topic_family_test.go b/beacon-chain/sync/gossipsub_topic_family_test.go new file mode 100644 index 000000000000..42ca5b7f310f --- /dev/null +++ b/beacon-chain/sync/gossipsub_topic_family_test.go @@ -0,0 +1,311 @@ +package sync + +import ( + "context" + "testing" + + p2ptest "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/testing" + "github.com/OffchainLabs/prysm/v6/config/features" + "github.com/OffchainLabs/prysm/v6/config/params" + "github.com/OffchainLabs/prysm/v6/consensus-types/primitives" + "github.com/OffchainLabs/prysm/v6/testing/assert" +) + +// createMinimalService creates a minimal Service instance for testing +func createMinimalService(t *testing.T) *Service { + p2pService := p2ptest.NewTestP2P(t) + return &Service{ + cfg: &config{ + p2p: p2pService, + }, + ctx: context.Background(), + } +} + +func TestTopicFamiliesForEpoch(t *testing.T) { + // Define test epochs + const ( + genesisEpoch = primitives.Epoch(0) + altairEpoch = primitives.Epoch(100) + bellatrixEpoch = primitives.Epoch(200) + capellaEpoch = primitives.Epoch(300) + denebEpoch = primitives.Epoch(400) + electraEpoch = primitives.Epoch(500) + fuluEpoch = primitives.Epoch(600) + ) + + // Define topic families for each fork + // These names must match what's returned by the Name() method of each topic family + genesisFamilies := []string{ + "BlockTopicFamily", + "AggregateAndProofTopicFamily", + "VoluntaryExitTopicFamily", + "ProposerSlashingTopicFamily", + "AttesterSlashingTopicFamily", + "AttestationTopicFamily", + } + + altairFamilies := []string{ + "SyncContributionAndProofTopicFamily", + "SyncCommitteeTopicFamily", + } + + altairLightClientFamilies := []string{ + "LightClientOptimisticUpdateTopicFamily", + "LightClientFinalityUpdateTopicFamily", + } + + capellaFamilies := []string{ + "BlsToExecutionChangeTopicFamily", + } + + denebBlobFamilies := []string{ + "BlobTopicFamily-0", + "BlobTopicFamily-1", + "BlobTopicFamily-2", + "BlobTopicFamily-3", + "BlobTopicFamily-4", + "BlobTopicFamily-5", + } + + electraBlobFamilies := denebBlobFamilies + electraBlobFamilies = append(denebBlobFamilies, "BlobTopicFamily-6", "BlobTopicFamily-7") + + fuluFamilies := []string{ + "DataColumnTopicFamily", + } + + // Helper function to combine fork families + combineForks := func(forkSets ...[]string) []string { + var combined []string + for _, forkSet := range forkSets { + combined = append(combined, forkSet...) + } + return combined + } + + tests := []struct { + name string + epoch primitives.Epoch + setupConfig func() + enableLightClient bool + expectedFamilies []string + }{ + { + name: "epoch before any fork activation should return empty", + epoch: primitives.Epoch(0), + setupConfig: func() { + config := params.BeaconConfig().Copy() + // Set all fork epochs to future epochs + config.GenesisEpoch = primitives.Epoch(1000) + config.AltairForkEpoch = primitives.Epoch(2000) + config.BellatrixForkEpoch = primitives.Epoch(3000) + config.CapellaForkEpoch = primitives.Epoch(4000) + config.DenebForkEpoch = primitives.Epoch(5000) + config.ElectraForkEpoch = primitives.Epoch(6000) + config.FuluForkEpoch = primitives.Epoch(7000) + params.OverrideBeaconConfig(config) + }, + expectedFamilies: []string{}, + }, + { + name: "epoch at genesis should return genesis topic families", + epoch: genesisEpoch, + setupConfig: func() { + config := params.BeaconConfig().Copy() + config.GenesisEpoch = genesisEpoch + config.AltairForkEpoch = altairEpoch + config.BellatrixForkEpoch = bellatrixEpoch + config.CapellaForkEpoch = capellaEpoch + config.DenebForkEpoch = denebEpoch + config.ElectraForkEpoch = electraEpoch + config.FuluForkEpoch = fuluEpoch + params.OverrideBeaconConfig(config) + }, + expectedFamilies: genesisFamilies, + }, + { + name: "epoch at Altair without light client should have genesis + Altair families", + epoch: altairEpoch, + enableLightClient: false, + setupConfig: func() { + config := params.BeaconConfig().Copy() + config.GenesisEpoch = genesisEpoch + config.AltairForkEpoch = altairEpoch + config.BellatrixForkEpoch = bellatrixEpoch + config.CapellaForkEpoch = capellaEpoch + config.DenebForkEpoch = denebEpoch + config.ElectraForkEpoch = electraEpoch + config.FuluForkEpoch = fuluEpoch + params.OverrideBeaconConfig(config) + }, + expectedFamilies: combineForks(genesisFamilies, altairFamilies), + }, + { + name: "epoch at Altair with light client enabled should include light client families", + epoch: altairEpoch, + enableLightClient: true, + setupConfig: func() { + config := params.BeaconConfig().Copy() + config.GenesisEpoch = genesisEpoch + config.AltairForkEpoch = altairEpoch + config.BellatrixForkEpoch = bellatrixEpoch + config.CapellaForkEpoch = capellaEpoch + config.DenebForkEpoch = denebEpoch + config.ElectraForkEpoch = electraEpoch + config.FuluForkEpoch = fuluEpoch + params.OverrideBeaconConfig(config) + }, + expectedFamilies: combineForks(genesisFamilies, altairFamilies, altairLightClientFamilies), + }, + { + name: "epoch at Capella should have genesis + Altair + Capella families", + epoch: capellaEpoch, + setupConfig: func() { + config := params.BeaconConfig().Copy() + config.GenesisEpoch = genesisEpoch + config.AltairForkEpoch = altairEpoch + config.BellatrixForkEpoch = bellatrixEpoch + config.CapellaForkEpoch = capellaEpoch + config.DenebForkEpoch = denebEpoch + config.ElectraForkEpoch = electraEpoch + config.FuluForkEpoch = fuluEpoch + params.OverrideBeaconConfig(config) + }, + expectedFamilies: combineForks(genesisFamilies, altairFamilies, capellaFamilies), + }, + { + name: "epoch at Deneb should include blob sidecars", + epoch: denebEpoch, + setupConfig: func() { + config := params.BeaconConfig().Copy() + config.GenesisEpoch = genesisEpoch + config.AltairForkEpoch = altairEpoch + config.BellatrixForkEpoch = bellatrixEpoch + config.CapellaForkEpoch = capellaEpoch + config.DenebForkEpoch = denebEpoch + config.ElectraForkEpoch = electraEpoch + config.FuluForkEpoch = fuluEpoch + config.BlobsidecarSubnetCount = 6 // Deneb has 6 blob subnets + params.OverrideBeaconConfig(config) + }, + expectedFamilies: combineForks(genesisFamilies, altairFamilies, capellaFamilies, denebBlobFamilies), + }, + { + name: "epoch at Electra should have Electra blobs not Deneb blobs", + epoch: electraEpoch, + setupConfig: func() { + config := params.BeaconConfig().Copy() + config.GenesisEpoch = genesisEpoch + config.AltairForkEpoch = altairEpoch + config.BellatrixForkEpoch = bellatrixEpoch + config.CapellaForkEpoch = capellaEpoch + config.DenebForkEpoch = denebEpoch + config.ElectraForkEpoch = electraEpoch + config.FuluForkEpoch = fuluEpoch + config.BlobsidecarSubnetCount = 6 + config.BlobsidecarSubnetCountElectra = 8 // Electra has 8 blob subnets + params.OverrideBeaconConfig(config) + }, + expectedFamilies: combineForks(genesisFamilies, altairFamilies, capellaFamilies, electraBlobFamilies), + }, + { + name: "epoch at Fulu should have data columns not blobs", + epoch: fuluEpoch, + setupConfig: func() { + config := params.BeaconConfig().Copy() + config.GenesisEpoch = genesisEpoch + config.AltairForkEpoch = altairEpoch + config.BellatrixForkEpoch = bellatrixEpoch + config.CapellaForkEpoch = capellaEpoch + config.DenebForkEpoch = denebEpoch + config.ElectraForkEpoch = electraEpoch + config.FuluForkEpoch = fuluEpoch + config.BlobsidecarSubnetCount = 6 + config.BlobsidecarSubnetCountElectra = 8 + params.OverrideBeaconConfig(config) + }, + expectedFamilies: combineForks(genesisFamilies, altairFamilies, capellaFamilies, fuluFamilies), + }, + { + name: "epoch after Fulu should maintain Fulu families", + epoch: fuluEpoch + 100, + setupConfig: func() { + config := params.BeaconConfig().Copy() + config.GenesisEpoch = genesisEpoch + config.AltairForkEpoch = altairEpoch + config.BellatrixForkEpoch = bellatrixEpoch + config.CapellaForkEpoch = capellaEpoch + config.DenebForkEpoch = denebEpoch + config.ElectraForkEpoch = electraEpoch + config.FuluForkEpoch = fuluEpoch + config.BlobsidecarSubnetCount = 6 + config.BlobsidecarSubnetCountElectra = 8 + params.OverrideBeaconConfig(config) + }, + expectedFamilies: combineForks(genesisFamilies, altairFamilies, capellaFamilies, fuluFamilies), + }, + { + name: "edge case - epoch exactly at deactivation should not include deactivated family", + epoch: electraEpoch, // This deactivates Deneb blobs + setupConfig: func() { + config := params.BeaconConfig().Copy() + config.GenesisEpoch = genesisEpoch + config.AltairForkEpoch = altairEpoch + config.BellatrixForkEpoch = bellatrixEpoch + config.CapellaForkEpoch = capellaEpoch + config.DenebForkEpoch = denebEpoch + config.ElectraForkEpoch = electraEpoch + config.FuluForkEpoch = fuluEpoch + config.BlobsidecarSubnetCount = 6 + config.BlobsidecarSubnetCountElectra = 8 + params.OverrideBeaconConfig(config) + }, + expectedFamilies: combineForks(genesisFamilies, altairFamilies, capellaFamilies, electraBlobFamilies), + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + params.SetupTestConfigCleanup(t) + if tt.enableLightClient { + resetFlags := features.InitWithReset(&features.Flags{ + EnableLightClient: true, + }) + defer resetFlags() + } + tt.setupConfig() + service := createMinimalService(t) + families := TopicFamiliesForEpoch(tt.epoch, service, params.NetworkScheduleEntry{}) + + // Collect actual family names + actualFamilies := make([]string, 0, len(families)) + for _, family := range families { + actualFamilies = append(actualFamilies, family.Name()) + } + + // Assert exact match - families should have exactly the expected families and nothing more + assert.Equal(t, len(tt.expectedFamilies), len(actualFamilies), + "Expected %d families but got %d", len(tt.expectedFamilies), len(actualFamilies)) + + // Create a map for efficient lookup + expectedMap := make(map[string]bool) + for _, expected := range tt.expectedFamilies { + expectedMap[expected] = true + } + + // Check each actual family is expected + for _, actual := range actualFamilies { + if !expectedMap[actual] { + t.Errorf("Unexpected topic family found: %s", actual) + } + delete(expectedMap, actual) // Remove from map as we find it + } + + // Check all expected families were found (anything left in map was missing) + for missing := range expectedMap { + t.Errorf("Expected topic family not found: %s", missing) + } + }) + } +} diff --git a/beacon-chain/sync/subscriber_test.go b/beacon-chain/sync/subscriber_test.go index a29d330abf24..cda0532dc91e 100644 --- a/beacon-chain/sync/subscriber_test.go +++ b/beacon-chain/sync/subscriber_test.go @@ -92,6 +92,10 @@ type testDynamicFamily struct { subnetsForCast func(primitives.Slot) map[uint64]bool } +func (t *testDynamicFamily) Name() string { + return "TestDynamicFamily" +} + func (t *testDynamicFamily) Validator() wrappedVal { return t.validator } diff --git a/beacon-chain/sync/topic_families_dynamic_subnets.go b/beacon-chain/sync/topic_families_dynamic_subnets.go index 59d61750e390..710b16576eaa 100644 --- a/beacon-chain/sync/topic_families_dynamic_subnets.go +++ b/beacon-chain/sync/topic_families_dynamic_subnets.go @@ -63,6 +63,10 @@ func NewAttestationTopicFamily(s *Service, nse params.NetworkScheduleEntry) *Att return attestationTopicFamily } +func (a *AttestationTopicFamily) Name() string { + return "AttestationTopicFamily" +} + // Validator returns the validator function for attestation subnets. func (a *AttestationTopicFamily) Validator() wrappedVal { return a.syncService.validateCommitteeIndexBeaconAttestation @@ -121,6 +125,10 @@ func NewSyncCommitteeTopicFamily(s *Service, nse params.NetworkScheduleEntry) *S } } +func (s *SyncCommitteeTopicFamily) Name() string { + return "SyncCommitteeTopicFamily" +} + // Validator returns the validator function for sync committee subnets. func (s *SyncCommitteeTopicFamily) Validator() wrappedVal { return s.syncService.validateSyncCommitteeMessage @@ -179,6 +187,10 @@ func NewDataColumnTopicFamily(s *Service, nse params.NetworkScheduleEntry) *Data } } +func (d *DataColumnTopicFamily) Name() string { + return "DataColumnTopicFamily" +} + // Validator returns the validator function for data column subnets. func (d *DataColumnTopicFamily) Validator() wrappedVal { return d.syncService.validateDataColumn diff --git a/beacon-chain/sync/topic_families_static_subnets.go b/beacon-chain/sync/topic_families_static_subnets.go index d6224f979c0e..bcec01ef2178 100644 --- a/beacon-chain/sync/topic_families_static_subnets.go +++ b/beacon-chain/sync/topic_families_static_subnets.go @@ -26,6 +26,10 @@ func NewBlobTopicFamily(s *Service, nse params.NetworkScheduleEntry, subnetIndex } } +func (b *BlobTopicFamily) Name() string { + return fmt.Sprintf("BlobTopicFamily-%d", b.subnetIndex) +} + func (b *BlobTopicFamily) Validator() wrappedVal { return b.syncService.validateBlob } diff --git a/beacon-chain/sync/topic_families_without_subnets.go b/beacon-chain/sync/topic_families_without_subnets.go index e5358fcc8953..64944a453313 100644 --- a/beacon-chain/sync/topic_families_without_subnets.go +++ b/beacon-chain/sync/topic_families_without_subnets.go @@ -23,6 +23,10 @@ func NewBlockTopicFamily(s *Service, nse params.NetworkScheduleEntry) *BlockTopi } } +func (b *BlockTopicFamily) Name() string { + return "BlockTopicFamily" +} + func (b *BlockTopicFamily) Validator() wrappedVal { return b.syncService.validateBeaconBlockPubSub } @@ -59,6 +63,10 @@ func NewAggregateAndProofTopicFamily(s *Service, nse params.NetworkScheduleEntry } } +func (a *AggregateAndProofTopicFamily) Name() string { + return "AggregateAndProofTopicFamily" +} + func (a *AggregateAndProofTopicFamily) Validator() wrappedVal { return a.syncService.validateAggregateAndProof } @@ -95,6 +103,10 @@ func NewVoluntaryExitTopicFamily(s *Service, nse params.NetworkScheduleEntry) *V } } +func (v *VoluntaryExitTopicFamily) Name() string { + return "VoluntaryExitTopicFamily" +} + func (v *VoluntaryExitTopicFamily) Validator() wrappedVal { return v.syncService.validateVoluntaryExit } @@ -131,6 +143,10 @@ func NewProposerSlashingTopicFamily(s *Service, nse params.NetworkScheduleEntry) } } +func (p *ProposerSlashingTopicFamily) Name() string { + return "ProposerSlashingTopicFamily" +} + func (p *ProposerSlashingTopicFamily) Validator() wrappedVal { return p.syncService.validateProposerSlashing } @@ -167,6 +183,10 @@ func NewAttesterSlashingTopicFamily(s *Service, nse params.NetworkScheduleEntry) } } +func (a *AttesterSlashingTopicFamily) Name() string { + return "AttesterSlashingTopicFamily" +} + func (a *AttesterSlashingTopicFamily) Validator() wrappedVal { return a.syncService.validateAttesterSlashing } @@ -202,6 +222,10 @@ func NewSyncContributionAndProofTopicFamily(s *Service, nse params.NetworkSchedu } } +func (sc *SyncContributionAndProofTopicFamily) Name() string { + return "SyncContributionAndProofTopicFamily" +} + func (sc *SyncContributionAndProofTopicFamily) Validator() wrappedVal { return sc.syncService.validateSyncContributionAndProof } @@ -238,6 +262,10 @@ func NewLightClientOptimisticUpdateTopicFamily(s *Service, nse params.NetworkSch } } +func (l *LightClientOptimisticUpdateTopicFamily) Name() string { + return "LightClientOptimisticUpdateTopicFamily" +} + func (l *LightClientOptimisticUpdateTopicFamily) Validator() wrappedVal { return l.syncService.validateLightClientOptimisticUpdate } @@ -274,6 +302,10 @@ func NewLightClientFinalityUpdateTopicFamily(s *Service, nse params.NetworkSched } } +func (l *LightClientFinalityUpdateTopicFamily) Name() string { + return "LightClientFinalityUpdateTopicFamily" +} + func (l *LightClientFinalityUpdateTopicFamily) Validator() wrappedVal { return l.syncService.validateLightClientFinalityUpdate } @@ -309,6 +341,10 @@ func NewBlsToExecutionChangeTopicFamily(s *Service, nse params.NetworkScheduleEn } } +func (b *BlsToExecutionChangeTopicFamily) Name() string { + return "BlsToExecutionChangeTopicFamily" +} + func (b *BlsToExecutionChangeTopicFamily) Validator() wrappedVal { return b.syncService.validateBlsToExecutionChange } From 798376b1d7f10b5fce43d1e5dfb6b6867c0f35e2 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Wed, 5 Nov 2025 21:21:51 +0400 Subject: [PATCH 11/39] fix bazel --- beacon-chain/sync/BUILD.bazel | 2 -- 1 file changed, 2 deletions(-) diff --git a/beacon-chain/sync/BUILD.bazel b/beacon-chain/sync/BUILD.bazel index c03e0abf5f05..d62bc1542d06 100644 --- a/beacon-chain/sync/BUILD.bazel +++ b/beacon-chain/sync/BUILD.bazel @@ -307,8 +307,6 @@ go_test( "@com_github_pkg_errors//:go_default_library", "@com_github_prysmaticlabs_fastssz//:go_default_library", "@com_github_prysmaticlabs_go_bitfield//:go_default_library", - "@com_github_prysmaticlabs_prysm_v4//config/params:go_default_library", - "@com_github_prysmaticlabs_prysm_v4//consensus-types/primitives:go_default_library", "@com_github_sirupsen_logrus//:go_default_library", "@com_github_sirupsen_logrus//hooks/test:go_default_library", "@com_github_stretchr_testify//require:go_default_library", From a15a1ade17c93b06e1bea1bca7524fc114c5705a Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Wed, 5 Nov 2025 21:31:04 +0400 Subject: [PATCH 12/39] fix test --- beacon-chain/sync/gossipsub_topic_family_test.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/beacon-chain/sync/gossipsub_topic_family_test.go b/beacon-chain/sync/gossipsub_topic_family_test.go index 42ca5b7f310f..5f329fed7b7a 100644 --- a/beacon-chain/sync/gossipsub_topic_family_test.go +++ b/beacon-chain/sync/gossipsub_topic_family_test.go @@ -68,8 +68,7 @@ func TestTopicFamiliesForEpoch(t *testing.T) { "BlobTopicFamily-5", } - electraBlobFamilies := denebBlobFamilies - electraBlobFamilies = append(denebBlobFamilies, "BlobTopicFamily-6", "BlobTopicFamily-7") + electraBlobFamilies := append(append([]string{}, denebBlobFamilies...), "BlobTopicFamily-6", "BlobTopicFamily-7") fuluFamilies := []string{ "DataColumnTopicFamily", From 3f7371445b77ec75cf84d31d57b8f13fe01b3dc2 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Wed, 5 Nov 2025 22:23:26 +0400 Subject: [PATCH 13/39] fix test in sync --- testing/endtoend/evaluators/node.go | 1 + 1 file changed, 1 insertion(+) diff --git a/testing/endtoend/evaluators/node.go b/testing/endtoend/evaluators/node.go index bdbd0dbdca76..111c1b72591e 100644 --- a/testing/endtoend/evaluators/node.go +++ b/testing/endtoend/evaluators/node.go @@ -54,6 +54,7 @@ var AllNodesHaveSameHead = e2etypes.Evaluator{ } func healthzCheck(_ *e2etypes.EvaluationContext, conns ...*grpc.ClientConn) error { + count := len(conns) for i := 0; i < count; i++ { resp, err := http.Get(fmt.Sprintf("http://localhost:%d/healthz", e2e.TestParams.Ports.PrysmBeaconNodeMetricsPort+i)) From 614367ddcf4899c7af9667c6d6d2f2fda4f94f27 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Wed, 5 Nov 2025 22:26:55 +0400 Subject: [PATCH 14/39] fix lint --- testing/endtoend/evaluators/node.go | 1 - 1 file changed, 1 deletion(-) diff --git a/testing/endtoend/evaluators/node.go b/testing/endtoend/evaluators/node.go index 111c1b72591e..bdbd0dbdca76 100644 --- a/testing/endtoend/evaluators/node.go +++ b/testing/endtoend/evaluators/node.go @@ -54,7 +54,6 @@ var AllNodesHaveSameHead = e2etypes.Evaluator{ } func healthzCheck(_ *e2etypes.EvaluationContext, conns ...*grpc.ClientConn) error { - count := len(conns) for i := 0; i < count; i++ { resp, err := http.Get(fmt.Sprintf("http://localhost:%d/healthz", e2e.TestParams.Ports.PrysmBeaconNodeMetricsPort+i)) From 71050ab076bac94601d8747ad4be04687b34b652 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Thu, 6 Nov 2025 19:10:19 +0400 Subject: [PATCH 15/39] fix bazel --- beacon-chain/p2p/testing/p2p.go | 3 + beacon-chain/sync/BUILD.bazel | 1 + beacon-chain/sync/fork_watcher_test.go | 17 -- beacon-chain/sync/gossipsub_controller.go | 2 +- .../sync/gossipsub_controller_test.go | 181 ++++++++++++++++++ beacon-chain/sync/service.go | 1 - 6 files changed, 186 insertions(+), 19 deletions(-) create mode 100644 beacon-chain/sync/gossipsub_controller_test.go diff --git a/beacon-chain/p2p/testing/p2p.go b/beacon-chain/p2p/testing/p2p.go index d3be2285f0b8..65bf86d49df3 100644 --- a/beacon-chain/p2p/testing/p2p.go +++ b/beacon-chain/p2p/testing/p2p.go @@ -274,6 +274,9 @@ func (p *TestP2P) SubscribeToTopic(topic string, opts ...pubsub.SubOpt) (*pubsub // LeaveTopic closes topic and removes corresponding handler from list of joined topics. // This method will return error if there are outstanding event handlers or subscriptions. func (p *TestP2P) LeaveTopic(topic string) error { + p.mu.Lock() + defer p.mu.Unlock() + if t, ok := p.joinedTopics[topic]; ok { if err := t.Close(); err != nil { return err diff --git a/beacon-chain/sync/BUILD.bazel b/beacon-chain/sync/BUILD.bazel index d62bc1542d06..ffb9a5d5728c 100644 --- a/beacon-chain/sync/BUILD.bazel +++ b/beacon-chain/sync/BUILD.bazel @@ -179,6 +179,7 @@ go_test( "decode_pubsub_test.go", "error_test.go", "fork_watcher_test.go", + "gossipsub_controller_test.go", "gossipsub_topic_family_test.go", "kzg_batch_verifier_test.go", "once_test.go", diff --git a/beacon-chain/sync/fork_watcher_test.go b/beacon-chain/sync/fork_watcher_test.go index 97832f60c4de..abbec8af070b 100644 --- a/beacon-chain/sync/fork_watcher_test.go +++ b/beacon-chain/sync/fork_watcher_test.go @@ -2,7 +2,6 @@ package sync import ( "context" - "sync" "testing" "time" @@ -147,9 +146,7 @@ func TestService_CheckForNextEpochFork(t *testing.T) { t.Run(tt.name, func(t *testing.T) { current := tt.epochAtRegistration(tt.forkEpoch) s := testForkWatcherService(t, current) - wg := attachSpawner(s) require.NoError(t, s.ensureRPCRegistrationsForEpoch(s.cfg.clock.CurrentEpoch())) - wg.Wait() tt.checkRegistration(t, s) if current != tt.forkEpoch-1 { @@ -163,28 +160,14 @@ func TestService_CheckForNextEpochFork(t *testing.T) { } // Move the clock to just before the next fork epoch and ensure deregistration is correct - wg = attachSpawner(s) s.cfg.clock = defaultClockWithTimeAtEpoch(tt.nextForkEpoch - 1) require.NoError(t, s.ensureRPCRegistrationsForEpoch(s.cfg.clock.CurrentEpoch())) - wg.Wait() require.NoError(t, s.ensureRPCDeregistrationForEpoch(tt.nextForkEpoch)) }) } } -func attachSpawner(s *Service) *sync.WaitGroup { - wg := new(sync.WaitGroup) - s.subscriptionSpawner = func(f func()) { - wg.Add(1) - go func() { - defer wg.Done() - f() - }() - } - return wg -} - // oneEpoch returns the duration of one epoch. func oneEpoch() time.Duration { return time.Duration(params.BeaconConfig().SlotsPerEpoch.Mul(params.BeaconConfig().SecondsPerSlot)) * time.Second diff --git a/beacon-chain/sync/gossipsub_controller.go b/beacon-chain/sync/gossipsub_controller.go index 21e8dbdc7147..2e8ba9643699 100644 --- a/beacon-chain/sync/gossipsub_controller.go +++ b/beacon-chain/sync/gossipsub_controller.go @@ -17,7 +17,7 @@ type topicFamilyKey struct { } func topicFamilyKeyFrom(tf GossipsubTopicFamily) topicFamilyKey { - return topicFamilyKey{topicName: fmt.Sprintf("%T", tf), forkDigest: tf.NetworkScheduleEntry().ForkDigest} + return topicFamilyKey{topicName: fmt.Sprintf("%s", tf.Name()), forkDigest: tf.NetworkScheduleEntry().ForkDigest} } type GossipsubController struct { diff --git a/beacon-chain/sync/gossipsub_controller_test.go b/beacon-chain/sync/gossipsub_controller_test.go new file mode 100644 index 000000000000..2059ca2fcecd --- /dev/null +++ b/beacon-chain/sync/gossipsub_controller_test.go @@ -0,0 +1,181 @@ +package sync + +import ( + "context" + "fmt" + "strings" + "testing" + "time" + + "github.com/OffchainLabs/prysm/v6/async/abool" + mockChain "github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/testing" + "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p" + p2ptest "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/testing" + mockSync "github.com/OffchainLabs/prysm/v6/beacon-chain/sync/initial-sync/testing" + "github.com/OffchainLabs/prysm/v6/config/params" + "github.com/OffchainLabs/prysm/v6/consensus-types/primitives" + "github.com/OffchainLabs/prysm/v6/genesis" + "github.com/OffchainLabs/prysm/v6/testing/assert" +) + +func testGossipsubControllerService(t *testing.T, current primitives.Epoch) *Service { + closedChan := make(chan struct{}) + close(closedChan) + peer2peer := p2ptest.NewTestP2P(t) + chainService := &mockChain.ChainService{ + Genesis: genesis.Time(), + ValidatorsRoot: genesis.ValidatorsRoot(), + } + ctx, cancel := context.WithTimeout(t.Context(), 10*time.Millisecond) + r := &Service{ + ctx: ctx, + cancel: cancel, + cfg: &config{ + p2p: peer2peer, + chain: chainService, + clock: defaultClockWithTimeAtEpoch(current), + initialSync: &mockSync.Sync{IsSyncing: false}, + }, + chainStarted: abool.New(), + subHandler: newSubTopicHandler(), + initialSyncComplete: closedChan, + } + r.gossipsubController = NewGossipsubController(context.Background(), r) + return r +} + +func TestGossipsubController_CheckForNextEpochForkSubscriptions(t *testing.T) { + closedChan := make(chan struct{}) + close(closedChan) + params.SetupTestConfigCleanup(t) + genesis.StoreEmbeddedDuringTest(t, params.BeaconConfig().ConfigName) + params.BeaconConfig().FuluForkEpoch = params.BeaconConfig().ElectraForkEpoch + 4096*2 + params.BeaconConfig().InitializeForkSchedule() + + tests := []struct { + name string + svcCreator func(t *testing.T) *Service + checkRegistration func(t *testing.T, s *Service) + forkEpoch primitives.Epoch + epochAtRegistration func(primitives.Epoch) primitives.Epoch + nextForkEpoch primitives.Epoch + }{ + { + name: "no fork in the next epoch", + forkEpoch: params.BeaconConfig().AltairForkEpoch, + epochAtRegistration: func(e primitives.Epoch) primitives.Epoch { return e - 2 }, + nextForkEpoch: params.BeaconConfig().BellatrixForkEpoch, + checkRegistration: func(t *testing.T, s *Service) {}, + }, + { + name: "altair fork in the next epoch", + forkEpoch: params.BeaconConfig().AltairForkEpoch, + epochAtRegistration: func(e primitives.Epoch) primitives.Epoch { return e - 1 }, + nextForkEpoch: params.BeaconConfig().BellatrixForkEpoch, + checkRegistration: func(t *testing.T, s *Service) { + digest := params.ForkDigest(params.BeaconConfig().AltairForkEpoch) + expected := fmt.Sprintf(p2p.SyncContributionAndProofSubnetTopicFormat+s.cfg.p2p.Encoding().ProtocolSuffix(), digest) + assert.Equal(t, true, s.subHandler.topicExists(expected), "subnet topic doesn't exist") + }, + }, + { + name: "capella fork in the next epoch", + checkRegistration: func(t *testing.T, s *Service) { + digest := params.ForkDigest(params.BeaconConfig().CapellaForkEpoch) + rpcMap := make(map[string]bool) + for _, p := range s.cfg.p2p.Host().Mux().Protocols() { + rpcMap[string(p)] = true + } + + expected := fmt.Sprintf(p2p.BlsToExecutionChangeSubnetTopicFormat+s.cfg.p2p.Encoding().ProtocolSuffix(), digest) + assert.Equal(t, true, s.subHandler.topicExists(expected), "subnet topic doesn't exist") + }, + forkEpoch: params.BeaconConfig().CapellaForkEpoch, + nextForkEpoch: params.BeaconConfig().DenebForkEpoch, + epochAtRegistration: func(e primitives.Epoch) primitives.Epoch { return e - 1 }, + }, + { + name: "deneb fork in the next epoch", + checkRegistration: func(t *testing.T, s *Service) { + digest := params.ForkDigest(params.BeaconConfig().DenebForkEpoch) + subIndices := mapFromCount(params.BeaconConfig().BlobsidecarSubnetCount) + for idx := range subIndices { + topic := fmt.Sprintf(p2p.BlobSubnetTopicFormat, digest, idx) + expected := topic + s.cfg.p2p.Encoding().ProtocolSuffix() + assert.Equal(t, true, s.subHandler.topicExists(expected), fmt.Sprintf("subnet topic %s doesn't exist", expected)) + } + }, + forkEpoch: params.BeaconConfig().DenebForkEpoch, + nextForkEpoch: params.BeaconConfig().ElectraForkEpoch, + epochAtRegistration: func(e primitives.Epoch) primitives.Epoch { return e - 1 }, + }, + { + name: "electra fork in the next epoch", + checkRegistration: func(t *testing.T, s *Service) { + digest := params.ForkDigest(params.BeaconConfig().ElectraForkEpoch) + subIndices := mapFromCount(params.BeaconConfig().BlobsidecarSubnetCountElectra) + for idx := range subIndices { + topic := fmt.Sprintf(p2p.BlobSubnetTopicFormat, digest, idx) + expected := topic + s.cfg.p2p.Encoding().ProtocolSuffix() + assert.Equal(t, true, s.subHandler.topicExists(expected), fmt.Sprintf("subnet topic %s doesn't exist", expected)) + } + }, + forkEpoch: params.BeaconConfig().ElectraForkEpoch, + nextForkEpoch: params.BeaconConfig().FuluForkEpoch, + epochAtRegistration: func(e primitives.Epoch) primitives.Epoch { return e - 1 }, + }, + { + name: "fulu fork in the next epoch; should not have blob topics", + checkRegistration: func(t *testing.T, s *Service) { + // Advance to two epochs after Fulu activation and assert no blob topics remain. + fulu := params.BeaconConfig().FuluForkEpoch + target := fulu + 2 + s.cfg.clock = defaultClockWithTimeAtEpoch(target) + s.gossipsubController.updateActiveTopicFamilies(s.cfg.clock.CurrentEpoch()) + + for _, topic := range s.subHandler.allTopics() { + if strings.Contains(topic, "/"+p2p.GossipBlobSidecarMessage) { + t.Fatalf("blob topic still exists after Fulu+2: %s", topic) + } + } + }, + forkEpoch: params.BeaconConfig().FuluForkEpoch, + nextForkEpoch: params.BeaconConfig().FuluForkEpoch, + epochAtRegistration: func(e primitives.Epoch) primitives.Epoch { return e - 1 }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + current := tt.epochAtRegistration(tt.forkEpoch) + s := testGossipsubControllerService(t, current) + s.gossipsubController.updateActiveTopicFamilies(s.cfg.clock.CurrentEpoch()) + tt.checkRegistration(t, s) + + if current != tt.forkEpoch-1 { + return + } + + // Ensure the topics were registered for the upcoming fork + digest := params.ForkDigest(tt.forkEpoch) + assert.Equal(t, true, s.subHandler.digestExists(digest)) + + // After this point we are checking deregistration, which doesn't apply if there isn't a higher + // nextForkEpoch. + if tt.forkEpoch >= tt.nextForkEpoch { + return + } + + nextDigest := params.ForkDigest(tt.nextForkEpoch) + // Move the clock to just before the next fork epoch and ensure deregistration is correct + s.cfg.clock = defaultClockWithTimeAtEpoch(tt.nextForkEpoch - 1) + s.gossipsubController.updateActiveTopicFamilies(s.cfg.clock.CurrentEpoch()) + + s.gossipsubController.updateActiveTopicFamilies(tt.nextForkEpoch) + assert.Equal(t, true, s.subHandler.digestExists(digest)) + // deregister as if it is the epoch after the next fork epoch + s.gossipsubController.updateActiveTopicFamilies(tt.nextForkEpoch + 1) + assert.Equal(t, false, s.subHandler.digestExists(digest)) + assert.Equal(t, true, s.subHandler.digestExists(nextDigest)) + }) + } +} diff --git a/beacon-chain/sync/service.go b/beacon-chain/sync/service.go index b8456fc4caf4..1b47a6dfc15a 100644 --- a/beacon-chain/sync/service.go +++ b/beacon-chain/sync/service.go @@ -181,7 +181,6 @@ type Service struct { lcStore *lightClient.Store dataColumnLogCh chan dataColumnLogEntry digestActions perDigestSet - subscriptionSpawner func(func()) // see Service.spawn for details gossipsubController *GossipsubController } From 5161f087fcc0361d8ad6815f650afe231e4bfe6a Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Thu, 6 Nov 2025 19:38:59 +0400 Subject: [PATCH 16/39] fix compilation --- beacon-chain/sync/gossipsub_controller_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/beacon-chain/sync/gossipsub_controller_test.go b/beacon-chain/sync/gossipsub_controller_test.go index 2059ca2fcecd..50e1d403231e 100644 --- a/beacon-chain/sync/gossipsub_controller_test.go +++ b/beacon-chain/sync/gossipsub_controller_test.go @@ -48,6 +48,7 @@ func TestGossipsubController_CheckForNextEpochForkSubscriptions(t *testing.T) { closedChan := make(chan struct{}) close(closedChan) params.SetupTestConfigCleanup(t) + genesis.StoreEmbeddedDuringTest(t, params.BeaconConfig().ConfigName) params.BeaconConfig().FuluForkEpoch = params.BeaconConfig().ElectraForkEpoch + 4096*2 params.BeaconConfig().InitializeForkSchedule() From fa7596bceb33781ea99e890f0bba2ea1c4dc1b63 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Wed, 12 Nov 2025 12:54:41 +0400 Subject: [PATCH 17/39] first draft --- beacon-chain/p2p/BUILD.bazel | 3 + beacon-chain/p2p/discovery.go | 1 + beacon-chain/p2p/gossipsub_peer_crawler.go | 410 ++++++++++++++++++ beacon-chain/p2p/gossipsubcrawler/BUILD.bazel | 17 + .../p2p/gossipsubcrawler/interface.go | 22 + beacon-chain/p2p/handshake.go | 4 + beacon-chain/p2p/interfaces.go | 2 + beacon-chain/p2p/service.go | 17 + beacon-chain/p2p/testing/fuzz_p2p.go | 6 + beacon-chain/p2p/testing/mock_peermanager.go | 6 + beacon-chain/p2p/testing/p2p.go | 33 ++ beacon-chain/sync/BUILD.bazel | 1 + beacon-chain/sync/gossipsub_controller.go | 46 ++ .../sync/gossipsub_controller_test.go | 207 ++++++++- beacon-chain/sync/service.go | 14 + beacon-chain/sync/subscriber.go | 5 + 16 files changed, 793 insertions(+), 1 deletion(-) create mode 100644 beacon-chain/p2p/gossipsub_peer_crawler.go create mode 100644 beacon-chain/p2p/gossipsubcrawler/BUILD.bazel create mode 100644 beacon-chain/p2p/gossipsubcrawler/interface.go diff --git a/beacon-chain/p2p/BUILD.bazel b/beacon-chain/p2p/BUILD.bazel index 406f756b398d..b70fc857150e 100644 --- a/beacon-chain/p2p/BUILD.bazel +++ b/beacon-chain/p2p/BUILD.bazel @@ -8,6 +8,7 @@ go_library( "config.go", "connection_gater.go", "custody.go", + "gossipsub_peer_crawler.go", "dial_relay_node.go", "discovery.go", "doc.go", @@ -51,6 +52,7 @@ go_library( "//beacon-chain/db:go_default_library", "//beacon-chain/db/kv:go_default_library", "//beacon-chain/p2p/encoder:go_default_library", + "//beacon-chain/p2p/gossipsubcrawler:go_default_library", "//beacon-chain/p2p/peers:go_default_library", "//beacon-chain/p2p/peers/peerdata:go_default_library", "//beacon-chain/p2p/peers/scorers:go_default_library", @@ -113,6 +115,7 @@ go_library( "@com_github_prysmaticlabs_fastssz//:go_default_library", "@com_github_prysmaticlabs_go_bitfield//:go_default_library", "@com_github_sirupsen_logrus//:go_default_library", + "@org_golang_x_sync//semaphore:go_default_library", "@org_golang_google_protobuf//proto:go_default_library", ], ) diff --git a/beacon-chain/p2p/discovery.go b/beacon-chain/p2p/discovery.go index a5f324ad5c1c..6d4b1b2fb3cf 100644 --- a/beacon-chain/p2p/discovery.go +++ b/beacon-chain/p2p/discovery.go @@ -554,6 +554,7 @@ func (s *Service) createListener( Bootnodes: bootNodes, PingInterval: s.cfg.PingInterval, NoFindnodeLivenessCheck: s.cfg.DisableLivenessCheck, + V5RespTimeout: 300 * time.Millisecond, } listener, err := discover.ListenV5(conn, localNode, dv5Cfg) diff --git a/beacon-chain/p2p/gossipsub_peer_crawler.go b/beacon-chain/p2p/gossipsub_peer_crawler.go new file mode 100644 index 000000000000..aa7a6ebd2390 --- /dev/null +++ b/beacon-chain/p2p/gossipsub_peer_crawler.go @@ -0,0 +1,410 @@ +package p2p + +import ( + "context" + "sync" + "time" + + "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/gossipsubcrawler" + "github.com/pkg/errors" + "golang.org/x/sync/semaphore" + + "github.com/libp2p/go-libp2p/core/peer" + + "github.com/ethereum/go-ethereum/p2p/enode" +) + +type peerStatus int + +const ( + peerStatusUnknown peerStatus = iota + peerStatusCrawled + peerStatusPinged +) + +type peerNode struct { + id enode.ID + status peerStatus + node *enode.Node + peerID peer.ID + topics map[gossipsubcrawler.Topic]struct{} +} + +type crawledPeers struct { + g *GossipsubPeerCrawler + mu sync.RWMutex + + byEnode map[enode.ID]*peerNode + byPeerId map[peer.ID]*peerNode + byTopic map[gossipsubcrawler.Topic]map[peer.ID]struct{} +} + +func (cp *crawledPeers) updateStatusToPinged(node *enode.Node) { + cp.mu.Lock() + defer cp.mu.Unlock() + + enodeID := node.ID() + existingPNode, ok := cp.byEnode[enodeID] + if !ok { + return + } + if existingPNode.node.Seq() == node.Seq() { + existingPNode.status = peerStatusPinged + return + } +} + +func (cp *crawledPeers) removePeerOnPingFailure(node *enode.Node) { + cp.mu.Lock() + defer cp.mu.Unlock() + + enodeID := node.ID() + existingPNode, ok := cp.byEnode[enodeID] + if !ok { + return + } + if existingPNode.node.Seq() == node.Seq() { + cp.updateTopicsUnlocked(existingPNode, nil) + } +} + +func (cp *crawledPeers) updateCrawledIfNewer(node *enode.Node, topics []string) { + cp.mu.Lock() + + enodeID := node.ID() + existingPNode, ok := cp.byEnode[enodeID] + if ok && existingPNode.node.Seq() >= node.Seq() { + cp.mu.Unlock() + return + } + if !ok { + peerID, err := enodeToPeerID(node) + if err != nil { + log.WithError(err).WithField("node", node.ID()).Debug("Failed to convert enode to peer ID") + cp.mu.Unlock() + return + } + existingPNode = &peerNode{ + id: enodeID, + node: node, + peerID: peerID, + topics: make(map[gossipsubcrawler.Topic]struct{}), + } + cp.byEnode[enodeID] = existingPNode + cp.byPeerId[peerID] = existingPNode + } else { + existingPNode.node = node + } + + cp.updateTopicsUnlocked(existingPNode, topics) + cp.mu.Unlock() + if len(topics) == 0 { + return + } + select { + case cp.g.pingCh <- *node: + case <-cp.g.ctx.Done(): + return + } +} + +func (cp *crawledPeers) removeTopic(topic gossipsubcrawler.Topic) { + cp.mu.Lock() + defer cp.mu.Unlock() + + // Get all peers subscribed to this topic + peers, ok := cp.byTopic[topic] + if !ok { + return // Topic doesn't exist + } + + // Remove the topic from each peer's topic list + for peerID := range peers { + if pnode, exists := cp.byPeerId[peerID]; exists { + delete(pnode.topics, topic) + } + } + + // Remove the topic from byTopic map + delete(cp.byTopic, topic) +} + +func (cp *crawledPeers) removePeerId(peerID peer.ID) { + cp.mu.Lock() + defer cp.mu.Unlock() + + pnode, exists := cp.byPeerId[peerID] + if !exists { + return + } + + // Use updateTopicsUnlocked with empty topics to remove the peer + cp.updateTopicsUnlocked(pnode, nil) +} + +func (cp *crawledPeers) removePeer(enodeID enode.ID) { + cp.mu.Lock() + defer cp.mu.Unlock() + pnode, exists := cp.byEnode[enodeID] + if !exists { + return + } + cp.updateTopicsUnlocked(pnode, nil) +} + +// setting topics to empty will remove the peer completely. +func (cp *crawledPeers) updateTopicsUnlocked(pnode *peerNode, topics []string) { + // If topics is empty, remove the peer completely. + if len(topics) == 0 { + delete(cp.byPeerId, pnode.peerID) + delete(cp.byEnode, pnode.id) + for t := range pnode.topics { + if peers, ok := cp.byTopic[t]; ok { + delete(peers, pnode.peerID) + if len(peers) == 0 { + delete(cp.byTopic, t) + } + } + } + pnode.topics = nil // Clear topics to indicate removal. + return + } + + newTopics := make(map[gossipsubcrawler.Topic]struct{}) + for _, t := range topics { + newTopics[gossipsubcrawler.Topic(t)] = struct{}{} + } + + // Remove old topics that are no longer present. + for oldTopic := range pnode.topics { + if _, exists := newTopics[oldTopic]; !exists { + if peers, ok := cp.byTopic[oldTopic]; ok { + delete(peers, pnode.peerID) + if len(peers) == 0 { + delete(cp.byTopic, oldTopic) + } + } + } + } + + // Add new topics. + for newTopic := range newTopics { + if _, exists := pnode.topics[newTopic]; !exists { + if _, ok := cp.byTopic[newTopic]; !ok { + cp.byTopic[newTopic] = make(map[peer.ID]struct{}) + } + cp.byTopic[newTopic][pnode.peerID] = struct{}{} + } + } + pnode.topics = newTopics +} + +type GossipsubPeerCrawler struct { + ctx context.Context + cancel context.CancelFunc + + crawlInterval time.Duration + crawlTimeout time.Duration + + crawledPeers *crawledPeers + + // Discovery interface for finding peers + dv5 ListenerRebooter + + service *Service + + topicExtractor gossipsubcrawler.TopicExtractor + + maxConcurrentPings int + pingCh chan enode.Node + pingSemaphore *semaphore.Weighted + + wg sync.WaitGroup + once sync.Once +} + +func NewGossipsubPeerCrawler( + service *Service, + dv5 ListenerRebooter, + crawlTimeout time.Duration, + crawlInterval time.Duration, + maxConcurrentPings int, +) (*GossipsubPeerCrawler, error) { + if service == nil { + return nil, errors.New("service is nil") + } + if dv5 == nil { + return nil, errors.New("dv5 is nil") + } + if crawlTimeout <= 0 { + return nil, errors.New("crawl timeout must be greater than 0") + } + if crawlInterval <= 0 { + return nil, errors.New("crawl interval must be greater than 0") + } + if maxConcurrentPings <= 0 { + return nil, errors.New("max concurrent pings must be greater than 0") + } + + ctx, cancel := context.WithCancel(context.Background()) + g := &GossipsubPeerCrawler{ + ctx: ctx, + cancel: cancel, + crawlInterval: crawlInterval, + crawlTimeout: crawlTimeout, + service: service, + dv5: dv5, + maxConcurrentPings: maxConcurrentPings, + } + g.pingCh = make(chan enode.Node, 4*g.maxConcurrentPings) + g.pingSemaphore = semaphore.NewWeighted(int64(g.maxConcurrentPings)) + g.crawledPeers = &crawledPeers{ + g: g, + byEnode: make(map[enode.ID]*peerNode), + byPeerId: make(map[peer.ID]*peerNode), + byTopic: make(map[gossipsubcrawler.Topic]map[peer.ID]struct{}), + } + return g, nil +} +func (g *GossipsubPeerCrawler) PeersForTopic(topic gossipsubcrawler.Topic) []*enode.Node { + g.crawledPeers.mu.RLock() + defer g.crawledPeers.mu.RUnlock() + + peerIDs, ok := g.crawledPeers.byTopic[topic] + if !ok { + return nil + } + + var nodes []*enode.Node + for peerID := range peerIDs { + peerNode, ok := g.crawledPeers.byPeerId[peerID] + if !ok { + continue + } + if peerNode.status == peerStatusPinged { + nodes = append(nodes, peerNode.node) + } + } + + return nodes +} + +func (g *GossipsubPeerCrawler) RemovePeerId(peerID peer.ID) { + g.crawledPeers.removePeerId(peerID) +} + +func (g *GossipsubPeerCrawler) RemoveTopic(topic gossipsubcrawler.Topic) { + g.crawledPeers.removeTopic(topic) +} + +// Start runs the crawler's loops in the background. +func (g *GossipsubPeerCrawler) Start(te gossipsubcrawler.TopicExtractor) error { + if te == nil { + return errors.New("topic extractor is nil") + } + g.once.Do(func() { + g.topicExtractor = te + g.wg.Go(func() { + g.crawlLoop() + }) + g.wg.Go(func() { + g.pingLoop() + }) + }) + + return nil +} + +// Stop terminates the crawler. +func (g *GossipsubPeerCrawler) Stop() { + g.cancel() + g.wg.Wait() +} + +func (g *GossipsubPeerCrawler) pingLoop() { + for { + select { + case node := <-g.pingCh: + if err := g.pingSemaphore.Acquire(g.ctx, 1); err != nil { + return // Context cancelled, exit loop. + } + go func(node *enode.Node) { + defer g.pingSemaphore.Release(1) + + if err := g.dv5.Ping(node); err != nil { + g.crawledPeers.removePeerOnPingFailure(node) + return + } + + g.crawledPeers.updateStatusToPinged(node) + }(&node) + + case <-g.ctx.Done(): + return + } + } +} + +func (g *GossipsubPeerCrawler) crawlLoop() { + ticker := time.NewTicker(g.crawlInterval) + defer ticker.Stop() + + g.crawl() + for { + select { + case <-ticker.C: + g.crawl() + case <-g.ctx.Done(): + return + } + } +} + +func (g *GossipsubPeerCrawler) crawl() { + ctx, cancel := context.WithTimeout(g.ctx, g.crawlTimeout) + defer cancel() + + iterator := g.dv5.RandomNodes() + + // Ensure iterator unblocks on context cancellation or timeout + go func() { + <-ctx.Done() + iterator.Close() + }() + + for iterator.Next() { + if ctx.Err() != nil { + return + } + + node := iterator.Node() + if node == nil { + continue + } + + if !g.service.filterPeer(node) { + g.crawledPeers.removePeer(node.ID()) + continue + } + + topics, err := g.topicExtractor(ctx, node) + if err != nil { + log.WithError(err).WithField("node", node.ID()).Debug("Failed to extract topics, skipping") + continue + } + + g.crawledPeers.updateCrawledIfNewer(node, topics) + } +} + +// enodeToPeerID converts an enode record to a peer ID. +func enodeToPeerID(n *enode.Node) (peer.ID, error) { + info, _, err := convertToAddrInfo(n) + if err != nil { + return "", err + } + if info == nil { + return "", errors.New("peer info is nil") + } + return info.ID, nil +} diff --git a/beacon-chain/p2p/gossipsubcrawler/BUILD.bazel b/beacon-chain/p2p/gossipsubcrawler/BUILD.bazel new file mode 100644 index 000000000000..1221c4583cf1 --- /dev/null +++ b/beacon-chain/p2p/gossipsubcrawler/BUILD.bazel @@ -0,0 +1,17 @@ +load("@prysm//tools/go:def.bzl", "go_library") + +go_library( + name = "go_default_library", + srcs = [ + "interface.go", + ], + importpath = "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/gossipsubcrawler", + visibility = [ + "//visibility:public", + ], + deps = [ + "@com_github_ethereum_go_ethereum//p2p/enode:go_default_library", + "@com_github_libp2p_go_libp2p//core/peer:go_default_library", + ], +) + diff --git a/beacon-chain/p2p/gossipsubcrawler/interface.go b/beacon-chain/p2p/gossipsubcrawler/interface.go new file mode 100644 index 000000000000..5e2a5895ab68 --- /dev/null +++ b/beacon-chain/p2p/gossipsubcrawler/interface.go @@ -0,0 +1,22 @@ +package gossipsubcrawler + +import ( + "context" + + "github.com/ethereum/go-ethereum/p2p/enode" + "github.com/libp2p/go-libp2p/core/peer" +) + +type Topic string + +// TopicExtractor is a function that can determine the set of topics a current or potential peer +// is subscribed to based on key/value pairs from the ENR record. +type TopicExtractor func(ctx context.Context, node *enode.Node) ([]string, error) + +type Crawler interface { + Start(te TopicExtractor) error + Stop() + RemovePeerId(peerID peer.ID) + RemoveTopic(topic Topic) + PeersForTopic(topic Topic) []*enode.Node +} diff --git a/beacon-chain/p2p/handshake.go b/beacon-chain/p2p/handshake.go index 0b7f66c14a20..851ad720b91c 100644 --- a/beacon-chain/p2p/handshake.go +++ b/beacon-chain/p2p/handshake.go @@ -225,6 +225,10 @@ func (s *Service) AddDisconnectionHandler(handler func(ctx context.Context, id p return } + if s.crawler != nil { + s.crawler.RemovePeerId(peerID) + } + priorState, err := s.peers.ConnectionState(peerID) if err != nil { // Can happen if the peer has already disconnected, so... diff --git a/beacon-chain/p2p/interfaces.go b/beacon-chain/p2p/interfaces.go index 82e79b3f987c..7cdbf4db0d79 100644 --- a/beacon-chain/p2p/interfaces.go +++ b/beacon-chain/p2p/interfaces.go @@ -4,6 +4,7 @@ import ( "context" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/encoder" + "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/gossipsubcrawler" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers" "github.com/OffchainLabs/prysm/v6/consensus-types/blocks" "github.com/OffchainLabs/prysm/v6/consensus-types/interfaces" @@ -102,6 +103,7 @@ type ( RefreshPersistentSubnets() FindAndDialPeersWithSubnets(ctx context.Context, fullTopicForSubnet func(uint64) string, minimumPeersPerSubnet int, subnets map[uint64]bool) error AddPingMethod(reqFunc func(ctx context.Context, id peer.ID) error) + Crawler() gossipsubcrawler.Crawler } // Sender abstracts the sending functionality from libp2p. diff --git a/beacon-chain/p2p/service.go b/beacon-chain/p2p/service.go index c9e3a916b257..35355507e9d0 100644 --- a/beacon-chain/p2p/service.go +++ b/beacon-chain/p2p/service.go @@ -11,6 +11,7 @@ import ( "github.com/OffchainLabs/prysm/v6/async" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/encoder" + "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/gossipsubcrawler" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers/scorers" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/types" @@ -95,6 +96,7 @@ type Service struct { custodyInfoLock sync.RWMutex // Lock access to custodyInfo custodyInfoSet chan struct{} allForkDigests map[[4]byte]struct{} + crawler gossipsubcrawler.Crawler } type custodyInfo struct { @@ -241,6 +243,14 @@ func (s *Service) Start() { s.dv5Listener = listener go s.listenForNewNodes() + // Create the crawler using the local constructor, passing the service reference + crawler, err := NewGossipsubPeerCrawler(s, s.dv5Listener, 10*time.Second, 10*time.Second, 10) + if err != nil { + log.WithError(err).Fatal("Failed to create peer crawler") + s.startupErr = err + return + } + s.crawler = crawler } s.started = true @@ -311,12 +321,19 @@ func (s *Service) Start() { func (s *Service) Stop() error { defer s.cancel() s.started = false + if s.dv5Listener != nil { s.dv5Listener.Close() } + return nil } +// Crawler returns the p2p service's peer crawler. +func (s *Service) Crawler() gossipsubcrawler.Crawler { + return s.crawler +} + // Status of the p2p service. Will return an error if the service is considered unhealthy to // indicate that this node should not serve traffic until the issue has been resolved. func (s *Service) Status() error { diff --git a/beacon-chain/p2p/testing/fuzz_p2p.go b/beacon-chain/p2p/testing/fuzz_p2p.go index 4920d4921808..babe0bd99db2 100644 --- a/beacon-chain/p2p/testing/fuzz_p2p.go +++ b/beacon-chain/p2p/testing/fuzz_p2p.go @@ -4,6 +4,7 @@ import ( "context" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/encoder" + "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/gossipsubcrawler" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers" "github.com/OffchainLabs/prysm/v6/consensus-types/blocks" "github.com/OffchainLabs/prysm/v6/consensus-types/interfaces" @@ -40,6 +41,11 @@ func (*FakeP2P) AddConnectionHandler(_, _ func(ctx context.Context, id peer.ID) } +// Crawler -- fake. +func (*FakeP2P) Crawler() gossipsubcrawler.Crawler { + return &MockCrawler{} +} + // AddDisconnectionHandler -- fake. func (*FakeP2P) AddDisconnectionHandler(_ func(ctx context.Context, id peer.ID) error) { } diff --git a/beacon-chain/p2p/testing/mock_peermanager.go b/beacon-chain/p2p/testing/mock_peermanager.go index 83bb656a637f..6134a19cc51e 100644 --- a/beacon-chain/p2p/testing/mock_peermanager.go +++ b/beacon-chain/p2p/testing/mock_peermanager.go @@ -4,6 +4,7 @@ import ( "context" "errors" + "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/gossipsubcrawler" "github.com/ethereum/go-ethereum/p2p/enode" "github.com/ethereum/go-ethereum/p2p/enr" "github.com/libp2p/go-libp2p/core/host" @@ -63,3 +64,8 @@ func (*MockPeerManager) FindAndDialPeersWithSubnets(ctx context.Context, fullTop // AddPingMethod . func (*MockPeerManager) AddPingMethod(_ func(ctx context.Context, id peer.ID) error) {} + +// Crawler. +func (*MockPeerManager) Crawler() gossipsubcrawler.Crawler { + return nil +} diff --git a/beacon-chain/p2p/testing/p2p.go b/beacon-chain/p2p/testing/p2p.go index 65bf86d49df3..3bfa10844dd3 100644 --- a/beacon-chain/p2p/testing/p2p.go +++ b/beacon-chain/p2p/testing/p2p.go @@ -13,6 +13,7 @@ import ( "github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/encoder" + "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/gossipsubcrawler" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers/scorers" "github.com/OffchainLabs/prysm/v6/config/params" @@ -556,3 +557,35 @@ func (s *TestP2P) custodyGroupCountFromPeerENR(pid peer.ID) uint64 { return custodyGroupCount } + +// MockCrawler is a minimal mock implementation of PeerCrawler for testing +type MockCrawler struct{} + +// Start does nothing as this is a mock +func (m *MockCrawler) Start(gossipsubcrawler.TopicExtractor) error { + return nil +} + +// Stop does nothing as this is a mock +func (m *MockCrawler) Stop() {} + +// SetTopicExtractor does nothing as this is a mock +func (m *MockCrawler) SetTopicExtractor(extractor func(context.Context, *enode.Node) ([]string, error)) error { + return nil +} + +// RemoveTopic does nothing as this is a mock +func (m *MockCrawler) RemoveTopic(topic gossipsubcrawler.Topic) {} + +// RemovePeerID does nothing as this is a mock +func (m *MockCrawler) RemovePeerId(pid peer.ID) {} + +// PeersForTopic returns empty list as this is a mock +func (m *MockCrawler) PeersForTopic(topic gossipsubcrawler.Topic) []*enode.Node { + return []*enode.Node{} +} + +// Crawler returns a mock crawler implementation for testing. +func (*TestP2P) Crawler() gossipsubcrawler.Crawler { + return &MockCrawler{} +} diff --git a/beacon-chain/sync/BUILD.bazel b/beacon-chain/sync/BUILD.bazel index ffb9a5d5728c..bdd112de7565 100644 --- a/beacon-chain/sync/BUILD.bazel +++ b/beacon-chain/sync/BUILD.bazel @@ -101,6 +101,7 @@ go_library( "//beacon-chain/operations/synccommittee:go_default_library", "//beacon-chain/operations/voluntaryexits:go_default_library", "//beacon-chain/p2p:go_default_library", + "//beacon-chain/p2p/gossipsubcrawler:go_default_library", "//beacon-chain/p2p/encoder:go_default_library", "//beacon-chain/p2p/peers:go_default_library", "//beacon-chain/p2p/types:go_default_library", diff --git a/beacon-chain/sync/gossipsub_controller.go b/beacon-chain/sync/gossipsub_controller.go index 2e8ba9643699..75eb25e2175a 100644 --- a/beacon-chain/sync/gossipsub_controller.go +++ b/beacon-chain/sync/gossipsub_controller.go @@ -8,6 +8,8 @@ import ( "github.com/OffchainLabs/prysm/v6/config/params" "github.com/OffchainLabs/prysm/v6/consensus-types/primitives" "github.com/OffchainLabs/prysm/v6/time/slots" + "github.com/ethereum/go-ethereum/p2p/enode" + "github.com/pkg/errors" "github.com/sirupsen/logrus" ) @@ -121,6 +123,50 @@ func (g *GossipsubController) Stop() { g.wg.Wait() } +func (g *GossipsubController) ExtractTopics(ctx context.Context, node *enode.Node) ([]string, error) { + if err := ctx.Err(); err != nil { + return nil, err + } + if node == nil { + return nil, errors.New("enode is nil") + } + + g.mu.RLock() + families := make([]GossipsubTopicFamilyWithDynamicSubnets, 0, len(g.activeTopicFamilies)) + for _, f := range g.activeTopicFamilies { + if tfm, ok := f.(GossipsubTopicFamilyWithDynamicSubnets); ok { + families = append(families, tfm) + } + } + g.mu.RUnlock() + + // Collect topics from dynamic families only, de-duplicated. + topicSet := make(map[string]struct{}) + for _, df := range families { + if err := ctx.Err(); err != nil { + return nil, err + } + + topics, err := df.GetTopicsForNode(node) + if err != nil { + log.WithError(err).WithFields(logrus.Fields{ + "topicFamily": fmt.Sprintf("%T", df), + }).Debug("Failed to get topics for node from family") + continue + } + for _, t := range topics { + topicSet[t] = struct{}{} + } + } + + // Flatten set to slice with stable but unspecified order. + out := make([]string, 0, len(topicSet)) + for t := range topicSet { + out = append(out, t) + } + return out, nil +} + func isNextEpochForkBoundary(currentEpoch primitives.Epoch) (bool, params.NetworkScheduleEntry) { current := params.GetNetworkScheduleEntry(currentEpoch) next := params.GetNetworkScheduleEntry(currentEpoch + 1) diff --git a/beacon-chain/sync/gossipsub_controller_test.go b/beacon-chain/sync/gossipsub_controller_test.go index 50e1d403231e..1394a94b915e 100644 --- a/beacon-chain/sync/gossipsub_controller_test.go +++ b/beacon-chain/sync/gossipsub_controller_test.go @@ -16,8 +16,78 @@ import ( "github.com/OffchainLabs/prysm/v6/consensus-types/primitives" "github.com/OffchainLabs/prysm/v6/genesis" "github.com/OffchainLabs/prysm/v6/testing/assert" + "github.com/OffchainLabs/prysm/v6/testing/require" + "github.com/ethereum/go-ethereum/p2p/enode" ) +// fakeDynFamily is a test implementation of a dynamic-subnet topic family. +type fakeDynFamily struct { + baseGossipsubTopicFamily + topics []string + name string +} + +func (f *fakeDynFamily) Name() string { + return f.name +} + +func (f *fakeDynFamily) Validator() wrappedVal { + return nil +} + +func (f *fakeDynFamily) Handler() subHandler { + return noopHandler +} + +func (f *fakeDynFamily) Subscribe() { + +} + +func (f *fakeDynFamily) Unsubscribe() { + +} + +func (f *fakeDynFamily) GetFullTopicString(subnet uint64) string { + return fmt.Sprintf("topic-%d", subnet) +} + +func (f *fakeDynFamily) GetSubnetsToJoin(_ primitives.Slot) map[uint64]bool { + return nil +} + +func (f *fakeDynFamily) GetSubnetsForBroadcast(_ primitives.Slot) map[uint64]bool { + return nil +} + +func (f *fakeDynFamily) GetTopicsForNode(_ *enode.Node) ([]string, error) { + return append([]string{}, f.topics...), nil +} + +type fakeStaticFamily struct { + baseGossipsubTopicFamily + name string +} + +func (f *fakeStaticFamily) Name() string { + return f.name +} + +func (f *fakeStaticFamily) Validator() wrappedVal { + return nil +} + +func (f *fakeStaticFamily) Handler() subHandler { + return noopHandler +} + +func (f *fakeStaticFamily) Subscribe() { + +} + +func (f *fakeStaticFamily) Unsubscribe() { + +} + func testGossipsubControllerService(t *testing.T, current primitives.Epoch) *Service { closedChan := make(chan struct{}) close(closedChan) @@ -48,7 +118,6 @@ func TestGossipsubController_CheckForNextEpochForkSubscriptions(t *testing.T) { closedChan := make(chan struct{}) close(closedChan) params.SetupTestConfigCleanup(t) - genesis.StoreEmbeddedDuringTest(t, params.BeaconConfig().ConfigName) params.BeaconConfig().FuluForkEpoch = params.BeaconConfig().ElectraForkEpoch + 4096*2 params.BeaconConfig().InitializeForkSchedule() @@ -180,3 +249,139 @@ func TestGossipsubController_CheckForNextEpochForkSubscriptions(t *testing.T) { }) } } + +func TestGossipsubController_ExtractTopics(t *testing.T) { + params.SetupTestConfigCleanup(t) + genesis.StoreEmbeddedDuringTest(t, params.BeaconConfig().ConfigName) + + type tc struct { + name string + setup func(*GossipsubController) + ctx func() context.Context + node *enode.Node + want []string + wantErr bool + } + + dummyNode := new(enode.Node) + + tests := []tc{ + { + name: "nil node returns error", + setup: func(g *GossipsubController) {}, + ctx: func() context.Context { return context.Background() }, + node: nil, + want: nil, + wantErr: true, + }, + { + name: "no families yields empty", + setup: func(g *GossipsubController) {}, + ctx: func() context.Context { return context.Background() }, + node: dummyNode, + want: []string{}, + wantErr: false, + }, + { + name: "static family ignored", + setup: func(g *GossipsubController) { + g.mu.Lock() + g.activeTopicFamilies[topicFamilyKey{topicName: "static", forkDigest: [4]byte{1, 2, 3, 4}}] = &fakeStaticFamily{name: "StaticFam"} + g.mu.Unlock() + }, + ctx: func() context.Context { return context.Background() }, + node: dummyNode, + want: []string{}, + wantErr: false, + }, + { + name: "single dynamic family topics returned", + setup: func(g *GossipsubController) { + fam := &fakeDynFamily{topics: []string{"t1", "t2"}, name: "Dyn1"} + g.mu.Lock() + g.activeTopicFamilies[topicFamilyKey{topicName: "dyn1", forkDigest: [4]byte{0}}] = fam + g.mu.Unlock() + }, + ctx: func() context.Context { return context.Background() }, + node: dummyNode, + want: []string{"t1", "t2"}, + wantErr: false, + }, + { + name: "multiple dynamic families de-dup", + setup: func(g *GossipsubController) { + f1 := &fakeDynFamily{topics: []string{"t1", "t2"}, name: "Dyn1"} + f2 := &fakeDynFamily{topics: []string{"t2", "t3"}, name: "Dyn2"} + g.mu.Lock() + g.activeTopicFamilies[topicFamilyKey{topicName: "static", forkDigest: [4]byte{1, 2, 3, 4}}] = &fakeStaticFamily{name: "StaticFam"} + g.activeTopicFamilies[topicFamilyKey{topicName: "dyn1", forkDigest: [4]byte{0}}] = f1 + g.activeTopicFamilies[topicFamilyKey{topicName: "dyn2", forkDigest: [4]byte{0}}] = f2 + g.mu.Unlock() + }, + ctx: func() context.Context { return context.Background() }, + node: dummyNode, + want: []string{"t1", "t2", "t3"}, + wantErr: false, + }, + { + name: "mixed static and dynamic", + setup: func(g *GossipsubController) { + f1 := &fakeDynFamily{topics: []string{"a", "b"}, name: "Dyn"} + s1 := &fakeStaticFamily{name: "Static"} + g.mu.Lock() + g.activeTopicFamilies[topicFamilyKey{topicName: "dyn", forkDigest: [4]byte{9}}] = f1 + g.activeTopicFamilies[topicFamilyKey{topicName: "static", forkDigest: [4]byte{9}}] = s1 + g.mu.Unlock() + }, + ctx: func() context.Context { return context.Background() }, + node: dummyNode, + want: []string{"a", "b"}, + wantErr: false, + }, + { + name: "context cancelled short-circuits", + setup: func(g *GossipsubController) { + f1 := &fakeDynFamily{topics: []string{"x"}, name: "Dyn"} + g.mu.Lock() + g.activeTopicFamilies[topicFamilyKey{topicName: "dyn", forkDigest: [4]byte{0}}] = f1 + g.mu.Unlock() + }, + ctx: func() context.Context { c, cancel := context.WithCancel(context.Background()); cancel(); return c }, + node: dummyNode, + want: nil, + wantErr: true, + }, + } + + s := &Service{} + g := NewGossipsubController(context.Background(), s) + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + // Reset families for each subtest + g.mu.Lock() + g.activeTopicFamilies = make(map[topicFamilyKey]GossipsubTopicFamily) + g.mu.Unlock() + + tt.setup(g) + topics, err := g.ExtractTopics(tt.ctx(), tt.node) + if tt.wantErr { + require.NotNil(t, err) + return + } + require.NoError(t, err) + + got := map[string]bool{} + for _, tpc := range topics { + got[tpc] = true + } + want := map[string]bool{} + for _, tpc := range tt.want { + want[tpc] = true + } + require.Equal(t, len(want), len(got)) + for k := range want { + require.Equal(t, true, got[k], "missing topic %s", k) + } + }) + } +} diff --git a/beacon-chain/sync/service.go b/beacon-chain/sync/service.go index 1b47a6dfc15a..1e179d4110ac 100644 --- a/beacon-chain/sync/service.go +++ b/beacon-chain/sync/service.go @@ -328,6 +328,10 @@ func (s *Service) Stop() error { s.unSubscribeFromTopic(t) } + if s.cfg.p2p.Crawler() != nil { + s.cfg.p2p.Crawler().Stop() + } + // Stop the gossipsub controller. s.gossipsubController.Stop() @@ -414,6 +418,16 @@ func (s *Service) startDiscoveryAndSubscriptions() { // Start the gossipsub controller. go s.gossipsubController.Start() + + // Configure the crawler with the topic extractor if available + if crawler := s.cfg.p2p.Crawler(); crawler != nil { + // Start the crawler now that it has the extractor + if err := crawler.Start(s.gossipsubController.ExtractTopics); err != nil { + log.WithError(err).Warn("Failed to start peer crawler") + } + } else { + log.Info("No crawler available, topic extraction disabled") + } } func (s *Service) writeErrorResponseToStream(responseCode byte, reason string, stream libp2pcore.Stream) { diff --git a/beacon-chain/sync/subscriber.go b/beacon-chain/sync/subscriber.go index e6d91e715f57..24ecdf1b3f8e 100644 --- a/beacon-chain/sync/subscriber.go +++ b/beacon-chain/sync/subscriber.go @@ -14,6 +14,7 @@ import ( "github.com/OffchainLabs/prysm/v6/beacon-chain/core/helpers" "github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p" + "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/gossipsubcrawler" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers" "github.com/OffchainLabs/prysm/v6/beacon-chain/startup" "github.com/OffchainLabs/prysm/v6/cmd/beacon-chain/flags" @@ -456,6 +457,10 @@ func (s *Service) unSubscribeFromTopic(topic string) { if err := s.cfg.p2p.LeaveTopic(topic); err != nil { log.WithError(err).Error("Unable to leave topic") } + + if crawler := s.cfg.p2p.Crawler(); crawler != nil { + crawler.RemoveTopic(gossipsubcrawler.Topic(topic)) + } } // connectedPeersCount counts how many peer for a given topic are connected to the node. From 14dca4078654d6d3358daf998923f433695929c9 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Wed, 12 Nov 2025 16:36:53 +0400 Subject: [PATCH 18/39] draft --- beacon-chain/p2p/BUILD.bazel | 4 +- beacon-chain/p2p/gossipsub_peer_crawler.go | 22 +++++++++-- beacon-chain/p2p/gossipsubcrawler/BUILD.bazel | 5 +-- beacon-chain/p2p/service.go | 7 +++- beacon-chain/p2p/subnets.go | 32 ++++++++-------- beacon-chain/p2p/subnets_test.go | 37 +++++++++++++++++-- beacon-chain/p2p/testing/BUILD.bazel | 1 + beacon-chain/sync/BUILD.bazel | 2 +- 8 files changed, 80 insertions(+), 30 deletions(-) diff --git a/beacon-chain/p2p/BUILD.bazel b/beacon-chain/p2p/BUILD.bazel index b70fc857150e..0e74041a0e84 100644 --- a/beacon-chain/p2p/BUILD.bazel +++ b/beacon-chain/p2p/BUILD.bazel @@ -8,7 +8,6 @@ go_library( "config.go", "connection_gater.go", "custody.go", - "gossipsub_peer_crawler.go", "dial_relay_node.go", "discovery.go", "doc.go", @@ -16,6 +15,7 @@ go_library( "fork_watcher.go", "gossip_scoring_params.go", "gossip_topic_mappings.go", + "gossipsub_peer_crawler.go", "handshake.go", "info.go", "interfaces.go", @@ -115,8 +115,8 @@ go_library( "@com_github_prysmaticlabs_fastssz//:go_default_library", "@com_github_prysmaticlabs_go_bitfield//:go_default_library", "@com_github_sirupsen_logrus//:go_default_library", - "@org_golang_x_sync//semaphore:go_default_library", "@org_golang_google_protobuf//proto:go_default_library", + "@org_golang_x_sync//semaphore:go_default_library", ], ) diff --git a/beacon-chain/p2p/gossipsub_peer_crawler.go b/beacon-chain/p2p/gossipsub_peer_crawler.go index aa7a6ebd2390..e287bae3dbb4 100644 --- a/beacon-chain/p2p/gossipsub_peer_crawler.go +++ b/beacon-chain/p2p/gossipsub_peer_crawler.go @@ -2,6 +2,7 @@ package p2p import ( "context" + "sort" "sync" "time" @@ -266,6 +267,7 @@ func NewGossipsubPeerCrawler( } return g, nil } + func (g *GossipsubPeerCrawler) PeersForTopic(topic gossipsubcrawler.Topic) []*enode.Node { g.crawledPeers.mu.RLock() defer g.crawledPeers.mu.RUnlock() @@ -275,17 +277,31 @@ func (g *GossipsubPeerCrawler) PeersForTopic(topic gossipsubcrawler.Topic) []*en return nil } - var nodes []*enode.Node + var peerNodes []*peerNode for peerID := range peerIDs { peerNode, ok := g.crawledPeers.byPeerId[peerID] if !ok { continue } - if peerNode.status == peerStatusPinged { - nodes = append(nodes, peerNode.node) + if peerNode.status == peerStatusPinged && g.service.filterPeer(peerNode.node) { + peerNodes = append(peerNodes, peerNode) } } + scorer := g.service.Peers().Scorers() + + // Sort peerNodes in descending order of their scores. + sort.Slice(peerNodes, func(i, j int) bool { + scoreI := scorer.Score(peerNodes[i].peerID) + scoreJ := scorer.Score(peerNodes[j].peerID) + return scoreI > scoreJ + }) + + nodes := make([]*enode.Node, len(peerNodes)) + for i, pn := range peerNodes { + nodes[i] = pn.node + } + return nodes } diff --git a/beacon-chain/p2p/gossipsubcrawler/BUILD.bazel b/beacon-chain/p2p/gossipsubcrawler/BUILD.bazel index 1221c4583cf1..07cd6bb31c09 100644 --- a/beacon-chain/p2p/gossipsubcrawler/BUILD.bazel +++ b/beacon-chain/p2p/gossipsubcrawler/BUILD.bazel @@ -2,9 +2,7 @@ load("@prysm//tools/go:def.bzl", "go_library") go_library( name = "go_default_library", - srcs = [ - "interface.go", - ], + srcs = ["interface.go"], importpath = "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/gossipsubcrawler", visibility = [ "//visibility:public", @@ -14,4 +12,3 @@ go_library( "@com_github_libp2p_go_libp2p//core/peer:go_default_library", ], ) - diff --git a/beacon-chain/p2p/service.go b/beacon-chain/p2p/service.go index 35355507e9d0..1b528a112605 100644 --- a/beacon-chain/p2p/service.go +++ b/beacon-chain/p2p/service.go @@ -62,6 +62,10 @@ var ( // for the current peer limit status for the time period // defined below. pollingPeriod = 6 * time.Second + + crawlTimeout = 30 * time.Second + crawlInterval = 1 * time.Second + maxConcurrentDials = 256 ) // Service for managing peer to peer (p2p) networking. @@ -244,7 +248,8 @@ func (s *Service) Start() { s.dv5Listener = listener go s.listenForNewNodes() // Create the crawler using the local constructor, passing the service reference - crawler, err := NewGossipsubPeerCrawler(s, s.dv5Listener, 10*time.Second, 10*time.Second, 10) + crawler, err := NewGossipsubPeerCrawler(s, s.dv5Listener, crawlTimeout, crawlInterval, + maxConcurrentDials) if err != nil { log.WithError(err).Fatal("Failed to create peer crawler") s.startupErr = err diff --git a/beacon-chain/p2p/subnets.go b/beacon-chain/p2p/subnets.go index f020dcda9bb6..0475a5896d8c 100644 --- a/beacon-chain/p2p/subnets.go +++ b/beacon-chain/p2p/subnets.go @@ -12,6 +12,7 @@ import ( "github.com/OffchainLabs/prysm/v6/beacon-chain/cache" "github.com/OffchainLabs/prysm/v6/beacon-chain/core/helpers" "github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas" + "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/gossipsubcrawler" "github.com/OffchainLabs/prysm/v6/cmd/beacon-chain/flags" "github.com/OffchainLabs/prysm/v6/config/params" "github.com/OffchainLabs/prysm/v6/consensus-types/primitives" @@ -103,31 +104,32 @@ func (s *Service) FindAndDialPeersWithSubnets( defectiveSubnets := s.defectiveSubnets(fullTopicForSubnet, minimumPeersPerSubnet, subnets) for len(defectiveSubnets) > 0 { + defectiveSubnets = s.defectiveSubnets(fullTopicForSubnet, minimumPeersPerSubnet, subnets) + // Stop the search/dialing loop if the context is canceled. if err := ctx.Err(); err != nil { return err } - peersToDial, err := func() ([]*enode.Node, error) { - ctx, cancel := context.WithTimeout(ctx, batchPeriod) - defer cancel() - - peersToDial, err := s.findPeersWithSubnets(ctx, fullTopicForSubnet, minimumPeersPerSubnet, defectiveSubnets) - if err != nil && !errors.Is(err, context.DeadlineExceeded) { - return nil, errors.Wrap(err, "find peers with subnets") + var peersToDial []*enode.Node + for subnet := range defectiveSubnets { + topic := fullTopicForSubnet(subnet) + peersToDial = append(peersToDial, s.crawler.PeersForTopic(gossipsubcrawler.Topic(topic))...) + } + if len(peersToDial) > minimumPeersPerSubnet { + peersToDial = peersToDial[:minimumPeersPerSubnet] + } + if len(peersToDial) == 0 { + select { + case <-time.After(100 * time.Millisecond): + case <-ctx.Done(): + return ctx.Err() } - - return peersToDial, nil - }() - - if err != nil { - return err + continue } // Dial new peers in batches. s.dialPeers(s.ctx, maxConcurrentDials, peersToDial) - - defectiveSubnets = s.defectiveSubnets(fullTopicForSubnet, minimumPeersPerSubnet, subnets) } return nil diff --git a/beacon-chain/p2p/subnets_test.go b/beacon-chain/p2p/subnets_test.go index 69d77b9a494d..ceeaa65f8840 100644 --- a/beacon-chain/p2p/subnets_test.go +++ b/beacon-chain/p2p/subnets_test.go @@ -114,8 +114,23 @@ func TestStartDiscV5_FindAndDialPeersWithSubnet(t *testing.T) { require.NoError(t, err) require.Equal(t, true, nodeForkDigest == bootNodeForkDigest, "fork digest of the node doesn't match the boot node") - // Start the service. - service.Start() + // Start the service. + service.Start() + + // start the crawler with a topic extractor that maps ENR attestation subnets + // to full attestation topics for the current fork digest and encoding. + _ = service.Crawler().Start(func(ctx context.Context, node *enode.Node) ([]string, error) { + subs, err := attestationSubnets(node.Record()) + if err != nil { + return nil, err + } + var topics []string + for subnet := range subs { + t := fmt.Sprintf(AttestationSubnetTopicFormat, bootNodeForkDigest, subnet) + service.Encoding().ProtocolSuffix() + topics = append(topics, t) + } + return topics, nil + }) // Set the ENR `attnets`, used by Prysm to filter peers by subnet. bitV := bitfield.NewBitvector64() @@ -161,7 +176,21 @@ func TestStartDiscV5_FindAndDialPeersWithSubnet(t *testing.T) { service.genesisValidatorsRoot = params.BeaconConfig().GenesisValidatorsRoot[:] service.custodyInfo = &custodyInfo{} - service.Start() + service.Start() + // start the crawler with a topic extractor that maps ENR attestation subnets + // to full attestation topics for the current fork digest and encoding. + _ = service.Crawler().Start(func(ctx context.Context, node *enode.Node) ([]string, error) { + subs, err := attestationSubnets(node.Record()) + if err != nil { + return nil, err + } + var topics []string + for subnet := range subs { + t := fmt.Sprintf(AttestationSubnetTopicFormat, bootNodeForkDigest, subnet) + service.Encoding().ProtocolSuffix() + topics = append(topics, t) + } + return topics, nil + }) defer func() { err := service.Stop() require.NoError(t, err) @@ -174,7 +203,7 @@ func TestStartDiscV5_FindAndDialPeersWithSubnet(t *testing.T) { defectiveSubnets := service.defectiveSubnets(builder, minimumPeersPerSubnet, subnets) require.Equal(t, subnetCount, len(defectiveSubnets)) - ctxWithTimeOut, cancel := context.WithTimeout(ctx, 5*time.Second) + ctxWithTimeOut, cancel := context.WithTimeout(ctx, 15*time.Second) defer cancel() err = service.FindAndDialPeersWithSubnets(ctxWithTimeOut, builder, minimumPeersPerSubnet, subnets) diff --git a/beacon-chain/p2p/testing/BUILD.bazel b/beacon-chain/p2p/testing/BUILD.bazel index 6206705016e9..90babd27c7f2 100644 --- a/beacon-chain/p2p/testing/BUILD.bazel +++ b/beacon-chain/p2p/testing/BUILD.bazel @@ -21,6 +21,7 @@ go_library( deps = [ "//beacon-chain/core/peerdas:go_default_library", "//beacon-chain/p2p/encoder:go_default_library", + "//beacon-chain/p2p/gossipsubcrawler:go_default_library", "//beacon-chain/p2p/peers:go_default_library", "//beacon-chain/p2p/peers/scorers:go_default_library", "//config/params:go_default_library", diff --git a/beacon-chain/sync/BUILD.bazel b/beacon-chain/sync/BUILD.bazel index bdd112de7565..48889e32a1d4 100644 --- a/beacon-chain/sync/BUILD.bazel +++ b/beacon-chain/sync/BUILD.bazel @@ -101,8 +101,8 @@ go_library( "//beacon-chain/operations/synccommittee:go_default_library", "//beacon-chain/operations/voluntaryexits:go_default_library", "//beacon-chain/p2p:go_default_library", - "//beacon-chain/p2p/gossipsubcrawler:go_default_library", "//beacon-chain/p2p/encoder:go_default_library", + "//beacon-chain/p2p/gossipsubcrawler:go_default_library", "//beacon-chain/p2p/peers:go_default_library", "//beacon-chain/p2p/types:go_default_library", "//beacon-chain/slasher/types:go_default_library", From 929115639d9a778418c4c4fedf8a5e82fefe288c Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Wed, 12 Nov 2025 16:38:07 +0400 Subject: [PATCH 19/39] draft --- changelog/aarshkshah1992-gossipsub-peer-crawler.md | 3 +++ 1 file changed, 3 insertions(+) create mode 100644 changelog/aarshkshah1992-gossipsub-peer-crawler.md diff --git a/changelog/aarshkshah1992-gossipsub-peer-crawler.md b/changelog/aarshkshah1992-gossipsub-peer-crawler.md new file mode 100644 index 000000000000..6a5326b554dd --- /dev/null +++ b/changelog/aarshkshah1992-gossipsub-peer-crawler.md @@ -0,0 +1,3 @@ +### Added + +- A peer crawler for Gossipsub. \ No newline at end of file From 1397a79b4c3943d8939dde06be1513200e3bfa41 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Wed, 12 Nov 2025 17:09:33 +0400 Subject: [PATCH 20/39] changelog --- changelog/aarshkshah1992-gossipsub-peer-crawler.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/changelog/aarshkshah1992-gossipsub-peer-crawler.md b/changelog/aarshkshah1992-gossipsub-peer-crawler.md index 6a5326b554dd..1145054f46a6 100644 --- a/changelog/aarshkshah1992-gossipsub-peer-crawler.md +++ b/changelog/aarshkshah1992-gossipsub-peer-crawler.md @@ -1,3 +1,3 @@ ### Added -- A peer crawler for Gossipsub. \ No newline at end of file +- A discV5 peer crawler for Gossipsub. \ No newline at end of file From 9b07f13cd31857c30eeb3e9c9a272c7a847a902e Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Thu, 13 Nov 2025 16:58:49 +0400 Subject: [PATCH 21/39] tests for the crawler --- beacon-chain/p2p/BUILD.bazel | 3 + beacon-chain/p2p/gossipsub_peer_crawler.go | 157 +++- .../p2p/gossipsub_peer_crawler_test.go | 795 ++++++++++++++++++ .../p2p/gossipsubcrawler/interface.go | 4 + beacon-chain/p2p/service.go | 12 +- 5 files changed, 932 insertions(+), 39 deletions(-) create mode 100644 beacon-chain/p2p/gossipsub_peer_crawler_test.go diff --git a/beacon-chain/p2p/BUILD.bazel b/beacon-chain/p2p/BUILD.bazel index 0e74041a0e84..ae164db0d0e7 100644 --- a/beacon-chain/p2p/BUILD.bazel +++ b/beacon-chain/p2p/BUILD.bazel @@ -132,6 +132,7 @@ go_test( "fork_test.go", "gossip_scoring_params_test.go", "gossip_topic_mappings_test.go", + "gossipsub_peer_crawler_test.go", "message_id_test.go", "options_test.go", "parameter_test.go", @@ -157,6 +158,7 @@ go_test( "//beacon-chain/core/signing:go_default_library", "//beacon-chain/db/testing:go_default_library", "//beacon-chain/p2p/encoder:go_default_library", + "//beacon-chain/p2p/gossipsubcrawler:go_default_library", "//beacon-chain/p2p/peers:go_default_library", "//beacon-chain/p2p/peers/peerdata:go_default_library", "//beacon-chain/p2p/peers/scorers:go_default_library", @@ -204,6 +206,7 @@ go_test( "@com_github_pkg_errors//:go_default_library", "@com_github_prysmaticlabs_go_bitfield//:go_default_library", "@com_github_sirupsen_logrus//hooks/test:go_default_library", + "@com_github_stretchr_testify//require:go_default_library", "@org_golang_google_protobuf//proto:go_default_library", ], ) diff --git a/beacon-chain/p2p/gossipsub_peer_crawler.go b/beacon-chain/p2p/gossipsub_peer_crawler.go index e287bae3dbb4..92a8e645d58f 100644 --- a/beacon-chain/p2p/gossipsub_peer_crawler.go +++ b/beacon-chain/p2p/gossipsub_peer_crawler.go @@ -15,58 +15,53 @@ import ( "github.com/ethereum/go-ethereum/p2p/enode" ) -type peerStatus int - -const ( - peerStatusUnknown peerStatus = iota - peerStatusCrawled - peerStatusPinged -) - type peerNode struct { - id enode.ID - status peerStatus - node *enode.Node - peerID peer.ID - topics map[gossipsubcrawler.Topic]struct{} + id enode.ID + isPinged bool + node *enode.Node + peerID peer.ID + topics map[gossipsubcrawler.Topic]struct{} } type crawledPeers struct { - g *GossipsubPeerCrawler - mu sync.RWMutex + g *GossipsubPeerCrawler + mu sync.RWMutex byEnode map[enode.ID]*peerNode byPeerId map[peer.ID]*peerNode byTopic map[gossipsubcrawler.Topic]map[peer.ID]struct{} } -func (cp *crawledPeers) updateStatusToPinged(node *enode.Node) { +func (cp *crawledPeers) updateStatusToPinged(enodeID enode.ID) { cp.mu.Lock() defer cp.mu.Unlock() - enodeID := node.ID() existingPNode, ok := cp.byEnode[enodeID] if !ok { return } - if existingPNode.node.Seq() == node.Seq() { - existingPNode.status = peerStatusPinged - return - } + + // we only want to ping a node with a given NodeId once -> not on every sequence number change + // as ping is simply a test of a node being reachable and not fake + existingPNode.isPinged = true } -func (cp *crawledPeers) removePeerOnPingFailure(node *enode.Node) { +func (cp *crawledPeers) removePeerOnPingFailure(enodeID enode.ID) { cp.mu.Lock() defer cp.mu.Unlock() - enodeID := node.ID() existingPNode, ok := cp.byEnode[enodeID] if !ok { return } - if existingPNode.node.Seq() == node.Seq() { - cp.updateTopicsUnlocked(existingPNode, nil) - } + + // same idea as in "updateStatusToPinged" above. + // We don't want to test pings for every sequence number change for a given node as that + // can lead to an explosion in the number of pings the crawler needs to do. + // So, remove the peer when the first ping fails. If the node becomes reachable later, + // we will discover it during a re-crawl and ping it again to test for reachability. + // we're not blacklisting this peer anyways. + cp.updateTopicsUnlocked(existingPNode, nil) } func (cp *crawledPeers) updateCrawledIfNewer(node *enode.Node, topics []string) { @@ -74,11 +69,21 @@ func (cp *crawledPeers) updateCrawledIfNewer(node *enode.Node, topics []string) enodeID := node.ID() existingPNode, ok := cp.byEnode[enodeID] + + if ok && existingPNode.node == nil { + log.WithField("enodeId", enodeID).Error("enode is nil for enodeId") + cp.mu.Unlock() + return + } + + // we don't want to update enodes with a lower sequence number as they're stale records if ok && existingPNode.node.Seq() >= node.Seq() { cp.mu.Unlock() return } + if !ok { + // this is a new peer peerID, err := enodeToPeerID(node) if err != nil { log.WithError(err).WithField("node", node.ID()).Debug("Failed to convert enode to peer ID") @@ -98,10 +103,13 @@ func (cp *crawledPeers) updateCrawledIfNewer(node *enode.Node, topics []string) } cp.updateTopicsUnlocked(existingPNode, topics) - cp.mu.Unlock() - if len(topics) == 0 { + + if existingPNode.isPinged || len(topics) == 0 { + cp.mu.Unlock() return } + cp.mu.Unlock() + select { case cp.g.pingCh <- *node: case <-cp.g.ctx.Done(): @@ -123,6 +131,10 @@ func (cp *crawledPeers) removeTopic(topic gossipsubcrawler.Topic) { for peerID := range peers { if pnode, exists := cp.byPeerId[peerID]; exists { delete(pnode.topics, topic) + // remove the peer if it has no more topics left + if len(pnode.topics) == 0 { + cp.updateTopicsUnlocked(pnode, nil) + } } } @@ -216,6 +228,9 @@ type GossipsubPeerCrawler struct { topicExtractor gossipsubcrawler.TopicExtractor + peerFilter gossipsubcrawler.PeerFilterFunc + scorer PeerScoreFunc + maxConcurrentPings int pingCh chan enode.Node pingSemaphore *semaphore.Weighted @@ -224,12 +239,22 @@ type GossipsubPeerCrawler struct { once sync.Once } +// cleanupInterval controls how frequently we sweep crawled peers and prune +// those that are no longer useful. +const cleanupInterval = 5 * time.Minute + +// PeerScoreFunc provides a way to calculate a score for a given peer ID. +// Higher scores should indicate better peers. +type PeerScoreFunc func(peer.ID) float64 + func NewGossipsubPeerCrawler( service *Service, dv5 ListenerRebooter, crawlTimeout time.Duration, crawlInterval time.Duration, maxConcurrentPings int, + peerFilter gossipsubcrawler.PeerFilterFunc, + scorer PeerScoreFunc, ) (*GossipsubPeerCrawler, error) { if service == nil { return nil, errors.New("service is nil") @@ -246,6 +271,12 @@ func NewGossipsubPeerCrawler( if maxConcurrentPings <= 0 { return nil, errors.New("max concurrent pings must be greater than 0") } + if peerFilter == nil { + return nil, errors.New("peer filter is nil") + } + if scorer == nil { + return nil, errors.New("peer scorer is nil") + } ctx, cancel := context.WithCancel(context.Background()) g := &GossipsubPeerCrawler{ @@ -256,6 +287,8 @@ func NewGossipsubPeerCrawler( service: service, dv5: dv5, maxConcurrentPings: maxConcurrentPings, + peerFilter: peerFilter, + scorer: scorer, } g.pingCh = make(chan enode.Node, 4*g.maxConcurrentPings) g.pingSemaphore = semaphore.NewWeighted(int64(g.maxConcurrentPings)) @@ -283,17 +316,15 @@ func (g *GossipsubPeerCrawler) PeersForTopic(topic gossipsubcrawler.Topic) []*en if !ok { continue } - if peerNode.status == peerStatusPinged && g.service.filterPeer(peerNode.node) { + if peerNode.isPinged && g.peerFilter(peerNode.node) { peerNodes = append(peerNodes, peerNode) } } - scorer := g.service.Peers().Scorers() - // Sort peerNodes in descending order of their scores. sort.Slice(peerNodes, func(i, j int) bool { - scoreI := scorer.Score(peerNodes[i].peerID) - scoreJ := scorer.Score(peerNodes[j].peerID) + scoreI := g.scorer(peerNodes[i].peerID) + scoreJ := g.scorer(peerNodes[j].peerID) return scoreI > scoreJ }) @@ -326,6 +357,9 @@ func (g *GossipsubPeerCrawler) Start(te gossipsubcrawler.TopicExtractor) error { g.wg.Go(func() { g.pingLoop() }) + g.wg.Go(func() { + g.cleanupLoop() + }) }) return nil @@ -348,11 +382,11 @@ func (g *GossipsubPeerCrawler) pingLoop() { defer g.pingSemaphore.Release(1) if err := g.dv5.Ping(node); err != nil { - g.crawledPeers.removePeerOnPingFailure(node) + g.crawledPeers.removePeerOnPingFailure(node.ID()) return } - g.crawledPeers.updateStatusToPinged(node) + g.crawledPeers.updateStatusToPinged(node.ID()) }(&node) case <-g.ctx.Done(): @@ -398,7 +432,7 @@ func (g *GossipsubPeerCrawler) crawl() { continue } - if !g.service.filterPeer(node) { + if !g.peerFilter(node) { g.crawledPeers.removePeer(node.ID()) continue } @@ -413,6 +447,57 @@ func (g *GossipsubPeerCrawler) crawl() { } } +// cleanupLoop periodically removes peers that the filter rejects or that +// have no topics of interest. It uses the same context lifecycle as other +// background loops. +func (g *GossipsubPeerCrawler) cleanupLoop() { + ticker := time.NewTicker(cleanupInterval) + defer ticker.Stop() + + // Initial cleanup to catch any leftovers from startup state + g.cleanup() + + for { + select { + case <-ticker.C: + g.cleanup() + case <-g.ctx.Done(): + return + } + } +} + +// cleanup scans the crawled peer set and removes entries that either fail +// the current peer filter or have no topics of interest remaining. +func (g *GossipsubPeerCrawler) cleanup() { + cp := g.crawledPeers + + // Snapshot current peers to evaluate without holding the lock during + // filter and topic extraction. + cp.mu.RLock() + peers := make([]*peerNode, 0, len(cp.byPeerId)) + for _, p := range cp.byPeerId { + peers = append(peers, p) + } + cp.mu.RUnlock() + + for _, p := range peers { + p := p + + // Remove peers that no longer pass the filter + if !g.peerFilter(p.node) { + cp.removePeer(p.id) + continue + } + + // Re-extract topics; if the extractor errors or yields none, drop the peer. + topics, err := g.topicExtractor(g.ctx, p.node) + if err != nil || len(topics) == 0 { + cp.removePeer(p.id) + } + } +} + // enodeToPeerID converts an enode record to a peer ID. func enodeToPeerID(n *enode.Node) (peer.ID, error) { info, _, err := convertToAddrInfo(n) diff --git a/beacon-chain/p2p/gossipsub_peer_crawler_test.go b/beacon-chain/p2p/gossipsub_peer_crawler_test.go new file mode 100644 index 000000000000..1c0c2b858b03 --- /dev/null +++ b/beacon-chain/p2p/gossipsub_peer_crawler_test.go @@ -0,0 +1,795 @@ +package p2p + +import ( + "context" + "net" + "testing" + "time" + + "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/gossipsubcrawler" + p2ptest "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/testing" + "github.com/ethereum/go-ethereum/p2p/enode" + "github.com/libp2p/go-libp2p/core/peer" + "github.com/stretchr/testify/require" +) + +// Helpers for crawledPeers tests +func newTestCrawledPeers() *crawledPeers { + return &crawledPeers{ + byEnode: make(map[enode.ID]*peerNode), + byPeerId: make(map[peer.ID]*peerNode), + byTopic: make(map[gossipsubcrawler.Topic]map[peer.ID]struct{}), + } +} + +func addPeerWithTopics(t *testing.T, cp *crawledPeers, node *enode.Node, topics []string, pinged bool) *peerNode { + t.Helper() + pid, err := enodeToPeerID(node) + require.NoError(t, err) + p := &peerNode{ + id: node.ID(), + isPinged: pinged, + node: node, + peerID: pid, + topics: make(map[gossipsubcrawler.Topic]struct{}), + } + cp.mu.Lock() + cp.byEnode[p.id] = p + cp.byPeerId[p.peerID] = p + cp.updateTopicsUnlocked(p, topics) + cp.mu.Unlock() + return p +} + +func TestUpdateStatusToPinged(t *testing.T) { + localNode := createTestNodeRandom(t) + node1 := localNode.Node() + localNode2 := createTestNodeRandom(t) + node2 := localNode2.Node() + + cases := []struct { + name string + prep func(*crawledPeers) + target *enode.Node + expectPinged map[enode.ID]bool + }{ + { + name: "sets pinged for existing peer", + prep: func(cp *crawledPeers) { + addPeerWithTopics(t, cp, node1, []string{"a"}, false) + }, + target: node1, + expectPinged: map[enode.ID]bool{ + node1.ID(): true, + }, + }, + { + name: "idempotent when already pinged", + prep: func(cp *crawledPeers) { + addPeerWithTopics(t, cp, node1, []string{"a"}, true) + }, + target: node1, + expectPinged: map[enode.ID]bool{ + node1.ID(): true, + }, + }, + { + name: "no change when peer missing", + prep: func(cp *crawledPeers) { + addPeerWithTopics(t, cp, node1, []string{"a"}, false) + }, + target: node2, + expectPinged: map[enode.ID]bool{ + node1.ID(): false, + }, + }, + } + + for _, tc := range cases { + t.Run(tc.name, func(t *testing.T) { + cp := newTestCrawledPeers() + tc.prep(cp) + cp.updateStatusToPinged(tc.target.ID()) + cp.mu.RLock() + defer cp.mu.RUnlock() + for id, exp := range tc.expectPinged { + if p := cp.byEnode[id]; p != nil { + require.Equal(t, exp, p.isPinged) + } + } + }) + } +} + +func TestRemovePeerOnPingFailure(t *testing.T) { + localNode := createTestNodeRandom(t) + node1 := localNode.Node() + localNode2 := createTestNodeRandom(t) + node2 := localNode2.Node() + + cases := []struct { + name string + prep func(*crawledPeers) + target *enode.Node + wantLeft int // peers left byPeerId + check func(*testing.T, *crawledPeers) + }{ + { + name: "removes existing peer and prunes sole topic", + prep: func(cp *crawledPeers) { + addPeerWithTopics(t, cp, node1, []string{"t1"}, true) + }, + target: node1, + wantLeft: 0, + check: func(t *testing.T, cp *crawledPeers) { + _, ok := cp.byTopic[gossipsubcrawler.Topic("t1")] + require.False(t, ok, "topic should be pruned when empty") + }, + }, + { + name: "removes only target peer, keeps topic for others", + prep: func(cp *crawledPeers) { + addPeerWithTopics(t, cp, node1, []string{"t1"}, true) + addPeerWithTopics(t, cp, node2, []string{"t1"}, true) + }, + target: node1, + wantLeft: 1, + check: func(t *testing.T, cp *crawledPeers) { + peers := cp.byTopic[gossipsubcrawler.Topic("t1")] + require.Len(t, peers, 1) + }, + }, + { + name: "no-op when peer missing", + prep: func(cp *crawledPeers) { + addPeerWithTopics(t, cp, node1, []string{"t1"}, true) + }, + target: node2, + wantLeft: 1, + check: func(t *testing.T, cp *crawledPeers) {}, + }, + } + + for _, tc := range cases { + t.Run(tc.name, func(t *testing.T) { + cp := newTestCrawledPeers() + tc.prep(cp) + cp.removePeerOnPingFailure(tc.target.ID()) + cp.mu.RLock() + defer cp.mu.RUnlock() + require.Len(t, cp.byPeerId, tc.wantLeft) + tc.check(t, cp) + }) + } +} + +func TestRemoveTopic(t *testing.T) { + localNode := createTestNodeRandom(t) + node1 := localNode.Node() + localNode2 := createTestNodeRandom(t) + node2 := localNode2.Node() + + topic1 := gossipsubcrawler.Topic("t1") + topic2 := gossipsubcrawler.Topic("t2") + + cases := []struct { + name string + prep func(*crawledPeers) + topic gossipsubcrawler.Topic + check func(*testing.T, *crawledPeers) + }{ + { + name: "removes topic from all peers and index", + prep: func(cp *crawledPeers) { + addPeerWithTopics(t, cp, node1, []string{"t1", "t2"}, true) + addPeerWithTopics(t, cp, node2, []string{"t1"}, true) + }, + topic: topic1, + check: func(t *testing.T, cp *crawledPeers) { + _, ok := cp.byTopic[topic1] + require.False(t, ok) + for _, p := range cp.byPeerId { + _, has := p.topics[topic1] + require.False(t, has) + } + // Ensure other topics remain + _, ok = cp.byTopic[topic2] + require.True(t, ok) + }, + }, + { + name: "no-op when topic missing", + prep: func(cp *crawledPeers) { + addPeerWithTopics(t, cp, node1, []string{"t2"}, true) + }, + topic: topic1, + check: func(t *testing.T, cp *crawledPeers) { + _, ok := cp.byTopic[topic2] + require.True(t, ok) + }, + }, + } + + for _, tc := range cases { + t.Run(tc.name, func(t *testing.T) { + cp := newTestCrawledPeers() + tc.prep(cp) + cp.removeTopic(tc.topic) + tc.check(t, cp) + }) + } +} + +func TestRemovePeer(t *testing.T) { + localNode := createTestNodeRandom(t) + node1 := localNode.Node() + localNode2 := createTestNodeRandom(t) + node2 := localNode2.Node() + + cases := []struct { + name string + prep func(*crawledPeers) + target enode.ID + wantTopics int + }{ + { + name: "removes existing peer and prunes empty topic", + prep: func(cp *crawledPeers) { + addPeerWithTopics(t, cp, node1, []string{"t1"}, true) + }, + target: node1.ID(), + wantTopics: 0, + }, + { + name: "removes only targeted peer; keeps topic for other", + prep: func(cp *crawledPeers) { + addPeerWithTopics(t, cp, node1, []string{"t1"}, true) + addPeerWithTopics(t, cp, node2, []string{"t1"}, true) + }, + target: node1.ID(), + wantTopics: 1, // byTopic should still have t1 with one peer + }, + { + name: "no-op when peer missing", + prep: func(cp *crawledPeers) { + addPeerWithTopics(t, cp, node1, []string{"t1"}, true) + }, + target: node2.ID(), + wantTopics: 1, + }, + } + + for _, tc := range cases { + t.Run(tc.name, func(t *testing.T) { + cp := newTestCrawledPeers() + tc.prep(cp) + cp.removePeer(tc.target) + cp.mu.RLock() + defer cp.mu.RUnlock() + require.Len(t, cp.byTopic, tc.wantTopics) + }) + } +} + +func TestRemovePeerId(t *testing.T) { + localNode := createTestNodeRandom(t) + node1 := localNode.Node() + localNode2 := createTestNodeRandom(t) + node2 := localNode2.Node() + + pid1, err := enodeToPeerID(node1) + require.NoError(t, err) + pid2, err := enodeToPeerID(node2) + require.NoError(t, err) + + cases := []struct { + name string + prep func(*crawledPeers) + target peer.ID + wantTopics int + wantPeers int + }{ + { + name: "removes existing peer by id and prunes topic", + prep: func(cp *crawledPeers) { + addPeerWithTopics(t, cp, node1, []string{"t1"}, true) + }, + target: pid1, + wantTopics: 0, + wantPeers: 0, + }, + { + name: "removes only targeted peer id; keeps topic for other", + prep: func(cp *crawledPeers) { + addPeerWithTopics(t, cp, node1, []string{"t1"}, true) + addPeerWithTopics(t, cp, node2, []string{"t1"}, true) + }, + target: pid1, + wantTopics: 1, + wantPeers: 1, + }, + { + name: "no-op when peer id missing", + prep: func(cp *crawledPeers) { + addPeerWithTopics(t, cp, node1, []string{"t1"}, true) + }, + target: pid2, + wantTopics: 1, + wantPeers: 1, + }, + } + + for _, tc := range cases { + t.Run(tc.name, func(t *testing.T) { + cp := newTestCrawledPeers() + tc.prep(cp) + cp.removePeerId(tc.target) + cp.mu.RLock() + defer cp.mu.RUnlock() + require.Len(t, cp.byTopic, tc.wantTopics) + require.Len(t, cp.byPeerId, tc.wantPeers) + }) + } +} + +func TestUpdateCrawledIfNewer(t *testing.T) { + newCrawler := func() (*crawledPeers, *GossipsubPeerCrawler, func()) { + ctx, cancel := context.WithCancel(context.Background()) + g := &GossipsubPeerCrawler{ + ctx: ctx, + cancel: cancel, + pingCh: make(chan enode.Node, 8), + } + cp := newTestCrawledPeers() + cp.g = g + return cp, g, cancel + } + + // Helper: non-blocking receive from ping channel + recvPing := func(ch <-chan enode.Node) (enode.Node, bool) { + select { + case n := <-ch: + return n, true + default: + return enode.Node{}, false + } + } + + // Helper: local node with specific sequence + newNodeWithSeq := func(t *testing.T, seq uint64) *enode.Node { + ln := createTestNodeRandom(t) + setNodeSeq(ln, seq) + return ln.Node() + } + + // Helper: local node that will cause enodeToPeerID to fail (no TCP/UDP multiaddrs) + newNodeNoPorts := func(t *testing.T) *enode.Node { + _, privKey := createAddrAndPrivKey(t) + db, err := enode.OpenDB("") + require.NoError(t, err) + t.Cleanup(func() { db.Close() }) + ln := enode.NewLocalNode(db, privKey) + // Do not set TCP/UDP; keep only IP + ln.SetStaticIP(net.ParseIP("127.0.0.1")) + return ln.Node() + } + + nodeA1 := newNodeWithSeq(t, 1) + nodeA2 := newNodeWithSeq(t, 2) // same ID but different record required; we will rebuild to keep IDs stable + // Ensure both A nodes have the same enode.ID but differing seq + // Recreate using same LocalNode to keep ID; using helper to set seq + { + ln := createTestNodeRandom(t) + nodeA1 = ln.Node() + setNodeSeq(ln, nodeA1.Seq()) + nodeA1 = ln.Node() + setNodeSeq(ln, nodeA1.Seq()+1) + nodeA2 = ln.Node() + } + + tests := []struct { + name string + arrange func(*crawledPeers) + invokeNode *enode.Node + invokeTopics []string + assert func(*testing.T, *crawledPeers, <-chan enode.Node) + }{ + { + name: "new peer with topics adds and pings once", + arrange: func(cp *crawledPeers) {}, + invokeNode: nodeA1, + invokeTopics: []string{"a"}, + assert: func(t *testing.T, cp *crawledPeers, ch <-chan enode.Node) { + cp.mu.RLock() + require.Len(t, cp.byEnode, 1) + require.Len(t, cp.byPeerId, 1) + require.Contains(t, cp.byTopic, gossipsubcrawler.Topic("a")) + cp.mu.RUnlock() + if n, ok := recvPing(ch); !ok || n.ID() != nodeA1.ID() { + t.Fatalf("expected one ping for nodeA1") + } + if _, ok := recvPing(ch); ok { + t.Fatalf("expected exactly one ping") + } + }, + }, + { + name: "new peer with empty topics is removed and not pinged", + arrange: func(cp *crawledPeers) {}, + invokeNode: nodeA1, + invokeTopics: nil, + assert: func(t *testing.T, cp *crawledPeers, ch <-chan enode.Node) { + cp.mu.RLock() + require.Empty(t, cp.byEnode) + require.Empty(t, cp.byPeerId) + require.Empty(t, cp.byTopic) + cp.mu.RUnlock() + if _, ok := recvPing(ch); ok { + t.Fatalf("did not expect ping when topics empty") + } + }, + }, + { + name: "existing peer lower seq is ignored (no update, no ping)", + arrange: func(cp *crawledPeers) { + addPeerWithTopics(t, cp, nodeA2, []string{"x"}, false) // higher seq exists + }, + invokeNode: nodeA1, // lower seq + invokeTopics: []string{"a", "b"}, + assert: func(t *testing.T, cp *crawledPeers, ch <-chan enode.Node) { + cp.mu.RLock() + require.Contains(t, cp.byTopic, gossipsubcrawler.Topic("x")) + require.NotContains(t, cp.byTopic, gossipsubcrawler.Topic("a")) + cp.mu.RUnlock() + if _, ok := recvPing(ch); ok { + t.Fatalf("did not expect ping for lower/equal seq") + } + }, + }, + { + name: "existing peer equal seq is ignored (no update, no ping)", + arrange: func(cp *crawledPeers) { + addPeerWithTopics(t, cp, nodeA1, []string{"x"}, false) + }, + invokeNode: nodeA1, + invokeTopics: []string{"a"}, + assert: func(t *testing.T, cp *crawledPeers, ch <-chan enode.Node) { + cp.mu.RLock() + require.Contains(t, cp.byTopic, gossipsubcrawler.Topic("x")) + require.NotContains(t, cp.byTopic, gossipsubcrawler.Topic("a")) + cp.mu.RUnlock() + if _, ok := recvPing(ch); ok { + t.Fatalf("did not expect ping for equal seq") + } + }, + }, + { + name: "existing peer higher seq updates topics and pings if not pinged", + arrange: func(cp *crawledPeers) { + addPeerWithTopics(t, cp, nodeA1, []string{"x"}, false) + }, + invokeNode: nodeA2, + invokeTopics: []string{"a"}, + assert: func(t *testing.T, cp *crawledPeers, ch <-chan enode.Node) { + cp.mu.RLock() + require.NotContains(t, cp.byTopic, gossipsubcrawler.Topic("x")) + require.Contains(t, cp.byTopic, gossipsubcrawler.Topic("a")) + cp.mu.RUnlock() + if n, ok := recvPing(ch); !ok || n.ID() != nodeA2.ID() { + t.Fatalf("expected one ping for updated node") + } + }, + }, + { + name: "existing peer higher seq with already pinged does not ping", + arrange: func(cp *crawledPeers) { + p := addPeerWithTopics(t, cp, nodeA1, []string{"x"}, true) + // ensure pinged flag set + require.True(t, p.isPinged) + }, + invokeNode: nodeA2, + invokeTopics: []string{"a"}, + assert: func(t *testing.T, cp *crawledPeers, ch <-chan enode.Node) { + cp.mu.RLock() + require.Contains(t, cp.byTopic, gossipsubcrawler.Topic("a")) + cp.mu.RUnlock() + if _, ok := recvPing(ch); ok { + t.Fatalf("did not expect ping when already pinged") + } + }, + }, + { + name: "existing peer higher seq but empty topics removes peer and doesn't ping", + arrange: func(cp *crawledPeers) { + addPeerWithTopics(t, cp, nodeA1, []string{"x"}, false) + }, + invokeNode: nodeA2, + invokeTopics: nil, + assert: func(t *testing.T, cp *crawledPeers, ch <-chan enode.Node) { + cp.mu.RLock() + require.Empty(t, cp.byEnode) + require.Empty(t, cp.byPeerId) + cp.mu.RUnlock() + if _, ok := recvPing(ch); ok { + t.Fatalf("did not expect ping when topics empty on update") + } + }, + }, + { + name: "corrupted existing entry with nil node is ignored (no change, no ping)", + arrange: func(cp *crawledPeers) { + pid, _ := enodeToPeerID(nodeA1) + cp.mu.Lock() + cp.byEnode[nodeA1.ID()] = &peerNode{id: nodeA1.ID(), node: nil, peerID: pid, topics: map[gossipsubcrawler.Topic]struct{}{gossipsubcrawler.Topic("x"): {}}} + cp.byPeerId[pid] = cp.byEnode[nodeA1.ID()] + cp.byTopic[gossipsubcrawler.Topic("x")] = map[peer.ID]struct{}{pid: {}} + cp.mu.Unlock() + }, + invokeNode: nodeA2, + invokeTopics: []string{"a"}, + assert: func(t *testing.T, cp *crawledPeers, ch <-chan enode.Node) { + cp.mu.RLock() + require.Contains(t, cp.byTopic, gossipsubcrawler.Topic("x")) + cp.mu.RUnlock() + if _, ok := recvPing(ch); ok { + t.Fatalf("did not expect ping for corrupted entry") + } + }, + }, + { + name: "new peer with no ports causes enodeToPeerID error; no add, no ping", + arrange: func(cp *crawledPeers) {}, + invokeNode: newNodeNoPorts(t), + invokeTopics: []string{"a"}, + assert: func(t *testing.T, cp *crawledPeers, ch <-chan enode.Node) { + cp.mu.RLock() + require.Empty(t, cp.byEnode) + require.Empty(t, cp.byPeerId) + require.Empty(t, cp.byTopic) + cp.mu.RUnlock() + if _, ok := recvPing(ch); ok { + t.Fatalf("did not expect ping when enodeToPeerID fails") + } + }, + }, + } + + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + cp, g, cancel := newCrawler() + defer cancel() + tc.arrange(cp) + cp.updateCrawledIfNewer(tc.invokeNode, tc.invokeTopics) + tc.assert(t, cp, g.pingCh) + }) + } +} + +func TestPeersForTopic(t *testing.T) { + t.Parallel() + + newCrawler := func(filter gossipsubcrawler.PeerFilterFunc) (*GossipsubPeerCrawler, *crawledPeers) { + g := &GossipsubPeerCrawler{ + peerFilter: filter, + scorer: func(peer.ID) float64 { return 0 }, + crawledPeers: newTestCrawledPeers(), + } + return g, g.crawledPeers + } + + // Prepare nodes + ln1 := createTestNodeRandom(t) + ln2 := createTestNodeRandom(t) + ln3 := createTestNodeRandom(t) + n1, n2, n3 := ln1.Node(), ln2.Node(), ln3.Node() + topic := gossipsubcrawler.Topic("top") + + cases := []struct { + name string + filter gossipsubcrawler.PeerFilterFunc + setup func(t *testing.T, g *GossipsubPeerCrawler, cp *crawledPeers) + wantIDs []enode.ID + }{ + { + name: "no peers for topic returns empty", + filter: func(*enode.Node) bool { return true }, + setup: func(t *testing.T, g *GossipsubPeerCrawler, cp *crawledPeers) {}, + wantIDs: nil, + }, + { + name: "excludes unpinged peers", + filter: func(*enode.Node) bool { return true }, + setup: func(t *testing.T, g *GossipsubPeerCrawler, cp *crawledPeers) { + // Add one pinged and one not pinged on same topic + addPeerWithTopics(t, cp, n1, []string{string(topic)}, true) + addPeerWithTopics(t, cp, n2, []string{string(topic)}, false) + }, + wantIDs: []enode.ID{n1.ID()}, + }, + { + name: "applies peer filter to exclude", + filter: func(n *enode.Node) bool { return n.ID() != n2.ID() }, + setup: func(t *testing.T, g *GossipsubPeerCrawler, cp *crawledPeers) { + addPeerWithTopics(t, cp, n1, []string{string(topic)}, true) + addPeerWithTopics(t, cp, n2, []string{string(topic)}, true) + }, + wantIDs: []enode.ID{n1.ID()}, + }, + { + name: "ignores dangling peerID in byTopic", + filter: func(*enode.Node) bool { return true }, + setup: func(t *testing.T, g *GossipsubPeerCrawler, cp *crawledPeers) { + addPeerWithTopics(t, cp, n1, []string{string(topic)}, true) + // Add n2 then remove it from byPeerId to simulate dangling + p2 := addPeerWithTopics(t, cp, n2, []string{string(topic)}, true) + cp.mu.Lock() + delete(cp.byPeerId, p2.peerID) + cp.mu.Unlock() + }, + wantIDs: []enode.ID{n1.ID()}, + }, + { + name: "sorted by score descending", + filter: func(*enode.Node) bool { return true }, + setup: func(t *testing.T, g *GossipsubPeerCrawler, cp *crawledPeers) { + // Add three pinged peers + p1 := addPeerWithTopics(t, cp, n1, []string{string(topic)}, true) + p2 := addPeerWithTopics(t, cp, n2, []string{string(topic)}, true) + p3 := addPeerWithTopics(t, cp, n3, []string{string(topic)}, true) + // Provide a deterministic scoring function + scores := map[peer.ID]float64{ + p1.peerID: 3.0, + p2.peerID: 2.0, + p3.peerID: 1.0, + } + g.scorer = func(id peer.ID) float64 { return scores[id] } + }, + wantIDs: []enode.ID{n1.ID(), n2.ID(), n3.ID()}, + }, + } + + for _, tc := range cases { + t.Run(tc.name, func(t *testing.T) { + g, cp := newCrawler(tc.filter) + cp.g = g + tc.setup(t, g, cp) + got := g.PeersForTopic(topic) + var gotIDs []enode.ID + for _, n := range got { + gotIDs = append(gotIDs, n.ID()) + } + if tc.wantIDs == nil { + require.Empty(t, gotIDs) + return + } + require.Equal(t, tc.wantIDs, gotIDs) + }) + } +} + +func TestCrawler_AddsAndPingsPeer(t *testing.T) { + // Create a test node with valid ENR entries (IP/TCP/UDP) + localNode := createTestNodeRandom(t) + node := localNode.Node() + + // Prepare a mock iterator returning our single node + iterator := p2ptest.NewMockIterator([]*enode.Node{node}) + // Prepare a mock listener with successful Ping + mockListener := p2ptest.NewMockListener(localNode, iterator) + mockListener.PingFunc = func(*enode.Node) error { return nil } + + // Inject a permissive peer filter + filter := gossipsubcrawler.PeerFilterFunc(func(n *enode.Node) bool { return true }) + + // Create crawler with small intervals + scorer := func(peer.ID) float64 { return 0 } + g, err := NewGossipsubPeerCrawler(&Service{}, mockListener, 2*time.Second, 10*time.Millisecond, 4, filter, scorer) + require.NoError(t, err) + + // Assign a simple topic extractor + topic := "test/topic" + topicExtractor := func(ctx context.Context, n *enode.Node) ([]string, error) { + return []string{topic}, nil + } + + // Run ping loop in background and perform a single crawl + g.Start(topicExtractor) + defer g.Stop() + + // Verify that the peer has been indexed under the topic and marked as pinged + require.Eventually(t, func() bool { + g.crawledPeers.mu.RLock() + defer g.crawledPeers.mu.RUnlock() + + peersByTopic := g.crawledPeers.byTopic[gossipsubcrawler.Topic(topic)] + if len(peersByTopic) == 0 { + return false + } + // Fetch the single peerNode and check status + for pid := range peersByTopic { + pn := g.crawledPeers.byPeerId[pid] + if pn == nil { + return false + } + return pn.isPinged + } + return false + }, 2*time.Second, 10*time.Millisecond) +} + +func TestCrawler_SkipsPeer_WhenFilterRejects(t *testing.T) { + t.Parallel() + + localNode := createTestNodeRandom(t) + node := localNode.Node() + iterator := p2ptest.NewMockIterator([]*enode.Node{node}) + mockListener := p2ptest.NewMockListener(localNode, iterator) + mockListener.PingFunc = func(*enode.Node) error { return nil } + + // Reject all peers via injected filter + filter := gossipsubcrawler.PeerFilterFunc(func(n *enode.Node) bool { return false }) + + scorer := func(peer.ID) float64 { return 0 } + g, err := NewGossipsubPeerCrawler(&Service{}, mockListener, 2*time.Second, 10*time.Millisecond, 2, filter, scorer) + if err != nil { + t.Fatalf("NewGossipsubPeerCrawler error: %v", err) + } + + topic := "test/topic" + g.topicExtractor = func(ctx context.Context, n *enode.Node) ([]string, error) { return []string{topic}, nil } + + g.crawl() + + // Verify no peers are indexed, because filter rejected the node + g.crawledPeers.mu.RLock() + defer g.crawledPeers.mu.RUnlock() + if len(g.crawledPeers.byEnode) != 0 || len(g.crawledPeers.byPeerId) != 0 || len(g.crawledPeers.byTopic) != 0 { + t.Fatalf("expected no peers indexed, got byEnode=%d byPeerId=%d byTopic=%d", + len(g.crawledPeers.byEnode), len(g.crawledPeers.byPeerId), len(g.crawledPeers.byTopic)) + } +} + +func TestCrawler_RemoveTopic_RemovesTopicFromIndexes(t *testing.T) { + t.Parallel() + + localNode := createTestNodeRandom(t) + node := localNode.Node() + iterator := p2ptest.NewMockIterator([]*enode.Node{node}) + mockListener := p2ptest.NewMockListener(localNode, iterator) + mockListener.PingFunc = func(*enode.Node) error { return nil } + + filter := gossipsubcrawler.PeerFilterFunc(func(n *enode.Node) bool { return true }) + + scorer := func(peer.ID) float64 { return 0 } + g, err := NewGossipsubPeerCrawler(&Service{}, mockListener, 2*time.Second, 10*time.Millisecond, 2, filter, scorer) + if err != nil { + t.Fatalf("NewGossipsubPeerCrawler error: %v", err) + } + + topic1 := "test/topic1" + topic2 := "test/topic2" + g.topicExtractor = func(ctx context.Context, n *enode.Node) ([]string, error) { return []string{topic1, topic2}, nil } + + // Single crawl to index topics + g.crawl() + + // Remove one topic and assert it is pruned from all indexes + g.RemoveTopic(gossipsubcrawler.Topic(topic1)) + + g.crawledPeers.mu.RLock() + defer g.crawledPeers.mu.RUnlock() + + if _, ok := g.crawledPeers.byTopic[gossipsubcrawler.Topic(topic1)]; ok { + t.Fatalf("expected topic1 to be removed from byTopic") + } + + // Ensure peer still exists and retains topic2 + for _, pn := range g.crawledPeers.byEnode { + if _, has1 := pn.topics[gossipsubcrawler.Topic(topic1)]; has1 { + t.Fatalf("expected topic1 to be removed from peer topics") + } + if _, has2 := pn.topics[gossipsubcrawler.Topic(topic2)]; !has2 { + t.Fatalf("expected topic2 to remain for peer") + } + } +} diff --git a/beacon-chain/p2p/gossipsubcrawler/interface.go b/beacon-chain/p2p/gossipsubcrawler/interface.go index 5e2a5895ab68..928cf56ebd23 100644 --- a/beacon-chain/p2p/gossipsubcrawler/interface.go +++ b/beacon-chain/p2p/gossipsubcrawler/interface.go @@ -13,6 +13,10 @@ type Topic string // is subscribed to based on key/value pairs from the ENR record. type TopicExtractor func(ctx context.Context, node *enode.Node) ([]string, error) +// PeerFilterFunc defines the filtering interface used by the crawler to decide if a node +// is a valid candidate to index in the crawler. +type PeerFilterFunc func(*enode.Node) bool + type Crawler interface { Start(te TopicExtractor) error Stop() diff --git a/beacon-chain/p2p/service.go b/beacon-chain/p2p/service.go index 1b528a112605..e240b489c401 100644 --- a/beacon-chain/p2p/service.go +++ b/beacon-chain/p2p/service.go @@ -247,9 +247,15 @@ func (s *Service) Start() { s.dv5Listener = listener go s.listenForNewNodes() - // Create the crawler using the local constructor, passing the service reference - crawler, err := NewGossipsubPeerCrawler(s, s.dv5Listener, crawlTimeout, crawlInterval, - maxConcurrentDials) + crawler, err := NewGossipsubPeerCrawler( + s, + s.dv5Listener, + crawlTimeout, + crawlInterval, + maxConcurrentDials, + gossipsubcrawler.PeerFilterFunc(s.filterPeer), + s.Peers().Scorers().Score, + ) if err != nil { log.WithError(err).Fatal("Failed to create peer crawler") s.startupErr = err From 61628efd44dc67acae06b6ac2eedb09242bb67af Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Thu, 13 Nov 2025 17:04:41 +0400 Subject: [PATCH 22/39] run CI --- beacon-chain/p2p/gossipsub_peer_crawler_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/beacon-chain/p2p/gossipsub_peer_crawler_test.go b/beacon-chain/p2p/gossipsub_peer_crawler_test.go index 1c0c2b858b03..756fd6b2f05e 100644 --- a/beacon-chain/p2p/gossipsub_peer_crawler_test.go +++ b/beacon-chain/p2p/gossipsub_peer_crawler_test.go @@ -711,6 +711,7 @@ func TestCrawler_AddsAndPingsPeer(t *testing.T) { if pn == nil { return false } + return pn.isPinged } return false From 63279bcadf3f17d03504d61250aa9bd0d466380e Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Thu, 13 Nov 2025 17:06:47 +0400 Subject: [PATCH 23/39] revert new line change --- beacon-chain/p2p/gossipsub_peer_crawler_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/beacon-chain/p2p/gossipsub_peer_crawler_test.go b/beacon-chain/p2p/gossipsub_peer_crawler_test.go index 756fd6b2f05e..1c0c2b858b03 100644 --- a/beacon-chain/p2p/gossipsub_peer_crawler_test.go +++ b/beacon-chain/p2p/gossipsub_peer_crawler_test.go @@ -711,7 +711,6 @@ func TestCrawler_AddsAndPingsPeer(t *testing.T) { if pn == nil { return false } - return pn.isPinged } return false From 08f038fe808cd4c8e1af804fc8d96af786112086 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Thu, 13 Nov 2025 18:57:18 +0400 Subject: [PATCH 24/39] fix lint --- .../p2p/gossipsub_peer_crawler_test.go | 24 ++++--------------- 1 file changed, 5 insertions(+), 19 deletions(-) diff --git a/beacon-chain/p2p/gossipsub_peer_crawler_test.go b/beacon-chain/p2p/gossipsub_peer_crawler_test.go index 1c0c2b858b03..1786d2c947fe 100644 --- a/beacon-chain/p2p/gossipsub_peer_crawler_test.go +++ b/beacon-chain/p2p/gossipsub_peer_crawler_test.go @@ -355,13 +355,6 @@ func TestUpdateCrawledIfNewer(t *testing.T) { } } - // Helper: local node with specific sequence - newNodeWithSeq := func(t *testing.T, seq uint64) *enode.Node { - ln := createTestNodeRandom(t) - setNodeSeq(ln, seq) - return ln.Node() - } - // Helper: local node that will cause enodeToPeerID to fail (no TCP/UDP multiaddrs) newNodeNoPorts := func(t *testing.T) *enode.Node { _, privKey := createAddrAndPrivKey(t) @@ -374,18 +367,11 @@ func TestUpdateCrawledIfNewer(t *testing.T) { return ln.Node() } - nodeA1 := newNodeWithSeq(t, 1) - nodeA2 := newNodeWithSeq(t, 2) // same ID but different record required; we will rebuild to keep IDs stable // Ensure both A nodes have the same enode.ID but differing seq - // Recreate using same LocalNode to keep ID; using helper to set seq - { - ln := createTestNodeRandom(t) - nodeA1 = ln.Node() - setNodeSeq(ln, nodeA1.Seq()) - nodeA1 = ln.Node() - setNodeSeq(ln, nodeA1.Seq()+1) - nodeA2 = ln.Node() - } + ln := createTestNodeRandom(t) + nodeA1 := ln.Node() + setNodeSeq(ln, nodeA1.Seq()+1) + nodeA2 := ln.Node() tests := []struct { name string @@ -693,7 +679,7 @@ func TestCrawler_AddsAndPingsPeer(t *testing.T) { } // Run ping loop in background and perform a single crawl - g.Start(topicExtractor) + require.NoError(t, g.Start(topicExtractor)) defer g.Stop() // Verify that the peer has been indexed under the topic and marked as pinged From 1ff836e5494e40221193b73933323ea45bb45227 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Mon, 17 Nov 2025 19:39:18 +0400 Subject: [PATCH 25/39] peer controller first draft --- beacon-chain/p2p/BUILD.bazel | 1 + beacon-chain/p2p/broadcaster.go | 17 ++- beacon-chain/p2p/broadcaster_test.go | 11 ++ beacon-chain/p2p/discovery.go | 10 +- beacon-chain/p2p/gossipsub_peer_controller.go | 129 ++++++++++++++++++ .../p2p/gossipsubcrawler/interface.go | 13 ++ beacon-chain/p2p/interfaces.go | 3 +- beacon-chain/p2p/service.go | 10 ++ beacon-chain/p2p/subnets.go | 66 +-------- beacon-chain/p2p/subnets_test.go | 76 ++++++----- beacon-chain/p2p/testing/fuzz_p2p.go | 10 ++ beacon-chain/p2p/testing/p2p.go | 10 ++ beacon-chain/sync/gossipsub_controller.go | 22 ++- beacon-chain/sync/service.go | 32 ++++- beacon-chain/sync/subscriber.go | 30 ---- 15 files changed, 287 insertions(+), 153 deletions(-) create mode 100644 beacon-chain/p2p/gossipsub_peer_controller.go diff --git a/beacon-chain/p2p/BUILD.bazel b/beacon-chain/p2p/BUILD.bazel index ae164db0d0e7..d82368615c0e 100644 --- a/beacon-chain/p2p/BUILD.bazel +++ b/beacon-chain/p2p/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "fork_watcher.go", "gossip_scoring_params.go", "gossip_topic_mappings.go", + "gossipsub_peer_controller.go", "gossipsub_peer_crawler.go", "handshake.go", "info.go", diff --git a/beacon-chain/p2p/broadcaster.go b/beacon-chain/p2p/broadcaster.go index ec09343ce788..d5634f5eb640 100644 --- a/beacon-chain/p2p/broadcaster.go +++ b/beacon-chain/p2p/broadcaster.go @@ -131,8 +131,8 @@ func (s *Service) internalBroadcastAttestation(ctx context.Context, subnet uint6 s.subnetLocker(subnet).Lock() defer s.subnetLocker(subnet).Unlock() - builder := func(idx uint64) string { return attestationToTopic(idx, forkDigest) + s.Encoding().ProtocolSuffix() } - if err := s.FindAndDialPeersWithSubnets(ctx, builder, minimumPeersPerSubnetForBroadcast, map[uint64]bool{subnet: true}); err != nil { + topicStr := attestationToTopic(subnet, forkDigest) + s.Encoding().ProtocolSuffix() + if err := s.gossipsubDialer.DialPeersForTopicBlocking(topicStr, minimumPeersPerSubnetForBroadcast); err != nil { return errors.Wrap(err, "find peers with subnets") } @@ -188,8 +188,8 @@ func (s *Service) broadcastSyncCommittee(ctx context.Context, subnet uint64, sMs if err := func() error { s.subnetLocker(wrappedSubIdx).Lock() defer s.subnetLocker(wrappedSubIdx).Unlock() - builder := func(idx uint64) string { return syncCommitteeToTopic(idx, forkDigest) + s.Encoding().ProtocolSuffix() } - if err := s.FindAndDialPeersWithSubnets(ctx, builder, minimumPeersPerSubnetForBroadcast, map[uint64]bool{subnet: true}); err != nil { + topicStr := syncCommitteeToTopic(subnet, forkDigest) + s.Encoding().ProtocolSuffix() + if err := s.gossipsubDialer.DialPeersForTopicBlocking(topicStr, minimumPeersPerSubnetForBroadcast); err != nil { return errors.Wrap(err, "find peers with subnets") } @@ -254,8 +254,8 @@ func (s *Service) internalBroadcastBlob(ctx context.Context, subnet uint64, blob s.subnetLocker(wrappedSubIdx).Lock() defer s.subnetLocker(wrappedSubIdx).Unlock() - builder := func(idx uint64) string { return blobSubnetToTopic(idx, forkDigest) + s.Encoding().ProtocolSuffix() } - if err := s.FindAndDialPeersWithSubnets(ctx, builder, minimumPeersPerSubnetForBroadcast, map[uint64]bool{subnet: true}); err != nil { + topicStr := blobSubnetToTopic(subnet, forkDigest) + s.Encoding().ProtocolSuffix() + if err := s.gossipsubDialer.DialPeersForTopicBlocking(topicStr, minimumPeersPerSubnetForBroadcast); err != nil { return errors.Wrap(err, "find peers with subnets") } @@ -501,11 +501,10 @@ func (s *Service) findPeersIfNeeded( s.subnetLocker(wrappedSubIdx).Lock() defer s.subnetLocker(wrappedSubIdx).Unlock() - // No peers found, attempt to find peers with this subnet. - if err := s.FindAndDialPeersWithSubnets(ctx, fullTopicForSubnet, minimumPeersPerSubnetForBroadcast, map[uint64]bool{subnet: true}); err != nil { + topicStr := fullTopicForSubnet(subnet) + if err := s.gossipsubDialer.DialPeersForTopicBlocking(topicStr, minimumPeersPerSubnetForBroadcast); err != nil { return errors.Wrap(err, "find peers with subnet") } - return nil } diff --git a/beacon-chain/p2p/broadcaster_test.go b/beacon-chain/p2p/broadcaster_test.go index 956500831894..0a8406ea8f67 100644 --- a/beacon-chain/p2p/broadcaster_test.go +++ b/beacon-chain/p2p/broadcaster_test.go @@ -30,6 +30,7 @@ import ( "github.com/OffchainLabs/prysm/v6/testing/require" "github.com/OffchainLabs/prysm/v6/testing/util" "github.com/OffchainLabs/prysm/v6/time/slots" + "github.com/ethereum/go-ethereum/p2p/enode" pubsub "github.com/libp2p/go-libp2p-pubsub" "github.com/libp2p/go-libp2p/core/host" "google.golang.org/protobuf/proto" @@ -737,6 +738,16 @@ func TestService_BroadcastDataColumn(t *testing.T) { subnet := peerdas.ComputeSubnetForDataColumnSidecar(columnIndex) topic := fmt.Sprintf(topicFormat, digest, subnet) + service.Encoding().ProtocolSuffix() + crawler, err := NewGossipsubPeerCrawler(service, listener, 1*time.Second, 1*time.Second, 10, + func(n *enode.Node) bool { return true }, + service.Peers().Scorers().Score) + require.NoError(t, err) + err = crawler.Start(func(ctx context.Context, node *enode.Node) ([]string, error) { + return []string{topic}, nil + }) + require.NoError(t, err) + service.gossipsubDialer = NewGossipsubPeerDialer(service, crawler) + _, verifiedRoSidecars := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{{Index: columnIndex}}) verifiedRoSidecar := verifiedRoSidecars[0] diff --git a/beacon-chain/p2p/discovery.go b/beacon-chain/p2p/discovery.go index 6d4b1b2fb3cf..2445807a5b42 100644 --- a/beacon-chain/p2p/discovery.go +++ b/beacon-chain/p2p/discovery.go @@ -369,11 +369,11 @@ func (s *Service) listenForNewNodes() { } } -// FindAndDialPeersWithSubnets ensures that our node is connected to enough peers. -// If, the threshold is met, then this function immediately returns. +// findAndDialPeers ensures that our node is connected to enough peers. +// If the threshold is met, then this function immediately returns. // Otherwise, it searches for new peers and dials them. -// If `ctx“ is canceled while searching for peers, search is stopped, but new found peers are still dialed. -// In this case, the function returns an error. +// If `ctx` is canceled while searching for peers, search is stopped, but newly +// found peers are still dialed. In this case, the function returns an error. func (s *Service) findAndDialPeers(ctx context.Context) error { // Restrict dials if limit is applied. maxConcurrentDials := math.MaxInt @@ -404,7 +404,7 @@ func (s *Service) findAndDialPeers(ctx context.Context) error { return err } - dialedPeerCount := s.dialPeers(s.ctx, maxConcurrentDials, peersToDial) + dialedPeerCount := s.DialPeers(s.ctx, maxConcurrentDials, peersToDial) if dialedPeerCount > missingPeerCount { missingPeerCount = 0 diff --git a/beacon-chain/p2p/gossipsub_peer_controller.go b/beacon-chain/p2p/gossipsub_peer_controller.go new file mode 100644 index 000000000000..e394b4f78964 --- /dev/null +++ b/beacon-chain/p2p/gossipsub_peer_controller.go @@ -0,0 +1,129 @@ +package p2p + +import ( + "context" + "math" + "sync" + "time" + + "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/gossipsubcrawler" + "github.com/OffchainLabs/prysm/v6/cmd/beacon-chain/flags" + "github.com/ethereum/go-ethereum/p2p/enode" + "github.com/pkg/errors" +) + +var ( + peerPerTopic = 20 +) + +type GossipsubPeerDialer struct { + ctx context.Context + cancel context.CancelFunc + + service *Service + + crawler gossipsubcrawler.Crawler + topicsProvider gossipsubcrawler.SubnetTopicsProvider + + wg sync.WaitGroup + once sync.Once +} + +func NewGossipsubPeerDialer(service *Service, crawler gossipsubcrawler.Crawler) *GossipsubPeerDialer { + ctx, cancel := context.WithCancel(context.Background()) + return &GossipsubPeerDialer{ + service: service, + crawler: crawler, + ctx: ctx, + cancel: cancel, + } +} + +func (g *GossipsubPeerDialer) Stop() { + g.cancel() + g.wg.Wait() +} + +func (g *GossipsubPeerDialer) Start(provider gossipsubcrawler.SubnetTopicsProvider) error { + if provider == nil { + return errors.New("topics provider is nil") + } + + g.once.Do(func() { + g.topicsProvider = provider + g.wg.Go(func() { + g.dialLoop() + }) + }) + + return nil +} + +func (g *GossipsubPeerDialer) dialLoop() { + ticker := time.NewTicker(time.Second) + defer ticker.Stop() + + for { + select { + case <-ticker.C: + topics := g.topicsProvider() + var peersToDial []*enode.Node + + for _, topic := range topics { + peers := g.service.PubSub().ListPeers(topic) + peerCount := len(peers) + if peerCount >= peerPerTopic { + continue + } + missing := peerPerTopic - peerCount + // this is fine as "PeersForTopic" does not return peers we are already connected to + newPeers := g.crawler.PeersForTopic(gossipsubcrawler.Topic(topic)) + if len(newPeers) > missing { + newPeers = newPeers[:missing] + } + peersToDial = append(peersToDial, newPeers...) + } + + if len(peersToDial) > 0 { + // Dial new peers in batches. + maxConcurrentDials := math.MaxInt + if flags.MaxDialIsActive() { + maxConcurrentDials = flags.Get().MaxConcurrentDials + } + g.service.DialPeers(g.ctx, maxConcurrentDials, peersToDial) + } + + case <-g.ctx.Done(): + return + } + } +} + +func (g *GossipsubPeerDialer) DialPeersForTopicBlocking(topic string, nPeers int) error { + for { + peers := g.service.PubSub().ListPeers(topic) + if len(peers) >= nPeers { + return nil + } + + missing := nPeers - len(peers) + // this is fine as "PeersForTopic" does not return peers we are already connected to + newPeers := g.crawler.PeersForTopic(gossipsubcrawler.Topic(topic)) + if len(newPeers) > 0 { + if len(newPeers) > missing { + newPeers = newPeers[:missing] + } + maxConcurrentDials := math.MaxInt + if flags.MaxDialIsActive() { + maxConcurrentDials = flags.Get().MaxConcurrentDials + } + g.service.DialPeers(g.ctx, maxConcurrentDials, newPeers) + } + + select { + case <-time.After(100 * time.Millisecond): + case <-g.ctx.Done(): + return g.ctx.Err() + } + } +} diff --git a/beacon-chain/p2p/gossipsubcrawler/interface.go b/beacon-chain/p2p/gossipsubcrawler/interface.go index 928cf56ebd23..d5da2b3f8f73 100644 --- a/beacon-chain/p2p/gossipsubcrawler/interface.go +++ b/beacon-chain/p2p/gossipsubcrawler/interface.go @@ -24,3 +24,16 @@ type Crawler interface { RemoveTopic(topic Topic) PeersForTopic(topic Topic) []*enode.Node } + +// SubnetTopicsProvider returns the set of gossipsub topics the node +// should currently maintain peer connections for (e.g. attestation, +// sync committee subnets). +type SubnetTopicsProvider func() []string + +// GossipsubDialer controls dialing peers for gossipsub topics based +// on a provided SubnetTopicsProvider and the p2p crawler. +type GossipsubDialer interface { + Start(provider SubnetTopicsProvider) error + Stop() + DialPeersForTopicBlocking(topic string, nPeers int) error +} diff --git a/beacon-chain/p2p/interfaces.go b/beacon-chain/p2p/interfaces.go index 7cdbf4db0d79..266f46d1c4d4 100644 --- a/beacon-chain/p2p/interfaces.go +++ b/beacon-chain/p2p/interfaces.go @@ -98,10 +98,11 @@ type ( PeerID() peer.ID Host() host.Host ENR() *enr.Record + GossipsubDialer() gossipsubcrawler.GossipsubDialer NodeID() enode.ID DiscoveryAddresses() ([]multiaddr.Multiaddr, error) RefreshPersistentSubnets() - FindAndDialPeersWithSubnets(ctx context.Context, fullTopicForSubnet func(uint64) string, minimumPeersPerSubnet int, subnets map[uint64]bool) error + DialPeers(ctx context.Context, maxConcurrentDials int, nodes []*enode.Node) uint AddPingMethod(reqFunc func(ctx context.Context, id peer.ID) error) Crawler() gossipsubcrawler.Crawler } diff --git a/beacon-chain/p2p/service.go b/beacon-chain/p2p/service.go index e240b489c401..13f206c96e8d 100644 --- a/beacon-chain/p2p/service.go +++ b/beacon-chain/p2p/service.go @@ -101,6 +101,7 @@ type Service struct { custodyInfoSet chan struct{} allForkDigests map[[4]byte]struct{} crawler gossipsubcrawler.Crawler + gossipsubDialer gossipsubcrawler.GossipsubDialer } type custodyInfo struct { @@ -262,6 +263,9 @@ func (s *Service) Start() { return } s.crawler = crawler + // Initialise the gossipsub dialer which will be started + // once the sync service is ready to provide subnet topics. + s.gossipsubDialer = NewGossipsubPeerDialer(s, s.crawler) } s.started = true @@ -345,6 +349,12 @@ func (s *Service) Crawler() gossipsubcrawler.Crawler { return s.crawler } +// GossipsubDialer returns the dialer responsible for maintaining +// peer counts per gossipsub topic, if discovery is enabled. +func (s *Service) GossipsubDialer() gossipsubcrawler.GossipsubDialer { + return s.gossipsubDialer +} + // Status of the p2p service. Will return an error if the service is considered unhealthy to // indicate that this node should not serve traffic until the issue has been resolved. func (s *Service) Status() error { diff --git a/beacon-chain/p2p/subnets.go b/beacon-chain/p2p/subnets.go index 0475a5896d8c..58361e599576 100644 --- a/beacon-chain/p2p/subnets.go +++ b/beacon-chain/p2p/subnets.go @@ -3,7 +3,6 @@ package p2p import ( "context" "fmt" - "math" "strings" "sync" "time" @@ -12,14 +11,12 @@ import ( "github.com/OffchainLabs/prysm/v6/beacon-chain/cache" "github.com/OffchainLabs/prysm/v6/beacon-chain/core/helpers" "github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas" - "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/gossipsubcrawler" "github.com/OffchainLabs/prysm/v6/cmd/beacon-chain/flags" "github.com/OffchainLabs/prysm/v6/config/params" "github.com/OffchainLabs/prysm/v6/consensus-types/primitives" "github.com/OffchainLabs/prysm/v6/consensus-types/wrapper" "github.com/OffchainLabs/prysm/v6/crypto/hash" "github.com/OffchainLabs/prysm/v6/encoding/bytesutil" - "github.com/OffchainLabs/prysm/v6/monitoring/tracing/trace" pb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1" "github.com/ethereum/go-ethereum/p2p/enode" "github.com/ethereum/go-ethereum/p2p/enr" @@ -76,65 +73,6 @@ func (s *Service) nodeFilter(topic string, indices map[uint64]int) (func(node *e } } -// FindAndDialPeersWithSubnets ensures that our node is connected to at least `minimumPeersPerSubnet` -// peers for each subnet listed in `subnets`. -// If, for all subnets, the threshold is met, then this function immediately returns. -// Otherwise, it searches for new peers for defective subnets, and dials them. -// If `ctx“ is canceled while searching for peers, search is stopped, but new found peers are still dialed. -// In this case, the function returns an error. -func (s *Service) FindAndDialPeersWithSubnets( - ctx context.Context, - fullTopicForSubnet func(uint64) string, - minimumPeersPerSubnet int, - subnets map[uint64]bool, -) error { - ctx, span := trace.StartSpan(ctx, "p2p.FindAndDialPeersWithSubnet") - defer span.End() - - // Return early if the discovery listener isn't set. - if s.dv5Listener == nil { - return nil - } - - // Restrict dials if limit is applied. - maxConcurrentDials := math.MaxInt - if flags.MaxDialIsActive() { - maxConcurrentDials = flags.Get().MaxConcurrentDials - } - - defectiveSubnets := s.defectiveSubnets(fullTopicForSubnet, minimumPeersPerSubnet, subnets) - for len(defectiveSubnets) > 0 { - defectiveSubnets = s.defectiveSubnets(fullTopicForSubnet, minimumPeersPerSubnet, subnets) - - // Stop the search/dialing loop if the context is canceled. - if err := ctx.Err(); err != nil { - return err - } - - var peersToDial []*enode.Node - for subnet := range defectiveSubnets { - topic := fullTopicForSubnet(subnet) - peersToDial = append(peersToDial, s.crawler.PeersForTopic(gossipsubcrawler.Topic(topic))...) - } - if len(peersToDial) > minimumPeersPerSubnet { - peersToDial = peersToDial[:minimumPeersPerSubnet] - } - if len(peersToDial) == 0 { - select { - case <-time.After(100 * time.Millisecond): - case <-ctx.Done(): - return ctx.Err() - } - continue - } - - // Dial new peers in batches. - s.dialPeers(s.ctx, maxConcurrentDials, peersToDial) - } - - return nil -} - // updateDefectiveSubnets updates the defective subnets map when a node with matching subnets is found. // It decrements the defective count for each subnet the node satisfies and removes subnets // that are fully satisfied (count reaches 0). @@ -280,9 +218,9 @@ func (s *Service) defectiveSubnets( return missingCountPerSubnet } -// dialPeers dials multiple peers concurrently up to `maxConcurrentDials` at a time. +// DialPeers dials multiple peers concurrently up to `maxConcurrentDials` at a time. // In case of a dial failure, it logs the error but continues dialing other peers. -func (s *Service) dialPeers(ctx context.Context, maxConcurrentDials int, nodes []*enode.Node) uint { +func (s *Service) DialPeers(ctx context.Context, maxConcurrentDials int, nodes []*enode.Node) uint { var mut sync.Mutex counter := uint(0) diff --git a/beacon-chain/p2p/subnets_test.go b/beacon-chain/p2p/subnets_test.go index ceeaa65f8840..da95dd362943 100644 --- a/beacon-chain/p2p/subnets_test.go +++ b/beacon-chain/p2p/subnets_test.go @@ -114,23 +114,23 @@ func TestStartDiscV5_FindAndDialPeersWithSubnet(t *testing.T) { require.NoError(t, err) require.Equal(t, true, nodeForkDigest == bootNodeForkDigest, "fork digest of the node doesn't match the boot node") - // Start the service. - service.Start() - - // start the crawler with a topic extractor that maps ENR attestation subnets - // to full attestation topics for the current fork digest and encoding. - _ = service.Crawler().Start(func(ctx context.Context, node *enode.Node) ([]string, error) { - subs, err := attestationSubnets(node.Record()) - if err != nil { - return nil, err - } - var topics []string - for subnet := range subs { - t := fmt.Sprintf(AttestationSubnetTopicFormat, bootNodeForkDigest, subnet) + service.Encoding().ProtocolSuffix() - topics = append(topics, t) - } - return topics, nil - }) + // Start the service. + service.Start() + + // start the crawler with a topic extractor that maps ENR attestation subnets + // to full attestation topics for the current fork digest and encoding. + _ = service.Crawler().Start(func(ctx context.Context, node *enode.Node) ([]string, error) { + subs, err := attestationSubnets(node.Record()) + if err != nil { + return nil, err + } + var topics []string + for subnet := range subs { + t := fmt.Sprintf(AttestationSubnetTopicFormat, bootNodeForkDigest, subnet) + service.Encoding().ProtocolSuffix() + topics = append(topics, t) + } + return topics, nil + }) // Set the ENR `attnets`, used by Prysm to filter peers by subnet. bitV := bitfield.NewBitvector64() @@ -176,21 +176,21 @@ func TestStartDiscV5_FindAndDialPeersWithSubnet(t *testing.T) { service.genesisValidatorsRoot = params.BeaconConfig().GenesisValidatorsRoot[:] service.custodyInfo = &custodyInfo{} - service.Start() - // start the crawler with a topic extractor that maps ENR attestation subnets - // to full attestation topics for the current fork digest and encoding. - _ = service.Crawler().Start(func(ctx context.Context, node *enode.Node) ([]string, error) { - subs, err := attestationSubnets(node.Record()) - if err != nil { - return nil, err - } - var topics []string - for subnet := range subs { - t := fmt.Sprintf(AttestationSubnetTopicFormat, bootNodeForkDigest, subnet) + service.Encoding().ProtocolSuffix() - topics = append(topics, t) - } - return topics, nil - }) + service.Start() + // start the crawler with a topic extractor that maps ENR attestation subnets + // to full attestation topics for the current fork digest and encoding. + _ = service.Crawler().Start(func(ctx context.Context, node *enode.Node) ([]string, error) { + subs, err := attestationSubnets(node.Record()) + if err != nil { + return nil, err + } + var topics []string + for subnet := range subs { + t := fmt.Sprintf(AttestationSubnetTopicFormat, bootNodeForkDigest, subnet) + service.Encoding().ProtocolSuffix() + topics = append(topics, t) + } + return topics, nil + }) defer func() { err := service.Stop() require.NoError(t, err) @@ -203,11 +203,15 @@ func TestStartDiscV5_FindAndDialPeersWithSubnet(t *testing.T) { defectiveSubnets := service.defectiveSubnets(builder, minimumPeersPerSubnet, subnets) require.Equal(t, subnetCount, len(defectiveSubnets)) - ctxWithTimeOut, cancel := context.WithTimeout(ctx, 15*time.Second) - defer cancel() + var topicsToDial []string + for s := range subnets { + topicsToDial = append(topicsToDial, builder(s)) + } - err = service.FindAndDialPeersWithSubnets(ctxWithTimeOut, builder, minimumPeersPerSubnet, subnets) - require.NoError(t, err) + for _, topic := range topicsToDial { + err = service.GossipsubDialer().DialPeersForTopicBlocking(topic, minimumPeersPerSubnet) + require.NoError(t, err) + } defectiveSubnets = service.defectiveSubnets(builder, minimumPeersPerSubnet, subnets) require.Equal(t, 0, len(defectiveSubnets)) diff --git a/beacon-chain/p2p/testing/fuzz_p2p.go b/beacon-chain/p2p/testing/fuzz_p2p.go index babe0bd99db2..7e66ba6896aa 100644 --- a/beacon-chain/p2p/testing/fuzz_p2p.go +++ b/beacon-chain/p2p/testing/fuzz_p2p.go @@ -46,6 +46,11 @@ func (*FakeP2P) Crawler() gossipsubcrawler.Crawler { return &MockCrawler{} } +// GossipsubDialer -- fake. +func (*FakeP2P) GossipsubDialer() gossipsubcrawler.GossipsubDialer { + return nil +} + // AddDisconnectionHandler -- fake. func (*FakeP2P) AddDisconnectionHandler(_ func(ctx context.Context, id peer.ID) error) { } @@ -98,6 +103,11 @@ func (*FakeP2P) Peers() *peers.Status { return nil } +// DialPeers -- fake. +func (*FakeP2P) DialPeers(ctx context.Context, maxConcurrentDials int, nodes []*enode.Node) uint { + return 0 +} + // PublishToTopic -- fake. func (*FakeP2P) PublishToTopic(_ context.Context, _ string, _ []byte, _ ...pubsub.PubOpt) error { return nil diff --git a/beacon-chain/p2p/testing/p2p.go b/beacon-chain/p2p/testing/p2p.go index 3bfa10844dd3..d9de566055b1 100644 --- a/beacon-chain/p2p/testing/p2p.go +++ b/beacon-chain/p2p/testing/p2p.go @@ -66,6 +66,7 @@ type TestP2P struct { earliestAvailableSlot primitives.Slot custodyGroupCount uint64 enr *enr.Record + dialer gossipsubcrawler.GossipsubDialer } // NewTestP2P initializes a new p2p test service. @@ -418,6 +419,10 @@ func (p *TestP2P) Peers() *peers.Status { return p.peers } +func (p *TestP2P) DialPeers(ctx context.Context, maxConcurrentDials int, nodes []*enode.Node) uint { + return 0 +} + // FindAndDialPeersWithSubnets mocks the p2p func. func (*TestP2P) FindAndDialPeersWithSubnets(ctx context.Context, fullTopicForSubnet func(uint64) string, minimumPeersPerSubnet int, subnets map[uint64]bool) error { return nil @@ -589,3 +594,8 @@ func (m *MockCrawler) PeersForTopic(topic gossipsubcrawler.Topic) []*enode.Node func (*TestP2P) Crawler() gossipsubcrawler.Crawler { return &MockCrawler{} } + +// GossipsubDialer returns nil for tests that do not exercise dialer behaviour. +func (p *TestP2P) GossipsubDialer() gossipsubcrawler.GossipsubDialer { + return p.dialer +} diff --git a/beacon-chain/sync/gossipsub_controller.go b/beacon-chain/sync/gossipsub_controller.go index 75eb25e2175a..f8a94757cd51 100644 --- a/beacon-chain/sync/gossipsub_controller.go +++ b/beacon-chain/sync/gossipsub_controller.go @@ -27,8 +27,7 @@ type GossipsubController struct { cancel context.CancelFunc syncService *Service - - wg sync.WaitGroup + wg sync.WaitGroup mu sync.RWMutex activeTopicFamilies map[topicFamilyKey]GossipsubTopicFamily @@ -123,6 +122,25 @@ func (g *GossipsubController) Stop() { g.wg.Wait() } +func (g *GossipsubController) GetCurrentSubnetTopics(slot primitives.Slot) []string { + g.mu.RLock() + var topics []string + for _, f := range g.activeTopicFamilies { + if tfm, ok := f.(GossipsubTopicFamilyWithDynamicSubnets); ok { + bsubnets := tfm.GetSubnetsForBroadcast(slot) + for subnet := range bsubnets { + topics = append(topics, tfm.GetFullTopicString(subnet)) + } + jsubnets := tfm.GetSubnetsToJoin(slot) + for subnet := range jsubnets { + topics = append(topics, tfm.GetFullTopicString(subnet)) + } + } + } + g.mu.RUnlock() + return topics +} + func (g *GossipsubController) ExtractTopics(ctx context.Context, node *enode.Node) ([]string, error) { if err := ctx.Err(); err != nil { return nil, err diff --git a/beacon-chain/sync/service.go b/beacon-chain/sync/service.go index 1e179d4110ac..db11d58d5cee 100644 --- a/beacon-chain/sync/service.go +++ b/beacon-chain/sync/service.go @@ -328,6 +328,10 @@ func (s *Service) Stop() error { s.unSubscribeFromTopic(t) } + // Stop gossipsub dialer and crawler if present. + if dialer := s.cfg.p2p.GossipsubDialer(); dialer != nil { + dialer.Stop() + } if s.cfg.p2p.Crawler() != nil { s.cfg.p2p.Crawler().Stop() } @@ -419,14 +423,30 @@ func (s *Service) startDiscoveryAndSubscriptions() { // Start the gossipsub controller. go s.gossipsubController.Start() - // Configure the crawler with the topic extractor if available - if crawler := s.cfg.p2p.Crawler(); crawler != nil { - // Start the crawler now that it has the extractor - if err := crawler.Start(s.gossipsubController.ExtractTopics); err != nil { - log.WithError(err).Warn("Failed to start peer crawler") + // Configure the crawler and dialer with the topic extractor / subnet topics + // provider if available. + crawler := s.cfg.p2p.Crawler() + if crawler == nil { + log.Info("No crawler available, topic extraction disabled") + return + } + + // Start the crawler now that it has the extractor. + if err := crawler.Start(s.gossipsubController.ExtractTopics); err != nil { + log.WithError(err).Warn("Failed to start peer crawler") + return + } + + // Start the gossipsub dialer if available. + if dialer := s.cfg.p2p.GossipsubDialer(); dialer != nil { + provider := func() []string { + return s.gossipsubController.GetCurrentSubnetTopics(s.cfg.clock.CurrentSlot()) + } + if err := dialer.Start(provider); err != nil { + log.WithError(err).Warn("Failed to start gossipsub peer dialer") } } else { - log.Info("No crawler available, topic extraction disabled") + log.Info("No gossipsub peer dialer available") } } diff --git a/beacon-chain/sync/subscriber.go b/beacon-chain/sync/subscriber.go index 24ecdf1b3f8e..67e341f2c998 100644 --- a/beacon-chain/sync/subscriber.go +++ b/beacon-chain/sync/subscriber.go @@ -339,7 +339,6 @@ func (s *Service) subscribeToSubnets(tf GossipsubTopicFamilyWithDynamicSubnets, ctx, cancel := context.WithCancel(s.ctx) defer cancel() - go s.ensurePeers(ctx, tracker) go s.logMinimumPeersPerSubnet(ctx, tf) s.trySubscribeSubnets(tracker) @@ -378,35 +377,6 @@ func (s *Service) trySubscribeSubnets(t *subnetTracker) { } } -func (s *Service) ensurePeers(ctx context.Context, tracker *subnetTracker) { - // Try once immediately so we don't have to wait until the next slot. - s.tryEnsurePeers(ctx, tracker) - - oncePerSlot := slots.NewSlotTicker(s.cfg.clock.GenesisTime(), params.BeaconConfig().SecondsPerSlot) - defer oncePerSlot.Done() - for { - select { - case <-oncePerSlot.C(): - s.tryEnsurePeers(ctx, tracker) - case <-ctx.Done(): - return - } - } -} - -func (s *Service) tryEnsurePeers(ctx context.Context, tracker *subnetTracker) { - timeout := (time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Second) - 100*time.Millisecond - minPeers := flags.Get().MinimumPeersPerSubnet - neededSubnets := computeAllNeededSubnets(s.cfg.clock.CurrentSlot(), tracker.family) - ctx, cancel := context.WithTimeout(ctx, timeout) - defer cancel() - builder := func(idx uint64) string { return tracker.family.GetFullTopicString(idx) } - err := s.cfg.p2p.FindAndDialPeersWithSubnets(ctx, builder, minPeers, neededSubnets) - if err != nil && !errors.Is(err, context.DeadlineExceeded) { - log.WithFields(familyLogFields(tracker.family)).WithError(err).Debug("Could not find peers with subnets") - } -} - func (s *Service) logMinimumPeersPerSubnet(ctx context.Context, tf GossipsubTopicFamilyWithDynamicSubnets) { logFields := familyLogFields(tf) minimumPeersPerSubnet := flags.Get().MinimumPeersPerSubnet From 4a6d88d9fb0159bc98ac83cec5cced0bfcb0baf5 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Fri, 21 Nov 2025 12:05:57 +0400 Subject: [PATCH 26/39] finish all changes --- beacon-chain/p2p/BUILD.bazel | 1 + beacon-chain/p2p/broadcaster.go | 73 ++-- beacon-chain/p2p/broadcaster_test.go | 4 + beacon-chain/p2p/gossipsub_peer_controller.go | 81 +++-- beacon-chain/p2p/gossipsub_peer_crawler.go | 6 + .../p2p/gossipsub_peer_crawler_test.go | 3 +- .../p2p/gossipsubcrawler/interface.go | 2 +- beacon-chain/p2p/service.go | 66 ++++ beacon-chain/p2p/subnets.go | 240 +------------ beacon-chain/p2p/subnets_test.go | 224 ++++++------ beacon-chain/sync/BUILD.bazel | 1 + beacon-chain/sync/fork_watcher_test.go | 1 - beacon-chain/sync/gossipsub_base.go | 187 ++++++++++ beacon-chain/sync/gossipsub_controller.go | 77 +++-- .../sync/gossipsub_controller_test.go | 66 ++-- beacon-chain/sync/gossipsub_topic_family.go | 29 +- beacon-chain/sync/service.go | 6 +- beacon-chain/sync/subscriber.go | 318 ------------------ beacon-chain/sync/subscriber_test.go | 251 ++++---------- .../sync/subscription_topic_handler.go | 4 + .../sync/topic_families_dynamic_subnets.go | 255 +++++++------- .../sync/topic_families_static_subnets.go | 36 +- .../sync/topic_families_without_subnets.go | 308 +++++++---------- 23 files changed, 841 insertions(+), 1398 deletions(-) create mode 100644 beacon-chain/sync/gossipsub_base.go diff --git a/beacon-chain/p2p/BUILD.bazel b/beacon-chain/p2p/BUILD.bazel index d82368615c0e..1e59b72b493f 100644 --- a/beacon-chain/p2p/BUILD.bazel +++ b/beacon-chain/p2p/BUILD.bazel @@ -157,6 +157,7 @@ go_test( "//beacon-chain/core/helpers:go_default_library", "//beacon-chain/core/peerdas:go_default_library", "//beacon-chain/core/signing:go_default_library", + "//beacon-chain/db:go_default_library", "//beacon-chain/db/testing:go_default_library", "//beacon-chain/p2p/encoder:go_default_library", "//beacon-chain/p2p/gossipsubcrawler:go_default_library", diff --git a/beacon-chain/p2p/broadcaster.go b/beacon-chain/p2p/broadcaster.go index d5634f5eb640..7dc695e2488c 100644 --- a/beacon-chain/p2p/broadcaster.go +++ b/beacon-chain/p2p/broadcaster.go @@ -60,7 +60,10 @@ func (s *Service) Broadcast(ctx context.Context, msg proto.Message) error { if !ok { return errors.Errorf("message of %T does not support marshaller interface", msg) } - return s.broadcastObject(ctx, castMsg, fmt.Sprintf(topic, forkDigest)) + + fullTopic := fmt.Sprintf(topic, forkDigest) + s.Encoding().ProtocolSuffix() + + return s.broadcastObject(ctx, castMsg, fullTopic) } // BroadcastAttestation broadcasts an attestation to the p2p network, the message is assumed to be @@ -106,6 +109,7 @@ func (s *Service) BroadcastSyncCommitteeMessage(ctx context.Context, subnet uint } func (s *Service) internalBroadcastAttestation(ctx context.Context, subnet uint64, att ethpb.Att, forkDigest [fieldparams.VersionLength]byte) { + topic := AttestationSubnetTopic(forkDigest, subnet) _, span := trace.StartSpan(ctx, "p2p.internalBroadcastAttestation") defer span.End() ctx = trace.NewContext(context.Background(), span) // clear parent context / deadline. @@ -116,7 +120,7 @@ func (s *Service) internalBroadcastAttestation(ctx context.Context, subnet uint6 // Ensure we have peers with this subnet. s.subnetLocker(subnet).RLock() - hasPeer := s.hasPeerWithSubnet(attestationToTopic(subnet, forkDigest)) + hasPeer := s.hasPeerWithTopic(topic) s.subnetLocker(subnet).RUnlock() span.SetAttributes( @@ -131,8 +135,7 @@ func (s *Service) internalBroadcastAttestation(ctx context.Context, subnet uint6 s.subnetLocker(subnet).Lock() defer s.subnetLocker(subnet).Unlock() - topicStr := attestationToTopic(subnet, forkDigest) + s.Encoding().ProtocolSuffix() - if err := s.gossipsubDialer.DialPeersForTopicBlocking(topicStr, minimumPeersPerSubnetForBroadcast); err != nil { + if err := s.gossipsubDialer.DialPeersForTopicBlocking(ctx, topic, minimumPeersPerSubnetForBroadcast); err != nil { return errors.Wrap(err, "find peers with subnets") } @@ -154,13 +157,14 @@ func (s *Service) internalBroadcastAttestation(ctx context.Context, subnet uint6 return } - if err := s.broadcastObject(ctx, att, attestationToTopic(subnet, forkDigest)); err != nil { + if err := s.broadcastObject(ctx, att, topic); err != nil { log.WithError(err).Error("Failed to broadcast attestation") tracing.AnnotateError(span, err) } } func (s *Service) broadcastSyncCommittee(ctx context.Context, subnet uint64, sMsg *ethpb.SyncCommitteeMessage, forkDigest [fieldparams.VersionLength]byte) { + topic := SyncCommitteeSubnetTopic(forkDigest, subnet) _, span := trace.StartSpan(ctx, "p2p.broadcastSyncCommittee") defer span.End() ctx = trace.NewContext(context.Background(), span) // clear parent context / deadline. @@ -174,7 +178,7 @@ func (s *Service) broadcastSyncCommittee(ctx context.Context, subnet uint64, sMs // to ensure that we can reuse the same subnet locker. wrappedSubIdx := subnet + syncLockerVal s.subnetLocker(wrappedSubIdx).RLock() - hasPeer := s.hasPeerWithSubnet(syncCommitteeToTopic(subnet, forkDigest)) + hasPeer := s.hasPeerWithTopic(topic) s.subnetLocker(wrappedSubIdx).RUnlock() span.SetAttributes( @@ -188,8 +192,7 @@ func (s *Service) broadcastSyncCommittee(ctx context.Context, subnet uint64, sMs if err := func() error { s.subnetLocker(wrappedSubIdx).Lock() defer s.subnetLocker(wrappedSubIdx).Unlock() - topicStr := syncCommitteeToTopic(subnet, forkDigest) + s.Encoding().ProtocolSuffix() - if err := s.gossipsubDialer.DialPeersForTopicBlocking(topicStr, minimumPeersPerSubnetForBroadcast); err != nil { + if err := s.gossipsubDialer.DialPeersForTopicBlocking(ctx, topic, minimumPeersPerSubnetForBroadcast); err != nil { return errors.Wrap(err, "find peers with subnets") } @@ -207,7 +210,7 @@ func (s *Service) broadcastSyncCommittee(ctx context.Context, subnet uint64, sMs return } - if err := s.broadcastObject(ctx, sMsg, syncCommitteeToTopic(subnet, forkDigest)); err != nil { + if err := s.broadcastObject(ctx, sMsg, topic); err != nil { log.WithError(err).Error("Failed to broadcast sync committee message") tracing.AnnotateError(span, err) } @@ -235,6 +238,7 @@ func (s *Service) BroadcastBlob(ctx context.Context, subnet uint64, blob *ethpb. } func (s *Service) internalBroadcastBlob(ctx context.Context, subnet uint64, blobSidecar *ethpb.BlobSidecar, forkDigest [fieldparams.VersionLength]byte) { + topic := BlobSubnetTopic(forkDigest, subnet) _, span := trace.StartSpan(ctx, "p2p.internalBroadcastBlob") defer span.End() ctx = trace.NewContext(context.Background(), span) // clear parent context / deadline. @@ -245,7 +249,7 @@ func (s *Service) internalBroadcastBlob(ctx context.Context, subnet uint64, blob wrappedSubIdx := subnet + blobSubnetLockerVal s.subnetLocker(wrappedSubIdx).RLock() - hasPeer := s.hasPeerWithSubnet(blobSubnetToTopic(subnet, forkDigest)) + hasPeer := s.hasPeerWithTopic(topic) s.subnetLocker(wrappedSubIdx).RUnlock() if !hasPeer { @@ -254,8 +258,7 @@ func (s *Service) internalBroadcastBlob(ctx context.Context, subnet uint64, blob s.subnetLocker(wrappedSubIdx).Lock() defer s.subnetLocker(wrappedSubIdx).Unlock() - topicStr := blobSubnetToTopic(subnet, forkDigest) + s.Encoding().ProtocolSuffix() - if err := s.gossipsubDialer.DialPeersForTopicBlocking(topicStr, minimumPeersPerSubnetForBroadcast); err != nil { + if err := s.gossipsubDialer.DialPeersForTopicBlocking(ctx, topic, minimumPeersPerSubnetForBroadcast); err != nil { return errors.Wrap(err, "find peers with subnets") } @@ -267,7 +270,7 @@ func (s *Service) internalBroadcastBlob(ctx context.Context, subnet uint64, blob } } - if err := s.broadcastObject(ctx, blobSidecar, blobSubnetToTopic(subnet, forkDigest)); err != nil { + if err := s.broadcastObject(ctx, blobSidecar, topic); err != nil { log.WithError(err).Error("Failed to broadcast blob sidecar") tracing.AnnotateError(span, err) } @@ -296,7 +299,7 @@ func (s *Service) BroadcastLightClientOptimisticUpdate(ctx context.Context, upda } digest := params.ForkDigest(slots.ToEpoch(update.AttestedHeader().Beacon().Slot)) - if err := s.broadcastObject(ctx, update, lcOptimisticToTopic(digest)); err != nil { + if err := s.broadcastObject(ctx, update, LcOptimisticToTopic(digest)); err != nil { log.WithError(err).Debug("Failed to broadcast light client optimistic update") err := errors.Wrap(err, "could not publish message") tracing.AnnotateError(span, err) @@ -330,7 +333,7 @@ func (s *Service) BroadcastLightClientFinalityUpdate(ctx context.Context, update } forkDigest := params.ForkDigest(slots.ToEpoch(update.AttestedHeader().Beacon().Slot)) - if err := s.broadcastObject(ctx, update, lcFinalityToTopic(forkDigest)); err != nil { + if err := s.broadcastObject(ctx, update, LcFinalityToTopic(forkDigest)); err != nil { log.WithError(err).Debug("Failed to broadcast light client finality update") err := errors.Wrap(err, "could not publish message") tracing.AnnotateError(span, err) @@ -389,16 +392,14 @@ func (s *Service) broadcastDataColumnSidecars(ctx context.Context, forkDigest [f subnet := peerdas.ComputeSubnetForDataColumnSidecar(sidecar.Index) // Build the topic corresponding to subnet column subnet and this fork digest. - topic := dataColumnSubnetToTopic(subnet, forkDigest) + topic := DataColumnSubnetTopic(forkDigest, subnet) // Compute the wrapped subnet index. wrappedSubIdx := subnet + dataColumnSubnetVal // Find peers if needed. - builder := func(idx uint64) string { - return dataColumnSubnetToTopic(idx, forkDigest) + s.Encoding().ProtocolSuffix() - } - if err := s.findPeersIfNeeded(ctx, wrappedSubIdx, builder, subnet); err != nil { + + if err := s.findPeersIfNeeded(ctx, wrappedSubIdx, topic); err != nil { tracing.AnnotateError(span, err) log.WithError(err).Error("Cannot find peers if needed") return @@ -493,16 +494,14 @@ func (s *Service) broadcastDataColumnSidecars(ctx context.Context, forkDigest [f func (s *Service) findPeersIfNeeded( ctx context.Context, wrappedSubIdx uint64, - fullTopicForSubnet func(uint64) string, - subnet uint64, + topic string, ) error { // Sending a data column sidecar to only one peer is not ideal, // but it ensures at least one peer receives it. s.subnetLocker(wrappedSubIdx).Lock() defer s.subnetLocker(wrappedSubIdx).Unlock() - topicStr := fullTopicForSubnet(subnet) - if err := s.gossipsubDialer.DialPeersForTopicBlocking(topicStr, minimumPeersPerSubnetForBroadcast); err != nil { + if err := s.gossipsubDialer.DialPeersForTopicBlocking(ctx, topic, minimumPeersPerSubnetForBroadcast); err != nil { return errors.Wrap(err, "find peers with subnet") } return nil @@ -529,34 +528,10 @@ func (s *Service) broadcastObject(ctx context.Context, obj ssz.Marshaler, topic iid := int64(id) span = trace.AddMessageSendEvent(span, iid, messageLen /*uncompressed*/, messageLen /*compressed*/) } - if err := s.PublishToTopic(ctx, topic+s.Encoding().ProtocolSuffix(), buf.Bytes()); err != nil { + if err := s.PublishToTopic(ctx, topic, buf.Bytes()); err != nil { err := errors.Wrap(err, "could not publish message") tracing.AnnotateError(span, err) return err } return nil } - -func attestationToTopic(subnet uint64, forkDigest [fieldparams.VersionLength]byte) string { - return fmt.Sprintf(AttestationSubnetTopicFormat, forkDigest, subnet) -} - -func syncCommitteeToTopic(subnet uint64, forkDigest [fieldparams.VersionLength]byte) string { - return fmt.Sprintf(SyncCommitteeSubnetTopicFormat, forkDigest, subnet) -} - -func blobSubnetToTopic(subnet uint64, forkDigest [fieldparams.VersionLength]byte) string { - return fmt.Sprintf(BlobSubnetTopicFormat, forkDigest, subnet) -} - -func lcOptimisticToTopic(forkDigest [4]byte) string { - return fmt.Sprintf(LightClientOptimisticUpdateTopicFormat, forkDigest) -} - -func lcFinalityToTopic(forkDigest [4]byte) string { - return fmt.Sprintf(LightClientFinalityUpdateTopicFormat, forkDigest) -} - -func dataColumnSubnetToTopic(subnet uint64, forkDigest [fieldparams.VersionLength]byte) string { - return fmt.Sprintf(DataColumnSubnetTopicFormat, forkDigest, subnet) -} diff --git a/beacon-chain/p2p/broadcaster_test.go b/beacon-chain/p2p/broadcaster_test.go index 0a8406ea8f67..e6b2d7fcfaab 100644 --- a/beacon-chain/p2p/broadcaster_test.go +++ b/beacon-chain/p2p/broadcaster_test.go @@ -773,3 +773,7 @@ func TestService_BroadcastDataColumn(t *testing.T) { require.NoError(t, service.Encoding().DecodeGossip(msg.Data, &result)) require.DeepEqual(t, &result, verifiedRoSidecar) } + +func attestationToTopic(subnet uint64, forkDigest [fieldparams.VersionLength]byte) string { + return fmt.Sprintf(AttestationSubnetTopicFormat, forkDigest, subnet) +} diff --git a/beacon-chain/p2p/gossipsub_peer_controller.go b/beacon-chain/p2p/gossipsub_peer_controller.go index e394b4f78964..123869138fb5 100644 --- a/beacon-chain/p2p/gossipsub_peer_controller.go +++ b/beacon-chain/p2p/gossipsub_peer_controller.go @@ -12,7 +12,7 @@ import ( "github.com/pkg/errors" ) -var ( +const ( peerPerTopic = 20 ) @@ -51,9 +51,11 @@ func (g *GossipsubPeerDialer) Start(provider gossipsubcrawler.SubnetTopicsProvid g.once.Do(func() { g.topicsProvider = provider - g.wg.Go(func() { + g.wg.Add(1) + go func() { + defer g.wg.Done() g.dialLoop() - }) + }() }) return nil @@ -70,28 +72,26 @@ func (g *GossipsubPeerDialer) dialLoop() { var peersToDial []*enode.Node for _, topic := range topics { - peers := g.service.PubSub().ListPeers(topic) - peerCount := len(peers) - if peerCount >= peerPerTopic { - continue - } - missing := peerPerTopic - peerCount - // this is fine as "PeersForTopic" does not return peers we are already connected to - newPeers := g.crawler.PeersForTopic(gossipsubcrawler.Topic(topic)) - if len(newPeers) > missing { - newPeers = newPeers[:missing] - } + newPeers := g.peersForTopic(topic, peerPerTopic) peersToDial = append(peersToDial, newPeers...) } - if len(peersToDial) > 0 { - // Dial new peers in batches. - maxConcurrentDials := math.MaxInt - if flags.MaxDialIsActive() { - maxConcurrentDials = flags.Get().MaxConcurrentDials + if len(peersToDial) == 0 { + continue + } + + // Deduplicate peers to avoid dialing the same peer multiple times. + uniquePeers := make([]*enode.Node, 0, len(peersToDial)) + seen := make(map[enode.ID]struct{}) + for _, p := range peersToDial { + if _, ok := seen[p.ID()]; !ok { + seen[p.ID()] = struct{}{} + uniquePeers = append(uniquePeers, p) } - g.service.DialPeers(g.ctx, maxConcurrentDials, peersToDial) } + peersToDial = uniquePeers + + g.dialPeers(peersToDial) case <-g.ctx.Done(): return @@ -99,31 +99,48 @@ func (g *GossipsubPeerDialer) dialLoop() { } } -func (g *GossipsubPeerDialer) DialPeersForTopicBlocking(topic string, nPeers int) error { +func (g *GossipsubPeerDialer) DialPeersForTopicBlocking(ctx context.Context, topic string, nPeers int) error { for { peers := g.service.PubSub().ListPeers(topic) if len(peers) >= nPeers { return nil } - missing := nPeers - len(peers) - // this is fine as "PeersForTopic" does not return peers we are already connected to - newPeers := g.crawler.PeersForTopic(gossipsubcrawler.Topic(topic)) + newPeers := g.peersForTopic(topic, nPeers) if len(newPeers) > 0 { - if len(newPeers) > missing { - newPeers = newPeers[:missing] - } - maxConcurrentDials := math.MaxInt - if flags.MaxDialIsActive() { - maxConcurrentDials = flags.Get().MaxConcurrentDials - } - g.service.DialPeers(g.ctx, maxConcurrentDials, newPeers) + g.dialPeers(newPeers) } select { + case <-ctx.Done(): + return ctx.Err() case <-time.After(100 * time.Millisecond): case <-g.ctx.Done(): return g.ctx.Err() } } } + +func (g *GossipsubPeerDialer) peersForTopic(topic string, targetCount int) []*enode.Node { + peers := g.service.PubSub().ListPeers(topic) + peerCount := len(peers) + if peerCount >= targetCount { + return nil + } + missing := targetCount - peerCount + // this is fine as "PeersForTopic" does not return peers we are already connected to + newPeers := g.crawler.PeersForTopic(gossipsubcrawler.Topic(topic)) + if len(newPeers) > missing { + newPeers = newPeers[:missing] + } + return newPeers +} + +func (g *GossipsubPeerDialer) dialPeers(peers []*enode.Node) { + // Dial new peers in batches. + maxConcurrentDials := math.MaxInt + if flags.MaxDialIsActive() { + maxConcurrentDials = flags.Get().MaxConcurrentDials + } + g.service.DialPeers(g.ctx, maxConcurrentDials, peers) +} diff --git a/beacon-chain/p2p/gossipsub_peer_crawler.go b/beacon-chain/p2p/gossipsub_peer_crawler.go index 92a8e645d58f..563dc032d9c2 100644 --- a/beacon-chain/p2p/gossipsub_peer_crawler.go +++ b/beacon-chain/p2p/gossipsub_peer_crawler.go @@ -311,12 +311,18 @@ func (g *GossipsubPeerCrawler) PeersForTopic(topic gossipsubcrawler.Topic) []*en } var peerNodes []*peerNode + seen := make(map[enode.ID]bool) for peerID := range peerIDs { peerNode, ok := g.crawledPeers.byPeerId[peerID] if !ok { continue } if peerNode.isPinged && g.peerFilter(peerNode.node) { + // Skip if we've already seen this enode ID + if seen[peerNode.id] { + continue + } + seen[peerNode.id] = true peerNodes = append(peerNodes, peerNode) } } diff --git a/beacon-chain/p2p/gossipsub_peer_crawler_test.go b/beacon-chain/p2p/gossipsub_peer_crawler_test.go index 1786d2c947fe..5e14769ad853 100644 --- a/beacon-chain/p2p/gossipsub_peer_crawler_test.go +++ b/beacon-chain/p2p/gossipsub_peer_crawler_test.go @@ -11,6 +11,7 @@ import ( "github.com/ethereum/go-ethereum/p2p/enode" "github.com/libp2p/go-libp2p/core/peer" "github.com/stretchr/testify/require" + require2 "github.com/stretchr/testify/require" ) // Helpers for crawledPeers tests @@ -683,7 +684,7 @@ func TestCrawler_AddsAndPingsPeer(t *testing.T) { defer g.Stop() // Verify that the peer has been indexed under the topic and marked as pinged - require.Eventually(t, func() bool { + require2.Eventually(t, func() bool { g.crawledPeers.mu.RLock() defer g.crawledPeers.mu.RUnlock() diff --git a/beacon-chain/p2p/gossipsubcrawler/interface.go b/beacon-chain/p2p/gossipsubcrawler/interface.go index d5da2b3f8f73..003435d7c710 100644 --- a/beacon-chain/p2p/gossipsubcrawler/interface.go +++ b/beacon-chain/p2p/gossipsubcrawler/interface.go @@ -35,5 +35,5 @@ type SubnetTopicsProvider func() []string type GossipsubDialer interface { Start(provider SubnetTopicsProvider) error Stop() - DialPeersForTopicBlocking(topic string, nPeers int) error + DialPeersForTopicBlocking(ctx context.Context, topic string, nPeers int) error } diff --git a/beacon-chain/p2p/service.go b/beacon-chain/p2p/service.go index 13f206c96e8d..06ccac233bc5 100644 --- a/beacon-chain/p2p/service.go +++ b/beacon-chain/p2p/service.go @@ -6,6 +6,7 @@ package p2p import ( "context" "crypto/ecdsa" + "fmt" "sync" "time" @@ -607,3 +608,68 @@ func SyncSubnets(nodeID enode.ID, node *enode.Node, record *enr.Record) (map[uin func DataColumnSubnets(nodeID enode.ID, node *enode.Node, record *enr.Record) (map[uint64]bool, error) { return dataColumnSubnets(nodeID, record) } + +func DataColumnSubnetTopic(digest [4]byte, subnet uint64) string { + e := &encoder.SszNetworkEncoder{} + return fmt.Sprintf(DataColumnSubnetTopicFormat, digest, subnet) + e.ProtocolSuffix() +} + +func SyncCommitteeSubnetTopic(digest [4]byte, subnet uint64) string { + e := &encoder.SszNetworkEncoder{} + return fmt.Sprintf(SyncCommitteeSubnetTopicFormat, digest, subnet) + e.ProtocolSuffix() +} + +func AttestationSubnetTopic(digest [4]byte, subnet uint64) string { + e := &encoder.SszNetworkEncoder{} + return fmt.Sprintf(AttestationSubnetTopicFormat, digest, subnet) + e.ProtocolSuffix() +} + +func BlobSubnetTopic(digest [4]byte, subnet uint64) string { + e := &encoder.SszNetworkEncoder{} + return fmt.Sprintf(BlobSubnetTopicFormat, digest, subnet) + e.ProtocolSuffix() +} + +func LcOptimisticToTopic(forkDigest [4]byte) string { + e := &encoder.SszNetworkEncoder{} + return fmt.Sprintf(LightClientOptimisticUpdateTopicFormat, forkDigest) + e.ProtocolSuffix() +} + +func LcFinalityToTopic(forkDigest [4]byte) string { + e := &encoder.SszNetworkEncoder{} + return fmt.Sprintf(LightClientFinalityUpdateTopicFormat, forkDigest) + e.ProtocolSuffix() +} + +func BlockSubnetTopic(forkDigest [4]byte) string { + e := &encoder.SszNetworkEncoder{} + return fmt.Sprintf(BlockSubnetTopicFormat, forkDigest) + e.ProtocolSuffix() +} + +func AggregateAndProofSubnetTopic(forkDigest [4]byte) string { + e := &encoder.SszNetworkEncoder{} + return fmt.Sprintf(AggregateAndProofSubnetTopicFormat, forkDigest) + e.ProtocolSuffix() +} + +func VoluntaryExitSubnetTopic(forkDigest [4]byte) string { + e := &encoder.SszNetworkEncoder{} + return fmt.Sprintf(ExitSubnetTopicFormat, forkDigest) + e.ProtocolSuffix() +} + +func ProposerSlashingSubnetTopic(forkDigest [4]byte) string { + e := &encoder.SszNetworkEncoder{} + return fmt.Sprintf(ProposerSlashingSubnetTopicFormat, forkDigest) + e.ProtocolSuffix() +} + +func AttesterSlashingSubnetTopic(forkDigest [4]byte) string { + e := &encoder.SszNetworkEncoder{} + return fmt.Sprintf(AttesterSlashingSubnetTopicFormat, forkDigest) + e.ProtocolSuffix() +} + +func SyncContributionAndProofSubnetTopic(forkDigest [4]byte) string { + e := &encoder.SszNetworkEncoder{} + return fmt.Sprintf(SyncContributionAndProofSubnetTopicFormat, forkDigest) + e.ProtocolSuffix() +} + +func BlsToExecutionChangeSubnetTopic(forkDigest [4]byte) string { + e := &encoder.SszNetworkEncoder{} + return fmt.Sprintf(BlsToExecutionChangeSubnetTopicFormat, forkDigest) + e.ProtocolSuffix() +} diff --git a/beacon-chain/p2p/subnets.go b/beacon-chain/p2p/subnets.go index 58361e599576..6866d5b930fe 100644 --- a/beacon-chain/p2p/subnets.go +++ b/beacon-chain/p2p/subnets.go @@ -3,7 +3,6 @@ package p2p import ( "context" "fmt" - "strings" "sync" "time" @@ -22,7 +21,6 @@ import ( "github.com/ethereum/go-ethereum/p2p/enr" "github.com/holiman/uint256" "github.com/pkg/errors" - "github.com/sirupsen/logrus" ) var ( @@ -57,167 +55,6 @@ const dataColumnSubnetVal = 150 const errSavingSequenceNumber = "saving sequence number after updating subnets: %w" -// nodeFilter returns a function that filters nodes based on the subnet topic and subnet index. -func (s *Service) nodeFilter(topic string, indices map[uint64]int) (func(node *enode.Node) (map[uint64]bool, error), error) { - switch { - case strings.Contains(topic, GossipAttestationMessage): - return s.filterPeerForAttSubnet(indices), nil - case strings.Contains(topic, GossipSyncCommitteeMessage): - return s.filterPeerForSyncSubnet(indices), nil - case strings.Contains(topic, GossipBlobSidecarMessage): - return s.filterPeerForBlobSubnet(indices), nil - case strings.Contains(topic, GossipDataColumnSidecarMessage): - return s.filterPeerForDataColumnsSubnet(indices), nil - default: - return nil, errors.Errorf("no subnet exists for provided topic: %s", topic) - } -} - -// updateDefectiveSubnets updates the defective subnets map when a node with matching subnets is found. -// It decrements the defective count for each subnet the node satisfies and removes subnets -// that are fully satisfied (count reaches 0). -func updateDefectiveSubnets( - nodeSubnets map[uint64]bool, - defectiveSubnets map[uint64]int, -) { - for subnet := range defectiveSubnets { - if !nodeSubnets[subnet] { - continue - } - defectiveSubnets[subnet]-- - if defectiveSubnets[subnet] == 0 { - delete(defectiveSubnets, subnet) - } - } -} - -// findPeersWithSubnets finds peers subscribed to defective subnets in batches -// until enough peers are found or the context is canceled. -// It returns new peers found during the search. -func (s *Service) findPeersWithSubnets( - ctx context.Context, - fullTopicForSubnet func(uint64) string, - minimumPeersPerSubnet int, - defectiveSubnetsOrigin map[uint64]int, -) ([]*enode.Node, error) { - // Copy the defective subnets map to avoid modifying the original map. - defectiveSubnets := make(map[uint64]int, len(defectiveSubnetsOrigin)) - for k, v := range defectiveSubnetsOrigin { - defectiveSubnets[k] = v - } - - // Create an discovery iterator to find new peers. - iterator := s.dv5Listener.RandomNodes() - - // `iterator.Next` can block indefinitely. `iterator.Close` unblocks it. - // So it is important to close the iterator when the context is done to ensure - // that the search does not hang indefinitely. - go func() { - <-ctx.Done() - iterator.Close() - }() - - // Retrieve the filter function that will be used to filter nodes based on the defective subnets. - // Use any subnet's full topic to infer the family type from the topic string. - var sampleTopic string - for k := range defectiveSubnets { - sampleTopic = fullTopicForSubnet(k) - break - } - filter, err := s.nodeFilter(sampleTopic, defectiveSubnets) - if err != nil { - return nil, errors.Wrap(err, "node filter") - } - - // Crawl the network for peers subscribed to the defective subnets. - nodeByNodeID := make(map[enode.ID]*enode.Node) - - for len(defectiveSubnets) > 0 && iterator.Next() { - if err := ctx.Err(); err != nil { - // Convert the map to a slice. - peersToDial := make([]*enode.Node, 0, len(nodeByNodeID)) - for _, node := range nodeByNodeID { - peersToDial = append(peersToDial, node) - } - - return peersToDial, err - } - - node := iterator.Node() - - // Remove duplicates, keeping the node with higher seq. - existing, ok := nodeByNodeID[node.ID()] - if ok && existing.Seq() >= node.Seq() { - continue // keep existing and skip. - } - - // Treat nodes that exist in nodeByNodeID with higher seq numbers as new peers - // Skip peer not matching the filter. - if !s.filterPeer(node) { - if ok { - // this means the existing peer with the lower sequence number is no longer valid - delete(nodeByNodeID, existing.ID()) - // Note: We are choosing to not rollback changes to the defective subnets map in favor of calling s.defectiveSubnets once again after dialing peers. - // This is a case that should rarely happen and should be handled through a second iteration in FindAndDialPeersWithSubnets - } - continue - } - - // Get all needed subnets that the node is subscribed to. - // Skip nodes that are not subscribed to any of the defective subnets. - nodeSubnets, err := filter(node) - if err != nil { - log.WithError(err).WithFields(logrus.Fields{ - "nodeID": node.ID(), - "topic": sampleTopic, - }).Debug("Could not get needed subnets from peer") - - continue - } - - if len(nodeSubnets) == 0 { - continue - } - - // We found a new peer. Modify the defective subnets map - // and the filter accordingly. - nodeByNodeID[node.ID()] = node - - updateDefectiveSubnets(nodeSubnets, defectiveSubnets) - filter, err = s.nodeFilter(sampleTopic, defectiveSubnets) - if err != nil { - return nil, errors.Wrap(err, "node filter") - } - } - - // Convert the map to a slice. - peersToDial := make([]*enode.Node, 0, len(nodeByNodeID)) - for _, node := range nodeByNodeID { - peersToDial = append(peersToDial, node) - } - - return peersToDial, nil -} - -// defectiveSubnets returns a map of subnets that have fewer than the minimum peer count. -func (s *Service) defectiveSubnets( - fullTopicForSubnet func(uint64) string, - minimumPeersPerSubnet int, - subnets map[uint64]bool, -) map[uint64]int { - missingCountPerSubnet := make(map[uint64]int, len(subnets)) - for subnet := range subnets { - topic := fullTopicForSubnet(subnet) - peers := s.pubsub.ListPeers(topic) - peerCount := len(peers) - if peerCount < minimumPeersPerSubnet { - missingCountPerSubnet[subnet] = minimumPeersPerSubnet - peerCount - } - } - - return missingCountPerSubnet -} - // DialPeers dials multiple peers concurrently up to `maxConcurrentDials` at a time. // In case of a dial failure, it logs the error but continues dialing other peers. func (s *Service) DialPeers(ctx context.Context, maxConcurrentDials int, nodes []*enode.Node) uint { @@ -264,75 +101,13 @@ func (s *Service) DialPeers(ctx context.Context, maxConcurrentDials int, nodes [ return counter } -// filterPeerForAttSubnet returns a method with filters peers specifically for a particular attestation subnet. -func (s *Service) filterPeerForAttSubnet(indices map[uint64]int) func(node *enode.Node) (map[uint64]bool, error) { - return func(node *enode.Node) (map[uint64]bool, error) { - if !s.filterPeer(node) { - return map[uint64]bool{}, nil - } - - subnets, err := attestationSubnets(node.Record()) - if err != nil { - return nil, errors.Wrap(err, "attestation subnets") - } - - return intersect(indices, subnets), nil - } -} - -// returns a method with filters peers specifically for a particular sync subnet. -func (s *Service) filterPeerForSyncSubnet(indices map[uint64]int) func(node *enode.Node) (map[uint64]bool, error) { - return func(node *enode.Node) (map[uint64]bool, error) { - if !s.filterPeer(node) { - return map[uint64]bool{}, nil - } - - subnets, err := syncSubnets(node.Record()) - if err != nil { - return nil, errors.Wrap(err, "sync subnets") - } - - return intersect(indices, subnets), nil - } -} - -// returns a method with filters peers specifically for a particular blob subnet. -// All peers are supposed to be subscribed to all blob subnets. -func (s *Service) filterPeerForBlobSubnet(indices map[uint64]int) func(_ *enode.Node) (map[uint64]bool, error) { - result := make(map[uint64]bool, len(indices)) - for i := range indices { - result[i] = true - } - - return func(_ *enode.Node) (map[uint64]bool, error) { - return result, nil - } -} - -// returns a method with filters peers specifically for a particular data column subnet. -func (s *Service) filterPeerForDataColumnsSubnet(indices map[uint64]int) func(node *enode.Node) (map[uint64]bool, error) { - return func(node *enode.Node) (map[uint64]bool, error) { - if !s.filterPeer(node) { - return map[uint64]bool{}, nil - } - - subnets, err := dataColumnSubnets(node.ID(), node.Record()) - if err != nil { - return nil, errors.Wrap(err, "data column subnets") - } - - return intersect(indices, subnets), nil - } -} - // lower threshold to broadcast object compared to searching // for a subnet. So that even in the event of poor peer // connectivity, we can still broadcast an attestation. -func (s *Service) hasPeerWithSubnet(subnetTopic string) bool { +func (s *Service) hasPeerWithTopic(topic string) bool { // In the event peer threshold is lower, we will choose the lower // threshold. minPeers := min(1, flags.Get().MinimumPeersPerSubnet) - topic := subnetTopic + s.Encoding().ProtocolSuffix() peersWithSubnet := s.pubsub.ListPeers(topic) peersWithSubnetCount := len(peersWithSubnet) @@ -657,16 +432,3 @@ func byteCount(bitCount int) int { } return numOfBytes } - -// interesect intersects two maps and returns a new map containing only the keys -// that are present in both maps. -func intersect(left map[uint64]int, right map[uint64]bool) map[uint64]bool { - result := make(map[uint64]bool, min(len(left), len(right))) - for i := range left { - if right[i] { - result[i] = true - } - } - - return result -} diff --git a/beacon-chain/p2p/subnets_test.go b/beacon-chain/p2p/subnets_test.go index da95dd362943..de7070a877c0 100644 --- a/beacon-chain/p2p/subnets_test.go +++ b/beacon-chain/p2p/subnets_test.go @@ -10,7 +10,9 @@ import ( "github.com/OffchainLabs/go-bitfield" "github.com/OffchainLabs/prysm/v6/beacon-chain/cache" "github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas" + "github.com/OffchainLabs/prysm/v6/beacon-chain/db" testDB "github.com/OffchainLabs/prysm/v6/beacon-chain/db/testing" + "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/gossipsubcrawler" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers/scorers" testp2p "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/testing" @@ -24,6 +26,7 @@ import ( "github.com/ethereum/go-ethereum/p2p/enr" "github.com/libp2p/go-libp2p/core/crypto" "github.com/libp2p/go-libp2p/core/network" + require2 "github.com/stretchr/testify/require" ) func TestStartDiscV5_FindAndDialPeersWithSubnet(t *testing.T) { @@ -177,6 +180,14 @@ func TestStartDiscV5_FindAndDialPeersWithSubnet(t *testing.T) { service.custodyInfo = &custodyInfo{} service.Start() + + subnets := map[uint64]bool{1: true, 2: true, 3: true} + var topics []string + for subnet := range subnets { + t := fmt.Sprintf(AttestationSubnetTopicFormat, bootNodeForkDigest, subnet) + service.Encoding().ProtocolSuffix() + topics = append(topics, t) + } + // start the crawler with a topic extractor that maps ENR attestation subnets // to full attestation topics for the current fork digest and encoding. _ = service.Crawler().Start(func(ctx context.Context, node *enode.Node) ([]string, error) { @@ -184,7 +195,6 @@ func TestStartDiscV5_FindAndDialPeersWithSubnet(t *testing.T) { if err != nil { return nil, err } - var topics []string for subnet := range subs { t := fmt.Sprintf(AttestationSubnetTopicFormat, bootNodeForkDigest, subnet) + service.Encoding().ProtocolSuffix() topics = append(topics, t) @@ -196,12 +206,11 @@ func TestStartDiscV5_FindAndDialPeersWithSubnet(t *testing.T) { require.NoError(t, err) }() - subnets := map[uint64]bool{1: true, 2: true, 3: true} builder := func(idx uint64) string { return fmt.Sprintf(AttestationSubnetTopicFormat, bootNodeForkDigest, idx) + service.Encoding().ProtocolSuffix() } - defectiveSubnets := service.defectiveSubnets(builder, minimumPeersPerSubnet, subnets) - require.Equal(t, subnetCount, len(defectiveSubnets)) + defectiveSubnetsCount := defectiveSubnets(service, topics, minimumPeersPerSubnet) + require.Equal(t, subnetCount, defectiveSubnetsCount) var topicsToDial []string for s := range subnets { @@ -209,12 +218,23 @@ func TestStartDiscV5_FindAndDialPeersWithSubnet(t *testing.T) { } for _, topic := range topicsToDial { - err = service.GossipsubDialer().DialPeersForTopicBlocking(topic, minimumPeersPerSubnet) + err = service.GossipsubDialer().DialPeersForTopicBlocking(ctx, topic, minimumPeersPerSubnet) require.NoError(t, err) } - defectiveSubnets = service.defectiveSubnets(builder, minimumPeersPerSubnet, subnets) - require.Equal(t, 0, len(defectiveSubnets)) + defectiveSubnetsCount = defectiveSubnets(service, topics, minimumPeersPerSubnet) + require.Equal(t, 0, defectiveSubnetsCount) +} + +func defectiveSubnets(service *Service, topics []string, minimumPeersPerSubnet int) int { + count := 0 + for _, topic := range topics { + peers := service.pubsub.ListPeers(topic) + if len(peers) < minimumPeersPerSubnet { + count++ + } + } + return count } func Test_AttSubnets(t *testing.T) { @@ -610,7 +630,6 @@ func TestFindPeersWithSubnets_NodeDeduplication(t *testing.T) { cache.SubnetIDs.EmptyAllCaches() defer cache.SubnetIDs.EmptyAllCaches() - ctx := context.Background() db := testDB.SetupDB(t) localNode1 := createTestNodeWithID(t, "node1") @@ -771,49 +790,13 @@ func TestFindPeersWithSubnets_NodeDeduplication(t *testing.T) { flags.Init(gFlags) defer flags.Init(new(flags.GlobalFlags)) - fakePeer := testp2p.NewTestP2P(t) - - s := &Service{ - cfg: &Config{ - MaxPeers: 30, - DB: db, - }, - genesisTime: time.Now(), - genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32), - peers: peers.NewStatus(ctx, &peers.StatusConfig{ - PeerLimit: 30, - ScorerParams: &scorers.Config{}, - }), - host: fakePeer.BHost, - } - + s := createTestService(t, db) localNode := createTestNodeRandom(t) - mockIter := testp2p.NewMockIterator(tt.nodes) s.dv5Listener = testp2p.NewMockListener(localNode, mockIter) - digest, err := s.currentForkDigest() - require.NoError(t, err) - - ctxWithTimeout, cancel := context.WithTimeout(ctx, 100*time.Millisecond) - defer cancel() - - builder := func(idx uint64) string { - return fmt.Sprintf(AttestationSubnetTopicFormat, digest, idx) + s.Encoding().ProtocolSuffix() - } - result, err := s.findPeersWithSubnets( - ctxWithTimeout, - builder, - 1, - tt.defectiveSubnets, - ) - - require.NoError(t, err, tt.description) - require.Equal(t, tt.expectedCount, len(result), tt.description) - - if tt.eval != nil { - tt.eval(t, result) - } + crawler := startTestCrawler(t, s, s.dv5Listener.(*testp2p.MockListener)) + verifyCrawlerPeers(t, crawler, s, tt.defectiveSubnets, tt.expectedCount, tt.description, tt.eval) }) } } @@ -823,7 +806,6 @@ func TestFindPeersWithSubnets_FilterPeerRemoval(t *testing.T) { cache.SubnetIDs.EmptyAllCaches() defer cache.SubnetIDs.EmptyAllCaches() - ctx := context.Background() db := testDB.SetupDB(t) localNode1 := createTestNodeWithID(t, "node1") @@ -976,23 +958,7 @@ func TestFindPeersWithSubnets_FilterPeerRemoval(t *testing.T) { flags.Init(gFlags) defer flags.Init(new(flags.GlobalFlags)) - // Create test P2P instance - fakePeer := testp2p.NewTestP2P(t) - - // Create mock service - s := &Service{ - cfg: &Config{ - MaxPeers: 30, - DB: db, - }, - genesisTime: time.Now(), - genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32), - peers: peers.NewStatus(ctx, &peers.StatusConfig{ - PeerLimit: 30, - ScorerParams: &scorers.Config{}, - }), - host: fakePeer.BHost, - } + s := createTestService(t, db) // Mark specific node versions as "bad" to simulate filterPeer failures for _, node := range tt.nodes { @@ -1010,32 +976,11 @@ func TestFindPeersWithSubnets_FilterPeerRemoval(t *testing.T) { } localNode := createTestNodeRandom(t) - mockIter := testp2p.NewMockIterator(tt.nodes) s.dv5Listener = testp2p.NewMockListener(localNode, mockIter) - digest, err := s.currentForkDigest() - require.NoError(t, err) - - ctxWithTimeout, cancel := context.WithTimeout(ctx, 100*time.Millisecond) - defer cancel() - - builder := func(idx uint64) string { - return fmt.Sprintf(AttestationSubnetTopicFormat, digest, idx) + s.Encoding().ProtocolSuffix() - } - result, err := s.findPeersWithSubnets( - ctxWithTimeout, - builder, - 1, - tt.defectiveSubnets, - ) - - require.NoError(t, err, tt.description) - require.Equal(t, tt.expectedCount, len(result), tt.description) - - if tt.eval != nil { - tt.eval(t, result) - } + crawler := startTestCrawler(t, s, s.dv5Listener.(*testp2p.MockListener)) + verifyCrawlerPeers(t, crawler, s, tt.defectiveSubnets, tt.expectedCount, tt.description, tt.eval) }) } } @@ -1079,7 +1024,6 @@ func TestFindPeersWithSubnets_received_bad_existing_node(t *testing.T) { cache.SubnetIDs.EmptyAllCaches() defer cache.SubnetIDs.EmptyAllCaches() - ctx := context.Background() db := testDB.SetupDB(t) // Create LocalNode with same ID but different sequences @@ -1100,21 +1044,7 @@ func TestFindPeersWithSubnets_received_bad_existing_node(t *testing.T) { flags.Init(gFlags) defer flags.Init(new(flags.GlobalFlags)) - fakePeer := testp2p.NewTestP2P(t) - - service := &Service{ - cfg: &Config{ - MaxPeers: 30, - DB: db, - }, - genesisTime: time.Now(), - genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32), - peers: peers.NewStatus(ctx, &peers.StatusConfig{ - PeerLimit: 30, - ScorerParams: &scorers.Config{}, - }), - host: fakePeer.BHost, - } + service := createTestService(t, db) // Create iterator with callback that marks peer as bad before processing node1_seq2 iter := &callbackIteratorForSubnets{ @@ -1138,24 +1068,80 @@ func TestFindPeersWithSubnets_received_bad_existing_node(t *testing.T) { localNode := createTestNodeRandom(t) service.dv5Listener = testp2p.NewMockListener(localNode, iter) - digest, err := service.currentForkDigest() - require.NoError(t, err) + crawler := startTestCrawler(t, service, service.dv5Listener.(*testp2p.MockListener)) - // Run findPeersWithSubnets - node1_seq1 gets processed first, then callback marks peer bad, then node1_seq2 fails - ctxWithTimeout, cancel := context.WithTimeout(ctx, 1*time.Second) - defer cancel() + // Verification using verifyCrawlerPeers with a custom eval function + verifyCrawlerPeers(t, crawler, service, map[uint64]int{1: 1}, 1, "only node2 should remain", func(t *testing.T, result []*enode.Node) { + require.Equal(t, localNode2.Node().ID(), result[0].ID()) + }) +} - builder := func(idx uint64) string { - return fmt.Sprintf(AttestationSubnetTopicFormat, digest, idx) + service.Encoding().ProtocolSuffix() +func createTestService(t *testing.T, d db.Database) *Service { + fakePeer := testp2p.NewTestP2P(t) + s := &Service{ + cfg: &Config{ + MaxPeers: 30, + DB: d, + }, + genesisTime: time.Now(), + genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32), + peers: peers.NewStatus(t.Context(), &peers.StatusConfig{ + PeerLimit: 30, + ScorerParams: &scorers.Config{}, + }), + host: fakePeer.BHost, } - result, err := service.findPeersWithSubnets( - ctxWithTimeout, - builder, - 1, - map[uint64]int{1: 2}, // Need 2 peers for subnet 1 - ) + return s +} + +func startTestCrawler(t *testing.T, s *Service, listener *testp2p.MockListener) *GossipsubPeerCrawler { + digest, err := s.currentForkDigest() + require.NoError(t, err) + crawler, err := NewGossipsubPeerCrawler(s, listener, + 1*time.Second, 100*time.Millisecond, 10, gossipsubcrawler.PeerFilterFunc(s.filterPeer), + s.Peers().Scorers().Score) + require.NoError(t, err) + s.crawler = crawler + require.NoError(t, crawler.Start(func(ctx context.Context, n *enode.Node) ([]string, error) { + subs, err := attestationSubnets(n.Record()) + if err != nil { + return nil, err + } + var topics []string + for subnet := range subs { + t := fmt.Sprintf(AttestationSubnetTopicFormat, digest, subnet) + s.Encoding().ProtocolSuffix() + topics = append(topics, t) + } + return topics, nil + })) + return crawler +} +func verifyCrawlerPeers(t *testing.T, crawler *GossipsubPeerCrawler, s *Service, subnets map[uint64]int, expectedCount int, description string, eval func(t *testing.T, result []*enode.Node)) { + digest, err := s.currentForkDigest() require.NoError(t, err) - require.Equal(t, 1, len(result)) - require.Equal(t, localNode2.Node().ID(), result[0].ID()) // only node2 should remain + var topics []string + for subnet := range subnets { + topics = append(topics, fmt.Sprintf(AttestationSubnetTopicFormat, digest, subnet)+s.Encoding().ProtocolSuffix()) + } + + var results []*enode.Node + require2.Eventually(t, func() bool { + results = results[:0] + seen := make(map[enode.ID]struct{}) + for _, topic := range topics { + peers := crawler.PeersForTopic(gossipsubcrawler.Topic(topic)) + for _, peer := range peers { + if _, ok := seen[peer.ID()]; !ok { + seen[peer.ID()] = struct{}{} + results = append(results, peer) + } + } + } + return len(results) == expectedCount + }, 1*time.Second, 100*time.Millisecond, description) + + if eval != nil { + eval(t, results) + } } diff --git a/beacon-chain/sync/BUILD.bazel b/beacon-chain/sync/BUILD.bazel index 48889e32a1d4..9cb5c0837cef 100644 --- a/beacon-chain/sync/BUILD.bazel +++ b/beacon-chain/sync/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "error.go", "fork_watcher.go", "fuzz_exports.go", # keep + "gossipsub_base.go", "gossipsub_controller.go", "gossipsub_topic_family.go", "log.go", diff --git a/beacon-chain/sync/fork_watcher_test.go b/beacon-chain/sync/fork_watcher_test.go index abbec8af070b..d0dd8af628c0 100644 --- a/beacon-chain/sync/fork_watcher_test.go +++ b/beacon-chain/sync/fork_watcher_test.go @@ -42,7 +42,6 @@ func testForkWatcherService(t *testing.T, current primitives.Epoch) *Service { initialSync: &mockSync.Sync{IsSyncing: false}, }, chainStarted: abool.New(), - subHandler: newSubTopicHandler(), initialSyncComplete: closedChan, } return r diff --git a/beacon-chain/sync/gossipsub_base.go b/beacon-chain/sync/gossipsub_base.go new file mode 100644 index 000000000000..08c3c627e787 --- /dev/null +++ b/beacon-chain/sync/gossipsub_base.go @@ -0,0 +1,187 @@ +package sync + +import ( + "context" + "fmt" + "runtime/debug" + "sync" + + "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/gossipsubcrawler" + "github.com/OffchainLabs/prysm/v6/config/params" + "github.com/OffchainLabs/prysm/v6/monitoring/tracing" + "github.com/OffchainLabs/prysm/v6/monitoring/tracing/trace" + pubsub "github.com/libp2p/go-libp2p-pubsub" + "github.com/pkg/errors" + "google.golang.org/protobuf/proto" +) + +type baseGossipsubTopicFamily struct { + syncService *Service + nse params.NetworkScheduleEntry + validator wrappedVal + handler subHandler + + tf GossipsubTopicFamily + + mu sync.Mutex + subscriptions map[string]*pubsub.Subscription +} + +func newBaseGossipsubTopicFamily(syncService *Service, nse params.NetworkScheduleEntry, validator wrappedVal, + handler subHandler, tf GossipsubTopicFamily) *baseGossipsubTopicFamily { + return &baseGossipsubTopicFamily{ + syncService: syncService, + nse: nse, + validator: validator, + handler: handler, + tf: tf, + subscriptions: make(map[string]*pubsub.Subscription), + } +} + +func (b *baseGossipsubTopicFamily) NetworkScheduleEntry() params.NetworkScheduleEntry { + return b.nse +} + +// idempotent for a topic +func (b *baseGossipsubTopicFamily) subscribeToTopics(topics []string) { + b.mu.Lock() + defer b.mu.Unlock() + + for _, topic := range topics { + log := log.WithField("topic", topic) + s := b.syncService + + // Do not resubscribe to topics that we already have a subscription for. + _, ok := b.subscriptions[topic] + if ok { + continue + } + + if err := s.cfg.p2p.PubSub().RegisterTopicValidator(s.wrapAndReportValidation(topic, b.validator)); err != nil { + log.WithError(err).Error("Could not register validator for topic") + continue + } + + 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 + // subscription filter. + log.WithError(err).Error("Could not subscribe topic") + continue + } + + // Pipeline decodes the incoming subscription data, runs the validation, and handles the + // message. + pipeline := func(msg *pubsub.Message) { + ctx, cancel := context.WithTimeout(s.ctx, pubsubMessageTimeout) + defer cancel() + + ctx, span := trace.StartSpan(ctx, "sync.pubsub") + defer span.End() + + defer func() { + if r := recover(); r != nil { + tracing.AnnotateError(span, fmt.Errorf("panic occurred: %v", r)) + log.WithField("error", r). + WithField("recoveredAt", "subscribeWithBase"). + WithField("stack", string(debug.Stack())). + Error("Panic occurred") + } + }() + + span.SetAttributes(trace.StringAttribute("topic", topic)) + + if msg.ValidatorData == nil { + log.Error("Received nil message on pubsub") + messageFailedProcessingCounter.WithLabelValues(topic).Inc() + return + } + + if err := b.handler(ctx, msg.ValidatorData.(proto.Message)); err != nil { + tracing.AnnotateError(span, err) + log.WithError(err).Error("Could not handle p2p pubsub") + messageFailedProcessingCounter.WithLabelValues(topic).Inc() + return + } + } + + // The main message loop for receiving incoming messages from this subscription. + messageLoop := func() { + for { + msg, err := sub.Next(s.ctx) + if err != nil { + // This should only happen when the context is cancelled or subscription is cancelled. + if !errors.Is(err, pubsub.ErrSubscriptionCancelled) { // Only log a warning on unexpected errors. + log.WithError(err).Warn("Subscription next failed") + } + // Cancel subscription in the event of an error, as we are + // now exiting topic event loop. + sub.Cancel() + return + } + + if msg.ReceivedFrom == s.cfg.p2p.PeerID() { + continue + } + + go pipeline(msg) + } + } + + go messageLoop() + log.WithField("topic", topic).Info("Subscribed to") + b.subscriptions[topic] = sub + s.subHandler.addTopic(topic, sub) + } +} + +func (b *baseGossipsubTopicFamily) unsubscribeAll() { + b.mu.Lock() + defer b.mu.Unlock() + + for topic, sub := range b.subscriptions { + b.cleanupSubscription(topic, sub) + } + + b.subscriptions = make(map[string]*pubsub.Subscription) +} + +func (b *baseGossipsubTopicFamily) removeUnwantedTopics(wantedTopics []string) { + b.mu.Lock() + defer b.mu.Unlock() + + topics := wantedTopics + neededMap := make(map[string]bool, len(topics)) + for _, t := range topics { + neededMap[t] = true + } + + for topic, sub := range b.subscriptions { + if !neededMap[topic] { + b.cleanupSubscription(topic, sub) + } + } +} + +func (b *baseGossipsubTopicFamily) cleanupSubscription(topic string, sub *pubsub.Subscription) { + s := b.syncService + log.WithField("topic", topic).Info("Unsubscribed from") + if err := s.cfg.p2p.PubSub().UnregisterTopicValidator(topic); err != nil { + log.WithError(err).Error("Could not unregister topic validator") + } + + if sub != nil { + sub.Cancel() + } + if err := s.cfg.p2p.LeaveTopic(topic); err != nil { + log.WithError(err).Error("Unable to leave topic") + } + + if crawler := s.cfg.p2p.Crawler(); crawler != nil { + crawler.RemoveTopic(gossipsubcrawler.Topic(topic)) + } + delete(b.subscriptions, topic) + s.subHandler.removeTopic(topic) +} diff --git a/beacon-chain/sync/gossipsub_controller.go b/beacon-chain/sync/gossipsub_controller.go index f8a94757cd51..8072d791eb77 100644 --- a/beacon-chain/sync/gossipsub_controller.go +++ b/beacon-chain/sync/gossipsub_controller.go @@ -19,7 +19,7 @@ type topicFamilyKey struct { } func topicFamilyKeyFrom(tf GossipsubTopicFamily) topicFamilyKey { - return topicFamilyKey{topicName: fmt.Sprintf("%s", tf.Name()), forkDigest: tf.NetworkScheduleEntry().ForkDigest} + return topicFamilyKey{topicName: tf.Name(), forkDigest: tf.NetworkScheduleEntry().ForkDigest} } type GossipsubController struct { @@ -73,6 +73,7 @@ func (g *GossipsubController) controlLoop() { } func (g *GossipsubController) updateActiveTopicFamilies(currentEpoch primitives.Epoch) { + slot := g.syncService.cfg.clock.CurrentSlot() currentNSE := params.GetNetworkScheduleEntry(currentEpoch) families := TopicFamiliesForEpoch(currentEpoch, g.syncService, currentNSE) @@ -86,18 +87,26 @@ func (g *GossipsubController) updateActiveTopicFamilies(currentEpoch primitives. // register topic families for the current NSE -> this is idempotent for _, family := range families { key := topicFamilyKeyFrom(family) - if _, ok := g.activeTopicFamilies[key]; ok { - continue + existing, seen := g.activeTopicFamilies[key] + if !seen { + g.activeTopicFamilies[key] = family + existing = family + log.WithFields(logrus.Fields{ + "topicName": key.topicName, + "forkDigest": fmt.Sprintf("%#x", key.forkDigest), + "epoch": currentEpoch, + }).Info("Registered topic family") } - g.activeTopicFamilies[key] = family - - family.Subscribe() - log.WithFields(logrus.Fields{ - "topicName": key.topicName, - "forkDigest": fmt.Sprintf("%#x", key.forkDigest), - "epoch": currentEpoch, - }).Info("Registered topic family") + switch tf := existing.(type) { + case GossipsubTopicFamilyWithDynamicSubnets: + tf.UnsubscribeForSlot(slot) + tf.SubscribeForSlot(slot) + case GossipsubTopicFamilyWithoutDynamicSubnets: + if !seen { + tf.Subscribe() + } + } } // remove topic families for the previous NSE -> this is idempotent @@ -105,9 +114,9 @@ func (g *GossipsubController) updateActiveTopicFamilies(currentEpoch primitives. for key, family := range g.activeTopicFamilies { if key.forkDigest == previous.ForkDigest { - family.Unsubscribe() - + family.UnsubscribeAll() delete(g.activeTopicFamilies, key) + log.WithFields(logrus.Fields{ "topicName": key.topicName, "forkDigest": fmt.Sprintf("%#x", key.forkDigest), @@ -120,52 +129,50 @@ func (g *GossipsubController) updateActiveTopicFamilies(currentEpoch primitives. func (g *GossipsubController) Stop() { g.cancel() g.wg.Wait() + + g.mu.Lock() + defer g.mu.Unlock() + + for _, family := range g.activeTopicFamilies { + family.UnsubscribeAll() + } } -func (g *GossipsubController) GetCurrentSubnetTopics(slot primitives.Slot) []string { +func (g *GossipsubController) GetCurrentActiveTopics() []string { g.mu.RLock() + defer g.mu.RUnlock() + + slot := g.syncService.cfg.clock.CurrentSlot() var topics []string for _, f := range g.activeTopicFamilies { - if tfm, ok := f.(GossipsubTopicFamilyWithDynamicSubnets); ok { - bsubnets := tfm.GetSubnetsForBroadcast(slot) - for subnet := range bsubnets { - topics = append(topics, tfm.GetFullTopicString(subnet)) - } - jsubnets := tfm.GetSubnetsToJoin(slot) - for subnet := range jsubnets { - topics = append(topics, tfm.GetFullTopicString(subnet)) - } + tfm, ok := f.(GossipsubTopicFamilyWithDynamicSubnets) + if !ok { + continue } + topics = append(topics, tfm.TopicsToSubscribeForSlot(slot)...) } - g.mu.RUnlock() return topics } -func (g *GossipsubController) ExtractTopics(ctx context.Context, node *enode.Node) ([]string, error) { - if err := ctx.Err(); err != nil { - return nil, err - } +func (g *GossipsubController) ExtractTopics(_ context.Context, node *enode.Node) ([]string, error) { if node == nil { return nil, errors.New("enode is nil") } g.mu.RLock() + defer g.mu.RUnlock() + families := make([]GossipsubTopicFamilyWithDynamicSubnets, 0, len(g.activeTopicFamilies)) for _, f := range g.activeTopicFamilies { if tfm, ok := f.(GossipsubTopicFamilyWithDynamicSubnets); ok { families = append(families, tfm) } } - g.mu.RUnlock() // Collect topics from dynamic families only, de-duplicated. topicSet := make(map[string]struct{}) for _, df := range families { - if err := ctx.Err(); err != nil { - return nil, err - } - - topics, err := df.GetTopicsForNode(node) + topics, err := df.ExtractTopicsForNode(node) if err != nil { log.WithError(err).WithFields(logrus.Fields{ "topicFamily": fmt.Sprintf("%T", df), @@ -177,7 +184,6 @@ func (g *GossipsubController) ExtractTopics(ctx context.Context, node *enode.Nod } } - // Flatten set to slice with stable but unspecified order. out := make([]string, 0, len(topicSet)) for t := range topicSet { out = append(out, t) @@ -193,7 +199,6 @@ func isNextEpochForkBoundary(currentEpoch primitives.Epoch) (bool, params.Networ } return true, next // there is a fork in the next epoch } - func isOneEpochBeyondForkBoundary(currentEpoch primitives.Epoch) (bool, params.NetworkScheduleEntry) { current := params.GetNetworkScheduleEntry(currentEpoch) previous := params.GetNetworkScheduleEntry(current.Epoch - 1) diff --git a/beacon-chain/sync/gossipsub_controller_test.go b/beacon-chain/sync/gossipsub_controller_test.go index 1394a94b915e..e6161beea3f0 100644 --- a/beacon-chain/sync/gossipsub_controller_test.go +++ b/beacon-chain/sync/gossipsub_controller_test.go @@ -39,53 +39,52 @@ func (f *fakeDynFamily) Handler() subHandler { return noopHandler } -func (f *fakeDynFamily) Subscribe() { - -} - -func (f *fakeDynFamily) Unsubscribe() { - +func (f *fakeDynFamily) UnsubscribeAll() { + f.unsubscribeAll() } func (f *fakeDynFamily) GetFullTopicString(subnet uint64) string { return fmt.Sprintf("topic-%d", subnet) } -func (f *fakeDynFamily) GetSubnetsToJoin(_ primitives.Slot) map[uint64]bool { - return nil +func (f *fakeDynFamily) TopicsToSubscribeForSlot(_ primitives.Slot) []string { + return f.topics } -func (f *fakeDynFamily) GetSubnetsForBroadcast(_ primitives.Slot) map[uint64]bool { - return nil +func (f *fakeDynFamily) ExtractTopicsForNode(_ *enode.Node) ([]string, error) { + return append([]string{}, f.topics...), nil } -func (f *fakeDynFamily) GetTopicsForNode(_ *enode.Node) ([]string, error) { - return append([]string{}, f.topics...), nil +func (f *fakeDynFamily) SubscribeForSlot(_ primitives.Slot) { + f.baseGossipsubTopicFamily.subscribeToTopics(f.topics) } -type fakeStaticFamily struct { - baseGossipsubTopicFamily - name string +func (f *fakeDynFamily) UnsubscribeForSlot(_ primitives.Slot) {} + +type staticTopicFamily struct { + *baseGossipsubTopicFamily + name string + topics []string } -func (f *fakeStaticFamily) Name() string { +func (f *staticTopicFamily) Name() string { return f.name } -func (f *fakeStaticFamily) Validator() wrappedVal { - return nil +func (f *staticTopicFamily) Validator() wrappedVal { + return f.validator } -func (f *fakeStaticFamily) Handler() subHandler { - return noopHandler +func (f *staticTopicFamily) Handler() subHandler { + return f.handler } -func (f *fakeStaticFamily) Subscribe() { - +func (f *staticTopicFamily) Subscribe() { + f.baseGossipsubTopicFamily.subscribeToTopics(f.topics) } -func (f *fakeStaticFamily) Unsubscribe() { - +func (f *staticTopicFamily) UnsubscribeAll() { + f.baseGossipsubTopicFamily.unsubscribeAll() } func testGossipsubControllerService(t *testing.T, current primitives.Epoch) *Service { @@ -286,7 +285,7 @@ func TestGossipsubController_ExtractTopics(t *testing.T) { name: "static family ignored", setup: func(g *GossipsubController) { g.mu.Lock() - g.activeTopicFamilies[topicFamilyKey{topicName: "static", forkDigest: [4]byte{1, 2, 3, 4}}] = &fakeStaticFamily{name: "StaticFam"} + g.activeTopicFamilies[topicFamilyKey{topicName: "static", forkDigest: [4]byte{1, 2, 3, 4}}] = &staticTopicFamily{name: "StaticFam"} g.mu.Unlock() }, ctx: func() context.Context { return context.Background() }, @@ -313,7 +312,7 @@ func TestGossipsubController_ExtractTopics(t *testing.T) { f1 := &fakeDynFamily{topics: []string{"t1", "t2"}, name: "Dyn1"} f2 := &fakeDynFamily{topics: []string{"t2", "t3"}, name: "Dyn2"} g.mu.Lock() - g.activeTopicFamilies[topicFamilyKey{topicName: "static", forkDigest: [4]byte{1, 2, 3, 4}}] = &fakeStaticFamily{name: "StaticFam"} + g.activeTopicFamilies[topicFamilyKey{topicName: "static", forkDigest: [4]byte{1, 2, 3, 4}}] = &staticTopicFamily{name: "StaticFam"} g.activeTopicFamilies[topicFamilyKey{topicName: "dyn1", forkDigest: [4]byte{0}}] = f1 g.activeTopicFamilies[topicFamilyKey{topicName: "dyn2", forkDigest: [4]byte{0}}] = f2 g.mu.Unlock() @@ -327,7 +326,7 @@ func TestGossipsubController_ExtractTopics(t *testing.T) { name: "mixed static and dynamic", setup: func(g *GossipsubController) { f1 := &fakeDynFamily{topics: []string{"a", "b"}, name: "Dyn"} - s1 := &fakeStaticFamily{name: "Static"} + s1 := &staticTopicFamily{name: "Static"} g.mu.Lock() g.activeTopicFamilies[topicFamilyKey{topicName: "dyn", forkDigest: [4]byte{9}}] = f1 g.activeTopicFamilies[topicFamilyKey{topicName: "static", forkDigest: [4]byte{9}}] = s1 @@ -338,19 +337,6 @@ func TestGossipsubController_ExtractTopics(t *testing.T) { want: []string{"a", "b"}, wantErr: false, }, - { - name: "context cancelled short-circuits", - setup: func(g *GossipsubController) { - f1 := &fakeDynFamily{topics: []string{"x"}, name: "Dyn"} - g.mu.Lock() - g.activeTopicFamilies[topicFamilyKey{topicName: "dyn", forkDigest: [4]byte{0}}] = f1 - g.mu.Unlock() - }, - ctx: func() context.Context { c, cancel := context.WithCancel(context.Background()); cancel(); return c }, - node: dummyNode, - want: nil, - wantErr: true, - }, } s := &Service{} diff --git a/beacon-chain/sync/gossipsub_topic_family.go b/beacon-chain/sync/gossipsub_topic_family.go index b6dd538814de..4b66e3ea4a04 100644 --- a/beacon-chain/sync/gossipsub_topic_family.go +++ b/beacon-chain/sync/gossipsub_topic_family.go @@ -23,36 +23,27 @@ var noopHandler subHandler = func(ctx context.Context, msg proto.Message) error return nil } -type baseGossipsubTopicFamily struct { - syncService *Service - protocolSuffix string - nse params.NetworkScheduleEntry -} - -func (b *baseGossipsubTopicFamily) NetworkScheduleEntry() params.NetworkScheduleEntry { - return b.nse -} - type GossipsubTopicFamily interface { Name() string - Validator() wrappedVal - Handler() subHandler NetworkScheduleEntry() params.NetworkScheduleEntry - Subscribe() - Unsubscribe() + UnsubscribeAll() } type GossipsubTopicFamilyWithoutDynamicSubnets interface { GossipsubTopicFamily - GetFullTopicString() string + Subscribe() } type GossipsubTopicFamilyWithDynamicSubnets interface { GossipsubTopicFamily - GetFullTopicString(subnet uint64) string - GetSubnetsToJoin(slot primitives.Slot) map[uint64]bool - GetSubnetsForBroadcast(slot primitives.Slot) map[uint64]bool - GetTopicsForNode(node *enode.Node) ([]string, error) + + TopicsToSubscribeForSlot(slot primitives.Slot) []string + + ExtractTopicsForNode(node *enode.Node) ([]string, error) + + SubscribeForSlot(slot primitives.Slot) + + UnsubscribeForSlot(slot primitives.Slot) } type topicFamilyEntry struct { diff --git a/beacon-chain/sync/service.go b/beacon-chain/sync/service.go index db11d58d5cee..25e7cce33c6a 100644 --- a/beacon-chain/sync/service.go +++ b/beacon-chain/sync/service.go @@ -233,6 +233,7 @@ func NewService(ctx context.Context, opts ...Option) *Service { delete(r.seenPendingBlocks, root) } }) + r.subHandler = newSubTopicHandler() r.rateLimiter = newRateLimiter(r.cfg.p2p) r.initCaches() @@ -324,9 +325,6 @@ func (s *Service) Stop() error { for _, p := range s.cfg.p2p.Host().Mux().Protocols() { s.cfg.p2p.Host().RemoveStreamHandler(p) } - for _, t := range s.cfg.p2p.PubSub().GetTopics() { - s.unSubscribeFromTopic(t) - } // Stop gossipsub dialer and crawler if present. if dialer := s.cfg.p2p.GossipsubDialer(); dialer != nil { @@ -440,7 +438,7 @@ func (s *Service) startDiscoveryAndSubscriptions() { // Start the gossipsub dialer if available. if dialer := s.cfg.p2p.GossipsubDialer(); dialer != nil { provider := func() []string { - return s.gossipsubController.GetCurrentSubnetTopics(s.cfg.clock.CurrentSlot()) + return s.gossipsubController.GetCurrentActiveTopics() } if err := dialer.Start(provider); err != nil { log.WithError(err).Warn("Failed to start gossipsub peer dialer") diff --git a/beacon-chain/sync/subscriber.go b/beacon-chain/sync/subscriber.go index 67e341f2c998..0a26995be990 100644 --- a/beacon-chain/sync/subscriber.go +++ b/beacon-chain/sync/subscriber.go @@ -4,9 +4,7 @@ import ( "context" "fmt" "reflect" - "runtime/debug" "strings" - "sync" "time" "github.com/OffchainLabs/prysm/v6/beacon-chain/cache" @@ -14,15 +12,12 @@ import ( "github.com/OffchainLabs/prysm/v6/beacon-chain/core/helpers" "github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p" - "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/gossipsubcrawler" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers" "github.com/OffchainLabs/prysm/v6/beacon-chain/startup" "github.com/OffchainLabs/prysm/v6/cmd/beacon-chain/flags" "github.com/OffchainLabs/prysm/v6/config/features" "github.com/OffchainLabs/prysm/v6/config/params" "github.com/OffchainLabs/prysm/v6/consensus-types/primitives" - "github.com/OffchainLabs/prysm/v6/monitoring/tracing" - "github.com/OffchainLabs/prysm/v6/monitoring/tracing/trace" ethpb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1" "github.com/OffchainLabs/prysm/v6/runtime/messagehandler" "github.com/OffchainLabs/prysm/v6/time/slots" @@ -32,87 +27,12 @@ import ( "github.com/libp2p/go-libp2p/core/peer" "github.com/pkg/errors" "github.com/sirupsen/logrus" - "google.golang.org/protobuf/proto" ) const pubsubMessageTimeout = 30 * time.Second var errInvalidDigest = errors.New("invalid digest") -func familyLogFields(tf GossipsubTopicFamilyWithDynamicSubnets) logrus.Fields { - nse := tf.NetworkScheduleEntry() - return logrus.Fields{ - "topicFamily": fmt.Sprintf("%T", tf), - "digest": nse.ForkDigest, - "forkEpoch": nse.Epoch, - } -} - -// subnetTracker keeps track of which subnets we are subscribed to for a given -// dynamic topic family (attestations, sync-committee, data-column, etc.). -type subnetTracker struct { - family GossipsubTopicFamilyWithDynamicSubnets - mu sync.RWMutex - subscriptions map[uint64]*pubsub.Subscription -} - -func newSubnetTracker(tf GossipsubTopicFamilyWithDynamicSubnets) *subnetTracker { - return &subnetTracker{ - family: tf, - subscriptions: make(map[uint64]*pubsub.Subscription), - } -} - -// unwanted takes a list of wanted subnets and returns a list of currently subscribed subnets that are not included. -func (t *subnetTracker) unwanted(wanted map[uint64]bool) []uint64 { - t.mu.RLock() - defer t.mu.RUnlock() - unwanted := make([]uint64, 0, len(t.subscriptions)) - for subnet := range t.subscriptions { - if wanted == nil || !wanted[subnet] { - unwanted = append(unwanted, subnet) - } - } - return unwanted -} - -// missing takes a list of wanted subnets and returns a list of wanted subnets that are not currently tracked. -func (t *subnetTracker) missing(wanted map[uint64]bool) []uint64 { - t.mu.RLock() - defer t.mu.RUnlock() - missing := make([]uint64, 0, len(wanted)) - for subnet := range wanted { - if _, ok := t.subscriptions[subnet]; !ok { - missing = append(missing, subnet) - } - } - return missing -} - -// cancelSubscription cancels and removes the subscription for a given subnet. -func (t *subnetTracker) cancelSubscription(subnet uint64) { - t.mu.Lock() - defer t.mu.Unlock() - defer delete(t.subscriptions, subnet) - - sub := t.subscriptions[subnet] - if sub == nil { - return - } - sub.Cancel() -} - -// track asks subscriptionTracker to hold on to the subscription for a given subnet so -// that we can remember that it is tracked and cancel its context when it's time to unsubscribe. -func (t *subnetTracker) track(subnet uint64, sub *pubsub.Subscription) { - if sub == nil { - return - } - t.mu.Lock() - defer t.mu.Unlock() - t.subscriptions[subnet] = sub -} - // noopValidator is a no-op that only decodes the message, but does not check its contents. func (s *Service) noopValidator(_ context.Context, _ peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) { m, err := s.decodePubsubMessage(msg) @@ -156,100 +76,6 @@ func (s *Service) activeSyncSubnetIndices(currentSlot primitives.Slot) map[uint6 return mapFromSlice(subscriptions) } -func (s *Service) subscriptionRequestExpired(nse params.NetworkScheduleEntry) bool { - next := params.NextNetworkScheduleEntry(nse.Epoch) - return next.Epoch != nse.Epoch && s.cfg.clock.CurrentEpoch() > next.Epoch -} - -func (s *Service) subscribe(topic string, validator wrappedVal, handle subHandler) *pubsub.Subscription { - log := log.WithField("topic", topic) - - // Do not resubscribe already seen subscriptions. - ok := s.subHandler.topicExists(topic) - if ok { - log.WithField("topic", topic).Error("Provided topic already has an active subscription running") - return 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) - 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 - // subscription filter. - log.WithError(err).Error("Could not subscribe topic") - return nil - } - - s.subHandler.addTopic(sub.Topic(), sub) - - // Pipeline decodes the incoming subscription data, runs the validation, and handles the - // message. - pipeline := func(msg *pubsub.Message) { - ctx, cancel := context.WithTimeout(s.ctx, pubsubMessageTimeout) - defer cancel() - - ctx, span := trace.StartSpan(ctx, "sync.pubsub") - defer span.End() - - defer func() { - if r := recover(); r != nil { - tracing.AnnotateError(span, fmt.Errorf("panic occurred: %v", r)) - log.WithField("error", r). - WithField("recoveredAt", "subscribeWithBase"). - WithField("stack", string(debug.Stack())). - Error("Panic occurred") - } - }() - - span.SetAttributes(trace.StringAttribute("topic", topic)) - - if msg.ValidatorData == nil { - log.Error("Received nil message on pubsub") - messageFailedProcessingCounter.WithLabelValues(topic).Inc() - return - } - - if err := handle(ctx, msg.ValidatorData.(proto.Message)); err != nil { - tracing.AnnotateError(span, err) - log.WithError(err).Error("Could not handle p2p pubsub") - messageFailedProcessingCounter.WithLabelValues(topic).Inc() - return - } - } - - // The main message loop for receiving incoming messages from this subscription. - messageLoop := func() { - for { - msg, err := sub.Next(s.ctx) - if err != nil { - // This should only happen when the context is cancelled or subscription is cancelled. - if !errors.Is(err, pubsub.ErrSubscriptionCancelled) { // Only log a warning on unexpected errors. - log.WithError(err).Warn("Subscription next failed") - } - // Cancel subscription in the event of an error, as we are - // now exiting topic event loop. - sub.Cancel() - return - } - - if msg.ReceivedFrom == s.cfg.p2p.PeerID() { - continue - } - - go pipeline(msg) - } - } - - go messageLoop() - log.WithField("topic", topic).Info("Subscribed to") - return sub -} - // Wrap the pubsub validator with a metric monitoring function. This function increments the // appropriate counter if the particular message fails to validate. func (s *Service) wrapAndReportValidation(topic string, v wrappedVal) (string, pubsub.ValidatorEx) { @@ -319,126 +145,6 @@ func (s *Service) wrapAndReportValidation(topic string, v wrappedVal) (string, p } } -// pruneNotWanted unsubscribes from topics we are currently subscribed to but that are -// not in the list of wanted subnets. -func (s *Service) pruneNotWanted(t *subnetTracker, wantedSubnets map[uint64]bool) { - for _, subnet := range t.unwanted(wantedSubnets) { - t.cancelSubscription(subnet) - s.unSubscribeFromTopic(t.family.GetFullTopicString(subnet)) - } -} - -// subscribeToDynamicSubnetFamily subscribes to a list of subnets. -func (s *Service) subscribeToDynamicSubnetFamily(tf GossipsubTopicFamilyWithDynamicSubnets) *subnetTracker { - tracker := newSubnetTracker(tf) - go s.subscribeToSubnets(tf, tracker) - return tracker -} - -func (s *Service) subscribeToSubnets(tf GossipsubTopicFamilyWithDynamicSubnets, tracker *subnetTracker) { - ctx, cancel := context.WithCancel(s.ctx) - defer cancel() - - go s.logMinimumPeersPerSubnet(ctx, tf) - - s.trySubscribeSubnets(tracker) - - slotTicker := slots.NewSlotTicker(s.cfg.clock.GenesisTime(), params.BeaconConfig().SecondsPerSlot) - defer slotTicker.Done() - for { - select { - case <-slotTicker.C(): - // Check if this subscribe request is still valid - we may have crossed another fork epoch while waiting for initial sync. - if s.subscriptionRequestExpired(tf.NetworkScheduleEntry()) { - // If we are already past the next fork epoch, do not subscribe to this topic. - log.WithFields(logrus.Fields{ - "topicFamily": fmt.Sprintf("%T", tf), - "digest": tf.NetworkScheduleEntry().ForkDigest, - "epoch": tf.NetworkScheduleEntry().Epoch, - "currentEpoch": s.cfg.clock.CurrentEpoch(), - }).Debug("Exiting topic subnet subscription loop") - return - } - s.trySubscribeSubnets(tracker) - case <-s.ctx.Done(): - return - } - } -} - -// trySubscribeSubnets attempts to subscribe to any missing subnets that we should be subscribed to. -// Only if initial sync is complete. -func (s *Service) trySubscribeSubnets(t *subnetTracker) { - subnetsToJoin := t.family.GetSubnetsToJoin(s.cfg.clock.CurrentSlot()) - s.pruneNotWanted(t, subnetsToJoin) - for _, subnet := range t.missing(subnetsToJoin) { - topic := t.family.GetFullTopicString(subnet) - t.track(subnet, s.subscribe(topic, t.family.Validator(), t.family.Handler())) - } -} - -func (s *Service) logMinimumPeersPerSubnet(ctx context.Context, tf GossipsubTopicFamilyWithDynamicSubnets) { - logFields := familyLogFields(tf) - minimumPeersPerSubnet := flags.Get().MinimumPeersPerSubnet - // Warn the user if we are not subscribed to enough peers in the subnets. - log := log.WithField("minimum", minimumPeersPerSubnet) - logTicker := time.NewTicker(5 * time.Minute) - defer logTicker.Stop() - - for { - select { - case <-logTicker.C: - currentSlot := s.cfg.clock.CurrentSlot() - subnetsToFindPeersIndex := computeAllNeededSubnets(currentSlot, tf) - - isSubnetWithMissingPeers := false - // Find new peers for wanted subnets if needed. - for index := range subnetsToFindPeersIndex { - topic := tf.GetFullTopicString(index) - - // Check if we have enough peers in the subnet. Skip if we do. - if count := s.connectedPeersCount(topic); count < minimumPeersPerSubnet { - isSubnetWithMissingPeers = true - log.WithFields(logrus.Fields{ - "topic": topic, - "actual": count, - }).Warning("Not enough connected peers") - } - } - if !isSubnetWithMissingPeers { - log.WithFields(logFields).Debug("All subnets have enough connected peers") - } - case <-ctx.Done(): - return - } - } -} - -func (s *Service) unSubscribeFromTopic(topic string) { - log.WithField("topic", topic).Info("Unsubscribed from") - if err := s.cfg.p2p.PubSub().UnregisterTopicValidator(topic); err != nil { - log.WithError(err).Error("Could not unregister topic validator") - } - sub := s.subHandler.subForTopic(topic) - if sub != nil { - sub.Cancel() - } - s.subHandler.removeTopic(topic) - if err := s.cfg.p2p.LeaveTopic(topic); err != nil { - log.WithError(err).Error("Unable to leave topic") - } - - if crawler := s.cfg.p2p.Crawler(); crawler != nil { - crawler.RemoveTopic(gossipsubcrawler.Topic(topic)) - } -} - -// connectedPeersCount counts how many peer for a given topic are connected to the node. -func (s *Service) connectedPeersCount(fullTopic string) int { - peersWithSubnet := s.cfg.p2p.PubSub().ListPeers(fullTopic) - return len(peersWithSubnet) -} - func (s *Service) dataColumnSubnetIndices(primitives.Slot) map[uint64]bool { nodeID := s.cfg.p2p.NodeID() @@ -581,30 +287,6 @@ func isDigestValid(digest [4]byte, clock *startup.Clock) (bool, error) { return params.ForkDigest(current) == digest, nil } -// computeAllNeededSubnets computes the subnets we want to join -// and the subnets for which we want to find peers. -func computeAllNeededSubnets( - currentSlot primitives.Slot, - dtf GossipsubTopicFamilyWithDynamicSubnets, -) map[uint64]bool { - // Retrieve the subnets we want to join. - subnetsToJoin := dtf.GetSubnetsToJoin(currentSlot) - - // Retrieve the subnets we want to find peers into. - subnetsRequiringPeers := dtf.GetSubnetsForBroadcast(currentSlot) - - // Combine the two maps to get all needed subnets. - neededSubnets := make(map[uint64]bool, len(subnetsToJoin)+len(subnetsRequiringPeers)) - for subnet := range subnetsToJoin { - neededSubnets[subnet] = true - } - for subnet := range subnetsRequiringPeers { - neededSubnets[subnet] = true - } - - return neededSubnets -} - func agentString(pid peer.ID, hst host.Host) string { rawVersion, storeErr := hst.Peerstore().Get(pid, "AgentVersion") agString, ok := rawVersion.(string) diff --git a/beacon-chain/sync/subscriber_test.go b/beacon-chain/sync/subscriber_test.go index cda0532dc91e..31688997b714 100644 --- a/beacon-chain/sync/subscriber_test.go +++ b/beacon-chain/sync/subscriber_test.go @@ -3,7 +3,6 @@ package sync import ( "context" "fmt" - "reflect" "sync" "testing" "time" @@ -30,123 +29,12 @@ import ( "github.com/OffchainLabs/prysm/v6/testing/require" "github.com/OffchainLabs/prysm/v6/testing/util" "github.com/OffchainLabs/prysm/v6/time/slots" - "github.com/ethereum/go-ethereum/p2p/enode" pubsub "github.com/libp2p/go-libp2p-pubsub" pubsubpb "github.com/libp2p/go-libp2p-pubsub/pb" "github.com/libp2p/go-libp2p/core/peer" - logTest "github.com/sirupsen/logrus/hooks/test" "google.golang.org/protobuf/proto" ) -// testStaticFamily implements a minimal static topic family for tests. -type testStaticFamily struct { - nse params.NetworkScheduleEntry - topicFmt string - protocolSuffix string - validator wrappedVal - handler subHandler -} - -func (t testStaticFamily) Validator() wrappedVal { - return t.validator -} - -func (t testStaticFamily) Handler() subHandler { - return t.handler -} - -func (t testStaticFamily) NetworkScheduleEntry() params.NetworkScheduleEntry { - return t.nse -} - -func (t testStaticFamily) GetFullTopicString() string { - return fmt.Sprintf(t.topicFmt, t.nse.ForkDigest) + t.protocolSuffix -} - -func (t testStaticFamily) Subscribe() {} - -func (t testStaticFamily) Unsubscribe() {} - -func makeTestFamily(nse params.NetworkScheduleEntry, topicFmt, suffix string, validator wrappedVal, handler subHandler) testStaticFamily { - return testStaticFamily{ - nse: nse, - topicFmt: topicFmt, - protocolSuffix: suffix, - validator: validator, - handler: handler, - } -} - -func makeFullTopic(topicFmt string, nse params.NetworkScheduleEntry, suffix string) string { - return fmt.Sprintf(topicFmt, nse.ForkDigest) + suffix -} - -// testDynamicFamily implements a minimal dynamic topic family for tests. -type testDynamicFamily struct { - nse params.NetworkScheduleEntry - topicFmt string - protocolSuffix string - validator wrappedVal - handler subHandler - subnetsToJoin func(primitives.Slot) map[uint64]bool - subnetsForCast func(primitives.Slot) map[uint64]bool -} - -func (t *testDynamicFamily) Name() string { - return "TestDynamicFamily" -} - -func (t *testDynamicFamily) Validator() wrappedVal { - return t.validator -} - -func (t *testDynamicFamily) Handler() subHandler { - return t.handler -} - -func (t *testDynamicFamily) NetworkScheduleEntry() params.NetworkScheduleEntry { - return t.nse -} - -func (t *testDynamicFamily) GetFullTopicString(subnet uint64) string { - return fmt.Sprintf(t.topicFmt, t.nse.ForkDigest, subnet) + t.protocolSuffix -} - -func (t *testDynamicFamily) GetSubnetsToJoin(slot primitives.Slot) map[uint64]bool { - if t.subnetsToJoin != nil { - return t.subnetsToJoin(slot) - } - return nil -} - -func (t *testDynamicFamily) GetSubnetsForBroadcast(slot primitives.Slot) map[uint64]bool { - if t.subnetsForCast != nil { - return t.subnetsForCast(slot) - } - return nil -} - -func (t *testDynamicFamily) Subscribe() {} - -func (t *testDynamicFamily) Unsubscribe() {} - -func (t *testDynamicFamily) GetTopicsForNode(_ *enode.Node) ([]string, error) { - return nil, nil -} - -func makeTestDynamicFamily(nse params.NetworkScheduleEntry, topicFmt, suffix string, validator wrappedVal, handler subHandler, - getJoin func(primitives.Slot) map[uint64]bool, getCast func(primitives.Slot) map[uint64]bool) *testDynamicFamily { - return &testDynamicFamily{ - nse: nse, - topicFmt: topicFmt, - protocolSuffix: suffix, - validator: validator, - handler: handler, - subnetsToJoin: getJoin, - subnetsForCast: getCast, - } -} - func TestSubscribe_ReceivesValidMessage(t *testing.T) { p2pService := p2ptest.NewTestP2P(t) gt := time.Now() @@ -170,11 +58,10 @@ func TestSubscribe_ReceivesValidMessage(t *testing.T) { require.NoError(t, err) nse := params.GetNetworkScheduleEntry(r.cfg.clock.CurrentEpoch()) p2pService.Digest = nse.ForkDigest - topic := "/eth2/%x/voluntary_exit" var wg sync.WaitGroup wg.Add(1) - tf := makeTestFamily(nse, topic, r.cfg.p2p.Encoding().ProtocolSuffix(), r.noopValidator, func(_ context.Context, msg proto.Message) error { + handler := func(_ context.Context, msg proto.Message) error { m, ok := msg.(*pb.SignedVoluntaryExit) assert.Equal(t, true, ok, "Object is not of type *pb.SignedVoluntaryExit") if m.Exit == nil || m.Exit.Epoch != 55 { @@ -182,10 +69,15 @@ func TestSubscribe_ReceivesValidMessage(t *testing.T) { } wg.Done() return nil - }) - r.subscribe(tf.GetFullTopicString(), tf.Validator(), tf.Handler()) + } + + tf := NewVoluntaryExitTopicFamily(&r, nse) + base := newBaseGossipsubTopicFamily(&r, nse, r.noopValidator, handler, tf) + tf.baseGossipsubTopicFamily = base + + tf.Subscribe() r.markForChainStart() - p2pService.ReceivePubSub(tf.GetFullTopicString(), &pb.SignedVoluntaryExit{Exit: &pb.VoluntaryExit{Epoch: 55}, Signature: make([]byte, fieldparams.BLSSignatureLength)}) + p2pService.ReceivePubSub(tf.getFullTopicString(), &pb.SignedVoluntaryExit{Exit: &pb.VoluntaryExit{Epoch: 55}, Signature: make([]byte, fieldparams.BLSSignatureLength)}) if util.WaitTimeout(&wg, time.Second) { t.Fatal("Did not receive PubSub in 1 second") @@ -220,17 +112,22 @@ func TestSubscribe_UnsubscribeTopic(t *testing.T) { p2pService.Digest = nse.ForkDigest topic := "/eth2/%x/voluntary_exit" - tf := makeTestFamily(nse, topic, r.cfg.p2p.Encoding().ProtocolSuffix(), r.noopValidator, func(_ context.Context, msg proto.Message) error { return nil }) - r.subscribe(tf.GetFullTopicString(), tf.Validator(), tf.Handler()) + tf := staticTopicFamily{ + name: "VoluntaryExitTopicFamily", + topics: []string{topic}, + } + base := newBaseGossipsubTopicFamily(&r, nse, r.noopValidator, noopHandler, &tf) + tf.baseGossipsubTopicFamily = base + + tf.Subscribe() r.markForChainStart() - fullTopic := tf.GetFullTopicString() - assert.Equal(t, true, r.subHandler.topicExists(fullTopic)) + assert.Equal(t, true, r.subHandler.topicExists(topic)) topics := p2pService.PubSub().GetTopics() - assert.Equal(t, fullTopic, topics[0]) + assert.Equal(t, topic, topics[0]) - r.unSubscribeFromTopic(fullTopic) + tf.UnsubscribeAll() - assert.Equal(t, false, r.subHandler.topicExists(fullTopic)) + assert.Equal(t, false, r.subHandler.topicExists(topic)) assert.Equal(t, 0, len(p2pService.PubSub().GetTopics())) } @@ -265,17 +162,20 @@ func TestSubscribe_ReceivesAttesterSlashing(t *testing.T) { subHandler: newSubTopicHandler(), } markInitSyncComplete(t, &r) - topic := "/eth2/%x/attester_slashing" var wg sync.WaitGroup wg.Add(1) nse := params.GetNetworkScheduleEntry(r.cfg.clock.CurrentEpoch()) p2pService.Digest = nse.ForkDigest - tf := makeTestFamily(nse, topic, r.cfg.p2p.Encoding().ProtocolSuffix(), r.noopValidator, func(ctx context.Context, msg proto.Message) error { + + tf := NewAttesterSlashingTopicFamily(&r, nse) + tf.baseGossipsubTopicFamily.validator = r.noopValidator + tf.baseGossipsubTopicFamily.handler = func(ctx context.Context, msg proto.Message) error { require.NoError(t, r.attesterSlashingSubscriber(ctx, msg)) wg.Done() return nil - }) - r.subscribe(tf.GetFullTopicString(), tf.Validator(), tf.Handler()) + } + tf.Subscribe() + beaconState, privKeys := util.DeterministicGenesisState(t, 64) chainService.State = beaconState r.markForChainStart() @@ -287,7 +187,7 @@ func TestSubscribe_ReceivesAttesterSlashing(t *testing.T) { require.NoError(t, err, "Error generating attester slashing") err = r.cfg.beaconDB.SaveState(ctx, beaconState, bytesutil.ToBytes32(attesterSlashing.FirstAttestation().GetData().BeaconBlockRoot)) require.NoError(t, err) - p2pService.ReceivePubSub(tf.GetFullTopicString(), attesterSlashing) + p2pService.ReceivePubSub(tf.getFullTopicString(), attesterSlashing) if util.WaitTimeout(&wg, time.Second) { t.Fatal("Did not receive PubSub in 1 second") @@ -319,19 +219,22 @@ func TestSubscribe_ReceivesProposerSlashing(t *testing.T) { subHandler: newSubTopicHandler(), } markInitSyncComplete(t, &r) - topic := "/eth2/%x/proposer_slashing" var wg sync.WaitGroup wg.Add(1) params.SetupTestConfigCleanup(t) params.OverrideBeaconConfig(params.MainnetConfig()) nse := params.GetNetworkScheduleEntry(r.cfg.clock.CurrentEpoch()) p2pService.Digest = nse.ForkDigest - tf := makeTestFamily(nse, topic, r.cfg.p2p.Encoding().ProtocolSuffix(), r.noopValidator, func(ctx context.Context, msg proto.Message) error { + + tf := NewProposerSlashingTopicFamily(&r, nse) + tf.baseGossipsubTopicFamily.validator = r.noopValidator + tf.baseGossipsubTopicFamily.handler = func(ctx context.Context, msg proto.Message) error { require.NoError(t, r.proposerSlashingSubscriber(ctx, msg)) wg.Done() return nil - }) - r.subscribe(tf.GetFullTopicString(), tf.Validator(), tf.Handler()) + } + tf.Subscribe() + beaconState, privKeys := util.DeterministicGenesisState(t, 64) chainService.State = beaconState r.markForChainStart() @@ -342,7 +245,7 @@ func TestSubscribe_ReceivesProposerSlashing(t *testing.T) { ) require.NoError(t, err, "Error generating proposer slashing") - p2pService.ReceivePubSub(tf.GetFullTopicString(), proposerSlashing) + p2pService.ReceivePubSub(tf.getFullTopicString(), proposerSlashing) if util.WaitTimeout(&wg, time.Second) { t.Fatal("Did not receive PubSub in 1 second") @@ -372,68 +275,27 @@ func TestSubscribe_HandlesPanic(t *testing.T) { nse := params.GetNetworkScheduleEntry(r.cfg.clock.CurrentEpoch()) p.Digest = nse.ForkDigest - topic := p2p.GossipTypeMapping[reflect.TypeOf(&pb.SignedVoluntaryExit{})] var wg sync.WaitGroup wg.Add(1) - tf := makeTestFamily(nse, topic, p.Encoding().ProtocolSuffix(), r.noopValidator, func(_ context.Context, msg proto.Message) error { + tf := NewVoluntaryExitTopicFamily(&r, nse) + handler := func(_ context.Context, msg proto.Message) error { defer wg.Done() panic("bad") - }) - r.subscribe(tf.GetFullTopicString(), tf.Validator(), tf.Handler()) + } + base := newBaseGossipsubTopicFamily(&r, nse, r.noopValidator, handler, tf) + tf.baseGossipsubTopicFamily = base + + tf.Subscribe() + r.markForChainStart() - p.ReceivePubSub(tf.GetFullTopicString(), &pb.SignedVoluntaryExit{Exit: &pb.VoluntaryExit{Epoch: 55}, Signature: make([]byte, fieldparams.BLSSignatureLength)}) + p.ReceivePubSub(tf.getFullTopicString(), &pb.SignedVoluntaryExit{Exit: &pb.VoluntaryExit{Epoch: 55}, Signature: make([]byte, fieldparams.BLSSignatureLength)}) if util.WaitTimeout(&wg, time.Second) { t.Fatal("Did not receive PubSub in 1 second") } } -func TestRevalidateSubscription_CorrectlyFormatsTopic(t *testing.T) { - p := p2ptest.NewTestP2P(t) - hook := logTest.NewGlobal() - chain := &mockChain.ChainService{ - Genesis: time.Now(), - ValidatorsRoot: [32]byte{'A'}, - } - r := Service{ - ctx: t.Context(), - cfg: &config{ - chain: chain, - clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot), - p2p: p, - }, - chainStarted: abool.New(), - subHandler: newSubTopicHandler(), - } - nse := params.GetNetworkScheduleEntry(r.cfg.clock.CurrentEpoch()) - - tfDyn := makeTestDynamicFamily(nse, "/eth2/testing/%#x/committee%d", r.cfg.p2p.Encoding().ProtocolSuffix(), r.noopValidator, nil, nil, nil) - tracker := newSubnetTracker(tfDyn) - - // committee index 1 - c1 := uint64(1) - fullTopic := tfDyn.GetFullTopicString(c1) - _, topVal := r.wrapAndReportValidation(fullTopic, r.noopValidator) - require.NoError(t, r.cfg.p2p.PubSub().RegisterTopicValidator(fullTopic, topVal)) - sub1, err := r.cfg.p2p.SubscribeToTopic(fullTopic) - require.NoError(t, err) - tracker.track(c1, sub1) - - // committee index 2 - c2 := uint64(2) - fullTopic = tfDyn.GetFullTopicString(c2) - _, topVal = r.wrapAndReportValidation(fullTopic, r.noopValidator) - err = r.cfg.p2p.PubSub().RegisterTopicValidator(fullTopic, topVal) - require.NoError(t, err) - sub2, err := r.cfg.p2p.SubscribeToTopic(fullTopic) - require.NoError(t, err) - tracker.track(c2, sub2) - - r.pruneNotWanted(tracker, map[uint64]bool{c2: true}) - require.LogsDoNotContain(t, hook, "Could not unregister topic validator") -} - func Test_wrapAndReportValidation(t *testing.T) { mChain := &mockChain.ChainService{ Genesis: time.Now(), @@ -661,8 +523,12 @@ func TestSubscribeWithSyncSubnets_DynamicOK(t *testing.T) { currEpoch := slots.ToEpoch(slot) cache.SyncSubnetIDs.AddSyncCommitteeSubnets([]byte("pubkey"), currEpoch, []uint64{0, 1}, 10*time.Second) nse := params.GetNetworkScheduleEntry(r.cfg.clock.CurrentEpoch()) - tfDyn := makeTestDynamicFamily(nse, p2p.SyncCommitteeSubnetTopicFormat, r.cfg.p2p.Encoding().ProtocolSuffix(), r.noopValidator, nil, r.activeSyncSubnetIndices, nil) - go r.subscribeToDynamicSubnetFamily(tfDyn) + + tfDyn := NewSyncCommitteeTopicFamily(&r, nse) + base := newBaseGossipsubTopicFamily(&r, nse, r.noopValidator, noopHandler, tfDyn) + tfDyn.baseGossipsubTopicFamily = base + tfDyn.SubscribeForSlot(slot) + time.Sleep(2 * time.Second) assert.Equal(t, 2, len(r.cfg.p2p.PubSub().GetTopics())) topicMap := map[string]bool{} @@ -707,9 +573,11 @@ func TestSubscribeWithSyncSubnets_DynamicSwitchFork(t *testing.T) { require.Equal(t, [4]byte(params.BeaconConfig().DenebForkVersion), nse.ForkVersion) require.Equal(t, params.BeaconConfig().DenebForkEpoch, nse.Epoch) - tfDyn2 := makeTestDynamicFamily(nse, p2p.SyncCommitteeSubnetTopicFormat, r.cfg.p2p.Encoding().ProtocolSuffix(), r.noopValidator, nil, r.activeSyncSubnetIndices, nil) - sp := newSubnetTracker(tfDyn2) - r.trySubscribeSubnets(sp) + tfDyn2 := NewSyncCommitteeTopicFamily(&r, nse) + base := newBaseGossipsubTopicFamily(&r, nse, r.noopValidator, noopHandler, tfDyn2) + tfDyn2.baseGossipsubTopicFamily = base + tfDyn2.SubscribeForSlot(r.cfg.clock.CurrentSlot()) + assert.Equal(t, 2, len(r.cfg.p2p.PubSub().GetTopics())) topicMap := map[string]bool{} for _, t := range r.cfg.p2p.PubSub().GetTopics() { @@ -732,7 +600,10 @@ func TestSubscribeWithSyncSubnets_DynamicSwitchFork(t *testing.T) { // clear the cache and re-subscribe to subnets. // this should result in the subscriptions being removed cache.SyncSubnetIDs.EmptyAllCaches() - r.trySubscribeSubnets(sp) + + slot := r.cfg.clock.CurrentSlot() + tfDyn2.UnsubscribeForSlot(slot) + tfDyn2.SubscribeForSlot(r.cfg.clock.CurrentSlot()) assert.Equal(t, 0, len(r.cfg.p2p.PubSub().GetTopics())) } diff --git a/beacon-chain/sync/subscription_topic_handler.go b/beacon-chain/sync/subscription_topic_handler.go index bfb42dd1999e..2d4be4f7e60d 100644 --- a/beacon-chain/sync/subscription_topic_handler.go +++ b/beacon-chain/sync/subscription_topic_handler.go @@ -35,6 +35,7 @@ func (s *subTopicHandler) addTopic(topic string, sub *pubsub.Subscription) { s.digestMap[digest] += 1 } +// topicExists checks if a topic is currently tracked. func (s *subTopicHandler) topicExists(topic string) bool { s.RLock() defer s.RUnlock() @@ -64,6 +65,7 @@ func (s *subTopicHandler) removeTopic(topic string) { } } +// digestExists checks if a fork digest is currently tracked. func (s *subTopicHandler) digestExists(digest [4]byte) bool { s.RLock() defer s.RUnlock() @@ -72,6 +74,7 @@ func (s *subTopicHandler) digestExists(digest [4]byte) bool { return ok && count > 0 } +// allTopics returns all currently tracked topics. func (s *subTopicHandler) allTopics() []string { s.RLock() defer s.RUnlock() @@ -83,6 +86,7 @@ func (s *subTopicHandler) allTopics() []string { return topics } +// subForTopic returns the subscription for a given topic. func (s *subTopicHandler) subForTopic(topic string) *pubsub.Subscription { s.RLock() defer s.RUnlock() diff --git a/beacon-chain/sync/topic_families_dynamic_subnets.go b/beacon-chain/sync/topic_families_dynamic_subnets.go index 710b16576eaa..17a9f60544bf 100644 --- a/beacon-chain/sync/topic_families_dynamic_subnets.go +++ b/beacon-chain/sync/topic_families_dynamic_subnets.go @@ -1,239 +1,202 @@ package sync import ( - "fmt" - "sync" - "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p" "github.com/OffchainLabs/prysm/v6/config/params" "github.com/OffchainLabs/prysm/v6/consensus-types/primitives" "github.com/ethereum/go-ethereum/p2p/enode" "github.com/ethereum/go-ethereum/p2p/enr" "github.com/pkg/errors" - "github.com/sirupsen/logrus" ) // AttestationTopicFamily var _ GossipsubTopicFamilyWithDynamicSubnets = (*AttestationTopicFamily)(nil) -type baseGossipsubTopicFamilyWithDynamicSubnets struct { - baseGossipsubTopicFamily - - mu sync.Mutex - tracker *subnetTracker - unsubscribed bool -} - -func (b *baseGossipsubTopicFamilyWithDynamicSubnets) Subscribe(tf GossipsubTopicFamilyWithDynamicSubnets) { - b.mu.Lock() - defer b.mu.Unlock() - if b.unsubscribed { - log.WithFields(logrus.Fields{ - "topicFamily": fmt.Sprintf("%T", tf), - "digest": b.nse.ForkDigest, - "epoch": b.nse.Epoch, - }).Error("Cannot subscribe after unsubscribing") - return - } - b.tracker = b.syncService.subscribeToDynamicSubnetFamily(tf) -} - -func (b *baseGossipsubTopicFamilyWithDynamicSubnets) Unsubscribe() { - b.mu.Lock() - defer b.mu.Unlock() - b.unsubscribed = true - b.syncService.pruneNotWanted(b.tracker, nil) // unsubscribe from all subnets -} - type AttestationTopicFamily struct { - baseGossipsubTopicFamilyWithDynamicSubnets + *baseGossipsubTopicFamily } // NewAttestationTopicFamily creates a new AttestationTopicFamily. func NewAttestationTopicFamily(s *Service, nse params.NetworkScheduleEntry) *AttestationTopicFamily { - attestationTopicFamily := &AttestationTopicFamily{ - baseGossipsubTopicFamilyWithDynamicSubnets: baseGossipsubTopicFamilyWithDynamicSubnets{ - baseGossipsubTopicFamily: baseGossipsubTopicFamily{ - syncService: s, - nse: nse, - protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix(), - }, - }, - } - return attestationTopicFamily + a := &AttestationTopicFamily{} + base := newBaseGossipsubTopicFamily(s, nse, s.validateCommitteeIndexBeaconAttestation, s.committeeIndexBeaconAttestationSubscriber, a) + a.baseGossipsubTopicFamily = base + return a } func (a *AttestationTopicFamily) Name() string { return "AttestationTopicFamily" } -// Validator returns the validator function for attestation subnets. -func (a *AttestationTopicFamily) Validator() wrappedVal { - return a.syncService.validateCommitteeIndexBeaconAttestation +// SubscribeForSlot subscribes to the topics for the given slot. +func (a *AttestationTopicFamily) SubscribeForSlot(slot primitives.Slot) { + a.subscribeToTopics(a.TopicsToSubscribeForSlot(slot)) } -// Handler returns the message handler for attestation subnets. -func (a *AttestationTopicFamily) Handler() subHandler { - return a.syncService.committeeIndexBeaconAttestationSubscriber +// UnsubscribeForSlot unsubscribes from topics we no longer need for the slot. +func (a *AttestationTopicFamily) UnsubscribeForSlot(slot primitives.Slot) { + a.removeUnwantedTopics(a.TopicsToSubscribeForSlot(slot)) } -// GetFullTopicString builds the full topic string for an attestation subnet. -func (a *AttestationTopicFamily) GetFullTopicString(subnet uint64) string { - return fmt.Sprintf(p2p.AttestationSubnetTopicFormat, a.nse.ForkDigest, subnet) + a.protocolSuffix +// UnsubscribeAll unsubscribes from all topics in the family. +func (a *AttestationTopicFamily) UnsubscribeAll() { + a.unsubscribeAll() } -// GetSubnetsToJoin returns persistent and aggregator subnets. -func (a *AttestationTopicFamily) GetSubnetsToJoin(slot primitives.Slot) map[uint64]bool { - return a.syncService.persistentAndAggregatorSubnetIndices(slot) +// TopicsToSubscribeFor returns the topics to subscribe to for a given slot. +func (a *AttestationTopicFamily) TopicsToSubscribeForSlot(slot primitives.Slot) []string { + return topicsFromSubnets(computeNeededSubnets(a, slot), a) } -// GetSubnetsForBroadcast returns subnets needed for attestation duties. -func (a *AttestationTopicFamily) GetSubnetsForBroadcast(slot primitives.Slot) map[uint64]bool { - return attesterSubnetIndices(slot) +// getFullTopicString builds the full topic string for an attestation subnet. +func (a *AttestationTopicFamily) getFullTopicString(subnet uint64) string { + return p2p.AttestationSubnetTopic(a.nse.ForkDigest, subnet) } -// GetTopicsForNode returns all topics for the given node that are relevant to this topic family. -func (a *AttestationTopicFamily) GetTopicsForNode(node *enode.Node) ([]string, error) { - return getTopicsForNode(a.syncService, a, node, p2p.AttestationSubnets) +// getSubnetsToJoin returns persistent and aggregator subnets. +func (a *AttestationTopicFamily) getSubnetsToJoin(slot primitives.Slot) map[uint64]bool { + return a.syncService.persistentAndAggregatorSubnetIndices(slot) } -func (a *AttestationTopicFamily) Subscribe() { - a.baseGossipsubTopicFamilyWithDynamicSubnets.Subscribe(a) +// getSubnetsForBroadcast returns subnets needed for attestation duties. +func (a *AttestationTopicFamily) getSubnetsForBroadcast(slot primitives.Slot) map[uint64]bool { + return attesterSubnetIndices(slot) } -func (a *AttestationTopicFamily) Unsubscribe() { - a.baseGossipsubTopicFamilyWithDynamicSubnets.Unsubscribe() +// ExtractTopicsForNode returns all topics for the given node that are relevant to this topic family. +func (a *AttestationTopicFamily) ExtractTopicsForNode(node *enode.Node) ([]string, error) { + return getTopicsForNode(a.syncService, a, node, p2p.AttestationSubnets) } // SyncCommitteeTopicFamily var _ GossipsubTopicFamilyWithDynamicSubnets = (*SyncCommitteeTopicFamily)(nil) type SyncCommitteeTopicFamily struct { - baseGossipsubTopicFamilyWithDynamicSubnets + *baseGossipsubTopicFamily } // NewSyncCommitteeTopicFamily creates a new SyncCommitteeTopicFamily. func NewSyncCommitteeTopicFamily(s *Service, nse params.NetworkScheduleEntry) *SyncCommitteeTopicFamily { - return &SyncCommitteeTopicFamily{ - baseGossipsubTopicFamilyWithDynamicSubnets: baseGossipsubTopicFamilyWithDynamicSubnets{ - baseGossipsubTopicFamily: baseGossipsubTopicFamily{ - syncService: s, - nse: nse, - protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix(), - }, - }, - } + sc := &SyncCommitteeTopicFamily{} + base := newBaseGossipsubTopicFamily(s, nse, s.validateSyncCommitteeMessage, s.syncCommitteeMessageSubscriber, sc) + sc.baseGossipsubTopicFamily = base + return sc } func (s *SyncCommitteeTopicFamily) Name() string { return "SyncCommitteeTopicFamily" } -// Validator returns the validator function for sync committee subnets. -func (s *SyncCommitteeTopicFamily) Validator() wrappedVal { - return s.syncService.validateSyncCommitteeMessage +// SubscribeFor subscribes to the topics for the given slot. +func (s *SyncCommitteeTopicFamily) SubscribeForSlot(slot primitives.Slot) { + s.subscribeToTopics(s.TopicsToSubscribeForSlot(slot)) } -// Handler returns the message handler for sync committee subnets. -func (s *SyncCommitteeTopicFamily) Handler() subHandler { - return s.syncService.syncCommitteeMessageSubscriber +// UnsubscribeFor unsubscribes from topics we no longer need for the slot. +func (s *SyncCommitteeTopicFamily) UnsubscribeForSlot(slot primitives.Slot) { + s.removeUnwantedTopics(s.TopicsToSubscribeForSlot(slot)) } -// GetFullTopicString builds the full topic string for a sync committee subnet. -func (s *SyncCommitteeTopicFamily) GetFullTopicString(subnet uint64) string { - return fmt.Sprintf(p2p.SyncCommitteeSubnetTopicFormat, s.nse.ForkDigest, subnet) + s.protocolSuffix +// UnsubscribeAll unsubscribes from all topics in the family. +func (s *SyncCommitteeTopicFamily) UnsubscribeAll() { + s.unsubscribeAll() } -// GetSubnetsToJoin returns active sync committee subnets. -func (s *SyncCommitteeTopicFamily) GetSubnetsToJoin(slot primitives.Slot) map[uint64]bool { - return s.syncService.activeSyncSubnetIndices(slot) +// TopicsToSubscribeFor returns the topics to subscribe to for a given slot. +func (s *SyncCommitteeTopicFamily) TopicsToSubscribeForSlot(slot primitives.Slot) []string { + return topicsFromSubnets(computeNeededSubnets(s, slot), s) } -// GetSubnetsForBroadcast returns nil as there are no separate peer requirements. -func (s *SyncCommitteeTopicFamily) GetSubnetsForBroadcast(slot primitives.Slot) map[uint64]bool { - return nil +// getFullTopicString builds the full topic string for a sync committee subnet. +func (s *SyncCommitteeTopicFamily) getFullTopicString(subnet uint64) string { + return p2p.SyncCommitteeSubnetTopic(s.nse.ForkDigest, subnet) } -// GetTopicsForNode returns all topics for the given node that are relevant to this topic family. -func (s *SyncCommitteeTopicFamily) GetTopicsForNode(node *enode.Node) ([]string, error) { - return getTopicsForNode(s.syncService, s, node, p2p.SyncSubnets) +// getSubnetsToJoin returns active sync committee subnets. +func (s *SyncCommitteeTopicFamily) getSubnetsToJoin(slot primitives.Slot) map[uint64]bool { + return s.syncService.activeSyncSubnetIndices(slot) } -func (s *SyncCommitteeTopicFamily) Subscribe() { - s.baseGossipsubTopicFamilyWithDynamicSubnets.Subscribe(s) +// getSubnetsForBroadcast returns nil as there are no separate peer requirements. +func (s *SyncCommitteeTopicFamily) getSubnetsForBroadcast(slot primitives.Slot) map[uint64]bool { + return nil } -func (s *SyncCommitteeTopicFamily) Unsubscribe() { - s.baseGossipsubTopicFamilyWithDynamicSubnets.Unsubscribe() +// ExtractTopicsForNode returns all topics for the given node that are relevant to this topic family. +func (s *SyncCommitteeTopicFamily) ExtractTopicsForNode(node *enode.Node) ([]string, error) { + return getTopicsForNode(s.syncService, s, node, p2p.SyncSubnets) } // DataColumnTopicFamily var _ GossipsubTopicFamilyWithDynamicSubnets = (*DataColumnTopicFamily)(nil) type DataColumnTopicFamily struct { - baseGossipsubTopicFamilyWithDynamicSubnets + *baseGossipsubTopicFamily } // NewDataColumnTopicFamily creates a new DataColumnTopicFamily. func NewDataColumnTopicFamily(s *Service, nse params.NetworkScheduleEntry) *DataColumnTopicFamily { - return &DataColumnTopicFamily{ - baseGossipsubTopicFamilyWithDynamicSubnets: baseGossipsubTopicFamilyWithDynamicSubnets{ - baseGossipsubTopicFamily: baseGossipsubTopicFamily{ - syncService: s, - nse: nse, - protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix(), - }, - }, - } + d := &DataColumnTopicFamily{} + base := newBaseGossipsubTopicFamily(s, nse, s.validateDataColumn, s.dataColumnSubscriber, d) + d.baseGossipsubTopicFamily = base + return d } func (d *DataColumnTopicFamily) Name() string { return "DataColumnTopicFamily" } -// Validator returns the validator function for data column subnets. -func (d *DataColumnTopicFamily) Validator() wrappedVal { - return d.syncService.validateDataColumn +// SubscribeFor subscribes to the topics for the given slot. +func (d *DataColumnTopicFamily) SubscribeForSlot(slot primitives.Slot) { + d.subscribeToTopics(d.TopicsToSubscribeForSlot(slot)) } -// Handler returns the message handler for data column subnets. -func (d *DataColumnTopicFamily) Handler() subHandler { - return d.syncService.dataColumnSubscriber +// UnsubscribeForSlot unsubscribes from topics we no longer need for the slot. +func (d *DataColumnTopicFamily) UnsubscribeForSlot(slot primitives.Slot) { + d.removeUnwantedTopics(d.TopicsToSubscribeForSlot(slot)) } -// GetFullTopicString builds the full topic string for a data column subnet. -func (d *DataColumnTopicFamily) GetFullTopicString(subnet uint64) string { - return fmt.Sprintf(p2p.DataColumnSubnetTopicFormat, d.nse.ForkDigest, subnet) + d.protocolSuffix +// UnsubscribeAll unsubscribes from all topics in the family. +func (d *DataColumnTopicFamily) UnsubscribeAll() { + d.unsubscribeAll() } -// GetSubnetsToJoin returns data column subnets. -func (d *DataColumnTopicFamily) GetSubnetsToJoin(slot primitives.Slot) map[uint64]bool { - return d.syncService.dataColumnSubnetIndices(slot) +// TopicsToSubscribeFor returns the topics to subscribe to for a given slot. +func (d *DataColumnTopicFamily) TopicsToSubscribeForSlot(slot primitives.Slot) []string { + return topicsFromSubnets(computeNeededSubnets(d, slot), d) } -// GetSubnetsForBroadcast returns all data column subnets. -func (d *DataColumnTopicFamily) GetSubnetsForBroadcast(slot primitives.Slot) map[uint64]bool { - return d.syncService.allDataColumnSubnets(slot) +// getFullTopicString builds the full topic string for a data column subnet. +func (d *DataColumnTopicFamily) getFullTopicString(subnet uint64) string { + return p2p.DataColumnSubnetTopic(d.nse.ForkDigest, subnet) } -// GetTopicsForNode returns all topics for the given node that are relevant to this topic family. -func (d *DataColumnTopicFamily) GetTopicsForNode(node *enode.Node) ([]string, error) { - return getTopicsForNode(d.syncService, d, node, p2p.DataColumnSubnets) +// getSubnetsToJoin returns data column subnets. +func (d *DataColumnTopicFamily) getSubnetsToJoin(slot primitives.Slot) map[uint64]bool { + return d.syncService.dataColumnSubnetIndices(slot) } -func (d *DataColumnTopicFamily) Subscribe() { - d.baseGossipsubTopicFamilyWithDynamicSubnets.Subscribe(d) +// getSubnetsForBroadcast returns all data column subnets. +func (d *DataColumnTopicFamily) getSubnetsForBroadcast(slot primitives.Slot) map[uint64]bool { + return d.syncService.allDataColumnSubnets(slot) } -func (d *DataColumnTopicFamily) Unsubscribe() { - d.baseGossipsubTopicFamilyWithDynamicSubnets.Unsubscribe() +// ExtractTopicsForNode returns all topics for the given node that are relevant to this topic family. +func (d *DataColumnTopicFamily) ExtractTopicsForNode(node *enode.Node) ([]string, error) { + return getTopicsForNode(d.syncService, d, node, p2p.DataColumnSubnets) } type nodeSubnetExtractor func(id enode.ID, n *enode.Node, r *enr.Record) (map[uint64]bool, error) +type dynamicSubnetFamily interface { + getSubnetsToJoin(primitives.Slot) map[uint64]bool + getSubnetsForBroadcast(primitives.Slot) map[uint64]bool + getFullTopicString(subnet uint64) string +} + func getTopicsForNode( s *Service, - tf GossipsubTopicFamilyWithDynamicSubnets, + tf dynamicSubnetFamily, node *enode.Node, extractor nodeSubnetExtractor, ) ([]string, error) { @@ -241,10 +204,8 @@ func getTopicsForNode( return nil, errors.New("enode is nil") } currentSlot := s.cfg.clock.CurrentSlot() - neededSubnets := computeAllNeededSubnets( - currentSlot, - tf, - ) + + neededSubnets := computeNeededSubnets(tf, currentSlot) nodeSubnets, err := extractor(node.ID(), node, node.Record()) if err != nil { @@ -254,8 +215,30 @@ func getTopicsForNode( var topics []string for subnet := range neededSubnets { if nodeSubnets[subnet] { - topics = append(topics, tf.GetFullTopicString(subnet)) + topics = append(topics, tf.getFullTopicString(subnet)) } } return topics, nil } + +func computeNeededSubnets(tf dynamicSubnetFamily, slot primitives.Slot) map[uint64]bool { + subnetsToJoin := tf.getSubnetsToJoin(slot) + subnetsRequiringPeers := tf.getSubnetsForBroadcast(slot) + + neededSubnets := make(map[uint64]bool, len(subnetsToJoin)+len(subnetsRequiringPeers)) + for subnet := range subnetsToJoin { + neededSubnets[subnet] = true + } + for subnet := range subnetsRequiringPeers { + neededSubnets[subnet] = true + } + return neededSubnets +} + +func topicsFromSubnets(subnets map[uint64]bool, tf dynamicSubnetFamily) []string { + topics := make([]string, 0, len(subnets)) + for s := range subnets { + topics = append(topics, tf.getFullTopicString(s)) + } + return topics +} diff --git a/beacon-chain/sync/topic_families_static_subnets.go b/beacon-chain/sync/topic_families_static_subnets.go index bcec01ef2178..d1c63f0d3190 100644 --- a/beacon-chain/sync/topic_families_static_subnets.go +++ b/beacon-chain/sync/topic_families_static_subnets.go @@ -11,41 +11,33 @@ var _ GossipsubTopicFamilyWithoutDynamicSubnets = (*BlobTopicFamily)(nil) // BlobTopicFamily represents a static-subnet family instance for a specific blob subnet index. type BlobTopicFamily struct { - baseGossipsubTopicFamily + *baseGossipsubTopicFamily subnetIndex uint64 } func NewBlobTopicFamily(s *Service, nse params.NetworkScheduleEntry, subnetIndex uint64) *BlobTopicFamily { - return &BlobTopicFamily{ - baseGossipsubTopicFamily{ - syncService: s, - nse: nse, - protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix(), - }, - subnetIndex, + b := &BlobTopicFamily{ + subnetIndex: subnetIndex, } + base := newBaseGossipsubTopicFamily(s, nse, s.validateBlob, s.blobSubscriber, b) + b.baseGossipsubTopicFamily = base + return b } func (b *BlobTopicFamily) Name() string { return fmt.Sprintf("BlobTopicFamily-%d", b.subnetIndex) } -func (b *BlobTopicFamily) Validator() wrappedVal { - return b.syncService.validateBlob -} - -func (b *BlobTopicFamily) Handler() subHandler { - return b.syncService.blobSubscriber -} - -func (b *BlobTopicFamily) GetFullTopicString() string { - return fmt.Sprintf(p2p.BlobSubnetTopicFormat, b.nse.ForkDigest, b.subnetIndex) + b.protocolSuffix +// Subscribe subscribes to the static subnet topic. Slot is ignored for this topic family. +func (b *BlobTopicFamily) Subscribe() { + b.subscribeToTopics([]string{b.getFullTopicString()}) } -func (b *BlobTopicFamily) Subscribe() { - b.syncService.subscribe(b.GetFullTopicString(), b.Validator(), b.Handler()) +// UnsubscribeAll unsubscribes from all topics in the family. +func (b *BlobTopicFamily) UnsubscribeAll() { + b.unsubscribeAll() } -func (b *BlobTopicFamily) Unsubscribe() { - b.syncService.unSubscribeFromTopic(b.GetFullTopicString()) +func (b *BlobTopicFamily) getFullTopicString() string { + return p2p.BlobSubnetTopic(b.nse.ForkDigest, b.subnetIndex) } diff --git a/beacon-chain/sync/topic_families_without_subnets.go b/beacon-chain/sync/topic_families_without_subnets.go index 64944a453313..f531b2a9300d 100644 --- a/beacon-chain/sync/topic_families_without_subnets.go +++ b/beacon-chain/sync/topic_families_without_subnets.go @@ -1,8 +1,6 @@ package sync import ( - "fmt" - "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p" "github.com/OffchainLabs/prysm/v6/config/params" ) @@ -11,356 +9,284 @@ import ( var _ GossipsubTopicFamilyWithoutDynamicSubnets = (*BlockTopicFamily)(nil) type BlockTopicFamily struct { - baseGossipsubTopicFamily + *baseGossipsubTopicFamily } func NewBlockTopicFamily(s *Service, nse params.NetworkScheduleEntry) *BlockTopicFamily { - return &BlockTopicFamily{ - baseGossipsubTopicFamily{ - syncService: s, - nse: nse, - protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix()}, - } + b := &BlockTopicFamily{} + base := newBaseGossipsubTopicFamily(s, nse, s.validateBeaconBlockPubSub, s.beaconBlockSubscriber, b) + b.baseGossipsubTopicFamily = base + return b } func (b *BlockTopicFamily) Name() string { return "BlockTopicFamily" } -func (b *BlockTopicFamily) Validator() wrappedVal { - return b.syncService.validateBeaconBlockPubSub -} - -func (b *BlockTopicFamily) Handler() subHandler { - return b.syncService.beaconBlockSubscriber -} - -func (b *BlockTopicFamily) GetFullTopicString() string { - return fmt.Sprintf(p2p.BlockSubnetTopicFormat, b.nse.ForkDigest) + b.protocolSuffix +// Subscribe subscribes to the topic. +func (b *BlockTopicFamily) Subscribe() { + b.subscribeToTopics([]string{b.getFullTopicString()}) } -func (b *BlockTopicFamily) Subscribe() { - b.syncService.subscribe(b.GetFullTopicString(), b.Validator(), b.Handler()) +// UnsubscribeAll unsubscribes from all topics in the family. +func (b *BlockTopicFamily) UnsubscribeAll() { + b.unsubscribeAll() } -func (b *BlockTopicFamily) Unsubscribe() { - b.syncService.unSubscribeFromTopic(b.GetFullTopicString()) +func (b *BlockTopicFamily) getFullTopicString() string { + return p2p.BlockSubnetTopic(b.nse.ForkDigest) } // Aggregate and Proof var _ GossipsubTopicFamilyWithoutDynamicSubnets = (*AggregateAndProofTopicFamily)(nil) type AggregateAndProofTopicFamily struct { - baseGossipsubTopicFamily + *baseGossipsubTopicFamily } func NewAggregateAndProofTopicFamily(s *Service, nse params.NetworkScheduleEntry) *AggregateAndProofTopicFamily { - return &AggregateAndProofTopicFamily{ - baseGossipsubTopicFamily{ - syncService: s, - nse: nse, - protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix()}, - } + a := &AggregateAndProofTopicFamily{} + base := newBaseGossipsubTopicFamily(s, nse, s.validateAggregateAndProof, s.beaconAggregateProofSubscriber, a) + a.baseGossipsubTopicFamily = base + return a } func (a *AggregateAndProofTopicFamily) Name() string { return "AggregateAndProofTopicFamily" } -func (a *AggregateAndProofTopicFamily) Validator() wrappedVal { - return a.syncService.validateAggregateAndProof -} - -func (a *AggregateAndProofTopicFamily) Handler() subHandler { - return a.syncService.beaconAggregateProofSubscriber -} - -func (a *AggregateAndProofTopicFamily) GetFullTopicString() string { - return fmt.Sprintf(p2p.AggregateAndProofSubnetTopicFormat, a.nse.ForkDigest) + a.protocolSuffix +// Subscribe subscribes to the topic. +func (a *AggregateAndProofTopicFamily) Subscribe() { + a.subscribeToTopics([]string{a.getFullTopicString()}) } -func (a *AggregateAndProofTopicFamily) Subscribe() { - a.syncService.subscribe(a.GetFullTopicString(), a.Validator(), a.Handler()) +// UnsubscribeAll unsubscribes from all topics in the family. +func (a *AggregateAndProofTopicFamily) UnsubscribeAll() { + a.unsubscribeAll() } -func (a *AggregateAndProofTopicFamily) Unsubscribe() { - a.syncService.unSubscribeFromTopic(a.GetFullTopicString()) +func (a *AggregateAndProofTopicFamily) getFullTopicString() string { + return p2p.AggregateAndProofSubnetTopic(a.nse.ForkDigest) } // Voluntary Exit var _ GossipsubTopicFamilyWithoutDynamicSubnets = (*VoluntaryExitTopicFamily)(nil) type VoluntaryExitTopicFamily struct { - baseGossipsubTopicFamily + *baseGossipsubTopicFamily } func NewVoluntaryExitTopicFamily(s *Service, nse params.NetworkScheduleEntry) *VoluntaryExitTopicFamily { - return &VoluntaryExitTopicFamily{ - baseGossipsubTopicFamily{ - syncService: s, - nse: nse, - protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix()}, - } + v := &VoluntaryExitTopicFamily{} + base := newBaseGossipsubTopicFamily(s, nse, s.validateVoluntaryExit, s.voluntaryExitSubscriber, v) + v.baseGossipsubTopicFamily = base + return v } func (v *VoluntaryExitTopicFamily) Name() string { return "VoluntaryExitTopicFamily" } -func (v *VoluntaryExitTopicFamily) Validator() wrappedVal { - return v.syncService.validateVoluntaryExit -} - -func (v *VoluntaryExitTopicFamily) Handler() subHandler { - return v.syncService.voluntaryExitSubscriber +// Subscribe subscribes to the topic. Slot is ignored for this topic family. +func (v *VoluntaryExitTopicFamily) Subscribe() { + v.subscribeToTopics([]string{v.getFullTopicString()}) } -func (v *VoluntaryExitTopicFamily) GetFullTopicString() string { - return fmt.Sprintf(p2p.ExitSubnetTopicFormat, v.nse.ForkDigest) + v.protocolSuffix +// UnsubscribeAll unsubscribes from all topics in the family. +func (v *VoluntaryExitTopicFamily) UnsubscribeAll() { + v.unsubscribeAll() } -func (v *VoluntaryExitTopicFamily) Subscribe() { - v.syncService.subscribe(v.GetFullTopicString(), v.Validator(), v.Handler()) -} - -func (v *VoluntaryExitTopicFamily) Unsubscribe() { - v.syncService.unSubscribeFromTopic(v.GetFullTopicString()) +func (v *VoluntaryExitTopicFamily) getFullTopicString() string { + return p2p.VoluntaryExitSubnetTopic(v.nse.ForkDigest) } // Proposer Slashing var _ GossipsubTopicFamilyWithoutDynamicSubnets = (*ProposerSlashingTopicFamily)(nil) type ProposerSlashingTopicFamily struct { - baseGossipsubTopicFamily + *baseGossipsubTopicFamily } func NewProposerSlashingTopicFamily(s *Service, nse params.NetworkScheduleEntry) *ProposerSlashingTopicFamily { - return &ProposerSlashingTopicFamily{ - baseGossipsubTopicFamily{ - syncService: s, - nse: nse, - protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix()}, - } + p := &ProposerSlashingTopicFamily{} + base := newBaseGossipsubTopicFamily(s, nse, s.validateProposerSlashing, s.proposerSlashingSubscriber, p) + p.baseGossipsubTopicFamily = base + return p } func (p *ProposerSlashingTopicFamily) Name() string { return "ProposerSlashingTopicFamily" } -func (p *ProposerSlashingTopicFamily) Validator() wrappedVal { - return p.syncService.validateProposerSlashing -} - -func (p *ProposerSlashingTopicFamily) Handler() subHandler { - return p.syncService.proposerSlashingSubscriber -} - -func (p *ProposerSlashingTopicFamily) GetFullTopicString() string { - return fmt.Sprintf(p2p.ProposerSlashingSubnetTopicFormat, p.nse.ForkDigest) + p.protocolSuffix +// Subscribe subscribes to the topic. Slot is ignored for this topic family. +func (p *ProposerSlashingTopicFamily) Subscribe() { + p.subscribeToTopics([]string{p.getFullTopicString()}) } -func (p *ProposerSlashingTopicFamily) Subscribe() { - p.syncService.subscribe(p.GetFullTopicString(), p.Validator(), p.Handler()) +// UnsubscribeAll unsubscribes from all topics in the family. +func (p *ProposerSlashingTopicFamily) UnsubscribeAll() { + p.unsubscribeAll() } -func (p *ProposerSlashingTopicFamily) Unsubscribe() { - p.syncService.unSubscribeFromTopic(p.GetFullTopicString()) +func (p *ProposerSlashingTopicFamily) getFullTopicString() string { + return p2p.ProposerSlashingSubnetTopic(p.nse.ForkDigest) } // Attester Slashing var _ GossipsubTopicFamilyWithoutDynamicSubnets = (*AttesterSlashingTopicFamily)(nil) type AttesterSlashingTopicFamily struct { - baseGossipsubTopicFamily + *baseGossipsubTopicFamily } func NewAttesterSlashingTopicFamily(s *Service, nse params.NetworkScheduleEntry) *AttesterSlashingTopicFamily { - return &AttesterSlashingTopicFamily{ - baseGossipsubTopicFamily{ - syncService: s, - nse: nse, - protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix()}, - } + a := &AttesterSlashingTopicFamily{} + base := newBaseGossipsubTopicFamily(s, nse, s.validateAttesterSlashing, s.attesterSlashingSubscriber, a) + a.baseGossipsubTopicFamily = base + return a } func (a *AttesterSlashingTopicFamily) Name() string { return "AttesterSlashingTopicFamily" } -func (a *AttesterSlashingTopicFamily) Validator() wrappedVal { - return a.syncService.validateAttesterSlashing -} - -func (a *AttesterSlashingTopicFamily) Handler() subHandler { - return a.syncService.attesterSlashingSubscriber -} - -func (a *AttesterSlashingTopicFamily) GetFullTopicString() string { - return fmt.Sprintf(p2p.AttesterSlashingSubnetTopicFormat, a.nse.ForkDigest) + a.protocolSuffix +// Subscribe subscribes to the topic. Slot is ignored for this topic family. +func (a *AttesterSlashingTopicFamily) Subscribe() { + a.subscribeToTopics([]string{a.getFullTopicString()}) } -// TODO: Do we really need to spawn go-routines here ? -func (a *AttesterSlashingTopicFamily) Subscribe() { - a.syncService.subscribe(a.GetFullTopicString(), a.Validator(), a.Handler()) +// UnsubscribeAll unsubscribes from all topics in the family. +func (a *AttesterSlashingTopicFamily) UnsubscribeAll() { + a.unsubscribeAll() } -func (a *AttesterSlashingTopicFamily) Unsubscribe() { - a.syncService.unSubscribeFromTopic(a.GetFullTopicString()) +func (a *AttesterSlashingTopicFamily) getFullTopicString() string { + return p2p.AttesterSlashingSubnetTopic(a.nse.ForkDigest) } // Sync Contribution and Proof (Altair+) var _ GossipsubTopicFamilyWithoutDynamicSubnets = (*SyncContributionAndProofTopicFamily)(nil) -type SyncContributionAndProofTopicFamily struct{ baseGossipsubTopicFamily } +type SyncContributionAndProofTopicFamily struct{ *baseGossipsubTopicFamily } func NewSyncContributionAndProofTopicFamily(s *Service, nse params.NetworkScheduleEntry) *SyncContributionAndProofTopicFamily { - return &SyncContributionAndProofTopicFamily{ - baseGossipsubTopicFamily{ - syncService: s, - nse: nse, - protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix()}, - } + sc := &SyncContributionAndProofTopicFamily{} + base := newBaseGossipsubTopicFamily(s, nse, s.validateSyncContributionAndProof, s.syncContributionAndProofSubscriber, sc) + sc.baseGossipsubTopicFamily = base + return sc } func (sc *SyncContributionAndProofTopicFamily) Name() string { return "SyncContributionAndProofTopicFamily" } -func (sc *SyncContributionAndProofTopicFamily) Validator() wrappedVal { - return sc.syncService.validateSyncContributionAndProof -} - -func (sc *SyncContributionAndProofTopicFamily) Handler() subHandler { - return sc.syncService.syncContributionAndProofSubscriber -} - -func (sc *SyncContributionAndProofTopicFamily) GetFullTopicString() string { - return fmt.Sprintf(p2p.SyncContributionAndProofSubnetTopicFormat, sc.nse.ForkDigest) + sc.protocolSuffix +// Subscribe subscribes to the topic. Slot is ignored for this topic family. +func (sc *SyncContributionAndProofTopicFamily) Subscribe() { + sc.subscribeToTopics([]string{sc.getFullTopicString()}) } -func (sc *SyncContributionAndProofTopicFamily) Subscribe() { - sc.syncService.subscribe(sc.GetFullTopicString(), sc.Validator(), sc.Handler()) +// UnsubscribeAll unsubscribes from all topics in the family. +func (sc *SyncContributionAndProofTopicFamily) UnsubscribeAll() { + sc.unsubscribeAll() } -func (sc *SyncContributionAndProofTopicFamily) Unsubscribe() { - sc.syncService.unSubscribeFromTopic(sc.GetFullTopicString()) +func (sc *SyncContributionAndProofTopicFamily) getFullTopicString() string { + return p2p.SyncContributionAndProofSubnetTopic(sc.nse.ForkDigest) } // Light Client Optimistic Update (Altair+) var _ GossipsubTopicFamilyWithoutDynamicSubnets = (*LightClientOptimisticUpdateTopicFamily)(nil) type LightClientOptimisticUpdateTopicFamily struct { - baseGossipsubTopicFamily + *baseGossipsubTopicFamily } func NewLightClientOptimisticUpdateTopicFamily(s *Service, nse params.NetworkScheduleEntry) *LightClientOptimisticUpdateTopicFamily { - return &LightClientOptimisticUpdateTopicFamily{ - baseGossipsubTopicFamily{ - syncService: s, - nse: nse, - protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix()}, - } + l := &LightClientOptimisticUpdateTopicFamily{} + base := newBaseGossipsubTopicFamily(s, nse, s.validateLightClientOptimisticUpdate, noopHandler, l) + l.baseGossipsubTopicFamily = base + return l } func (l *LightClientOptimisticUpdateTopicFamily) Name() string { return "LightClientOptimisticUpdateTopicFamily" } -func (l *LightClientOptimisticUpdateTopicFamily) Validator() wrappedVal { - return l.syncService.validateLightClientOptimisticUpdate -} - -func (l *LightClientOptimisticUpdateTopicFamily) Handler() subHandler { - return noopHandler -} - -func (l *LightClientOptimisticUpdateTopicFamily) GetFullTopicString() string { - return fmt.Sprintf(p2p.LightClientOptimisticUpdateTopicFormat, l.nse.ForkDigest) + l.protocolSuffix +// Subscribe subscribes to the topic. Slot is ignored for this topic family. +func (l *LightClientOptimisticUpdateTopicFamily) Subscribe() { + l.subscribeToTopics([]string{l.getFullTopicString()}) } -func (l *LightClientOptimisticUpdateTopicFamily) Subscribe() { - l.syncService.subscribe(l.GetFullTopicString(), l.Validator(), l.Handler()) +// UnsubscribeAll unsubscribes from all topics in the family. +func (l *LightClientOptimisticUpdateTopicFamily) UnsubscribeAll() { + l.unsubscribeAll() } -func (l *LightClientOptimisticUpdateTopicFamily) Unsubscribe() { - l.syncService.unSubscribeFromTopic(l.GetFullTopicString()) +func (l *LightClientOptimisticUpdateTopicFamily) getFullTopicString() string { + return p2p.LcOptimisticToTopic(l.nse.ForkDigest) } // Light Client Finality Update (Altair+) var _ GossipsubTopicFamilyWithoutDynamicSubnets = (*LightClientFinalityUpdateTopicFamily)(nil) type LightClientFinalityUpdateTopicFamily struct { - baseGossipsubTopicFamily + *baseGossipsubTopicFamily } func NewLightClientFinalityUpdateTopicFamily(s *Service, nse params.NetworkScheduleEntry) *LightClientFinalityUpdateTopicFamily { - return &LightClientFinalityUpdateTopicFamily{ - baseGossipsubTopicFamily{ - syncService: s, - nse: nse, - protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix()}, - } + l := &LightClientFinalityUpdateTopicFamily{} + base := newBaseGossipsubTopicFamily(s, nse, s.validateLightClientFinalityUpdate, noopHandler, l) + l.baseGossipsubTopicFamily = base + return l } func (l *LightClientFinalityUpdateTopicFamily) Name() string { return "LightClientFinalityUpdateTopicFamily" } -func (l *LightClientFinalityUpdateTopicFamily) Validator() wrappedVal { - return l.syncService.validateLightClientFinalityUpdate -} - -func (l *LightClientFinalityUpdateTopicFamily) Handler() subHandler { - return noopHandler +// Subscribe subscribes to the topic. Slot is ignored for this topic family. +func (l *LightClientFinalityUpdateTopicFamily) Subscribe() { + l.subscribeToTopics([]string{l.getFullTopicString()}) } -func (l *LightClientFinalityUpdateTopicFamily) GetFullTopicString() string { - return fmt.Sprintf(p2p.LightClientFinalityUpdateTopicFormat, l.nse.ForkDigest) + l.protocolSuffix +// UnsubscribeAll unsubscribes from all topics in the family. +func (l *LightClientFinalityUpdateTopicFamily) UnsubscribeAll() { + l.unsubscribeAll() } -func (l *LightClientFinalityUpdateTopicFamily) Subscribe() { - l.syncService.subscribe(l.GetFullTopicString(), l.Validator(), l.Handler()) -} -func (l *LightClientFinalityUpdateTopicFamily) Unsubscribe() { - l.syncService.unSubscribeFromTopic(l.GetFullTopicString()) +func (l *LightClientFinalityUpdateTopicFamily) getFullTopicString() string { + return p2p.LcFinalityToTopic(l.nse.ForkDigest) } // BLS to Execution Change (Capella+) var _ GossipsubTopicFamilyWithoutDynamicSubnets = (*BlsToExecutionChangeTopicFamily)(nil) type BlsToExecutionChangeTopicFamily struct { - baseGossipsubTopicFamily + *baseGossipsubTopicFamily } func NewBlsToExecutionChangeTopicFamily(s *Service, nse params.NetworkScheduleEntry) *BlsToExecutionChangeTopicFamily { - return &BlsToExecutionChangeTopicFamily{ - baseGossipsubTopicFamily{ - syncService: s, - nse: nse, - protocolSuffix: s.cfg.p2p.Encoding().ProtocolSuffix()}, - } + b := &BlsToExecutionChangeTopicFamily{} + base := newBaseGossipsubTopicFamily(s, nse, s.validateBlsToExecutionChange, s.blsToExecutionChangeSubscriber, b) + b.baseGossipsubTopicFamily = base + return b } func (b *BlsToExecutionChangeTopicFamily) Name() string { return "BlsToExecutionChangeTopicFamily" } -func (b *BlsToExecutionChangeTopicFamily) Validator() wrappedVal { - return b.syncService.validateBlsToExecutionChange -} - -func (b *BlsToExecutionChangeTopicFamily) Handler() subHandler { - return b.syncService.blsToExecutionChangeSubscriber -} - -func (b *BlsToExecutionChangeTopicFamily) GetFullTopicString() string { - return fmt.Sprintf(p2p.BlsToExecutionChangeSubnetTopicFormat, b.nse.ForkDigest) + b.protocolSuffix +// Subscribe subscribes to the topic. Slot is ignored for this topic family. +func (b *BlsToExecutionChangeTopicFamily) Subscribe() { + b.subscribeToTopics([]string{b.getFullTopicString()}) } -func (b *BlsToExecutionChangeTopicFamily) Subscribe() { - b.syncService.subscribe(b.GetFullTopicString(), b.Validator(), b.Handler()) +// UnsubscribeAll unsubscribes from all topics in the family. +func (b *BlsToExecutionChangeTopicFamily) UnsubscribeAll() { + b.unsubscribeAll() } -func (b *BlsToExecutionChangeTopicFamily) Unsubscribe() { - b.syncService.unSubscribeFromTopic(b.GetFullTopicString()) +func (b *BlsToExecutionChangeTopicFamily) getFullTopicString() string { + return p2p.BlsToExecutionChangeSubnetTopic(b.nse.ForkDigest) } From 672de432a2df43c839d1dd34a26124bbf699b628 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Fri, 21 Nov 2025 12:08:17 +0400 Subject: [PATCH 27/39] finish all changes --- beacon-chain/sync/BUILD.bazel | 2 +- .../sync/{subscriber_test.go => gossipsub_base_test.go} | 0 2 files changed, 1 insertion(+), 1 deletion(-) rename beacon-chain/sync/{subscriber_test.go => gossipsub_base_test.go} (100%) diff --git a/beacon-chain/sync/BUILD.bazel b/beacon-chain/sync/BUILD.bazel index 9cb5c0837cef..3eace18dd626 100644 --- a/beacon-chain/sync/BUILD.bazel +++ b/beacon-chain/sync/BUILD.bazel @@ -181,6 +181,7 @@ go_test( "decode_pubsub_test.go", "error_test.go", "fork_watcher_test.go", + "gossipsub_base_test.go", "gossipsub_controller_test.go", "gossipsub_topic_family_test.go", "kzg_batch_verifier_test.go", @@ -208,7 +209,6 @@ go_test( "subscriber_beacon_aggregate_proof_test.go", "subscriber_beacon_blocks_test.go", "subscriber_data_column_sidecar_test.go", - "subscriber_test.go", "subscription_topic_handler_test.go", "sync_fuzz_test.go", "sync_test.go", diff --git a/beacon-chain/sync/subscriber_test.go b/beacon-chain/sync/gossipsub_base_test.go similarity index 100% rename from beacon-chain/sync/subscriber_test.go rename to beacon-chain/sync/gossipsub_base_test.go From 76975a134d6c9619c3df1d23138de22c16ab10f6 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Fri, 21 Nov 2025 14:00:40 +0400 Subject: [PATCH 28/39] fix build --- beacon-chain/p2p/testing/fuzz_p2p.go | 5 ----- beacon-chain/p2p/testing/mock_peermanager.go | 6 +++--- beacon-chain/p2p/testing/p2p.go | 5 ----- 3 files changed, 3 insertions(+), 13 deletions(-) diff --git a/beacon-chain/p2p/testing/fuzz_p2p.go b/beacon-chain/p2p/testing/fuzz_p2p.go index 7e66ba6896aa..7712e6225ee2 100644 --- a/beacon-chain/p2p/testing/fuzz_p2p.go +++ b/beacon-chain/p2p/testing/fuzz_p2p.go @@ -80,11 +80,6 @@ func (*FakeP2P) DiscoveryAddresses() ([]multiaddr.Multiaddr, error) { return nil, nil } -// FindAndDialPeersWithSubnets mocks the p2p func. -func (*FakeP2P) FindAndDialPeersWithSubnets(ctx context.Context, fullTopicForSubnet func(uint64) string, minimumPeersPerSubnet int, subnets map[uint64]bool) error { - return nil -} - // RefreshPersistentSubnets mocks the p2p func. func (*FakeP2P) RefreshPersistentSubnets() {} diff --git a/beacon-chain/p2p/testing/mock_peermanager.go b/beacon-chain/p2p/testing/mock_peermanager.go index 6134a19cc51e..5beb4dbab083 100644 --- a/beacon-chain/p2p/testing/mock_peermanager.go +++ b/beacon-chain/p2p/testing/mock_peermanager.go @@ -57,9 +57,9 @@ func (m *MockPeerManager) DiscoveryAddresses() ([]multiaddr.Multiaddr, error) { // RefreshPersistentSubnets . func (*MockPeerManager) RefreshPersistentSubnets() {} -// FindAndDialPeersWithSubnet . -func (*MockPeerManager) FindAndDialPeersWithSubnets(ctx context.Context, fullTopicForSubnet func(uint64) string, minimumPeersPerSubnet int, subnets map[uint64]bool) error { - return nil +// DialPeers +func (p *MockPeerManager) DialPeers(ctx context.Context, maxConcurrentDials int, nodes []*enode.Node) uint { + return 0 } // AddPingMethod . diff --git a/beacon-chain/p2p/testing/p2p.go b/beacon-chain/p2p/testing/p2p.go index d9de566055b1..e9ee6cbec315 100644 --- a/beacon-chain/p2p/testing/p2p.go +++ b/beacon-chain/p2p/testing/p2p.go @@ -423,11 +423,6 @@ func (p *TestP2P) DialPeers(ctx context.Context, maxConcurrentDials int, nodes [ return 0 } -// FindAndDialPeersWithSubnets mocks the p2p func. -func (*TestP2P) FindAndDialPeersWithSubnets(ctx context.Context, fullTopicForSubnet func(uint64) string, minimumPeersPerSubnet int, subnets map[uint64]bool) error { - return nil -} - // RefreshPersistentSubnets mocks the p2p func. func (*TestP2P) RefreshPersistentSubnets() {} From 707abe61123e76766d9819372a5146c85d964887 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Fri, 21 Nov 2025 14:06:36 +0400 Subject: [PATCH 29/39] fix mock peer manager --- beacon-chain/p2p/testing/mock_peermanager.go | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/beacon-chain/p2p/testing/mock_peermanager.go b/beacon-chain/p2p/testing/mock_peermanager.go index 5beb4dbab083..70a651d97bdb 100644 --- a/beacon-chain/p2p/testing/mock_peermanager.go +++ b/beacon-chain/p2p/testing/mock_peermanager.go @@ -4,6 +4,7 @@ import ( "context" "errors" + "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/gossipsubcrawler" "github.com/ethereum/go-ethereum/p2p/enode" "github.com/ethereum/go-ethereum/p2p/enr" @@ -12,6 +13,8 @@ import ( "github.com/multiformats/go-multiaddr" ) +var _ p2p.PeerManager = (*MockPeerManager)(nil) + // MockPeerManager is mock of the PeerManager interface. type MockPeerManager struct { Enr *enr.Record @@ -19,6 +22,7 @@ type MockPeerManager struct { BHost host.Host DiscoveryAddr []multiaddr.Multiaddr FailDiscoveryAddr bool + Dialer gossipsubcrawler.GossipsubDialer } // Disconnect . @@ -46,6 +50,11 @@ func (m MockPeerManager) NodeID() enode.ID { return enode.ID{} } +// GossipsubDialer returns the configured dialer mock, if any. +func (m MockPeerManager) GossipsubDialer() gossipsubcrawler.GossipsubDialer { + return m.Dialer +} + // DiscoveryAddresses . func (m *MockPeerManager) DiscoveryAddresses() ([]multiaddr.Multiaddr, error) { if m.FailDiscoveryAddr { From 41c9f160a2513bfe17ceb6227da7fdfb17267a21 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Fri, 21 Nov 2025 14:08:02 +0400 Subject: [PATCH 30/39] fix bazel --- beacon-chain/p2p/testing/BUILD.bazel | 1 + 1 file changed, 1 insertion(+) diff --git a/beacon-chain/p2p/testing/BUILD.bazel b/beacon-chain/p2p/testing/BUILD.bazel index 90babd27c7f2..25ecb401ccef 100644 --- a/beacon-chain/p2p/testing/BUILD.bazel +++ b/beacon-chain/p2p/testing/BUILD.bazel @@ -20,6 +20,7 @@ go_library( ], deps = [ "//beacon-chain/core/peerdas:go_default_library", + "//beacon-chain/p2p:go_default_library", "//beacon-chain/p2p/encoder:go_default_library", "//beacon-chain/p2p/gossipsubcrawler:go_default_library", "//beacon-chain/p2p/peers:go_default_library", From 893cf6092117cd307f80975f4a3ce30a754e1575 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Fri, 21 Nov 2025 15:37:15 +0400 Subject: [PATCH 31/39] fix tests --- beacon-chain/p2p/BUILD.bazel | 1 + beacon-chain/p2p/subnets_test.go | 3 +++ beacon-chain/p2p/testing/BUILD.bazel | 1 - beacon-chain/p2p/testing/mock_peermanager.go | 3 --- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/beacon-chain/p2p/BUILD.bazel b/beacon-chain/p2p/BUILD.bazel index 1e59b72b493f..7177e21738a9 100644 --- a/beacon-chain/p2p/BUILD.bazel +++ b/beacon-chain/p2p/BUILD.bazel @@ -133,6 +133,7 @@ go_test( "fork_test.go", "gossip_scoring_params_test.go", "gossip_topic_mappings_test.go", + "gossipsub_peer_controller_test.go", "gossipsub_peer_crawler_test.go", "message_id_test.go", "options_test.go", diff --git a/beacon-chain/p2p/subnets_test.go b/beacon-chain/p2p/subnets_test.go index de7070a877c0..2551b2b85927 100644 --- a/beacon-chain/p2p/subnets_test.go +++ b/beacon-chain/p2p/subnets_test.go @@ -217,6 +217,9 @@ func TestStartDiscV5_FindAndDialPeersWithSubnet(t *testing.T) { topicsToDial = append(topicsToDial, builder(s)) } + ctx, cancel := context.WithTimeout(ctx, 15*time.Second) + defer cancel() + for _, topic := range topicsToDial { err = service.GossipsubDialer().DialPeersForTopicBlocking(ctx, topic, minimumPeersPerSubnet) require.NoError(t, err) diff --git a/beacon-chain/p2p/testing/BUILD.bazel b/beacon-chain/p2p/testing/BUILD.bazel index 25ecb401ccef..90babd27c7f2 100644 --- a/beacon-chain/p2p/testing/BUILD.bazel +++ b/beacon-chain/p2p/testing/BUILD.bazel @@ -20,7 +20,6 @@ go_library( ], deps = [ "//beacon-chain/core/peerdas:go_default_library", - "//beacon-chain/p2p:go_default_library", "//beacon-chain/p2p/encoder:go_default_library", "//beacon-chain/p2p/gossipsubcrawler:go_default_library", "//beacon-chain/p2p/peers:go_default_library", diff --git a/beacon-chain/p2p/testing/mock_peermanager.go b/beacon-chain/p2p/testing/mock_peermanager.go index 70a651d97bdb..3adc2ee8aeb5 100644 --- a/beacon-chain/p2p/testing/mock_peermanager.go +++ b/beacon-chain/p2p/testing/mock_peermanager.go @@ -4,7 +4,6 @@ import ( "context" "errors" - "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p" "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/gossipsubcrawler" "github.com/ethereum/go-ethereum/p2p/enode" "github.com/ethereum/go-ethereum/p2p/enr" @@ -13,8 +12,6 @@ import ( "github.com/multiformats/go-multiaddr" ) -var _ p2p.PeerManager = (*MockPeerManager)(nil) - // MockPeerManager is mock of the PeerManager interface. type MockPeerManager struct { Enr *enr.Record From e9dac060375c0cafaa0479d75fad0c8a612c12d5 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Fri, 21 Nov 2025 16:38:00 +0400 Subject: [PATCH 32/39] finish tests --- beacon-chain/p2p/broadcaster_test.go | 2 +- beacon-chain/p2p/gossipsub_peer_controller.go | 26 +- .../p2p/gossipsub_peer_controller_test.go | 318 ++++++++++++++++++ beacon-chain/p2p/service.go | 2 +- 4 files changed, 337 insertions(+), 11 deletions(-) create mode 100644 beacon-chain/p2p/gossipsub_peer_controller_test.go diff --git a/beacon-chain/p2p/broadcaster_test.go b/beacon-chain/p2p/broadcaster_test.go index e6b2d7fcfaab..6604eb33e089 100644 --- a/beacon-chain/p2p/broadcaster_test.go +++ b/beacon-chain/p2p/broadcaster_test.go @@ -746,7 +746,7 @@ func TestService_BroadcastDataColumn(t *testing.T) { return []string{topic}, nil }) require.NoError(t, err) - service.gossipsubDialer = NewGossipsubPeerDialer(service, crawler) + service.gossipsubDialer = NewGossipsubPeerDialer(crawler, service.PubSub().ListPeers, service.DialPeers) _, verifiedRoSidecars := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{{Index: columnIndex}}) verifiedRoSidecar := verifiedRoSidecars[0] diff --git a/beacon-chain/p2p/gossipsub_peer_controller.go b/beacon-chain/p2p/gossipsub_peer_controller.go index 123869138fb5..5821be7031d2 100644 --- a/beacon-chain/p2p/gossipsub_peer_controller.go +++ b/beacon-chain/p2p/gossipsub_peer_controller.go @@ -9,6 +9,7 @@ import ( "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/gossipsubcrawler" "github.com/OffchainLabs/prysm/v6/cmd/beacon-chain/flags" "github.com/ethereum/go-ethereum/p2p/enode" + "github.com/libp2p/go-libp2p/core/peer" "github.com/pkg/errors" ) @@ -20,7 +21,8 @@ type GossipsubPeerDialer struct { ctx context.Context cancel context.CancelFunc - service *Service + listPeersFunc func(topic string) []peer.ID + dialPeersFunc func(ctx context.Context, maxConcurrentDials int, nodes []*enode.Node) uint crawler gossipsubcrawler.Crawler topicsProvider gossipsubcrawler.SubnetTopicsProvider @@ -29,13 +31,18 @@ type GossipsubPeerDialer struct { once sync.Once } -func NewGossipsubPeerDialer(service *Service, crawler gossipsubcrawler.Crawler) *GossipsubPeerDialer { +func NewGossipsubPeerDialer( + crawler gossipsubcrawler.Crawler, + listPeers func(topic string) []peer.ID, + dialPeers func(ctx context.Context, maxConcurrentDials int, nodes []*enode.Node) uint, +) *GossipsubPeerDialer { ctx, cancel := context.WithCancel(context.Background()) return &GossipsubPeerDialer{ - service: service, - crawler: crawler, - ctx: ctx, - cancel: cancel, + listPeersFunc: listPeers, + dialPeersFunc: dialPeers, + crawler: crawler, + ctx: ctx, + cancel: cancel, } } @@ -101,7 +108,7 @@ func (g *GossipsubPeerDialer) dialLoop() { func (g *GossipsubPeerDialer) DialPeersForTopicBlocking(ctx context.Context, topic string, nPeers int) error { for { - peers := g.service.PubSub().ListPeers(topic) + peers := g.listPeersFunc(topic) if len(peers) >= nPeers { return nil } @@ -114,6 +121,7 @@ func (g *GossipsubPeerDialer) DialPeersForTopicBlocking(ctx context.Context, top select { case <-ctx.Done(): return ctx.Err() + // some wait here is good after dialing as connections take some time to show up in pubsub case <-time.After(100 * time.Millisecond): case <-g.ctx.Done(): return g.ctx.Err() @@ -122,7 +130,7 @@ func (g *GossipsubPeerDialer) DialPeersForTopicBlocking(ctx context.Context, top } func (g *GossipsubPeerDialer) peersForTopic(topic string, targetCount int) []*enode.Node { - peers := g.service.PubSub().ListPeers(topic) + peers := g.listPeersFunc(topic) peerCount := len(peers) if peerCount >= targetCount { return nil @@ -142,5 +150,5 @@ func (g *GossipsubPeerDialer) dialPeers(peers []*enode.Node) { if flags.MaxDialIsActive() { maxConcurrentDials = flags.Get().MaxConcurrentDials } - g.service.DialPeers(g.ctx, maxConcurrentDials, peers) + g.dialPeersFunc(g.ctx, maxConcurrentDials, peers) } diff --git a/beacon-chain/p2p/gossipsub_peer_controller_test.go b/beacon-chain/p2p/gossipsub_peer_controller_test.go new file mode 100644 index 000000000000..688652fd5d24 --- /dev/null +++ b/beacon-chain/p2p/gossipsub_peer_controller_test.go @@ -0,0 +1,318 @@ +package p2p + +import ( + "context" + "crypto/rand" + "net" + "sync" + "testing" + "time" + + "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/gossipsubcrawler" + "github.com/OffchainLabs/prysm/v6/crypto/ecdsa" + "github.com/ethereum/go-ethereum/p2p/enode" + "github.com/ethereum/go-ethereum/p2p/enr" + "github.com/libp2p/go-libp2p/core/crypto" + "github.com/libp2p/go-libp2p/core/peer" + "github.com/stretchr/testify/require" +) + +func TestGossipsubPeerDialer_Start(t *testing.T) { + tests := []struct { + name string + newCrawler func(t *testing.T) *mockCrawler + provider gossipsubcrawler.SubnetTopicsProvider + expectedConnects int + expectStartErr bool + }{ + { + name: "nil provider errors", + newCrawler: func(t *testing.T) *mockCrawler { return &mockCrawler{} }, + provider: nil, + expectStartErr: true, + }, + { + name: "dials unique peers across topics", + newCrawler: func(t *testing.T) *mockCrawler { + nodeA := newTestNode(t, "127.0.0.1", 30101) + nodeB := newTestNode(t, "127.0.0.1", 30102) + return &mockCrawler{ + consume: true, + peers: map[gossipsubcrawler.Topic][]*enode.Node{ + "topic/a": {nodeA, nodeB}, + "topic/b": {nodeA}, + }, + } + }, + provider: func() []string { + return []string{"topic/a", "topic/b"} + }, + expectedConnects: 2, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + md := &mockDialer{} + listPeers := func(topic string) []peer.ID { return nil } + + dialer := NewGossipsubPeerDialer(tt.newCrawler(t), listPeers, md.DialPeers) + defer dialer.Stop() + + err := dialer.Start(tt.provider) + if tt.expectStartErr { + require.Error(t, err) + return + } + require.NoError(t, err) + + require.Eventually(t, func() bool { + return md.dialCount() >= tt.expectedConnects + }, 2*time.Second, 20*time.Millisecond) + + require.Equal(t, tt.expectedConnects, md.dialCount()) + }) + } +} + +func TestGossipsubPeerDialer_DialPeersForTopicBlocking(t *testing.T) { + tests := []struct { + name string + connectedPeers int + newCrawler func(t *testing.T) *mockCrawler + targetPeers int + ctx func() (context.Context, context.CancelFunc) + expectedConnects int + expectErr bool + }{ + { + name: "returns immediately when enough peers", + connectedPeers: 1, + newCrawler: func(t *testing.T) *mockCrawler { + return &mockCrawler{} + }, + targetPeers: 1, + ctx: func() (context.Context, context.CancelFunc) { return context.WithCancel(context.Background()) }, + expectedConnects: 0, + expectErr: false, + }, + { + name: "dials when peers are missing", + connectedPeers: 0, + newCrawler: func(t *testing.T) *mockCrawler { + nodeA := newTestNode(t, "127.0.0.1", 30201) + nodeB := newTestNode(t, "127.0.0.1", 30202) + return &mockCrawler{ + peers: map[gossipsubcrawler.Topic][]*enode.Node{ + "topic/a": {nodeA, nodeB}, + }, + } + }, + targetPeers: 2, + ctx: func() (context.Context, context.CancelFunc) { + return context.WithTimeout(context.Background(), 1*time.Second) + }, + expectedConnects: 2, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + md := &mockDialer{} + var mu sync.Mutex + connected := make([]peer.ID, 0) + for i := 0; i < tt.connectedPeers; i++ { + connected = append(connected, peer.ID(string(rune(i)))) + } + + listPeers := func(topic string) []peer.ID { + mu.Lock() + defer mu.Unlock() + return connected + } + + dialPeers := func(ctx context.Context, max int, nodes []*enode.Node) uint { + cnt := md.DialPeers(ctx, max, nodes) + mu.Lock() + defer mu.Unlock() + for range nodes { + // Just add a dummy peer ID to simulate connection success + connected = append(connected, peer.ID("dummy")) + } + return cnt + } + + crawler := tt.newCrawler(t) + dialer := NewGossipsubPeerDialer(crawler, listPeers, dialPeers) + topic := "topic/a" + + ctx, cancel := tt.ctx() + defer cancel() + + err := dialer.DialPeersForTopicBlocking(ctx, topic, tt.targetPeers) + if tt.expectErr { + require.Error(t, err) + } else { + require.NoError(t, err) + } + + require.Equal(t, tt.expectedConnects, md.dialCount()) + dialer.Stop() + }) + } +} + +func TestGossipsubPeerDialer_peersForTopic(t *testing.T) { + tests := []struct { + name string + connected int + targetCount int + buildPeers func(t *testing.T) ([]*enode.Node, []*enode.Node) + }{ + { + name: "returns nil when enough peers already connected", + connected: 1, + targetCount: 1, + buildPeers: func(t *testing.T) ([]*enode.Node, []*enode.Node) { + return []*enode.Node{newTestNode(t, "127.0.0.1", 30301)}, nil + }, + }, + { + name: "returns crawler peers when none connected", + connected: 0, + targetCount: 2, + buildPeers: func(t *testing.T) ([]*enode.Node, []*enode.Node) { + nodeA := newTestNode(t, "127.0.0.1", 30311) + nodeB := newTestNode(t, "127.0.0.1", 30312) + return []*enode.Node{nodeA, nodeB}, []*enode.Node{nodeA, nodeB} + }, + }, + { + name: "truncates peers when more than needed", + connected: 0, + targetCount: 1, + buildPeers: func(t *testing.T) ([]*enode.Node, []*enode.Node) { + nodeA := newTestNode(t, "127.0.0.1", 30321) + nodeB := newTestNode(t, "127.0.0.1", 30322) + nodeC := newTestNode(t, "127.0.0.1", 30323) + return []*enode.Node{nodeA, nodeB, nodeC}, []*enode.Node{nodeA} + }, + }, + { + name: "only returns missing peers", + connected: 1, + targetCount: 3, + buildPeers: func(t *testing.T) ([]*enode.Node, []*enode.Node) { + nodeA := newTestNode(t, "127.0.0.1", 30331) + nodeB := newTestNode(t, "127.0.0.1", 30332) + nodeC := newTestNode(t, "127.0.0.1", 30333) + return []*enode.Node{nodeA, nodeB, nodeC}, []*enode.Node{nodeA, nodeB} + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + listPeers := func(topic string) []peer.ID { + peers := make([]peer.ID, tt.connected) + for i := 0; i < tt.connected; i++ { + peers[i] = peer.ID(string(rune(i))) // Fake peer ID + } + return peers + } + + crawlerPeers, expected := tt.buildPeers(t) + crawler := &mockCrawler{ + peers: map[gossipsubcrawler.Topic][]*enode.Node{gossipsubcrawler.Topic("topic/test"): crawlerPeers}, + consume: false, + } + dialer := NewGossipsubPeerDialer(crawler, listPeers, func(ctx context.Context, maxConcurrentDials int, nodes []*enode.Node) uint { return 0 }) + + got := dialer.peersForTopic("topic/test", tt.targetCount) + if expected == nil { + require.Nil(t, got) + return + } + + require.Equal(t, len(expected), len(got)) + + for i := range expected { + require.Equal(t, expected[i], got[i]) + } + }) + } +} + +type mockCrawler struct { + mu sync.Mutex + peers map[gossipsubcrawler.Topic][]*enode.Node + consume bool +} + +func (m *mockCrawler) Start(gossipsubcrawler.TopicExtractor) error { + return nil +} + +func (m *mockCrawler) Stop() {} +func (m *mockCrawler) RemovePeerId(peer.ID) {} +func (m *mockCrawler) RemoveTopic(gossipsubcrawler.Topic) {} +func (m *mockCrawler) PeersForTopic(topic gossipsubcrawler.Topic) []*enode.Node { + m.mu.Lock() + defer m.mu.Unlock() + + nodes := m.peers[topic] + if len(nodes) == 0 { + return nil + } + + copied := append([]*enode.Node(nil), nodes...) + if m.consume { + m.peers[topic] = nil + } + return copied +} + +type mockDialer struct { + mu sync.Mutex + dials []*enode.Node +} + +func (m *mockDialer) DialPeers(ctx context.Context, maxConcurrentDials int, nodes []*enode.Node) uint { + m.mu.Lock() + defer m.mu.Unlock() + m.dials = append(m.dials, nodes...) + return uint(len(nodes)) +} + +func (m *mockDialer) dialCount() int { + m.mu.Lock() + defer m.mu.Unlock() + return len(m.dials) +} + +func newTestNode(t *testing.T, ip string, tcpPort uint16) *enode.Node { + priv, _, err := crypto.GenerateSecp256k1Key(rand.Reader) + require.NoError(t, err) + + return newTestNodeWithPriv(t, priv, ip, tcpPort) +} + +func newTestNodeWithPriv(t *testing.T, priv crypto.PrivKey, ip string, tcpPort uint16) *enode.Node { + t.Helper() + + db, err := enode.OpenDB("") + require.NoError(t, err) + t.Cleanup(func() { + db.Close() + }) + + convertedKey, err := ecdsa.ConvertFromInterfacePrivKey(priv) + require.NoError(t, err) + + localNode := enode.NewLocalNode(db, convertedKey) + localNode.SetStaticIP(net.ParseIP(ip)) + localNode.Set(enr.TCP(tcpPort)) + localNode.Set(enr.UDP(tcpPort)) + + return localNode.Node() +} diff --git a/beacon-chain/p2p/service.go b/beacon-chain/p2p/service.go index 06ccac233bc5..f22500e08c4d 100644 --- a/beacon-chain/p2p/service.go +++ b/beacon-chain/p2p/service.go @@ -266,7 +266,7 @@ func (s *Service) Start() { s.crawler = crawler // Initialise the gossipsub dialer which will be started // once the sync service is ready to provide subnet topics. - s.gossipsubDialer = NewGossipsubPeerDialer(s, s.crawler) + s.gossipsubDialer = NewGossipsubPeerDialer(s.crawler, s.PubSub().ListPeers, s.DialPeers) } s.started = true From 86b65e091237b4b4bdc26af0e2ff8692d1895dfe Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Fri, 21 Nov 2025 17:10:35 +0400 Subject: [PATCH 33/39] re-run test --- beacon-chain/sync/rpc_beacon_blocks_by_range_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon-chain/sync/rpc_beacon_blocks_by_range_test.go b/beacon-chain/sync/rpc_beacon_blocks_by_range_test.go index 0b893b50aa70..004fc0af869c 100644 --- a/beacon-chain/sync/rpc_beacon_blocks_by_range_test.go +++ b/beacon-chain/sync/rpc_beacon_blocks_by_range_test.go @@ -77,7 +77,7 @@ func TestRPCBeaconBlocksByRange_RPCHandlerReturnsBlocks(t *testing.T) { res := util.NewBeaconBlock() assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, res)) if res.Block.Slot.SubSlot(req.StartSlot).Mod(1) != 0 { - t.Errorf("Received unexpected block slot %d", res.Block.Slot) + t.Errorf("Received wrong unexpected block slot %d", res.Block.Slot) } } }) From e6f3b636ac49d1e228f88ff8696812902d32a0f9 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Mon, 24 Nov 2025 14:54:23 +0400 Subject: [PATCH 34/39] fix lint --- beacon-chain/p2p/testing/mock_peermanager.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon-chain/p2p/testing/mock_peermanager.go b/beacon-chain/p2p/testing/mock_peermanager.go index ca6278b316e4..0f279b414498 100644 --- a/beacon-chain/p2p/testing/mock_peermanager.go +++ b/beacon-chain/p2p/testing/mock_peermanager.go @@ -4,8 +4,8 @@ import ( "context" "errors" - "github.com/ethereum/go-ethereum/p2p/enode" "github.com/OffchainLabs/prysm/v7/beacon-chain/p2p/gossipsubcrawler" + "github.com/ethereum/go-ethereum/p2p/enode" "github.com/ethereum/go-ethereum/p2p/enr" "github.com/libp2p/go-libp2p/core/host" "github.com/libp2p/go-libp2p/core/peer" From 10804bbb5690ee5eaf564312b7f1bf5dd9304130 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Mon, 24 Nov 2025 14:58:09 +0400 Subject: [PATCH 35/39] fix conflicts --- beacon-chain/sync/gossipsub_topic_family_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/beacon-chain/sync/gossipsub_topic_family_test.go b/beacon-chain/sync/gossipsub_topic_family_test.go index d6e29c32f4fb..6b236c34c659 100644 --- a/beacon-chain/sync/gossipsub_topic_family_test.go +++ b/beacon-chain/sync/gossipsub_topic_family_test.go @@ -2,6 +2,7 @@ package sync import ( "context" + "slices" "testing" p2ptest "github.com/OffchainLabs/prysm/v7/beacon-chain/p2p/testing" @@ -68,7 +69,7 @@ func TestTopicFamiliesForEpoch(t *testing.T) { "BlobTopicFamily-5", } - electraBlobFamilies := append(append([]string{}, denebBlobFamilies...), "BlobTopicFamily-6", "BlobTopicFamily-7") + electraBlobFamilies := append(slices.Clone(denebBlobFamilies), "BlobTopicFamily-6", "BlobTopicFamily-7") fuluFamilies := []string{ "DataColumnTopicFamily", From 11c5c6fb8ba99aeda32c20d088f79bf179143444 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Mon, 24 Nov 2025 15:06:35 +0400 Subject: [PATCH 36/39] fix dynamic families --- .../sync/gossipsub_controller_test.go | 32 +++++++++---------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/beacon-chain/sync/gossipsub_controller_test.go b/beacon-chain/sync/gossipsub_controller_test.go index d9ad2bfe05ed..03afade8cd50 100644 --- a/beacon-chain/sync/gossipsub_controller_test.go +++ b/beacon-chain/sync/gossipsub_controller_test.go @@ -20,46 +20,46 @@ import ( "github.com/ethereum/go-ethereum/p2p/enode" ) -// fakeDynFamily is a test implementation of a dynamic-subnet topic family. -type fakeDynFamily struct { +// testDynFamly is a test implementation of a dynamic-subnet topic family. +type testDynFamly struct { baseGossipsubTopicFamily topics []string name string } -func (f *fakeDynFamily) Name() string { +func (f *testDynFamly) Name() string { return f.name } -func (f *fakeDynFamily) Validator() wrappedVal { +func (f *testDynFamly) Validator() wrappedVal { return nil } -func (f *fakeDynFamily) Handler() subHandler { +func (f *testDynFamly) Handler() subHandler { return noopHandler } -func (f *fakeDynFamily) UnsubscribeAll() { +func (f *testDynFamly) UnsubscribeAll() { f.unsubscribeAll() } -func (f *fakeDynFamily) GetFullTopicString(subnet uint64) string { +func (f *testDynFamly) GetFullTopicString(subnet uint64) string { return fmt.Sprintf("topic-%d", subnet) } -func (f *fakeDynFamily) TopicsToSubscribeForSlot(_ primitives.Slot) []string { +func (f *testDynFamly) TopicsToSubscribeForSlot(_ primitives.Slot) []string { return f.topics } -func (f *fakeDynFamily) ExtractTopicsForNode(_ *enode.Node) ([]string, error) { - return append([]string{}, f.topics...), nil +func (f *testDynFamly) ExtractTopicsForNode(_ *enode.Node) ([]string, error) { + return f.topics, nil } -func (f *fakeDynFamily) SubscribeForSlot(_ primitives.Slot) { +func (f *testDynFamly) SubscribeForSlot(_ primitives.Slot) { f.baseGossipsubTopicFamily.subscribeToTopics(f.topics) } -func (f *fakeDynFamily) UnsubscribeForSlot(_ primitives.Slot) {} +func (f *testDynFamly) UnsubscribeForSlot(_ primitives.Slot) {} type staticTopicFamily struct { *baseGossipsubTopicFamily @@ -296,7 +296,7 @@ func TestGossipsubController_ExtractTopics(t *testing.T) { { name: "single dynamic family topics returned", setup: func(g *GossipsubController) { - fam := &fakeDynFamily{topics: []string{"t1", "t2"}, name: "Dyn1"} + fam := &testDynFamly{topics: []string{"t1", "t2"}, name: "Dyn1"} g.mu.Lock() g.activeTopicFamilies[topicFamilyKey{topicName: "dyn1", forkDigest: [4]byte{0}}] = fam g.mu.Unlock() @@ -309,8 +309,8 @@ func TestGossipsubController_ExtractTopics(t *testing.T) { { name: "multiple dynamic families de-dup", setup: func(g *GossipsubController) { - f1 := &fakeDynFamily{topics: []string{"t1", "t2"}, name: "Dyn1"} - f2 := &fakeDynFamily{topics: []string{"t2", "t3"}, name: "Dyn2"} + f1 := &testDynFamly{topics: []string{"t1", "t2"}, name: "Dyn1"} + f2 := &testDynFamly{topics: []string{"t2", "t3"}, name: "Dyn2"} g.mu.Lock() g.activeTopicFamilies[topicFamilyKey{topicName: "static", forkDigest: [4]byte{1, 2, 3, 4}}] = &staticTopicFamily{name: "StaticFam"} g.activeTopicFamilies[topicFamilyKey{topicName: "dyn1", forkDigest: [4]byte{0}}] = f1 @@ -325,7 +325,7 @@ func TestGossipsubController_ExtractTopics(t *testing.T) { { name: "mixed static and dynamic", setup: func(g *GossipsubController) { - f1 := &fakeDynFamily{topics: []string{"a", "b"}, name: "Dyn"} + f1 := &testDynFamly{topics: []string{"a", "b"}, name: "Dyn"} s1 := &staticTopicFamily{name: "Static"} g.mu.Lock() g.activeTopicFamilies[topicFamilyKey{topicName: "dyn", forkDigest: [4]byte{9}}] = f1 From 09d886c676754b6cf4c0ad8033ae8b4f42feffed Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Mon, 24 Nov 2025 15:18:27 +0400 Subject: [PATCH 37/39] fix test --- beacon-chain/p2p/broadcaster_test.go | 2 +- beacon-chain/p2p/gossipsub_peer_controller_test.go | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/beacon-chain/p2p/broadcaster_test.go b/beacon-chain/p2p/broadcaster_test.go index 02e90bf600a6..880bbc8ba682 100644 --- a/beacon-chain/p2p/broadcaster_test.go +++ b/beacon-chain/p2p/broadcaster_test.go @@ -267,7 +267,7 @@ func TestService_BroadcastAttestationWithDiscoveryAttempts(t *testing.T) { // Create a new config for each service to avoid shared mutations cfg := &Config{ - Discv5BootStrapAddrs: append([]string{}, baseCfg.Discv5BootStrapAddrs...), + Discv5BootStrapAddrs: baseCfg.Discv5BootStrapAddrs, MaxPeers: baseCfg.MaxPeers, PingInterval: baseCfg.PingInterval, DB: baseCfg.DB, diff --git a/beacon-chain/p2p/gossipsub_peer_controller_test.go b/beacon-chain/p2p/gossipsub_peer_controller_test.go index 63fc4658becf..e87ce141ebd9 100644 --- a/beacon-chain/p2p/gossipsub_peer_controller_test.go +++ b/beacon-chain/p2p/gossipsub_peer_controller_test.go @@ -4,6 +4,7 @@ import ( "context" "crypto/rand" "net" + "slices" "sync" "testing" "time" @@ -265,7 +266,7 @@ func (m *mockCrawler) PeersForTopic(topic gossipsubcrawler.Topic) []*enode.Node return nil } - copied := append([]*enode.Node(nil), nodes...) + copied := slices.Clone(nodes) if m.consume { m.peers[topic] = nil } From affdab77765e2a870d73914f992ad371d6414806 Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Mon, 24 Nov 2025 17:49:47 +0400 Subject: [PATCH 38/39] fix changelog --- beacon-chain/sync/rpc_beacon_blocks_by_range_test.go | 2 +- changelog/aarshkshah1992-gossipsub-control-pane.md | 3 +-- changelog/aarshkshah1992-gossipsub-peer-crawler.md | 3 --- 3 files changed, 2 insertions(+), 6 deletions(-) delete mode 100644 changelog/aarshkshah1992-gossipsub-peer-crawler.md diff --git a/beacon-chain/sync/rpc_beacon_blocks_by_range_test.go b/beacon-chain/sync/rpc_beacon_blocks_by_range_test.go index 37e2a5cae4a3..c279087f5a22 100644 --- a/beacon-chain/sync/rpc_beacon_blocks_by_range_test.go +++ b/beacon-chain/sync/rpc_beacon_blocks_by_range_test.go @@ -77,7 +77,7 @@ func TestRPCBeaconBlocksByRange_RPCHandlerReturnsBlocks(t *testing.T) { res := util.NewBeaconBlock() assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, res)) if res.Block.Slot.SubSlot(req.StartSlot).Mod(1) != 0 { - t.Errorf("Received wrong unexpected block slot %d", res.Block.Slot) + t.Errorf("Received unexpected block slot %d", res.Block.Slot) } } }) diff --git a/changelog/aarshkshah1992-gossipsub-control-pane.md b/changelog/aarshkshah1992-gossipsub-control-pane.md index a369fb02d1eb..36d639d280f6 100644 --- a/changelog/aarshkshah1992-gossipsub-control-pane.md +++ b/changelog/aarshkshah1992-gossipsub-control-pane.md @@ -1,4 +1,3 @@ ### Added -- A Gossipsub controller that uses well defined inetrfaces and implementations for ALL topics (subnetted and otherwise) -that Prysm subscribes to to the hide the gnarly details of gossipsub subscription management. \ No newline at end of file +- A Gossipsub control pane with topic abstractions, a peer crawler and a peer controller. \ No newline at end of file diff --git a/changelog/aarshkshah1992-gossipsub-peer-crawler.md b/changelog/aarshkshah1992-gossipsub-peer-crawler.md deleted file mode 100644 index 1145054f46a6..000000000000 --- a/changelog/aarshkshah1992-gossipsub-peer-crawler.md +++ /dev/null @@ -1,3 +0,0 @@ -### Added - -- A discV5 peer crawler for Gossipsub. \ No newline at end of file From 2061fc8a2ff26999e3135c3306ad91dcc9af19bf Mon Sep 17 00:00:00 2001 From: aarshkshah1992 Date: Tue, 25 Nov 2025 15:49:37 +0400 Subject: [PATCH 39/39] design doc --- .../docs/gossipsub_control_plane_design.md | 398 ++++++++++++++++++ 1 file changed, 398 insertions(+) create mode 100644 beacon-chain/sync/docs/gossipsub_control_plane_design.md diff --git a/beacon-chain/sync/docs/gossipsub_control_plane_design.md b/beacon-chain/sync/docs/gossipsub_control_plane_design.md new file mode 100644 index 000000000000..b80dcafe3c84 --- /dev/null +++ b/beacon-chain/sync/docs/gossipsub_control_plane_design.md @@ -0,0 +1,398 @@ +# Gossipsub Control Plane Design Document + +## Overview + +This branch introduces a declarative, fork-aware gossipsub control plane that manages topic subscriptions and peer discovery for subnet-based topics. The system replaces ad-hoc topic management with a structured approach centered on **Topic Families**. + +### Key Components + +| Component | Location | Responsibility | +|-----------|----------|----------------| +| **GossipsubController** | `sync/gossipsub_controller.go` | Orchestrates topic family lifecycle across forks | +| **GossipsubPeerCrawler** | `p2p/gossipsub_peer_crawler.go` | Discovers and indexes peers by topic via discv5 | +| **GossipsubPeerDialer** | `p2p/gossipsub_peer_controller.go` | Maintains peer connections for required topics | +| **Topic Family Abstractions** | `sync/gossipsub_topic_family.go` | Interfaces for topic subscription management | + +--- + +## 1. Topic Family Abstraction + +### 1.1 Design Goals + +- **Declarative Fork Management**: Topic families declare when they activate/deactivate based on fork epochs +- **Unified Subscription Logic**: Common base handles validator registration, message loops, and cleanup +- **Dynamic vs Static Distinction**: Clear separation between global topics and subnet-based topics that change per slot + +### 1.2 Interface Hierarchy + +``` +GossipsubTopicFamily (base) +├── Name() +├── NetworkScheduleEntry() +└── UnsubscribeAll() + +GossipsubTopicFamilyWithoutDynamicSubnets +└── Subscribe() // Called once when registered + +GossipsubTopicFamilyWithDynamicSubnets +├── TopicsToSubscribeForSlot(slot) +├── ExtractTopicsForNode(node) // For peer discovery +├── SubscribeForSlot(slot) +└── UnsubscribeForSlot(slot) +``` + +### 1.3 Implementation Categories + +**Global Topics** (subscribed once per fork): +- Block, AggregateAndProof, VoluntaryExit, ProposerSlashing, AttesterSlashing +- SyncContributionAndProof (Altair+), BlsToExecutionChange (Capella+) +- LightClient updates (Altair+, feature-flagged) + +**Static Per-Subnet**: +- BlobTopicFamily - One instance per blob subnet (Deneb/Electra) + +**Dynamic Subnets** (change per slot based on validator duties): +- **AttestationTopicFamily** - Subnets based on attestation committee assignments +- **SyncCommitteeTopicFamily** - Subnets based on sync committee membership +- **DataColumnTopicFamily** - Subnets based on data column custody (Fulu+) + +### 1.4 Base Implementation Features + +`baseGossipsubTopicFamily` provides: +- **Idempotent subscriptions** - Safe to call multiple times for same topic +- **Automatic validator registration** - Registers message validator with pubsub +- **Message loop management** - Spawns goroutine to process incoming messages +- **Cleanup coordination** - Notifies crawler when topics are unsubscribed + +### 1.5 Dynamic Subnet Selection + +Dynamic families combine two subnet sources: +- **Subnets to Join**: Topics we must subscribe to (persistent duties, aggregator responsibilities) +- **Subnets for Broadcast**: Topics we need peers for but may not subscribe to + +| Family | Subnets to Join | Subnets for Broadcast | +|--------|-----------------|----------------------| +| Attestation | Persistent + aggregator subnets | Attester duty subnets | +| SyncCommittee | Active sync committee subnets | (none) | +| DataColumn | Custody column subnets | All column subnets | + +### 1.6 Fork Schedule + +Topic families declare activation and optional deactivation epochs: + +| Fork | Activations | Deactivations | +|------|-------------|---------------| +| Genesis | Block, AggregateAndProof, VoluntaryExit, ProposerSlashing, AttesterSlashing, Attestation | - | +| Altair | SyncContributionAndProof, SyncCommittee, [LightClient*] | - | +| Capella | BlsToExecutionChange | - | +| Deneb | Blob (6 subnets) | - | +| Electra | Blob (9 subnets) | Blob (Deneb config) | +| Fulu | DataColumn | Blob (all) | + +--- + +## 2. GossipsubController + +### 2.1 Responsibilities + +- **Fork-Aware Topic Management**: Automatically subscribes/unsubscribes based on fork schedule +- **Smooth Fork Transitions**: Pre-subscribes 1 epoch before fork, unsubscribes 1 epoch after +- **Slot-Based Updates**: Updates dynamic subnet subscriptions every slot +- **Topic Extraction**: Provides interface for crawler to determine peer topic relevance + +### 2.2 Lifecycle + +1. **Startup**: Waits for initial sync to complete +2. **Control Loop**: Runs on slot ticker, calling `updateActiveTopicFamilies()` +3. **Shutdown**: Unsubscribes all families, cancels context + +### 2.3 Fork Transition Handling + +**Timeline for Fork at Epoch N:** +``` +Epoch N-1: Subscribe to both old and new fork topics (overlap period) +Epoch N: Fork occurs, both topic sets remain active +Epoch N+1: Unsubscribe from old fork topics, only new fork active +``` + +This ensures no message loss during the transition window. + +### 2.4 Update Logic (per slot) + +1. **Get families for current epoch** from declarative schedule +2. **Check for upcoming fork** - if next epoch is fork boundary, include next fork's families +3. **Register new families** - add to active map, subscribe based on type: + - Static families: `Subscribe()` once + - Dynamic families: `SubscribeForSlot()` and `UnsubscribeForSlot()` every slot +4. **Remove old fork families** - if 1 epoch past fork boundary, unsubscribe and remove + +### 2.5 Topic Extraction for Peer Discovery + +The controller exposes `ExtractTopics(node)` which: +- Iterates all active **dynamic** subnet families +- Calls `ExtractTopicsForNode(node)` on each +- Returns deduplicated list of topics the node can serve + +This is used by the peer crawler to index discovered peers by topic. + +### 2.6 Topics Provider + +The controller exposes `GetCurrentActiveTopics()` which: +- Returns all topics from dynamic families for the current slot +- Used by the peer dialer to know which topics need peer connections + +--- + +## 3. GossipsubPeerCrawler + +### 3.1 Purpose + +Discovers peers via discv5, indexes them by topic, and verifies reachability via ping. Provides the peer dialer with a pool of verified, scored peers for each topic. + +### 3.2 Key Design Decisions + +**Triple Index Structure:** +- `byEnode` - Fast lookup by enode ID +- `byPeerId` - Fast lookup by libp2p peer ID +- `byTopic` - Fast lookup of peers serving a topic + +**Ping-Once Guarantee:** +- A node is pinged exactly **once** regardless of ENR sequence number updates +- Prevents ping explosion when nodes frequently update their records +- Ping success sets `isPinged=true`, failure removes peer entirely + +**Sequence Number Handling:** +- Only updates peer record if new sequence number is higher +- Stale records are ignored to prevent processing outdated data + +### 3.3 Three Concurrent Loops + +| Loop | Interval | Purpose | +|------|----------|---------| +| **crawlLoop** | `crawlInterval` | Iterates discv5 `RandomNodes()`, extracts topics, updates index | +| **pingLoop** | Continuous | Consumes ping queue, verifies reachability | +| **cleanupLoop** | 5 minutes | Prunes peers that fail filter or have no relevant topics | + +### 3.4 Crawl Flow + +1. Create timeout context for crawl iteration +2. Get random nodes iterator from discv5 +3. For each node: + - Apply peer filter (reject bad/incompatible peers) + - Extract topics via `topicExtractor` (provided by controller) + - Update index if sequence number is newer + - Queue for ping if not already pinged and has topics + +### 3.5 Ping Queue and Backpressure + +- **Channel capacity**: `4 * maxConcurrentPings` +- **Backpressure**: When queue is full, crawl loop blocks on send +- **Semaphore**: Limits concurrent ping goroutines to `maxConcurrentPings` +- **Ping failure**: Removes peer from index entirely (unreachable) +- **Ping success**: Marks peer as verified (`isPinged=true`) + +### 3.6 Peer Retrieval (`PeersForTopic`) + +Returns peers for a topic with guarantees: +1. **Only pinged peers** - Verified reachable +2. **Filter applied** - Passes current peer filter +3. **Sorted by score** - Best peers first (using p2p scorer) + +### 3.7 Peer Removal Triggers + +| Trigger | Behavior | +|---------|----------| +| Ping failure | Remove immediately | +| Peer disconnection | `RemovePeerId()` called from disconnect handler | +| Topic unsubscription | `RemoveTopic()` called from base family cleanup | +| Filter rejection during crawl | Remove if previously indexed | +| Cleanup loop | Remove if no longer passes filter or has no topics | + +### 3.8 Topic Extraction for Dynamic Subnets + +For each dynamic family, extraction: +1. Gets subnets we currently need (union of join + broadcast) +2. Reads subnet bitfield from node's ENR record +3. Returns intersection - topics both we need AND the node advertises + +--- + +## 4. GossipsubPeerDialer + +### 4.1 Purpose + +Maintains peer connections for topics we need. Works with the crawler to dial verified peers when topic peer counts fall below threshold. + +### 4.2 Key Design Decisions + +**Target Peer Count**: 20 peers per topic (`peerPerTopic` constant) + +**Dial Loop Frequency**: Every 1 second + +**Deduplication**: Peers appearing for multiple topics are only dialed once + +### 4.3 Dial Flow + +1. Get current topics from `topicsProvider` (controller's `GetCurrentActiveTopics`) +2. For each topic: + - Check current connected peer count via `listPeersFunc` + - If below target, calculate how many more needed + - Get peers from crawler (already filtered, scored, pinged) + - Limit to what's needed +3. Deduplicate peer list across all topics +4. Dial peers via `dialPeersFunc` + +### 4.4 Blocking Dial + +`DialPeersForTopicBlocking(ctx, topic, nPeers)` provides synchronous peer acquisition: +- Loops until target peer count reached or context cancelled +- Used for critical operations that need guaranteed peer connectivity +- Polls every 100ms to check connection status + +--- + +## 5. Component Interactions + +### 5.1 Architecture Diagram + +``` +┌─────────────────────────────────────────────────────────────────────────────┐ +│ Sync Service │ +│ ┌───────────────────────────────────────────────────────────────────────┐ │ +│ │ GossipsubController │ │ +│ │ │ │ +│ │ ┌─────────────────┐ ┌─────────────────┐ ┌─────────────────┐ │ │ +│ │ │ AttestationTF │ │ SyncCommitteeTF │ │ DataColumnTF │ │ │ +│ │ │ (dynamic) │ │ (dynamic) │ │ (dynamic) │ │ │ +│ │ └─────────────────┘ └─────────────────┘ └─────────────────┘ │ │ +│ │ │ │ +│ │ ┌─────────────────┐ ┌─────────────────┐ ┌─────────────────┐ │ │ +│ │ │ BlockTF, etc. │ │ BlobTF (static) │ │ baseTopicFamily │ │ │ +│ │ │ (global) │ │ │ │ (shared logic) │ │ │ +│ │ └─────────────────┘ └─────────────────┘ └─────────────────┘ │ │ +│ │ │ │ +│ └──────────────────┬─────────────────────────────┬──────────────────────┘ │ +│ │ │ │ +│ GetCurrentActiveTopics() ExtractTopics() │ +│ │ │ │ +└─────────────────────┼─────────────────────────────┼─────────────────────────┘ + │ │ + ▼ ▼ +┌─────────────────────────────────┐ ┌─────────────────────────────────────┐ +│ GossipsubPeerDialer │ │ GossipsubPeerCrawler │ +│ │ │ │ +│ - Polls topics every 1 second │ │ - Crawls discv5 periodically │ +│ - Checks peer count per topic │ │ - Indexes peers by topic │ +│ - Dials missing peers │ │ - Verifies via ping │ +│ │ │ - Filters and scores peers │ +│ │ │ │ │ +│ │ PeersForTopic() │ │ │ │ +│ └───────────────────────┼──┼─────────┘ │ +│ │ │ │ +└─────────────────────────────────┘ └──────────────────┬──────────────────┘ + │ + │ RemovePeerId() + ┌──────────────────┘ + │ + ▼ + ┌─────────────────────────────────┐ + │ P2P Service │ + │ │ + │ - Disconnect handler calls │ + │ RemovePeerId() on crawler │ + │ - Provides filterPeer, scorer │ + └─────────────────────────────────┘ +``` + +### 5.2 Data Flow Summary + +| Flow | Description | +|------|-------------| +| **Discovery** | discv5 → crawlLoop → topicExtractor → crawledPeers index → pingCh | +| **Ping** | pingCh → semaphore → dv5.Ping() → isPinged=true or remove | +| **Dial** | controller topics → dialer → crawler.PeersForTopic() → dialPeers | +| **Cleanup** | disconnect/unsubscribe → RemovePeerId()/RemoveTopic() | + +### 5.3 Key Invariants + +**Peers from `PeersForTopic()` are always:** +- Successfully pinged (reachable) +- Passing the peer filter +- Sorted by score (best first) + +**Topic subscriptions are:** +- Pre-subscribed 1 epoch before fork +- Unsubscribed 1 epoch after fork +- Updated every slot for dynamic families + +**Ping behavior:** +- Each node ID pinged at most once +- Ping failures remove peer entirely +- Sequence number updates don't trigger re-ping + +**Backpressure:** +- Ping queue blocks crawl when full +- Semaphore limits concurrent pings +- Natural rate limiting without explicit throttling + +--- + +## 6. Initialization Sequence + +``` +PHASE 1: P2P Service Start +══════════════════════════ + ├─► Start discv5 listener + ├─► Create GossipsubPeerCrawler (with filterPeer, scorer) + └─► Create GossipsubPeerDialer (not started yet) + +PHASE 2: Sync Service Start +═══════════════════════════ + ├─► Create GossipsubController + └─► Launch startDiscoveryAndSubscriptions goroutine + +PHASE 3: Discovery and Subscriptions (after chain start) +════════════════════════════════════════════════════════ + ├─► Start GossipsubController (control loop) + ├─► Start Crawler with topicExtractor from controller + └─► Start Dialer with topicsProvider from controller +``` + +### Dependency Injection + +| Component | Dependencies | Provider | +|-----------|-------------|----------| +| Crawler | discv5, filterPeer, scorer | P2P Service | +| Crawler | topicExtractor | GossipsubController | +| Dialer | crawler, listPeers, dialPeers | P2P Service | +| Dialer | topicsProvider | GossipsubController | + +--- + +## 7. Configuration Parameters + +| Parameter | Default | Description | +|-----------|---------|-------------| +| `crawlInterval` | configurable | How often to crawl discv5 | +| `crawlTimeout` | configurable | Max duration per crawl iteration | +| `maxConcurrentPings` | configurable | Parallel ping limit | +| `cleanupInterval` | 5 minutes | Stale peer pruning frequency | +| `peerPerTopic` | 20 | Target peer count per topic | +| `dialLoop interval` | 1 second | Topic peer check frequency | + +--- + +## 8. Key Files + +| File | Purpose | +|------|---------| +| `sync/gossipsub_controller.go` | Controller orchestrating topic families | +| `sync/gossipsub_topic_family.go` | Interface definitions and fork schedule | +| `sync/gossipsub_base.go` | Base implementation for all topic families | +| `sync/topic_families_without_subnets.go` | Global topic family implementations | +| `sync/topic_families_static_subnets.go` | Blob topic family | +| `sync/topic_families_dynamic_subnets.go` | Dynamic subnet families | +| `p2p/gossipsub_peer_crawler.go` | Peer discovery and indexing | +| `p2p/gossipsub_peer_controller.go` | Peer dialing logic | +| `p2p/gossipsubcrawler/interface.go` | Shared interfaces | +| `p2p/handshake.go` | Disconnect handler integration |