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:
parent
5f2002bbcc
commit
90caa2cabb
637
p2p/dial.go
637
p2p/dial.go
@ -17,11 +17,17 @@
|
|||||||
package p2p
|
package p2p
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"context"
|
||||||
|
crand "crypto/rand"
|
||||||
|
"encoding/binary"
|
||||||
"errors"
|
"errors"
|
||||||
"fmt"
|
"fmt"
|
||||||
|
mrand "math/rand"
|
||||||
"net"
|
"net"
|
||||||
|
"sync"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/ethereum/go-ethereum/common/mclock"
|
||||||
"github.com/ethereum/go-ethereum/log"
|
"github.com/ethereum/go-ethereum/log"
|
||||||
"github.com/ethereum/go-ethereum/p2p/enode"
|
"github.com/ethereum/go-ethereum/p2p/enode"
|
||||||
"github.com/ethereum/go-ethereum/p2p/netutil"
|
"github.com/ethereum/go-ethereum/p2p/netutil"
|
||||||
@ -33,8 +39,9 @@ const (
|
|||||||
// private networks.
|
// private networks.
|
||||||
dialHistoryExpiration = inboundThrottleTime + 5*time.Second
|
dialHistoryExpiration = inboundThrottleTime + 5*time.Second
|
||||||
|
|
||||||
// If no peers are found for this amount of time, the initial bootnodes are dialed.
|
// Config for the "Looking for peers" message.
|
||||||
fallbackInterval = 20 * time.Second
|
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.
|
// Endpoint resolution is throttled with bounded backoff.
|
||||||
initialResolveDelay = 60 * time.Second
|
initialResolveDelay = 60 * time.Second
|
||||||
@ -42,161 +49,29 @@ const (
|
|||||||
)
|
)
|
||||||
|
|
||||||
// NodeDialer is used to connect to nodes in the network, typically by using
|
// 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 {
|
type NodeDialer interface {
|
||||||
Dial(*enode.Node) (net.Conn, error)
|
Dial(context.Context, *enode.Node) (net.Conn, error)
|
||||||
}
|
}
|
||||||
|
|
||||||
type nodeResolver interface {
|
type nodeResolver interface {
|
||||||
Resolve(*enode.Node) *enode.Node
|
Resolve(*enode.Node) *enode.Node
|
||||||
}
|
}
|
||||||
|
|
||||||
// TCPDialer implements the NodeDialer interface by using a net.Dialer to
|
// tcpDialer implements NodeDialer using real TCP connections.
|
||||||
// create TCP connections to nodes in the network
|
type tcpDialer struct {
|
||||||
type TCPDialer struct {
|
d *net.Dialer
|
||||||
*net.Dialer
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Dial creates a TCP connection to the node
|
func (t tcpDialer) Dial(ctx context.Context, dest *enode.Node) (net.Conn, error) {
|
||||||
func (t TCPDialer) Dial(dest *enode.Node) (net.Conn, error) {
|
return t.d.DialContext(ctx, "tcp", nodeAddr(dest).String())
|
||||||
addr := &net.TCPAddr{IP: dest.IP(), Port: dest.TCP()}
|
|
||||||
return t.Dialer.Dial("tcp", addr.String())
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// dialstate schedules dials and discovery lookups.
|
func nodeAddr(n *enode.Node) net.Addr {
|
||||||
// It gets a chance to compute new tasks on every iteration
|
return &net.TCPAddr{IP: n.IP(), Port: n.TCP()}
|
||||||
// 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
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// checkDial errors:
|
||||||
var (
|
var (
|
||||||
errSelf = errors.New("is self")
|
errSelf = errors.New("is self")
|
||||||
errAlreadyDialing = errors.New("already dialing")
|
errAlreadyDialing = errors.New("already dialing")
|
||||||
@ -205,56 +80,412 @@ var (
|
|||||||
errNotWhitelisted = errors.New("not contained in netrestrict whitelist")
|
errNotWhitelisted = errors.New("not contained in netrestrict whitelist")
|
||||||
)
|
)
|
||||||
|
|
||||||
func (s *dialstate) checkDial(n *enode.Node, peers map[enode.ID]*Peer) error {
|
// dialer creates outbound connections and submits them into Server.
|
||||||
_, dialing := s.dialing[n.ID()]
|
// Two types of peer connections can be created:
|
||||||
switch {
|
//
|
||||||
case dialing:
|
// - static dials are pre-configured connections. The dialer attempts
|
||||||
return errAlreadyDialing
|
// keep these nodes connected at all times.
|
||||||
case peers[n.ID()] != nil:
|
//
|
||||||
return errAlreadyConnected
|
// - dynamic dials are created from node discovery results. The dialer
|
||||||
case n.ID() == s.self:
|
// 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
|
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
|
return errNotWhitelisted
|
||||||
case s.hist.contains(string(n.ID().Bytes())):
|
}
|
||||||
|
if d.history.contains(string(n.ID().Bytes())) {
|
||||||
return errRecentlyDialed
|
return errRecentlyDialed
|
||||||
}
|
}
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *dialstate) taskDone(t task, now time.Time) {
|
// startStaticDials starts n static dial tasks.
|
||||||
switch t := t.(type) {
|
func (d *dialScheduler) startStaticDials(n int) (started int) {
|
||||||
case *dialTask:
|
for started = 0; started < n && len(d.staticPool) > 0; started++ {
|
||||||
s.hist.add(string(t.dest.ID().Bytes()), now.Add(dialHistoryExpiration))
|
idx := d.rand.Intn(len(d.staticPool))
|
||||||
delete(s.dialing, t.dest.ID())
|
task := d.staticPool[idx]
|
||||||
case *discoverTask:
|
d.startDial(task)
|
||||||
s.lookupRunning = false
|
d.removeFromStaticPool(idx)
|
||||||
s.lookupBuf = append(s.lookupBuf, t.results...)
|
}
|
||||||
|
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
|
func (d *dialScheduler) addToStaticPool(task *dialTask) {
|
||||||
// fields cannot be accessed while the task is running.
|
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 {
|
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
|
dest *enode.Node
|
||||||
lastResolved time.Time
|
lastResolved mclock.AbsTime
|
||||||
resolveDelay time.Duration
|
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.dest.Incomplete() {
|
||||||
if !t.resolve(srv) {
|
if !t.resolve(d) {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
err := t.dial(srv, t.dest)
|
|
||||||
|
err := t.dial(d, t.dest)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
srv.log.Trace("Dial error", "task", t, "err", err)
|
|
||||||
// Try resolving the ID of static nodes if dialing failed.
|
// Try resolving the ID of static nodes if dialing failed.
|
||||||
if _, ok := err.(*dialError); ok && t.flags&staticDialedConn != 0 {
|
if _, ok := err.(*dialError); ok && t.flags&staticDialedConn != 0 {
|
||||||
if t.resolve(srv) {
|
if t.resolve(d) {
|
||||||
t.dial(srv, t.dest)
|
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
|
// Resolve operations are throttled with backoff to avoid flooding the
|
||||||
// discovery network with useless queries for nodes that don't exist.
|
// discovery network with useless queries for nodes that don't exist.
|
||||||
// The backoff delay resets when the node is found.
|
// The backoff delay resets when the node is found.
|
||||||
func (t *dialTask) resolve(srv *Server) bool {
|
func (t *dialTask) resolve(d *dialScheduler) bool {
|
||||||
if srv.staticNodeResolver == nil {
|
if d.resolver == nil {
|
||||||
srv.log.Debug("Can't resolve node", "id", t.dest.ID(), "err", "discovery is disabled")
|
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
if t.resolveDelay == 0 {
|
if t.resolveDelay == 0 {
|
||||||
t.resolveDelay = initialResolveDelay
|
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
|
return false
|
||||||
}
|
}
|
||||||
resolved := srv.staticNodeResolver.Resolve(t.dest)
|
resolved := d.resolver.Resolve(t.dest)
|
||||||
t.lastResolved = time.Now()
|
t.lastResolved = d.clock.Now()
|
||||||
if resolved == nil {
|
if resolved == nil {
|
||||||
t.resolveDelay *= 2
|
t.resolveDelay *= 2
|
||||||
if t.resolveDelay > maxResolveDelay {
|
if t.resolveDelay > maxResolveDelay {
|
||||||
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
|
return false
|
||||||
}
|
}
|
||||||
// The node was found.
|
// The node was found.
|
||||||
t.resolveDelay = initialResolveDelay
|
t.resolveDelay = initialResolveDelay
|
||||||
t.dest = resolved
|
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
|
return true
|
||||||
}
|
}
|
||||||
|
|
||||||
type dialError struct {
|
|
||||||
error
|
|
||||||
}
|
|
||||||
|
|
||||||
// dial performs the actual connection attempt.
|
// dial performs the actual connection attempt.
|
||||||
func (t *dialTask) dial(srv *Server, dest *enode.Node) error {
|
func (t *dialTask) dial(d *dialScheduler, dest *enode.Node) error {
|
||||||
fd, err := srv.Dialer.Dial(dest)
|
fd, err := d.dialer.Dial(d.ctx, t.dest)
|
||||||
if err != nil {
|
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}
|
return &dialError{err}
|
||||||
}
|
}
|
||||||
mfd := newMeteredConn(fd, false, &net.TCPAddr{IP: dest.IP(), Port: dest.TCP()})
|
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 {
|
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())
|
return fmt.Sprintf("%v %x %v:%d", t.flags, id[:8], t.dest.IP(), t.dest.TCP())
|
||||||
}
|
}
|
||||||
|
|
||||||
// discoverTask runs discovery table operations.
|
func cleanupDialErr(err error) error {
|
||||||
// Only one discoverTask is active at any time.
|
if netErr, ok := err.(*net.OpError); ok && netErr.Op == "dial" {
|
||||||
// discoverTask.Do performs a random lookup.
|
return netErr.Err
|
||||||
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)
|
|
||||||
}
|
}
|
||||||
return s
|
return err
|
||||||
}
|
|
||||||
|
|
||||||
// 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)
|
|
||||||
}
|
}
|
||||||
|
1084
p2p/dial_test.go
1084
p2p/dial_test.go
File diff suppressed because it is too large
Load Diff
@ -17,15 +17,20 @@
|
|||||||
package p2p
|
package p2p
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"encoding/binary"
|
||||||
"errors"
|
"errors"
|
||||||
"fmt"
|
"fmt"
|
||||||
"math/rand"
|
"math/rand"
|
||||||
"net"
|
"net"
|
||||||
"reflect"
|
"reflect"
|
||||||
|
"strconv"
|
||||||
|
"strings"
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/ethereum/go-ethereum/log"
|
"github.com/ethereum/go-ethereum/log"
|
||||||
|
"github.com/ethereum/go-ethereum/p2p/enode"
|
||||||
|
"github.com/ethereum/go-ethereum/p2p/enr"
|
||||||
)
|
)
|
||||||
|
|
||||||
var discard = Protocol{
|
var discard = Protocol{
|
||||||
@ -45,10 +50,45 @@ var discard = Protocol{
|
|||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// uintID encodes i into a node ID.
|
||||||
|
func uintID(i uint16) enode.ID {
|
||||||
|
var id enode.ID
|
||||||
|
binary.BigEndian.PutUint16(id[:], i)
|
||||||
|
return id
|
||||||
|
}
|
||||||
|
|
||||||
|
// newNode creates a node record with the given address.
|
||||||
|
func newNode(id enode.ID, addr string) *enode.Node {
|
||||||
|
var r enr.Record
|
||||||
|
if addr != "" {
|
||||||
|
// Set the port if present.
|
||||||
|
if strings.Contains(addr, ":") {
|
||||||
|
hs, ps, err := net.SplitHostPort(addr)
|
||||||
|
if err != nil {
|
||||||
|
panic(fmt.Errorf("invalid address %q", addr))
|
||||||
|
}
|
||||||
|
port, err := strconv.Atoi(ps)
|
||||||
|
if err != nil {
|
||||||
|
panic(fmt.Errorf("invalid port in %q", addr))
|
||||||
|
}
|
||||||
|
r.Set(enr.TCP(port))
|
||||||
|
r.Set(enr.UDP(port))
|
||||||
|
addr = hs
|
||||||
|
}
|
||||||
|
// Set the IP.
|
||||||
|
ip := net.ParseIP(addr)
|
||||||
|
if ip == nil {
|
||||||
|
panic(fmt.Errorf("invalid IP %q", addr))
|
||||||
|
}
|
||||||
|
r.Set(enr.IP(ip))
|
||||||
|
}
|
||||||
|
return enode.SignNull(&r, id)
|
||||||
|
}
|
||||||
|
|
||||||
func testPeer(protos []Protocol) (func(), *conn, *Peer, <-chan error) {
|
func testPeer(protos []Protocol) (func(), *conn, *Peer, <-chan error) {
|
||||||
fd1, fd2 := net.Pipe()
|
fd1, fd2 := net.Pipe()
|
||||||
c1 := &conn{fd: fd1, node: newNode(randomID(), nil), transport: newTestTransport(&newkey().PublicKey, fd1)}
|
c1 := &conn{fd: fd1, node: newNode(randomID(), ""), transport: newTestTransport(&newkey().PublicKey, fd1)}
|
||||||
c2 := &conn{fd: fd2, node: newNode(randomID(), nil), transport: newTestTransport(&newkey().PublicKey, fd2)}
|
c2 := &conn{fd: fd2, node: newNode(randomID(), ""), transport: newTestTransport(&newkey().PublicKey, fd2)}
|
||||||
for _, p := range protos {
|
for _, p := range protos {
|
||||||
c1.caps = append(c1.caps, p.cap())
|
c1.caps = append(c1.caps, p.cap())
|
||||||
c2.caps = append(c2.caps, p.cap())
|
c2.caps = append(c2.caps, p.cap())
|
||||||
|
320
p2p/server.go
320
p2p/server.go
@ -51,7 +51,6 @@ const (
|
|||||||
discmixTimeout = 5 * time.Second
|
discmixTimeout = 5 * time.Second
|
||||||
|
|
||||||
// Connectivity defaults.
|
// Connectivity defaults.
|
||||||
maxActiveDialTasks = 16
|
|
||||||
defaultMaxPendingPeers = 50
|
defaultMaxPendingPeers = 50
|
||||||
defaultDialRatio = 3
|
defaultDialRatio = 3
|
||||||
|
|
||||||
@ -156,6 +155,8 @@ type Config struct {
|
|||||||
|
|
||||||
// Logger is a custom logger to use with the p2p.Server.
|
// Logger is a custom logger to use with the p2p.Server.
|
||||||
Logger log.Logger `toml:",omitempty"`
|
Logger log.Logger `toml:",omitempty"`
|
||||||
|
|
||||||
|
clock mclock.Clock
|
||||||
}
|
}
|
||||||
|
|
||||||
// Server manages all peer connections.
|
// Server manages all peer connections.
|
||||||
@ -183,13 +184,10 @@ type Server struct {
|
|||||||
ntab *discover.UDPv4
|
ntab *discover.UDPv4
|
||||||
DiscV5 *discv5.Network
|
DiscV5 *discv5.Network
|
||||||
discmix *enode.FairMix
|
discmix *enode.FairMix
|
||||||
|
dialsched *dialScheduler
|
||||||
staticNodeResolver nodeResolver
|
|
||||||
|
|
||||||
// Channels into the run loop.
|
// Channels into the run loop.
|
||||||
quit chan struct{}
|
quit chan struct{}
|
||||||
addstatic chan *enode.Node
|
|
||||||
removestatic chan *enode.Node
|
|
||||||
addtrusted chan *enode.Node
|
addtrusted chan *enode.Node
|
||||||
removetrusted chan *enode.Node
|
removetrusted chan *enode.Node
|
||||||
peerOp chan peerOpFunc
|
peerOp chan peerOpFunc
|
||||||
@ -302,47 +300,57 @@ func (srv *Server) LocalNode() *enode.LocalNode {
|
|||||||
// Peers returns all connected peers.
|
// Peers returns all connected peers.
|
||||||
func (srv *Server) Peers() []*Peer {
|
func (srv *Server) Peers() []*Peer {
|
||||||
var ps []*Peer
|
var ps []*Peer
|
||||||
select {
|
srv.doPeerOp(func(peers map[enode.ID]*Peer) {
|
||||||
// Note: We'd love to put this function into a variable but
|
|
||||||
// that seems to cause a weird compiler error in some
|
|
||||||
// environments.
|
|
||||||
case srv.peerOp <- func(peers map[enode.ID]*Peer) {
|
|
||||||
for _, p := range peers {
|
for _, p := range peers {
|
||||||
ps = append(ps, p)
|
ps = append(ps, p)
|
||||||
}
|
}
|
||||||
}:
|
})
|
||||||
<-srv.peerOpDone
|
|
||||||
case <-srv.quit:
|
|
||||||
}
|
|
||||||
return ps
|
return ps
|
||||||
}
|
}
|
||||||
|
|
||||||
// PeerCount returns the number of connected peers.
|
// PeerCount returns the number of connected peers.
|
||||||
func (srv *Server) PeerCount() int {
|
func (srv *Server) PeerCount() int {
|
||||||
var count int
|
var count int
|
||||||
select {
|
srv.doPeerOp(func(ps map[enode.ID]*Peer) {
|
||||||
case srv.peerOp <- func(ps map[enode.ID]*Peer) { count = len(ps) }:
|
count = len(ps)
|
||||||
<-srv.peerOpDone
|
})
|
||||||
case <-srv.quit:
|
|
||||||
}
|
|
||||||
return count
|
return count
|
||||||
}
|
}
|
||||||
|
|
||||||
// AddPeer connects to the given node and maintains the connection until the
|
// AddPeer adds the given node to the static node set. When there is room in the peer set,
|
||||||
// server is shut down. If the connection fails for any reason, the server will
|
// the server will connect to the node. If the connection fails for any reason, the server
|
||||||
// attempt to reconnect the peer.
|
// will attempt to reconnect the peer.
|
||||||
func (srv *Server) AddPeer(node *enode.Node) {
|
func (srv *Server) AddPeer(node *enode.Node) {
|
||||||
select {
|
srv.dialsched.addStatic(node)
|
||||||
case srv.addstatic <- node:
|
|
||||||
case <-srv.quit:
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// RemovePeer disconnects from the given node
|
// RemovePeer removes a node from the static node set. It also disconnects from the given
|
||||||
|
// node if it is currently connected as a peer.
|
||||||
|
//
|
||||||
|
// This method blocks until all protocols have exited and the peer is removed. Do not use
|
||||||
|
// RemovePeer in protocol implementations, call Disconnect on the Peer instead.
|
||||||
func (srv *Server) RemovePeer(node *enode.Node) {
|
func (srv *Server) RemovePeer(node *enode.Node) {
|
||||||
select {
|
var (
|
||||||
case srv.removestatic <- node:
|
ch chan *PeerEvent
|
||||||
case <-srv.quit:
|
sub event.Subscription
|
||||||
|
)
|
||||||
|
// Disconnect the peer on the main loop.
|
||||||
|
srv.doPeerOp(func(peers map[enode.ID]*Peer) {
|
||||||
|
srv.dialsched.removeStatic(node)
|
||||||
|
if peer := peers[node.ID()]; peer != nil {
|
||||||
|
ch = make(chan *PeerEvent, 1)
|
||||||
|
sub = srv.peerFeed.Subscribe(ch)
|
||||||
|
peer.Disconnect(DiscRequested)
|
||||||
|
}
|
||||||
|
})
|
||||||
|
// Wait for the peer connection to end.
|
||||||
|
if ch != nil {
|
||||||
|
defer sub.Unsubscribe()
|
||||||
|
for ev := range ch {
|
||||||
|
if ev.Peer == node.ID() && ev.Type == PeerEventTypeDrop {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -437,6 +445,9 @@ func (srv *Server) Start() (err error) {
|
|||||||
if srv.log == nil {
|
if srv.log == nil {
|
||||||
srv.log = log.Root()
|
srv.log = log.Root()
|
||||||
}
|
}
|
||||||
|
if srv.clock == nil {
|
||||||
|
srv.clock = mclock.System{}
|
||||||
|
}
|
||||||
if srv.NoDial && srv.ListenAddr == "" {
|
if srv.NoDial && srv.ListenAddr == "" {
|
||||||
srv.log.Warn("P2P server will be useless, neither dialing nor listening")
|
srv.log.Warn("P2P server will be useless, neither dialing nor listening")
|
||||||
}
|
}
|
||||||
@ -451,15 +462,10 @@ func (srv *Server) Start() (err error) {
|
|||||||
if srv.listenFunc == nil {
|
if srv.listenFunc == nil {
|
||||||
srv.listenFunc = net.Listen
|
srv.listenFunc = net.Listen
|
||||||
}
|
}
|
||||||
if srv.Dialer == nil {
|
|
||||||
srv.Dialer = TCPDialer{&net.Dialer{Timeout: defaultDialTimeout}}
|
|
||||||
}
|
|
||||||
srv.quit = make(chan struct{})
|
srv.quit = make(chan struct{})
|
||||||
srv.delpeer = make(chan peerDrop)
|
srv.delpeer = make(chan peerDrop)
|
||||||
srv.checkpointPostHandshake = make(chan *conn)
|
srv.checkpointPostHandshake = make(chan *conn)
|
||||||
srv.checkpointAddPeer = make(chan *conn)
|
srv.checkpointAddPeer = make(chan *conn)
|
||||||
srv.addstatic = make(chan *enode.Node)
|
|
||||||
srv.removestatic = make(chan *enode.Node)
|
|
||||||
srv.addtrusted = make(chan *enode.Node)
|
srv.addtrusted = make(chan *enode.Node)
|
||||||
srv.removetrusted = make(chan *enode.Node)
|
srv.removetrusted = make(chan *enode.Node)
|
||||||
srv.peerOp = make(chan peerOpFunc)
|
srv.peerOp = make(chan peerOpFunc)
|
||||||
@ -476,11 +482,10 @@ func (srv *Server) Start() (err error) {
|
|||||||
if err := srv.setupDiscovery(); err != nil {
|
if err := srv.setupDiscovery(); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
srv.setupDialScheduler()
|
||||||
|
|
||||||
dynPeers := srv.maxDialedConns()
|
|
||||||
dialer := newDialState(srv.localnode.ID(), dynPeers, &srv.Config)
|
|
||||||
srv.loopWG.Add(1)
|
srv.loopWG.Add(1)
|
||||||
go srv.run(dialer)
|
go srv.run()
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -583,7 +588,6 @@ func (srv *Server) setupDiscovery() error {
|
|||||||
}
|
}
|
||||||
srv.ntab = ntab
|
srv.ntab = ntab
|
||||||
srv.discmix.AddSource(ntab.RandomNodes())
|
srv.discmix.AddSource(ntab.RandomNodes())
|
||||||
srv.staticNodeResolver = ntab
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Discovery V5
|
// Discovery V5
|
||||||
@ -606,6 +610,47 @@ func (srv *Server) setupDiscovery() error {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (srv *Server) setupDialScheduler() {
|
||||||
|
config := dialConfig{
|
||||||
|
self: srv.localnode.ID(),
|
||||||
|
maxDialPeers: srv.maxDialedConns(),
|
||||||
|
maxActiveDials: srv.MaxPendingPeers,
|
||||||
|
log: srv.Logger,
|
||||||
|
netRestrict: srv.NetRestrict,
|
||||||
|
dialer: srv.Dialer,
|
||||||
|
clock: srv.clock,
|
||||||
|
}
|
||||||
|
if srv.ntab != nil {
|
||||||
|
config.resolver = srv.ntab
|
||||||
|
}
|
||||||
|
if config.dialer == nil {
|
||||||
|
config.dialer = tcpDialer{&net.Dialer{Timeout: defaultDialTimeout}}
|
||||||
|
}
|
||||||
|
srv.dialsched = newDialScheduler(config, srv.discmix, srv.SetupConn)
|
||||||
|
for _, n := range srv.StaticNodes {
|
||||||
|
srv.dialsched.addStatic(n)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (srv *Server) maxInboundConns() int {
|
||||||
|
return srv.MaxPeers - srv.maxDialedConns()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (srv *Server) maxDialedConns() (limit int) {
|
||||||
|
if srv.NoDial || srv.MaxPeers == 0 {
|
||||||
|
return 0
|
||||||
|
}
|
||||||
|
if srv.DialRatio == 0 {
|
||||||
|
limit = srv.MaxPeers / defaultDialRatio
|
||||||
|
} else {
|
||||||
|
limit = srv.MaxPeers / srv.DialRatio
|
||||||
|
}
|
||||||
|
if limit == 0 {
|
||||||
|
limit = 1
|
||||||
|
}
|
||||||
|
return limit
|
||||||
|
}
|
||||||
|
|
||||||
func (srv *Server) setupListening() error {
|
func (srv *Server) setupListening() error {
|
||||||
// Launch the listener.
|
// Launch the listener.
|
||||||
listener, err := srv.listenFunc("tcp", srv.ListenAddr)
|
listener, err := srv.listenFunc("tcp", srv.ListenAddr)
|
||||||
@ -632,112 +677,55 @@ func (srv *Server) setupListening() error {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
type dialer interface {
|
// doPeerOp runs fn on the main loop.
|
||||||
newTasks(running int, peers map[enode.ID]*Peer, now time.Time) []task
|
func (srv *Server) doPeerOp(fn peerOpFunc) {
|
||||||
taskDone(task, time.Time)
|
select {
|
||||||
addStatic(*enode.Node)
|
case srv.peerOp <- fn:
|
||||||
removeStatic(*enode.Node)
|
<-srv.peerOpDone
|
||||||
|
case <-srv.quit:
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (srv *Server) run(dialstate dialer) {
|
// run is the main loop of the server.
|
||||||
|
func (srv *Server) run() {
|
||||||
srv.log.Info("Started P2P networking", "self", srv.localnode.Node().URLv4())
|
srv.log.Info("Started P2P networking", "self", srv.localnode.Node().URLv4())
|
||||||
defer srv.loopWG.Done()
|
defer srv.loopWG.Done()
|
||||||
defer srv.nodedb.Close()
|
defer srv.nodedb.Close()
|
||||||
defer srv.discmix.Close()
|
defer srv.discmix.Close()
|
||||||
|
defer srv.dialsched.stop()
|
||||||
|
|
||||||
var (
|
var (
|
||||||
peers = make(map[enode.ID]*Peer)
|
peers = make(map[enode.ID]*Peer)
|
||||||
inboundCount = 0
|
inboundCount = 0
|
||||||
trusted = make(map[enode.ID]bool, len(srv.TrustedNodes))
|
trusted = make(map[enode.ID]bool, len(srv.TrustedNodes))
|
||||||
taskdone = make(chan task, maxActiveDialTasks)
|
|
||||||
tick = time.NewTicker(30 * time.Second)
|
|
||||||
runningTasks []task
|
|
||||||
queuedTasks []task // tasks that can't run yet
|
|
||||||
)
|
)
|
||||||
defer tick.Stop()
|
|
||||||
|
|
||||||
// Put trusted nodes into a map to speed up checks.
|
// Put trusted nodes into a map to speed up checks.
|
||||||
// Trusted peers are loaded on startup or added via AddTrustedPeer RPC.
|
// Trusted peers are loaded on startup or added via AddTrustedPeer RPC.
|
||||||
for _, n := range srv.TrustedNodes {
|
for _, n := range srv.TrustedNodes {
|
||||||
trusted[n.ID()] = true
|
trusted[n.ID()] = true
|
||||||
}
|
}
|
||||||
|
|
||||||
// removes t from runningTasks
|
|
||||||
delTask := func(t task) {
|
|
||||||
for i := range runningTasks {
|
|
||||||
if runningTasks[i] == t {
|
|
||||||
runningTasks = append(runningTasks[:i], runningTasks[i+1:]...)
|
|
||||||
break
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// starts until max number of active tasks is satisfied
|
|
||||||
startTasks := func(ts []task) (rest []task) {
|
|
||||||
i := 0
|
|
||||||
for ; len(runningTasks) < maxActiveDialTasks && i < len(ts); i++ {
|
|
||||||
t := ts[i]
|
|
||||||
srv.log.Trace("New dial task", "task", t)
|
|
||||||
go func() { t.Do(srv); taskdone <- t }()
|
|
||||||
runningTasks = append(runningTasks, t)
|
|
||||||
}
|
|
||||||
return ts[i:]
|
|
||||||
}
|
|
||||||
scheduleTasks := func() {
|
|
||||||
// Start from queue first.
|
|
||||||
queuedTasks = append(queuedTasks[:0], startTasks(queuedTasks)...)
|
|
||||||
// Query dialer for new tasks and start as many as possible now.
|
|
||||||
if len(runningTasks) < maxActiveDialTasks {
|
|
||||||
nt := dialstate.newTasks(len(runningTasks)+len(queuedTasks), peers, time.Now())
|
|
||||||
queuedTasks = append(queuedTasks, startTasks(nt)...)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
running:
|
running:
|
||||||
for {
|
for {
|
||||||
scheduleTasks()
|
|
||||||
|
|
||||||
select {
|
select {
|
||||||
case <-tick.C:
|
|
||||||
// This is just here to ensure the dial scheduler runs occasionally.
|
|
||||||
|
|
||||||
case <-srv.quit:
|
case <-srv.quit:
|
||||||
// The server was stopped. Run the cleanup logic.
|
// The server was stopped. Run the cleanup logic.
|
||||||
break running
|
break running
|
||||||
|
|
||||||
case n := <-srv.addstatic:
|
|
||||||
// This channel is used by AddPeer to add to the
|
|
||||||
// ephemeral static peer list. Add it to the dialer,
|
|
||||||
// it will keep the node connected.
|
|
||||||
srv.log.Trace("Adding static node", "node", n)
|
|
||||||
dialstate.addStatic(n)
|
|
||||||
|
|
||||||
case n := <-srv.removestatic:
|
|
||||||
// This channel is used by RemovePeer to send a
|
|
||||||
// disconnect request to a peer and begin the
|
|
||||||
// stop keeping the node connected.
|
|
||||||
srv.log.Trace("Removing static node", "node", n)
|
|
||||||
dialstate.removeStatic(n)
|
|
||||||
if p, ok := peers[n.ID()]; ok {
|
|
||||||
p.Disconnect(DiscRequested)
|
|
||||||
}
|
|
||||||
|
|
||||||
case n := <-srv.addtrusted:
|
case n := <-srv.addtrusted:
|
||||||
// This channel is used by AddTrustedPeer to add an enode
|
// This channel is used by AddTrustedPeer to add a node
|
||||||
// to the trusted node set.
|
// to the trusted node set.
|
||||||
srv.log.Trace("Adding trusted node", "node", n)
|
srv.log.Trace("Adding trusted node", "node", n)
|
||||||
trusted[n.ID()] = true
|
trusted[n.ID()] = true
|
||||||
// Mark any already-connected peer as trusted
|
|
||||||
if p, ok := peers[n.ID()]; ok {
|
if p, ok := peers[n.ID()]; ok {
|
||||||
p.rw.set(trustedConn, true)
|
p.rw.set(trustedConn, true)
|
||||||
}
|
}
|
||||||
|
|
||||||
case n := <-srv.removetrusted:
|
case n := <-srv.removetrusted:
|
||||||
// This channel is used by RemoveTrustedPeer to remove an enode
|
// This channel is used by RemoveTrustedPeer to remove a node
|
||||||
// from the trusted node set.
|
// from the trusted node set.
|
||||||
srv.log.Trace("Removing trusted node", "node", n)
|
srv.log.Trace("Removing trusted node", "node", n)
|
||||||
delete(trusted, n.ID())
|
delete(trusted, n.ID())
|
||||||
|
|
||||||
// Unmark any already-connected peer as trusted
|
|
||||||
if p, ok := peers[n.ID()]; ok {
|
if p, ok := peers[n.ID()]; ok {
|
||||||
p.rw.set(trustedConn, false)
|
p.rw.set(trustedConn, false)
|
||||||
}
|
}
|
||||||
@ -747,14 +735,6 @@ running:
|
|||||||
op(peers)
|
op(peers)
|
||||||
srv.peerOpDone <- struct{}{}
|
srv.peerOpDone <- struct{}{}
|
||||||
|
|
||||||
case t := <-taskdone:
|
|
||||||
// A task got done. Tell dialstate about it so it
|
|
||||||
// can update its state and remove it from the active
|
|
||||||
// tasks list.
|
|
||||||
srv.log.Trace("Dial task done", "task", t)
|
|
||||||
dialstate.taskDone(t, time.Now())
|
|
||||||
delTask(t)
|
|
||||||
|
|
||||||
case c := <-srv.checkpointPostHandshake:
|
case c := <-srv.checkpointPostHandshake:
|
||||||
// A connection has passed the encryption handshake so
|
// A connection has passed the encryption handshake so
|
||||||
// the remote identity is known (but hasn't been verified yet).
|
// the remote identity is known (but hasn't been verified yet).
|
||||||
@ -771,33 +751,25 @@ running:
|
|||||||
err := srv.addPeerChecks(peers, inboundCount, c)
|
err := srv.addPeerChecks(peers, inboundCount, c)
|
||||||
if err == nil {
|
if err == nil {
|
||||||
// The handshakes are done and it passed all checks.
|
// The handshakes are done and it passed all checks.
|
||||||
p := newPeer(srv.log, c, srv.Protocols)
|
p := srv.launchPeer(c)
|
||||||
// If message events are enabled, pass the peerFeed
|
|
||||||
// to the peer
|
|
||||||
if srv.EnableMsgEvents {
|
|
||||||
p.events = &srv.peerFeed
|
|
||||||
}
|
|
||||||
name := truncateName(c.name)
|
|
||||||
p.log.Debug("Adding p2p peer", "addr", p.RemoteAddr(), "peers", len(peers)+1, "name", name)
|
|
||||||
go srv.runPeer(p)
|
|
||||||
peers[c.node.ID()] = p
|
peers[c.node.ID()] = p
|
||||||
if p.Inbound() {
|
srv.log.Debug("Adding p2p peer", "peercount", len(peers), "id", p.ID(), "conn", c.flags, "addr", p.RemoteAddr(), "name", truncateName(c.name))
|
||||||
inboundCount++
|
srv.dialsched.peerAdded(c)
|
||||||
}
|
|
||||||
if conn, ok := c.fd.(*meteredConn); ok {
|
if conn, ok := c.fd.(*meteredConn); ok {
|
||||||
conn.handshakeDone(p)
|
conn.handshakeDone(p)
|
||||||
}
|
}
|
||||||
|
if p.Inbound() {
|
||||||
|
inboundCount++
|
||||||
|
}
|
||||||
}
|
}
|
||||||
// The dialer logic relies on the assumption that
|
|
||||||
// dial tasks complete after the peer has been added or
|
|
||||||
// discarded. Unblock the task last.
|
|
||||||
c.cont <- err
|
c.cont <- err
|
||||||
|
|
||||||
case pd := <-srv.delpeer:
|
case pd := <-srv.delpeer:
|
||||||
// A peer disconnected.
|
// A peer disconnected.
|
||||||
d := common.PrettyDuration(mclock.Now() - pd.created)
|
d := common.PrettyDuration(mclock.Now() - pd.created)
|
||||||
pd.log.Debug("Removing p2p peer", "addr", pd.RemoteAddr(), "peers", len(peers)-1, "duration", d, "req", pd.requested, "err", pd.err)
|
|
||||||
delete(peers, pd.ID())
|
delete(peers, pd.ID())
|
||||||
|
srv.log.Debug("Removing p2p peer", "peercount", len(peers), "id", pd.ID(), "duration", d, "req", pd.requested, "err", pd.err)
|
||||||
|
srv.dialsched.peerRemoved(pd.rw)
|
||||||
if pd.Inbound() {
|
if pd.Inbound() {
|
||||||
inboundCount--
|
inboundCount--
|
||||||
}
|
}
|
||||||
@ -822,14 +794,14 @@ running:
|
|||||||
// is closed.
|
// is closed.
|
||||||
for len(peers) > 0 {
|
for len(peers) > 0 {
|
||||||
p := <-srv.delpeer
|
p := <-srv.delpeer
|
||||||
p.log.Trace("<-delpeer (spindown)", "remainingTasks", len(runningTasks))
|
p.log.Trace("<-delpeer (spindown)")
|
||||||
delete(peers, p.ID())
|
delete(peers, p.ID())
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (srv *Server) postHandshakeChecks(peers map[enode.ID]*Peer, inboundCount int, c *conn) error {
|
func (srv *Server) postHandshakeChecks(peers map[enode.ID]*Peer, inboundCount int, c *conn) error {
|
||||||
switch {
|
switch {
|
||||||
case !c.is(trustedConn|staticDialedConn) && len(peers) >= srv.MaxPeers:
|
case !c.is(trustedConn) && len(peers) >= srv.MaxPeers:
|
||||||
return DiscTooManyPeers
|
return DiscTooManyPeers
|
||||||
case !c.is(trustedConn) && c.is(inboundConn) && inboundCount >= srv.maxInboundConns():
|
case !c.is(trustedConn) && c.is(inboundConn) && inboundCount >= srv.maxInboundConns():
|
||||||
return DiscTooManyPeers
|
return DiscTooManyPeers
|
||||||
@ -852,21 +824,6 @@ func (srv *Server) addPeerChecks(peers map[enode.ID]*Peer, inboundCount int, c *
|
|||||||
return srv.postHandshakeChecks(peers, inboundCount, c)
|
return srv.postHandshakeChecks(peers, inboundCount, c)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (srv *Server) maxInboundConns() int {
|
|
||||||
return srv.MaxPeers - srv.maxDialedConns()
|
|
||||||
}
|
|
||||||
|
|
||||||
func (srv *Server) maxDialedConns() int {
|
|
||||||
if srv.NoDiscovery || srv.NoDial {
|
|
||||||
return 0
|
|
||||||
}
|
|
||||||
r := srv.DialRatio
|
|
||||||
if r == 0 {
|
|
||||||
r = defaultDialRatio
|
|
||||||
}
|
|
||||||
return srv.MaxPeers / r
|
|
||||||
}
|
|
||||||
|
|
||||||
// listenLoop runs in its own goroutine and accepts
|
// listenLoop runs in its own goroutine and accepts
|
||||||
// inbound connections.
|
// inbound connections.
|
||||||
func (srv *Server) listenLoop() {
|
func (srv *Server) listenLoop() {
|
||||||
@ -935,18 +892,20 @@ func (srv *Server) listenLoop() {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (srv *Server) checkInboundConn(fd net.Conn, remoteIP net.IP) error {
|
func (srv *Server) checkInboundConn(fd net.Conn, remoteIP net.IP) error {
|
||||||
if remoteIP != nil {
|
if remoteIP == nil {
|
||||||
// Reject connections that do not match NetRestrict.
|
return nil
|
||||||
if srv.NetRestrict != nil && !srv.NetRestrict.Contains(remoteIP) {
|
|
||||||
return fmt.Errorf("not whitelisted in NetRestrict")
|
|
||||||
}
|
|
||||||
// Reject Internet peers that try too often.
|
|
||||||
srv.inboundHistory.expire(time.Now())
|
|
||||||
if !netutil.IsLAN(remoteIP) && srv.inboundHistory.contains(remoteIP.String()) {
|
|
||||||
return fmt.Errorf("too many attempts")
|
|
||||||
}
|
|
||||||
srv.inboundHistory.add(remoteIP.String(), time.Now().Add(inboundThrottleTime))
|
|
||||||
}
|
}
|
||||||
|
// Reject connections that do not match NetRestrict.
|
||||||
|
if srv.NetRestrict != nil && !srv.NetRestrict.Contains(remoteIP) {
|
||||||
|
return fmt.Errorf("not whitelisted in NetRestrict")
|
||||||
|
}
|
||||||
|
// Reject Internet peers that try too often.
|
||||||
|
now := srv.clock.Now()
|
||||||
|
srv.inboundHistory.expire(now, nil)
|
||||||
|
if !netutil.IsLAN(remoteIP) && srv.inboundHistory.contains(remoteIP.String()) {
|
||||||
|
return fmt.Errorf("too many attempts")
|
||||||
|
}
|
||||||
|
srv.inboundHistory.add(remoteIP.String(), now.Add(inboundThrottleTime))
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -958,7 +917,6 @@ func (srv *Server) SetupConn(fd net.Conn, flags connFlag, dialDest *enode.Node)
|
|||||||
err := srv.setupConn(c, flags, dialDest)
|
err := srv.setupConn(c, flags, dialDest)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
c.close(err)
|
c.close(err)
|
||||||
srv.log.Trace("Setting up connection failed", "addr", fd.RemoteAddr(), "err", err)
|
|
||||||
}
|
}
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
@ -977,7 +935,9 @@ func (srv *Server) setupConn(c *conn, flags connFlag, dialDest *enode.Node) erro
|
|||||||
if dialDest != nil {
|
if dialDest != nil {
|
||||||
dialPubkey = new(ecdsa.PublicKey)
|
dialPubkey = new(ecdsa.PublicKey)
|
||||||
if err := dialDest.Load((*enode.Secp256k1)(dialPubkey)); err != nil {
|
if err := dialDest.Load((*enode.Secp256k1)(dialPubkey)); err != nil {
|
||||||
return errors.New("dial destination doesn't have a secp256k1 public key")
|
err = errors.New("dial destination doesn't have a secp256k1 public key")
|
||||||
|
srv.log.Trace("Setting up connection failed", "addr", c.fd.RemoteAddr(), "conn", c.flags, "err", err)
|
||||||
|
return err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1006,7 +966,7 @@ func (srv *Server) setupConn(c *conn, flags connFlag, dialDest *enode.Node) erro
|
|||||||
// Run the capability negotiation handshake.
|
// Run the capability negotiation handshake.
|
||||||
phs, err := c.doProtoHandshake(srv.ourHandshake)
|
phs, err := c.doProtoHandshake(srv.ourHandshake)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
clog.Trace("Failed proto handshake", "err", err)
|
clog.Trace("Failed p2p handshake", "err", err)
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
if id := c.node.ID(); !bytes.Equal(crypto.Keccak256(phs.ID), id[:]) {
|
if id := c.node.ID(); !bytes.Equal(crypto.Keccak256(phs.ID), id[:]) {
|
||||||
@ -1020,9 +980,6 @@ func (srv *Server) setupConn(c *conn, flags connFlag, dialDest *enode.Node) erro
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// If the checks completed successfully, the connection has been added as a peer and
|
|
||||||
// runPeer has been launched.
|
|
||||||
clog.Trace("Connection set up", "inbound", dialDest == nil)
|
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1054,15 +1011,22 @@ func (srv *Server) checkpoint(c *conn, stage chan<- *conn) error {
|
|||||||
return <-c.cont
|
return <-c.cont
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (srv *Server) launchPeer(c *conn) *Peer {
|
||||||
|
p := newPeer(srv.log, c, srv.Protocols)
|
||||||
|
if srv.EnableMsgEvents {
|
||||||
|
// If message events are enabled, pass the peerFeed
|
||||||
|
// to the peer.
|
||||||
|
p.events = &srv.peerFeed
|
||||||
|
}
|
||||||
|
go srv.runPeer(p)
|
||||||
|
return p
|
||||||
|
}
|
||||||
|
|
||||||
// runPeer runs in its own goroutine for each peer.
|
// runPeer runs in its own goroutine for each peer.
|
||||||
// it waits until the Peer logic returns and removes
|
|
||||||
// the peer.
|
|
||||||
func (srv *Server) runPeer(p *Peer) {
|
func (srv *Server) runPeer(p *Peer) {
|
||||||
if srv.newPeerHook != nil {
|
if srv.newPeerHook != nil {
|
||||||
srv.newPeerHook(p)
|
srv.newPeerHook(p)
|
||||||
}
|
}
|
||||||
|
|
||||||
// broadcast peer add
|
|
||||||
srv.peerFeed.Send(&PeerEvent{
|
srv.peerFeed.Send(&PeerEvent{
|
||||||
Type: PeerEventTypeAdd,
|
Type: PeerEventTypeAdd,
|
||||||
Peer: p.ID(),
|
Peer: p.ID(),
|
||||||
@ -1070,10 +1034,18 @@ func (srv *Server) runPeer(p *Peer) {
|
|||||||
LocalAddress: p.LocalAddr().String(),
|
LocalAddress: p.LocalAddr().String(),
|
||||||
})
|
})
|
||||||
|
|
||||||
// run the protocol
|
// Run the per-peer main loop.
|
||||||
remoteRequested, err := p.run()
|
remoteRequested, err := p.run()
|
||||||
|
|
||||||
// broadcast peer drop
|
// Announce disconnect on the main loop to update the peer set.
|
||||||
|
// The main loop waits for existing peers to be sent on srv.delpeer
|
||||||
|
// before returning, so this send should not select on srv.quit.
|
||||||
|
srv.delpeer <- peerDrop{p, err, remoteRequested}
|
||||||
|
|
||||||
|
// Broadcast peer drop to external subscribers. This needs to be
|
||||||
|
// after the send to delpeer so subscribers have a consistent view of
|
||||||
|
// the peer set (i.e. Server.Peers() doesn't include the peer when the
|
||||||
|
// event is received.
|
||||||
srv.peerFeed.Send(&PeerEvent{
|
srv.peerFeed.Send(&PeerEvent{
|
||||||
Type: PeerEventTypeDrop,
|
Type: PeerEventTypeDrop,
|
||||||
Peer: p.ID(),
|
Peer: p.ID(),
|
||||||
@ -1081,10 +1053,6 @@ func (srv *Server) runPeer(p *Peer) {
|
|||||||
RemoteAddress: p.RemoteAddr().String(),
|
RemoteAddress: p.RemoteAddr().String(),
|
||||||
LocalAddress: p.LocalAddr().String(),
|
LocalAddress: p.LocalAddr().String(),
|
||||||
})
|
})
|
||||||
|
|
||||||
// Note: run waits for existing peers to be sent on srv.delpeer
|
|
||||||
// before returning, so this send should not select on srv.quit.
|
|
||||||
srv.delpeer <- peerDrop{p, err, remoteRequested}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// NodeInfo represents a short summary of the information known about the host.
|
// NodeInfo represents a short summary of the information known about the host.
|
||||||
|
@ -34,10 +34,6 @@ import (
|
|||||||
"golang.org/x/crypto/sha3"
|
"golang.org/x/crypto/sha3"
|
||||||
)
|
)
|
||||||
|
|
||||||
// func init() {
|
|
||||||
// log.Root().SetHandler(log.LvlFilterHandler(log.LvlTrace, log.StreamHandler(os.Stderr, log.TerminalFormat(false))))
|
|
||||||
// }
|
|
||||||
|
|
||||||
type testTransport struct {
|
type testTransport struct {
|
||||||
rpub *ecdsa.PublicKey
|
rpub *ecdsa.PublicKey
|
||||||
*rlpx
|
*rlpx
|
||||||
@ -72,11 +68,12 @@ func (c *testTransport) close(err error) {
|
|||||||
|
|
||||||
func startTestServer(t *testing.T, remoteKey *ecdsa.PublicKey, pf func(*Peer)) *Server {
|
func startTestServer(t *testing.T, remoteKey *ecdsa.PublicKey, pf func(*Peer)) *Server {
|
||||||
config := Config{
|
config := Config{
|
||||||
Name: "test",
|
Name: "test",
|
||||||
MaxPeers: 10,
|
MaxPeers: 10,
|
||||||
ListenAddr: "127.0.0.1:0",
|
ListenAddr: "127.0.0.1:0",
|
||||||
PrivateKey: newkey(),
|
NoDiscovery: true,
|
||||||
Logger: testlog.Logger(t, log.LvlTrace),
|
PrivateKey: newkey(),
|
||||||
|
Logger: testlog.Logger(t, log.LvlTrace),
|
||||||
}
|
}
|
||||||
server := &Server{
|
server := &Server{
|
||||||
Config: config,
|
Config: config,
|
||||||
@ -131,11 +128,10 @@ func TestServerDial(t *testing.T) {
|
|||||||
t.Fatalf("could not setup listener: %v", err)
|
t.Fatalf("could not setup listener: %v", err)
|
||||||
}
|
}
|
||||||
defer listener.Close()
|
defer listener.Close()
|
||||||
accepted := make(chan net.Conn)
|
accepted := make(chan net.Conn, 1)
|
||||||
go func() {
|
go func() {
|
||||||
conn, err := listener.Accept()
|
conn, err := listener.Accept()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Error("accept error:", err)
|
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
accepted <- conn
|
accepted <- conn
|
||||||
@ -205,155 +201,38 @@ func TestServerDial(t *testing.T) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// This test checks that tasks generated by dialstate are
|
// This test checks that RemovePeer disconnects the peer if it is connected.
|
||||||
// actually executed and taskdone is called for them.
|
func TestServerRemovePeerDisconnect(t *testing.T) {
|
||||||
func TestServerTaskScheduling(t *testing.T) {
|
srv1 := &Server{Config: Config{
|
||||||
var (
|
PrivateKey: newkey(),
|
||||||
done = make(chan *testTask)
|
MaxPeers: 1,
|
||||||
quit, returned = make(chan struct{}), make(chan struct{})
|
NoDiscovery: true,
|
||||||
tc = 0
|
Logger: testlog.Logger(t, log.LvlTrace).New("server", "1"),
|
||||||
tg = taskgen{
|
}}
|
||||||
newFunc: func(running int, peers map[enode.ID]*Peer) []task {
|
srv2 := &Server{Config: Config{
|
||||||
tc++
|
PrivateKey: newkey(),
|
||||||
return []task{&testTask{index: tc - 1}}
|
MaxPeers: 1,
|
||||||
},
|
NoDiscovery: true,
|
||||||
doneFunc: func(t task) {
|
NoDial: true,
|
||||||
select {
|
ListenAddr: "127.0.0.1:0",
|
||||||
case done <- t.(*testTask):
|
Logger: testlog.Logger(t, log.LvlTrace).New("server", "2"),
|
||||||
case <-quit:
|
}}
|
||||||
}
|
srv1.Start()
|
||||||
},
|
defer srv1.Stop()
|
||||||
}
|
srv2.Start()
|
||||||
)
|
defer srv2.Stop()
|
||||||
|
|
||||||
// The Server in this test isn't actually running
|
if !syncAddPeer(srv1, srv2.Self()) {
|
||||||
// because we're only interested in what run does.
|
t.Fatal("peer not connected")
|
||||||
db, _ := enode.OpenDB("")
|
|
||||||
srv := &Server{
|
|
||||||
Config: Config{MaxPeers: 10},
|
|
||||||
localnode: enode.NewLocalNode(db, newkey()),
|
|
||||||
nodedb: db,
|
|
||||||
discmix: enode.NewFairMix(0),
|
|
||||||
quit: make(chan struct{}),
|
|
||||||
running: true,
|
|
||||||
log: log.New(),
|
|
||||||
}
|
}
|
||||||
srv.loopWG.Add(1)
|
srv1.RemovePeer(srv2.Self())
|
||||||
go func() {
|
if srv1.PeerCount() > 0 {
|
||||||
srv.run(tg)
|
t.Fatal("removed peer still connected")
|
||||||
close(returned)
|
|
||||||
}()
|
|
||||||
|
|
||||||
var gotdone []*testTask
|
|
||||||
for i := 0; i < 100; i++ {
|
|
||||||
gotdone = append(gotdone, <-done)
|
|
||||||
}
|
|
||||||
for i, task := range gotdone {
|
|
||||||
if task.index != i {
|
|
||||||
t.Errorf("task %d has wrong index, got %d", i, task.index)
|
|
||||||
break
|
|
||||||
}
|
|
||||||
if !task.called {
|
|
||||||
t.Errorf("task %d was not called", i)
|
|
||||||
break
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
close(quit)
|
|
||||||
srv.Stop()
|
|
||||||
select {
|
|
||||||
case <-returned:
|
|
||||||
case <-time.After(500 * time.Millisecond):
|
|
||||||
t.Error("Server.run did not return within 500ms")
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// This test checks that Server doesn't drop tasks,
|
// This test checks that connections are disconnected just after the encryption handshake
|
||||||
// even if newTasks returns more than the maximum number of tasks.
|
// when the server is at capacity. Trusted connections should still be accepted.
|
||||||
func TestServerManyTasks(t *testing.T) {
|
|
||||||
alltasks := make([]task, 300)
|
|
||||||
for i := range alltasks {
|
|
||||||
alltasks[i] = &testTask{index: i}
|
|
||||||
}
|
|
||||||
|
|
||||||
var (
|
|
||||||
db, _ = enode.OpenDB("")
|
|
||||||
srv = &Server{
|
|
||||||
quit: make(chan struct{}),
|
|
||||||
localnode: enode.NewLocalNode(db, newkey()),
|
|
||||||
nodedb: db,
|
|
||||||
running: true,
|
|
||||||
log: log.New(),
|
|
||||||
discmix: enode.NewFairMix(0),
|
|
||||||
}
|
|
||||||
done = make(chan *testTask)
|
|
||||||
start, end = 0, 0
|
|
||||||
)
|
|
||||||
defer srv.Stop()
|
|
||||||
srv.loopWG.Add(1)
|
|
||||||
go srv.run(taskgen{
|
|
||||||
newFunc: func(running int, peers map[enode.ID]*Peer) []task {
|
|
||||||
start, end = end, end+maxActiveDialTasks+10
|
|
||||||
if end > len(alltasks) {
|
|
||||||
end = len(alltasks)
|
|
||||||
}
|
|
||||||
return alltasks[start:end]
|
|
||||||
},
|
|
||||||
doneFunc: func(tt task) {
|
|
||||||
done <- tt.(*testTask)
|
|
||||||
},
|
|
||||||
})
|
|
||||||
|
|
||||||
doneset := make(map[int]bool)
|
|
||||||
timeout := time.After(2 * time.Second)
|
|
||||||
for len(doneset) < len(alltasks) {
|
|
||||||
select {
|
|
||||||
case tt := <-done:
|
|
||||||
if doneset[tt.index] {
|
|
||||||
t.Errorf("task %d got done more than once", tt.index)
|
|
||||||
} else {
|
|
||||||
doneset[tt.index] = true
|
|
||||||
}
|
|
||||||
case <-timeout:
|
|
||||||
t.Errorf("%d of %d tasks got done within 2s", len(doneset), len(alltasks))
|
|
||||||
for i := 0; i < len(alltasks); i++ {
|
|
||||||
if !doneset[i] {
|
|
||||||
t.Logf("task %d not done", i)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
type taskgen struct {
|
|
||||||
newFunc func(running int, peers map[enode.ID]*Peer) []task
|
|
||||||
doneFunc func(task)
|
|
||||||
}
|
|
||||||
|
|
||||||
func (tg taskgen) newTasks(running int, peers map[enode.ID]*Peer, now time.Time) []task {
|
|
||||||
return tg.newFunc(running, peers)
|
|
||||||
}
|
|
||||||
func (tg taskgen) taskDone(t task, now time.Time) {
|
|
||||||
tg.doneFunc(t)
|
|
||||||
}
|
|
||||||
func (tg taskgen) addStatic(*enode.Node) {
|
|
||||||
}
|
|
||||||
func (tg taskgen) removeStatic(*enode.Node) {
|
|
||||||
}
|
|
||||||
|
|
||||||
type testTask struct {
|
|
||||||
index int
|
|
||||||
called bool
|
|
||||||
}
|
|
||||||
|
|
||||||
func (t *testTask) Do(srv *Server) {
|
|
||||||
t.called = true
|
|
||||||
}
|
|
||||||
|
|
||||||
// This test checks that connections are disconnected
|
|
||||||
// just after the encryption handshake when the server is
|
|
||||||
// at capacity. Trusted connections should still be accepted.
|
|
||||||
func TestServerAtCap(t *testing.T) {
|
func TestServerAtCap(t *testing.T) {
|
||||||
trustedNode := newkey()
|
trustedNode := newkey()
|
||||||
trustedID := enode.PubkeyToIDV4(&trustedNode.PublicKey)
|
trustedID := enode.PubkeyToIDV4(&trustedNode.PublicKey)
|
||||||
@ -363,7 +242,8 @@ func TestServerAtCap(t *testing.T) {
|
|||||||
MaxPeers: 10,
|
MaxPeers: 10,
|
||||||
NoDial: true,
|
NoDial: true,
|
||||||
NoDiscovery: true,
|
NoDiscovery: true,
|
||||||
TrustedNodes: []*enode.Node{newNode(trustedID, nil)},
|
TrustedNodes: []*enode.Node{newNode(trustedID, "")},
|
||||||
|
Logger: testlog.Logger(t, log.LvlTrace),
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
if err := srv.Start(); err != nil {
|
if err := srv.Start(); err != nil {
|
||||||
@ -401,14 +281,14 @@ func TestServerAtCap(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Remove from trusted set and try again
|
// Remove from trusted set and try again
|
||||||
srv.RemoveTrustedPeer(newNode(trustedID, nil))
|
srv.RemoveTrustedPeer(newNode(trustedID, ""))
|
||||||
c = newconn(trustedID)
|
c = newconn(trustedID)
|
||||||
if err := srv.checkpoint(c, srv.checkpointPostHandshake); err != DiscTooManyPeers {
|
if err := srv.checkpoint(c, srv.checkpointPostHandshake); err != DiscTooManyPeers {
|
||||||
t.Error("wrong error for insert:", err)
|
t.Error("wrong error for insert:", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Add anotherID to trusted set and try again
|
// Add anotherID to trusted set and try again
|
||||||
srv.AddTrustedPeer(newNode(anotherID, nil))
|
srv.AddTrustedPeer(newNode(anotherID, ""))
|
||||||
c = newconn(anotherID)
|
c = newconn(anotherID)
|
||||||
if err := srv.checkpoint(c, srv.checkpointPostHandshake); err != nil {
|
if err := srv.checkpoint(c, srv.checkpointPostHandshake); err != nil {
|
||||||
t.Error("unexpected error for trusted conn @posthandshake:", err)
|
t.Error("unexpected error for trusted conn @posthandshake:", err)
|
||||||
@ -439,9 +319,9 @@ func TestServerPeerLimits(t *testing.T) {
|
|||||||
NoDial: true,
|
NoDial: true,
|
||||||
NoDiscovery: true,
|
NoDiscovery: true,
|
||||||
Protocols: []Protocol{discard},
|
Protocols: []Protocol{discard},
|
||||||
|
Logger: testlog.Logger(t, log.LvlTrace),
|
||||||
},
|
},
|
||||||
newTransport: func(fd net.Conn) transport { return tp },
|
newTransport: func(fd net.Conn) transport { return tp },
|
||||||
log: log.New(),
|
|
||||||
}
|
}
|
||||||
if err := srv.Start(); err != nil {
|
if err := srv.Start(); err != nil {
|
||||||
t.Fatalf("couldn't start server: %v", err)
|
t.Fatalf("couldn't start server: %v", err)
|
||||||
@ -724,3 +604,23 @@ func (l *fakeAddrListener) Accept() (net.Conn, error) {
|
|||||||
func (c *fakeAddrConn) RemoteAddr() net.Addr {
|
func (c *fakeAddrConn) RemoteAddr() net.Addr {
|
||||||
return c.remoteAddr
|
return c.remoteAddr
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func syncAddPeer(srv *Server, node *enode.Node) bool {
|
||||||
|
var (
|
||||||
|
ch = make(chan *PeerEvent)
|
||||||
|
sub = srv.SubscribeEvents(ch)
|
||||||
|
timeout = time.After(2 * time.Second)
|
||||||
|
)
|
||||||
|
defer sub.Unsubscribe()
|
||||||
|
srv.AddPeer(node)
|
||||||
|
for {
|
||||||
|
select {
|
||||||
|
case ev := <-ch:
|
||||||
|
if ev.Type == PeerEventTypeAdd && ev.Peer == node.ID() {
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
case <-timeout:
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
@ -17,6 +17,7 @@
|
|||||||
package adapters
|
package adapters
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"context"
|
||||||
"errors"
|
"errors"
|
||||||
"fmt"
|
"fmt"
|
||||||
"math"
|
"math"
|
||||||
@ -126,7 +127,7 @@ func (s *SimAdapter) NewNode(config *NodeConfig) (Node, error) {
|
|||||||
|
|
||||||
// Dial implements the p2p.NodeDialer interface by connecting to the node using
|
// Dial implements the p2p.NodeDialer interface by connecting to the node using
|
||||||
// an in-memory net.Pipe
|
// an in-memory net.Pipe
|
||||||
func (s *SimAdapter) Dial(dest *enode.Node) (conn net.Conn, err error) {
|
func (s *SimAdapter) Dial(ctx context.Context, dest *enode.Node) (conn net.Conn, err error) {
|
||||||
node, ok := s.GetNode(dest.ID())
|
node, ok := s.GetNode(dest.ID())
|
||||||
if !ok {
|
if !ok {
|
||||||
return nil, fmt.Errorf("unknown node: %s", dest.ID())
|
return nil, fmt.Errorf("unknown node: %s", dest.ID())
|
||||||
|
20
p2p/util.go
20
p2p/util.go
@ -18,7 +18,8 @@ package p2p
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"container/heap"
|
"container/heap"
|
||||||
"time"
|
|
||||||
|
"github.com/ethereum/go-ethereum/common/mclock"
|
||||||
)
|
)
|
||||||
|
|
||||||
// expHeap tracks strings and their expiry time.
|
// expHeap tracks strings and their expiry time.
|
||||||
@ -27,16 +28,16 @@ type expHeap []expItem
|
|||||||
// expItem is an entry in addrHistory.
|
// expItem is an entry in addrHistory.
|
||||||
type expItem struct {
|
type expItem struct {
|
||||||
item string
|
item string
|
||||||
exp time.Time
|
exp mclock.AbsTime
|
||||||
}
|
}
|
||||||
|
|
||||||
// nextExpiry returns the next expiry time.
|
// nextExpiry returns the next expiry time.
|
||||||
func (h *expHeap) nextExpiry() time.Time {
|
func (h *expHeap) nextExpiry() mclock.AbsTime {
|
||||||
return (*h)[0].exp
|
return (*h)[0].exp
|
||||||
}
|
}
|
||||||
|
|
||||||
// add adds an item and sets its expiry time.
|
// add adds an item and sets its expiry time.
|
||||||
func (h *expHeap) add(item string, exp time.Time) {
|
func (h *expHeap) add(item string, exp mclock.AbsTime) {
|
||||||
heap.Push(h, expItem{item, exp})
|
heap.Push(h, expItem{item, exp})
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -51,15 +52,18 @@ func (h expHeap) contains(item string) bool {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// expire removes items with expiry time before 'now'.
|
// expire removes items with expiry time before 'now'.
|
||||||
func (h *expHeap) expire(now time.Time) {
|
func (h *expHeap) expire(now mclock.AbsTime, onExp func(string)) {
|
||||||
for h.Len() > 0 && h.nextExpiry().Before(now) {
|
for h.Len() > 0 && h.nextExpiry() < now {
|
||||||
heap.Pop(h)
|
item := heap.Pop(h)
|
||||||
|
if onExp != nil {
|
||||||
|
onExp(item.(expItem).item)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// heap.Interface boilerplate
|
// heap.Interface boilerplate
|
||||||
func (h expHeap) Len() int { return len(h) }
|
func (h expHeap) Len() int { return len(h) }
|
||||||
func (h expHeap) Less(i, j int) bool { return h[i].exp.Before(h[j].exp) }
|
func (h expHeap) Less(i, j int) bool { return h[i].exp < h[j].exp }
|
||||||
func (h expHeap) Swap(i, j int) { h[i], h[j] = h[j], h[i] }
|
func (h expHeap) Swap(i, j int) { h[i], h[j] = h[j], h[i] }
|
||||||
func (h *expHeap) Push(x interface{}) { *h = append(*h, x.(expItem)) }
|
func (h *expHeap) Push(x interface{}) { *h = append(*h, x.(expItem)) }
|
||||||
func (h *expHeap) Pop() interface{} {
|
func (h *expHeap) Pop() interface{} {
|
||||||
|
@ -19,30 +19,32 @@ package p2p
|
|||||||
import (
|
import (
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/ethereum/go-ethereum/common/mclock"
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestExpHeap(t *testing.T) {
|
func TestExpHeap(t *testing.T) {
|
||||||
var h expHeap
|
var h expHeap
|
||||||
|
|
||||||
var (
|
var (
|
||||||
basetime = time.Unix(4000, 0)
|
basetime = mclock.AbsTime(10)
|
||||||
exptimeA = basetime.Add(2 * time.Second)
|
exptimeA = basetime.Add(2 * time.Second)
|
||||||
exptimeB = basetime.Add(3 * time.Second)
|
exptimeB = basetime.Add(3 * time.Second)
|
||||||
exptimeC = basetime.Add(4 * time.Second)
|
exptimeC = basetime.Add(4 * time.Second)
|
||||||
)
|
)
|
||||||
h.add("a", exptimeA)
|
|
||||||
h.add("b", exptimeB)
|
h.add("b", exptimeB)
|
||||||
|
h.add("a", exptimeA)
|
||||||
h.add("c", exptimeC)
|
h.add("c", exptimeC)
|
||||||
|
|
||||||
if !h.nextExpiry().Equal(exptimeA) {
|
if h.nextExpiry() != exptimeA {
|
||||||
t.Fatal("wrong nextExpiry")
|
t.Fatal("wrong nextExpiry")
|
||||||
}
|
}
|
||||||
if !h.contains("a") || !h.contains("b") || !h.contains("c") {
|
if !h.contains("a") || !h.contains("b") || !h.contains("c") {
|
||||||
t.Fatal("heap doesn't contain all live items")
|
t.Fatal("heap doesn't contain all live items")
|
||||||
}
|
}
|
||||||
|
|
||||||
h.expire(exptimeA.Add(1))
|
h.expire(exptimeA.Add(1), nil)
|
||||||
if !h.nextExpiry().Equal(exptimeB) {
|
if h.nextExpiry() != exptimeB {
|
||||||
t.Fatal("wrong nextExpiry")
|
t.Fatal("wrong nextExpiry")
|
||||||
}
|
}
|
||||||
if h.contains("a") {
|
if h.contains("a") {
|
||||||
|
Loading…
Reference in New Issue
Block a user