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:
@ -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())
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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) {
|
||||
|
Reference in New Issue
Block a user