p2p: new dial scheduler (#20592)
* p2p: new dial scheduler This change replaces the peer-to-peer dial scheduler with a new and improved implementation. The new code is better than the previous implementation in two key aspects: - The time between discovery of a node and dialing that node is significantly lower in the new version. The old dialState kept a buffer of nodes and launched a task to refill it whenever the buffer became empty. This worked well with the discovery interface we used to have, but doesn't really work with the new iterator-based discovery API. - Selection of static dial candidates (created by Server.AddPeer or through static-nodes.json) performs much better for large amounts of static peers. Connections to static nodes are now limited like dynanic dials and can no longer overstep MaxPeers or the dial ratio. * p2p/simulations/adapters: adapt to new NodeDialer interface * p2p: re-add check for self in checkDial * p2p: remove peersetCh * p2p: allow static dials when discovery is disabled * p2p: add test for dialScheduler.removeStatic * p2p: remove blank line * p2p: fix documentation of maxDialPeers * p2p: change "ok" to "added" in static node log * p2p: improve dialTask docs Also increase log level for "Can't resolve node" * p2p: ensure dial resolver is truly nil without discovery * p2p: add "looking for peers" log message * p2p: clean up Server.run comments * p2p: fix maxDialedConns for maxpeers < dialRatio Always allocate at least one dial slot unless dialing is disabled using NoDial or MaxPeers == 0. Most importantly, this fixes MaxPeers == 1 to dedicate the sole slot to dialing instead of listening. * p2p: fix RemovePeer to disconnect the peer again Also make RemovePeer synchronous and add a test. * p2p: remove "Connection set up" log message * p2p: clean up connection logging We previously logged outgoing connection failures up to three times. - in SetupConn() as "Setting up connection failed addr=..." - in setupConn() with an error-specific message and "id=... addr=..." - in dial() as "Dial error task=..." This commit ensures a single log message is emitted per failure and adds "id=... addr=... conn=..." everywhere (id= omitted when the ID isn't known yet). Also avoid printing a log message when a static dial fails but can't be resolved because discv4 is disabled. The light client hit this case all the time, increasing the message count to four lines per failed connection. * p2p: document that RemovePeer blocks
This commit is contained in:
637
p2p/dial.go
637
p2p/dial.go
@ -17,11 +17,17 @@
|
||||
package p2p
|
||||
|
||||
import (
|
||||
"context"
|
||||
crand "crypto/rand"
|
||||
"encoding/binary"
|
||||
"errors"
|
||||
"fmt"
|
||||
mrand "math/rand"
|
||||
"net"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common/mclock"
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
"github.com/ethereum/go-ethereum/p2p/enode"
|
||||
"github.com/ethereum/go-ethereum/p2p/netutil"
|
||||
@ -33,8 +39,9 @@ const (
|
||||
// private networks.
|
||||
dialHistoryExpiration = inboundThrottleTime + 5*time.Second
|
||||
|
||||
// If no peers are found for this amount of time, the initial bootnodes are dialed.
|
||||
fallbackInterval = 20 * time.Second
|
||||
// Config for the "Looking for peers" message.
|
||||
dialStatsLogInterval = 10 * time.Second // printed at most this often
|
||||
dialStatsPeerLimit = 3 // but not if more than this many dialed peers
|
||||
|
||||
// Endpoint resolution is throttled with bounded backoff.
|
||||
initialResolveDelay = 60 * time.Second
|
||||
@ -42,161 +49,29 @@ const (
|
||||
)
|
||||
|
||||
// NodeDialer is used to connect to nodes in the network, typically by using
|
||||
// an underlying net.Dialer but also using net.Pipe in tests
|
||||
// an underlying net.Dialer but also using net.Pipe in tests.
|
||||
type NodeDialer interface {
|
||||
Dial(*enode.Node) (net.Conn, error)
|
||||
Dial(context.Context, *enode.Node) (net.Conn, error)
|
||||
}
|
||||
|
||||
type nodeResolver interface {
|
||||
Resolve(*enode.Node) *enode.Node
|
||||
}
|
||||
|
||||
// TCPDialer implements the NodeDialer interface by using a net.Dialer to
|
||||
// create TCP connections to nodes in the network
|
||||
type TCPDialer struct {
|
||||
*net.Dialer
|
||||
// tcpDialer implements NodeDialer using real TCP connections.
|
||||
type tcpDialer struct {
|
||||
d *net.Dialer
|
||||
}
|
||||
|
||||
// Dial creates a TCP connection to the node
|
||||
func (t TCPDialer) Dial(dest *enode.Node) (net.Conn, error) {
|
||||
addr := &net.TCPAddr{IP: dest.IP(), Port: dest.TCP()}
|
||||
return t.Dialer.Dial("tcp", addr.String())
|
||||
func (t tcpDialer) Dial(ctx context.Context, dest *enode.Node) (net.Conn, error) {
|
||||
return t.d.DialContext(ctx, "tcp", nodeAddr(dest).String())
|
||||
}
|
||||
|
||||
// dialstate schedules dials and discovery lookups.
|
||||
// It gets a chance to compute new tasks on every iteration
|
||||
// of the main loop in Server.run.
|
||||
type dialstate struct {
|
||||
maxDynDials int
|
||||
netrestrict *netutil.Netlist
|
||||
self enode.ID
|
||||
bootnodes []*enode.Node // default dials when there are no peers
|
||||
log log.Logger
|
||||
|
||||
start time.Time // time when the dialer was first used
|
||||
lookupRunning bool
|
||||
dialing map[enode.ID]connFlag
|
||||
lookupBuf []*enode.Node // current discovery lookup results
|
||||
static map[enode.ID]*dialTask
|
||||
hist expHeap
|
||||
}
|
||||
|
||||
type task interface {
|
||||
Do(*Server)
|
||||
}
|
||||
|
||||
func newDialState(self enode.ID, maxdyn int, cfg *Config) *dialstate {
|
||||
s := &dialstate{
|
||||
maxDynDials: maxdyn,
|
||||
self: self,
|
||||
netrestrict: cfg.NetRestrict,
|
||||
log: cfg.Logger,
|
||||
static: make(map[enode.ID]*dialTask),
|
||||
dialing: make(map[enode.ID]connFlag),
|
||||
bootnodes: make([]*enode.Node, len(cfg.BootstrapNodes)),
|
||||
}
|
||||
copy(s.bootnodes, cfg.BootstrapNodes)
|
||||
if s.log == nil {
|
||||
s.log = log.Root()
|
||||
}
|
||||
for _, n := range cfg.StaticNodes {
|
||||
s.addStatic(n)
|
||||
}
|
||||
return s
|
||||
}
|
||||
|
||||
func (s *dialstate) addStatic(n *enode.Node) {
|
||||
// This overwrites the task instead of updating an existing
|
||||
// entry, giving users the opportunity to force a resolve operation.
|
||||
s.static[n.ID()] = &dialTask{flags: staticDialedConn, dest: n}
|
||||
}
|
||||
|
||||
func (s *dialstate) removeStatic(n *enode.Node) {
|
||||
// This removes a task so future attempts to connect will not be made.
|
||||
delete(s.static, n.ID())
|
||||
}
|
||||
|
||||
func (s *dialstate) newTasks(nRunning int, peers map[enode.ID]*Peer, now time.Time) []task {
|
||||
var newtasks []task
|
||||
addDial := func(flag connFlag, n *enode.Node) bool {
|
||||
if err := s.checkDial(n, peers); err != nil {
|
||||
s.log.Trace("Skipping dial candidate", "id", n.ID(), "addr", &net.TCPAddr{IP: n.IP(), Port: n.TCP()}, "err", err)
|
||||
return false
|
||||
}
|
||||
s.dialing[n.ID()] = flag
|
||||
newtasks = append(newtasks, &dialTask{flags: flag, dest: n})
|
||||
return true
|
||||
}
|
||||
|
||||
if s.start.IsZero() {
|
||||
s.start = now
|
||||
}
|
||||
s.hist.expire(now)
|
||||
|
||||
// Create dials for static nodes if they are not connected.
|
||||
for id, t := range s.static {
|
||||
err := s.checkDial(t.dest, peers)
|
||||
switch err {
|
||||
case errNotWhitelisted, errSelf:
|
||||
s.log.Warn("Removing static dial candidate", "id", t.dest.ID, "addr", &net.TCPAddr{IP: t.dest.IP(), Port: t.dest.TCP()}, "err", err)
|
||||
delete(s.static, t.dest.ID())
|
||||
case nil:
|
||||
s.dialing[id] = t.flags
|
||||
newtasks = append(newtasks, t)
|
||||
}
|
||||
}
|
||||
|
||||
// Compute number of dynamic dials needed.
|
||||
needDynDials := s.maxDynDials
|
||||
for _, p := range peers {
|
||||
if p.rw.is(dynDialedConn) {
|
||||
needDynDials--
|
||||
}
|
||||
}
|
||||
for _, flag := range s.dialing {
|
||||
if flag&dynDialedConn != 0 {
|
||||
needDynDials--
|
||||
}
|
||||
}
|
||||
|
||||
// If we don't have any peers whatsoever, try to dial a random bootnode. This
|
||||
// scenario is useful for the testnet (and private networks) where the discovery
|
||||
// table might be full of mostly bad peers, making it hard to find good ones.
|
||||
if len(peers) == 0 && len(s.bootnodes) > 0 && needDynDials > 0 && now.Sub(s.start) > fallbackInterval {
|
||||
bootnode := s.bootnodes[0]
|
||||
s.bootnodes = append(s.bootnodes[:0], s.bootnodes[1:]...)
|
||||
s.bootnodes = append(s.bootnodes, bootnode)
|
||||
if addDial(dynDialedConn, bootnode) {
|
||||
needDynDials--
|
||||
}
|
||||
}
|
||||
|
||||
// Create dynamic dials from discovery results.
|
||||
i := 0
|
||||
for ; i < len(s.lookupBuf) && needDynDials > 0; i++ {
|
||||
if addDial(dynDialedConn, s.lookupBuf[i]) {
|
||||
needDynDials--
|
||||
}
|
||||
}
|
||||
s.lookupBuf = s.lookupBuf[:copy(s.lookupBuf, s.lookupBuf[i:])]
|
||||
|
||||
// Launch a discovery lookup if more candidates are needed.
|
||||
if len(s.lookupBuf) < needDynDials && !s.lookupRunning {
|
||||
s.lookupRunning = true
|
||||
newtasks = append(newtasks, &discoverTask{want: needDynDials - len(s.lookupBuf)})
|
||||
}
|
||||
|
||||
// Launch a timer to wait for the next node to expire if all
|
||||
// candidates have been tried and no task is currently active.
|
||||
// This should prevent cases where the dialer logic is not ticked
|
||||
// because there are no pending events.
|
||||
if nRunning == 0 && len(newtasks) == 0 && s.hist.Len() > 0 {
|
||||
t := &waitExpireTask{s.hist.nextExpiry().Sub(now)}
|
||||
newtasks = append(newtasks, t)
|
||||
}
|
||||
return newtasks
|
||||
func nodeAddr(n *enode.Node) net.Addr {
|
||||
return &net.TCPAddr{IP: n.IP(), Port: n.TCP()}
|
||||
}
|
||||
|
||||
// checkDial errors:
|
||||
var (
|
||||
errSelf = errors.New("is self")
|
||||
errAlreadyDialing = errors.New("already dialing")
|
||||
@ -205,56 +80,412 @@ var (
|
||||
errNotWhitelisted = errors.New("not contained in netrestrict whitelist")
|
||||
)
|
||||
|
||||
func (s *dialstate) checkDial(n *enode.Node, peers map[enode.ID]*Peer) error {
|
||||
_, dialing := s.dialing[n.ID()]
|
||||
switch {
|
||||
case dialing:
|
||||
return errAlreadyDialing
|
||||
case peers[n.ID()] != nil:
|
||||
return errAlreadyConnected
|
||||
case n.ID() == s.self:
|
||||
// dialer creates outbound connections and submits them into Server.
|
||||
// Two types of peer connections can be created:
|
||||
//
|
||||
// - static dials are pre-configured connections. The dialer attempts
|
||||
// keep these nodes connected at all times.
|
||||
//
|
||||
// - dynamic dials are created from node discovery results. The dialer
|
||||
// continuously reads candidate nodes from its input iterator and attempts
|
||||
// to create peer connections to nodes arriving through the iterator.
|
||||
//
|
||||
type dialScheduler struct {
|
||||
dialConfig
|
||||
setupFunc dialSetupFunc
|
||||
wg sync.WaitGroup
|
||||
cancel context.CancelFunc
|
||||
ctx context.Context
|
||||
nodesIn chan *enode.Node
|
||||
doneCh chan *dialTask
|
||||
addStaticCh chan *enode.Node
|
||||
remStaticCh chan *enode.Node
|
||||
addPeerCh chan *conn
|
||||
remPeerCh chan *conn
|
||||
|
||||
// Everything below here belongs to loop and
|
||||
// should only be accessed by code on the loop goroutine.
|
||||
dialing map[enode.ID]*dialTask // active tasks
|
||||
peers map[enode.ID]connFlag // all connected peers
|
||||
dialPeers int // current number of dialed peers
|
||||
|
||||
// The static map tracks all static dial tasks. The subset of usable static dial tasks
|
||||
// (i.e. those passing checkDial) is kept in staticPool. The scheduler prefers
|
||||
// launching random static tasks from the pool over launching dynamic dials from the
|
||||
// iterator.
|
||||
static map[enode.ID]*dialTask
|
||||
staticPool []*dialTask
|
||||
|
||||
// The dial history keeps recently dialed nodes. Members of history are not dialed.
|
||||
history expHeap
|
||||
historyTimer mclock.Timer
|
||||
historyTimerTime mclock.AbsTime
|
||||
|
||||
// for logStats
|
||||
lastStatsLog mclock.AbsTime
|
||||
doneSinceLastLog int
|
||||
}
|
||||
|
||||
type dialSetupFunc func(net.Conn, connFlag, *enode.Node) error
|
||||
|
||||
type dialConfig struct {
|
||||
self enode.ID // our own ID
|
||||
maxDialPeers int // maximum number of dialed peers
|
||||
maxActiveDials int // maximum number of active dials
|
||||
netRestrict *netutil.Netlist // IP whitelist, disabled if nil
|
||||
resolver nodeResolver
|
||||
dialer NodeDialer
|
||||
log log.Logger
|
||||
clock mclock.Clock
|
||||
rand *mrand.Rand
|
||||
}
|
||||
|
||||
func (cfg dialConfig) withDefaults() dialConfig {
|
||||
if cfg.maxActiveDials == 0 {
|
||||
cfg.maxActiveDials = defaultMaxPendingPeers
|
||||
}
|
||||
if cfg.log == nil {
|
||||
cfg.log = log.Root()
|
||||
}
|
||||
if cfg.clock == nil {
|
||||
cfg.clock = mclock.System{}
|
||||
}
|
||||
if cfg.rand == nil {
|
||||
seedb := make([]byte, 8)
|
||||
crand.Read(seedb)
|
||||
seed := int64(binary.BigEndian.Uint64(seedb))
|
||||
cfg.rand = mrand.New(mrand.NewSource(seed))
|
||||
}
|
||||
return cfg
|
||||
}
|
||||
|
||||
func newDialScheduler(config dialConfig, it enode.Iterator, setupFunc dialSetupFunc) *dialScheduler {
|
||||
d := &dialScheduler{
|
||||
dialConfig: config.withDefaults(),
|
||||
setupFunc: setupFunc,
|
||||
dialing: make(map[enode.ID]*dialTask),
|
||||
static: make(map[enode.ID]*dialTask),
|
||||
peers: make(map[enode.ID]connFlag),
|
||||
doneCh: make(chan *dialTask),
|
||||
nodesIn: make(chan *enode.Node),
|
||||
addStaticCh: make(chan *enode.Node),
|
||||
remStaticCh: make(chan *enode.Node),
|
||||
addPeerCh: make(chan *conn),
|
||||
remPeerCh: make(chan *conn),
|
||||
}
|
||||
d.lastStatsLog = d.clock.Now()
|
||||
d.ctx, d.cancel = context.WithCancel(context.Background())
|
||||
d.wg.Add(2)
|
||||
go d.readNodes(it)
|
||||
go d.loop(it)
|
||||
return d
|
||||
}
|
||||
|
||||
// stop shuts down the dialer, canceling all current dial tasks.
|
||||
func (d *dialScheduler) stop() {
|
||||
d.cancel()
|
||||
d.wg.Wait()
|
||||
}
|
||||
|
||||
// addStatic adds a static dial candidate.
|
||||
func (d *dialScheduler) addStatic(n *enode.Node) {
|
||||
select {
|
||||
case d.addStaticCh <- n:
|
||||
case <-d.ctx.Done():
|
||||
}
|
||||
}
|
||||
|
||||
// removeStatic removes a static dial candidate.
|
||||
func (d *dialScheduler) removeStatic(n *enode.Node) {
|
||||
select {
|
||||
case d.remStaticCh <- n:
|
||||
case <-d.ctx.Done():
|
||||
}
|
||||
}
|
||||
|
||||
// peerAdded updates the peer set.
|
||||
func (d *dialScheduler) peerAdded(c *conn) {
|
||||
select {
|
||||
case d.addPeerCh <- c:
|
||||
case <-d.ctx.Done():
|
||||
}
|
||||
}
|
||||
|
||||
// peerRemoved updates the peer set.
|
||||
func (d *dialScheduler) peerRemoved(c *conn) {
|
||||
select {
|
||||
case d.remPeerCh <- c:
|
||||
case <-d.ctx.Done():
|
||||
}
|
||||
}
|
||||
|
||||
// loop is the main loop of the dialer.
|
||||
func (d *dialScheduler) loop(it enode.Iterator) {
|
||||
var (
|
||||
nodesCh chan *enode.Node
|
||||
historyExp = make(chan struct{}, 1)
|
||||
)
|
||||
|
||||
loop:
|
||||
for {
|
||||
// Launch new dials if slots are available.
|
||||
slots := d.freeDialSlots()
|
||||
slots -= d.startStaticDials(slots)
|
||||
if slots > 0 {
|
||||
nodesCh = d.nodesIn
|
||||
} else {
|
||||
nodesCh = nil
|
||||
}
|
||||
d.rearmHistoryTimer(historyExp)
|
||||
d.logStats()
|
||||
|
||||
select {
|
||||
case node := <-nodesCh:
|
||||
if err := d.checkDial(node); err != nil {
|
||||
d.log.Trace("Discarding dial candidate", "id", node.ID(), "ip", node.IP(), "reason", err)
|
||||
} else {
|
||||
d.startDial(newDialTask(node, dynDialedConn))
|
||||
}
|
||||
|
||||
case task := <-d.doneCh:
|
||||
id := task.dest.ID()
|
||||
delete(d.dialing, id)
|
||||
d.updateStaticPool(id)
|
||||
d.doneSinceLastLog++
|
||||
|
||||
case c := <-d.addPeerCh:
|
||||
if c.is(dynDialedConn) || c.is(staticDialedConn) {
|
||||
d.dialPeers++
|
||||
}
|
||||
id := c.node.ID()
|
||||
d.peers[id] = c.flags
|
||||
// Remove from static pool because the node is now connected.
|
||||
task := d.static[id]
|
||||
if task != nil && task.staticPoolIndex >= 0 {
|
||||
d.removeFromStaticPool(task.staticPoolIndex)
|
||||
}
|
||||
// TODO: cancel dials to connected peers
|
||||
|
||||
case c := <-d.remPeerCh:
|
||||
if c.is(dynDialedConn) || c.is(staticDialedConn) {
|
||||
d.dialPeers--
|
||||
}
|
||||
delete(d.peers, c.node.ID())
|
||||
d.updateStaticPool(c.node.ID())
|
||||
|
||||
case node := <-d.addStaticCh:
|
||||
id := node.ID()
|
||||
_, exists := d.static[id]
|
||||
d.log.Trace("Adding static node", "id", id, "ip", node.IP(), "added", !exists)
|
||||
if exists {
|
||||
continue loop
|
||||
}
|
||||
task := newDialTask(node, staticDialedConn)
|
||||
d.static[id] = task
|
||||
if d.checkDial(node) == nil {
|
||||
d.addToStaticPool(task)
|
||||
}
|
||||
|
||||
case node := <-d.remStaticCh:
|
||||
id := node.ID()
|
||||
task := d.static[id]
|
||||
d.log.Trace("Removing static node", "id", id, "ok", task != nil)
|
||||
if task != nil {
|
||||
delete(d.static, id)
|
||||
if task.staticPoolIndex >= 0 {
|
||||
d.removeFromStaticPool(task.staticPoolIndex)
|
||||
}
|
||||
}
|
||||
|
||||
case <-historyExp:
|
||||
d.expireHistory()
|
||||
|
||||
case <-d.ctx.Done():
|
||||
it.Close()
|
||||
break loop
|
||||
}
|
||||
}
|
||||
|
||||
d.stopHistoryTimer(historyExp)
|
||||
for range d.dialing {
|
||||
<-d.doneCh
|
||||
}
|
||||
d.wg.Done()
|
||||
}
|
||||
|
||||
// readNodes runs in its own goroutine and delivers nodes from
|
||||
// the input iterator to the nodesIn channel.
|
||||
func (d *dialScheduler) readNodes(it enode.Iterator) {
|
||||
defer d.wg.Done()
|
||||
|
||||
for it.Next() {
|
||||
select {
|
||||
case d.nodesIn <- it.Node():
|
||||
case <-d.ctx.Done():
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// logStats prints dialer statistics to the log. The message is suppressed when enough
|
||||
// peers are connected because users should only see it while their client is starting up
|
||||
// or comes back online.
|
||||
func (d *dialScheduler) logStats() {
|
||||
now := d.clock.Now()
|
||||
if d.lastStatsLog.Add(dialStatsLogInterval) > now {
|
||||
return
|
||||
}
|
||||
if d.dialPeers < dialStatsPeerLimit && d.dialPeers < d.maxDialPeers {
|
||||
d.log.Info("Looking for peers", "peercount", len(d.peers), "tried", d.doneSinceLastLog, "static", len(d.static))
|
||||
}
|
||||
d.doneSinceLastLog = 0
|
||||
d.lastStatsLog = now
|
||||
}
|
||||
|
||||
// rearmHistoryTimer configures d.historyTimer to fire when the
|
||||
// next item in d.history expires.
|
||||
func (d *dialScheduler) rearmHistoryTimer(ch chan struct{}) {
|
||||
if len(d.history) == 0 || d.historyTimerTime == d.history.nextExpiry() {
|
||||
return
|
||||
}
|
||||
d.stopHistoryTimer(ch)
|
||||
d.historyTimerTime = d.history.nextExpiry()
|
||||
timeout := time.Duration(d.historyTimerTime - d.clock.Now())
|
||||
d.historyTimer = d.clock.AfterFunc(timeout, func() { ch <- struct{}{} })
|
||||
}
|
||||
|
||||
// stopHistoryTimer stops the timer and drains the channel it sends on.
|
||||
func (d *dialScheduler) stopHistoryTimer(ch chan struct{}) {
|
||||
if d.historyTimer != nil && !d.historyTimer.Stop() {
|
||||
<-ch
|
||||
}
|
||||
}
|
||||
|
||||
// expireHistory removes expired items from d.history.
|
||||
func (d *dialScheduler) expireHistory() {
|
||||
d.historyTimer.Stop()
|
||||
d.historyTimer = nil
|
||||
d.historyTimerTime = 0
|
||||
d.history.expire(d.clock.Now(), func(hkey string) {
|
||||
var id enode.ID
|
||||
copy(id[:], hkey)
|
||||
d.updateStaticPool(id)
|
||||
})
|
||||
}
|
||||
|
||||
// freeDialSlots returns the number of free dial slots. The result can be negative
|
||||
// when peers are connected while their task is still running.
|
||||
func (d *dialScheduler) freeDialSlots() int {
|
||||
slots := (d.maxDialPeers - d.dialPeers) * 2
|
||||
if slots > d.maxActiveDials {
|
||||
slots = d.maxActiveDials
|
||||
}
|
||||
free := slots - len(d.dialing)
|
||||
return free
|
||||
}
|
||||
|
||||
// checkDial returns an error if node n should not be dialed.
|
||||
func (d *dialScheduler) checkDial(n *enode.Node) error {
|
||||
if n.ID() == d.self {
|
||||
return errSelf
|
||||
case s.netrestrict != nil && !s.netrestrict.Contains(n.IP()):
|
||||
}
|
||||
if _, ok := d.dialing[n.ID()]; ok {
|
||||
return errAlreadyDialing
|
||||
}
|
||||
if _, ok := d.peers[n.ID()]; ok {
|
||||
return errAlreadyConnected
|
||||
}
|
||||
if d.netRestrict != nil && !d.netRestrict.Contains(n.IP()) {
|
||||
return errNotWhitelisted
|
||||
case s.hist.contains(string(n.ID().Bytes())):
|
||||
}
|
||||
if d.history.contains(string(n.ID().Bytes())) {
|
||||
return errRecentlyDialed
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *dialstate) taskDone(t task, now time.Time) {
|
||||
switch t := t.(type) {
|
||||
case *dialTask:
|
||||
s.hist.add(string(t.dest.ID().Bytes()), now.Add(dialHistoryExpiration))
|
||||
delete(s.dialing, t.dest.ID())
|
||||
case *discoverTask:
|
||||
s.lookupRunning = false
|
||||
s.lookupBuf = append(s.lookupBuf, t.results...)
|
||||
// startStaticDials starts n static dial tasks.
|
||||
func (d *dialScheduler) startStaticDials(n int) (started int) {
|
||||
for started = 0; started < n && len(d.staticPool) > 0; started++ {
|
||||
idx := d.rand.Intn(len(d.staticPool))
|
||||
task := d.staticPool[idx]
|
||||
d.startDial(task)
|
||||
d.removeFromStaticPool(idx)
|
||||
}
|
||||
return started
|
||||
}
|
||||
|
||||
// updateStaticPool attempts to move the given static dial back into staticPool.
|
||||
func (d *dialScheduler) updateStaticPool(id enode.ID) {
|
||||
task, ok := d.static[id]
|
||||
if ok && task.staticPoolIndex < 0 && d.checkDial(task.dest) == nil {
|
||||
d.addToStaticPool(task)
|
||||
}
|
||||
}
|
||||
|
||||
// A dialTask is generated for each node that is dialed. Its
|
||||
// fields cannot be accessed while the task is running.
|
||||
func (d *dialScheduler) addToStaticPool(task *dialTask) {
|
||||
if task.staticPoolIndex >= 0 {
|
||||
panic("attempt to add task to staticPool twice")
|
||||
}
|
||||
d.staticPool = append(d.staticPool, task)
|
||||
task.staticPoolIndex = len(d.staticPool) - 1
|
||||
}
|
||||
|
||||
// removeFromStaticPool removes the task at idx from staticPool. It does that by moving the
|
||||
// current last element of the pool to idx and then shortening the pool by one.
|
||||
func (d *dialScheduler) removeFromStaticPool(idx int) {
|
||||
task := d.staticPool[idx]
|
||||
end := len(d.staticPool) - 1
|
||||
d.staticPool[idx] = d.staticPool[end]
|
||||
d.staticPool[idx].staticPoolIndex = idx
|
||||
d.staticPool[end] = nil
|
||||
d.staticPool = d.staticPool[:end]
|
||||
task.staticPoolIndex = -1
|
||||
}
|
||||
|
||||
// startDial runs the given dial task in a separate goroutine.
|
||||
func (d *dialScheduler) startDial(task *dialTask) {
|
||||
d.log.Trace("Starting p2p dial", "id", task.dest.ID(), "ip", task.dest.IP(), "flag", task.flags)
|
||||
hkey := string(task.dest.ID().Bytes())
|
||||
d.history.add(hkey, d.clock.Now().Add(dialHistoryExpiration))
|
||||
d.dialing[task.dest.ID()] = task
|
||||
go func() {
|
||||
task.run(d)
|
||||
d.doneCh <- task
|
||||
}()
|
||||
}
|
||||
|
||||
// A dialTask generated for each node that is dialed.
|
||||
type dialTask struct {
|
||||
flags connFlag
|
||||
staticPoolIndex int
|
||||
flags connFlag
|
||||
// These fields are private to the task and should not be
|
||||
// accessed by dialScheduler while the task is running.
|
||||
dest *enode.Node
|
||||
lastResolved time.Time
|
||||
lastResolved mclock.AbsTime
|
||||
resolveDelay time.Duration
|
||||
}
|
||||
|
||||
func (t *dialTask) Do(srv *Server) {
|
||||
func newDialTask(dest *enode.Node, flags connFlag) *dialTask {
|
||||
return &dialTask{dest: dest, flags: flags, staticPoolIndex: -1}
|
||||
}
|
||||
|
||||
type dialError struct {
|
||||
error
|
||||
}
|
||||
|
||||
func (t *dialTask) run(d *dialScheduler) {
|
||||
if t.dest.Incomplete() {
|
||||
if !t.resolve(srv) {
|
||||
if !t.resolve(d) {
|
||||
return
|
||||
}
|
||||
}
|
||||
err := t.dial(srv, t.dest)
|
||||
|
||||
err := t.dial(d, t.dest)
|
||||
if err != nil {
|
||||
srv.log.Trace("Dial error", "task", t, "err", err)
|
||||
// Try resolving the ID of static nodes if dialing failed.
|
||||
if _, ok := err.(*dialError); ok && t.flags&staticDialedConn != 0 {
|
||||
if t.resolve(srv) {
|
||||
t.dial(srv, t.dest)
|
||||
if t.resolve(d) {
|
||||
t.dial(d, t.dest)
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -266,46 +497,42 @@ func (t *dialTask) Do(srv *Server) {
|
||||
// Resolve operations are throttled with backoff to avoid flooding the
|
||||
// discovery network with useless queries for nodes that don't exist.
|
||||
// The backoff delay resets when the node is found.
|
||||
func (t *dialTask) resolve(srv *Server) bool {
|
||||
if srv.staticNodeResolver == nil {
|
||||
srv.log.Debug("Can't resolve node", "id", t.dest.ID(), "err", "discovery is disabled")
|
||||
func (t *dialTask) resolve(d *dialScheduler) bool {
|
||||
if d.resolver == nil {
|
||||
return false
|
||||
}
|
||||
if t.resolveDelay == 0 {
|
||||
t.resolveDelay = initialResolveDelay
|
||||
}
|
||||
if time.Since(t.lastResolved) < t.resolveDelay {
|
||||
if t.lastResolved > 0 && time.Duration(d.clock.Now()-t.lastResolved) < t.resolveDelay {
|
||||
return false
|
||||
}
|
||||
resolved := srv.staticNodeResolver.Resolve(t.dest)
|
||||
t.lastResolved = time.Now()
|
||||
resolved := d.resolver.Resolve(t.dest)
|
||||
t.lastResolved = d.clock.Now()
|
||||
if resolved == nil {
|
||||
t.resolveDelay *= 2
|
||||
if t.resolveDelay > maxResolveDelay {
|
||||
t.resolveDelay = maxResolveDelay
|
||||
}
|
||||
srv.log.Debug("Resolving node failed", "id", t.dest.ID(), "newdelay", t.resolveDelay)
|
||||
d.log.Debug("Resolving node failed", "id", t.dest.ID(), "newdelay", t.resolveDelay)
|
||||
return false
|
||||
}
|
||||
// The node was found.
|
||||
t.resolveDelay = initialResolveDelay
|
||||
t.dest = resolved
|
||||
srv.log.Debug("Resolved node", "id", t.dest.ID(), "addr", &net.TCPAddr{IP: t.dest.IP(), Port: t.dest.TCP()})
|
||||
d.log.Debug("Resolved node", "id", t.dest.ID(), "addr", &net.TCPAddr{IP: t.dest.IP(), Port: t.dest.TCP()})
|
||||
return true
|
||||
}
|
||||
|
||||
type dialError struct {
|
||||
error
|
||||
}
|
||||
|
||||
// dial performs the actual connection attempt.
|
||||
func (t *dialTask) dial(srv *Server, dest *enode.Node) error {
|
||||
fd, err := srv.Dialer.Dial(dest)
|
||||
func (t *dialTask) dial(d *dialScheduler, dest *enode.Node) error {
|
||||
fd, err := d.dialer.Dial(d.ctx, t.dest)
|
||||
if err != nil {
|
||||
d.log.Trace("Dial error", "id", t.dest.ID(), "addr", nodeAddr(t.dest), "conn", t.flags, "err", cleanupDialErr(err))
|
||||
return &dialError{err}
|
||||
}
|
||||
mfd := newMeteredConn(fd, false, &net.TCPAddr{IP: dest.IP(), Port: dest.TCP()})
|
||||
return srv.SetupConn(mfd, t.flags, dest)
|
||||
return d.setupFunc(mfd, t.flags, dest)
|
||||
}
|
||||
|
||||
func (t *dialTask) String() string {
|
||||
@ -313,37 +540,9 @@ func (t *dialTask) String() string {
|
||||
return fmt.Sprintf("%v %x %v:%d", t.flags, id[:8], t.dest.IP(), t.dest.TCP())
|
||||
}
|
||||
|
||||
// discoverTask runs discovery table operations.
|
||||
// Only one discoverTask is active at any time.
|
||||
// discoverTask.Do performs a random lookup.
|
||||
type discoverTask struct {
|
||||
want int
|
||||
results []*enode.Node
|
||||
}
|
||||
|
||||
func (t *discoverTask) Do(srv *Server) {
|
||||
t.results = enode.ReadNodes(srv.discmix, t.want)
|
||||
}
|
||||
|
||||
func (t *discoverTask) String() string {
|
||||
s := "discovery query"
|
||||
if len(t.results) > 0 {
|
||||
s += fmt.Sprintf(" (%d results)", len(t.results))
|
||||
} else {
|
||||
s += fmt.Sprintf(" (want %d)", t.want)
|
||||
func cleanupDialErr(err error) error {
|
||||
if netErr, ok := err.(*net.OpError); ok && netErr.Op == "dial" {
|
||||
return netErr.Err
|
||||
}
|
||||
return s
|
||||
}
|
||||
|
||||
// A waitExpireTask is generated if there are no other tasks
|
||||
// to keep the loop in Server.run ticking.
|
||||
type waitExpireTask struct {
|
||||
time.Duration
|
||||
}
|
||||
|
||||
func (t waitExpireTask) Do(*Server) {
|
||||
time.Sleep(t.Duration)
|
||||
}
|
||||
func (t waitExpireTask) String() string {
|
||||
return fmt.Sprintf("wait for dial hist expire (%v)", t.Duration)
|
||||
return err
|
||||
}
|
||||
|
Reference in New Issue
Block a user