mirror of
https://github.com/OffchainLabs/prysm.git
synced 2026-01-10 13:58:09 -05:00
Compare commits
26 Commits
eas-bug
...
feat/peer-
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
1ff836e549 | ||
|
|
08f038fe80 | ||
|
|
63279bcadf | ||
|
|
61628efd44 | ||
|
|
9b07f13cd3 | ||
|
|
1397a79b4c | ||
|
|
929115639d | ||
|
|
14dca40786 | ||
|
|
fa7596bceb | ||
|
|
5161f087fc | ||
|
|
71050ab076 | ||
|
|
614367ddcf | ||
|
|
3f7371445b | ||
|
|
a15a1ade17 | ||
|
|
798376b1d7 | ||
|
|
93271050bf | ||
|
|
8dfbabc691 | ||
|
|
af2522e5f0 | ||
|
|
452d42bd10 | ||
|
|
3e985377ce | ||
|
|
ab2e836d3f | ||
|
|
14158bea9c | ||
|
|
e14590636f | ||
|
|
ce3660d2e7 | ||
|
|
7853cb9db0 | ||
|
|
8cfeda1473 |
@@ -15,6 +15,8 @@ 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",
|
||||
"interfaces.go",
|
||||
@@ -51,6 +53,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",
|
||||
@@ -114,6 +117,7 @@ go_library(
|
||||
"@com_github_prysmaticlabs_go_bitfield//:go_default_library",
|
||||
"@com_github_sirupsen_logrus//:go_default_library",
|
||||
"@org_golang_google_protobuf//proto:go_default_library",
|
||||
"@org_golang_x_sync//semaphore:go_default_library",
|
||||
],
|
||||
)
|
||||
|
||||
@@ -129,6 +133,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",
|
||||
@@ -154,6 +159,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",
|
||||
@@ -201,6 +207,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",
|
||||
],
|
||||
)
|
||||
|
||||
@@ -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 {
|
||||
topicStr := attestationToTopic(subnet, forkDigest) + s.Encoding().ProtocolSuffix()
|
||||
if err := s.gossipsubDialer.DialPeersForTopicBlocking(topicStr, minimumPeersPerSubnetForBroadcast); 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 {
|
||||
topicStr := syncCommitteeToTopic(subnet, forkDigest) + s.Encoding().ProtocolSuffix()
|
||||
if err := s.gossipsubDialer.DialPeersForTopicBlocking(topicStr, minimumPeersPerSubnetForBroadcast); 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 {
|
||||
topicStr := blobSubnetToTopic(subnet, forkDigest) + s.Encoding().ProtocolSuffix()
|
||||
if err := s.gossipsubDialer.DialPeersForTopicBlocking(topicStr, minimumPeersPerSubnetForBroadcast); 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,
|
||||
@@ -496,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, topicFormat, forkDigest, 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
|
||||
}
|
||||
|
||||
|
||||
@@ -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]
|
||||
|
||||
|
||||
@@ -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
|
||||
@@ -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)
|
||||
|
||||
129
beacon-chain/p2p/gossipsub_peer_controller.go
Normal file
129
beacon-chain/p2p/gossipsub_peer_controller.go
Normal file
@@ -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()
|
||||
}
|
||||
}
|
||||
}
|
||||
511
beacon-chain/p2p/gossipsub_peer_crawler.go
Normal file
511
beacon-chain/p2p/gossipsub_peer_crawler.go
Normal file
@@ -0,0 +1,511 @@
|
||||
package p2p
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sort"
|
||||
"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 peerNode 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
|
||||
byEnode map[enode.ID]*peerNode
|
||||
byPeerId map[peer.ID]*peerNode
|
||||
byTopic map[gossipsubcrawler.Topic]map[peer.ID]struct{}
|
||||
}
|
||||
|
||||
func (cp *crawledPeers) updateStatusToPinged(enodeID enode.ID) {
|
||||
cp.mu.Lock()
|
||||
defer cp.mu.Unlock()
|
||||
|
||||
existingPNode, ok := cp.byEnode[enodeID]
|
||||
if !ok {
|
||||
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(enodeID enode.ID) {
|
||||
cp.mu.Lock()
|
||||
defer cp.mu.Unlock()
|
||||
|
||||
existingPNode, ok := cp.byEnode[enodeID]
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
|
||||
// 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) {
|
||||
cp.mu.Lock()
|
||||
|
||||
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")
|
||||
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)
|
||||
|
||||
if existingPNode.isPinged || len(topics) == 0 {
|
||||
cp.mu.Unlock()
|
||||
return
|
||||
}
|
||||
cp.mu.Unlock()
|
||||
|
||||
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 peer if it has no more topics left
|
||||
if len(pnode.topics) == 0 {
|
||||
cp.updateTopicsUnlocked(pnode, nil)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// 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
|
||||
|
||||
peerFilter gossipsubcrawler.PeerFilterFunc
|
||||
scorer PeerScoreFunc
|
||||
|
||||
maxConcurrentPings int
|
||||
pingCh chan enode.Node
|
||||
pingSemaphore *semaphore.Weighted
|
||||
|
||||
wg sync.WaitGroup
|
||||
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")
|
||||
}
|
||||
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")
|
||||
}
|
||||
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{
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
crawlInterval: crawlInterval,
|
||||
crawlTimeout: crawlTimeout,
|
||||
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))
|
||||
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 peerNodes []*peerNode
|
||||
for peerID := range peerIDs {
|
||||
peerNode, ok := g.crawledPeers.byPeerId[peerID]
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
if peerNode.isPinged && g.peerFilter(peerNode.node) {
|
||||
peerNodes = append(peerNodes, peerNode)
|
||||
}
|
||||
}
|
||||
|
||||
// Sort peerNodes in descending order of their scores.
|
||||
sort.Slice(peerNodes, func(i, j int) bool {
|
||||
scoreI := g.scorer(peerNodes[i].peerID)
|
||||
scoreJ := g.scorer(peerNodes[j].peerID)
|
||||
return scoreI > scoreJ
|
||||
})
|
||||
|
||||
nodes := make([]*enode.Node, len(peerNodes))
|
||||
for i, pn := range peerNodes {
|
||||
nodes[i] = pn.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()
|
||||
})
|
||||
g.wg.Go(func() {
|
||||
g.cleanupLoop()
|
||||
})
|
||||
})
|
||||
|
||||
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.ID())
|
||||
return
|
||||
}
|
||||
|
||||
g.crawledPeers.updateStatusToPinged(node.ID())
|
||||
}(&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.peerFilter(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)
|
||||
}
|
||||
}
|
||||
|
||||
// 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)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
if info == nil {
|
||||
return "", errors.New("peer info is nil")
|
||||
}
|
||||
return info.ID, nil
|
||||
}
|
||||
781
beacon-chain/p2p/gossipsub_peer_crawler_test.go
Normal file
781
beacon-chain/p2p/gossipsub_peer_crawler_test.go
Normal file
@@ -0,0 +1,781 @@
|
||||
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 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()
|
||||
}
|
||||
|
||||
// Ensure both A nodes have the same enode.ID but differing seq
|
||||
ln := createTestNodeRandom(t)
|
||||
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
|
||||
require.NoError(t, 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")
|
||||
}
|
||||
}
|
||||
}
|
||||
14
beacon-chain/p2p/gossipsubcrawler/BUILD.bazel
Normal file
14
beacon-chain/p2p/gossipsubcrawler/BUILD.bazel
Normal file
@@ -0,0 +1,14 @@
|
||||
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",
|
||||
],
|
||||
)
|
||||
39
beacon-chain/p2p/gossipsubcrawler/interface.go
Normal file
39
beacon-chain/p2p/gossipsubcrawler/interface.go
Normal file
@@ -0,0 +1,39 @@
|
||||
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)
|
||||
|
||||
// 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()
|
||||
RemovePeerId(peerID peer.ID)
|
||||
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
|
||||
}
|
||||
@@ -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...
|
||||
|
||||
@@ -4,8 +4,8 @@ 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"
|
||||
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"
|
||||
@@ -98,11 +98,13 @@ 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, topicFormat string, digest [fieldparams.VersionLength]byte, 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
|
||||
}
|
||||
|
||||
// Sender abstracts the sending functionality from libp2p.
|
||||
|
||||
@@ -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"
|
||||
@@ -61,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.
|
||||
@@ -95,6 +100,8 @@ type Service struct {
|
||||
custodyInfoLock sync.RWMutex // Lock access to custodyInfo
|
||||
custodyInfoSet chan struct{}
|
||||
allForkDigests map[[4]byte]struct{}
|
||||
crawler gossipsubcrawler.Crawler
|
||||
gossipsubDialer gossipsubcrawler.GossipsubDialer
|
||||
}
|
||||
|
||||
type custodyInfo struct {
|
||||
@@ -241,6 +248,24 @@ func (s *Service) Start() {
|
||||
|
||||
s.dv5Listener = listener
|
||||
go s.listenForNewNodes()
|
||||
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
|
||||
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
|
||||
@@ -311,12 +336,25 @@ 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
|
||||
}
|
||||
|
||||
// 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 {
|
||||
@@ -557,3 +595,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 AttestationSubnets(nodeID enode.ID, node *enode.Node, record *enr.Record) (map[uint64]bool, error) {
|
||||
return attestationSubnets(record)
|
||||
}
|
||||
|
||||
func SyncSubnets(nodeID enode.ID, node *enode.Node, record *enr.Record) (map[uint64]bool, error) {
|
||||
return syncSubnets(record)
|
||||
}
|
||||
|
||||
func DataColumnSubnets(nodeID enode.ID, node *enode.Node, record *enr.Record) (map[uint64]bool, error) {
|
||||
return dataColumnSubnets(nodeID, record)
|
||||
}
|
||||
|
||||
@@ -3,7 +3,6 @@ package p2p
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"math"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
@@ -13,13 +12,11 @@ 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"
|
||||
"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,
|
||||
topicFormat string,
|
||||
digest [fieldparams.VersionLength]byte,
|
||||
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(topicFormat, digest, minimumPeersPerSubnet, subnets)
|
||||
for len(defectiveSubnets) > 0 {
|
||||
// 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, topicFormat, digest, minimumPeersPerSubnet, defectiveSubnets)
|
||||
if err != nil && !errors.Is(err, context.DeadlineExceeded) {
|
||||
return nil, errors.Wrap(err, "find peers with subnets")
|
||||
}
|
||||
|
||||
return peersToDial, nil
|
||||
}()
|
||||
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Dial new peers in batches.
|
||||
s.dialPeers(s.ctx, maxConcurrentDials, peersToDial)
|
||||
|
||||
defectiveSubnets = s.defectiveSubnets(topicFormat, digest, minimumPeersPerSubnet, subnets)
|
||||
}
|
||||
|
||||
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).
|
||||
@@ -158,8 +96,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 +118,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 +168,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 +184,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 +201,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 {
|
||||
@@ -276,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)
|
||||
|
||||
@@ -117,6 +117,21 @@ func TestStartDiscV5_FindAndDialPeersWithSubnet(t *testing.T) {
|
||||
// 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()
|
||||
bitV.SetBitAt(i, true)
|
||||
@@ -162,22 +177,43 @@ func TestStartDiscV5_FindAndDialPeersWithSubnet(t *testing.T) {
|
||||
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
|
||||
})
|
||||
defer func() {
|
||||
err := service.Stop()
|
||||
require.NoError(t, err)
|
||||
}()
|
||||
|
||||
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()
|
||||
var topicsToDial []string
|
||||
for s := range subnets {
|
||||
topicsToDial = append(topicsToDial, builder(s))
|
||||
}
|
||||
|
||||
err = service.FindAndDialPeersWithSubnets(ctxWithTimeOut, AttestationSubnetTopicFormat, bootNodeForkDigest, minimumPeersPerSubnet, subnets)
|
||||
require.NoError(t, err)
|
||||
for _, topic := range topicsToDial {
|
||||
err = service.GossipsubDialer().DialPeersForTopicBlocking(topic, minimumPeersPerSubnet)
|
||||
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 +798,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 +1020,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 +1145,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
|
||||
)
|
||||
|
||||
@@ -21,9 +21,9 @@ 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/fieldparams:go_default_library",
|
||||
"//config/params:go_default_library",
|
||||
"//consensus-types/blocks:go_default_library",
|
||||
"//consensus-types/interfaces:go_default_library",
|
||||
|
||||
@@ -4,8 +4,8 @@ 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"
|
||||
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"
|
||||
@@ -41,6 +41,16 @@ func (*FakeP2P) AddConnectionHandler(_, _ func(ctx context.Context, id peer.ID)
|
||||
|
||||
}
|
||||
|
||||
// Crawler -- fake.
|
||||
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) {
|
||||
}
|
||||
@@ -71,7 +81,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
|
||||
}
|
||||
|
||||
@@ -93,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
|
||||
|
||||
@@ -4,7 +4,7 @@ import (
|
||||
"context"
|
||||
"errors"
|
||||
|
||||
fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams"
|
||||
"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"
|
||||
@@ -58,9 +58,14 @@ 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
|
||||
}
|
||||
|
||||
// AddPingMethod .
|
||||
func (*MockPeerManager) AddPingMethod(_ func(ctx context.Context, id peer.ID) error) {}
|
||||
|
||||
// Crawler.
|
||||
func (*MockPeerManager) Crawler() gossipsubcrawler.Crawler {
|
||||
return nil
|
||||
}
|
||||
|
||||
@@ -13,9 +13,9 @@ 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"
|
||||
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"
|
||||
@@ -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.
|
||||
@@ -183,11 +184,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)
|
||||
}
|
||||
@@ -279,6 +276,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
|
||||
@@ -419,8 +419,12 @@ 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, 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
|
||||
}
|
||||
|
||||
@@ -558,3 +562,40 @@ 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{}
|
||||
}
|
||||
|
||||
// GossipsubDialer returns nil for tests that do not exercise dialer behaviour.
|
||||
func (p *TestP2P) GossipsubDialer() gossipsubcrawler.GossipsubDialer {
|
||||
return p.dialer
|
||||
}
|
||||
|
||||
@@ -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",
|
||||
@@ -97,6 +102,7 @@ go_library(
|
||||
"//beacon-chain/operations/voluntaryexits: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",
|
||||
"//beacon-chain/p2p/types:go_default_library",
|
||||
"//beacon-chain/slasher/types:go_default_library",
|
||||
@@ -135,6 +141,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",
|
||||
@@ -172,6 +180,8 @@ 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",
|
||||
"pending_attestations_queue_bucket_test.go",
|
||||
@@ -283,6 +293,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",
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -2,8 +2,6 @@ package sync
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
@@ -50,30 +48,6 @@ func testForkWatcherService(t *testing.T, current primitives.Epoch) *Service {
|
||||
return r
|
||||
}
|
||||
|
||||
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.ensureRegistrationsForEpoch(fulu))
|
||||
}
|
||||
|
||||
func TestService_CheckForNextEpochFork(t *testing.T) {
|
||||
closedChan := make(chan struct{})
|
||||
close(closedChan)
|
||||
@@ -103,7 +77,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 +84,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)
|
||||
@@ -127,14 +98,10 @@ func TestService_CheckForNextEpochFork(t *testing.T) {
|
||||
{
|
||||
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 +110,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 +122,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 },
|
||||
@@ -194,54 +146,28 @@ 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.ensureRegistrationsForEpoch(s.cfg.clock.CurrentEpoch()))
|
||||
wg.Wait()
|
||||
require.NoError(t, s.ensureRPCRegistrationsForEpoch(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
|
||||
wg = attachSpawner(s)
|
||||
s.cfg.clock = defaultClockWithTimeAtEpoch(tt.nextForkEpoch - 1)
|
||||
require.NoError(t, s.ensureRegistrationsForEpoch(s.cfg.clock.CurrentEpoch()))
|
||||
wg.Wait()
|
||||
require.NoError(t, s.ensureRPCRegistrationsForEpoch(s.cfg.clock.CurrentEpoch()))
|
||||
|
||||
require.NoError(t, s.ensureDeregistrationForEpoch(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))
|
||||
assert.Equal(t, false, s.subHandler.digestExists(digest))
|
||||
assert.Equal(t, true, s.subHandler.digestExists(nextDigest))
|
||||
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
|
||||
|
||||
208
beacon-chain/sync/gossipsub_controller.go
Normal file
208
beacon-chain/sync/gossipsub_controller.go
Normal file
@@ -0,0 +1,208 @@
|
||||
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/ethereum/go-ethereum/p2p/enode"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
type topicFamilyKey struct {
|
||||
topicName string
|
||||
forkDigest [4]byte
|
||||
}
|
||||
|
||||
func topicFamilyKeyFrom(tf GossipsubTopicFamily) topicFamilyKey {
|
||||
return topicFamilyKey{topicName: fmt.Sprintf("%s", tf.Name()), 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()
|
||||
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() })
|
||||
|
||||
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 (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
|
||||
}
|
||||
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)
|
||||
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
|
||||
}
|
||||
387
beacon-chain/sync/gossipsub_controller_test.go
Normal file
387
beacon-chain/sync/gossipsub_controller_test.go
Normal file
@@ -0,0 +1,387 @@
|
||||
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"
|
||||
"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)
|
||||
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))
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
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)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
152
beacon-chain/sync/gossipsub_topic_family.go
Normal file
152
beacon-chain/sync/gossipsub_topic_family.go
Normal file
@@ -0,0 +1,152 @@
|
||||
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 {
|
||||
Name() string
|
||||
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
|
||||
}
|
||||
|
||||
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: 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: 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: 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: 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: 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() {
|
||||
if epoch < entry.activationEpoch {
|
||||
continue
|
||||
}
|
||||
if entry.deactivationEpoch != nil && epoch >= *entry.deactivationEpoch {
|
||||
continue
|
||||
}
|
||||
activeFamilies = append(activeFamilies, entry.factory(s, nse)...)
|
||||
}
|
||||
return activeFamilies
|
||||
}
|
||||
310
beacon-chain/sync/gossipsub_topic_family_test.go
Normal file
310
beacon-chain/sync/gossipsub_topic_family_test.go
Normal file
@@ -0,0 +1,310 @@
|
||||
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 := append(append([]string{}, 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)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
@@ -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{})))
|
||||
@@ -945,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{})))
|
||||
|
||||
@@ -181,7 +181,7 @@ type Service struct {
|
||||
lcStore *lightClient.Store
|
||||
dataColumnLogCh chan dataColumnLogEntry
|
||||
digestActions perDigestSet
|
||||
subscriptionSpawner func(func()) // see Service.spawn for details
|
||||
gossipsubController *GossipsubController
|
||||
}
|
||||
|
||||
// NewService initializes new regular sync service.
|
||||
@@ -198,6 +198,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 +327,18 @@ func (s *Service) Stop() error {
|
||||
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 {
|
||||
dialer.Stop()
|
||||
}
|
||||
if s.cfg.p2p.Crawler() != nil {
|
||||
s.cfg.p2p.Crawler().Stop()
|
||||
}
|
||||
|
||||
// Stop the gossipsub controller.
|
||||
s.gossipsubController.Stop()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
@@ -405,7 +418,36 @@ func (s *Service) startDiscoveryAndSubscriptions() {
|
||||
}
|
||||
|
||||
// Start the fork watcher.
|
||||
go s.p2pHandlerControlLoop()
|
||||
go s.p2pRPCHandlerControlLoop()
|
||||
|
||||
// Start the gossipsub controller.
|
||||
go s.gossipsubController.Start()
|
||||
|
||||
// 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 gossipsub peer dialer available")
|
||||
}
|
||||
}
|
||||
|
||||
func (s *Service) writeErrorResponseToStream(responseCode byte, reason string, stream libp2pcore.Stream) {
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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"
|
||||
@@ -38,64 +39,27 @@ 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 {
|
||||
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),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -192,184 +156,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()
|
||||
}
|
||||
}
|
||||
|
||||
// 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
|
||||
}
|
||||
|
||||
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(topic string, validator wrappedVal, handle subHandler, nse params.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
|
||||
}
|
||||
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)
|
||||
}
|
||||
|
||||
func (s *Service) subscribeWithBase(topic string, validator wrappedVal, handle subHandler) *pubsub.Subscription {
|
||||
topic += s.cfg.p2p.Encoding().ProtocolSuffix()
|
||||
func (s *Service) subscribe(topic string, validator wrappedVal, handle subHandler) *pubsub.Subscription {
|
||||
log := log.WithField("topic", topic)
|
||||
|
||||
// Do not resubscribe already seen subscriptions.
|
||||
@@ -532,36 +324,37 @@ 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
|
||||
@@ -576,45 +369,16 @@ 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()))
|
||||
}
|
||||
}
|
||||
|
||||
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.getSubnetsToJoin, tracker.getSubnetsRequiringPeers)
|
||||
ctx, cancel := context.WithTimeout(ctx, timeout)
|
||||
defer cancel()
|
||||
err := s.cfg.p2p.FindAndDialPeersWithSubnets(ctx, tracker.topicFormat, tracker.nse.ForkDigest, minPeers, neededSubnets)
|
||||
if err != nil && !errors.Is(err, context.DeadlineExceeded) {
|
||||
log.WithFields(tracker.logFields()).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)
|
||||
@@ -625,12 +389,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 {
|
||||
@@ -663,12 +427,15 @@ 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.
|
||||
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)
|
||||
}
|
||||
|
||||
@@ -818,17 +585,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))
|
||||
|
||||
@@ -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,115 @@ 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) 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()
|
||||
@@ -64,7 +174,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 +182,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 +220,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 +270,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 +287,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 +326,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 +342,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 +376,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 +408,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 +422,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)
|
||||
@@ -479,6 +587,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()
|
||||
@@ -552,11 +661,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 +707,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 +728,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()
|
||||
|
||||
261
beacon-chain/sync/topic_families_dynamic_subnets.go
Normal file
261
beacon-chain/sync/topic_families_dynamic_subnets.go
Normal file
@@ -0,0 +1,261 @@
|
||||
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
|
||||
}
|
||||
|
||||
// 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
|
||||
}
|
||||
|
||||
func (a *AttestationTopicFamily) Name() string {
|
||||
return "AttestationTopicFamily"
|
||||
}
|
||||
|
||||
// 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, p2p.AttestationSubnets)
|
||||
}
|
||||
|
||||
func (a *AttestationTopicFamily) Subscribe() {
|
||||
a.baseGossipsubTopicFamilyWithDynamicSubnets.Subscribe(a)
|
||||
}
|
||||
|
||||
func (a *AttestationTopicFamily) Unsubscribe() {
|
||||
a.baseGossipsubTopicFamilyWithDynamicSubnets.Unsubscribe()
|
||||
}
|
||||
|
||||
// SyncCommitteeTopicFamily
|
||||
var _ GossipsubTopicFamilyWithDynamicSubnets = (*SyncCommitteeTopicFamily)(nil)
|
||||
|
||||
type SyncCommitteeTopicFamily struct {
|
||||
baseGossipsubTopicFamilyWithDynamicSubnets
|
||||
}
|
||||
|
||||
// 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(),
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
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
|
||||
}
|
||||
|
||||
// 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, p2p.SyncSubnets)
|
||||
}
|
||||
|
||||
func (s *SyncCommitteeTopicFamily) Subscribe() {
|
||||
s.baseGossipsubTopicFamilyWithDynamicSubnets.Subscribe(s)
|
||||
}
|
||||
|
||||
func (s *SyncCommitteeTopicFamily) Unsubscribe() {
|
||||
s.baseGossipsubTopicFamilyWithDynamicSubnets.Unsubscribe()
|
||||
}
|
||||
|
||||
// DataColumnTopicFamily
|
||||
var _ GossipsubTopicFamilyWithDynamicSubnets = (*DataColumnTopicFamily)(nil)
|
||||
|
||||
type DataColumnTopicFamily struct {
|
||||
baseGossipsubTopicFamilyWithDynamicSubnets
|
||||
}
|
||||
|
||||
// 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(),
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
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
|
||||
}
|
||||
|
||||
// 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, p2p.DataColumnSubnets)
|
||||
}
|
||||
|
||||
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)
|
||||
|
||||
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,
|
||||
)
|
||||
|
||||
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
|
||||
}
|
||||
51
beacon-chain/sync/topic_families_static_subnets.go
Normal file
51
beacon-chain/sync/topic_families_static_subnets.go
Normal file
@@ -0,0 +1,51 @@
|
||||
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) 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
|
||||
}
|
||||
|
||||
func (b *BlobTopicFamily) Subscribe() {
|
||||
b.syncService.subscribe(b.GetFullTopicString(), b.Validator(), b.Handler())
|
||||
}
|
||||
|
||||
func (b *BlobTopicFamily) Unsubscribe() {
|
||||
b.syncService.unSubscribeFromTopic(b.GetFullTopicString())
|
||||
}
|
||||
366
beacon-chain/sync/topic_families_without_subnets.go
Normal file
366
beacon-chain/sync/topic_families_without_subnets.go
Normal file
@@ -0,0 +1,366 @@
|
||||
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) 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
|
||||
}
|
||||
|
||||
func (b *BlockTopicFamily) Subscribe() {
|
||||
b.syncService.subscribe(b.GetFullTopicString(), b.Validator(), b.Handler())
|
||||
}
|
||||
|
||||
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) 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
|
||||
}
|
||||
|
||||
func (a *AggregateAndProofTopicFamily) Subscribe() {
|
||||
a.syncService.subscribe(a.GetFullTopicString(), a.Validator(), a.Handler())
|
||||
}
|
||||
|
||||
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) Name() string {
|
||||
return "VoluntaryExitTopicFamily"
|
||||
}
|
||||
|
||||
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.GetFullTopicString(), v.Validator(), v.Handler())
|
||||
}
|
||||
|
||||
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) 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
|
||||
}
|
||||
|
||||
func (p *ProposerSlashingTopicFamily) Subscribe() {
|
||||
p.syncService.subscribe(p.GetFullTopicString(), p.Validator(), p.Handler())
|
||||
}
|
||||
|
||||
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) 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
|
||||
}
|
||||
|
||||
// TODO: Do we really need to spawn go-routines here ?
|
||||
func (a *AttesterSlashingTopicFamily) Subscribe() {
|
||||
a.syncService.subscribe(a.GetFullTopicString(), a.Validator(), a.Handler())
|
||||
}
|
||||
|
||||
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) 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
|
||||
}
|
||||
|
||||
func (sc *SyncContributionAndProofTopicFamily) Subscribe() {
|
||||
sc.syncService.subscribe(sc.GetFullTopicString(), sc.Validator(), sc.Handler())
|
||||
}
|
||||
|
||||
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) 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
|
||||
}
|
||||
|
||||
func (l *LightClientOptimisticUpdateTopicFamily) Subscribe() {
|
||||
l.syncService.subscribe(l.GetFullTopicString(), l.Validator(), l.Handler())
|
||||
}
|
||||
|
||||
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) Name() string {
|
||||
return "LightClientFinalityUpdateTopicFamily"
|
||||
}
|
||||
|
||||
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.GetFullTopicString(), l.Validator(), l.Handler())
|
||||
}
|
||||
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) 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
|
||||
}
|
||||
|
||||
func (b *BlsToExecutionChangeTopicFamily) Subscribe() {
|
||||
b.syncService.subscribe(b.GetFullTopicString(), b.Validator(), b.Handler())
|
||||
}
|
||||
|
||||
func (b *BlsToExecutionChangeTopicFamily) Unsubscribe() {
|
||||
b.syncService.unSubscribeFromTopic(b.GetFullTopicString())
|
||||
}
|
||||
4
changelog/aarshkshah1992-gossipsub-control-pane.md
Normal file
4
changelog/aarshkshah1992-gossipsub-control-pane.md
Normal file
@@ -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.
|
||||
3
changelog/aarshkshah1992-gossipsub-peer-crawler.md
Normal file
3
changelog/aarshkshah1992-gossipsub-peer-crawler.md
Normal file
@@ -0,0 +1,3 @@
|
||||
### Added
|
||||
|
||||
- A discV5 peer crawler for Gossipsub.
|
||||
Reference in New Issue
Block a user