p2p, p2p/discover: add signed ENR generation (#17753)

This PR adds enode.LocalNode and integrates it into the p2p
subsystem. This new object is the keeper of the local node
record. For now, a new version of the record is produced every
time the client restarts. We'll make it smarter to avoid that in
the future.

There are a couple of other changes in this commit: discovery now
waits for all of its goroutines at shutdown and the p2p server
now closes the node database after discovery has shut down. This
fixes a leveldb crash in tests. p2p server startup is faster
because it doesn't need to wait for the external IP query
anymore.
This commit is contained in:
Felix Lange
2018-10-12 11:47:24 +02:00
committed by GitHub
parent dcae0d348b
commit 6f607de5d5
24 changed files with 979 additions and 278 deletions

View File

@ -72,21 +72,20 @@ type Table struct {
ips netutil.DistinctNetSet
db *enode.DB // database of known nodes
net transport
refreshReq chan chan struct{}
initDone chan struct{}
closeReq chan struct{}
closed chan struct{}
nodeAddedHook func(*node) // for testing
net transport
self *node // metadata of the local node
}
// transport is implemented by the UDP transport.
// it is an interface so we can test without opening lots of UDP
// sockets and without generating a private key.
type transport interface {
self() *enode.Node
ping(enode.ID, *net.UDPAddr) error
findnode(toid enode.ID, addr *net.UDPAddr, target encPubkey) ([]*node, error)
close()
@ -100,11 +99,10 @@ type bucket struct {
ips netutil.DistinctNetSet
}
func newTable(t transport, self *enode.Node, db *enode.DB, bootnodes []*enode.Node) (*Table, error) {
func newTable(t transport, db *enode.DB, bootnodes []*enode.Node) (*Table, error) {
tab := &Table{
net: t,
db: db,
self: wrapNode(self),
refreshReq: make(chan chan struct{}),
initDone: make(chan struct{}),
closeReq: make(chan struct{}),
@ -127,6 +125,10 @@ func newTable(t transport, self *enode.Node, db *enode.DB, bootnodes []*enode.No
return tab, nil
}
func (tab *Table) self() *enode.Node {
return tab.net.self()
}
func (tab *Table) seedRand() {
var b [8]byte
crand.Read(b[:])
@ -136,11 +138,6 @@ func (tab *Table) seedRand() {
tab.mutex.Unlock()
}
// Self returns the local node.
func (tab *Table) Self() *enode.Node {
return unwrapNode(tab.self)
}
// ReadRandomNodes fills the given slice with random nodes from the table. The results
// are guaranteed to be unique for a single invocation, no node will appear twice.
func (tab *Table) ReadRandomNodes(buf []*enode.Node) (n int) {
@ -183,6 +180,10 @@ func (tab *Table) ReadRandomNodes(buf []*enode.Node) (n int) {
// Close terminates the network listener and flushes the node database.
func (tab *Table) Close() {
if tab.net != nil {
tab.net.close()
}
select {
case <-tab.closed:
// already closed.
@ -257,7 +258,7 @@ func (tab *Table) lookup(targetKey encPubkey, refreshIfEmpty bool) []*node {
)
// don't query further if we hit ourself.
// unlikely to happen often in practice.
asked[tab.self.ID()] = true
asked[tab.self().ID()] = true
for {
tab.mutex.Lock()
@ -340,8 +341,8 @@ func (tab *Table) loop() {
revalidate = time.NewTimer(tab.nextRevalidateTime())
refresh = time.NewTicker(refreshInterval)
copyNodes = time.NewTicker(copyNodesInterval)
revalidateDone = make(chan struct{})
refreshDone = make(chan struct{}) // where doRefresh reports completion
revalidateDone chan struct{} // where doRevalidate reports completion
waiting = []chan struct{}{tab.initDone} // holds waiting callers while doRefresh runs
)
defer refresh.Stop()
@ -372,9 +373,11 @@ loop:
}
waiting, refreshDone = nil, nil
case <-revalidate.C:
revalidateDone = make(chan struct{})
go tab.doRevalidate(revalidateDone)
case <-revalidateDone:
revalidate.Reset(tab.nextRevalidateTime())
revalidateDone = nil
case <-copyNodes.C:
go tab.copyLiveNodes()
case <-tab.closeReq:
@ -382,15 +385,15 @@ loop:
}
}
if tab.net != nil {
tab.net.close()
}
if refreshDone != nil {
<-refreshDone
}
for _, ch := range waiting {
close(ch)
}
if revalidateDone != nil {
<-revalidateDone
}
close(tab.closed)
}
@ -408,7 +411,7 @@ func (tab *Table) doRefresh(done chan struct{}) {
// Run self lookup to discover new neighbor nodes.
// We can only do this if we have a secp256k1 identity.
var key ecdsa.PublicKey
if err := tab.self.Load((*enode.Secp256k1)(&key)); err == nil {
if err := tab.self().Load((*enode.Secp256k1)(&key)); err == nil {
tab.lookup(encodePubkey(&key), false)
}
@ -530,7 +533,7 @@ func (tab *Table) len() (n int) {
// bucket returns the bucket for the given node ID hash.
func (tab *Table) bucket(id enode.ID) *bucket {
d := enode.LogDist(tab.self.ID(), id)
d := enode.LogDist(tab.self().ID(), id)
if d <= bucketMinDistance {
return tab.buckets[0]
}
@ -543,7 +546,7 @@ func (tab *Table) bucket(id enode.ID) *bucket {
//
// The caller must not hold tab.mutex.
func (tab *Table) add(n *node) {
if n.ID() == tab.self.ID() {
if n.ID() == tab.self().ID() {
return
}
@ -576,7 +579,7 @@ func (tab *Table) stuff(nodes []*node) {
defer tab.mutex.Unlock()
for _, n := range nodes {
if n.ID() == tab.self.ID() {
if n.ID() == tab.self().ID() {
continue // don't add self
}
b := tab.bucket(n.ID())

View File

@ -141,7 +141,7 @@ func TestTable_IPLimit(t *testing.T) {
defer db.Close()
for i := 0; i < tableIPLimit+1; i++ {
n := nodeAtDistance(tab.self.ID(), i, net.IP{172, 0, 1, byte(i)})
n := nodeAtDistance(tab.self().ID(), i, net.IP{172, 0, 1, byte(i)})
tab.add(n)
}
if tab.len() > tableIPLimit {
@ -158,7 +158,7 @@ func TestTable_BucketIPLimit(t *testing.T) {
d := 3
for i := 0; i < bucketIPLimit+1; i++ {
n := nodeAtDistance(tab.self.ID(), d, net.IP{172, 0, 1, byte(i)})
n := nodeAtDistance(tab.self().ID(), d, net.IP{172, 0, 1, byte(i)})
tab.add(n)
}
if tab.len() > bucketIPLimit {
@ -240,7 +240,7 @@ func TestTable_ReadRandomNodesGetAll(t *testing.T) {
for i := 0; i < len(buf); i++ {
ld := cfg.Rand.Intn(len(tab.buckets))
tab.stuff([]*node{nodeAtDistance(tab.self.ID(), ld, intIP(ld))})
tab.stuff([]*node{nodeAtDistance(tab.self().ID(), ld, intIP(ld))})
}
gotN := tab.ReadRandomNodes(buf)
if gotN != tab.len() {
@ -510,6 +510,10 @@ type preminedTestnet struct {
dists [hashBits + 1][]encPubkey
}
func (tn *preminedTestnet) self() *enode.Node {
return nullNode
}
func (tn *preminedTestnet) findnode(toid enode.ID, toaddr *net.UDPAddr, target encPubkey) ([]*node, error) {
// current log distance is encoded in port number
// fmt.Println("findnode query at dist", toaddr.Port)

View File

@ -28,12 +28,17 @@ import (
"github.com/ethereum/go-ethereum/p2p/enr"
)
func newTestTable(t transport) (*Table, *enode.DB) {
var nullNode *enode.Node
func init() {
var r enr.Record
r.Set(enr.IP{0, 0, 0, 0})
n := enode.SignNull(&r, enode.ID{})
nullNode = enode.SignNull(&r, enode.ID{})
}
func newTestTable(t transport) (*Table, *enode.DB) {
db, _ := enode.OpenDB("")
tab, _ := newTable(t, n, db, nil)
tab, _ := newTable(t, db, nil)
return tab, db
}
@ -70,10 +75,10 @@ func intIP(i int) net.IP {
// fillBucket inserts nodes into the given bucket until it is full.
func fillBucket(tab *Table, n *node) (last *node) {
ld := enode.LogDist(tab.self.ID(), n.ID())
ld := enode.LogDist(tab.self().ID(), n.ID())
b := tab.bucket(n.ID())
for len(b.entries) < bucketSize {
b.entries = append(b.entries, nodeAtDistance(tab.self.ID(), ld, intIP(ld)))
b.entries = append(b.entries, nodeAtDistance(tab.self().ID(), ld, intIP(ld)))
}
return b.entries[bucketSize-1]
}
@ -81,15 +86,25 @@ func fillBucket(tab *Table, n *node) (last *node) {
type pingRecorder struct {
mu sync.Mutex
dead, pinged map[enode.ID]bool
n *enode.Node
}
func newPingRecorder() *pingRecorder {
var r enr.Record
r.Set(enr.IP{0, 0, 0, 0})
n := enode.SignNull(&r, enode.ID{})
return &pingRecorder{
dead: make(map[enode.ID]bool),
pinged: make(map[enode.ID]bool),
n: n,
}
}
func (t *pingRecorder) self() *enode.Node {
return nullNode
}
func (t *pingRecorder) findnode(toid enode.ID, toaddr *net.UDPAddr, target encPubkey) ([]*node, error) {
return nil, nil
}

View File

@ -23,12 +23,12 @@ import (
"errors"
"fmt"
"net"
"sync"
"time"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/p2p/enode"
"github.com/ethereum/go-ethereum/p2p/nat"
"github.com/ethereum/go-ethereum/p2p/netutil"
"github.com/ethereum/go-ethereum/rlp"
)
@ -118,9 +118,11 @@ type (
)
func makeEndpoint(addr *net.UDPAddr, tcpPort uint16) rpcEndpoint {
ip := addr.IP.To4()
if ip == nil {
ip = addr.IP.To16()
ip := net.IP{}
if ip4 := addr.IP.To4(); ip4 != nil {
ip = ip4
} else if ip6 := addr.IP.To16(); ip6 != nil {
ip = ip6
}
return rpcEndpoint{IP: ip, UDP: uint16(addr.Port), TCP: tcpPort}
}
@ -165,20 +167,19 @@ type conn interface {
LocalAddr() net.Addr
}
// udp implements the RPC protocol.
// udp implements the discovery v4 UDP wire protocol.
type udp struct {
conn conn
netrestrict *netutil.Netlist
priv *ecdsa.PrivateKey
ourEndpoint rpcEndpoint
localNode *enode.LocalNode
db *enode.DB
tab *Table
wg sync.WaitGroup
addpending chan *pending
gotreply chan reply
closing chan struct{}
nat nat.Interface
*Table
closing chan struct{}
}
// pending represents a pending reply.
@ -230,60 +231,57 @@ type Config struct {
PrivateKey *ecdsa.PrivateKey
// These settings are optional:
AnnounceAddr *net.UDPAddr // local address announced in the DHT
NodeDBPath string // if set, the node database is stored at this filesystem location
NetRestrict *netutil.Netlist // network whitelist
Bootnodes []*enode.Node // list of bootstrap nodes
Unhandled chan<- ReadPacket // unhandled packets are sent on this channel
NetRestrict *netutil.Netlist // network whitelist
Bootnodes []*enode.Node // list of bootstrap nodes
Unhandled chan<- ReadPacket // unhandled packets are sent on this channel
}
// ListenUDP returns a new table that listens for UDP packets on laddr.
func ListenUDP(c conn, cfg Config) (*Table, error) {
tab, _, err := newUDP(c, cfg)
func ListenUDP(c conn, ln *enode.LocalNode, cfg Config) (*Table, error) {
tab, _, err := newUDP(c, ln, cfg)
if err != nil {
return nil, err
}
log.Info("UDP listener up", "self", tab.self)
return tab, nil
}
func newUDP(c conn, cfg Config) (*Table, *udp, error) {
realaddr := c.LocalAddr().(*net.UDPAddr)
if cfg.AnnounceAddr != nil {
realaddr = cfg.AnnounceAddr
}
self := enode.NewV4(&cfg.PrivateKey.PublicKey, realaddr.IP, realaddr.Port, realaddr.Port)
db, err := enode.OpenDB(cfg.NodeDBPath)
if err != nil {
return nil, nil, err
}
func newUDP(c conn, ln *enode.LocalNode, cfg Config) (*Table, *udp, error) {
udp := &udp{
conn: c,
priv: cfg.PrivateKey,
netrestrict: cfg.NetRestrict,
localNode: ln,
db: ln.Database(),
closing: make(chan struct{}),
gotreply: make(chan reply),
addpending: make(chan *pending),
}
// TODO: separate TCP port
udp.ourEndpoint = makeEndpoint(realaddr, uint16(realaddr.Port))
tab, err := newTable(udp, self, db, cfg.Bootnodes)
tab, err := newTable(udp, ln.Database(), cfg.Bootnodes)
if err != nil {
return nil, nil, err
}
udp.Table = tab
udp.tab = tab
udp.wg.Add(2)
go udp.loop()
go udp.readLoop(cfg.Unhandled)
return udp.Table, udp, nil
return udp.tab, udp, nil
}
func (t *udp) self() *enode.Node {
return t.localNode.Node()
}
func (t *udp) close() {
close(t.closing)
t.conn.Close()
t.db.Close()
// TODO: wait for the loops to end.
t.wg.Wait()
}
func (t *udp) ourEndpoint() rpcEndpoint {
n := t.self()
a := &net.UDPAddr{IP: n.IP(), Port: n.UDP()}
return makeEndpoint(a, uint16(n.TCP()))
}
// ping sends a ping message to the given node and waits for a reply.
@ -296,7 +294,7 @@ func (t *udp) ping(toid enode.ID, toaddr *net.UDPAddr) error {
func (t *udp) sendPing(toid enode.ID, toaddr *net.UDPAddr, callback func()) <-chan error {
req := &ping{
Version: 4,
From: t.ourEndpoint,
From: t.ourEndpoint(),
To: makeEndpoint(toaddr, 0), // TODO: maybe use known TCP port from DB
Expiration: uint64(time.Now().Add(expiration).Unix()),
}
@ -313,6 +311,7 @@ func (t *udp) sendPing(toid enode.ID, toaddr *net.UDPAddr, callback func()) <-ch
}
return ok
})
t.localNode.UDPContact(toaddr)
t.write(toaddr, req.name(), packet)
return errc
}
@ -381,6 +380,8 @@ func (t *udp) handleReply(from enode.ID, ptype byte, req packet) bool {
// loop runs in its own goroutine. it keeps track of
// the refresh timer and the pending reply queue.
func (t *udp) loop() {
defer t.wg.Done()
var (
plist = list.New()
timeout = time.NewTimer(0)
@ -542,10 +543,11 @@ func encodePacket(priv *ecdsa.PrivateKey, ptype byte, req interface{}) (packet,
// readLoop runs in its own goroutine. it handles incoming UDP packets.
func (t *udp) readLoop(unhandled chan<- ReadPacket) {
defer t.conn.Close()
defer t.wg.Done()
if unhandled != nil {
defer close(unhandled)
}
// Discovery packets are defined to be no larger than 1280 bytes.
// Packets larger than this size will be cut at the end and treated
// as invalid because their hash won't match.
@ -629,10 +631,11 @@ func (req *ping) handle(t *udp, from *net.UDPAddr, fromKey encPubkey, mac []byte
n := wrapNode(enode.NewV4(key, from.IP, int(req.From.TCP), from.Port))
t.handleReply(n.ID(), pingPacket, req)
if time.Since(t.db.LastPongReceived(n.ID())) > bondExpiration {
t.sendPing(n.ID(), from, func() { t.addThroughPing(n) })
t.sendPing(n.ID(), from, func() { t.tab.addThroughPing(n) })
} else {
t.addThroughPing(n)
t.tab.addThroughPing(n)
}
t.localNode.UDPEndpointStatement(from, &net.UDPAddr{IP: req.To.IP, Port: int(req.To.UDP)})
t.db.UpdateLastPingReceived(n.ID(), time.Now())
return nil
}
@ -647,6 +650,7 @@ func (req *pong) handle(t *udp, from *net.UDPAddr, fromKey encPubkey, mac []byte
if !t.handleReply(fromID, pongPacket, req) {
return errUnsolicitedReply
}
t.localNode.UDPEndpointStatement(from, &net.UDPAddr{IP: req.To.IP, Port: int(req.To.UDP)})
t.db.UpdateLastPongReceived(fromID, time.Now())
return nil
}
@ -668,9 +672,9 @@ func (req *findnode) handle(t *udp, from *net.UDPAddr, fromKey encPubkey, mac []
return errUnknownNode
}
target := enode.ID(crypto.Keccak256Hash(req.Target[:]))
t.mutex.Lock()
closest := t.closest(target, bucketSize).entries
t.mutex.Unlock()
t.tab.mutex.Lock()
closest := t.tab.closest(target, bucketSize).entries
t.tab.mutex.Unlock()
p := neighbors{Expiration: uint64(time.Now().Add(expiration).Unix())}
var sent bool

View File

@ -71,7 +71,9 @@ func newUDPTest(t *testing.T) *udpTest {
remotekey: newkey(),
remoteaddr: &net.UDPAddr{IP: net.IP{10, 0, 1, 99}, Port: 30303},
}
test.table, test.udp, _ = newUDP(test.pipe, Config{PrivateKey: test.localkey})
db, _ := enode.OpenDB("")
ln := enode.NewLocalNode(db, test.localkey)
test.table, test.udp, _ = newUDP(test.pipe, ln, Config{PrivateKey: test.localkey})
// Wait for initial refresh so the table doesn't send unexpected findnode.
<-test.table.initDone
return test
@ -355,12 +357,13 @@ func TestUDP_successfulPing(t *testing.T) {
// remote is unknown, the table pings back.
hash, _ := test.waitPacketOut(func(p *ping) error {
if !reflect.DeepEqual(p.From, test.udp.ourEndpoint) {
t.Errorf("got ping.From %v, want %v", p.From, test.udp.ourEndpoint)
if !reflect.DeepEqual(p.From, test.udp.ourEndpoint()) {
t.Errorf("got ping.From %#v, want %#v", p.From, test.udp.ourEndpoint())
}
wantTo := rpcEndpoint{
// The mirrored UDP address is the UDP packet sender.
IP: test.remoteaddr.IP, UDP: uint16(test.remoteaddr.Port),
IP: test.remoteaddr.IP,
UDP: uint16(test.remoteaddr.Port),
TCP: 0,
}
if !reflect.DeepEqual(p.To, wantTo) {