p2p: enforce connection retry limit on server side (#19684)
The dialer limits itself to one attempt every 30s. Apply the same limit in Server and reject peers which try to connect too eagerly. The check against the limit happens right after accepting the connection. Further changes in this commit ensure we pass the Server logger down to Peer instances, discovery and dialState. Unit test logging now works in all Server tests.
This commit is contained in:
196
p2p/server.go
196
p2p/server.go
@ -22,6 +22,7 @@ import (
|
||||
"crypto/ecdsa"
|
||||
"encoding/hex"
|
||||
"errors"
|
||||
"fmt"
|
||||
"net"
|
||||
"sort"
|
||||
"sync"
|
||||
@ -49,6 +50,9 @@ const (
|
||||
defaultMaxPendingPeers = 50
|
||||
defaultDialRatio = 3
|
||||
|
||||
// This time limits inbound connection attempts per source IP.
|
||||
inboundThrottleTime = 30 * time.Second
|
||||
|
||||
// Maximum time allowed for reading a complete message.
|
||||
// This is effectively the amount of time a connection can be idle.
|
||||
frameReadTimeout = 30 * time.Second
|
||||
@ -158,6 +162,7 @@ type Server struct {
|
||||
// the whole protocol stack.
|
||||
newTransport func(net.Conn) transport
|
||||
newPeerHook func(*Peer)
|
||||
listenFunc func(network, addr string) (net.Listener, error)
|
||||
|
||||
lock sync.Mutex // protects running
|
||||
running bool
|
||||
@ -167,24 +172,26 @@ type Server struct {
|
||||
ntab discoverTable
|
||||
listener net.Listener
|
||||
ourHandshake *protoHandshake
|
||||
lastLookup time.Time
|
||||
DiscV5 *discv5.Network
|
||||
loopWG sync.WaitGroup // loop, listenLoop
|
||||
peerFeed event.Feed
|
||||
log log.Logger
|
||||
|
||||
// These are for Peers, PeerCount (and nothing else).
|
||||
peerOp chan peerOpFunc
|
||||
peerOpDone chan struct{}
|
||||
// Channels into the run loop.
|
||||
quit chan struct{}
|
||||
addstatic chan *enode.Node
|
||||
removestatic chan *enode.Node
|
||||
addtrusted chan *enode.Node
|
||||
removetrusted chan *enode.Node
|
||||
peerOp chan peerOpFunc
|
||||
peerOpDone chan struct{}
|
||||
delpeer chan peerDrop
|
||||
checkpointPostHandshake chan *conn
|
||||
checkpointAddPeer chan *conn
|
||||
|
||||
quit chan struct{}
|
||||
addstatic chan *enode.Node
|
||||
removestatic chan *enode.Node
|
||||
addtrusted chan *enode.Node
|
||||
removetrusted chan *enode.Node
|
||||
posthandshake chan *conn
|
||||
addpeer chan *conn
|
||||
delpeer chan peerDrop
|
||||
loopWG sync.WaitGroup // loop, listenLoop
|
||||
peerFeed event.Feed
|
||||
log log.Logger
|
||||
// State of run loop and listenLoop.
|
||||
lastLookup time.Time
|
||||
inboundHistory expHeap
|
||||
}
|
||||
|
||||
type peerOpFunc func(map[enode.ID]*Peer)
|
||||
@ -415,7 +422,7 @@ func (srv *Server) Start() (err error) {
|
||||
srv.running = true
|
||||
srv.log = srv.Config.Logger
|
||||
if srv.log == nil {
|
||||
srv.log = log.New()
|
||||
srv.log = log.Root()
|
||||
}
|
||||
if srv.NoDial && srv.ListenAddr == "" {
|
||||
srv.log.Warn("P2P server will be useless, neither dialing nor listening")
|
||||
@ -428,13 +435,16 @@ func (srv *Server) Start() (err error) {
|
||||
if srv.newTransport == nil {
|
||||
srv.newTransport = newRLPX
|
||||
}
|
||||
if srv.listenFunc == nil {
|
||||
srv.listenFunc = net.Listen
|
||||
}
|
||||
if srv.Dialer == nil {
|
||||
srv.Dialer = TCPDialer{&net.Dialer{Timeout: defaultDialTimeout}}
|
||||
}
|
||||
srv.quit = make(chan struct{})
|
||||
srv.addpeer = make(chan *conn)
|
||||
srv.delpeer = make(chan peerDrop)
|
||||
srv.posthandshake = make(chan *conn)
|
||||
srv.checkpointPostHandshake = 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)
|
||||
@ -455,7 +465,7 @@ func (srv *Server) Start() (err error) {
|
||||
}
|
||||
|
||||
dynPeers := srv.maxDialedConns()
|
||||
dialer := newDialState(srv.localnode.ID(), srv.StaticNodes, srv.BootstrapNodes, srv.ntab, dynPeers, srv.NetRestrict)
|
||||
dialer := newDialState(srv.localnode.ID(), srv.ntab, dynPeers, &srv.Config)
|
||||
srv.loopWG.Add(1)
|
||||
go srv.run(dialer)
|
||||
return nil
|
||||
@ -541,6 +551,7 @@ func (srv *Server) setupDiscovery() error {
|
||||
NetRestrict: srv.NetRestrict,
|
||||
Bootnodes: srv.BootstrapNodes,
|
||||
Unhandled: unhandled,
|
||||
Log: srv.log,
|
||||
}
|
||||
ntab, err := discover.ListenUDP(conn, srv.localnode, cfg)
|
||||
if err != nil {
|
||||
@ -569,27 +580,28 @@ func (srv *Server) setupDiscovery() error {
|
||||
}
|
||||
|
||||
func (srv *Server) setupListening() error {
|
||||
// Launch the TCP listener.
|
||||
listener, err := net.Listen("tcp", srv.ListenAddr)
|
||||
// Launch the listener.
|
||||
listener, err := srv.listenFunc("tcp", srv.ListenAddr)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
laddr := listener.Addr().(*net.TCPAddr)
|
||||
srv.ListenAddr = laddr.String()
|
||||
srv.listener = listener
|
||||
srv.localnode.Set(enr.TCP(laddr.Port))
|
||||
srv.ListenAddr = listener.Addr().String()
|
||||
|
||||
// Update the local node record and map the TCP listening port if NAT is configured.
|
||||
if tcp, ok := listener.Addr().(*net.TCPAddr); ok {
|
||||
srv.localnode.Set(enr.TCP(tcp.Port))
|
||||
if !tcp.IP.IsLoopback() && srv.NAT != nil {
|
||||
srv.loopWG.Add(1)
|
||||
go func() {
|
||||
nat.Map(srv.NAT, srv.quit, "tcp", tcp.Port, tcp.Port, "ethereum p2p")
|
||||
srv.loopWG.Done()
|
||||
}()
|
||||
}
|
||||
}
|
||||
|
||||
srv.loopWG.Add(1)
|
||||
go srv.listenLoop()
|
||||
|
||||
// Map the TCP listening port if NAT is configured.
|
||||
if !laddr.IP.IsLoopback() && srv.NAT != nil {
|
||||
srv.loopWG.Add(1)
|
||||
go func() {
|
||||
nat.Map(srv.NAT, srv.quit, "tcp", laddr.Port, laddr.Port, "ethereum p2p")
|
||||
srv.loopWG.Done()
|
||||
}()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -657,12 +669,14 @@ running:
|
||||
case <-srv.quit:
|
||||
// The server was stopped. Run the cleanup logic.
|
||||
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
|
||||
@ -672,6 +686,7 @@ running:
|
||||
if p, ok := peers[n.ID()]; ok {
|
||||
p.Disconnect(DiscRequested)
|
||||
}
|
||||
|
||||
case n := <-srv.addtrusted:
|
||||
// This channel is used by AddTrustedPeer to add an enode
|
||||
// to the trusted node set.
|
||||
@ -681,6 +696,7 @@ running:
|
||||
if p, ok := peers[n.ID()]; ok {
|
||||
p.rw.set(trustedConn, true)
|
||||
}
|
||||
|
||||
case n := <-srv.removetrusted:
|
||||
// This channel is used by RemoveTrustedPeer to remove an enode
|
||||
// from the trusted node set.
|
||||
@ -691,10 +707,12 @@ running:
|
||||
if p, ok := peers[n.ID()]; ok {
|
||||
p.rw.set(trustedConn, false)
|
||||
}
|
||||
|
||||
case op := <-srv.peerOp:
|
||||
// This channel is used by Peers and PeerCount.
|
||||
op(peers)
|
||||
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
|
||||
@ -702,7 +720,8 @@ running:
|
||||
srv.log.Trace("Dial task done", "task", t)
|
||||
dialstate.taskDone(t, time.Now())
|
||||
delTask(t)
|
||||
case c := <-srv.posthandshake:
|
||||
|
||||
case c := <-srv.checkpointPostHandshake:
|
||||
// A connection has passed the encryption handshake so
|
||||
// the remote identity is known (but hasn't been verified yet).
|
||||
if trusted[c.node.ID()] {
|
||||
@ -710,18 +729,15 @@ running:
|
||||
c.flags |= trustedConn
|
||||
}
|
||||
// TODO: track in-progress inbound node IDs (pre-Peer) to avoid dialing them.
|
||||
select {
|
||||
case c.cont <- srv.encHandshakeChecks(peers, inboundCount, c):
|
||||
case <-srv.quit:
|
||||
break running
|
||||
}
|
||||
case c := <-srv.addpeer:
|
||||
c.cont <- srv.postHandshakeChecks(peers, inboundCount, c)
|
||||
|
||||
case c := <-srv.checkpointAddPeer:
|
||||
// At this point the connection is past the protocol handshake.
|
||||
// Its capabilities are known and the remote identity is verified.
|
||||
err := srv.protoHandshakeChecks(peers, inboundCount, c)
|
||||
err := srv.addPeerChecks(peers, inboundCount, c)
|
||||
if err == nil {
|
||||
// The handshakes are done and it passed all checks.
|
||||
p := newPeer(c, srv.Protocols)
|
||||
p := newPeer(srv.log, c, srv.Protocols)
|
||||
// If message events are enabled, pass the peerFeed
|
||||
// to the peer
|
||||
if srv.EnableMsgEvents {
|
||||
@ -738,11 +754,8 @@ running:
|
||||
// The dialer logic relies on the assumption that
|
||||
// dial tasks complete after the peer has been added or
|
||||
// discarded. Unblock the task last.
|
||||
select {
|
||||
case c.cont <- err:
|
||||
case <-srv.quit:
|
||||
break running
|
||||
}
|
||||
c.cont <- err
|
||||
|
||||
case pd := <-srv.delpeer:
|
||||
// A peer disconnected.
|
||||
d := common.PrettyDuration(mclock.Now() - pd.created)
|
||||
@ -777,17 +790,7 @@ running:
|
||||
}
|
||||
}
|
||||
|
||||
func (srv *Server) protoHandshakeChecks(peers map[enode.ID]*Peer, inboundCount int, c *conn) error {
|
||||
// Drop connections with no matching protocols.
|
||||
if len(srv.Protocols) > 0 && countMatchingProtocols(srv.Protocols, c.caps) == 0 {
|
||||
return DiscUselessPeer
|
||||
}
|
||||
// Repeat the encryption handshake checks because the
|
||||
// peer set might have changed between the handshakes.
|
||||
return srv.encHandshakeChecks(peers, inboundCount, c)
|
||||
}
|
||||
|
||||
func (srv *Server) encHandshakeChecks(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 {
|
||||
case !c.is(trustedConn|staticDialedConn) && len(peers) >= srv.MaxPeers:
|
||||
return DiscTooManyPeers
|
||||
@ -802,9 +805,20 @@ func (srv *Server) encHandshakeChecks(peers map[enode.ID]*Peer, inboundCount int
|
||||
}
|
||||
}
|
||||
|
||||
func (srv *Server) addPeerChecks(peers map[enode.ID]*Peer, inboundCount int, c *conn) error {
|
||||
// Drop connections with no matching protocols.
|
||||
if len(srv.Protocols) > 0 && countMatchingProtocols(srv.Protocols, c.caps) == 0 {
|
||||
return DiscUselessPeer
|
||||
}
|
||||
// Repeat the post-handshake checks because the
|
||||
// peer set might have changed since those checks were performed.
|
||||
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
|
||||
@ -832,7 +846,7 @@ func (srv *Server) listenLoop() {
|
||||
}
|
||||
|
||||
for {
|
||||
// Wait for a handshake slot before accepting.
|
||||
// Wait for a free slot before accepting.
|
||||
<-slots
|
||||
|
||||
var (
|
||||
@ -851,21 +865,16 @@ func (srv *Server) listenLoop() {
|
||||
break
|
||||
}
|
||||
|
||||
// Reject connections that do not match NetRestrict.
|
||||
if srv.NetRestrict != nil {
|
||||
if tcp, ok := fd.RemoteAddr().(*net.TCPAddr); ok && !srv.NetRestrict.Contains(tcp.IP) {
|
||||
srv.log.Debug("Rejected conn (not whitelisted in NetRestrict)", "addr", fd.RemoteAddr())
|
||||
fd.Close()
|
||||
slots <- struct{}{}
|
||||
continue
|
||||
}
|
||||
remoteIP := netutil.AddrIP(fd.RemoteAddr())
|
||||
if err := srv.checkInboundConn(fd, remoteIP); err != nil {
|
||||
srv.log.Debug("Rejected inbound connnection", "addr", fd.RemoteAddr(), "err", err)
|
||||
fd.Close()
|
||||
slots <- struct{}{}
|
||||
continue
|
||||
}
|
||||
|
||||
var ip net.IP
|
||||
if tcp, ok := fd.RemoteAddr().(*net.TCPAddr); ok {
|
||||
ip = tcp.IP
|
||||
if remoteIP != nil {
|
||||
fd = newMeteredConn(fd, true, remoteIP)
|
||||
}
|
||||
fd = newMeteredConn(fd, true, ip)
|
||||
srv.log.Trace("Accepted connection", "addr", fd.RemoteAddr())
|
||||
go func() {
|
||||
srv.SetupConn(fd, inboundConn, nil)
|
||||
@ -874,6 +883,22 @@ func (srv *Server) listenLoop() {
|
||||
}
|
||||
}
|
||||
|
||||
func (srv *Server) checkInboundConn(fd net.Conn, remoteIP net.IP) error {
|
||||
if remoteIP != nil {
|
||||
// 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.
|
||||
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))
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// SetupConn runs the handshakes and attempts to add the connection
|
||||
// as a peer. It returns when the connection has been added as a peer
|
||||
// or the handshakes have failed.
|
||||
@ -895,6 +920,7 @@ func (srv *Server) setupConn(c *conn, flags connFlag, dialDest *enode.Node) erro
|
||||
if !running {
|
||||
return errServerStopped
|
||||
}
|
||||
|
||||
// If dialing, figure out the remote public key.
|
||||
var dialPubkey *ecdsa.PublicKey
|
||||
if dialDest != nil {
|
||||
@ -903,7 +929,8 @@ func (srv *Server) setupConn(c *conn, flags connFlag, dialDest *enode.Node) erro
|
||||
return errors.New("dial destination doesn't have a secp256k1 public key")
|
||||
}
|
||||
}
|
||||
// Run the encryption handshake.
|
||||
|
||||
// Run the RLPx handshake.
|
||||
remotePubkey, err := c.doEncHandshake(srv.PrivateKey, dialPubkey)
|
||||
if err != nil {
|
||||
srv.log.Trace("Failed RLPx handshake", "addr", c.fd.RemoteAddr(), "conn", c.flags, "err", err)
|
||||
@ -922,12 +949,13 @@ func (srv *Server) setupConn(c *conn, flags connFlag, dialDest *enode.Node) erro
|
||||
conn.handshakeDone(c.node.ID())
|
||||
}
|
||||
clog := srv.log.New("id", c.node.ID(), "addr", c.fd.RemoteAddr(), "conn", c.flags)
|
||||
err = srv.checkpoint(c, srv.posthandshake)
|
||||
err = srv.checkpoint(c, srv.checkpointPostHandshake)
|
||||
if err != nil {
|
||||
clog.Trace("Rejected peer before protocol handshake", "err", err)
|
||||
clog.Trace("Rejected peer", "err", err)
|
||||
return err
|
||||
}
|
||||
// Run the protocol handshake
|
||||
|
||||
// Run the capability negotiation handshake.
|
||||
phs, err := c.doProtoHandshake(srv.ourHandshake)
|
||||
if err != nil {
|
||||
clog.Trace("Failed proto handshake", "err", err)
|
||||
@ -938,14 +966,15 @@ func (srv *Server) setupConn(c *conn, flags connFlag, dialDest *enode.Node) erro
|
||||
return DiscUnexpectedIdentity
|
||||
}
|
||||
c.caps, c.name = phs.Caps, phs.Name
|
||||
err = srv.checkpoint(c, srv.addpeer)
|
||||
err = srv.checkpoint(c, srv.checkpointAddPeer)
|
||||
if err != nil {
|
||||
clog.Trace("Rejected peer", "err", err)
|
||||
return err
|
||||
}
|
||||
// If the checks completed successfully, runPeer has now been
|
||||
// launched by run.
|
||||
clog.Trace("connection set up", "inbound", dialDest == nil)
|
||||
|
||||
// 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
|
||||
}
|
||||
|
||||
@ -974,12 +1003,7 @@ func (srv *Server) checkpoint(c *conn, stage chan<- *conn) error {
|
||||
case <-srv.quit:
|
||||
return errServerStopped
|
||||
}
|
||||
select {
|
||||
case err := <-c.cont:
|
||||
return err
|
||||
case <-srv.quit:
|
||||
return errServerStopped
|
||||
}
|
||||
return <-c.cont
|
||||
}
|
||||
|
||||
// runPeer runs in its own goroutine for each peer.
|
||||
|
Reference in New Issue
Block a user