2021-06-26 19:00:33 +00:00
|
|
|
// Package p2p defines the network protocol implementation for Ethereum consensus
|
2020-04-29 17:40:33 +00:00
|
|
|
// used by beacon nodes, including peer discovery using discv5, gossip-sub
|
|
|
|
// using libp2p, and handing peer lifecycles + handshakes.
|
2019-08-13 21:12:00 +00:00
|
|
|
package p2p
|
|
|
|
|
|
|
|
import (
|
|
|
|
"context"
|
2019-09-01 22:29:58 +00:00
|
|
|
"crypto/ecdsa"
|
2020-07-27 00:56:55 +00:00
|
|
|
"sync"
|
2019-08-21 06:08:30 +00:00
|
|
|
"time"
|
2019-08-13 21:12:00 +00:00
|
|
|
|
2019-09-06 19:20:20 +00:00
|
|
|
"github.com/ethereum/go-ethereum/p2p/enode"
|
2020-03-23 14:41:47 +00:00
|
|
|
"github.com/ethereum/go-ethereum/p2p/enr"
|
2020-07-30 18:42:22 +00:00
|
|
|
"github.com/kevinms/leakybucket-go"
|
2019-09-01 22:29:58 +00:00
|
|
|
"github.com/libp2p/go-libp2p"
|
2019-08-13 21:12:00 +00:00
|
|
|
"github.com/libp2p/go-libp2p-core/host"
|
2019-09-01 22:29:58 +00:00
|
|
|
"github.com/libp2p/go-libp2p-core/network"
|
2019-08-16 20:03:11 +00:00
|
|
|
"github.com/libp2p/go-libp2p-core/peer"
|
2019-08-16 17:13:04 +00:00
|
|
|
"github.com/libp2p/go-libp2p-core/protocol"
|
2019-08-13 21:12:00 +00:00
|
|
|
pubsub "github.com/libp2p/go-libp2p-pubsub"
|
2021-01-08 11:08:11 +00:00
|
|
|
"github.com/libp2p/go-libp2p/p2p/protocol/identify"
|
2020-12-14 22:22:55 +00:00
|
|
|
"github.com/multiformats/go-multiaddr"
|
2019-08-13 21:12:00 +00:00
|
|
|
"github.com/pkg/errors"
|
2020-04-14 20:27:03 +00:00
|
|
|
"github.com/prysmaticlabs/prysm/beacon-chain/core/feed"
|
|
|
|
statefeed "github.com/prysmaticlabs/prysm/beacon-chain/core/feed/state"
|
2019-08-16 17:13:04 +00:00
|
|
|
"github.com/prysmaticlabs/prysm/beacon-chain/p2p/encoder"
|
2019-11-30 05:36:02 +00:00
|
|
|
"github.com/prysmaticlabs/prysm/beacon-chain/p2p/peers"
|
2020-12-14 22:22:55 +00:00
|
|
|
"github.com/prysmaticlabs/prysm/beacon-chain/p2p/peers/scorers"
|
2019-08-13 21:12:00 +00:00
|
|
|
"github.com/prysmaticlabs/prysm/shared"
|
2021-06-02 05:44:34 +00:00
|
|
|
"github.com/prysmaticlabs/prysm/shared/interfaces"
|
2020-07-01 09:47:59 +00:00
|
|
|
"github.com/prysmaticlabs/prysm/shared/params"
|
2019-12-13 23:14:56 +00:00
|
|
|
"github.com/prysmaticlabs/prysm/shared/runutil"
|
2020-05-13 03:28:17 +00:00
|
|
|
"github.com/prysmaticlabs/prysm/shared/slotutil"
|
2021-01-08 11:08:11 +00:00
|
|
|
"github.com/sirupsen/logrus"
|
2020-12-14 22:22:55 +00:00
|
|
|
"go.opencensus.io/trace"
|
2019-08-13 21:12:00 +00:00
|
|
|
)
|
|
|
|
|
2020-10-10 00:36:48 +00:00
|
|
|
var _ shared.Service = (*Service)(nil)
|
2019-09-21 16:21:44 +00:00
|
|
|
|
2020-06-24 01:09:09 +00:00
|
|
|
// In the event that we are at our peer limit, we
|
|
|
|
// stop looking for new peers and instead poll
|
|
|
|
// for the current peer limit status for the time period
|
|
|
|
// defined below.
|
|
|
|
var pollingPeriod = 6 * time.Second
|
2019-08-13 21:12:00 +00:00
|
|
|
|
2020-04-14 20:27:03 +00:00
|
|
|
// Refresh rate of ENR set at twice per slot.
|
2020-05-13 03:28:17 +00:00
|
|
|
var refreshRate = slotutil.DivideSlotBy(2)
|
2020-04-14 20:27:03 +00:00
|
|
|
|
2019-12-11 10:31:36 +00:00
|
|
|
// maxBadResponses is the maximum number of bad responses from a peer before we stop talking to it.
|
2020-05-14 03:08:58 +00:00
|
|
|
const maxBadResponses = 5
|
2019-12-11 10:31:36 +00:00
|
|
|
|
2020-07-11 15:18:25 +00:00
|
|
|
// maxDialTimeout is the timeout for a single peer dial.
|
2020-08-20 04:50:14 +00:00
|
|
|
var maxDialTimeout = params.BeaconNetworkConfig().RespTimeout
|
2020-07-11 15:18:25 +00:00
|
|
|
|
2019-08-13 21:12:00 +00:00
|
|
|
// Service for managing peer to peer (p2p) networking.
|
|
|
|
type Service struct {
|
2020-04-14 20:27:03 +00:00
|
|
|
started bool
|
|
|
|
isPreGenesis bool
|
2020-10-16 07:05:40 +00:00
|
|
|
currentForkDigest [4]byte
|
2020-04-14 20:27:03 +00:00
|
|
|
pingMethod func(ctx context.Context, id peer.ID) error
|
|
|
|
cancel context.CancelFunc
|
|
|
|
cfg *Config
|
|
|
|
peers *peers.Status
|
2020-12-14 22:22:55 +00:00
|
|
|
addrFilter *multiaddr.Filters
|
2020-07-30 18:42:22 +00:00
|
|
|
ipLimiter *leakybucket.Collector
|
2020-04-14 20:27:03 +00:00
|
|
|
privKey *ecdsa.PrivateKey
|
2021-06-02 05:44:34 +00:00
|
|
|
metaData interfaces.Metadata
|
2020-04-14 20:27:03 +00:00
|
|
|
pubsub *pubsub.PubSub
|
2020-07-13 02:28:40 +00:00
|
|
|
joinedTopics map[string]*pubsub.Topic
|
2020-07-27 00:56:55 +00:00
|
|
|
joinedTopicsLock sync.Mutex
|
2020-08-10 15:27:50 +00:00
|
|
|
subnetsLock map[uint64]*sync.RWMutex
|
|
|
|
subnetsLockLock sync.Mutex // Lock access to subnetsLock
|
2020-10-16 07:05:40 +00:00
|
|
|
initializationLock sync.Mutex
|
2020-04-14 20:27:03 +00:00
|
|
|
dv5Listener Listener
|
|
|
|
startupErr error
|
|
|
|
stateNotifier statefeed.Notifier
|
|
|
|
ctx context.Context
|
|
|
|
host host.Host
|
|
|
|
genesisTime time.Time
|
|
|
|
genesisValidatorsRoot []byte
|
2021-04-13 02:20:13 +00:00
|
|
|
activeValidatorCount uint64
|
2019-08-13 21:12:00 +00:00
|
|
|
}
|
|
|
|
|
2021-02-12 17:45:22 +00:00
|
|
|
// NewService initializes a new p2p service compatible with shared.Service interface. No
|
2019-08-13 21:12:00 +00:00
|
|
|
// connections are made until the Start function is called during the service registry startup.
|
2021-02-12 17:45:22 +00:00
|
|
|
func NewService(ctx context.Context, cfg *Config) (*Service, error) {
|
2019-09-01 22:29:58 +00:00
|
|
|
var err error
|
2020-09-09 09:48:52 +00:00
|
|
|
ctx, cancel := context.WithCancel(ctx)
|
2020-06-03 19:22:48 +00:00
|
|
|
_ = cancel // govet fix for lost cancel. Cancel is handled in service.Stop().
|
2019-08-30 20:15:40 +00:00
|
|
|
|
|
|
|
s := &Service{
|
2019-09-01 22:29:58 +00:00
|
|
|
ctx: ctx,
|
2020-04-14 20:27:03 +00:00
|
|
|
stateNotifier: cfg.StateNotifier,
|
2019-09-01 22:29:58 +00:00
|
|
|
cancel: cancel,
|
|
|
|
cfg: cfg,
|
2020-04-14 20:27:03 +00:00
|
|
|
isPreGenesis: true,
|
2020-07-13 02:28:40 +00:00
|
|
|
joinedTopics: make(map[string]*pubsub.Topic, len(GossipTopicMappings)),
|
2020-08-10 15:27:50 +00:00
|
|
|
subnetsLock: make(map[uint64]*sync.RWMutex),
|
2019-08-13 21:12:00 +00:00
|
|
|
}
|
|
|
|
|
2020-06-23 02:53:11 +00:00
|
|
|
dv5Nodes := parseBootStrapAddrs(s.cfg.BootstrapNodeAddr)
|
2019-09-23 17:24:16 +00:00
|
|
|
|
|
|
|
cfg.Discv5BootStrapAddr = dv5Nodes
|
|
|
|
|
2019-10-04 03:22:17 +00:00
|
|
|
ipAddr := ipAddr()
|
2019-09-01 22:29:58 +00:00
|
|
|
s.privKey, err = privKey(s.cfg)
|
2019-08-21 06:08:30 +00:00
|
|
|
if err != nil {
|
2019-08-23 21:46:54 +00:00
|
|
|
log.WithError(err).Error("Failed to generate p2p private key")
|
2019-08-30 20:15:40 +00:00
|
|
|
return nil, err
|
2019-08-21 06:08:30 +00:00
|
|
|
}
|
2020-04-14 20:27:03 +00:00
|
|
|
s.metaData, err = metaDataFromConfig(s.cfg)
|
|
|
|
if err != nil {
|
|
|
|
log.WithError(err).Error("Failed to create peer metadata")
|
|
|
|
return nil, err
|
|
|
|
}
|
2020-06-14 14:47:58 +00:00
|
|
|
s.addrFilter, err = configureFilter(s.cfg)
|
|
|
|
if err != nil {
|
|
|
|
log.WithError(err).Error("Failed to create address filter")
|
|
|
|
return nil, err
|
|
|
|
}
|
2020-07-30 18:42:22 +00:00
|
|
|
s.ipLimiter = leakybucket.NewCollector(ipLimit, ipBurst, true /* deleteEmptyBuckets */)
|
|
|
|
|
2020-06-14 07:35:05 +00:00
|
|
|
opts := s.buildOptions(ipAddr, s.privKey)
|
2019-08-21 06:08:30 +00:00
|
|
|
h, err := libp2p.New(s.ctx, opts...)
|
2019-08-13 21:12:00 +00:00
|
|
|
if err != nil {
|
2019-08-23 21:46:54 +00:00
|
|
|
log.WithError(err).Error("Failed to create p2p host")
|
2019-08-30 20:15:40 +00:00
|
|
|
return nil, err
|
2019-08-13 21:12:00 +00:00
|
|
|
}
|
2019-09-23 17:24:16 +00:00
|
|
|
|
2019-08-13 21:12:00 +00:00
|
|
|
s.host = h
|
2021-01-08 11:08:11 +00:00
|
|
|
s.host.RemoveStreamHandler(identify.IDDelta)
|
2019-08-27 15:23:22 +00:00
|
|
|
|
|
|
|
// Gossipsub registration is done before we add in any new peers
|
2019-08-27 22:01:27 +00:00
|
|
|
// due to libp2p's gossipsub implementation not taking into
|
2019-08-27 15:23:22 +00:00
|
|
|
// account previously added peers when creating the gossipsub
|
2019-08-27 22:01:27 +00:00
|
|
|
// object.
|
2019-09-11 18:38:35 +00:00
|
|
|
psOpts := []pubsub.Option{
|
2020-09-24 23:53:15 +00:00
|
|
|
pubsub.WithMessageSignaturePolicy(pubsub.StrictNoSign),
|
2020-08-25 11:08:42 +00:00
|
|
|
pubsub.WithNoAuthor(),
|
2019-12-17 05:17:54 +00:00
|
|
|
pubsub.WithMessageIdFn(msgIDFunction),
|
2020-10-16 07:05:40 +00:00
|
|
|
pubsub.WithSubscriptionFilter(s),
|
2020-11-22 23:54:58 +00:00
|
|
|
pubsub.WithPeerOutboundQueueSize(256),
|
2021-02-11 17:35:52 +00:00
|
|
|
pubsub.WithValidateQueueSize(256),
|
2021-04-13 02:20:13 +00:00
|
|
|
pubsub.WithPeerScore(peerScoringParams()),
|
|
|
|
pubsub.WithPeerScoreInspect(s.peerInspector, time.Minute),
|
2020-11-12 08:08:07 +00:00
|
|
|
}
|
2020-07-29 16:14:15 +00:00
|
|
|
// Set the pubsub global parameters that we require.
|
|
|
|
setPubSubParameters()
|
2020-04-19 01:53:32 +00:00
|
|
|
|
2020-06-26 17:33:18 +00:00
|
|
|
gs, err := pubsub.NewGossipSub(s.ctx, s.host, psOpts...)
|
2019-08-27 15:23:22 +00:00
|
|
|
if err != nil {
|
|
|
|
log.WithError(err).Error("Failed to start pubsub")
|
2019-08-30 20:15:40 +00:00
|
|
|
return nil, err
|
2019-08-27 15:23:22 +00:00
|
|
|
}
|
|
|
|
s.pubsub = gs
|
|
|
|
|
2020-07-19 03:45:08 +00:00
|
|
|
s.peers = peers.NewStatus(ctx, &peers.StatusConfig{
|
2020-07-20 15:12:59 +00:00
|
|
|
PeerLimit: int(s.cfg.MaxPeers),
|
2020-10-07 13:08:51 +00:00
|
|
|
ScorerParams: &scorers.Config{
|
|
|
|
BadResponsesScorerConfig: &scorers.BadResponsesScorerConfig{
|
2020-07-29 09:26:46 +00:00
|
|
|
Threshold: maxBadResponses,
|
|
|
|
DecayInterval: time.Hour,
|
|
|
|
},
|
2020-07-20 15:12:59 +00:00
|
|
|
},
|
2020-07-19 03:45:08 +00:00
|
|
|
})
|
2019-12-11 10:31:36 +00:00
|
|
|
|
2019-08-30 20:15:40 +00:00
|
|
|
return s, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// Start the p2p service.
|
|
|
|
func (s *Service) Start() {
|
|
|
|
if s.started {
|
|
|
|
log.Error("Attempted to start p2p service when it was already started")
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2020-04-14 20:27:03 +00:00
|
|
|
// Waits until the state is initialized via an event feed.
|
|
|
|
// Used for fork-related data when connecting peers.
|
|
|
|
s.awaitStateInitialized()
|
|
|
|
s.isPreGenesis = false
|
|
|
|
|
2019-09-23 21:43:53 +00:00
|
|
|
var peersToWatch []string
|
|
|
|
if s.cfg.RelayNodeAddr != "" {
|
|
|
|
peersToWatch = append(peersToWatch, s.cfg.RelayNodeAddr)
|
|
|
|
if err := dialRelayNode(s.ctx, s.host, s.cfg.RelayNodeAddr); err != nil {
|
|
|
|
log.WithError(err).Errorf("Could not dial relay node")
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-04-14 20:27:03 +00:00
|
|
|
if !s.cfg.NoDiscovery && !s.cfg.DisableDiscv5 {
|
2019-10-04 03:22:17 +00:00
|
|
|
ipAddr := ipAddr()
|
2020-04-14 20:27:03 +00:00
|
|
|
listener, err := s.startDiscoveryV5(
|
|
|
|
ipAddr,
|
|
|
|
s.privKey,
|
|
|
|
)
|
2019-08-30 20:15:40 +00:00
|
|
|
if err != nil {
|
2020-07-09 17:38:15 +00:00
|
|
|
log.WithError(err).Fatal("Failed to start discovery")
|
2019-08-30 20:15:40 +00:00
|
|
|
s.startupErr = err
|
|
|
|
return
|
|
|
|
}
|
2020-03-10 02:53:37 +00:00
|
|
|
err = s.connectToBootnodes()
|
2019-08-21 20:58:38 +00:00
|
|
|
if err != nil {
|
2019-09-01 22:29:58 +00:00
|
|
|
log.WithError(err).Error("Could not add bootnode to the exclusion list")
|
2019-08-21 20:58:38 +00:00
|
|
|
s.startupErr = err
|
|
|
|
return
|
|
|
|
}
|
|
|
|
s.dv5Listener = listener
|
|
|
|
go s.listenForNewNodes()
|
2019-09-23 17:24:16 +00:00
|
|
|
}
|
|
|
|
|
2019-08-27 15:23:22 +00:00
|
|
|
s.started = true
|
|
|
|
|
2019-08-22 15:23:16 +00:00
|
|
|
if len(s.cfg.StaticPeers) > 0 {
|
2020-03-05 14:49:26 +00:00
|
|
|
addrs, err := peersFromStringAddrs(s.cfg.StaticPeers)
|
2019-08-22 15:23:16 +00:00
|
|
|
if err != nil {
|
|
|
|
log.Errorf("Could not connect to static peer: %v", err)
|
|
|
|
}
|
|
|
|
s.connectWithAllPeers(addrs)
|
|
|
|
}
|
|
|
|
|
2019-12-19 15:02:11 +00:00
|
|
|
// Periodic functions.
|
2020-07-01 09:47:59 +00:00
|
|
|
runutil.RunEvery(s.ctx, params.BeaconNetworkConfig().TtfbTimeout, func() {
|
2019-12-19 15:02:11 +00:00
|
|
|
ensurePeerConnections(s.ctx, s.host, peersToWatch...)
|
|
|
|
})
|
2020-07-16 10:09:51 +00:00
|
|
|
runutil.RunEvery(s.ctx, 30*time.Minute, s.Peers().Prune)
|
2020-07-01 09:47:59 +00:00
|
|
|
runutil.RunEvery(s.ctx, params.BeaconNetworkConfig().RespTimeout, s.updateMetrics)
|
2020-04-14 20:27:03 +00:00
|
|
|
runutil.RunEvery(s.ctx, refreshRate, func() {
|
2020-05-05 19:26:20 +00:00
|
|
|
s.RefreshENR()
|
2020-04-14 20:27:03 +00:00
|
|
|
})
|
2021-01-08 11:08:11 +00:00
|
|
|
runutil.RunEvery(s.ctx, 1*time.Minute, func() {
|
|
|
|
log.WithFields(logrus.Fields{
|
2021-01-20 15:52:33 +00:00
|
|
|
"inbound": len(s.peers.InboundConnected()),
|
|
|
|
"outbound": len(s.peers.OutboundConnected()),
|
2021-01-08 11:08:11 +00:00
|
|
|
"activePeers": len(s.peers.Active()),
|
|
|
|
}).Info("Peer summary")
|
|
|
|
})
|
2019-12-19 15:02:11 +00:00
|
|
|
|
2019-08-22 15:23:16 +00:00
|
|
|
multiAddrs := s.host.Network().ListenAddresses()
|
2020-03-25 17:19:11 +00:00
|
|
|
logIPAddr(s.host.ID(), multiAddrs...)
|
2020-01-17 11:07:37 +00:00
|
|
|
|
|
|
|
p2pHostAddress := s.cfg.HostAddress
|
|
|
|
p2pTCPPort := s.cfg.TCPPort
|
2020-01-22 22:07:22 +00:00
|
|
|
|
|
|
|
if p2pHostAddress != "" {
|
2020-03-25 17:19:11 +00:00
|
|
|
logExternalIPAddr(s.host.ID(), p2pHostAddress, p2pTCPPort)
|
2020-06-10 23:41:11 +00:00
|
|
|
verifyConnectivity(p2pHostAddress, p2pTCPPort, "tcp")
|
2020-01-22 22:07:22 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
p2pHostDNS := s.cfg.HostDNS
|
|
|
|
if p2pHostDNS != "" {
|
|
|
|
logExternalDNSAddr(s.host.ID(), p2pHostDNS, p2pTCPPort)
|
|
|
|
}
|
2019-08-13 21:12:00 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// Stop the p2p service and terminate all peer connections.
|
|
|
|
func (s *Service) Stop() error {
|
2019-12-16 17:00:15 +00:00
|
|
|
defer s.cancel()
|
2019-08-13 21:12:00 +00:00
|
|
|
s.started = false
|
2019-08-28 10:59:34 +00:00
|
|
|
if s.dv5Listener != nil {
|
|
|
|
s.dv5Listener.Close()
|
|
|
|
}
|
2019-08-13 21:12:00 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// 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 {
|
2020-04-14 20:27:03 +00:00
|
|
|
if s.isPreGenesis {
|
|
|
|
return nil
|
|
|
|
}
|
2019-08-13 21:12:00 +00:00
|
|
|
if !s.started {
|
|
|
|
return errors.New("not running")
|
|
|
|
}
|
2020-04-14 20:27:03 +00:00
|
|
|
if s.startupErr != nil {
|
|
|
|
return s.startupErr
|
|
|
|
}
|
2019-08-13 21:12:00 +00:00
|
|
|
return nil
|
|
|
|
}
|
2019-08-16 17:13:04 +00:00
|
|
|
|
2019-08-21 20:58:38 +00:00
|
|
|
// Started returns true if the p2p service has successfully started.
|
|
|
|
func (s *Service) Started() bool {
|
|
|
|
return s.started
|
|
|
|
}
|
|
|
|
|
2019-08-16 17:13:04 +00:00
|
|
|
// Encoding returns the configured networking encoding.
|
|
|
|
func (s *Service) Encoding() encoder.NetworkEncoding {
|
2020-07-03 03:24:30 +00:00
|
|
|
return &encoder.SszNetworkEncoder{}
|
2019-08-16 17:13:04 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// PubSub returns the p2p pubsub framework.
|
|
|
|
func (s *Service) PubSub() *pubsub.PubSub {
|
|
|
|
return s.pubsub
|
|
|
|
}
|
|
|
|
|
2020-06-18 03:53:46 +00:00
|
|
|
// Host returns the currently running libp2p
|
|
|
|
// host of the service.
|
|
|
|
func (s *Service) Host() host.Host {
|
|
|
|
return s.host
|
|
|
|
}
|
|
|
|
|
2019-08-16 17:13:04 +00:00
|
|
|
// SetStreamHandler sets the protocol handler on the p2p host multiplexer.
|
|
|
|
// This method is a pass through to libp2pcore.Host.SetStreamHandler.
|
|
|
|
func (s *Service) SetStreamHandler(topic string, handler network.StreamHandler) {
|
|
|
|
s.host.SetStreamHandler(protocol.ID(topic), handler)
|
|
|
|
}
|
2019-08-16 20:03:11 +00:00
|
|
|
|
2019-08-24 18:41:24 +00:00
|
|
|
// PeerID returns the Peer ID of the local peer.
|
|
|
|
func (s *Service) PeerID() peer.ID {
|
|
|
|
return s.host.ID()
|
|
|
|
}
|
|
|
|
|
2019-08-16 20:03:11 +00:00
|
|
|
// Disconnect from a peer.
|
|
|
|
func (s *Service) Disconnect(pid peer.ID) error {
|
2019-08-23 16:53:38 +00:00
|
|
|
return s.host.Network().ClosePeer(pid)
|
2019-08-16 20:03:11 +00:00
|
|
|
}
|
2019-08-19 21:20:56 +00:00
|
|
|
|
2020-05-05 07:22:26 +00:00
|
|
|
// Connect to a specific peer.
|
|
|
|
func (s *Service) Connect(pi peer.AddrInfo) error {
|
|
|
|
return s.host.Connect(s.ctx, pi)
|
|
|
|
}
|
|
|
|
|
2019-12-11 10:31:36 +00:00
|
|
|
// Peers returns the peer status interface.
|
|
|
|
func (s *Service) Peers() *peers.Status {
|
|
|
|
return s.peers
|
2019-11-30 05:36:02 +00:00
|
|
|
}
|
|
|
|
|
2020-06-18 03:53:46 +00:00
|
|
|
// ENR returns the local node's current ENR.
|
|
|
|
func (s *Service) ENR() *enr.Record {
|
|
|
|
if s.dv5Listener == nil {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
return s.dv5Listener.Self().Record()
|
|
|
|
}
|
|
|
|
|
2021-01-07 13:35:42 +00:00
|
|
|
// DiscoveryAddresses represents our enr addresses as multiaddresses.
|
|
|
|
func (s *Service) DiscoveryAddresses() ([]multiaddr.Multiaddr, error) {
|
2020-12-17 18:03:18 +00:00
|
|
|
if s.dv5Listener == nil {
|
|
|
|
return nil, nil
|
|
|
|
}
|
|
|
|
return convertToUdpMultiAddr(s.dv5Listener.Self())
|
|
|
|
}
|
|
|
|
|
2020-04-14 20:27:03 +00:00
|
|
|
// Metadata returns a copy of the peer's metadata.
|
2021-06-02 05:44:34 +00:00
|
|
|
func (s *Service) Metadata() interfaces.Metadata {
|
|
|
|
return s.metaData.Copy()
|
2020-04-14 20:27:03 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// MetadataSeq returns the metadata sequence number.
|
|
|
|
func (s *Service) MetadataSeq() uint64 {
|
2021-06-02 05:44:34 +00:00
|
|
|
return s.metaData.SequenceNumber()
|
2020-04-14 20:27:03 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// AddPingMethod adds the metadata ping rpc method to the p2p service, so that it can
|
|
|
|
// be used to refresh ENR.
|
|
|
|
func (s *Service) AddPingMethod(reqFunc func(ctx context.Context, id peer.ID) error) {
|
|
|
|
s.pingMethod = reqFunc
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Service) pingPeers() {
|
|
|
|
if s.pingMethod == nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
for _, pid := range s.peers.Connected() {
|
|
|
|
go func(id peer.ID) {
|
|
|
|
if err := s.pingMethod(s.ctx, id); err != nil {
|
2020-08-20 03:30:41 +00:00
|
|
|
log.WithField("peer", id).WithError(err).Debug("Failed to ping peer")
|
2020-04-14 20:27:03 +00:00
|
|
|
}
|
|
|
|
}(pid)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Waits for the beacon state to be initialized, important
|
|
|
|
// for initializing the p2p service as p2p needs to be aware
|
|
|
|
// of genesis information for peering.
|
|
|
|
func (s *Service) awaitStateInitialized() {
|
2020-10-16 07:05:40 +00:00
|
|
|
s.initializationLock.Lock()
|
|
|
|
defer s.initializationLock.Unlock()
|
|
|
|
|
|
|
|
if s.isInitialized() {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2020-04-14 20:27:03 +00:00
|
|
|
stateChannel := make(chan *feed.Event, 1)
|
|
|
|
stateSub := s.stateNotifier.StateFeed().Subscribe(stateChannel)
|
2020-10-04 15:03:10 +00:00
|
|
|
cleanup := stateSub.Unsubscribe
|
|
|
|
defer cleanup()
|
2020-08-11 12:14:28 +00:00
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case event := <-stateChannel:
|
|
|
|
if event.Type == statefeed.Initialized {
|
|
|
|
data, ok := event.Data.(*statefeed.InitializedData)
|
|
|
|
if !ok {
|
2020-10-04 15:03:10 +00:00
|
|
|
// log.Fatalf will prevent defer from being called
|
|
|
|
cleanup()
|
2020-08-11 12:14:28 +00:00
|
|
|
log.Fatalf("Received wrong data over state initialized feed: %v", data)
|
|
|
|
}
|
|
|
|
s.genesisTime = data.StartTime
|
|
|
|
s.genesisValidatorsRoot = data.GenesisValidatorsRoot
|
2020-10-16 07:05:40 +00:00
|
|
|
_, err := s.forkDigest() // initialize fork digest cache
|
|
|
|
if err != nil {
|
|
|
|
log.WithError(err).Error("Could not initialize fork digest")
|
|
|
|
}
|
|
|
|
|
2020-08-11 12:14:28 +00:00
|
|
|
return
|
2020-04-14 20:27:03 +00:00
|
|
|
}
|
2020-10-12 16:12:00 +00:00
|
|
|
case <-s.ctx.Done():
|
|
|
|
log.Debug("Context closed, exiting goroutine")
|
|
|
|
return
|
2020-04-14 20:27:03 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-12-14 22:22:55 +00:00
|
|
|
func (s *Service) connectWithAllPeers(multiAddrs []multiaddr.Multiaddr) {
|
2019-08-21 06:08:30 +00:00
|
|
|
addrInfos, err := peer.AddrInfosFromP2pAddrs(multiAddrs...)
|
|
|
|
if err != nil {
|
|
|
|
log.Errorf("Could not convert to peer address info's from multiaddresses: %v", err)
|
|
|
|
return
|
|
|
|
}
|
|
|
|
for _, info := range addrInfos {
|
2020-03-06 14:57:47 +00:00
|
|
|
// make each dial non-blocking
|
|
|
|
go func(info peer.AddrInfo) {
|
2020-08-10 15:27:50 +00:00
|
|
|
if err := s.connectWithPeer(s.ctx, info); err != nil {
|
2020-04-22 01:21:12 +00:00
|
|
|
log.WithError(err).Tracef("Could not connect with peer %s", info.String())
|
2020-03-06 14:57:47 +00:00
|
|
|
}
|
|
|
|
}(info)
|
2019-08-21 06:08:30 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-08-10 15:27:50 +00:00
|
|
|
func (s *Service) connectWithPeer(ctx context.Context, info peer.AddrInfo) error {
|
|
|
|
ctx, span := trace.StartSpan(ctx, "p2p.connectWithPeer")
|
|
|
|
defer span.End()
|
|
|
|
|
2020-03-23 14:41:47 +00:00
|
|
|
if info.ID == s.host.ID() {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
if s.Peers().IsBad(info.ID) {
|
2020-11-21 20:09:07 +00:00
|
|
|
return errors.New("refused to connect to bad peer")
|
2020-03-23 14:41:47 +00:00
|
|
|
}
|
2020-08-10 15:27:50 +00:00
|
|
|
ctx, cancel := context.WithTimeout(ctx, maxDialTimeout)
|
2020-07-11 15:18:25 +00:00
|
|
|
defer cancel()
|
|
|
|
if err := s.host.Connect(ctx, info); err != nil {
|
2020-07-29 09:26:46 +00:00
|
|
|
s.Peers().Scorers().BadResponsesScorer().Increment(info.ID)
|
2020-03-23 14:41:47 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2020-03-10 02:53:37 +00:00
|
|
|
func (s *Service) connectToBootnodes() error {
|
|
|
|
nodes := make([]*enode.Node, 0, len(s.cfg.Discv5BootStrapAddr))
|
2019-09-23 17:24:16 +00:00
|
|
|
for _, addr := range s.cfg.Discv5BootStrapAddr {
|
|
|
|
bootNode, err := enode.Parse(enode.ValidSchemes, addr)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2020-04-14 20:27:03 +00:00
|
|
|
// do not dial bootnodes with their tcp ports not set
|
|
|
|
if err := bootNode.Record().Load(enr.WithEntry("tcp", new(enr.TCP))); err != nil {
|
|
|
|
if !enr.IsNotFound(err) {
|
|
|
|
log.WithError(err).Error("Could not retrieve tcp port")
|
|
|
|
}
|
|
|
|
continue
|
|
|
|
}
|
2020-03-10 02:53:37 +00:00
|
|
|
nodes = append(nodes, bootNode)
|
2019-09-01 22:29:58 +00:00
|
|
|
}
|
2020-03-10 02:53:37 +00:00
|
|
|
multiAddresses := convertToMultiAddr(nodes)
|
|
|
|
s.connectWithAllPeers(multiAddresses)
|
2019-09-23 17:24:16 +00:00
|
|
|
return nil
|
|
|
|
}
|
2020-10-16 07:05:40 +00:00
|
|
|
|
|
|
|
// Returns true if the service is aware of the genesis time and genesis validator root. This is
|
|
|
|
// required for discovery and pubsub validation.
|
|
|
|
func (s *Service) isInitialized() bool {
|
|
|
|
return !s.genesisTime.IsZero() && len(s.genesisValidatorsRoot) == 32
|
|
|
|
}
|