From f55cd9c00459888157bf774ce13ec72ffab945c3 Mon Sep 17 00:00:00 2001 From: Zsolt Felfoldi Date: Tue, 1 Dec 2020 11:20:08 +0100 Subject: [PATCH 01/10] les: switch to new discv5 --- cmd/utils/flags.go | 5 ++--- les/client.go | 19 ++++++++++--------- les/commons.go | 12 ------------ les/enr_entry.go | 42 ++++++++++++++++++++++++++++++++++++++---- les/retrieve.go | 9 ++++----- les/server.go | 21 --------------------- les/test_helper.go | 3 ++- p2p/server.go | 26 +++++++++++++------------- 8 files changed, 69 insertions(+), 68 deletions(-) diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index 6f4da583200d..3fe84e26764b 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -58,7 +58,6 @@ import ( "github.com/ethereum/go-ethereum/miner" "github.com/ethereum/go-ethereum/node" "github.com/ethereum/go-ethereum/p2p" - "github.com/ethereum/go-ethereum/p2p/discv5" "github.com/ethereum/go-ethereum/p2p/enode" "github.com/ethereum/go-ethereum/p2p/nat" "github.com/ethereum/go-ethereum/p2p/netutil" @@ -857,10 +856,10 @@ func setBootstrapNodesV5(ctx *cli.Context, cfg *p2p.Config) { return // already set, don't apply defaults. } - cfg.BootstrapNodesV5 = make([]*discv5.Node, 0, len(urls)) + cfg.BootstrapNodesV5 = make([]*enode.Node, 0, len(urls)) for _, url := range urls { if url != "" { - node, err := discv5.ParseNode(url) + node, err := enode.Parse(enode.ValidSchemes, url) if err != nil { log.Error("Bootstrap URL invalid", "enode", url, "err", err) continue diff --git a/les/client.go b/les/client.go index 47997a098b30..c073d97312b1 100644 --- a/les/client.go +++ b/les/client.go @@ -112,15 +112,7 @@ func New(stack *node.Node, config *eth.Config) (*LightEthereum, error) { } peers.subscribe((*vtSubscription)(leth.valueTracker)) - dnsdisc, err := leth.setupDiscovery() - if err != nil { - return nil, err - } - leth.serverPool = newServerPool(lespayDb, []byte("serverpool:"), leth.valueTracker, dnsdisc, time.Second, nil, &mclock.System{}, config.UltraLightServers) - peers.subscribe(leth.serverPool) - leth.dialCandidates = leth.serverPool.dialIterator - - leth.retriever = newRetrieveManager(peers, leth.reqDist, leth.serverPool.getTimeout) + leth.retriever = newRetrieveManager(peers, leth.reqDist) leth.relay = newLesTxRelay(peers, leth.retriever) leth.odr = NewLesOdr(chainDb, light.DefaultClientIndexerConfig, leth.retriever) @@ -140,6 +132,15 @@ func New(stack *node.Node, config *eth.Config) (*LightEthereum, error) { leth.chainReader = leth.blockchain leth.txPool = light.NewTxPool(leth.chainConfig, leth.blockchain, leth.relay) + discovery, err := leth.setupDiscovery(&stack.Config().P2P) + if err != nil { + return nil, err + } + leth.serverPool = newServerPool(lespayDb, []byte("serverpool:"), leth.valueTracker, discovery, time.Second, nil, &mclock.System{}, config.UltraLightServers) + peers.subscribe(leth.serverPool) + leth.dialCandidates = leth.serverPool.dialIterator + leth.retriever.softRequestTimeout = leth.serverPool.getTimeout + // Set up checkpoint oracle. leth.oracle = leth.setupOracle(stack, genesisHash, config) diff --git a/les/commons.go b/les/commons.go index 003e196d2b82..a2d845a51c1e 100644 --- a/les/commons.go +++ b/les/commons.go @@ -33,7 +33,6 @@ import ( "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/node" "github.com/ethereum/go-ethereum/p2p" - "github.com/ethereum/go-ethereum/p2p/discv5" "github.com/ethereum/go-ethereum/p2p/enode" "github.com/ethereum/go-ethereum/params" ) @@ -42,17 +41,6 @@ func errResp(code errCode, format string, v ...interface{}) error { return fmt.Errorf("%v - %v", code, fmt.Sprintf(format, v...)) } -func lesTopic(genesisHash common.Hash, protocolVersion uint) discv5.Topic { - var name string - switch protocolVersion { - case lpv2: - name = "LES2" - default: - panic(nil) - } - return discv5.Topic(name + "@" + common.Bytes2Hex(genesisHash.Bytes()[0:8])) -} - type chainReader interface { CurrentHeader() *types.Header } diff --git a/les/enr_entry.go b/les/enr_entry.go index a357f689df6e..eec415712030 100644 --- a/les/enr_entry.go +++ b/les/enr_entry.go @@ -17,8 +17,11 @@ package les import ( + "github.com/ethereum/go-ethereum/core/forkid" + "github.com/ethereum/go-ethereum/p2p" "github.com/ethereum/go-ethereum/p2p/dnsdisc" "github.com/ethereum/go-ethereum/p2p/enode" + "github.com/ethereum/go-ethereum/p2p/enr" "github.com/ethereum/go-ethereum/rlp" ) @@ -34,10 +37,41 @@ func (e lesEntry) ENRKey() string { } // setupDiscovery creates the node discovery source for the eth protocol. -func (eth *LightEthereum) setupDiscovery() (enode.Iterator, error) { - if len(eth.config.EthDiscoveryURLs) == 0 { +func (eth *LightEthereum) setupDiscovery(cfg *p2p.Config) (enode.Iterator, error) { + var it enode.Iterator + if len(eth.config.EthDiscoveryURLs) != 0 { + client := dnsdisc.NewClient(dnsdisc.Config{}) + var err error + it, err = client.NewIterator(eth.config.EthDiscoveryURLs...) + if err != nil { + return nil, err + } + } + if cfg.DiscoveryV5 && eth.p2pServer.DiscV5 != nil { + v5 := eth.p2pServer.DiscV5.RandomNodes() + if it == nil { + it = v5 + } else { + mix := enode.NewFairMix(0) + mix.AddSource(it) + mix.AddSource(v5) + it = mix + } + } + if it == nil { return nil, nil } - client := dnsdisc.NewClient(dnsdisc.Config{}) - return client.NewIterator(eth.config.EthDiscoveryURLs...) + forkFilter := forkid.NewFilter(eth.blockchain) + return enode.Filter(it, func(n *enode.Node) bool { + var ( + les struct { + _ []rlp.RawValue `rlp:"tail"` + } + eth struct { + ForkID forkid.ID + _ []rlp.RawValue `rlp:"tail"` + } + ) + return n.Load(enr.WithEntry("les", &les)) == nil && n.Load(enr.WithEntry("eth", ð)) == nil && forkFilter(eth.ForkID) == nil + }), nil } diff --git a/les/retrieve.go b/les/retrieve.go index ca4f867ea80f..e6d08312ed7c 100644 --- a/les/retrieve.go +++ b/les/retrieve.go @@ -92,12 +92,11 @@ const ( ) // newRetrieveManager creates the retrieve manager -func newRetrieveManager(peers *serverPeerSet, dist *requestDistributor, srto func() time.Duration) *retrieveManager { +func newRetrieveManager(peers *serverPeerSet, dist *requestDistributor) *retrieveManager { return &retrieveManager{ - peers: peers, - dist: dist, - sentReqs: make(map[uint64]*sentReq), - softRequestTimeout: srto, + peers: peers, + dist: dist, + sentReqs: make(map[uint64]*sentReq), } } diff --git a/les/server.go b/les/server.go index cbedce136c35..6b12b6f8f373 100644 --- a/les/server.go +++ b/les/server.go @@ -29,7 +29,6 @@ import ( "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/node" "github.com/ethereum/go-ethereum/p2p" - "github.com/ethereum/go-ethereum/p2p/discv5" "github.com/ethereum/go-ethereum/p2p/enode" "github.com/ethereum/go-ethereum/p2p/enr" "github.com/ethereum/go-ethereum/p2p/nodestate" @@ -58,7 +57,6 @@ type LesServer struct { archiveMode bool // Flag whether the ethereum node runs in archive mode. handler *serverHandler broadcaster *broadcaster - lesTopics []discv5.Topic privateKey *ecdsa.PrivateKey // Flow control and capacity management @@ -77,11 +75,6 @@ type LesServer struct { func NewLesServer(node *node.Node, e *eth.Ethereum, config *eth.Config) (*LesServer, error) { ns := nodestate.NewNodeStateMachine(nil, nil, mclock.System{}, serverSetup) - // Collect les protocol version information supported by local node. - lesTopics := make([]discv5.Topic, len(AdvertiseProtocolVersions)) - for i, pv := range AdvertiseProtocolVersions { - lesTopics[i] = lesTopic(e.BlockChain().Genesis().Hash(), pv) - } // Calculate the number of threads used to service the light client // requests based on the user-specified value. threads := config.LightServ * 4 / 100 @@ -103,7 +96,6 @@ func NewLesServer(node *node.Node, e *eth.Ethereum, config *eth.Config) (*LesSer ns: ns, archiveMode: e.ArchiveMode(), broadcaster: newBroadcaster(ns), - lesTopics: lesTopics, fcManager: flowcontrol.NewClientManager(nil, &mclock.System{}), servingQueue: newServingQueue(int64(time.Millisecond*10), float64(config.LightServ)/100), threadsBusy: config.LightServ/100 + 1, @@ -203,19 +195,6 @@ func (s *LesServer) Start() error { s.wg.Add(1) go s.capacityManagement() - if s.p2pSrv.DiscV5 != nil { - for _, topic := range s.lesTopics { - topic := topic - go func() { - logger := log.New("topic", topic) - logger.Info("Starting topic registration") - defer logger.Info("Terminated topic registration") - - s.p2pSrv.DiscV5.RegisterTopic(topic, s.closeCh) - }() - } - } - return nil } diff --git a/les/test_helper.go b/les/test_helper.go index d108a8dacefb..80fc34ce6def 100644 --- a/les/test_helper.go +++ b/les/test_helper.go @@ -514,7 +514,8 @@ func newClientServerEnv(t *testing.T, blocks int, protocol int, callback indexer clock = &mclock.Simulated{} } dist := newRequestDistributor(speers, clock) - rm := newRetrieveManager(speers, dist, func() time.Duration { return time.Millisecond * 500 }) + rm := newRetrieveManager(speers, dist) + rm.softRequestTimeout = func() time.Duration { return time.Millisecond * 500 } odr := NewLesOdr(cdb, light.TestClientIndexerConfig, rm) sindexers := testIndexers(sdb, nil, light.TestServerIndexerConfig, true) diff --git a/p2p/server.go b/p2p/server.go index 275cb5ea5c34..fc7154855490 100644 --- a/p2p/server.go +++ b/p2p/server.go @@ -35,7 +35,6 @@ import ( "github.com/ethereum/go-ethereum/event" "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/p2p/discover" - "github.com/ethereum/go-ethereum/p2p/discv5" "github.com/ethereum/go-ethereum/p2p/enode" "github.com/ethereum/go-ethereum/p2p/enr" "github.com/ethereum/go-ethereum/p2p/nat" @@ -105,7 +104,7 @@ type Config struct { // BootstrapNodesV5 are used to establish connectivity // with the rest of the network using the V5 discovery // protocol. - BootstrapNodesV5 []*discv5.Node `toml:",omitempty"` + BootstrapNodesV5 []*enode.Node `toml:",omitempty"` // Static nodes are used as pre-configured connections which are always // maintained and re-connected on disconnects. @@ -182,7 +181,7 @@ type Server struct { nodedb *enode.DB localnode *enode.LocalNode ntab *discover.UDPv4 - DiscV5 *discv5.Network + DiscV5 *discover.UDPv5 discmix *enode.FairMix dialsched *dialScheduler @@ -413,7 +412,7 @@ type sharedUDPConn struct { unhandled chan discover.ReadPacket } -// ReadFromUDP implements discv5.conn +// ReadFromUDP implements discover.UDPConn func (s *sharedUDPConn) ReadFromUDP(b []byte) (n int, addr *net.UDPAddr, err error) { packet, ok := <-s.unhandled if !ok { @@ -427,7 +426,7 @@ func (s *sharedUDPConn) ReadFromUDP(b []byte) (n int, addr *net.UDPAddr, err err return l, packet.Addr, nil } -// Close implements discv5.conn +// Close implements discover.UDPConn func (s *sharedUDPConn) Close() error { return nil } @@ -586,7 +585,7 @@ func (srv *Server) setupDiscovery() error { Unhandled: unhandled, Log: srv.log, } - ntab, err := discover.ListenUDP(conn, srv.localnode, cfg) + ntab, err := discover.ListenV4(conn, srv.localnode, cfg) if err != nil { return err } @@ -596,20 +595,21 @@ func (srv *Server) setupDiscovery() error { // Discovery V5 if srv.DiscoveryV5 { - var ntab *discv5.Network + cfg := discover.Config{ + PrivateKey: srv.PrivateKey, + NetRestrict: srv.NetRestrict, + Bootnodes: srv.BootstrapNodesV5, + Log: srv.log, + } var err error if sconn != nil { - ntab, err = discv5.ListenUDP(srv.PrivateKey, sconn, "", srv.NetRestrict) + srv.DiscV5, err = discover.ListenV5(sconn, srv.localnode, cfg) } else { - ntab, err = discv5.ListenUDP(srv.PrivateKey, conn, "", srv.NetRestrict) + srv.DiscV5, err = discover.ListenV5(conn, srv.localnode, cfg) } if err != nil { return err } - if err := ntab.SetFallbackNodes(srv.BootstrapNodesV5); err != nil { - return err - } - srv.DiscV5 = ntab } return nil } From 3fc456093af400e9974a5e0349f42669673b9a7c Mon Sep 17 00:00:00 2001 From: Zsolt Felfoldi Date: Wed, 2 Dec 2020 09:41:00 +0100 Subject: [PATCH 02/10] cmd/faucet: fixed faucet --- cmd/faucet/faucet.go | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/cmd/faucet/faucet.go b/cmd/faucet/faucet.go index b9c4e1819a06..763b8d25f876 100644 --- a/cmd/faucet/faucet.go +++ b/cmd/faucet/faucet.go @@ -55,7 +55,6 @@ import ( "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/node" "github.com/ethereum/go-ethereum/p2p" - "github.com/ethereum/go-ethereum/p2p/discv5" "github.com/ethereum/go-ethereum/p2p/enode" "github.com/ethereum/go-ethereum/p2p/nat" "github.com/ethereum/go-ethereum/params" @@ -154,9 +153,9 @@ func main() { log.Crit("Failed to parse genesis block json", "err", err) } // Convert the bootnodes to internal enode representations - var enodes []*discv5.Node + var enodes []*enode.Node for _, boot := range strings.Split(*bootFlag, ",") { - if url, err := discv5.ParseNode(boot); err == nil { + if url, err := enode.Parse(enode.ValidSchemes, boot); err == nil { enodes = append(enodes, url) } else { log.Error("Failed to parse bootnode URL", "url", boot, "err", err) @@ -228,7 +227,7 @@ type wsConn struct { wlock sync.Mutex } -func newFaucet(genesis *core.Genesis, port int, enodes []*discv5.Node, network uint64, stats string, ks *keystore.KeyStore, index []byte) (*faucet, error) { +func newFaucet(genesis *core.Genesis, port int, enodes []*enode.Node, network uint64, stats string, ks *keystore.KeyStore, index []byte) (*faucet, error) { // Assemble the raw devp2p protocol stack stack, err := node.New(&node.Config{ Name: "geth", From 367a2acd7c8dc491836dd23b9206e0c929ca147d Mon Sep 17 00:00:00 2001 From: Zsolt Felfoldi Date: Wed, 2 Dec 2020 09:52:29 +0100 Subject: [PATCH 03/10] mobile, cmd/bootnode: removed all references to old discv5 --- cmd/bootnode/main.go | 15 +++++++-------- mobile/discover.go | 12 ++++++------ mobile/params.go | 10 +++++++--- 3 files changed, 20 insertions(+), 17 deletions(-) diff --git a/cmd/bootnode/main.go b/cmd/bootnode/main.go index 6c9ff615a18b..036b968ef83d 100644 --- a/cmd/bootnode/main.go +++ b/cmd/bootnode/main.go @@ -28,7 +28,6 @@ import ( "github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/p2p/discover" - "github.com/ethereum/go-ethereum/p2p/discv5" "github.com/ethereum/go-ethereum/p2p/enode" "github.com/ethereum/go-ethereum/p2p/nat" "github.com/ethereum/go-ethereum/p2p/netutil" @@ -121,17 +120,17 @@ func main() { printNotice(&nodeKey.PublicKey, *realaddr) + db, _ := enode.OpenDB("") + ln := enode.NewLocalNode(db, nodeKey) + cfg := discover.Config{ + PrivateKey: nodeKey, + NetRestrict: restrictList, + } if *runv5 { - if _, err := discv5.ListenUDP(nodeKey, conn, "", restrictList); err != nil { + if _, err := discover.ListenV5(conn, ln, cfg); err != nil { utils.Fatalf("%v", err) } } else { - db, _ := enode.OpenDB("") - ln := enode.NewLocalNode(db, nodeKey) - cfg := discover.Config{ - PrivateKey: nodeKey, - NetRestrict: restrictList, - } if _, err := discover.ListenUDP(conn, ln, cfg); err != nil { utils.Fatalf("%v", err) } diff --git a/mobile/discover.go b/mobile/discover.go index 9b3c93ccd98f..2c699f08be04 100644 --- a/mobile/discover.go +++ b/mobile/discover.go @@ -22,12 +22,12 @@ package geth import ( "errors" - "github.com/ethereum/go-ethereum/p2p/discv5" + "github.com/ethereum/go-ethereum/p2p/enode" ) // Enode represents a host on the network. type Enode struct { - node *discv5.Node + node *enode.Node } // NewEnode parses a node designator. @@ -53,8 +53,8 @@ type Enode struct { // and UDP discovery port 30301. // // enode://@10.3.58.6:30303?discport=30301 -func NewEnode(rawurl string) (enode *Enode, _ error) { - node, err := discv5.ParseNode(rawurl) +func NewEnode(rawurl string) (*Enode, error) { + node, err := enode.Parse(enode.ValidSchemes, rawurl) if err != nil { return nil, err } @@ -62,12 +62,12 @@ func NewEnode(rawurl string) (enode *Enode, _ error) { } // Enodes represents a slice of accounts. -type Enodes struct{ nodes []*discv5.Node } +type Enodes struct{ nodes []*enode.Node } // NewEnodes creates a slice of uninitialized enodes. func NewEnodes(size int) *Enodes { return &Enodes{ - nodes: make([]*discv5.Node, size), + nodes: make([]*enode.Node, size), } } diff --git a/mobile/params.go b/mobile/params.go index 43ac00474081..0fc197c9e508 100644 --- a/mobile/params.go +++ b/mobile/params.go @@ -22,7 +22,7 @@ import ( "encoding/json" "github.com/ethereum/go-ethereum/core" - "github.com/ethereum/go-ethereum/p2p/discv5" + "github.com/ethereum/go-ethereum/p2p/enode" "github.com/ethereum/go-ethereum/params" ) @@ -62,9 +62,13 @@ func GoerliGenesis() string { // FoundationBootnodes returns the enode URLs of the P2P bootstrap nodes operated // by the foundation running the V5 discovery protocol. func FoundationBootnodes() *Enodes { - nodes := &Enodes{nodes: make([]*discv5.Node, len(params.MainnetBootnodes))} + nodes := &Enodes{nodes: make([]*enode.Node, len(params.MainnetBootnodes))} for i, url := range params.MainnetBootnodes { - nodes.nodes[i] = discv5.MustParseNode(url) + var err error + nodes.nodes[i], err = enode.Parse(enode.ValidSchemes, url) + if err != nil { + panic("invalid node URL: " + err.Error()) + } } return nodes } From 3468f353aa77d591533395851339454d02e2b28b Mon Sep 17 00:00:00 2001 From: Zsolt Felfoldi Date: Wed, 2 Dec 2020 09:54:24 +0100 Subject: [PATCH 04/10] p2p/discv5: removed old discv5 package --- p2p/discv5/README | 4 - p2p/discv5/database.go | 396 ---------- p2p/discv5/database_test.go | 380 --------- p2p/discv5/metrics.go | 24 - p2p/discv5/net.go | 1269 ------------------------------- p2p/discv5/net_test.go | 330 -------- p2p/discv5/node.go | 413 ---------- p2p/discv5/node_test.go | 305 -------- p2p/discv5/nodeevent_string.go | 17 - p2p/discv5/sim_run_test.go | 126 --- p2p/discv5/sim_test.go | 432 ----------- p2p/discv5/sim_testmain_test.go | 43 -- p2p/discv5/table.go | 318 -------- p2p/discv5/table_test.go | 238 ------ p2p/discv5/ticket.go | 884 --------------------- p2p/discv5/topic.go | 407 ---------- p2p/discv5/topic_test.go | 71 -- p2p/discv5/udp.go | 429 ----------- 18 files changed, 6086 deletions(-) delete mode 100644 p2p/discv5/README delete mode 100644 p2p/discv5/database.go delete mode 100644 p2p/discv5/database_test.go delete mode 100644 p2p/discv5/metrics.go delete mode 100644 p2p/discv5/net.go delete mode 100644 p2p/discv5/net_test.go delete mode 100644 p2p/discv5/node.go delete mode 100644 p2p/discv5/node_test.go delete mode 100644 p2p/discv5/nodeevent_string.go delete mode 100644 p2p/discv5/sim_run_test.go delete mode 100644 p2p/discv5/sim_test.go delete mode 100644 p2p/discv5/sim_testmain_test.go delete mode 100644 p2p/discv5/table.go delete mode 100644 p2p/discv5/table_test.go delete mode 100644 p2p/discv5/ticket.go delete mode 100644 p2p/discv5/topic.go delete mode 100644 p2p/discv5/topic_test.go delete mode 100644 p2p/discv5/udp.go diff --git a/p2p/discv5/README b/p2p/discv5/README deleted file mode 100644 index 617a473d7ff2..000000000000 --- a/p2p/discv5/README +++ /dev/null @@ -1,4 +0,0 @@ -This package is an early prototype of Discovery v5. Do not use this code. - -See https://github.com/ethereum/devp2p/blob/master/discv5/discv5.md for the -current Discovery v5 specification. \ No newline at end of file diff --git a/p2p/discv5/database.go b/p2p/discv5/database.go deleted file mode 100644 index ca118e7f80f2..000000000000 --- a/p2p/discv5/database.go +++ /dev/null @@ -1,396 +0,0 @@ -// Copyright 2015 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -// Contains the node database, storing previously seen nodes and any collected -// metadata about them for QoS purposes. - -package discv5 - -import ( - "bytes" - "crypto/rand" - "encoding/binary" - "fmt" - "os" - "sync" - "time" - - "github.com/ethereum/go-ethereum/crypto" - "github.com/ethereum/go-ethereum/log" - "github.com/ethereum/go-ethereum/rlp" - "github.com/syndtr/goleveldb/leveldb" - "github.com/syndtr/goleveldb/leveldb/errors" - "github.com/syndtr/goleveldb/leveldb/iterator" - "github.com/syndtr/goleveldb/leveldb/opt" - "github.com/syndtr/goleveldb/leveldb/storage" - "github.com/syndtr/goleveldb/leveldb/util" -) - -var ( - nodeDBNilNodeID = NodeID{} // Special node ID to use as a nil element. - nodeDBNodeExpiration = 24 * time.Hour // Time after which an unseen node should be dropped. - nodeDBCleanupCycle = time.Hour // Time period for running the expiration task. -) - -// nodeDB stores all nodes we know about. -type nodeDB struct { - lvl *leveldb.DB // Interface to the database itself - self NodeID // Own node id to prevent adding it into the database - runner sync.Once // Ensures we can start at most one expirer - quit chan struct{} // Channel to signal the expiring thread to stop -} - -// Schema layout for the node database -var ( - nodeDBVersionKey = []byte("version") // Version of the database to flush if changes - nodeDBItemPrefix = []byte("n:") // Identifier to prefix node entries with - - nodeDBDiscoverRoot = ":discover" - nodeDBDiscoverPing = nodeDBDiscoverRoot + ":lastping" - nodeDBDiscoverPong = nodeDBDiscoverRoot + ":lastpong" - nodeDBDiscoverFindFails = nodeDBDiscoverRoot + ":findfail" - nodeDBTopicRegTickets = ":tickets" -) - -// newNodeDB creates a new node database for storing and retrieving infos about -// known peers in the network. If no path is given, an in-memory, temporary -// database is constructed. -func newNodeDB(path string, version int, self NodeID) (*nodeDB, error) { - if path == "" { - return newMemoryNodeDB(self) - } - return newPersistentNodeDB(path, version, self) -} - -// newMemoryNodeDB creates a new in-memory node database without a persistent -// backend. -func newMemoryNodeDB(self NodeID) (*nodeDB, error) { - db, err := leveldb.Open(storage.NewMemStorage(), nil) - if err != nil { - return nil, err - } - return &nodeDB{ - lvl: db, - self: self, - quit: make(chan struct{}), - }, nil -} - -// newPersistentNodeDB creates/opens a leveldb backed persistent node database, -// also flushing its contents in case of a version mismatch. -func newPersistentNodeDB(path string, version int, self NodeID) (*nodeDB, error) { - opts := &opt.Options{OpenFilesCacheCapacity: 5} - db, err := leveldb.OpenFile(path, opts) - if _, iscorrupted := err.(*errors.ErrCorrupted); iscorrupted { - db, err = leveldb.RecoverFile(path, nil) - } - if err != nil { - return nil, err - } - // The nodes contained in the cache correspond to a certain protocol version. - // Flush all nodes if the version doesn't match. - currentVer := make([]byte, binary.MaxVarintLen64) - currentVer = currentVer[:binary.PutVarint(currentVer, int64(version))] - - blob, err := db.Get(nodeDBVersionKey, nil) - switch err { - case leveldb.ErrNotFound: - // Version not found (i.e. empty cache), insert it - if err := db.Put(nodeDBVersionKey, currentVer, nil); err != nil { - db.Close() - return nil, err - } - - case nil: - // Version present, flush if different - if !bytes.Equal(blob, currentVer) { - db.Close() - if err = os.RemoveAll(path); err != nil { - return nil, err - } - return newPersistentNodeDB(path, version, self) - } - } - return &nodeDB{ - lvl: db, - self: self, - quit: make(chan struct{}), - }, nil -} - -// makeKey generates the leveldb key-blob from a node id and its particular -// field of interest. -func makeKey(id NodeID, field string) []byte { - if bytes.Equal(id[:], nodeDBNilNodeID[:]) { - return []byte(field) - } - return append(nodeDBItemPrefix, append(id[:], field...)...) -} - -// splitKey tries to split a database key into a node id and a field part. -func splitKey(key []byte) (id NodeID, field string) { - // If the key is not of a node, return it plainly - if !bytes.HasPrefix(key, nodeDBItemPrefix) { - return NodeID{}, string(key) - } - // Otherwise split the id and field - item := key[len(nodeDBItemPrefix):] - copy(id[:], item[:len(id)]) - field = string(item[len(id):]) - - return id, field -} - -// fetchInt64 retrieves an integer instance associated with a particular -// database key. -func (db *nodeDB) fetchInt64(key []byte) int64 { - blob, err := db.lvl.Get(key, nil) - if err != nil { - return 0 - } - val, read := binary.Varint(blob) - if read <= 0 { - return 0 - } - return val -} - -// storeInt64 update a specific database entry to the current time instance as a -// unix timestamp. -func (db *nodeDB) storeInt64(key []byte, n int64) error { - blob := make([]byte, binary.MaxVarintLen64) - blob = blob[:binary.PutVarint(blob, n)] - return db.lvl.Put(key, blob, nil) -} - -func (db *nodeDB) storeRLP(key []byte, val interface{}) error { - blob, err := rlp.EncodeToBytes(val) - if err != nil { - return err - } - return db.lvl.Put(key, blob, nil) -} - -func (db *nodeDB) fetchRLP(key []byte, val interface{}) error { - blob, err := db.lvl.Get(key, nil) - if err != nil { - return err - } - err = rlp.DecodeBytes(blob, val) - if err != nil { - log.Warn(fmt.Sprintf("key %x (%T) %v", key, val, err)) - } - return err -} - -// node retrieves a node with a given id from the database. -func (db *nodeDB) node(id NodeID) *Node { - var node Node - if err := db.fetchRLP(makeKey(id, nodeDBDiscoverRoot), &node); err != nil { - return nil - } - node.sha = crypto.Keccak256Hash(node.ID[:]) - return &node -} - -// updateNode inserts - potentially overwriting - a node into the peer database. -func (db *nodeDB) updateNode(node *Node) error { - return db.storeRLP(makeKey(node.ID, nodeDBDiscoverRoot), node) -} - -// deleteNode deletes all information/keys associated with a node. -func (db *nodeDB) deleteNode(id NodeID) error { - deleter := db.lvl.NewIterator(util.BytesPrefix(makeKey(id, "")), nil) - for deleter.Next() { - if err := db.lvl.Delete(deleter.Key(), nil); err != nil { - return err - } - } - return nil -} - -// ensureExpirer is a small helper method ensuring that the data expiration -// mechanism is running. If the expiration goroutine is already running, this -// method simply returns. -// -// The goal is to start the data evacuation only after the network successfully -// bootstrapped itself (to prevent dumping potentially useful seed nodes). Since -// it would require significant overhead to exactly trace the first successful -// convergence, it's simpler to "ensure" the correct state when an appropriate -// condition occurs (i.e. a successful bonding), and discard further events. -func (db *nodeDB) ensureExpirer() { - db.runner.Do(func() { go db.expirer() }) -} - -// expirer should be started in a go routine, and is responsible for looping ad -// infinitum and dropping stale data from the database. -func (db *nodeDB) expirer() { - tick := time.NewTicker(nodeDBCleanupCycle) - defer tick.Stop() - for { - select { - case <-tick.C: - if err := db.expireNodes(); err != nil { - log.Error(fmt.Sprintf("Failed to expire nodedb items: %v", err)) - } - case <-db.quit: - return - } - } -} - -// expireNodes iterates over the database and deletes all nodes that have not -// been seen (i.e. received a pong from) for some allotted time. -func (db *nodeDB) expireNodes() error { - threshold := time.Now().Add(-nodeDBNodeExpiration) - - // Find discovered nodes that are older than the allowance - it := db.lvl.NewIterator(nil, nil) - defer it.Release() - - for it.Next() { - // Skip the item if not a discovery node - id, field := splitKey(it.Key()) - if field != nodeDBDiscoverRoot { - continue - } - // Skip the node if not expired yet (and not self) - if !bytes.Equal(id[:], db.self[:]) { - if seen := db.lastPong(id); seen.After(threshold) { - continue - } - } - // Otherwise delete all associated information - db.deleteNode(id) - } - return nil -} - -// lastPing retrieves the time of the last ping packet send to a remote node, -// requesting binding. -func (db *nodeDB) lastPing(id NodeID) time.Time { - return time.Unix(db.fetchInt64(makeKey(id, nodeDBDiscoverPing)), 0) -} - -// updateLastPing updates the last time we tried contacting a remote node. -func (db *nodeDB) updateLastPing(id NodeID, instance time.Time) error { - return db.storeInt64(makeKey(id, nodeDBDiscoverPing), instance.Unix()) -} - -// lastPong retrieves the time of the last successful contact from remote node. -func (db *nodeDB) lastPong(id NodeID) time.Time { - return time.Unix(db.fetchInt64(makeKey(id, nodeDBDiscoverPong)), 0) -} - -// updateLastPong updates the last time a remote node successfully contacted. -func (db *nodeDB) updateLastPong(id NodeID, instance time.Time) error { - return db.storeInt64(makeKey(id, nodeDBDiscoverPong), instance.Unix()) -} - -// findFails retrieves the number of findnode failures since bonding. -func (db *nodeDB) findFails(id NodeID) int { - return int(db.fetchInt64(makeKey(id, nodeDBDiscoverFindFails))) -} - -// updateFindFails updates the number of findnode failures since bonding. -func (db *nodeDB) updateFindFails(id NodeID, fails int) error { - return db.storeInt64(makeKey(id, nodeDBDiscoverFindFails), int64(fails)) -} - -// querySeeds retrieves random nodes to be used as potential seed nodes -// for bootstrapping. -func (db *nodeDB) querySeeds(n int, maxAge time.Duration) []*Node { - var ( - now = time.Now() - nodes = make([]*Node, 0, n) - it = db.lvl.NewIterator(nil, nil) - id NodeID - ) - defer it.Release() - -seek: - for seeks := 0; len(nodes) < n && seeks < n*5; seeks++ { - // Seek to a random entry. The first byte is incremented by a - // random amount each time in order to increase the likelihood - // of hitting all existing nodes in very small databases. - ctr := id[0] - rand.Read(id[:]) - id[0] = ctr + id[0]%16 - it.Seek(makeKey(id, nodeDBDiscoverRoot)) - - n := nextNode(it) - if n == nil { - id[0] = 0 - continue seek // iterator exhausted - } - if n.ID == db.self { - continue seek - } - if now.Sub(db.lastPong(n.ID)) > maxAge { - continue seek - } - for i := range nodes { - if nodes[i].ID == n.ID { - continue seek // duplicate - } - } - nodes = append(nodes, n) - } - return nodes -} - -func (db *nodeDB) fetchTopicRegTickets(id NodeID) (issued, used uint32) { - key := makeKey(id, nodeDBTopicRegTickets) - blob, _ := db.lvl.Get(key, nil) - if len(blob) != 8 { - return 0, 0 - } - issued = binary.BigEndian.Uint32(blob[0:4]) - used = binary.BigEndian.Uint32(blob[4:8]) - return -} - -func (db *nodeDB) updateTopicRegTickets(id NodeID, issued, used uint32) error { - key := makeKey(id, nodeDBTopicRegTickets) - blob := make([]byte, 8) - binary.BigEndian.PutUint32(blob[0:4], issued) - binary.BigEndian.PutUint32(blob[4:8], used) - return db.lvl.Put(key, blob, nil) -} - -// reads the next node record from the iterator, skipping over other -// database entries. -func nextNode(it iterator.Iterator) *Node { - for end := false; !end; end = !it.Next() { - id, field := splitKey(it.Key()) - if field != nodeDBDiscoverRoot { - continue - } - var n Node - if err := rlp.DecodeBytes(it.Value(), &n); err != nil { - log.Warn(fmt.Sprintf("invalid node %x: %v", id, err)) - continue - } - return &n - } - return nil -} - -// close flushes and closes the database files. -func (db *nodeDB) close() { - close(db.quit) - db.lvl.Close() -} diff --git a/p2p/discv5/database_test.go b/p2p/discv5/database_test.go deleted file mode 100644 index 2b86dc9ceccb..000000000000 --- a/p2p/discv5/database_test.go +++ /dev/null @@ -1,380 +0,0 @@ -// Copyright 2015 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -package discv5 - -import ( - "bytes" - "io/ioutil" - "net" - "os" - "path/filepath" - "reflect" - "testing" - "time" -) - -var nodeDBKeyTests = []struct { - id NodeID - field string - key []byte -}{ - { - id: NodeID{}, - field: "version", - key: []byte{0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e}, // field - }, - { - id: MustHexID("0x1dd9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439"), - field: ":discover", - key: []byte{0x6e, 0x3a, // prefix - 0x1d, 0xd9, 0xd6, 0x5c, 0x45, 0x52, 0xb5, 0xeb, // node id - 0x43, 0xd5, 0xad, 0x55, 0xa2, 0xee, 0x3f, 0x56, // - 0xc6, 0xcb, 0xc1, 0xc6, 0x4a, 0x5c, 0x8d, 0x65, // - 0x9f, 0x51, 0xfc, 0xd5, 0x1b, 0xac, 0xe2, 0x43, // - 0x51, 0x23, 0x2b, 0x8d, 0x78, 0x21, 0x61, 0x7d, // - 0x2b, 0x29, 0xb5, 0x4b, 0x81, 0xcd, 0xef, 0xb9, // - 0xb3, 0xe9, 0xc3, 0x7d, 0x7f, 0xd5, 0xf6, 0x32, // - 0x70, 0xbc, 0xc9, 0xe1, 0xa6, 0xf6, 0xa4, 0x39, // - 0x3a, 0x64, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, // field - }, - }, -} - -func TestNodeDBKeys(t *testing.T) { - for i, tt := range nodeDBKeyTests { - if key := makeKey(tt.id, tt.field); !bytes.Equal(key, tt.key) { - t.Errorf("make test %d: key mismatch: have 0x%x, want 0x%x", i, key, tt.key) - } - id, field := splitKey(tt.key) - if !bytes.Equal(id[:], tt.id[:]) { - t.Errorf("split test %d: id mismatch: have 0x%x, want 0x%x", i, id, tt.id) - } - if field != tt.field { - t.Errorf("split test %d: field mismatch: have 0x%x, want 0x%x", i, field, tt.field) - } - } -} - -var nodeDBInt64Tests = []struct { - key []byte - value int64 -}{ - {key: []byte{0x01}, value: 1}, - {key: []byte{0x02}, value: 2}, - {key: []byte{0x03}, value: 3}, -} - -func TestNodeDBInt64(t *testing.T) { - db, _ := newNodeDB("", Version, NodeID{}) - defer db.close() - - tests := nodeDBInt64Tests - for i := 0; i < len(tests); i++ { - // Insert the next value - if err := db.storeInt64(tests[i].key, tests[i].value); err != nil { - t.Errorf("test %d: failed to store value: %v", i, err) - } - // Check all existing and non existing values - for j := 0; j < len(tests); j++ { - num := db.fetchInt64(tests[j].key) - switch { - case j <= i && num != tests[j].value: - t.Errorf("test %d, item %d: value mismatch: have %v, want %v", i, j, num, tests[j].value) - case j > i && num != 0: - t.Errorf("test %d, item %d: value mismatch: have %v, want %v", i, j, num, 0) - } - } - } -} - -func TestNodeDBFetchStore(t *testing.T) { - node := NewNode( - MustHexID("0x1dd9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439"), - net.IP{192, 168, 0, 1}, - 30303, - 30303, - ) - inst := time.Now() - num := 314 - - db, _ := newNodeDB("", Version, NodeID{}) - defer db.close() - - // Check fetch/store operations on a node ping object - if stored := db.lastPing(node.ID); stored.Unix() != 0 { - t.Errorf("ping: non-existing object: %v", stored) - } - if err := db.updateLastPing(node.ID, inst); err != nil { - t.Errorf("ping: failed to update: %v", err) - } - if stored := db.lastPing(node.ID); stored.Unix() != inst.Unix() { - t.Errorf("ping: value mismatch: have %v, want %v", stored, inst) - } - // Check fetch/store operations on a node pong object - if stored := db.lastPong(node.ID); stored.Unix() != 0 { - t.Errorf("pong: non-existing object: %v", stored) - } - if err := db.updateLastPong(node.ID, inst); err != nil { - t.Errorf("pong: failed to update: %v", err) - } - if stored := db.lastPong(node.ID); stored.Unix() != inst.Unix() { - t.Errorf("pong: value mismatch: have %v, want %v", stored, inst) - } - // Check fetch/store operations on a node findnode-failure object - if stored := db.findFails(node.ID); stored != 0 { - t.Errorf("find-node fails: non-existing object: %v", stored) - } - if err := db.updateFindFails(node.ID, num); err != nil { - t.Errorf("find-node fails: failed to update: %v", err) - } - if stored := db.findFails(node.ID); stored != num { - t.Errorf("find-node fails: value mismatch: have %v, want %v", stored, num) - } - // Check fetch/store operations on an actual node object - if stored := db.node(node.ID); stored != nil { - t.Errorf("node: non-existing object: %v", stored) - } - if err := db.updateNode(node); err != nil { - t.Errorf("node: failed to update: %v", err) - } - if stored := db.node(node.ID); stored == nil { - t.Errorf("node: not found") - } else if !reflect.DeepEqual(stored, node) { - t.Errorf("node: data mismatch: have %v, want %v", stored, node) - } -} - -var nodeDBSeedQueryNodes = []struct { - node *Node - pong time.Time -}{ - // This one should not be in the result set because its last - // pong time is too far in the past. - { - node: NewNode( - MustHexID("0x84d9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439"), - net.IP{127, 0, 0, 3}, - 30303, - 30303, - ), - pong: time.Now().Add(-3 * time.Hour), - }, - // This one shouldn't be in the result set because its - // nodeID is the local node's ID. - { - node: NewNode( - MustHexID("0x57d9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439"), - net.IP{127, 0, 0, 3}, - 30303, - 30303, - ), - pong: time.Now().Add(-4 * time.Second), - }, - - // These should be in the result set. - { - node: NewNode( - MustHexID("0x22d9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439"), - net.IP{127, 0, 0, 1}, - 30303, - 30303, - ), - pong: time.Now().Add(-2 * time.Second), - }, - { - node: NewNode( - MustHexID("0x44d9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439"), - net.IP{127, 0, 0, 2}, - 30303, - 30303, - ), - pong: time.Now().Add(-3 * time.Second), - }, - { - node: NewNode( - MustHexID("0xe2d9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439"), - net.IP{127, 0, 0, 3}, - 30303, - 30303, - ), - pong: time.Now().Add(-1 * time.Second), - }, -} - -func TestNodeDBSeedQuery(t *testing.T) { - db, _ := newNodeDB("", Version, nodeDBSeedQueryNodes[1].node.ID) - defer db.close() - - // Insert a batch of nodes for querying - for i, seed := range nodeDBSeedQueryNodes { - if err := db.updateNode(seed.node); err != nil { - t.Fatalf("node %d: failed to insert: %v", i, err) - } - if err := db.updateLastPong(seed.node.ID, seed.pong); err != nil { - t.Fatalf("node %d: failed to insert lastPong: %v", i, err) - } - } - - // Retrieve the entire batch and check for duplicates - seeds := db.querySeeds(len(nodeDBSeedQueryNodes)*2, time.Hour) - have := make(map[NodeID]struct{}) - for _, seed := range seeds { - have[seed.ID] = struct{}{} - } - want := make(map[NodeID]struct{}) - for _, seed := range nodeDBSeedQueryNodes[2:] { - want[seed.node.ID] = struct{}{} - } - if len(seeds) != len(want) { - t.Errorf("seed count mismatch: have %v, want %v", len(seeds), len(want)) - } - for id := range have { - if _, ok := want[id]; !ok { - t.Errorf("extra seed: %v", id) - } - } - for id := range want { - if _, ok := have[id]; !ok { - t.Errorf("missing seed: %v", id) - } - } -} - -func TestNodeDBPersistency(t *testing.T) { - root, err := ioutil.TempDir("", "nodedb-") - if err != nil { - t.Fatalf("failed to create temporary data folder: %v", err) - } - defer os.RemoveAll(root) - - var ( - testKey = []byte("somekey") - testInt = int64(314) - ) - - // Create a persistent database and store some values - db, err := newNodeDB(filepath.Join(root, "database"), Version, NodeID{}) - if err != nil { - t.Fatalf("failed to create persistent database: %v", err) - } - if err := db.storeInt64(testKey, testInt); err != nil { - t.Fatalf("failed to store value: %v.", err) - } - db.close() - - // Reopen the database and check the value - db, err = newNodeDB(filepath.Join(root, "database"), Version, NodeID{}) - if err != nil { - t.Fatalf("failed to open persistent database: %v", err) - } - if val := db.fetchInt64(testKey); val != testInt { - t.Fatalf("value mismatch: have %v, want %v", val, testInt) - } - db.close() - - // Change the database version and check flush - db, err = newNodeDB(filepath.Join(root, "database"), Version+1, NodeID{}) - if err != nil { - t.Fatalf("failed to open persistent database: %v", err) - } - if val := db.fetchInt64(testKey); val != 0 { - t.Fatalf("value mismatch: have %v, want %v", val, 0) - } - db.close() -} - -var nodeDBExpirationNodes = []struct { - node *Node - pong time.Time - exp bool -}{ - { - node: NewNode( - MustHexID("0x01d9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439"), - net.IP{127, 0, 0, 1}, - 30303, - 30303, - ), - pong: time.Now().Add(-nodeDBNodeExpiration + time.Minute), - exp: false, - }, { - node: NewNode( - MustHexID("0x02d9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439"), - net.IP{127, 0, 0, 2}, - 30303, - 30303, - ), - pong: time.Now().Add(-nodeDBNodeExpiration - time.Minute), - exp: true, - }, -} - -func TestNodeDBExpiration(t *testing.T) { - db, _ := newNodeDB("", Version, NodeID{}) - defer db.close() - - // Add all the test nodes and set their last pong time - for i, seed := range nodeDBExpirationNodes { - if err := db.updateNode(seed.node); err != nil { - t.Fatalf("node %d: failed to insert: %v", i, err) - } - if err := db.updateLastPong(seed.node.ID, seed.pong); err != nil { - t.Fatalf("node %d: failed to update pong: %v", i, err) - } - } - // Expire some of them, and check the rest - if err := db.expireNodes(); err != nil { - t.Fatalf("failed to expire nodes: %v", err) - } - for i, seed := range nodeDBExpirationNodes { - node := db.node(seed.node.ID) - if (node == nil && !seed.exp) || (node != nil && seed.exp) { - t.Errorf("node %d: expiration mismatch: have %v, want %v", i, node, seed.exp) - } - } -} - -func TestNodeDBSelfExpiration(t *testing.T) { - // Find a node in the tests that shouldn't expire, and assign it as self - var self NodeID - for _, node := range nodeDBExpirationNodes { - if !node.exp { - self = node.node.ID - break - } - } - db, _ := newNodeDB("", Version, self) - defer db.close() - - // Add all the test nodes and set their last pong time - for i, seed := range nodeDBExpirationNodes { - if err := db.updateNode(seed.node); err != nil { - t.Fatalf("node %d: failed to insert: %v", i, err) - } - if err := db.updateLastPong(seed.node.ID, seed.pong); err != nil { - t.Fatalf("node %d: failed to update pong: %v", i, err) - } - } - // Expire the nodes and make sure self has been evacuated too - if err := db.expireNodes(); err != nil { - t.Fatalf("failed to expire nodes: %v", err) - } - node := db.node(self) - if node != nil { - t.Errorf("self not evacuated") - } -} diff --git a/p2p/discv5/metrics.go b/p2p/discv5/metrics.go deleted file mode 100644 index e68d53c13c63..000000000000 --- a/p2p/discv5/metrics.go +++ /dev/null @@ -1,24 +0,0 @@ -// Copyright 2018 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -package discv5 - -import "github.com/ethereum/go-ethereum/metrics" - -var ( - ingressTrafficMeter = metrics.NewRegisteredMeter("discv5/InboundTraffic", nil) - egressTrafficMeter = metrics.NewRegisteredMeter("discv5/OutboundTraffic", nil) -) diff --git a/p2p/discv5/net.go b/p2p/discv5/net.go deleted file mode 100644 index 53e00a38817a..000000000000 --- a/p2p/discv5/net.go +++ /dev/null @@ -1,1269 +0,0 @@ -// Copyright 2016 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -package discv5 - -import ( - "bytes" - "crypto/ecdsa" - "errors" - "fmt" - "net" - "time" - - "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/common/mclock" - "github.com/ethereum/go-ethereum/crypto" - "github.com/ethereum/go-ethereum/log" - "github.com/ethereum/go-ethereum/p2p/netutil" - "github.com/ethereum/go-ethereum/rlp" - "golang.org/x/crypto/sha3" -) - -var ( - errInvalidEvent = errors.New("invalid in current state") - errNoQuery = errors.New("no pending query") -) - -const ( - autoRefreshInterval = 1 * time.Hour - bucketRefreshInterval = 1 * time.Minute - seedCount = 30 - seedMaxAge = 5 * 24 * time.Hour - lowPort = 1024 -) - -const testTopic = "foo" - -const ( - printTestImgLogs = false -) - -// Network manages the table and all protocol interaction. -type Network struct { - db *nodeDB // database of known nodes - conn transport - netrestrict *netutil.Netlist - - closed chan struct{} // closed when loop is done - closeReq chan struct{} // 'request to close' - refreshReq chan []*Node // lookups ask for refresh on this channel - refreshResp chan (<-chan struct{}) // ...and get the channel to block on from this one - read chan ingressPacket // ingress packets arrive here - timeout chan timeoutEvent - queryReq chan *findnodeQuery // lookups submit findnode queries on this channel - tableOpReq chan func() - tableOpResp chan struct{} - topicRegisterReq chan topicRegisterReq - topicSearchReq chan topicSearchReq - - // State of the main loop. - tab *Table - topictab *topicTable - ticketStore *ticketStore - nursery []*Node - nodes map[NodeID]*Node // tracks active nodes with state != known - timeoutTimers map[timeoutEvent]*time.Timer -} - -// 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 { - sendPing(remote *Node, remoteAddr *net.UDPAddr, topics []Topic) (hash []byte) - sendNeighbours(remote *Node, nodes []*Node) - sendFindnodeHash(remote *Node, target common.Hash) - sendTopicRegister(remote *Node, topics []Topic, topicIdx int, pong []byte) - sendTopicNodes(remote *Node, queryHash common.Hash, nodes []*Node) - - send(remote *Node, ptype nodeEvent, p interface{}) (hash []byte) - - localAddr() *net.UDPAddr - Close() -} - -type findnodeQuery struct { - remote *Node - target common.Hash - reply chan<- []*Node -} - -type topicRegisterReq struct { - add bool - topic Topic -} - -type topicSearchReq struct { - topic Topic - found chan<- *Node - lookup chan<- bool - delay time.Duration -} - -type topicSearchResult struct { - target lookupInfo - nodes []*Node -} - -type timeoutEvent struct { - ev nodeEvent - node *Node -} - -func newNetwork(conn transport, ourPubkey ecdsa.PublicKey, dbPath string, netrestrict *netutil.Netlist) (*Network, error) { - ourID := PubkeyID(&ourPubkey) - - var db *nodeDB - if dbPath != "" { - var err error - if db, err = newNodeDB(dbPath, Version, ourID); err != nil { - return nil, err - } - } - - tab := newTable(ourID, conn.localAddr()) - net := &Network{ - db: db, - conn: conn, - netrestrict: netrestrict, - tab: tab, - topictab: newTopicTable(db, tab.self), - ticketStore: newTicketStore(), - refreshReq: make(chan []*Node), - refreshResp: make(chan (<-chan struct{})), - closed: make(chan struct{}), - closeReq: make(chan struct{}), - read: make(chan ingressPacket, 100), - timeout: make(chan timeoutEvent), - timeoutTimers: make(map[timeoutEvent]*time.Timer), - tableOpReq: make(chan func()), - tableOpResp: make(chan struct{}), - queryReq: make(chan *findnodeQuery), - topicRegisterReq: make(chan topicRegisterReq), - topicSearchReq: make(chan topicSearchReq), - nodes: make(map[NodeID]*Node), - } - go net.loop() - return net, nil -} - -// Close terminates the network listener and flushes the node database. -func (net *Network) Close() { - net.conn.Close() - select { - case <-net.closed: - case net.closeReq <- struct{}{}: - <-net.closed - } -} - -// Self returns the local node. -// The returned node should not be modified by the caller. -func (net *Network) Self() *Node { - return net.tab.self -} - -// ReadRandomNodes fills the given slice with random nodes from the -// table. It will not write the same node more than once. The nodes in -// the slice are copies and can be modified by the caller. -func (net *Network) ReadRandomNodes(buf []*Node) (n int) { - net.reqTableOp(func() { n = net.tab.readRandomNodes(buf) }) - return n -} - -// SetFallbackNodes sets the initial points of contact. These nodes -// are used to connect to the network if the table is empty and there -// are no known nodes in the database. -func (net *Network) SetFallbackNodes(nodes []*Node) error { - nursery := make([]*Node, 0, len(nodes)) - for _, n := range nodes { - if err := n.validateComplete(); err != nil { - return fmt.Errorf("bad bootstrap/fallback node %q (%v)", n, err) - } - // Recompute cpy.sha because the node might not have been - // created by NewNode or ParseNode. - cpy := *n - cpy.sha = crypto.Keccak256Hash(n.ID[:]) - nursery = append(nursery, &cpy) - } - net.reqRefresh(nursery) - return nil -} - -// Resolve searches for a specific node with the given ID. -// It returns nil if the node could not be found. -func (net *Network) Resolve(targetID NodeID) *Node { - result := net.lookup(crypto.Keccak256Hash(targetID[:]), true) - for _, n := range result { - if n.ID == targetID { - return n - } - } - return nil -} - -// Lookup performs a network search for nodes close -// to the given target. It approaches the target by querying -// nodes that are closer to it on each iteration. -// The given target does not need to be an actual node -// identifier. -// -// The local node may be included in the result. -func (net *Network) Lookup(targetID NodeID) []*Node { - return net.lookup(crypto.Keccak256Hash(targetID[:]), false) -} - -func (net *Network) lookup(target common.Hash, stopOnMatch bool) []*Node { - var ( - asked = make(map[NodeID]bool) - seen = make(map[NodeID]bool) - reply = make(chan []*Node, alpha) - result = nodesByDistance{target: target} - pendingQueries = 0 - ) - // Get initial answers from the local node. - result.push(net.tab.self, bucketSize) - for { - // Ask the α closest nodes that we haven't asked yet. - for i := 0; i < len(result.entries) && pendingQueries < alpha; i++ { - n := result.entries[i] - if !asked[n.ID] { - asked[n.ID] = true - pendingQueries++ - net.reqQueryFindnode(n, target, reply) - } - } - if pendingQueries == 0 { - // We have asked all closest nodes, stop the search. - break - } - // Wait for the next reply. - select { - case nodes := <-reply: - for _, n := range nodes { - if n != nil && !seen[n.ID] { - seen[n.ID] = true - result.push(n, bucketSize) - if stopOnMatch && n.sha == target { - return result.entries - } - } - } - pendingQueries-- - case <-time.After(respTimeout): - // forget all pending requests, start new ones - pendingQueries = 0 - reply = make(chan []*Node, alpha) - } - } - return result.entries -} - -func (net *Network) RegisterTopic(topic Topic, stop <-chan struct{}) { - select { - case net.topicRegisterReq <- topicRegisterReq{true, topic}: - case <-net.closed: - return - } - select { - case <-net.closed: - case <-stop: - select { - case net.topicRegisterReq <- topicRegisterReq{false, topic}: - case <-net.closed: - } - } -} - -func (net *Network) SearchTopic(topic Topic, setPeriod <-chan time.Duration, found chan<- *Node, lookup chan<- bool) { - for { - select { - case <-net.closed: - return - case delay, ok := <-setPeriod: - select { - case net.topicSearchReq <- topicSearchReq{topic: topic, found: found, lookup: lookup, delay: delay}: - case <-net.closed: - return - } - if !ok { - return - } - } - } -} - -func (net *Network) reqRefresh(nursery []*Node) <-chan struct{} { - select { - case net.refreshReq <- nursery: - return <-net.refreshResp - case <-net.closed: - return net.closed - } -} - -func (net *Network) reqQueryFindnode(n *Node, target common.Hash, reply chan []*Node) bool { - q := &findnodeQuery{remote: n, target: target, reply: reply} - select { - case net.queryReq <- q: - return true - case <-net.closed: - return false - } -} - -func (net *Network) reqReadPacket(pkt ingressPacket) { - select { - case net.read <- pkt: - case <-net.closed: - } -} - -func (net *Network) reqTableOp(f func()) (called bool) { - select { - case net.tableOpReq <- f: - <-net.tableOpResp - return true - case <-net.closed: - return false - } -} - -// TODO: external address handling. - -type topicSearchInfo struct { - lookupChn chan<- bool - period time.Duration -} - -const maxSearchCount = 5 - -func (net *Network) loop() { - var ( - refreshTimer = time.NewTicker(autoRefreshInterval) - bucketRefreshTimer = time.NewTimer(bucketRefreshInterval) - refreshDone chan struct{} // closed when the 'refresh' lookup has ended - ) - defer refreshTimer.Stop() - defer bucketRefreshTimer.Stop() - - // Tracking the next ticket to register. - var ( - nextTicket *ticketRef - nextRegisterTimer *time.Timer - nextRegisterTime <-chan time.Time - ) - defer func() { - if nextRegisterTimer != nil { - nextRegisterTimer.Stop() - } - }() - resetNextTicket := func() { - ticket, timeout := net.ticketStore.nextFilteredTicket() - if nextTicket != ticket { - nextTicket = ticket - if nextRegisterTimer != nil { - nextRegisterTimer.Stop() - nextRegisterTime = nil - } - if ticket != nil { - nextRegisterTimer = time.NewTimer(timeout) - nextRegisterTime = nextRegisterTimer.C - } - } - } - - // Tracking registration and search lookups. - var ( - topicRegisterLookupTarget lookupInfo - topicRegisterLookupDone chan []*Node - topicRegisterLookupTick = time.NewTimer(0) - searchReqWhenRefreshDone []topicSearchReq - searchInfo = make(map[Topic]topicSearchInfo) - activeSearchCount int - ) - defer topicRegisterLookupTick.Stop() - topicSearchLookupDone := make(chan topicSearchResult, 100) - topicSearch := make(chan Topic, 100) - <-topicRegisterLookupTick.C - - statsDump := time.NewTicker(10 * time.Second) - defer statsDump.Stop() - -loop: - for { - resetNextTicket() - - select { - case <-net.closeReq: - log.Trace("<-net.closeReq") - break loop - - // Ingress packet handling. - case pkt := <-net.read: - //fmt.Println("read", pkt.ev) - log.Trace("<-net.read") - n := net.internNode(&pkt) - prestate := n.state - status := "ok" - if err := net.handle(n, pkt.ev, &pkt); err != nil { - status = err.Error() - } - log.Trace("", "msg", log.Lazy{Fn: func() string { - return fmt.Sprintf("<<< (%d) %v from %x@%v: %v -> %v (%v)", - net.tab.count, pkt.ev, pkt.remoteID[:8], pkt.remoteAddr, prestate, n.state, status) - }}) - // TODO: persist state if n.state goes >= known, delete if it goes <= known - - // State transition timeouts. - case timeout := <-net.timeout: - log.Trace("<-net.timeout") - if net.timeoutTimers[timeout] == nil { - // Stale timer (was aborted). - continue - } - delete(net.timeoutTimers, timeout) - prestate := timeout.node.state - status := "ok" - if err := net.handle(timeout.node, timeout.ev, nil); err != nil { - status = err.Error() - } - log.Trace("", "msg", log.Lazy{Fn: func() string { - return fmt.Sprintf("--- (%d) %v for %x@%v: %v -> %v (%v)", - net.tab.count, timeout.ev, timeout.node.ID[:8], timeout.node.addr(), prestate, timeout.node.state, status) - }}) - - // Querying. - case q := <-net.queryReq: - log.Trace("<-net.queryReq") - if !q.start(net) { - q.remote.deferQuery(q) - } - - // Interacting with the table. - case f := <-net.tableOpReq: - log.Trace("<-net.tableOpReq") - f() - net.tableOpResp <- struct{}{} - - // Topic registration stuff. - case req := <-net.topicRegisterReq: - log.Trace("<-net.topicRegisterReq") - if !req.add { - net.ticketStore.removeRegisterTopic(req.topic) - continue - } - net.ticketStore.addTopic(req.topic, true) - // If we're currently waiting idle (nothing to look up), give the ticket store a - // chance to start it sooner. This should speed up convergence of the radius - // determination for new topics. - // if topicRegisterLookupDone == nil { - if topicRegisterLookupTarget.target == (common.Hash{}) { - log.Trace("topicRegisterLookupTarget == null") - if topicRegisterLookupTick.Stop() { - <-topicRegisterLookupTick.C - } - target, delay := net.ticketStore.nextRegisterLookup() - topicRegisterLookupTarget = target - topicRegisterLookupTick.Reset(delay) - } - - case nodes := <-topicRegisterLookupDone: - log.Trace("<-topicRegisterLookupDone") - net.ticketStore.registerLookupDone(topicRegisterLookupTarget, nodes, func(n *Node) []byte { - net.ping(n, n.addr()) - return n.pingEcho - }) - target, delay := net.ticketStore.nextRegisterLookup() - topicRegisterLookupTarget = target - topicRegisterLookupTick.Reset(delay) - topicRegisterLookupDone = nil - - case <-topicRegisterLookupTick.C: - log.Trace("<-topicRegisterLookupTick") - if (topicRegisterLookupTarget.target == common.Hash{}) { - target, delay := net.ticketStore.nextRegisterLookup() - topicRegisterLookupTarget = target - topicRegisterLookupTick.Reset(delay) - topicRegisterLookupDone = nil - } else { - topicRegisterLookupDone = make(chan []*Node) - target := topicRegisterLookupTarget.target - go func() { topicRegisterLookupDone <- net.lookup(target, false) }() - } - - case <-nextRegisterTime: - log.Trace("<-nextRegisterTime") - net.ticketStore.ticketRegistered(*nextTicket) - //fmt.Println("sendTopicRegister", nextTicket.t.node.addr().String(), nextTicket.t.topics, nextTicket.idx, nextTicket.t.pong) - net.conn.sendTopicRegister(nextTicket.t.node, nextTicket.t.topics, nextTicket.idx, nextTicket.t.pong) - - case req := <-net.topicSearchReq: - if refreshDone == nil { - log.Trace("<-net.topicSearchReq") - info, ok := searchInfo[req.topic] - if ok { - if req.delay == time.Duration(0) { - delete(searchInfo, req.topic) - net.ticketStore.removeSearchTopic(req.topic) - } else { - info.period = req.delay - searchInfo[req.topic] = info - } - continue - } - if req.delay != time.Duration(0) { - var info topicSearchInfo - info.period = req.delay - info.lookupChn = req.lookup - searchInfo[req.topic] = info - net.ticketStore.addSearchTopic(req.topic, req.found) - topicSearch <- req.topic - } - } else { - searchReqWhenRefreshDone = append(searchReqWhenRefreshDone, req) - } - - case topic := <-topicSearch: - if activeSearchCount < maxSearchCount { - activeSearchCount++ - target := net.ticketStore.nextSearchLookup(topic) - go func() { - nodes := net.lookup(target.target, false) - topicSearchLookupDone <- topicSearchResult{target: target, nodes: nodes} - }() - } - period := searchInfo[topic].period - if period != time.Duration(0) { - go func() { - time.Sleep(period) - topicSearch <- topic - }() - } - - case res := <-topicSearchLookupDone: - activeSearchCount-- - if lookupChn := searchInfo[res.target.topic].lookupChn; lookupChn != nil { - lookupChn <- net.ticketStore.radius[res.target.topic].converged - } - net.ticketStore.searchLookupDone(res.target, res.nodes, func(n *Node, topic Topic) []byte { - if n.state != nil && n.state.canQuery { - return net.conn.send(n, topicQueryPacket, topicQuery{Topic: topic}) // TODO: set expiration - } - if n.state == unknown { - net.ping(n, n.addr()) - } - return nil - }) - - case <-statsDump.C: - log.Trace("<-statsDump.C") - /*r, ok := net.ticketStore.radius[testTopic] - if !ok { - fmt.Printf("(%x) no radius @ %v\n", net.tab.self.ID[:8], time.Now()) - } else { - topics := len(net.ticketStore.tickets) - tickets := len(net.ticketStore.nodes) - rad := r.radius / (maxRadius/10000+1) - fmt.Printf("(%x) topics:%d radius:%d tickets:%d @ %v\n", net.tab.self.ID[:8], topics, rad, tickets, time.Now()) - }*/ - - tm := mclock.Now() - for topic, r := range net.ticketStore.radius { - if printTestImgLogs { - rad := r.radius / (maxRadius/1000000 + 1) - minrad := r.minRadius / (maxRadius/1000000 + 1) - fmt.Printf("*R %d %v %016x %v\n", tm/1000000, topic, net.tab.self.sha[:8], rad) - fmt.Printf("*MR %d %v %016x %v\n", tm/1000000, topic, net.tab.self.sha[:8], minrad) - } - } - for topic, t := range net.topictab.topics { - wp := t.wcl.nextWaitPeriod(tm) - if printTestImgLogs { - fmt.Printf("*W %d %v %016x %d\n", tm/1000000, topic, net.tab.self.sha[:8], wp/1000000) - } - } - - // Periodic / lookup-initiated bucket refresh. - case <-refreshTimer.C: - log.Trace("<-refreshTimer.C") - // TODO: ideally we would start the refresh timer after - // fallback nodes have been set for the first time. - if refreshDone == nil { - refreshDone = make(chan struct{}) - net.refresh(refreshDone) - } - case <-bucketRefreshTimer.C: - target := net.tab.chooseBucketRefreshTarget() - go func() { - net.lookup(target, false) - bucketRefreshTimer.Reset(bucketRefreshInterval) - }() - case newNursery := <-net.refreshReq: - log.Trace("<-net.refreshReq") - if newNursery != nil { - net.nursery = newNursery - } - if refreshDone == nil { - refreshDone = make(chan struct{}) - net.refresh(refreshDone) - } - net.refreshResp <- refreshDone - case <-refreshDone: - log.Trace("<-net.refreshDone", "table size", net.tab.count) - if net.tab.count != 0 { - refreshDone = nil - list := searchReqWhenRefreshDone - searchReqWhenRefreshDone = nil - go func() { - for _, req := range list { - net.topicSearchReq <- req - } - }() - } else { - refreshDone = make(chan struct{}) - net.refresh(refreshDone) - } - } - } - log.Trace("loop stopped") - - log.Debug("shutting down") - if net.conn != nil { - net.conn.Close() - } - // TODO: wait for pending refresh. - // if refreshDone != nil { - // <-refreshResults - // } - // Cancel all pending timeouts. - for _, timer := range net.timeoutTimers { - timer.Stop() - } - if net.db != nil { - net.db.close() - } - close(net.closed) -} - -// Everything below runs on the Network.loop goroutine -// and can modify Node, Table and Network at any time without locking. - -func (net *Network) refresh(done chan<- struct{}) { - var seeds []*Node - if net.db != nil { - seeds = net.db.querySeeds(seedCount, seedMaxAge) - } - if len(seeds) == 0 { - seeds = net.nursery - } - if len(seeds) == 0 { - log.Trace("no seed nodes found") - time.AfterFunc(time.Second*10, func() { close(done) }) - return - } - for _, n := range seeds { - log.Debug("", "msg", log.Lazy{Fn: func() string { - var age string - if net.db != nil { - age = time.Since(net.db.lastPong(n.ID)).String() - } else { - age = "unknown" - } - return fmt.Sprintf("seed node (age %s): %v", age, n) - }}) - n = net.internNodeFromDB(n) - if n.state == unknown { - net.transition(n, verifyinit) - } - // Force-add the seed node so Lookup does something. - // It will be deleted again if verification fails. - net.tab.add(n) - } - // Start self lookup to fill up the buckets. - go func() { - net.Lookup(net.tab.self.ID) - close(done) - }() -} - -// Node Interning. - -func (net *Network) internNode(pkt *ingressPacket) *Node { - if n := net.nodes[pkt.remoteID]; n != nil { - n.IP = pkt.remoteAddr.IP - n.UDP = uint16(pkt.remoteAddr.Port) - n.TCP = uint16(pkt.remoteAddr.Port) - return n - } - n := NewNode(pkt.remoteID, pkt.remoteAddr.IP, uint16(pkt.remoteAddr.Port), uint16(pkt.remoteAddr.Port)) - n.state = unknown - net.nodes[pkt.remoteID] = n - return n -} - -func (net *Network) internNodeFromDB(dbn *Node) *Node { - if n := net.nodes[dbn.ID]; n != nil { - return n - } - n := NewNode(dbn.ID, dbn.IP, dbn.UDP, dbn.TCP) - n.state = unknown - net.nodes[n.ID] = n - return n -} - -func (net *Network) internNodeFromNeighbours(sender *net.UDPAddr, rn rpcNode) (n *Node, err error) { - if rn.ID == net.tab.self.ID { - return nil, errors.New("is self") - } - if rn.UDP <= lowPort { - return nil, errors.New("low port") - } - n = net.nodes[rn.ID] - if n == nil { - // We haven't seen this node before. - n, err = nodeFromRPC(sender, rn) - if net.netrestrict != nil && !net.netrestrict.Contains(n.IP) { - return n, errors.New("not contained in netrestrict whitelist") - } - if err == nil { - n.state = unknown - net.nodes[n.ID] = n - } - return n, err - } - if !n.IP.Equal(rn.IP) || n.UDP != rn.UDP || n.TCP != rn.TCP { - if n.state == known { - // reject address change if node is known by us - err = fmt.Errorf("metadata mismatch: got %v, want %v", rn, n) - } else { - // accept otherwise; this will be handled nicer with signed ENRs - n.IP = rn.IP - n.UDP = rn.UDP - n.TCP = rn.TCP - } - } - return n, err -} - -// nodeNetGuts is embedded in Node and contains fields. -type nodeNetGuts struct { - // This is a cached copy of sha3(ID) which is used for node - // distance calculations. This is part of Node in order to make it - // possible to write tests that need a node at a certain distance. - // In those tests, the content of sha will not actually correspond - // with ID. - sha common.Hash - - // State machine fields. Access to these fields - // is restricted to the Network.loop goroutine. - state *nodeState - pingEcho []byte // hash of last ping sent by us - pingTopics []Topic // topic set sent by us in last ping - deferredQueries []*findnodeQuery // queries that can't be sent yet - pendingNeighbours *findnodeQuery // current query, waiting for reply - queryTimeouts int -} - -func (n *nodeNetGuts) deferQuery(q *findnodeQuery) { - n.deferredQueries = append(n.deferredQueries, q) -} - -func (n *nodeNetGuts) startNextQuery(net *Network) { - if len(n.deferredQueries) == 0 { - return - } - nextq := n.deferredQueries[0] - if nextq.start(net) { - n.deferredQueries = append(n.deferredQueries[:0], n.deferredQueries[1:]...) - } -} - -func (q *findnodeQuery) start(net *Network) bool { - // Satisfy queries against the local node directly. - if q.remote == net.tab.self { - closest := net.tab.closest(q.target, bucketSize) - q.reply <- closest.entries - return true - } - if q.remote.state.canQuery && q.remote.pendingNeighbours == nil { - net.conn.sendFindnodeHash(q.remote, q.target) - net.timedEvent(respTimeout, q.remote, neighboursTimeout) - q.remote.pendingNeighbours = q - return true - } - // If the node is not known yet, it won't accept queries. - // Initiate the transition to known. - // The request will be sent later when the node reaches known state. - if q.remote.state == unknown { - net.transition(q.remote, verifyinit) - } - return false -} - -// Node Events (the input to the state machine). - -type nodeEvent uint - -//go:generate stringer -type=nodeEvent - -const ( - - // Packet type events. - // These correspond to packet types in the UDP protocol. - pingPacket = iota + 1 - pongPacket - findnodePacket - neighborsPacket - findnodeHashPacket - topicRegisterPacket - topicQueryPacket - topicNodesPacket - - // Non-packet events. - // Event values in this category are allocated outside - // the packet type range (packet types are encoded as a single byte). - pongTimeout nodeEvent = iota + 256 - pingTimeout - neighboursTimeout -) - -// Node State Machine. - -type nodeState struct { - name string - handle func(*Network, *Node, nodeEvent, *ingressPacket) (next *nodeState, err error) - enter func(*Network, *Node) - canQuery bool -} - -func (s *nodeState) String() string { - return s.name -} - -var ( - unknown *nodeState - verifyinit *nodeState - verifywait *nodeState - remoteverifywait *nodeState - known *nodeState - contested *nodeState - unresponsive *nodeState -) - -func init() { - unknown = &nodeState{ - name: "unknown", - enter: func(net *Network, n *Node) { - net.tab.delete(n) - n.pingEcho = nil - // Abort active queries. - for _, q := range n.deferredQueries { - q.reply <- nil - } - n.deferredQueries = nil - if n.pendingNeighbours != nil { - n.pendingNeighbours.reply <- nil - n.pendingNeighbours = nil - } - n.queryTimeouts = 0 - }, - handle: func(net *Network, n *Node, ev nodeEvent, pkt *ingressPacket) (*nodeState, error) { - switch ev { - case pingPacket: - net.handlePing(n, pkt) - net.ping(n, pkt.remoteAddr) - return verifywait, nil - default: - return unknown, errInvalidEvent - } - }, - } - - verifyinit = &nodeState{ - name: "verifyinit", - enter: func(net *Network, n *Node) { - net.ping(n, n.addr()) - }, - handle: func(net *Network, n *Node, ev nodeEvent, pkt *ingressPacket) (*nodeState, error) { - switch ev { - case pingPacket: - net.handlePing(n, pkt) - return verifywait, nil - case pongPacket: - err := net.handleKnownPong(n, pkt) - return remoteverifywait, err - case pongTimeout: - return unknown, nil - default: - return verifyinit, errInvalidEvent - } - }, - } - - verifywait = &nodeState{ - name: "verifywait", - handle: func(net *Network, n *Node, ev nodeEvent, pkt *ingressPacket) (*nodeState, error) { - switch ev { - case pingPacket: - net.handlePing(n, pkt) - return verifywait, nil - case pongPacket: - err := net.handleKnownPong(n, pkt) - return known, err - case pongTimeout: - return unknown, nil - default: - return verifywait, errInvalidEvent - } - }, - } - - remoteverifywait = &nodeState{ - name: "remoteverifywait", - enter: func(net *Network, n *Node) { - net.timedEvent(respTimeout, n, pingTimeout) - }, - handle: func(net *Network, n *Node, ev nodeEvent, pkt *ingressPacket) (*nodeState, error) { - switch ev { - case pingPacket: - net.handlePing(n, pkt) - return remoteverifywait, nil - case pingTimeout: - return known, nil - default: - return remoteverifywait, errInvalidEvent - } - }, - } - - known = &nodeState{ - name: "known", - canQuery: true, - enter: func(net *Network, n *Node) { - n.queryTimeouts = 0 - n.startNextQuery(net) - // Insert into the table and start revalidation of the last node - // in the bucket if it is full. - last := net.tab.add(n) - if last != nil && last.state == known { - // TODO: do this asynchronously - net.transition(last, contested) - } - }, - handle: func(net *Network, n *Node, ev nodeEvent, pkt *ingressPacket) (*nodeState, error) { - switch ev { - case pingPacket: - net.handlePing(n, pkt) - return known, nil - case pongPacket: - err := net.handleKnownPong(n, pkt) - return known, err - default: - return net.handleQueryEvent(n, ev, pkt) - } - }, - } - - contested = &nodeState{ - name: "contested", - canQuery: true, - enter: func(net *Network, n *Node) { - net.ping(n, n.addr()) - }, - handle: func(net *Network, n *Node, ev nodeEvent, pkt *ingressPacket) (*nodeState, error) { - switch ev { - case pongPacket: - // Node is still alive. - err := net.handleKnownPong(n, pkt) - return known, err - case pongTimeout: - net.tab.deleteReplace(n) - return unresponsive, nil - case pingPacket: - net.handlePing(n, pkt) - return contested, nil - default: - return net.handleQueryEvent(n, ev, pkt) - } - }, - } - - unresponsive = &nodeState{ - name: "unresponsive", - canQuery: true, - handle: func(net *Network, n *Node, ev nodeEvent, pkt *ingressPacket) (*nodeState, error) { - switch ev { - case pingPacket: - net.handlePing(n, pkt) - return known, nil - case pongPacket: - err := net.handleKnownPong(n, pkt) - return known, err - default: - return net.handleQueryEvent(n, ev, pkt) - } - }, - } -} - -// handle processes packets sent by n and events related to n. -func (net *Network) handle(n *Node, ev nodeEvent, pkt *ingressPacket) error { - //fmt.Println("handle", n.addr().String(), n.state, ev) - if pkt != nil { - if err := net.checkPacket(n, ev, pkt); err != nil { - //fmt.Println("check err:", err) - return err - } - // Start the background expiration goroutine after the first - // successful communication. Subsequent calls have no effect if it - // is already running. We do this here instead of somewhere else - // so that the search for seed nodes also considers older nodes - // that would otherwise be removed by the expirer. - if net.db != nil { - net.db.ensureExpirer() - } - } - if ev == pongTimeout { - n.pingEcho = nil // clean up if pongtimeout - } - if n.state == nil { - n.state = unknown //??? - } - next, err := n.state.handle(net, n, ev, pkt) - net.transition(n, next) - //fmt.Println("new state:", n.state) - return err -} - -func (net *Network) checkPacket(n *Node, ev nodeEvent, pkt *ingressPacket) error { - // Replay prevention checks. - switch ev { - case pingPacket, findnodeHashPacket, neighborsPacket: - // TODO: check date is > last date seen - // TODO: check ping version - case pongPacket: - if !bytes.Equal(pkt.data.(*pong).ReplyTok, n.pingEcho) { - // fmt.Println("pong reply token mismatch") - return fmt.Errorf("pong reply token mismatch") - } - n.pingEcho = nil - } - // Address validation. - // TODO: Ideally we would do the following: - // - reject all packets with wrong address except ping. - // - for ping with new address, transition to verifywait but keep the - // previous node (with old address) around. if the new one reaches known, - // swap it out. - return nil -} - -func (net *Network) transition(n *Node, next *nodeState) { - if n.state != next { - n.state = next - if next.enter != nil { - next.enter(net, n) - } - } - - // TODO: persist/unpersist node -} - -func (net *Network) timedEvent(d time.Duration, n *Node, ev nodeEvent) { - timeout := timeoutEvent{ev, n} - net.timeoutTimers[timeout] = time.AfterFunc(d, func() { - select { - case net.timeout <- timeout: - case <-net.closed: - } - }) -} - -func (net *Network) abortTimedEvent(n *Node, ev nodeEvent) { - timer := net.timeoutTimers[timeoutEvent{ev, n}] - if timer != nil { - timer.Stop() - delete(net.timeoutTimers, timeoutEvent{ev, n}) - } -} - -func (net *Network) ping(n *Node, addr *net.UDPAddr) { - //fmt.Println("ping", n.addr().String(), n.ID.String(), n.sha.Hex()) - if n.pingEcho != nil || n.ID == net.tab.self.ID { - //fmt.Println(" not sent") - return - } - log.Trace("Pinging remote node", "node", n.ID) - n.pingTopics = net.ticketStore.regTopicSet() - n.pingEcho = net.conn.sendPing(n, addr, n.pingTopics) - net.timedEvent(respTimeout, n, pongTimeout) -} - -func (net *Network) handlePing(n *Node, pkt *ingressPacket) { - log.Trace("Handling remote ping", "node", n.ID) - ping := pkt.data.(*ping) - n.TCP = ping.From.TCP - t := net.topictab.getTicket(n, ping.Topics) - - pong := &pong{ - To: makeEndpoint(n.addr(), n.TCP), // TODO: maybe use known TCP port from DB - ReplyTok: pkt.hash, - Expiration: uint64(time.Now().Add(expiration).Unix()), - } - ticketToPong(t, pong) - net.conn.send(n, pongPacket, pong) -} - -func (net *Network) handleKnownPong(n *Node, pkt *ingressPacket) error { - log.Trace("Handling known pong", "node", n.ID) - net.abortTimedEvent(n, pongTimeout) - now := mclock.Now() - ticket, err := pongToTicket(now, n.pingTopics, n, pkt) - if err == nil { - // fmt.Printf("(%x) ticket: %+v\n", net.tab.self.ID[:8], pkt.data) - net.ticketStore.addTicket(now, pkt.data.(*pong).ReplyTok, ticket) - } else { - log.Trace("Failed to convert pong to ticket", "err", err) - } - n.pingEcho = nil - n.pingTopics = nil - return err -} - -func (net *Network) handleQueryEvent(n *Node, ev nodeEvent, pkt *ingressPacket) (*nodeState, error) { - switch ev { - case findnodePacket: - target := crypto.Keccak256Hash(pkt.data.(*findnode).Target[:]) - results := net.tab.closest(target, bucketSize).entries - net.conn.sendNeighbours(n, results) - return n.state, nil - case neighborsPacket: - err := net.handleNeighboursPacket(n, pkt) - return n.state, err - case neighboursTimeout: - if n.pendingNeighbours != nil { - n.pendingNeighbours.reply <- nil - n.pendingNeighbours = nil - } - n.queryTimeouts++ - if n.queryTimeouts > maxFindnodeFailures && n.state == known { - return contested, errors.New("too many timeouts") - } - return n.state, nil - - // v5 - - case findnodeHashPacket: - results := net.tab.closest(pkt.data.(*findnodeHash).Target, bucketSize).entries - net.conn.sendNeighbours(n, results) - return n.state, nil - case topicRegisterPacket: - //fmt.Println("got topicRegisterPacket") - regdata := pkt.data.(*topicRegister) - pong, err := net.checkTopicRegister(regdata) - if err != nil { - //fmt.Println(err) - return n.state, fmt.Errorf("bad waiting ticket: %v", err) - } - net.topictab.useTicket(n, pong.TicketSerial, regdata.Topics, int(regdata.Idx), pong.Expiration, pong.WaitPeriods) - return n.state, nil - case topicQueryPacket: - // TODO: handle expiration - topic := pkt.data.(*topicQuery).Topic - results := net.topictab.getEntries(topic) - if _, ok := net.ticketStore.tickets[topic]; ok { - results = append(results, net.tab.self) // we're not registering in our own table but if we're advertising, return ourselves too - } - if len(results) > 10 { - results = results[:10] - } - var hash common.Hash - copy(hash[:], pkt.hash) - net.conn.sendTopicNodes(n, hash, results) - return n.state, nil - case topicNodesPacket: - p := pkt.data.(*topicNodes) - if net.ticketStore.gotTopicNodes(n, p.Echo, p.Nodes) { - n.queryTimeouts++ - if n.queryTimeouts > maxFindnodeFailures && n.state == known { - return contested, errors.New("too many timeouts") - } - } - return n.state, nil - - default: - return n.state, errInvalidEvent - } -} - -func (net *Network) checkTopicRegister(data *topicRegister) (*pong, error) { - var pongpkt ingressPacket - if err := decodePacket(data.Pong, &pongpkt); err != nil { - return nil, err - } - if pongpkt.ev != pongPacket { - return nil, errors.New("is not pong packet") - } - if pongpkt.remoteID != net.tab.self.ID { - return nil, errors.New("not signed by us") - } - // check that we previously authorised all topics - // that the other side is trying to register. - if rlpHash(data.Topics) != pongpkt.data.(*pong).TopicHash { - return nil, errors.New("topic hash mismatch") - } - if data.Idx >= uint(len(data.Topics)) { - return nil, errors.New("topic index out of range") - } - return pongpkt.data.(*pong), nil -} - -func rlpHash(x interface{}) (h common.Hash) { - hw := sha3.NewLegacyKeccak256() - rlp.Encode(hw, x) - hw.Sum(h[:0]) - return h -} - -func (net *Network) handleNeighboursPacket(n *Node, pkt *ingressPacket) error { - if n.pendingNeighbours == nil { - return errNoQuery - } - net.abortTimedEvent(n, neighboursTimeout) - - req := pkt.data.(*neighbors) - nodes := make([]*Node, len(req.Nodes)) - for i, rn := range req.Nodes { - nn, err := net.internNodeFromNeighbours(pkt.remoteAddr, rn) - if err != nil { - log.Debug(fmt.Sprintf("invalid neighbour (%v) from %x@%v: %v", rn.IP, n.ID[:8], pkt.remoteAddr, err)) - continue - } - nodes[i] = nn - // Start validation of query results immediately. - // This fills the table quickly. - // TODO: generates way too many packets, maybe do it via queue. - if nn.state == unknown { - net.transition(nn, verifyinit) - } - } - // TODO: don't ignore second packet - n.pendingNeighbours.reply <- nodes - n.pendingNeighbours = nil - // Now that this query is done, start the next one. - n.startNextQuery(net) - return nil -} diff --git a/p2p/discv5/net_test.go b/p2p/discv5/net_test.go deleted file mode 100644 index 29321bc86f90..000000000000 --- a/p2p/discv5/net_test.go +++ /dev/null @@ -1,330 +0,0 @@ -// Copyright 2016 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -package discv5 - -import ( - "net" - "testing" - "time" - - "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/crypto" -) - -func TestNetwork_Lookup(t *testing.T) { - key, _ := crypto.GenerateKey() - network, err := newNetwork(lookupTestnet, key.PublicKey, "", nil) - if err != nil { - t.Fatal(err) - } - lookupTestnet.net = network - defer network.Close() - - // lookup on empty table returns no nodes - // if results := network.Lookup(lookupTestnet.target, false); len(results) > 0 { - // t.Fatalf("lookup on empty table returned %d results: %#v", len(results), results) - // } - // seed table with initial node (otherwise lookup will terminate immediately) - seeds := []*Node{NewNode(lookupTestnet.dists[256][0], net.IP{10, 0, 2, 99}, lowPort+256, 999)} - if err := network.SetFallbackNodes(seeds); err != nil { - t.Fatal(err) - } - time.Sleep(3 * time.Second) - - results := network.Lookup(lookupTestnet.target) - t.Logf("results:") - for _, e := range results { - t.Logf(" ld=%d, %x", logdist(lookupTestnet.targetSha, e.sha), e.sha[:]) - } - if len(results) != bucketSize { - t.Errorf("wrong number of results: got %d, want %d", len(results), bucketSize) - } - if hasDuplicates(results) { - t.Errorf("result set contains duplicate entries") - } - if !sortedByDistanceTo(lookupTestnet.targetSha, results) { - t.Errorf("result set not sorted by distance to target") - } - // TODO: check result nodes are actually closest -} - -// This is the test network for the Lookup test. -// The nodes were obtained by running testnet.mine with a random NodeID as target. -var lookupTestnet = &preminedTestnet{ - target: MustHexID("166aea4f556532c6d34e8b740e5d314af7e9ac0ca79833bd751d6b665f12dfd38ec563c363b32f02aef4a80b44fd3def94612d497b99cb5f17fd24de454927ec"), - targetSha: common.Hash{0x5c, 0x94, 0x4e, 0xe5, 0x1c, 0x5a, 0xe9, 0xf7, 0x2a, 0x95, 0xec, 0xcb, 0x8a, 0xed, 0x3, 0x74, 0xee, 0xcb, 0x51, 0x19, 0xd7, 0x20, 0xcb, 0xea, 0x68, 0x13, 0xe8, 0xe0, 0xd6, 0xad, 0x92, 0x61}, - dists: [257][]NodeID{ - 240: { - MustHexID("2001ad5e3e80c71b952161bc0186731cf5ffe942d24a79230a0555802296238e57ea7a32f5b6f18564eadc1c65389448481f8c9338df0a3dbd18f708cbc2cbcb"), - MustHexID("6ba3f4f57d084b6bf94cc4555b8c657e4a8ac7b7baf23c6874efc21dd1e4f56b7eb2721e07f5242d2f1d8381fc8cae535e860197c69236798ba1ad231b105794"), - }, - 244: { - MustHexID("696ba1f0a9d55c59246f776600542a9e6432490f0cd78f8bb55a196918df2081a9b521c3c3ba48e465a75c10768807717f8f689b0b4adce00e1c75737552a178"), - }, - 246: { - MustHexID("d6d32178bdc38416f46ffb8b3ec9e4cb2cfff8d04dd7e4311a70e403cb62b10be1b447311b60b4f9ee221a8131fc2cbd45b96dd80deba68a949d467241facfa8"), - MustHexID("3ea3d04a43a3dfb5ac11cffc2319248cf41b6279659393c2f55b8a0a5fc9d12581a9d97ef5d8ff9b5abf3321a290e8f63a4f785f450dc8a672aba3ba2ff4fdab"), - MustHexID("2fc897f05ae585553e5c014effd3078f84f37f9333afacffb109f00ca8e7a3373de810a3946be971cbccdfd40249f9fe7f322118ea459ac71acca85a1ef8b7f4"), - }, - 247: { - MustHexID("3155e1427f85f10a5c9a7755877748041af1bcd8d474ec065eb33df57a97babf54bfd2103575fa829115d224c523596b401065a97f74010610fce76382c0bf32"), - MustHexID("312c55512422cf9b8a4097e9a6ad79402e87a15ae909a4bfefa22398f03d20951933beea1e4dfa6f968212385e829f04c2d314fc2d4e255e0d3bc08792b069db"), - MustHexID("38643200b172dcfef857492156971f0e6aa2c538d8b74010f8e140811d53b98c765dd2d96126051913f44582e8c199ad7c6d6819e9a56483f637feaac9448aac"), - MustHexID("8dcab8618c3253b558d459da53bd8fa68935a719aff8b811197101a4b2b47dd2d47295286fc00cc081bb542d760717d1bdd6bec2c37cd72eca367d6dd3b9df73"), - MustHexID("8b58c6073dd98bbad4e310b97186c8f822d3a5c7d57af40e2136e88e315afd115edb27d2d0685a908cfe5aa49d0debdda6e6e63972691d6bd8c5af2d771dd2a9"), - MustHexID("2cbb718b7dc682da19652e7d9eb4fefaf7b7147d82c1c2b6805edf77b85e29fde9f6da195741467ff2638dc62c8d3e014ea5686693c15ed0080b6de90354c137"), - MustHexID("e84027696d3f12f2de30a9311afea8fbd313c2360daff52bb5fc8c7094d5295758bec3134e4eef24e4cdf377b40da344993284628a7a346eba94f74160998feb"), - MustHexID("f1357a4f04f9d33753a57c0b65ba20a5d8777abbffd04e906014491c9103fb08590e45548d37aa4bd70965e2e81ddba94f31860348df01469eec8c1829200a68"), - MustHexID("4ab0a75941b12892369b4490a1928c8ca52a9ad6d3dffbd1d8c0b907bc200fe74c022d011ec39b64808a39c0ca41f1d3254386c3e7733e7044c44259486461b6"), - MustHexID("d45150a72dc74388773e68e03133a3b5f51447fe91837d566706b3c035ee4b56f160c878c6273394daee7f56cc398985269052f22f75a8057df2fe6172765354"), - }, - 248: { - MustHexID("6aadfce366a189bab08ac84721567483202c86590642ea6d6a14f37ca78d82bdb6509eb7b8b2f6f63c78ae3ae1d8837c89509e41497d719b23ad53dd81574afa"), - MustHexID("a605ecfd6069a4cf4cf7f5840e5bc0ce10d23a3ac59e2aaa70c6afd5637359d2519b4524f56fc2ca180cdbebe54262f720ccaae8c1b28fd553c485675831624d"), - MustHexID("29701451cb9448ca33fc33680b44b840d815be90146eb521641efbffed0859c154e8892d3906eae9934bfacee72cd1d2fa9dd050fd18888eea49da155ab0efd2"), - MustHexID("3ed426322dee7572b08592e1e079f8b6c6b30e10e6243edd144a6a48fdbdb83df73a6e41b1143722cb82604f2203a32758610b5d9544f44a1a7921ba001528c1"), - MustHexID("b2e2a2b7fdd363572a3256e75435fab1da3b16f7891a8bd2015f30995dae665d7eabfd194d87d99d5df628b4bbc7b04e5b492c596422dd8272746c7a1b0b8e4f"), - MustHexID("0c69c9756162c593e85615b814ce57a2a8ca2df6c690b9c4e4602731b61e1531a3bbe3f7114271554427ffabea80ad8f36fa95a49fa77b675ae182c6ccac1728"), - MustHexID("8d28be21d5a97b0876442fa4f5e5387f5bf3faad0b6f13b8607b64d6e448c0991ca28dd7fe2f64eb8eadd7150bff5d5666aa6ed868b84c71311f4ba9a38569dd"), - MustHexID("2c677e1c64b9c9df6359348a7f5f33dc79e22f0177042486d125f8b6ca7f0dc756b1f672aceee5f1746bcff80aaf6f92a8dc0c9fbeb259b3fa0da060de5ab7e8"), - MustHexID("3994880f94a8678f0cd247a43f474a8af375d2a072128da1ad6cae84a244105ff85e94fc7d8496f639468de7ee998908a91c7e33ef7585fff92e984b210941a1"), - MustHexID("b45a9153c08d002a48090d15d61a7c7dad8c2af85d4ff5bd36ce23a9a11e0709bf8d56614c7b193bc028c16cbf7f20dfbcc751328b64a924995d47b41e452422"), - MustHexID("057ab3a9e53c7a84b0f3fc586117a525cdd18e313f52a67bf31798d48078e325abe5cfee3f6c2533230cb37d0549289d692a29dd400e899b8552d4b928f6f907"), - MustHexID("0ddf663d308791eb92e6bd88a2f8cb45e4f4f35bb16708a0e6ff7f1362aa6a73fedd0a1b1557fb3365e38e1b79d6918e2fae2788728b70c9ab6b51a3b94a4338"), - MustHexID("f637e07ff50cc1e3731735841c4798411059f2023abcf3885674f3e8032531b0edca50fd715df6feb489b6177c345374d64f4b07d257a7745de393a107b013a5"), - MustHexID("e24ec7c6eec094f63c7b3239f56d311ec5a3e45bc4e622a1095a65b95eea6fe13e29f3b6b7a2cbfe40906e3989f17ac834c3102dd0cadaaa26e16ee06d782b72"), - MustHexID("b76ea1a6fd6506ef6e3506a4f1f60ed6287fff8114af6141b2ff13e61242331b54082b023cfea5b3083354a4fb3f9eb8be01fb4a518f579e731a5d0707291a6b"), - MustHexID("9b53a37950ca8890ee349b325032d7b672cab7eced178d3060137b24ef6b92a43977922d5bdfb4a3409a2d80128e02f795f9dae6d7d99973ad0e23a2afb8442f"), - }, - 249: { - MustHexID("675ae65567c3c72c50c73bc0fd4f61f202ea5f93346ca57b551de3411ccc614fad61cb9035493af47615311b9d44ee7a161972ee4d77c28fe1ec029d01434e6a"), - MustHexID("8eb81408389da88536ae5800392b16ef5109d7ea132c18e9a82928047ecdb502693f6e4a4cdd18b54296caf561db937185731456c456c98bfe7de0baf0eaa495"), - MustHexID("2adba8b1612a541771cb93a726a38a4b88e97b18eced2593eb7daf82f05a5321ca94a72cc780c306ff21e551a932fc2c6d791e4681907b5ceab7f084c3fa2944"), - MustHexID("b1b4bfbda514d9b8f35b1c28961da5d5216fe50548f4066f69af3b7666a3b2e06eac646735e963e5c8f8138a2fb95af15b13b23ff00c6986eccc0efaa8ee6fb4"), - MustHexID("d2139281b289ad0e4d7b4243c4364f5c51aac8b60f4806135de06b12b5b369c9e43a6eb494eab860d115c15c6fbb8c5a1b0e382972e0e460af395b8385363de7"), - MustHexID("4a693df4b8fc5bdc7cec342c3ed2e228d7c5b4ab7321ddaa6cccbeb45b05a9f1d95766b4002e6d4791c2deacb8a667aadea6a700da28a3eea810a30395701bbc"), - MustHexID("ab41611195ec3c62bb8cd762ee19fb182d194fd141f4a66780efbef4b07ce916246c022b841237a3a6b512a93431157edd221e854ed2a259b72e9c5351f44d0c"), - MustHexID("68e8e26099030d10c3c703ae7045c0a48061fb88058d853b3e67880014c449d4311014da99d617d3150a20f1a3da5e34bf0f14f1c51fe4dd9d58afd222823176"), - MustHexID("3fbcacf546fb129cd70fc48de3b593ba99d3c473798bc309292aca280320e0eacc04442c914cad5c4cf6950345ba79b0d51302df88285d4e83ee3fe41339eee7"), - MustHexID("1d4a623659f7c8f80b6c3939596afdf42e78f892f682c768ad36eb7bfba402dbf97aea3a268f3badd8fe7636be216edf3d67ee1e08789ebbc7be625056bd7109"), - MustHexID("a283c474ab09da02bbc96b16317241d0627646fcc427d1fe790b76a7bf1989ced90f92101a973047ae9940c92720dffbac8eff21df8cae468a50f72f9e159417"), - MustHexID("dbf7e5ad7f87c3dfecae65d87c3039e14ed0bdc56caf00ce81931073e2e16719d746295512ff7937a15c3b03603e7c41a4f9df94fcd37bb200dd8f332767e9cb"), - MustHexID("caaa070a26692f64fc77f30d7b5ae980d419b4393a0f442b1c821ef58c0862898b0d22f74a4f8c5d83069493e3ec0b92f17dc1fe6e4cd437c1ec25039e7ce839"), - MustHexID("874cc8d1213beb65c4e0e1de38ef5d8165235893ac74ab5ea937c885eaab25c8d79dad0456e9fd3e9450626cac7e107b004478fb59842f067857f39a47cee695"), - MustHexID("d94193f236105010972f5df1b7818b55846592a0445b9cdc4eaed811b8c4c0f7c27dc8cc9837a4774656d6b34682d6d329d42b6ebb55da1d475c2474dc3dfdf4"), - MustHexID("edd9af6aded4094e9785637c28fccbd3980cbe28e2eb9a411048a23c2ace4bd6b0b7088a7817997b49a3dd05fc6929ca6c7abbb69438dbdabe65e971d2a794b2"), - }, - 250: { - MustHexID("53a5bd1215d4ab709ae8fdc2ced50bba320bced78bd9c5dc92947fb402250c914891786db0978c898c058493f86fc68b1c5de8a5cb36336150ac7a88655b6c39"), - MustHexID("b7f79e3ab59f79262623c9ccefc8f01d682323aee56ffbe295437487e9d5acaf556a9c92e1f1c6a9601f2b9eb6b027ae1aeaebac71d61b9b78e88676efd3e1a3"), - MustHexID("d374bf7e8d7ffff69cc00bebff38ef5bc1dcb0a8d51c1a3d70e61ac6b2e2d6617109254b0ac224354dfbf79009fe4239e09020c483cc60c071e00b9238684f30"), - MustHexID("1e1eac1c9add703eb252eb991594f8f5a173255d526a855fab24ae57dc277e055bc3c7a7ae0b45d437c4f47a72d97eb7b126f2ba344ba6c0e14b2c6f27d4b1e6"), - MustHexID("ae28953f63d4bc4e706712a59319c111f5ff8f312584f65d7436b4cd3d14b217b958f8486bad666b4481fe879019fb1f767cf15b3e3e2711efc33b56d460448a"), - MustHexID("934bb1edf9c7a318b82306aca67feb3d6b434421fa275d694f0b4927afd8b1d3935b727fd4ff6e3d012e0c82f1824385174e8c6450ade59c2a43281a4b3446b6"), - MustHexID("9eef3f28f70ce19637519a0916555bf76d26de31312ac656cf9d3e379899ea44e4dd7ffcce923b4f3563f8a00489a34bd6936db0cbb4c959d32c49f017e07d05"), - MustHexID("82200872e8f871c48f1fad13daec6478298099b591bb3dbc4ef6890aa28ebee5860d07d70be62f4c0af85085a90ae8179ee8f937cf37915c67ea73e704b03ee7"), - MustHexID("6c75a5834a08476b7fc37ff3dc2011dc3ea3b36524bad7a6d319b18878fad813c0ba76d1f4555cacd3890c865438c21f0e0aed1f80e0a157e642124c69f43a11"), - MustHexID("995b873742206cb02b736e73a88580c2aacb0bd4a3c97a647b647bcab3f5e03c0e0736520a8b3600da09edf4248991fb01091ec7ff3ec7cdc8a1beae011e7aae"), - MustHexID("c773a056594b5cdef2e850d30891ff0e927c3b1b9c35cd8e8d53a1017001e237468e1ece3ae33d612ca3e6abb0a9169aa352e9dcda358e5af2ad982b577447db"), - MustHexID("2b46a5f6923f475c6be99ec6d134437a6d11f6bb4b4ac6bcd94572fa1092639d1c08aeefcb51f0912f0a060f71d4f38ee4da70ecc16010b05dd4a674aab14c3a"), - MustHexID("af6ab501366debbaa0d22e20e9688f32ef6b3b644440580fd78de4fe0e99e2a16eb5636bbae0d1c259df8ddda77b35b9a35cbc36137473e9c68fbc9d203ba842"), - MustHexID("c9f6f2dd1a941926f03f770695bda289859e85fabaf94baaae20b93e5015dc014ba41150176a36a1884adb52f405194693e63b0c464a6891cc9cc1c80d450326"), - MustHexID("5b116f0751526868a909b61a30b0c5282c37df6925cc03ddea556ef0d0602a9595fd6c14d371f8ed7d45d89918a032dcd22be4342a8793d88fdbeb3ca3d75bd7"), - MustHexID("50f3222fb6b82481c7c813b2172e1daea43e2710a443b9c2a57a12bd160dd37e20f87aa968c82ad639af6972185609d47036c0d93b4b7269b74ebd7073221c10"), - }, - 251: { - MustHexID("9b8f702a62d1bee67bedfeb102eca7f37fa1713e310f0d6651cc0c33ea7c5477575289ccd463e5a2574a00a676a1fdce05658ba447bb9d2827f0ba47b947e894"), - MustHexID("b97532eb83054ed054b4abdf413bb30c00e4205545c93521554dbe77faa3cfaa5bd31ef466a107b0b34a71ec97214c0c83919720142cddac93aa7a3e928d4708"), - MustHexID("2f7a5e952bfb67f2f90b8441b5fadc9ee13b1dcde3afeeb3dd64bf937f86663cc5c55d1fa83952b5422763c7df1b7f2794b751c6be316ebc0beb4942e65ab8c1"), - MustHexID("42c7483781727051a0b3660f14faf39e0d33de5e643702ae933837d036508ab856ce7eec8ec89c4929a4901256e5233a3d847d5d4893f91bcf21835a9a880fee"), - MustHexID("873bae27bf1dc854408fba94046a53ab0c965cebe1e4e12290806fc62b88deb1f4a47f9e18f78fc0e7913a0c6e42ac4d0fc3a20cea6bc65f0c8a0ca90b67521e"), - MustHexID("a7e3a370bbd761d413f8d209e85886f68bf73d5c3089b2dc6fa42aab1ecb5162635497eed95dee2417f3c9c74a3e76319625c48ead2e963c7de877cd4551f347"), - MustHexID("528597534776a40df2addaaea15b6ff832ce36b9748a265768368f657e76d58569d9f30dbb91e91cf0ae7efe8f402f17aa0ae15f5c55051ba03ba830287f4c42"), - MustHexID("461d8bd4f13c3c09031fdb84f104ed737a52f630261463ce0bdb5704259bab4b737dda688285b8444dbecaecad7f50f835190b38684ced5e90c54219e5adf1bc"), - MustHexID("6ec50c0be3fd232737090fc0111caaf0bb6b18f72be453428087a11a97fd6b52db0344acbf789a689bd4f5f50f79017ea784f8fd6fe723ad6ae675b9e3b13e21"), - MustHexID("12fc5e2f77a83fdcc727b79d8ae7fe6a516881138d3011847ee136b400fed7cfba1f53fd7a9730253c7aa4f39abeacd04f138417ba7fcb0f36cccc3514e0dab6"), - MustHexID("4fdbe75914ccd0bce02101606a1ccf3657ec963e3b3c20239d5fec87673fe446d649b4f15f1fe1a40e6cfbd446dda2d31d40bb602b1093b8fcd5f139ba0eb46a"), - MustHexID("3753668a0f6281e425ea69b52cb2d17ab97afbe6eb84cf5d25425bc5e53009388857640668fadd7c110721e6047c9697803bd8a6487b43bb343bfa32ebf24039"), - MustHexID("2e81b16346637dec4410fd88e527346145b9c0a849dbf2628049ac7dae016c8f4305649d5659ec77f1e8a0fac0db457b6080547226f06283598e3740ad94849a"), - MustHexID("802c3cc27f91c89213223d758f8d2ecd41135b357b6d698f24d811cdf113033a81c38e0bdff574a5c005b00a8c193dc2531f8c1fa05fa60acf0ab6f2858af09f"), - MustHexID("fcc9a2e1ac3667026ff16192876d1813bb75abdbf39b929a92863012fe8b1d890badea7a0de36274d5c1eb1e8f975785532c50d80fd44b1a4b692f437303393f"), - MustHexID("6d8b3efb461151dd4f6de809b62726f5b89e9b38e9ba1391967f61cde844f7528fecf821b74049207cee5a527096b31f3ad623928cd3ce51d926fa345a6b2951"), - }, - 252: { - MustHexID("f1ae93157cc48c2075dd5868fbf523e79e06caf4b8198f352f6e526680b78ff4227263de92612f7d63472bd09367bb92a636fff16fe46ccf41614f7a72495c2a"), - MustHexID("587f482d111b239c27c0cb89b51dd5d574db8efd8de14a2e6a1400c54d4567e77c65f89c1da52841212080b91604104768350276b6682f2f961cdaf4039581c7"), - MustHexID("e3f88274d35cefdaabdf205afe0e80e936cc982b8e3e47a84ce664c413b29016a4fb4f3a3ebae0a2f79671f8323661ed462bf4390af94c424dc8ace0c301b90f"), - MustHexID("0ddc736077da9a12ba410dc5ea63cbcbe7659dd08596485b2bff3435221f82c10d263efd9af938e128464be64a178b7cd22e19f400d5802f4c9df54bf89f2619"), - MustHexID("784aa34d833c6ce63fcc1279630113c3272e82c4ae8c126c5a52a88ac461b6baeed4244e607b05dc14e5b2f41c70a273c3804dea237f14f7a1e546f6d1309d14"), - MustHexID("f253a2c354ee0e27cfcae786d726753d4ad24be6516b279a936195a487de4a59dbc296accf20463749ff55293263ed8c1b6365eecb248d44e75e9741c0d18205"), - MustHexID("a1910b80357b3ad9b4593e0628922939614dc9056a5fbf477279c8b2c1d0b4b31d89a0c09d0d41f795271d14d3360ef08a3f821e65e7e1f56c07a36afe49c7c5"), - MustHexID("f1168552c2efe541160f0909b0b4a9d6aeedcf595cdf0e9b165c97e3e197471a1ee6320e93389edfba28af6eaf10de98597ad56e7ab1b504ed762451996c3b98"), - MustHexID("b0c8e5d2c8634a7930e1a6fd082e448c6cf9d2d8b7293558b59238815a4df926c286bf297d2049f14e8296a6eb3256af614ec1812c4f2bbe807673b58bf14c8c"), - MustHexID("0fb346076396a38badc342df3679b55bd7f40a609ab103411fe45082c01f12ea016729e95914b2b5540e987ff5c9b133e85862648e7f36abdfd23100d248d234"), - MustHexID("f736e0cc83417feaa280d9483f5d4d72d1b036cd0c6d9cbdeb8ac35ceb2604780de46dddaa32a378474e1d5ccdf79b373331c30c7911ade2ae32f98832e5de1f"), - MustHexID("8b02991457602f42b38b342d3f2259ae4100c354b3843885f7e4e07bd644f64dab94bb7f38a3915f8b7f11d8e3f81c28e07a0078cf79d7397e38a7b7e0c857e2"), - MustHexID("9221d9f04a8a184993d12baa91116692bb685f887671302999d69300ad103eb2d2c75a09d8979404c6dd28f12362f58a1a43619c493d9108fd47588a23ce5824"), - MustHexID("652797801744dada833fff207d67484742eea6835d695925f3e618d71b68ec3c65bdd85b4302b2cdcb835ad3f94fd00d8da07e570b41bc0d2bcf69a8de1b3284"), - MustHexID("d84f06fe64debc4cd0625e36d19b99014b6218375262cc2209202bdbafd7dffcc4e34ce6398e182e02fd8faeed622c3e175545864902dfd3d1ac57647cddf4c6"), - MustHexID("d0ed87b294f38f1d741eb601020eeec30ac16331d05880fe27868f1e454446de367d7457b41c79e202eaf9525b029e4f1d7e17d85a55f83a557c005c68d7328a"), - }, - 253: { - MustHexID("ad4485e386e3cc7c7310366a7c38fb810b8896c0d52e55944bfd320ca294e7912d6c53c0a0cf85e7ce226e92491d60430e86f8f15cda0161ed71893fb4a9e3a1"), - MustHexID("36d0e7e5b7734f98c6183eeeb8ac5130a85e910a925311a19c4941b1290f945d4fc3996b12ef4966960b6fa0fb29b1604f83a0f81bd5fd6398d2e1a22e46af0c"), - MustHexID("7d307d8acb4a561afa23bdf0bd945d35c90245e26345ec3a1f9f7df354222a7cdcb81339c9ed6744526c27a1a0c8d10857e98df942fa433602facac71ac68a31"), - MustHexID("d97bf55f88c83fae36232661af115d66ca600fc4bd6d1fb35ff9bb4dad674c02cf8c8d05f317525b5522250db58bb1ecafb7157392bf5aa61b178c61f098d995"), - MustHexID("7045d678f1f9eb7a4613764d17bd5698796494d0bf977b16f2dbc272b8a0f7858a60805c022fc3d1fe4f31c37e63cdaca0416c0d053ef48a815f8b19121605e0"), - MustHexID("14e1f21418d445748de2a95cd9a8c3b15b506f86a0acabd8af44bb968ce39885b19c8822af61b3dd58a34d1f265baec30e3ae56149dc7d2aa4a538f7319f69c8"), - MustHexID("b9453d78281b66a4eac95a1546017111eaaa5f92a65d0de10b1122940e92b319728a24edf4dec6acc412321b1c95266d39c7b3a5d265c629c3e49a65fb022c09"), - MustHexID("e8a49248419e3824a00d86af422f22f7366e2d4922b304b7169937616a01d9d6fa5abf5cc01061a352dc866f48e1fa2240dbb453d872b1d7be62bdfc1d5e248c"), - MustHexID("bebcff24b52362f30e0589ee573ce2d86f073d58d18e6852a592fa86ceb1a6c9b96d7fb9ec7ed1ed98a51b6743039e780279f6bb49d0a04327ac7a182d9a56f6"), - MustHexID("d0835e5a4291db249b8d2fca9f503049988180c7d247bedaa2cf3a1bad0a76709360a85d4f9a1423b2cbc82bb4d94b47c0cde20afc430224834c49fe312a9ae3"), - MustHexID("6b087fe2a2da5e4f0b0f4777598a4a7fb66bf77dbd5bfc44e8a7eaa432ab585a6e226891f56a7d4f5ed11a7c57b90f1661bba1059590ca4267a35801c2802913"), - MustHexID("d901e5bde52d1a0f4ddf010a686a53974cdae4ebe5c6551b3c37d6b6d635d38d5b0e5f80bc0186a2c7809dbf3a42870dd09643e68d32db896c6da8ba734579e7"), - MustHexID("96419fb80efae4b674402bb969ebaab86c1274f29a83a311e24516d36cdf148fe21754d46c97688cdd7468f24c08b13e4727c29263393638a3b37b99ff60ebca"), - MustHexID("7b9c1889ae916a5d5abcdfb0aaedcc9c6f9eb1c1a4f68d0c2d034fe79ac610ce917c3abc670744150fa891bfcd8ab14fed6983fca964de920aa393fa7b326748"), - MustHexID("7a369b2b8962cc4c65900be046482fbf7c14f98a135bbbae25152c82ad168fb2097b3d1429197cf46d3ce9fdeb64808f908a489cc6019725db040060fdfe5405"), - MustHexID("47bcae48288da5ecc7f5058dfa07cf14d89d06d6e449cb946e237aa6652ea050d9f5a24a65efdc0013ccf232bf88670979eddef249b054f63f38da9d7796dbd8"), - }, - 254: { - MustHexID("099739d7abc8abd38ecc7a816c521a1168a4dbd359fa7212a5123ab583ffa1cf485a5fed219575d6475dbcdd541638b2d3631a6c7fce7474e7fe3cba1d4d5853"), - MustHexID("c2b01603b088a7182d0cf7ef29fb2b04c70acb320fccf78526bf9472e10c74ee70b3fcfa6f4b11d167bd7d3bc4d936b660f2c9bff934793d97cb21750e7c3d31"), - MustHexID("20e4d8f45f2f863e94b45548c1ef22a11f7d36f263e4f8623761e05a64c4572379b000a52211751e2561b0f14f4fc92dd4130410c8ccc71eb4f0e95a700d4ca9"), - MustHexID("27f4a16cc085e72d86e25c98bd2eca173eaaee7565c78ec5a52e9e12b2211f35de81b5b45e9195de2ebfe29106742c59112b951a04eb7ae48822911fc1f9389e"), - MustHexID("55db5ee7d98e7f0b1c3b9d5be6f2bc619a1b86c3cdd513160ad4dcf267037a5fffad527ac15d50aeb32c59c13d1d4c1e567ebbf4de0d25236130c8361f9aac63"), - MustHexID("883df308b0130fc928a8559fe50667a0fff80493bc09685d18213b2db241a3ad11310ed86b0ef662b3ce21fc3d9aa7f3fc24b8d9afe17c7407e9afd3345ae548"), - MustHexID("c7af968cc9bc8200c3ee1a387405f7563be1dce6710a3439f42ea40657d0eae9d2b3c16c42d779605351fcdece4da637b9804e60ca08cfb89aec32c197beffa6"), - MustHexID("3e66f2b788e3ff1d04106b80597915cd7afa06c405a7ae026556b6e583dca8e05cfbab5039bb9a1b5d06083ffe8de5780b1775550e7218f5e98624bf7af9a0a8"), - MustHexID("4fc7f53764de3337fdaec0a711d35d3a923e72fa65025444d12230b3552ed43d9b2d1ad08ccb11f2d50c58809e6dd74dde910e195294fca3b47ae5a3967cc479"), - MustHexID("bafdfdcf6ccaa989436752fa97c77477b6baa7deb374b16c095492c529eb133e8e2f99e1977012b64767b9d34b2cf6d2048ed489bd822b5139b523f6a423167b"), - MustHexID("7f5d78008a4312fe059104ce80202c82b8915c2eb4411c6b812b16f7642e57c00f2c9425121f5cbac4257fe0b3e81ef5dea97ea2dbaa98f6a8b6fd4d1e5980bb"), - MustHexID("598c37fe78f922751a052f463aeb0cb0bc7f52b7c2a4cf2da72ec0931c7c32175d4165d0f8998f7320e87324ac3311c03f9382a5385c55f0407b7a66b2acd864"), - MustHexID("f758c4136e1c148777a7f3275a76e2db0b2b04066fd738554ec398c1c6cc9fb47e14a3b4c87bd47deaeab3ffd2110514c3855685a374794daff87b605b27ee2e"), - MustHexID("0307bb9e4fd865a49dcf1fe4333d1b944547db650ab580af0b33e53c4fef6c789531110fac801bbcbce21fc4d6f61b6d5b24abdf5b22e3030646d579f6dca9c2"), - MustHexID("82504b6eb49bb2c0f91a7006ce9cefdbaf6df38706198502c2e06601091fc9dc91e4f15db3410d45c6af355bc270b0f268d3dff560f956985c7332d4b10bd1ed"), - MustHexID("b39b5b677b45944ceebe76e76d1f051de2f2a0ec7b0d650da52135743e66a9a5dba45f638258f9a7545d9a790c7fe6d3fdf82c25425c7887323e45d27d06c057"), - }, - 255: { - MustHexID("5c4d58d46e055dd1f093f81ee60a675e1f02f54da6206720adee4dccef9b67a31efc5c2a2949c31a04ee31beadc79aba10da31440a1f9ff2a24093c63c36d784"), - MustHexID("ea72161ffdd4b1e124c7b93b0684805f4c4b58d617ed498b37a145c670dbc2e04976f8785583d9c805ffbf343c31d492d79f841652bbbd01b61ed85640b23495"), - MustHexID("51caa1d93352d47a8e531692a3612adac1e8ac68d0a200d086c1c57ae1e1a91aa285ab242e8c52ef9d7afe374c9485b122ae815f1707b875569d0433c1c3ce85"), - MustHexID("c08397d5751b47bd3da044b908be0fb0e510d3149574dff7aeab33749b023bb171b5769990fe17469dbebc100bc150e798aeda426a2dcc766699a225fddd75c6"), - MustHexID("0222c1c194b749736e593f937fad67ee348ac57287a15c7e42877aa38a9b87732a408bca370f812efd0eedbff13e6d5b854bf3ba1dec431a796ed47f32552b09"), - MustHexID("03d859cd46ef02d9bfad5268461a6955426845eef4126de6be0fa4e8d7e0727ba2385b78f1a883a8239e95ebb814f2af8379632c7d5b100688eebc5841209582"), - MustHexID("64d5004b7e043c39ff0bd10cb20094c287721d5251715884c280a612b494b3e9e1c64ba6f67614994c7d969a0d0c0295d107d53fc225d47c44c4b82852d6f960"), - MustHexID("b0a5eefb2dab6f786670f35bf9641eefe6dd87fd3f1362bcab4aaa792903500ab23d88fae68411372e0813b057535a601d46e454323745a948017f6063a47b1f"), - MustHexID("0cc6df0a3433d448b5684d2a3ffa9d1a825388177a18f44ad0008c7bd7702f1ec0fc38b83506f7de689c3b6ecb552599927e29699eed6bb867ff08f80068b287"), - MustHexID("50772f7b8c03a4e153355fbbf79c8a80cf32af656ff0c7873c99911099d04a0dae0674706c357e0145ad017a0ade65e6052cb1b0d574fcd6f67da3eee0ace66b"), - MustHexID("1ae37829c9ef41f8b508b82259ebac76b1ed900d7a45c08b7970f25d2d48ddd1829e2f11423a18749940b6dab8598c6e416cef0efd47e46e51f29a0bc65b37cd"), - MustHexID("ba973cab31c2af091fc1644a93527d62b2394999e2b6ccbf158dd5ab9796a43d408786f1803ef4e29debfeb62fce2b6caa5ab2b24d1549c822a11c40c2856665"), - MustHexID("bc413ad270dd6ea25bddba78f3298b03b8ba6f8608ac03d06007d4116fa78ef5a0cfe8c80155089382fc7a193243ee5500082660cb5d7793f60f2d7d18650964"), - MustHexID("5a6a9ef07634d9eec3baa87c997b529b92652afa11473dfee41ef7037d5c06e0ddb9fe842364462d79dd31cff8a59a1b8d5bc2b810dea1d4cbbd3beb80ecec83"), - MustHexID("f492c6ee2696d5f682f7f537757e52744c2ae560f1090a07024609e903d334e9e174fc01609c5a229ddbcac36c9d21adaf6457dab38a25bfd44f2f0ee4277998"), - MustHexID("459e4db99298cb0467a90acee6888b08bb857450deac11015cced5104853be5adce5b69c740968bc7f931495d671a70cad9f48546d7cd203357fe9af0e8d2164"), - }, - 256: { - MustHexID("a8593af8a4aef7b806b5197612017951bac8845a1917ca9a6a15dd6086d608505144990b245785c4cd2d67a295701c7aac2aa18823fb0033987284b019656268"), - MustHexID("d2eebef914928c3aad77fc1b2a495f52d2294acf5edaa7d8a530b540f094b861a68fe8348a46a7c302f08ab609d85912a4968eacfea0740847b29421b4795d9e"), - MustHexID("b14bfcb31495f32b650b63cf7d08492e3e29071fdc73cf2da0da48d4b191a70ba1a65f42ad8c343206101f00f8a48e8db4b08bf3f622c0853e7323b250835b91"), - MustHexID("7feaee0d818c03eb30e4e0bf03ade0f3c21ca38e938a761aa1781cf70bda8cc5cd631a6cc53dd44f1d4a6d3e2dae6513c6c66ee50cb2f0e9ad6f7e319b309fd9"), - MustHexID("4ca3b657b139311db8d583c25dd5963005e46689e1317620496cc64129c7f3e52870820e0ec7941d28809311df6db8a2867bbd4f235b4248af24d7a9c22d1232"), - MustHexID("1181defb1d16851d42dd951d84424d6bd1479137f587fa184d5a8152be6b6b16ed08bcdb2c2ed8539bcde98c80c432875f9f724737c316a2bd385a39d3cab1d8"), - MustHexID("d9dd818769fa0c3ec9f553c759b92476f082817252a04a47dc1777740b1731d280058c66f982812f173a294acf4944a85ba08346e2de153ba3ba41ce8a62cb64"), - MustHexID("bd7c4f8a9e770aa915c771b15e107ca123d838762da0d3ffc53aa6b53e9cd076cffc534ec4d2e4c334c683f1f5ea72e0e123f6c261915ed5b58ac1b59f003d88"), - MustHexID("3dd5739c73649d510456a70e9d6b46a855864a4a3f744e088fd8c8da11b18e4c9b5f2d7da50b1c147b2bae5ca9609ae01f7a3cdea9dce34f80a91d29cd82f918"), - MustHexID("f0d7df1efc439b4bcc0b762118c1cfa99b2a6143a9f4b10e3c9465125f4c9fca4ab88a2504169bbcad65492cf2f50da9dd5d077c39574a944f94d8246529066b"), - MustHexID("dd598b9ba441448e5fb1a6ec6c5f5aa9605bad6e223297c729b1705d11d05f6bfd3d41988b694681ae69bb03b9a08bff4beab5596503d12a39bffb5cd6e94c7c"), - MustHexID("3fce284ac97e567aebae681b15b7a2b6df9d873945536335883e4bbc26460c064370537f323fd1ada828ea43154992d14ac0cec0940a2bd2a3f42ec156d60c83"), - MustHexID("7c8dfa8c1311cb14fb29a8ac11bca23ecc115e56d9fcf7b7ac1db9066aa4eb39f8b1dabf46e192a65be95ebfb4e839b5ab4533fef414921825e996b210dd53bd"), - MustHexID("cafa6934f82120456620573d7f801390ed5e16ed619613a37e409e44ab355ef755e83565a913b48a9466db786f8d4fbd590bfec474c2524d4a2608d4eafd6abd"), - MustHexID("9d16600d0dd310d77045769fed2cb427f32db88cd57d86e49390c2ba8a9698cfa856f775be2013237226e7bf47b248871cf865d23015937d1edeb20db5e3e760"), - MustHexID("17be6b6ba54199b1d80eff866d348ea11d8a4b341d63ad9a6681d3ef8a43853ac564d153eb2a8737f0afc9ab320f6f95c55aa11aaa13bbb1ff422fd16bdf8188"), - }, - }, -} - -type preminedTestnet struct { - target NodeID - targetSha common.Hash // sha3(target) - dists [hashBits + 1][]NodeID - net *Network -} - -func (tn *preminedTestnet) sendFindnodeHash(to *Node, target common.Hash) { - // current log distance is encoded in port number - // fmt.Println("findnode query at dist", toaddr.Port) - if to.UDP <= lowPort { - panic("query to node at or below distance 0") - } - next := to.UDP - 1 - var result []rpcNode - for i, id := range tn.dists[to.UDP-lowPort] { - result = append(result, nodeToRPC(NewNode(id, net.ParseIP("10.0.2.99"), next, uint16(i)+1+lowPort))) - } - injectResponse(tn.net, to, neighborsPacket, &neighbors{Nodes: result}) -} - -func (tn *preminedTestnet) sendPing(to *Node, addr *net.UDPAddr, topics []Topic) []byte { - injectResponse(tn.net, to, pongPacket, &pong{ReplyTok: []byte{1}}) - return []byte{1} -} - -func (tn *preminedTestnet) send(to *Node, ptype nodeEvent, data interface{}) (hash []byte) { - switch ptype { - case pingPacket: - injectResponse(tn.net, to, pongPacket, &pong{ReplyTok: []byte{1}}) - case pongPacket: - // ignored - case findnodeHashPacket: - // current log distance is encoded in port number - // fmt.Println("findnode query at dist", toaddr.Port-lowPort) - if to.UDP <= lowPort { - panic("query to node at or below distance 0") - } - next := to.UDP - 1 - var result []rpcNode - for i, id := range tn.dists[to.UDP-lowPort] { - result = append(result, nodeToRPC(NewNode(id, net.ParseIP("10.0.2.99"), next, uint16(i)+1+lowPort))) - } - injectResponse(tn.net, to, neighborsPacket, &neighbors{Nodes: result}) - default: - panic("send(" + ptype.String() + ")") - } - return []byte{2} -} - -func (tn *preminedTestnet) sendNeighbours(to *Node, nodes []*Node) { - panic("sendNeighbours called") -} - -func (tn *preminedTestnet) sendTopicNodes(to *Node, queryHash common.Hash, nodes []*Node) { - panic("sendTopicNodes called") -} - -func (tn *preminedTestnet) sendTopicRegister(to *Node, topics []Topic, idx int, pong []byte) { - panic("sendTopicRegister called") -} - -func (*preminedTestnet) Close() {} - -func (*preminedTestnet) localAddr() *net.UDPAddr { - return &net.UDPAddr{IP: net.ParseIP("10.0.1.1"), Port: 40000} -} - -func injectResponse(net *Network, from *Node, ev nodeEvent, packet interface{}) { - go net.reqReadPacket(ingressPacket{remoteID: from.ID, remoteAddr: from.addr(), ev: ev, data: packet}) -} diff --git a/p2p/discv5/node.go b/p2p/discv5/node.go deleted file mode 100644 index 44d3025b70c4..000000000000 --- a/p2p/discv5/node.go +++ /dev/null @@ -1,413 +0,0 @@ -// Copyright 2015 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -package discv5 - -import ( - "crypto/ecdsa" - "crypto/elliptic" - "encoding/hex" - "errors" - "fmt" - "math/big" - "math/rand" - "net" - "net/url" - "regexp" - "strconv" - "strings" - - "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/crypto" -) - -// Node represents a host on the network. -// The public fields of Node may not be modified. -type Node struct { - IP net.IP // len 4 for IPv4 or 16 for IPv6 - UDP, TCP uint16 // port numbers - ID NodeID // the node's public key - - // Network-related fields are contained in nodeNetGuts. - // These fields are not supposed to be used off the - // Network.loop goroutine. - nodeNetGuts -} - -// NewNode creates a new node. It is mostly meant to be used for -// testing purposes. -func NewNode(id NodeID, ip net.IP, udpPort, tcpPort uint16) *Node { - if ipv4 := ip.To4(); ipv4 != nil { - ip = ipv4 - } - return &Node{ - IP: ip, - UDP: udpPort, - TCP: tcpPort, - ID: id, - nodeNetGuts: nodeNetGuts{sha: crypto.Keccak256Hash(id[:])}, - } -} - -func (n *Node) addr() *net.UDPAddr { - return &net.UDPAddr{IP: n.IP, Port: int(n.UDP)} -} - -// Incomplete returns true for nodes with no IP address. -func (n *Node) Incomplete() bool { - return n.IP == nil -} - -// checks whether n is a valid complete node. -func (n *Node) validateComplete() error { - if n.Incomplete() { - return errors.New("incomplete node") - } - if n.UDP == 0 { - return errors.New("missing UDP port") - } - if n.TCP == 0 { - return errors.New("missing TCP port") - } - if n.IP.IsMulticast() || n.IP.IsUnspecified() { - return errors.New("invalid IP (multicast/unspecified)") - } - _, err := n.ID.Pubkey() // validate the key (on curve, etc.) - return err -} - -// The string representation of a Node is a URL. -// Please see ParseNode for a description of the format. -func (n *Node) String() string { - u := url.URL{Scheme: "enode"} - if n.Incomplete() { - u.Host = fmt.Sprintf("%x", n.ID[:]) - } else { - addr := net.TCPAddr{IP: n.IP, Port: int(n.TCP)} - u.User = url.User(fmt.Sprintf("%x", n.ID[:])) - u.Host = addr.String() - if n.UDP != n.TCP { - u.RawQuery = "discport=" + strconv.Itoa(int(n.UDP)) - } - } - return u.String() -} - -var incompleteNodeURL = regexp.MustCompile("(?i)^(?:enode://)?([0-9a-f]+)$") - -// ParseNode parses a node designator. -// -// There are two basic forms of node designators -// - incomplete nodes, which only have the public key (node ID) -// - complete nodes, which contain the public key and IP/Port information -// -// For incomplete nodes, the designator must look like one of these -// -// enode:// -// -// -// For complete nodes, the node ID is encoded in the username portion -// of the URL, separated from the host by an @ sign. The hostname can -// only be given as an IP address, DNS domain names are not allowed. -// The port in the host name section is the TCP listening port. If the -// TCP and UDP (discovery) ports differ, the UDP port is specified as -// query parameter "discport". -// -// In the following example, the node URL describes -// a node with IP address 10.3.58.6, TCP listening port 30303 -// and UDP discovery port 30301. -// -// enode://@10.3.58.6:30303?discport=30301 -func ParseNode(rawurl string) (*Node, error) { - if m := incompleteNodeURL.FindStringSubmatch(rawurl); m != nil { - id, err := HexID(m[1]) - if err != nil { - return nil, fmt.Errorf("invalid node ID (%v)", err) - } - return NewNode(id, nil, 0, 0), nil - } - return parseComplete(rawurl) -} - -func parseComplete(rawurl string) (*Node, error) { - var ( - id NodeID - ip net.IP - tcpPort, udpPort uint64 - ) - u, err := url.Parse(rawurl) - if err != nil { - return nil, err - } - if u.Scheme != "enode" { - return nil, errors.New("invalid URL scheme, want \"enode\"") - } - // Parse the Node ID from the user portion. - if u.User == nil { - return nil, errors.New("does not contain node ID") - } - if id, err = HexID(u.User.String()); err != nil { - return nil, fmt.Errorf("invalid node ID (%v)", err) - } - // Parse the IP address. - host, port, err := net.SplitHostPort(u.Host) - if err != nil { - return nil, fmt.Errorf("invalid host: %v", err) - } - if ip = net.ParseIP(host); ip == nil { - return nil, errors.New("invalid IP address") - } - // Ensure the IP is 4 bytes long for IPv4 addresses. - if ipv4 := ip.To4(); ipv4 != nil { - ip = ipv4 - } - // Parse the port numbers. - if tcpPort, err = strconv.ParseUint(port, 10, 16); err != nil { - return nil, errors.New("invalid port") - } - udpPort = tcpPort - qv := u.Query() - if qv.Get("discport") != "" { - udpPort, err = strconv.ParseUint(qv.Get("discport"), 10, 16) - if err != nil { - return nil, errors.New("invalid discport in query") - } - } - return NewNode(id, ip, uint16(udpPort), uint16(tcpPort)), nil -} - -// MustParseNode parses a node URL. It panics if the URL is not valid. -func MustParseNode(rawurl string) *Node { - n, err := ParseNode(rawurl) - if err != nil { - panic("invalid node URL: " + err.Error()) - } - return n -} - -// MarshalText implements encoding.TextMarshaler. -func (n *Node) MarshalText() ([]byte, error) { - return []byte(n.String()), nil -} - -// UnmarshalText implements encoding.TextUnmarshaler. -func (n *Node) UnmarshalText(text []byte) error { - dec, err := ParseNode(string(text)) - if err == nil { - *n = *dec - } - return err -} - -// type nodeQueue []*Node -// -// // pushNew adds n to the end if it is not present. -// func (nl *nodeList) appendNew(n *Node) { -// for _, entry := range n { -// if entry == n { -// return -// } -// } -// *nq = append(*nq, n) -// } -// -// // popRandom removes a random node. Nodes closer to -// // to the head of the beginning of the have a slightly higher probability. -// func (nl *nodeList) popRandom() *Node { -// ix := rand.Intn(len(*nq)) -// //TODO: probability as mentioned above. -// nl.removeIndex(ix) -// } -// -// func (nl *nodeList) removeIndex(i int) *Node { -// slice = *nl -// if len(*slice) <= i { -// return nil -// } -// *nl = append(slice[:i], slice[i+1:]...) -// } - -const nodeIDBits = 512 - -// NodeID is a unique identifier for each node. -// The node identifier is a marshaled elliptic curve public key. -type NodeID [nodeIDBits / 8]byte - -// NodeID prints as a long hexadecimal number. -func (n NodeID) String() string { - return fmt.Sprintf("%x", n[:]) -} - -// The Go syntax representation of a NodeID is a call to HexID. -func (n NodeID) GoString() string { - return fmt.Sprintf("discover.HexID(\"%x\")", n[:]) -} - -// TerminalString returns a shortened hex string for terminal logging. -func (n NodeID) TerminalString() string { - return hex.EncodeToString(n[:8]) -} - -// HexID converts a hex string to a NodeID. -// The string may be prefixed with 0x. -func HexID(in string) (NodeID, error) { - var id NodeID - b, err := hex.DecodeString(strings.TrimPrefix(in, "0x")) - if err != nil { - return id, err - } else if len(b) != len(id) { - return id, fmt.Errorf("wrong length, want %d hex chars", len(id)*2) - } - copy(id[:], b) - return id, nil -} - -// MustHexID converts a hex string to a NodeID. -// It panics if the string is not a valid NodeID. -func MustHexID(in string) NodeID { - id, err := HexID(in) - if err != nil { - panic(err) - } - return id -} - -// PubkeyID returns a marshaled representation of the given public key. -func PubkeyID(pub *ecdsa.PublicKey) NodeID { - var id NodeID - pbytes := elliptic.Marshal(pub.Curve, pub.X, pub.Y) - if len(pbytes)-1 != len(id) { - panic(fmt.Errorf("need %d bit pubkey, got %d bits", (len(id)+1)*8, len(pbytes))) - } - copy(id[:], pbytes[1:]) - return id -} - -// Pubkey returns the public key represented by the node ID. -// It returns an error if the ID is not a point on the curve. -func (n NodeID) Pubkey() (*ecdsa.PublicKey, error) { - p := &ecdsa.PublicKey{Curve: crypto.S256(), X: new(big.Int), Y: new(big.Int)} - half := len(n) / 2 - p.X.SetBytes(n[:half]) - p.Y.SetBytes(n[half:]) - if !p.Curve.IsOnCurve(p.X, p.Y) { - return nil, errors.New("id is invalid secp256k1 curve point") - } - return p, nil -} - -// recoverNodeID computes the public key used to sign the -// given hash from the signature. -func recoverNodeID(hash, sig []byte) (id NodeID, err error) { - pubkey, err := crypto.Ecrecover(hash, sig) - if err != nil { - return id, err - } - if len(pubkey)-1 != len(id) { - return id, fmt.Errorf("recovered pubkey has %d bits, want %d bits", len(pubkey)*8, (len(id)+1)*8) - } - for i := range id { - id[i] = pubkey[i+1] - } - return id, nil -} - -// distcmp compares the distances a->target and b->target. -// Returns -1 if a is closer to target, 1 if b is closer to target -// and 0 if they are equal. -func distcmp(target, a, b common.Hash) int { - for i := range target { - da := a[i] ^ target[i] - db := b[i] ^ target[i] - if da > db { - return 1 - } else if da < db { - return -1 - } - } - return 0 -} - -// table of leading zero counts for bytes [0..255] -var lzcount = [256]int{ - 8, 7, 6, 6, 5, 5, 5, 5, - 4, 4, 4, 4, 4, 4, 4, 4, - 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, - 2, 2, 2, 2, 2, 2, 2, 2, - 2, 2, 2, 2, 2, 2, 2, 2, - 2, 2, 2, 2, 2, 2, 2, 2, - 2, 2, 2, 2, 2, 2, 2, 2, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, -} - -// logdist returns the logarithmic distance between a and b, log2(a ^ b). -func logdist(a, b common.Hash) int { - lz := 0 - for i := range a { - x := a[i] ^ b[i] - if x == 0 { - lz += 8 - } else { - lz += lzcount[x] - break - } - } - return len(a)*8 - lz -} - -// hashAtDistance returns a random hash such that logdist(a, b) == n -func hashAtDistance(a common.Hash, n int) (b common.Hash) { - if n == 0 { - return a - } - // flip bit at position n, fill the rest with random bits - b = a - pos := len(a) - n/8 - 1 - bit := byte(0x01) << (byte(n%8) - 1) - if bit == 0 { - pos++ - bit = 0x80 - } - b[pos] = a[pos]&^bit | ^a[pos]&bit // TODO: randomize end bits - for i := pos + 1; i < len(a); i++ { - b[i] = byte(rand.Intn(255)) - } - return b -} diff --git a/p2p/discv5/node_test.go b/p2p/discv5/node_test.go deleted file mode 100644 index 4e0fdbe3db67..000000000000 --- a/p2p/discv5/node_test.go +++ /dev/null @@ -1,305 +0,0 @@ -// Copyright 2015 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -package discv5 - -import ( - "fmt" - "math/big" - "math/rand" - "net" - "reflect" - "strings" - "testing" - "testing/quick" - "time" - - "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/crypto" -) - -func ExampleNewNode() { - id := MustHexID("1dd9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439") - - // Complete nodes contain UDP and TCP endpoints: - n1 := NewNode(id, net.ParseIP("2001:db8:3c4d:15::abcd:ef12"), 52150, 30303) - fmt.Println("n1:", n1) - fmt.Println("n1.Incomplete() ->", n1.Incomplete()) - - // An incomplete node can be created by passing zero values - // for all parameters except id. - n2 := NewNode(id, nil, 0, 0) - fmt.Println("n2:", n2) - fmt.Println("n2.Incomplete() ->", n2.Incomplete()) - - // Output: - // n1: enode://1dd9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439@[2001:db8:3c4d:15::abcd:ef12]:30303?discport=52150 - // n1.Incomplete() -> false - // n2: enode://1dd9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439 - // n2.Incomplete() -> true -} - -var parseNodeTests = []struct { - rawurl string - wantError string - wantResult *Node -}{ - { - rawurl: "http://foobar", - wantError: `invalid URL scheme, want "enode"`, - }, - { - rawurl: "enode://01010101@123.124.125.126:3", - wantError: `invalid node ID (wrong length, want 128 hex chars)`, - }, - // Complete nodes with IP address. - { - rawurl: "enode://1dd9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439@hostname:3", - wantError: `invalid IP address`, - }, - { - rawurl: "enode://1dd9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439@127.0.0.1:foo", - wantError: `invalid port`, - }, - { - rawurl: "enode://1dd9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439@127.0.0.1:3?discport=foo", - wantError: `invalid discport in query`, - }, - { - rawurl: "enode://1dd9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439@127.0.0.1:52150", - wantResult: NewNode( - MustHexID("0x1dd9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439"), - net.IP{0x7f, 0x0, 0x0, 0x1}, - 52150, - 52150, - ), - }, - { - rawurl: "enode://1dd9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439@[::]:52150", - wantResult: NewNode( - MustHexID("0x1dd9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439"), - net.ParseIP("::"), - 52150, - 52150, - ), - }, - { - rawurl: "enode://1dd9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439@[2001:db8:3c4d:15::abcd:ef12]:52150", - wantResult: NewNode( - MustHexID("0x1dd9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439"), - net.ParseIP("2001:db8:3c4d:15::abcd:ef12"), - 52150, - 52150, - ), - }, - { - rawurl: "enode://1dd9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439@127.0.0.1:52150?discport=22334", - wantResult: NewNode( - MustHexID("0x1dd9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439"), - net.IP{0x7f, 0x0, 0x0, 0x1}, - 22334, - 52150, - ), - }, - // Incomplete nodes with no address. - { - rawurl: "1dd9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439", - wantResult: NewNode( - MustHexID("0x1dd9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439"), - nil, 0, 0, - ), - }, - { - rawurl: "enode://1dd9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439", - wantResult: NewNode( - MustHexID("0x1dd9d65c4552b5eb43d5ad55a2ee3f56c6cbc1c64a5c8d659f51fcd51bace24351232b8d7821617d2b29b54b81cdefb9b3e9c37d7fd5f63270bcc9e1a6f6a439"), - nil, 0, 0, - ), - }, - // Invalid URLs - { - rawurl: "01010101", - wantError: `invalid node ID (wrong length, want 128 hex chars)`, - }, - { - rawurl: "enode://01010101", - wantError: `invalid node ID (wrong length, want 128 hex chars)`, - }, - { - // This test checks that errors from url.Parse are handled. - rawurl: "://foo", - wantError: `missing protocol scheme`, - }, -} - -func TestParseNode(t *testing.T) { - for _, test := range parseNodeTests { - n, err := ParseNode(test.rawurl) - if test.wantError != "" { - if err == nil { - t.Errorf("test %q:\n got nil error, expected %#q", test.rawurl, test.wantError) - continue - } else if !strings.Contains(err.Error(), test.wantError) { - t.Errorf("test %q:\n got error %#q, expected %#q", test.rawurl, err.Error(), test.wantError) - continue - } - } else { - if err != nil { - t.Errorf("test %q:\n unexpected error: %v", test.rawurl, err) - continue - } - if !reflect.DeepEqual(n, test.wantResult) { - t.Errorf("test %q:\n result mismatch:\ngot: %#v, want: %#v", test.rawurl, n, test.wantResult) - } - } - } -} - -func TestNodeString(t *testing.T) { - for i, test := range parseNodeTests { - if test.wantError == "" && strings.HasPrefix(test.rawurl, "enode://") { - str := test.wantResult.String() - if str != test.rawurl { - t.Errorf("test %d: Node.String() mismatch:\ngot: %s\nwant: %s", i, str, test.rawurl) - } - } - } -} - -func TestHexID(t *testing.T) { - ref := NodeID{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 128, 106, 217, 182, 31, 165, 174, 1, 67, 7, 235, 220, 150, 66, 83, 173, 205, 159, 44, 10, 57, 42, 161, 26, 188} - id1 := MustHexID("0x000000000000000000000000000000000000000000000000000000000000000000000000000000806ad9b61fa5ae014307ebdc964253adcd9f2c0a392aa11abc") - id2 := MustHexID("000000000000000000000000000000000000000000000000000000000000000000000000000000806ad9b61fa5ae014307ebdc964253adcd9f2c0a392aa11abc") - - if id1 != ref { - t.Errorf("wrong id1\ngot %v\nwant %v", id1[:], ref[:]) - } - if id2 != ref { - t.Errorf("wrong id2\ngot %v\nwant %v", id2[:], ref[:]) - } -} - -func TestNodeID_recover(t *testing.T) { - prv := newkey() - hash := make([]byte, 32) - sig, err := crypto.Sign(hash, prv) - if err != nil { - t.Fatalf("signing error: %v", err) - } - - pub := PubkeyID(&prv.PublicKey) - recpub, err := recoverNodeID(hash, sig) - if err != nil { - t.Fatalf("recovery error: %v", err) - } - if pub != recpub { - t.Errorf("recovered wrong pubkey:\ngot: %v\nwant: %v", recpub, pub) - } - - ecdsa, err := pub.Pubkey() - if err != nil { - t.Errorf("Pubkey error: %v", err) - } - if !reflect.DeepEqual(ecdsa, &prv.PublicKey) { - t.Errorf("Pubkey mismatch:\n got: %#v\n want: %#v", ecdsa, &prv.PublicKey) - } -} - -func TestNodeID_pubkeyBad(t *testing.T) { - ecdsa, err := NodeID{}.Pubkey() - if err == nil { - t.Error("expected error for zero ID") - } - if ecdsa != nil { - t.Error("expected nil result") - } -} - -func TestNodeID_distcmp(t *testing.T) { - distcmpBig := func(target, a, b common.Hash) int { - tbig := new(big.Int).SetBytes(target[:]) - abig := new(big.Int).SetBytes(a[:]) - bbig := new(big.Int).SetBytes(b[:]) - return new(big.Int).Xor(tbig, abig).Cmp(new(big.Int).Xor(tbig, bbig)) - } - if err := quick.CheckEqual(distcmp, distcmpBig, quickcfg()); err != nil { - t.Error(err) - } -} - -// the random tests is likely to miss the case where they're equal. -func TestNodeID_distcmpEqual(t *testing.T) { - base := common.Hash{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15} - x := common.Hash{15, 14, 13, 12, 11, 10, 9, 8, 7, 6, 5, 4, 3, 2, 1, 0} - if distcmp(base, x, x) != 0 { - t.Errorf("distcmp(base, x, x) != 0") - } -} - -func TestNodeID_logdist(t *testing.T) { - logdistBig := func(a, b common.Hash) int { - abig, bbig := new(big.Int).SetBytes(a[:]), new(big.Int).SetBytes(b[:]) - return new(big.Int).Xor(abig, bbig).BitLen() - } - if err := quick.CheckEqual(logdist, logdistBig, quickcfg()); err != nil { - t.Error(err) - } -} - -// the random tests is likely to miss the case where they're equal. -func TestNodeID_logdistEqual(t *testing.T) { - x := common.Hash{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15} - if logdist(x, x) != 0 { - t.Errorf("logdist(x, x) != 0") - } -} - -func TestNodeID_hashAtDistance(t *testing.T) { - // we don't use quick.Check here because its output isn't - // very helpful when the test fails. - cfg := quickcfg() - for i := 0; i < cfg.MaxCount; i++ { - a := gen(common.Hash{}, cfg.Rand).(common.Hash) - dist := cfg.Rand.Intn(len(common.Hash{}) * 8) - result := hashAtDistance(a, dist) - actualdist := logdist(result, a) - - if dist != actualdist { - t.Log("a: ", a) - t.Log("result:", result) - t.Fatalf("#%d: distance of result is %d, want %d", i, actualdist, dist) - } - } -} - -func quickcfg() *quick.Config { - return &quick.Config{ - MaxCount: 5000, - Rand: rand.New(rand.NewSource(time.Now().Unix())), - } -} - -// TODO: The Generate method can be dropped when we require Go >= 1.5 -// because testing/quick learned to generate arrays in 1.5. - -func (NodeID) Generate(rand *rand.Rand, size int) reflect.Value { - var id NodeID - m := rand.Intn(len(id)) - for i := len(id) - 1; i > m; i-- { - id[i] = byte(rand.Uint32()) - } - return reflect.ValueOf(id) -} diff --git a/p2p/discv5/nodeevent_string.go b/p2p/discv5/nodeevent_string.go deleted file mode 100644 index 38c1993bacbf..000000000000 --- a/p2p/discv5/nodeevent_string.go +++ /dev/null @@ -1,17 +0,0 @@ -// Code generated by "stringer -type=nodeEvent"; DO NOT EDIT. - -package discv5 - -import "strconv" - -const _nodeEvent_name = "pongTimeoutpingTimeoutneighboursTimeout" - -var _nodeEvent_index = [...]uint8{0, 11, 22, 39} - -func (i nodeEvent) String() string { - i -= 264 - if i >= nodeEvent(len(_nodeEvent_index)-1) { - return "nodeEvent(" + strconv.FormatInt(int64(i+264), 10) + ")" - } - return _nodeEvent_name[_nodeEvent_index[i]:_nodeEvent_index[i+1]] -} diff --git a/p2p/discv5/sim_run_test.go b/p2p/discv5/sim_run_test.go deleted file mode 100644 index bded0cc02344..000000000000 --- a/p2p/discv5/sim_run_test.go +++ /dev/null @@ -1,126 +0,0 @@ -// Copyright 2016 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -package discv5 - -import ( - "bufio" - "bytes" - "encoding/binary" - "errors" - "fmt" - "io" - "os" - "os/exec" - "runtime" - "strings" - "testing" -) - -func getnacl() (string, error) { - switch runtime.GOARCH { - case "amd64": - _, err := exec.LookPath("sel_ldr_x86_64") - return "amd64p32", err - case "i386": - _, err := exec.LookPath("sel_ldr_i386") - return "i386", err - default: - return "", errors.New("nacl is not supported on " + runtime.GOARCH) - } -} - -// runWithPlaygroundTime executes the caller -// in the NaCl sandbox with faketime enabled. -// -// This function must be called from a Test* function -// and the caller must skip the actual test when isHost is true. -func runWithPlaygroundTime(t *testing.T) (isHost bool) { - if runtime.GOOS == "nacl" { - return false - } - - // Get the caller. - callerPC, _, _, ok := runtime.Caller(1) - if !ok { - panic("can't get caller") - } - callerFunc := runtime.FuncForPC(callerPC) - if callerFunc == nil { - panic("can't get caller") - } - callerName := callerFunc.Name()[strings.LastIndexByte(callerFunc.Name(), '.')+1:] - if !strings.HasPrefix(callerName, "Test") { - panic("must be called from witin a Test* function") - } - testPattern := "^" + callerName + "$" - - // Unfortunately runtime.faketime (playground time mode) only works on NaCl. The NaCl - // SDK must be installed and linked into PATH for this to work. - arch, err := getnacl() - if err != nil { - t.Skip(err) - } - - // Compile and run the calling test using NaCl. - // The extra tag ensures that the TestMain function in sim_main_test.go is used. - cmd := exec.Command("go", "test", "-v", "-tags", "faketime_simulation", "-timeout", "100h", "-run", testPattern, ".") - cmd.Env = append([]string{"GOOS=nacl", "GOARCH=" + arch}, os.Environ()...) - stdout, _ := cmd.StdoutPipe() - stderr, _ := cmd.StderrPipe() - go skipPlaygroundOutputHeaders(os.Stdout, stdout) - go skipPlaygroundOutputHeaders(os.Stderr, stderr) - if err := cmd.Run(); err != nil { - t.Error(err) - } - - // Ensure that the test function doesn't run in the (non-NaCl) host process. - return true -} - -func skipPlaygroundOutputHeaders(out io.Writer, in io.Reader) { - // Additional output can be printed without the headers - // before the NaCl binary starts running (e.g. compiler error messages). - bufin := bufio.NewReader(in) - output, err := bufin.ReadBytes(0) - output = bytes.TrimSuffix(output, []byte{0}) - if len(output) > 0 { - out.Write(output) - } - if err != nil { - return - } - bufin.UnreadByte() - - // Playback header: 0 0 P B <8-byte time> <4-byte data length> - head := make([]byte, 4+8+4) - for { - if _, err := io.ReadFull(bufin, head); err != nil { - if err != io.EOF { - fmt.Fprintln(out, "read error:", err) - } - return - } - if !bytes.HasPrefix(head, []byte{0x00, 0x00, 'P', 'B'}) { - fmt.Fprintf(out, "expected playback header, got %q\n", head) - io.Copy(out, bufin) - return - } - // Copy data until next header. - size := binary.BigEndian.Uint32(head[12:]) - io.CopyN(out, bufin, int64(size)) - } -} diff --git a/p2p/discv5/sim_test.go b/p2p/discv5/sim_test.go deleted file mode 100644 index 3d1e610d3adf..000000000000 --- a/p2p/discv5/sim_test.go +++ /dev/null @@ -1,432 +0,0 @@ -// Copyright 2016 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -package discv5 - -import ( - "crypto/ecdsa" - "encoding/binary" - "fmt" - "math/rand" - "net" - "strconv" - "sync" - "sync/atomic" - "testing" - "time" - - "github.com/ethereum/go-ethereum/common" -) - -// In this test, nodes try to randomly resolve each other. -func TestSimRandomResolve(t *testing.T) { - t.Skip("boring") - if runWithPlaygroundTime(t) { - return - } - - sim := newSimulation() - bootnode := sim.launchNode(false) - - // A new node joins every 10s. - launcher := time.NewTicker(10 * time.Second) - defer launcher.Stop() - go func() { - for range launcher.C { - net := sim.launchNode(false) - go randomResolves(t, sim, net) - if err := net.SetFallbackNodes([]*Node{bootnode.Self()}); err != nil { - panic(err) - } - t.Logf("launched @ %v: %x\n", time.Now(), net.Self().ID[:16]) - } - }() - - time.Sleep(3 * time.Hour) - sim.shutdown() - sim.printStats() -} - -func TestSimTopics(t *testing.T) { - t.Skip("NaCl test") - if runWithPlaygroundTime(t) { - return - } - sim := newSimulation() - bootnode := sim.launchNode(false) - - go func() { - nets := make([]*Network, 1024) - for i := range nets { - net := sim.launchNode(false) - nets[i] = net - if err := net.SetFallbackNodes([]*Node{bootnode.Self()}); err != nil { - panic(err) - } - time.Sleep(time.Second * 5) - } - - for i, net := range nets { - if i < 256 { - stop := make(chan struct{}) - go net.RegisterTopic(testTopic, stop) - go func() { - //time.Sleep(time.Second * 36000) - time.Sleep(time.Second * 40000) - close(stop) - }() - time.Sleep(time.Millisecond * 100) - } - // time.Sleep(time.Second * 10) - //time.Sleep(time.Second) - /*if i%500 == 499 { - time.Sleep(time.Second * 9501) - } else { - time.Sleep(time.Second) - }*/ - } - }() - - // A new node joins every 10s. - /* launcher := time.NewTicker(5 * time.Second) - cnt := 0 - var printNet *Network - go func() { - for range launcher.C { - cnt++ - if cnt <= 1000 { - log := false //(cnt == 500) - net := sim.launchNode(log) - if log { - printNet = net - } - if cnt > 500 { - go net.RegisterTopic(testTopic, nil) - } - if err := net.SetFallbackNodes([]*Node{bootnode.Self()}); err != nil { - panic(err) - } - } - //fmt.Printf("launched @ %v: %x\n", time.Now(), net.Self().ID[:16]) - } - }() - */ - time.Sleep(55000 * time.Second) - //launcher.Stop() - sim.shutdown() - //sim.printStats() - //printNet.log.printLogs() -} - -/*func testHierarchicalTopics(i int) []Topic { - digits := strconv.FormatInt(int64(256+i/4), 4) - res := make([]Topic, 5) - for i, _ := range res { - res[i] = Topic("foo" + digits[1:i+1]) - } - return res -}*/ - -func testHierarchicalTopics(i int) []Topic { - digits := strconv.FormatInt(int64(128+i/8), 2) - res := make([]Topic, 8) - for i := range res { - res[i] = Topic("foo" + digits[1:i+1]) - } - return res -} - -func TestSimTopicHierarchy(t *testing.T) { - t.Skip("NaCl test") - if runWithPlaygroundTime(t) { - return - } - sim := newSimulation() - bootnode := sim.launchNode(false) - - go func() { - nets := make([]*Network, 1024) - for i := range nets { - net := sim.launchNode(false) - nets[i] = net - if err := net.SetFallbackNodes([]*Node{bootnode.Self()}); err != nil { - panic(err) - } - time.Sleep(time.Second * 5) - } - - stop := make(chan struct{}) - for i, net := range nets { - //if i < 256 { - for _, topic := range testHierarchicalTopics(i)[:5] { - //fmt.Println("reg", topic) - go net.RegisterTopic(topic, stop) - } - time.Sleep(time.Millisecond * 100) - //} - } - time.Sleep(time.Second * 90000) - close(stop) - }() - - time.Sleep(100000 * time.Second) - sim.shutdown() -} - -func randomResolves(t *testing.T, s *simulation, net *Network) { - randtime := func() time.Duration { - return time.Duration(rand.Intn(50)+20) * time.Second - } - lookup := func(target NodeID) bool { - result := net.Resolve(target) - return result != nil && result.ID == target - } - - timer := time.NewTimer(randtime()) - defer timer.Stop() - for { - select { - case <-timer.C: - target := s.randomNode().Self().ID - if !lookup(target) { - t.Errorf("node %x: target %x not found", net.Self().ID[:8], target[:8]) - } - timer.Reset(randtime()) - case <-net.closed: - return - } - } -} - -type simulation struct { - mu sync.RWMutex - nodes map[NodeID]*Network - nodectr uint32 -} - -func newSimulation() *simulation { - return &simulation{nodes: make(map[NodeID]*Network)} -} - -func (s *simulation) shutdown() { - s.mu.RLock() - alive := make([]*Network, 0, len(s.nodes)) - for _, n := range s.nodes { - alive = append(alive, n) - } - defer s.mu.RUnlock() - - for _, n := range alive { - n.Close() - } -} - -func (s *simulation) printStats() { - s.mu.Lock() - defer s.mu.Unlock() - fmt.Println("node counter:", s.nodectr) - fmt.Println("alive nodes:", len(s.nodes)) - - // for _, n := range s.nodes { - // fmt.Printf("%x\n", n.tab.self.ID[:8]) - // transport := n.conn.(*simTransport) - // fmt.Println(" joined:", transport.joinTime) - // fmt.Println(" sends:", transport.hashctr) - // fmt.Println(" table size:", n.tab.count) - // } - - /*for _, n := range s.nodes { - fmt.Println() - fmt.Printf("*** Node %x\n", n.tab.self.ID[:8]) - n.log.printLogs() - }*/ - -} - -func (s *simulation) randomNode() *Network { - s.mu.Lock() - defer s.mu.Unlock() - - n := rand.Intn(len(s.nodes)) - for _, net := range s.nodes { - if n == 0 { - return net - } - n-- - } - return nil -} - -func (s *simulation) launchNode(log bool) *Network { - var ( - num = s.nodectr - key = newkey() - id = PubkeyID(&key.PublicKey) - ip = make(net.IP, 4) - ) - s.nodectr++ - binary.BigEndian.PutUint32(ip, num) - ip[0] = 10 - addr := &net.UDPAddr{IP: ip, Port: 30303} - - transport := &simTransport{joinTime: time.Now(), sender: id, senderAddr: addr, sim: s, priv: key} - net, err := newNetwork(transport, key.PublicKey, "", nil) - if err != nil { - panic("cannot launch new node: " + err.Error()) - } - - s.mu.Lock() - s.nodes[id] = net - s.mu.Unlock() - - return net -} - -type simTransport struct { - joinTime time.Time - sender NodeID - senderAddr *net.UDPAddr - sim *simulation - hashctr uint64 - priv *ecdsa.PrivateKey -} - -func (st *simTransport) localAddr() *net.UDPAddr { - return st.senderAddr -} - -func (st *simTransport) Close() {} - -func (st *simTransport) send(remote *Node, ptype nodeEvent, data interface{}) (hash []byte) { - hash = st.nextHash() - var raw []byte - if ptype == pongPacket { - var err error - raw, _, err = encodePacket(st.priv, byte(ptype), data) - if err != nil { - panic(err) - } - } - - st.sendPacket(remote.ID, ingressPacket{ - remoteID: st.sender, - remoteAddr: st.senderAddr, - hash: hash, - ev: ptype, - data: data, - rawData: raw, - }) - return hash -} - -func (st *simTransport) sendPing(remote *Node, remoteAddr *net.UDPAddr, topics []Topic) []byte { - hash := st.nextHash() - st.sendPacket(remote.ID, ingressPacket{ - remoteID: st.sender, - remoteAddr: st.senderAddr, - hash: hash, - ev: pingPacket, - data: &ping{ - Version: 4, - From: rpcEndpoint{IP: st.senderAddr.IP, UDP: uint16(st.senderAddr.Port), TCP: 30303}, - To: rpcEndpoint{IP: remoteAddr.IP, UDP: uint16(remoteAddr.Port), TCP: 30303}, - Expiration: uint64(time.Now().Unix() + int64(expiration)), - Topics: topics, - }, - }) - return hash -} - -func (st *simTransport) sendFindnodeHash(remote *Node, target common.Hash) { - st.sendPacket(remote.ID, ingressPacket{ - remoteID: st.sender, - remoteAddr: st.senderAddr, - hash: st.nextHash(), - ev: findnodeHashPacket, - data: &findnodeHash{ - Target: target, - Expiration: uint64(time.Now().Unix() + int64(expiration)), - }, - }) -} - -func (st *simTransport) sendTopicRegister(remote *Node, topics []Topic, idx int, pong []byte) { - //fmt.Println("send", topics, pong) - st.sendPacket(remote.ID, ingressPacket{ - remoteID: st.sender, - remoteAddr: st.senderAddr, - hash: st.nextHash(), - ev: topicRegisterPacket, - data: &topicRegister{ - Topics: topics, - Idx: uint(idx), - Pong: pong, - }, - }) -} - -func (st *simTransport) sendTopicNodes(remote *Node, queryHash common.Hash, nodes []*Node) { - rnodes := make([]rpcNode, len(nodes)) - for i := range nodes { - rnodes[i] = nodeToRPC(nodes[i]) - } - st.sendPacket(remote.ID, ingressPacket{ - remoteID: st.sender, - remoteAddr: st.senderAddr, - hash: st.nextHash(), - ev: topicNodesPacket, - data: &topicNodes{Echo: queryHash, Nodes: rnodes}, - }) -} - -func (st *simTransport) sendNeighbours(remote *Node, nodes []*Node) { - // TODO: send multiple packets - rnodes := make([]rpcNode, len(nodes)) - for i := range nodes { - rnodes[i] = nodeToRPC(nodes[i]) - } - st.sendPacket(remote.ID, ingressPacket{ - remoteID: st.sender, - remoteAddr: st.senderAddr, - hash: st.nextHash(), - ev: neighborsPacket, - data: &neighbors{ - Nodes: rnodes, - Expiration: uint64(time.Now().Unix() + int64(expiration)), - }, - }) -} - -func (st *simTransport) nextHash() []byte { - v := atomic.AddUint64(&st.hashctr, 1) - var hash common.Hash - binary.BigEndian.PutUint64(hash[:], v) - return hash[:] -} - -const packetLoss = 0 // 1/1000 - -func (st *simTransport) sendPacket(remote NodeID, p ingressPacket) { - if rand.Int31n(1000) >= packetLoss { - st.sim.mu.RLock() - recipient := st.sim.nodes[remote] - st.sim.mu.RUnlock() - - time.AfterFunc(200*time.Millisecond, func() { - recipient.reqReadPacket(p) - }) - } -} diff --git a/p2p/discv5/sim_testmain_test.go b/p2p/discv5/sim_testmain_test.go deleted file mode 100644 index 77e751c419fe..000000000000 --- a/p2p/discv5/sim_testmain_test.go +++ /dev/null @@ -1,43 +0,0 @@ -// Copyright 2016 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -// +build go1.4,nacl,faketime_simulation - -package discv5 - -import ( - "os" - "runtime" - "testing" - "unsafe" -) - -// Enable fake time mode in the runtime, like on the go playground. -// There is a slight chance that this won't work because some go code -// might have executed before the variable is set. - -//go:linkname faketime runtime.faketime -var faketime = 1 - -func TestMain(m *testing.M) { - // We need to use unsafe somehow in order to get access to go:linkname. - _ = unsafe.Sizeof(0) - - // Run the actual test. runWithPlaygroundTime ensures that the only test - // that runs is the one calling it. - runtime.GOMAXPROCS(8) - os.Exit(m.Run()) -} diff --git a/p2p/discv5/table.go b/p2p/discv5/table.go deleted file mode 100644 index 64c3ecd1c7b2..000000000000 --- a/p2p/discv5/table.go +++ /dev/null @@ -1,318 +0,0 @@ -// Copyright 2016 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -// Package discv5 is a prototype implementation of Discvery v5. -// Deprecated: do not use this package. -package discv5 - -import ( - "crypto/rand" - "encoding/binary" - "fmt" - "net" - "sort" - - "github.com/ethereum/go-ethereum/common" -) - -const ( - alpha = 3 // Kademlia concurrency factor - bucketSize = 16 // Kademlia bucket size - hashBits = len(common.Hash{}) * 8 - nBuckets = hashBits + 1 // Number of buckets - - maxFindnodeFailures = 5 -) - -type Table struct { - count int // number of nodes - buckets [nBuckets]*bucket // index of known nodes by distance - nodeAddedHook func(*Node) // for testing - self *Node // metadata of the local node -} - -// bucket contains nodes, ordered by their last activity. the entry -// that was most recently active is the first element in entries. -type bucket struct { - entries []*Node - replacements []*Node -} - -func newTable(ourID NodeID, ourAddr *net.UDPAddr) *Table { - self := NewNode(ourID, ourAddr.IP, uint16(ourAddr.Port), uint16(ourAddr.Port)) - tab := &Table{self: self} - for i := range tab.buckets { - tab.buckets[i] = new(bucket) - } - return tab -} - -const printTable = false - -// chooseBucketRefreshTarget selects random refresh targets to keep all Kademlia -// buckets filled with live connections and keep the network topology healthy. -// This requires selecting addresses closer to our own with a higher probability -// in order to refresh closer buckets too. -// -// This algorithm approximates the distance distribution of existing nodes in the -// table by selecting a random node from the table and selecting a target address -// with a distance less than twice of that of the selected node. -// This algorithm will be improved later to specifically target the least recently -// used buckets. -func (tab *Table) chooseBucketRefreshTarget() common.Hash { - entries := 0 - if printTable { - fmt.Println() - } - for i, b := range &tab.buckets { - entries += len(b.entries) - if printTable { - for _, e := range b.entries { - fmt.Println(i, e.state, e.addr().String(), e.ID.String(), e.sha.Hex()) - } - } - } - - prefix := binary.BigEndian.Uint64(tab.self.sha[0:8]) - dist := ^uint64(0) - entry := int(randUint(uint32(entries + 1))) - for _, b := range &tab.buckets { - if entry < len(b.entries) { - n := b.entries[entry] - dist = binary.BigEndian.Uint64(n.sha[0:8]) ^ prefix - break - } - entry -= len(b.entries) - } - - ddist := ^uint64(0) - if dist+dist > dist { - ddist = dist - } - targetPrefix := prefix ^ randUint64n(ddist) - - var target common.Hash - binary.BigEndian.PutUint64(target[0:8], targetPrefix) - rand.Read(target[8:]) - return target -} - -// readRandomNodes fills the given slice with random nodes from the -// table. It will not write the same node more than once. The nodes in -// the slice are copies and can be modified by the caller. -func (tab *Table) readRandomNodes(buf []*Node) (n int) { - // TODO: tree-based buckets would help here - // Find all non-empty buckets and get a fresh slice of their entries. - var buckets [][]*Node - for _, b := range &tab.buckets { - if len(b.entries) > 0 { - buckets = append(buckets, b.entries) - } - } - if len(buckets) == 0 { - return 0 - } - // Shuffle the buckets. - for i := uint32(len(buckets)) - 1; i > 0; i-- { - j := randUint(i) - buckets[i], buckets[j] = buckets[j], buckets[i] - } - // Move head of each bucket into buf, removing buckets that become empty. - var i, j int - for ; i < len(buf); i, j = i+1, (j+1)%len(buckets) { - b := buckets[j] - buf[i] = &(*b[0]) - buckets[j] = b[1:] - if len(b) == 1 { - buckets = append(buckets[:j], buckets[j+1:]...) - } - if len(buckets) == 0 { - break - } - } - return i + 1 -} - -func randUint(max uint32) uint32 { - if max < 2 { - return 0 - } - var b [4]byte - rand.Read(b[:]) - return binary.BigEndian.Uint32(b[:]) % max -} - -func randUint64n(max uint64) uint64 { - if max < 2 { - return 0 - } - var b [8]byte - rand.Read(b[:]) - return binary.BigEndian.Uint64(b[:]) % max -} - -// closest returns the n nodes in the table that are closest to the -// given id. The caller must hold tab.mutex. -func (tab *Table) closest(target common.Hash, nresults int) *nodesByDistance { - // This is a very wasteful way to find the closest nodes but - // obviously correct. I believe that tree-based buckets would make - // this easier to implement efficiently. - close := &nodesByDistance{target: target} - for _, b := range &tab.buckets { - for _, n := range b.entries { - close.push(n, nresults) - } - } - return close -} - -// add attempts to add the given node its corresponding bucket. If the -// bucket has space available, adding the node succeeds immediately. -// Otherwise, the node is added to the replacement cache for the bucket. -func (tab *Table) add(n *Node) (contested *Node) { - //fmt.Println("add", n.addr().String(), n.ID.String(), n.sha.Hex()) - if n.ID == tab.self.ID { - return - } - b := tab.buckets[logdist(tab.self.sha, n.sha)] - switch { - case b.bump(n): - // n exists in b. - return nil - case len(b.entries) < bucketSize: - // b has space available. - b.addFront(n) - tab.count++ - if tab.nodeAddedHook != nil { - tab.nodeAddedHook(n) - } - return nil - default: - // b has no space left, add to replacement cache - // and revalidate the last entry. - // TODO: drop previous node - b.replacements = append(b.replacements, n) - if len(b.replacements) > bucketSize { - copy(b.replacements, b.replacements[1:]) - b.replacements = b.replacements[:len(b.replacements)-1] - } - return b.entries[len(b.entries)-1] - } -} - -// stuff adds nodes the table to the end of their corresponding bucket -// if the bucket is not full. -func (tab *Table) stuff(nodes []*Node) { -outer: - for _, n := range nodes { - if n.ID == tab.self.ID { - continue // don't add self - } - bucket := tab.buckets[logdist(tab.self.sha, n.sha)] - for i := range bucket.entries { - if bucket.entries[i].ID == n.ID { - continue outer // already in bucket - } - } - if len(bucket.entries) < bucketSize { - bucket.entries = append(bucket.entries, n) - tab.count++ - if tab.nodeAddedHook != nil { - tab.nodeAddedHook(n) - } - } - } -} - -// delete removes an entry from the node table (used to evacuate -// failed/non-bonded discovery peers). -func (tab *Table) delete(node *Node) { - //fmt.Println("delete", node.addr().String(), node.ID.String(), node.sha.Hex()) - bucket := tab.buckets[logdist(tab.self.sha, node.sha)] - for i := range bucket.entries { - if bucket.entries[i].ID == node.ID { - bucket.entries = append(bucket.entries[:i], bucket.entries[i+1:]...) - tab.count-- - return - } - } -} - -func (tab *Table) deleteReplace(node *Node) { - b := tab.buckets[logdist(tab.self.sha, node.sha)] - i := 0 - for i < len(b.entries) { - if b.entries[i].ID == node.ID { - b.entries = append(b.entries[:i], b.entries[i+1:]...) - tab.count-- - } else { - i++ - } - } - // refill from replacement cache - // TODO: maybe use random index - if len(b.entries) < bucketSize && len(b.replacements) > 0 { - ri := len(b.replacements) - 1 - b.addFront(b.replacements[ri]) - tab.count++ - b.replacements[ri] = nil - b.replacements = b.replacements[:ri] - } -} - -func (b *bucket) addFront(n *Node) { - b.entries = append(b.entries, nil) - copy(b.entries[1:], b.entries) - b.entries[0] = n -} - -func (b *bucket) bump(n *Node) bool { - for i := range b.entries { - if b.entries[i].ID == n.ID { - // move it to the front - copy(b.entries[1:], b.entries[:i]) - b.entries[0] = n - return true - } - } - return false -} - -// nodesByDistance is a list of nodes, ordered by -// distance to target. -type nodesByDistance struct { - entries []*Node - target common.Hash -} - -// push adds the given node to the list, keeping the total size below maxElems. -func (h *nodesByDistance) push(n *Node, maxElems int) { - ix := sort.Search(len(h.entries), func(i int) bool { - return distcmp(h.target, h.entries[i].sha, n.sha) > 0 - }) - if len(h.entries) < maxElems { - h.entries = append(h.entries, n) - } - if ix == len(h.entries) { - // farther away than all nodes we already have. - // if there was room for it, the node is now the last element. - } else { - // slide existing entries down to make room - // this will overwrite the entry we just appended. - copy(h.entries[ix+1:], h.entries[ix:]) - h.entries[ix] = n - } -} diff --git a/p2p/discv5/table_test.go b/p2p/discv5/table_test.go deleted file mode 100644 index 872a4f68367e..000000000000 --- a/p2p/discv5/table_test.go +++ /dev/null @@ -1,238 +0,0 @@ -// Copyright 2016 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -package discv5 - -import ( - "crypto/ecdsa" - "fmt" - "math/rand" - - "net" - "reflect" - "testing" - "testing/quick" - "time" - - "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/crypto" -) - -func TestBucket_bumpNoDuplicates(t *testing.T) { - t.Parallel() - cfg := &quick.Config{ - MaxCount: 1000, - Rand: rand.New(rand.NewSource(time.Now().Unix())), - Values: func(args []reflect.Value, rand *rand.Rand) { - // generate a random list of nodes. this will be the content of the bucket. - n := rand.Intn(bucketSize-1) + 1 - nodes := make([]*Node, n) - for i := range nodes { - nodes[i] = nodeAtDistance(common.Hash{}, 200) - } - args[0] = reflect.ValueOf(nodes) - // generate random bump positions. - bumps := make([]int, rand.Intn(100)) - for i := range bumps { - bumps[i] = rand.Intn(len(nodes)) - } - args[1] = reflect.ValueOf(bumps) - }, - } - - prop := func(nodes []*Node, bumps []int) (ok bool) { - b := &bucket{entries: make([]*Node, len(nodes))} - copy(b.entries, nodes) - for i, pos := range bumps { - b.bump(b.entries[pos]) - if hasDuplicates(b.entries) { - t.Logf("bucket has duplicates after %d/%d bumps:", i+1, len(bumps)) - for _, n := range b.entries { - t.Logf(" %p", n) - } - return false - } - } - return true - } - if err := quick.Check(prop, cfg); err != nil { - t.Error(err) - } -} - -// nodeAtDistance creates a node for which logdist(base, n.sha) == ld. -// The node's ID does not correspond to n.sha. -func nodeAtDistance(base common.Hash, ld int) (n *Node) { - n = new(Node) - n.sha = hashAtDistance(base, ld) - copy(n.ID[:], n.sha[:]) // ensure the node still has a unique ID - return n -} - -func TestTable_closest(t *testing.T) { - t.Parallel() - - test := func(test *closeTest) bool { - // for any node table, Target and N - tab := newTable(test.Self, &net.UDPAddr{}) - tab.stuff(test.All) - - // check that doClosest(Target, N) returns nodes - result := tab.closest(test.Target, test.N).entries - if hasDuplicates(result) { - t.Errorf("result contains duplicates") - return false - } - if !sortedByDistanceTo(test.Target, result) { - t.Errorf("result is not sorted by distance to target") - return false - } - - // check that the number of results is min(N, tablen) - wantN := test.N - if tab.count < test.N { - wantN = tab.count - } - if len(result) != wantN { - t.Errorf("wrong number of nodes: got %d, want %d", len(result), wantN) - return false - } else if len(result) == 0 { - return true // no need to check distance - } - - // check that the result nodes have minimum distance to target. - for _, b := range tab.buckets { - for _, n := range b.entries { - if contains(result, n.ID) { - continue // don't run the check below for nodes in result - } - farthestResult := result[len(result)-1].sha - if distcmp(test.Target, n.sha, farthestResult) < 0 { - t.Errorf("table contains node that is closer to target but it's not in result") - t.Logf(" Target: %v", test.Target) - t.Logf(" Farthest Result: %v", farthestResult) - t.Logf(" ID: %v", n.ID) - return false - } - } - } - return true - } - if err := quick.Check(test, quickcfg()); err != nil { - t.Error(err) - } -} - -func TestTable_ReadRandomNodesGetAll(t *testing.T) { - cfg := &quick.Config{ - MaxCount: 200, - Rand: rand.New(rand.NewSource(time.Now().Unix())), - Values: func(args []reflect.Value, rand *rand.Rand) { - args[0] = reflect.ValueOf(make([]*Node, rand.Intn(1000))) - }, - } - test := func(buf []*Node) bool { - tab := newTable(NodeID{}, &net.UDPAddr{}) - for i := 0; i < len(buf); i++ { - ld := cfg.Rand.Intn(len(tab.buckets)) - tab.stuff([]*Node{nodeAtDistance(tab.self.sha, ld)}) - } - gotN := tab.readRandomNodes(buf) - if gotN != tab.count { - t.Errorf("wrong number of nodes, got %d, want %d", gotN, tab.count) - return false - } - if hasDuplicates(buf[:gotN]) { - t.Errorf("result contains duplicates") - return false - } - return true - } - if err := quick.Check(test, cfg); err != nil { - t.Error(err) - } -} - -type closeTest struct { - Self NodeID - Target common.Hash - All []*Node - N int -} - -func (*closeTest) Generate(rand *rand.Rand, size int) reflect.Value { - t := &closeTest{ - Self: gen(NodeID{}, rand).(NodeID), - Target: gen(common.Hash{}, rand).(common.Hash), - N: rand.Intn(bucketSize), - } - for _, id := range gen([]NodeID{}, rand).([]NodeID) { - t.All = append(t.All, &Node{ID: id}) - } - return reflect.ValueOf(t) -} - -func hasDuplicates(slice []*Node) bool { - seen := make(map[NodeID]bool) - for i, e := range slice { - if e == nil { - panic(fmt.Sprintf("nil *Node at %d", i)) - } - if seen[e.ID] { - return true - } - seen[e.ID] = true - } - return false -} - -func sortedByDistanceTo(distbase common.Hash, slice []*Node) bool { - var last common.Hash - for i, e := range slice { - if i > 0 && distcmp(distbase, e.sha, last) < 0 { - return false - } - last = e.sha - } - return true -} - -func contains(ns []*Node, id NodeID) bool { - for _, n := range ns { - if n.ID == id { - return true - } - } - return false -} - -// gen wraps quick.Value so it's easier to use. -// it generates a random value of the given value's type. -func gen(typ interface{}, rand *rand.Rand) interface{} { - v, ok := quick.Value(reflect.TypeOf(typ), rand) - if !ok { - panic(fmt.Sprintf("couldn't generate random value of type %T", typ)) - } - return v.Interface() -} - -func newkey() *ecdsa.PrivateKey { - key, err := crypto.GenerateKey() - if err != nil { - panic("couldn't generate key: " + err.Error()) - } - return key -} diff --git a/p2p/discv5/ticket.go b/p2p/discv5/ticket.go deleted file mode 100644 index c5e3d6c08f93..000000000000 --- a/p2p/discv5/ticket.go +++ /dev/null @@ -1,884 +0,0 @@ -// Copyright 2016 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -package discv5 - -import ( - "bytes" - "encoding/binary" - "fmt" - "math" - "math/rand" - "time" - - "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/common/mclock" - "github.com/ethereum/go-ethereum/crypto" - "github.com/ethereum/go-ethereum/log" -) - -const ( - ticketTimeBucketLen = time.Minute - collectFrequency = time.Second * 30 - registerFrequency = time.Second * 60 - maxCollectDebt = 10 - maxRegisterDebt = 5 - keepTicketConst = time.Minute * 10 - keepTicketExp = time.Minute * 5 - targetWaitTime = time.Minute * 10 - topicQueryTimeout = time.Second * 5 - topicQueryResend = time.Minute - // topic radius detection - maxRadius = 0xffffffffffffffff - radiusTC = time.Minute * 20 - radiusBucketsPerBit = 8 - minSlope = 1 - minPeakSize = 40 - maxNoAdjust = 20 - lookupWidth = 8 - minRightSum = 20 - searchForceQuery = 4 -) - -// timeBucket represents absolute monotonic time in minutes. -// It is used as the index into the per-topic ticket buckets. -type timeBucket int - -type ticket struct { - topics []Topic - regTime []mclock.AbsTime // Per-topic local absolute time when the ticket can be used. - - // The serial number that was issued by the server. - serial uint32 - // Used by registrar, tracks absolute time when the ticket was created. - issueTime mclock.AbsTime - - // Fields used only by registrants - node *Node // the registrar node that signed this ticket - refCnt int // tracks number of topics that will be registered using this ticket - pong []byte // encoded pong packet signed by the registrar -} - -// ticketRef refers to a single topic in a ticket. -type ticketRef struct { - t *ticket - idx int // index of the topic in t.topics and t.regTime -} - -func (ref ticketRef) topic() Topic { - return ref.t.topics[ref.idx] -} - -func (ref ticketRef) topicRegTime() mclock.AbsTime { - return ref.t.regTime[ref.idx] -} - -func pongToTicket(localTime mclock.AbsTime, topics []Topic, node *Node, p *ingressPacket) (*ticket, error) { - wps := p.data.(*pong).WaitPeriods - if len(topics) != len(wps) { - return nil, fmt.Errorf("bad wait period list: got %d values, want %d", len(topics), len(wps)) - } - if rlpHash(topics) != p.data.(*pong).TopicHash { - return nil, fmt.Errorf("bad topic hash") - } - t := &ticket{ - issueTime: localTime, - node: node, - topics: topics, - pong: p.rawData, - regTime: make([]mclock.AbsTime, len(wps)), - } - // Convert wait periods to local absolute time. - for i, wp := range wps { - t.regTime[i] = localTime + mclock.AbsTime(time.Second*time.Duration(wp)) - } - return t, nil -} - -func ticketToPong(t *ticket, pong *pong) { - pong.Expiration = uint64(t.issueTime / mclock.AbsTime(time.Second)) - pong.TopicHash = rlpHash(t.topics) - pong.TicketSerial = t.serial - pong.WaitPeriods = make([]uint32, len(t.regTime)) - for i, regTime := range t.regTime { - pong.WaitPeriods[i] = uint32(time.Duration(regTime-t.issueTime) / time.Second) - } -} - -type ticketStore struct { - // radius detector and target address generator - // exists for both searched and registered topics - radius map[Topic]*topicRadius - - // Contains buckets (for each absolute minute) of tickets - // that can be used in that minute. - // This is only set if the topic is being registered. - tickets map[Topic]*topicTickets - - regQueue []Topic // Topic registration queue for round robin attempts - regSet map[Topic]struct{} // Topic registration queue contents for fast filling - - nodes map[*Node]*ticket - nodeLastReq map[*Node]reqInfo - - lastBucketFetched timeBucket - nextTicketCached *ticketRef - - searchTopicMap map[Topic]searchTopic - nextTopicQueryCleanup mclock.AbsTime - queriesSent map[*Node]map[common.Hash]sentQuery -} - -type searchTopic struct { - foundChn chan<- *Node -} - -type sentQuery struct { - sent mclock.AbsTime - lookup lookupInfo -} - -type topicTickets struct { - buckets map[timeBucket][]ticketRef - nextLookup mclock.AbsTime - nextReg mclock.AbsTime -} - -func newTicketStore() *ticketStore { - return &ticketStore{ - radius: make(map[Topic]*topicRadius), - tickets: make(map[Topic]*topicTickets), - regSet: make(map[Topic]struct{}), - nodes: make(map[*Node]*ticket), - nodeLastReq: make(map[*Node]reqInfo), - searchTopicMap: make(map[Topic]searchTopic), - queriesSent: make(map[*Node]map[common.Hash]sentQuery), - } -} - -// addTopic starts tracking a topic. If register is true, -// the local node will register the topic and tickets will be collected. -func (s *ticketStore) addTopic(topic Topic, register bool) { - log.Trace("Adding discovery topic", "topic", topic, "register", register) - if s.radius[topic] == nil { - s.radius[topic] = newTopicRadius(topic) - } - if register && s.tickets[topic] == nil { - s.tickets[topic] = &topicTickets{buckets: make(map[timeBucket][]ticketRef)} - } -} - -func (s *ticketStore) addSearchTopic(t Topic, foundChn chan<- *Node) { - s.addTopic(t, false) - if s.searchTopicMap[t].foundChn == nil { - s.searchTopicMap[t] = searchTopic{foundChn: foundChn} - } -} - -func (s *ticketStore) removeSearchTopic(t Topic) { - if st := s.searchTopicMap[t]; st.foundChn != nil { - delete(s.searchTopicMap, t) - } -} - -// removeRegisterTopic deletes all tickets for the given topic. -func (s *ticketStore) removeRegisterTopic(topic Topic) { - log.Trace("Removing discovery topic", "topic", topic) - if s.tickets[topic] == nil { - log.Warn("Removing non-existent discovery topic", "topic", topic) - return - } - for _, list := range s.tickets[topic].buckets { - for _, ref := range list { - ref.t.refCnt-- - if ref.t.refCnt == 0 { - delete(s.nodes, ref.t.node) - delete(s.nodeLastReq, ref.t.node) - } - } - } - delete(s.tickets, topic) -} - -func (s *ticketStore) regTopicSet() []Topic { - topics := make([]Topic, 0, len(s.tickets)) - for topic := range s.tickets { - topics = append(topics, topic) - } - return topics -} - -// nextRegisterLookup returns the target of the next lookup for ticket collection. -func (s *ticketStore) nextRegisterLookup() (lookupInfo, time.Duration) { - // Queue up any new topics (or discarded ones), preserving iteration order - for topic := range s.tickets { - if _, ok := s.regSet[topic]; !ok { - s.regQueue = append(s.regQueue, topic) - s.regSet[topic] = struct{}{} - } - } - // Iterate over the set of all topics and look up the next suitable one - for len(s.regQueue) > 0 { - // Fetch the next topic from the queue, and ensure it still exists - topic := s.regQueue[0] - s.regQueue = s.regQueue[1:] - delete(s.regSet, topic) - - if s.tickets[topic] == nil { - continue - } - // If the topic needs more tickets, return it - if s.tickets[topic].nextLookup < mclock.Now() { - next, delay := s.radius[topic].nextTarget(false), 100*time.Millisecond - log.Trace("Found discovery topic to register", "topic", topic, "target", next.target, "delay", delay) - return next, delay - } - } - // No registration topics found or all exhausted, sleep - delay := 40 * time.Second - log.Trace("No topic found to register", "delay", delay) - return lookupInfo{}, delay -} - -func (s *ticketStore) nextSearchLookup(topic Topic) lookupInfo { - tr := s.radius[topic] - target := tr.nextTarget(tr.radiusLookupCnt >= searchForceQuery) - if target.radiusLookup { - tr.radiusLookupCnt++ - } else { - tr.radiusLookupCnt = 0 - } - return target -} - -func (s *ticketStore) addTicketRef(r ticketRef) { - topic := r.t.topics[r.idx] - tickets := s.tickets[topic] - if tickets == nil { - log.Warn("Adding ticket to non-existent topic", "topic", topic) - return - } - bucket := timeBucket(r.t.regTime[r.idx] / mclock.AbsTime(ticketTimeBucketLen)) - tickets.buckets[bucket] = append(tickets.buckets[bucket], r) - r.t.refCnt++ - - min := mclock.Now() - mclock.AbsTime(collectFrequency)*maxCollectDebt - if tickets.nextLookup < min { - tickets.nextLookup = min - } - tickets.nextLookup += mclock.AbsTime(collectFrequency) - - //s.removeExcessTickets(topic) -} - -func (s *ticketStore) nextFilteredTicket() (*ticketRef, time.Duration) { - now := mclock.Now() - for { - ticket, wait := s.nextRegisterableTicket() - if ticket == nil { - return ticket, wait - } - log.Trace("Found discovery ticket to register", "node", ticket.t.node, "serial", ticket.t.serial, "wait", wait) - - regTime := now + mclock.AbsTime(wait) - topic := ticket.t.topics[ticket.idx] - if s.tickets[topic] != nil && regTime >= s.tickets[topic].nextReg { - return ticket, wait - } - s.removeTicketRef(*ticket) - } -} - -func (s *ticketStore) ticketRegistered(ref ticketRef) { - now := mclock.Now() - - topic := ref.t.topics[ref.idx] - tickets := s.tickets[topic] - min := now - mclock.AbsTime(registerFrequency)*maxRegisterDebt - if min > tickets.nextReg { - tickets.nextReg = min - } - tickets.nextReg += mclock.AbsTime(registerFrequency) - s.tickets[topic] = tickets - - s.removeTicketRef(ref) -} - -// nextRegisterableTicket returns the next ticket that can be used -// to register. -// -// If the returned wait time <= zero the ticket can be used. For a positive -// wait time, the caller should requery the next ticket later. -// -// A ticket can be returned more than once with <= zero wait time in case -// the ticket contains multiple topics. -func (s *ticketStore) nextRegisterableTicket() (*ticketRef, time.Duration) { - now := mclock.Now() - if s.nextTicketCached != nil { - return s.nextTicketCached, time.Duration(s.nextTicketCached.topicRegTime() - now) - } - - for bucket := s.lastBucketFetched; ; bucket++ { - var ( - empty = true // true if there are no tickets - nextTicket ticketRef // uninitialized if this bucket is empty - ) - for _, tickets := range s.tickets { - //s.removeExcessTickets(topic) - if len(tickets.buckets) != 0 { - empty = false - - list := tickets.buckets[bucket] - for _, ref := range list { - //debugLog(fmt.Sprintf(" nrt bucket = %d node = %x sn = %v wait = %v", bucket, ref.t.node.ID[:8], ref.t.serial, time.Duration(ref.topicRegTime()-now))) - if nextTicket.t == nil || ref.topicRegTime() < nextTicket.topicRegTime() { - nextTicket = ref - } - } - } - } - if empty { - return nil, 0 - } - if nextTicket.t != nil { - s.nextTicketCached = &nextTicket - return &nextTicket, time.Duration(nextTicket.topicRegTime() - now) - } - s.lastBucketFetched = bucket - } -} - -// removeTicket removes a ticket from the ticket store -func (s *ticketStore) removeTicketRef(ref ticketRef) { - log.Trace("Removing discovery ticket reference", "node", ref.t.node.ID, "serial", ref.t.serial) - - // Make nextRegisterableTicket return the next available ticket. - s.nextTicketCached = nil - - topic := ref.topic() - tickets := s.tickets[topic] - - if tickets == nil { - log.Trace("Removing tickets from unknown topic", "topic", topic) - return - } - bucket := timeBucket(ref.t.regTime[ref.idx] / mclock.AbsTime(ticketTimeBucketLen)) - list := tickets.buckets[bucket] - idx := -1 - for i, bt := range list { - if bt.t == ref.t { - idx = i - break - } - } - if idx == -1 { - panic(nil) - } - list = append(list[:idx], list[idx+1:]...) - if len(list) != 0 { - tickets.buckets[bucket] = list - } else { - delete(tickets.buckets, bucket) - } - ref.t.refCnt-- - if ref.t.refCnt == 0 { - delete(s.nodes, ref.t.node) - delete(s.nodeLastReq, ref.t.node) - } -} - -type lookupInfo struct { - target common.Hash - topic Topic - radiusLookup bool -} - -type reqInfo struct { - pingHash []byte - lookup lookupInfo - time mclock.AbsTime -} - -// returns -1 if not found -func (t *ticket) findIdx(topic Topic) int { - for i, tt := range t.topics { - if tt == topic { - return i - } - } - return -1 -} - -func (s *ticketStore) registerLookupDone(lookup lookupInfo, nodes []*Node, ping func(n *Node) []byte) { - now := mclock.Now() - for i, n := range nodes { - if i == 0 || (binary.BigEndian.Uint64(n.sha[:8])^binary.BigEndian.Uint64(lookup.target[:8])) < s.radius[lookup.topic].minRadius { - if lookup.radiusLookup { - if lastReq, ok := s.nodeLastReq[n]; !ok || time.Duration(now-lastReq.time) > radiusTC { - s.nodeLastReq[n] = reqInfo{pingHash: ping(n), lookup: lookup, time: now} - } - } else { - if s.nodes[n] == nil { - s.nodeLastReq[n] = reqInfo{pingHash: ping(n), lookup: lookup, time: now} - } - } - } - } -} - -func (s *ticketStore) searchLookupDone(lookup lookupInfo, nodes []*Node, query func(n *Node, topic Topic) []byte) { - now := mclock.Now() - for i, n := range nodes { - if i == 0 || (binary.BigEndian.Uint64(n.sha[:8])^binary.BigEndian.Uint64(lookup.target[:8])) < s.radius[lookup.topic].minRadius { - if lookup.radiusLookup { - if lastReq, ok := s.nodeLastReq[n]; !ok || time.Duration(now-lastReq.time) > radiusTC { - s.nodeLastReq[n] = reqInfo{pingHash: nil, lookup: lookup, time: now} - } - } // else { - if s.canQueryTopic(n, lookup.topic) { - hash := query(n, lookup.topic) - if hash != nil { - s.addTopicQuery(common.BytesToHash(hash), n, lookup) - } - } - //} - } - } -} - -func (s *ticketStore) adjustWithTicket(now mclock.AbsTime, targetHash common.Hash, t *ticket) { - for i, topic := range t.topics { - if tt, ok := s.radius[topic]; ok { - tt.adjustWithTicket(now, targetHash, ticketRef{t, i}) - } - } -} - -func (s *ticketStore) addTicket(localTime mclock.AbsTime, pingHash []byte, ticket *ticket) { - log.Trace("Adding discovery ticket", "node", ticket.node.ID, "serial", ticket.serial) - - lastReq, ok := s.nodeLastReq[ticket.node] - if !(ok && bytes.Equal(pingHash, lastReq.pingHash)) { - return - } - s.adjustWithTicket(localTime, lastReq.lookup.target, ticket) - - if lastReq.lookup.radiusLookup || s.nodes[ticket.node] != nil { - return - } - - topic := lastReq.lookup.topic - topicIdx := ticket.findIdx(topic) - if topicIdx == -1 { - return - } - - bucket := timeBucket(localTime / mclock.AbsTime(ticketTimeBucketLen)) - if s.lastBucketFetched == 0 || bucket < s.lastBucketFetched { - s.lastBucketFetched = bucket - } - - if _, ok := s.tickets[topic]; ok { - wait := ticket.regTime[topicIdx] - localTime - rnd := rand.ExpFloat64() - if rnd > 10 { - rnd = 10 - } - if float64(wait) < float64(keepTicketConst)+float64(keepTicketExp)*rnd { - // use the ticket to register this topic - //fmt.Println("addTicket", ticket.node.ID[:8], ticket.node.addr().String(), ticket.serial, ticket.pong) - s.addTicketRef(ticketRef{ticket, topicIdx}) - } - } - - if ticket.refCnt > 0 { - s.nextTicketCached = nil - s.nodes[ticket.node] = ticket - } -} - -func (s *ticketStore) canQueryTopic(node *Node, topic Topic) bool { - qq := s.queriesSent[node] - if qq != nil { - now := mclock.Now() - for _, sq := range qq { - if sq.lookup.topic == topic && sq.sent > now-mclock.AbsTime(topicQueryResend) { - return false - } - } - } - return true -} - -func (s *ticketStore) addTopicQuery(hash common.Hash, node *Node, lookup lookupInfo) { - now := mclock.Now() - qq := s.queriesSent[node] - if qq == nil { - qq = make(map[common.Hash]sentQuery) - s.queriesSent[node] = qq - } - qq[hash] = sentQuery{sent: now, lookup: lookup} - s.cleanupTopicQueries(now) -} - -func (s *ticketStore) cleanupTopicQueries(now mclock.AbsTime) { - if s.nextTopicQueryCleanup > now { - return - } - exp := now - mclock.AbsTime(topicQueryResend) - for n, qq := range s.queriesSent { - for h, q := range qq { - if q.sent < exp { - delete(qq, h) - } - } - if len(qq) == 0 { - delete(s.queriesSent, n) - } - } - s.nextTopicQueryCleanup = now + mclock.AbsTime(topicQueryTimeout) -} - -func (s *ticketStore) gotTopicNodes(from *Node, hash common.Hash, nodes []rpcNode) (timeout bool) { - now := mclock.Now() - //fmt.Println("got", from.addr().String(), hash, len(nodes)) - qq := s.queriesSent[from] - if qq == nil { - return true - } - q, ok := qq[hash] - if !ok || now > q.sent+mclock.AbsTime(topicQueryTimeout) { - return true - } - inside := float64(0) - if len(nodes) > 0 { - inside = 1 - } - s.radius[q.lookup.topic].adjust(now, q.lookup.target, from.sha, inside) - chn := s.searchTopicMap[q.lookup.topic].foundChn - if chn == nil { - //fmt.Println("no channel") - return false - } - for _, node := range nodes { - ip := node.IP - if ip.IsUnspecified() || ip.IsLoopback() { - ip = from.IP - } - n := NewNode(node.ID, ip, node.UDP, node.TCP) - select { - case chn <- n: - default: - return false - } - } - return false -} - -type topicRadius struct { - topic Topic - topicHashPrefix uint64 - radius, minRadius uint64 - buckets []topicRadiusBucket - converged bool - radiusLookupCnt int -} - -type topicRadiusEvent int - -const ( - trOutside topicRadiusEvent = iota - trInside - trNoAdjust - trCount -) - -type topicRadiusBucket struct { - weights [trCount]float64 - lastTime mclock.AbsTime - value float64 - lookupSent map[common.Hash]mclock.AbsTime -} - -func (b *topicRadiusBucket) update(now mclock.AbsTime) { - if now == b.lastTime { - return - } - exp := math.Exp(-float64(now-b.lastTime) / float64(radiusTC)) - for i, w := range b.weights { - b.weights[i] = w * exp - } - b.lastTime = now - - for target, tm := range b.lookupSent { - if now-tm > mclock.AbsTime(respTimeout) { - b.weights[trNoAdjust] += 1 - delete(b.lookupSent, target) - } - } -} - -func (b *topicRadiusBucket) adjust(now mclock.AbsTime, inside float64) { - b.update(now) - if inside <= 0 { - b.weights[trOutside] += 1 - } else { - if inside >= 1 { - b.weights[trInside] += 1 - } else { - b.weights[trInside] += inside - b.weights[trOutside] += 1 - inside - } - } -} - -func newTopicRadius(t Topic) *topicRadius { - topicHash := crypto.Keccak256Hash([]byte(t)) - topicHashPrefix := binary.BigEndian.Uint64(topicHash[0:8]) - - return &topicRadius{ - topic: t, - topicHashPrefix: topicHashPrefix, - radius: maxRadius, - minRadius: maxRadius, - } -} - -func (r *topicRadius) getBucketIdx(addrHash common.Hash) int { - prefix := binary.BigEndian.Uint64(addrHash[0:8]) - var log2 float64 - if prefix != r.topicHashPrefix { - log2 = math.Log2(float64(prefix ^ r.topicHashPrefix)) - } - bucket := int((64 - log2) * radiusBucketsPerBit) - max := 64*radiusBucketsPerBit - 1 - if bucket > max { - return max - } - if bucket < 0 { - return 0 - } - return bucket -} - -func (r *topicRadius) targetForBucket(bucket int) common.Hash { - min := math.Pow(2, 64-float64(bucket+1)/radiusBucketsPerBit) - max := math.Pow(2, 64-float64(bucket)/radiusBucketsPerBit) - a := uint64(min) - b := randUint64n(uint64(max - min)) - xor := a + b - if xor < a { - xor = ^uint64(0) - } - prefix := r.topicHashPrefix ^ xor - var target common.Hash - binary.BigEndian.PutUint64(target[0:8], prefix) - globalRandRead(target[8:]) - return target -} - -// package rand provides a Read function in Go 1.6 and later, but -// we can't use it yet because we still support Go 1.5. -func globalRandRead(b []byte) { - pos := 0 - val := 0 - for n := 0; n < len(b); n++ { - if pos == 0 { - val = rand.Int() - pos = 7 - } - b[n] = byte(val) - val >>= 8 - pos-- - } -} - -func (r *topicRadius) chooseLookupBucket(a, b int) int { - if a < 0 { - a = 0 - } - if a > b { - return -1 - } - c := 0 - for i := a; i <= b; i++ { - if i >= len(r.buckets) || r.buckets[i].weights[trNoAdjust] < maxNoAdjust { - c++ - } - } - if c == 0 { - return -1 - } - rnd := randUint(uint32(c)) - for i := a; i <= b; i++ { - if i >= len(r.buckets) || r.buckets[i].weights[trNoAdjust] < maxNoAdjust { - if rnd == 0 { - return i - } - rnd-- - } - } - panic(nil) // should never happen -} - -func (r *topicRadius) needMoreLookups(a, b int, maxValue float64) bool { - var max float64 - if a < 0 { - a = 0 - } - if b >= len(r.buckets) { - b = len(r.buckets) - 1 - if r.buckets[b].value > max { - max = r.buckets[b].value - } - } - if b >= a { - for i := a; i <= b; i++ { - if r.buckets[i].value > max { - max = r.buckets[i].value - } - } - } - return maxValue-max < minPeakSize -} - -func (r *topicRadius) recalcRadius() (radius uint64, radiusLookup int) { - maxBucket := 0 - maxValue := float64(0) - now := mclock.Now() - v := float64(0) - for i := range r.buckets { - r.buckets[i].update(now) - v += r.buckets[i].weights[trOutside] - r.buckets[i].weights[trInside] - r.buckets[i].value = v - //fmt.Printf("%v %v | ", v, r.buckets[i].weights[trNoAdjust]) - } - //fmt.Println() - slopeCross := -1 - for i, b := range r.buckets { - v := b.value - if v < float64(i)*minSlope { - slopeCross = i - break - } - if v > maxValue { - maxValue = v - maxBucket = i + 1 - } - } - - minRadBucket := len(r.buckets) - sum := float64(0) - for minRadBucket > 0 && sum < minRightSum { - minRadBucket-- - b := r.buckets[minRadBucket] - sum += b.weights[trInside] + b.weights[trOutside] - } - r.minRadius = uint64(math.Pow(2, 64-float64(minRadBucket)/radiusBucketsPerBit)) - - lookupLeft := -1 - if r.needMoreLookups(0, maxBucket-lookupWidth-1, maxValue) { - lookupLeft = r.chooseLookupBucket(maxBucket-lookupWidth, maxBucket-1) - } - lookupRight := -1 - if slopeCross != maxBucket && (minRadBucket <= maxBucket || r.needMoreLookups(maxBucket+lookupWidth, len(r.buckets)-1, maxValue)) { - for len(r.buckets) <= maxBucket+lookupWidth { - r.buckets = append(r.buckets, topicRadiusBucket{lookupSent: make(map[common.Hash]mclock.AbsTime)}) - } - lookupRight = r.chooseLookupBucket(maxBucket, maxBucket+lookupWidth-1) - } - if lookupLeft == -1 { - radiusLookup = lookupRight - } else { - if lookupRight == -1 { - radiusLookup = lookupLeft - } else { - if randUint(2) == 0 { - radiusLookup = lookupLeft - } else { - radiusLookup = lookupRight - } - } - } - - //fmt.Println("mb", maxBucket, "sc", slopeCross, "mrb", minRadBucket, "ll", lookupLeft, "lr", lookupRight, "mv", maxValue) - - if radiusLookup == -1 { - // no more radius lookups needed at the moment, return a radius - r.converged = true - rad := maxBucket - if minRadBucket < rad { - rad = minRadBucket - } - radius = ^uint64(0) - if rad > 0 { - radius = uint64(math.Pow(2, 64-float64(rad)/radiusBucketsPerBit)) - } - r.radius = radius - } - - return -} - -func (r *topicRadius) nextTarget(forceRegular bool) lookupInfo { - if !forceRegular { - _, radiusLookup := r.recalcRadius() - if radiusLookup != -1 { - target := r.targetForBucket(radiusLookup) - r.buckets[radiusLookup].lookupSent[target] = mclock.Now() - return lookupInfo{target: target, topic: r.topic, radiusLookup: true} - } - } - - radExt := r.radius / 2 - if radExt > maxRadius-r.radius { - radExt = maxRadius - r.radius - } - rnd := randUint64n(r.radius) + randUint64n(2*radExt) - if rnd > radExt { - rnd -= radExt - } else { - rnd = radExt - rnd - } - - prefix := r.topicHashPrefix ^ rnd - var target common.Hash - binary.BigEndian.PutUint64(target[0:8], prefix) - globalRandRead(target[8:]) - return lookupInfo{target: target, topic: r.topic, radiusLookup: false} -} - -func (r *topicRadius) adjustWithTicket(now mclock.AbsTime, targetHash common.Hash, t ticketRef) { - wait := t.t.regTime[t.idx] - t.t.issueTime - inside := float64(wait)/float64(targetWaitTime) - 0.5 - if inside > 1 { - inside = 1 - } - if inside < 0 { - inside = 0 - } - r.adjust(now, targetHash, t.t.node.sha, inside) -} - -func (r *topicRadius) adjust(now mclock.AbsTime, targetHash, addrHash common.Hash, inside float64) { - bucket := r.getBucketIdx(addrHash) - //fmt.Println("adjust", bucket, len(r.buckets), inside) - if bucket >= len(r.buckets) { - return - } - r.buckets[bucket].adjust(now, inside) - delete(r.buckets[bucket].lookupSent, targetHash) -} diff --git a/p2p/discv5/topic.go b/p2p/discv5/topic.go deleted file mode 100644 index 609a41297f85..000000000000 --- a/p2p/discv5/topic.go +++ /dev/null @@ -1,407 +0,0 @@ -// Copyright 2016 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -package discv5 - -import ( - "container/heap" - "fmt" - "math" - "math/rand" - "time" - - "github.com/ethereum/go-ethereum/common/mclock" - "github.com/ethereum/go-ethereum/log" -) - -const ( - maxEntries = 10000 - maxEntriesPerTopic = 50 - - fallbackRegistrationExpiry = 1 * time.Hour -) - -type Topic string - -type topicEntry struct { - topic Topic - fifoIdx uint64 - node *Node - expire mclock.AbsTime -} - -type topicInfo struct { - entries map[uint64]*topicEntry - fifoHead, fifoTail uint64 - rqItem *topicRequestQueueItem - wcl waitControlLoop -} - -// removes tail element from the fifo -func (t *topicInfo) getFifoTail() *topicEntry { - for t.entries[t.fifoTail] == nil { - t.fifoTail++ - } - tail := t.entries[t.fifoTail] - t.fifoTail++ - return tail -} - -type nodeInfo struct { - entries map[Topic]*topicEntry - lastIssuedTicket, lastUsedTicket uint32 - // you can't register a ticket newer than lastUsedTicket before noRegUntil (absolute time) - noRegUntil mclock.AbsTime -} - -type topicTable struct { - db *nodeDB - self *Node - nodes map[*Node]*nodeInfo - topics map[Topic]*topicInfo - globalEntries uint64 - requested topicRequestQueue - requestCnt uint64 - lastGarbageCollection mclock.AbsTime -} - -func newTopicTable(db *nodeDB, self *Node) *topicTable { - if printTestImgLogs { - fmt.Printf("*N %016x\n", self.sha[:8]) - } - return &topicTable{ - db: db, - nodes: make(map[*Node]*nodeInfo), - topics: make(map[Topic]*topicInfo), - self: self, - } -} - -func (t *topicTable) getOrNewTopic(topic Topic) *topicInfo { - ti := t.topics[topic] - if ti == nil { - rqItem := &topicRequestQueueItem{ - topic: topic, - priority: t.requestCnt, - } - ti = &topicInfo{ - entries: make(map[uint64]*topicEntry), - rqItem: rqItem, - } - t.topics[topic] = ti - heap.Push(&t.requested, rqItem) - } - return ti -} - -func (t *topicTable) checkDeleteTopic(topic Topic) { - ti := t.topics[topic] - if ti == nil { - return - } - if len(ti.entries) == 0 && ti.wcl.hasMinimumWaitPeriod() { - delete(t.topics, topic) - heap.Remove(&t.requested, ti.rqItem.index) - } -} - -func (t *topicTable) getOrNewNode(node *Node) *nodeInfo { - n := t.nodes[node] - if n == nil { - //fmt.Printf("newNode %016x %016x\n", t.self.sha[:8], node.sha[:8]) - var issued, used uint32 - if t.db != nil { - issued, used = t.db.fetchTopicRegTickets(node.ID) - } - n = &nodeInfo{ - entries: make(map[Topic]*topicEntry), - lastIssuedTicket: issued, - lastUsedTicket: used, - } - t.nodes[node] = n - } - return n -} - -func (t *topicTable) checkDeleteNode(node *Node) { - if n, ok := t.nodes[node]; ok && len(n.entries) == 0 && n.noRegUntil < mclock.Now() { - //fmt.Printf("deleteNode %016x %016x\n", t.self.sha[:8], node.sha[:8]) - delete(t.nodes, node) - } -} - -func (t *topicTable) storeTicketCounters(node *Node) { - n := t.getOrNewNode(node) - if t.db != nil { - t.db.updateTopicRegTickets(node.ID, n.lastIssuedTicket, n.lastUsedTicket) - } -} - -func (t *topicTable) getEntries(topic Topic) []*Node { - t.collectGarbage() - - te := t.topics[topic] - if te == nil { - return nil - } - nodes := make([]*Node, len(te.entries)) - i := 0 - for _, e := range te.entries { - nodes[i] = e.node - i++ - } - t.requestCnt++ - t.requested.update(te.rqItem, t.requestCnt) - return nodes -} - -func (t *topicTable) addEntry(node *Node, topic Topic) { - n := t.getOrNewNode(node) - // clear previous entries by the same node - for _, e := range n.entries { - t.deleteEntry(e) - } - // *** - n = t.getOrNewNode(node) - - tm := mclock.Now() - te := t.getOrNewTopic(topic) - - if len(te.entries) == maxEntriesPerTopic { - t.deleteEntry(te.getFifoTail()) - } - - if t.globalEntries == maxEntries { - t.deleteEntry(t.leastRequested()) // not empty, no need to check for nil - } - - fifoIdx := te.fifoHead - te.fifoHead++ - entry := &topicEntry{ - topic: topic, - fifoIdx: fifoIdx, - node: node, - expire: tm + mclock.AbsTime(fallbackRegistrationExpiry), - } - if printTestImgLogs { - fmt.Printf("*+ %d %v %016x %016x\n", tm/1000000, topic, t.self.sha[:8], node.sha[:8]) - } - te.entries[fifoIdx] = entry - n.entries[topic] = entry - t.globalEntries++ - te.wcl.registered(tm) -} - -// removes least requested element from the fifo -func (t *topicTable) leastRequested() *topicEntry { - for t.requested.Len() > 0 && t.topics[t.requested[0].topic] == nil { - heap.Pop(&t.requested) - } - if t.requested.Len() == 0 { - return nil - } - return t.topics[t.requested[0].topic].getFifoTail() -} - -// entry should exist -func (t *topicTable) deleteEntry(e *topicEntry) { - if printTestImgLogs { - fmt.Printf("*- %d %v %016x %016x\n", mclock.Now()/1000000, e.topic, t.self.sha[:8], e.node.sha[:8]) - } - ne := t.nodes[e.node].entries - delete(ne, e.topic) - if len(ne) == 0 { - t.checkDeleteNode(e.node) - } - te := t.topics[e.topic] - delete(te.entries, e.fifoIdx) - if len(te.entries) == 0 { - t.checkDeleteTopic(e.topic) - } - t.globalEntries-- -} - -// It is assumed that topics and waitPeriods have the same length. -func (t *topicTable) useTicket(node *Node, serialNo uint32, topics []Topic, idx int, issueTime uint64, waitPeriods []uint32) (registered bool) { - log.Trace("Using discovery ticket", "serial", serialNo, "topics", topics, "waits", waitPeriods) - //fmt.Println("useTicket", serialNo, topics, waitPeriods) - t.collectGarbage() - - n := t.getOrNewNode(node) - if serialNo < n.lastUsedTicket { - return false - } - - tm := mclock.Now() - if serialNo > n.lastUsedTicket && tm < n.noRegUntil { - return false - } - if serialNo != n.lastUsedTicket { - n.lastUsedTicket = serialNo - n.noRegUntil = tm + mclock.AbsTime(noRegTimeout()) - t.storeTicketCounters(node) - } - - currTime := uint64(tm / mclock.AbsTime(time.Second)) - regTime := issueTime + uint64(waitPeriods[idx]) - relTime := int64(currTime - regTime) - if relTime >= -1 && relTime <= regTimeWindow+1 { // give clients a little security margin on both ends - if e := n.entries[topics[idx]]; e == nil { - t.addEntry(node, topics[idx]) - } else { - // if there is an active entry, don't move to the front of the FIFO but prolong expire time - e.expire = tm + mclock.AbsTime(fallbackRegistrationExpiry) - } - return true - } - - return false -} - -func (t *topicTable) getTicket(node *Node, topics []Topic) *ticket { - t.collectGarbage() - - now := mclock.Now() - n := t.getOrNewNode(node) - n.lastIssuedTicket++ - t.storeTicketCounters(node) - - tic := &ticket{ - issueTime: now, - topics: topics, - serial: n.lastIssuedTicket, - regTime: make([]mclock.AbsTime, len(topics)), - } - for i, topic := range topics { - var waitPeriod time.Duration - if topic := t.topics[topic]; topic != nil { - waitPeriod = topic.wcl.waitPeriod - } else { - waitPeriod = minWaitPeriod - } - - tic.regTime[i] = now + mclock.AbsTime(waitPeriod) - } - return tic -} - -const gcInterval = time.Minute - -func (t *topicTable) collectGarbage() { - tm := mclock.Now() - if time.Duration(tm-t.lastGarbageCollection) < gcInterval { - return - } - t.lastGarbageCollection = tm - - for node, n := range t.nodes { - for _, e := range n.entries { - if e.expire <= tm { - t.deleteEntry(e) - } - } - - t.checkDeleteNode(node) - } - - for topic := range t.topics { - t.checkDeleteTopic(topic) - } -} - -const ( - minWaitPeriod = time.Minute - regTimeWindow = 10 // seconds - avgnoRegTimeout = time.Minute * 10 - // target average interval between two incoming ad requests - wcTargetRegInterval = time.Minute * 10 / maxEntriesPerTopic - // - wcTimeConst = time.Minute * 10 -) - -// initialization is not required, will set to minWaitPeriod at first registration -type waitControlLoop struct { - lastIncoming mclock.AbsTime - waitPeriod time.Duration -} - -func (w *waitControlLoop) registered(tm mclock.AbsTime) { - w.waitPeriod = w.nextWaitPeriod(tm) - w.lastIncoming = tm -} - -func (w *waitControlLoop) nextWaitPeriod(tm mclock.AbsTime) time.Duration { - period := tm - w.lastIncoming - wp := time.Duration(float64(w.waitPeriod) * math.Exp((float64(wcTargetRegInterval)-float64(period))/float64(wcTimeConst))) - if wp < minWaitPeriod { - wp = minWaitPeriod - } - return wp -} - -func (w *waitControlLoop) hasMinimumWaitPeriod() bool { - return w.nextWaitPeriod(mclock.Now()) == minWaitPeriod -} - -func noRegTimeout() time.Duration { - e := rand.ExpFloat64() - if e > 100 { - e = 100 - } - return time.Duration(float64(avgnoRegTimeout) * e) -} - -type topicRequestQueueItem struct { - topic Topic - priority uint64 - index int -} - -// A topicRequestQueue implements heap.Interface and holds topicRequestQueueItems. -type topicRequestQueue []*topicRequestQueueItem - -func (tq topicRequestQueue) Len() int { return len(tq) } - -func (tq topicRequestQueue) Less(i, j int) bool { - return tq[i].priority < tq[j].priority -} - -func (tq topicRequestQueue) Swap(i, j int) { - tq[i], tq[j] = tq[j], tq[i] - tq[i].index = i - tq[j].index = j -} - -func (tq *topicRequestQueue) Push(x interface{}) { - n := len(*tq) - item := x.(*topicRequestQueueItem) - item.index = n - *tq = append(*tq, item) -} - -func (tq *topicRequestQueue) Pop() interface{} { - old := *tq - n := len(old) - item := old[n-1] - item.index = -1 - *tq = old[0 : n-1] - return item -} - -func (tq *topicRequestQueue) update(item *topicRequestQueueItem, priority uint64) { - item.priority = priority - heap.Fix(tq, item.index) -} diff --git a/p2p/discv5/topic_test.go b/p2p/discv5/topic_test.go deleted file mode 100644 index ba79993f29e9..000000000000 --- a/p2p/discv5/topic_test.go +++ /dev/null @@ -1,71 +0,0 @@ -// Copyright 2016 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -package discv5 - -import ( - "encoding/binary" - "testing" - "time" - - "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/common/mclock" -) - -func TestTopicRadius(t *testing.T) { - now := mclock.Now() - topic := Topic("qwerty") - rad := newTopicRadius(topic) - targetRad := (^uint64(0)) / 100 - - waitFn := func(addr common.Hash) time.Duration { - prefix := binary.BigEndian.Uint64(addr[0:8]) - dist := prefix ^ rad.topicHashPrefix - relDist := float64(dist) / float64(targetRad) - relTime := (1 - relDist/2) * 2 - if relTime < 0 { - relTime = 0 - } - return time.Duration(float64(targetWaitTime) * relTime) - } - - bcnt := 0 - cnt := 0 - var sum float64 - for cnt < 100 { - addr := rad.nextTarget(false).target - wait := waitFn(addr) - ticket := &ticket{ - topics: []Topic{topic}, - regTime: []mclock.AbsTime{mclock.AbsTime(wait)}, - node: &Node{nodeNetGuts: nodeNetGuts{sha: addr}}, - } - rad.adjustWithTicket(now, addr, ticketRef{ticket, 0}) - if rad.radius != maxRadius { - cnt++ - sum += float64(rad.radius) - } else { - bcnt++ - if bcnt > 500 { - t.Errorf("Radius did not converge in 500 iterations") - } - } - } - avgRel := sum / float64(cnt) / float64(targetRad) - if avgRel > 1.05 || avgRel < 0.95 { - t.Errorf("Average/target ratio is too far from 1 (%v)", avgRel) - } -} diff --git a/p2p/discv5/udp.go b/p2p/discv5/udp.go deleted file mode 100644 index 088f95cac6a2..000000000000 --- a/p2p/discv5/udp.go +++ /dev/null @@ -1,429 +0,0 @@ -// Copyright 2016 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -package discv5 - -import ( - "bytes" - "crypto/ecdsa" - "errors" - "fmt" - "net" - "time" - - "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/crypto" - "github.com/ethereum/go-ethereum/log" - "github.com/ethereum/go-ethereum/p2p/netutil" - "github.com/ethereum/go-ethereum/rlp" -) - -const Version = 4 - -// Errors -var ( - errPacketTooSmall = errors.New("too small") - errBadPrefix = errors.New("bad prefix") -) - -// Timeouts -const ( - respTimeout = 500 * time.Millisecond - expiration = 20 * time.Second -) - -// RPC request structures -type ( - ping struct { - Version uint - From, To rpcEndpoint - Expiration uint64 - - // v5 - Topics []Topic - - // Ignore additional fields (for forward compatibility). - Rest []rlp.RawValue `rlp:"tail"` - } - - // pong is the reply to ping. - pong struct { - // This field should mirror the UDP envelope address - // of the ping packet, which provides a way to discover the - // the external address (after NAT). - To rpcEndpoint - - ReplyTok []byte // This contains the hash of the ping packet. - Expiration uint64 // Absolute timestamp at which the packet becomes invalid. - - // v5 - TopicHash common.Hash - TicketSerial uint32 - WaitPeriods []uint32 - - // Ignore additional fields (for forward compatibility). - Rest []rlp.RawValue `rlp:"tail"` - } - - // findnode is a query for nodes close to the given target. - findnode struct { - Target NodeID // doesn't need to be an actual public key - Expiration uint64 - // Ignore additional fields (for forward compatibility). - Rest []rlp.RawValue `rlp:"tail"` - } - - // findnode is a query for nodes close to the given target. - findnodeHash struct { - Target common.Hash - Expiration uint64 - // Ignore additional fields (for forward compatibility). - Rest []rlp.RawValue `rlp:"tail"` - } - - // reply to findnode - neighbors struct { - Nodes []rpcNode - Expiration uint64 - // Ignore additional fields (for forward compatibility). - Rest []rlp.RawValue `rlp:"tail"` - } - - topicRegister struct { - Topics []Topic - Idx uint - Pong []byte - } - - topicQuery struct { - Topic Topic - Expiration uint64 - } - - // reply to topicQuery - topicNodes struct { - Echo common.Hash - Nodes []rpcNode - } - - rpcNode struct { - IP net.IP // len 4 for IPv4 or 16 for IPv6 - UDP uint16 // for discovery protocol - TCP uint16 // for RLPx protocol - ID NodeID - } - - rpcEndpoint struct { - IP net.IP // len 4 for IPv4 or 16 for IPv6 - UDP uint16 // for discovery protocol - TCP uint16 // for RLPx protocol - } -) - -var ( - versionPrefix = []byte("temporary discovery v5") - versionPrefixSize = len(versionPrefix) - sigSize = 520 / 8 - headSize = versionPrefixSize + sigSize // space of packet frame data -) - -// Neighbors replies are sent across multiple packets to -// stay below the 1280 byte limit. We compute the maximum number -// of entries by stuffing a packet until it grows too large. -var maxNeighbors = func() int { - p := neighbors{Expiration: ^uint64(0)} - maxSizeNode := rpcNode{IP: make(net.IP, 16), UDP: ^uint16(0), TCP: ^uint16(0)} - for n := 0; ; n++ { - p.Nodes = append(p.Nodes, maxSizeNode) - size, _, err := rlp.EncodeToReader(p) - if err != nil { - // If this ever happens, it will be caught by the unit tests. - panic("cannot encode: " + err.Error()) - } - if headSize+size+1 >= 1280 { - return n - } - } -}() - -var maxTopicNodes = func() int { - p := topicNodes{} - maxSizeNode := rpcNode{IP: make(net.IP, 16), UDP: ^uint16(0), TCP: ^uint16(0)} - for n := 0; ; n++ { - p.Nodes = append(p.Nodes, maxSizeNode) - size, _, err := rlp.EncodeToReader(p) - if err != nil { - // If this ever happens, it will be caught by the unit tests. - panic("cannot encode: " + err.Error()) - } - if headSize+size+1 >= 1280 { - return n - } - } -}() - -func makeEndpoint(addr *net.UDPAddr, tcpPort uint16) rpcEndpoint { - ip := addr.IP.To4() - if ip == nil { - ip = addr.IP.To16() - } - return rpcEndpoint{IP: ip, UDP: uint16(addr.Port), TCP: tcpPort} -} - -func nodeFromRPC(sender *net.UDPAddr, rn rpcNode) (*Node, error) { - if err := netutil.CheckRelayIP(sender.IP, rn.IP); err != nil { - return nil, err - } - n := NewNode(rn.ID, rn.IP, rn.UDP, rn.TCP) - err := n.validateComplete() - return n, err -} - -func nodeToRPC(n *Node) rpcNode { - return rpcNode{ID: n.ID, IP: n.IP, UDP: n.UDP, TCP: n.TCP} -} - -type ingressPacket struct { - remoteID NodeID - remoteAddr *net.UDPAddr - ev nodeEvent - hash []byte - data interface{} // one of the RPC structs - rawData []byte -} - -type conn interface { - ReadFromUDP(b []byte) (n int, addr *net.UDPAddr, err error) - WriteToUDP(b []byte, addr *net.UDPAddr) (n int, err error) - Close() error - LocalAddr() net.Addr -} - -// udp implements the RPC protocol. -type udp struct { - conn conn - priv *ecdsa.PrivateKey - ourEndpoint rpcEndpoint - net *Network -} - -// ListenUDP returns a new table that listens for UDP packets on laddr. -func ListenUDP(priv *ecdsa.PrivateKey, conn conn, nodeDBPath string, netrestrict *netutil.Netlist) (*Network, error) { - realaddr := conn.LocalAddr().(*net.UDPAddr) - transport, err := listenUDP(priv, conn, realaddr) - if err != nil { - return nil, err - } - net, err := newNetwork(transport, priv.PublicKey, nodeDBPath, netrestrict) - if err != nil { - return nil, err - } - log.Info("UDP listener up", "net", net.tab.self) - transport.net = net - go transport.readLoop() - return net, nil -} - -func listenUDP(priv *ecdsa.PrivateKey, conn conn, realaddr *net.UDPAddr) (*udp, error) { - return &udp{conn: conn, priv: priv, ourEndpoint: makeEndpoint(realaddr, uint16(realaddr.Port))}, nil -} - -func (t *udp) localAddr() *net.UDPAddr { - return t.conn.LocalAddr().(*net.UDPAddr) -} - -func (t *udp) Close() { - t.conn.Close() -} - -func (t *udp) send(remote *Node, ptype nodeEvent, data interface{}) (hash []byte) { - hash, _ = t.sendPacket(remote.ID, remote.addr(), byte(ptype), data) - return hash -} - -func (t *udp) sendPing(remote *Node, toaddr *net.UDPAddr, topics []Topic) (hash []byte) { - hash, _ = t.sendPacket(remote.ID, toaddr, byte(pingPacket), ping{ - Version: Version, - From: t.ourEndpoint, - To: makeEndpoint(toaddr, uint16(toaddr.Port)), // TODO: maybe use known TCP port from DB - Expiration: uint64(time.Now().Add(expiration).Unix()), - Topics: topics, - }) - return hash -} - -func (t *udp) sendNeighbours(remote *Node, results []*Node) { - // Send neighbors in chunks with at most maxNeighbors per packet - // to stay below the 1280 byte limit. - p := neighbors{Expiration: uint64(time.Now().Add(expiration).Unix())} - for i, result := range results { - p.Nodes = append(p.Nodes, nodeToRPC(result)) - if len(p.Nodes) == maxNeighbors || i == len(results)-1 { - t.sendPacket(remote.ID, remote.addr(), byte(neighborsPacket), p) - p.Nodes = p.Nodes[:0] - } - } -} - -func (t *udp) sendFindnodeHash(remote *Node, target common.Hash) { - t.sendPacket(remote.ID, remote.addr(), byte(findnodeHashPacket), findnodeHash{ - Target: target, - Expiration: uint64(time.Now().Add(expiration).Unix()), - }) -} - -func (t *udp) sendTopicRegister(remote *Node, topics []Topic, idx int, pong []byte) { - t.sendPacket(remote.ID, remote.addr(), byte(topicRegisterPacket), topicRegister{ - Topics: topics, - Idx: uint(idx), - Pong: pong, - }) -} - -func (t *udp) sendTopicNodes(remote *Node, queryHash common.Hash, nodes []*Node) { - p := topicNodes{Echo: queryHash} - var sent bool - for _, result := range nodes { - if result.IP.Equal(t.net.tab.self.IP) || netutil.CheckRelayIP(remote.IP, result.IP) == nil { - p.Nodes = append(p.Nodes, nodeToRPC(result)) - } - if len(p.Nodes) == maxTopicNodes { - t.sendPacket(remote.ID, remote.addr(), byte(topicNodesPacket), p) - p.Nodes = p.Nodes[:0] - sent = true - } - } - if !sent || len(p.Nodes) > 0 { - t.sendPacket(remote.ID, remote.addr(), byte(topicNodesPacket), p) - } -} - -func (t *udp) sendPacket(toid NodeID, toaddr *net.UDPAddr, ptype byte, req interface{}) (hash []byte, err error) { - //fmt.Println("sendPacket", nodeEvent(ptype), toaddr.String(), toid.String()) - packet, hash, err := encodePacket(t.priv, ptype, req) - if err != nil { - //fmt.Println(err) - return hash, err - } - log.Trace(fmt.Sprintf(">>> %v to %x@%v", nodeEvent(ptype), toid[:8], toaddr)) - if nbytes, err := t.conn.WriteToUDP(packet, toaddr); err != nil { - log.Trace(fmt.Sprint("UDP send failed:", err)) - } else { - egressTrafficMeter.Mark(int64(nbytes)) - } - //fmt.Println(err) - return hash, err -} - -// zeroed padding space for encodePacket. -var headSpace = make([]byte, headSize) - -func encodePacket(priv *ecdsa.PrivateKey, ptype byte, req interface{}) (p, hash []byte, err error) { - b := new(bytes.Buffer) - b.Write(headSpace) - b.WriteByte(ptype) - if err := rlp.Encode(b, req); err != nil { - log.Error(fmt.Sprint("error encoding packet:", err)) - return nil, nil, err - } - packet := b.Bytes() - sig, err := crypto.Sign(crypto.Keccak256(packet[headSize:]), priv) - if err != nil { - log.Error(fmt.Sprint("could not sign packet:", err)) - return nil, nil, err - } - copy(packet, versionPrefix) - copy(packet[versionPrefixSize:], sig) - hash = crypto.Keccak256(packet[versionPrefixSize:]) - return packet, hash, nil -} - -// readLoop runs in its own goroutine. it injects ingress UDP packets -// into the network loop. -func (t *udp) readLoop() { - defer t.conn.Close() - // 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. - buf := make([]byte, 1280) - for { - nbytes, from, err := t.conn.ReadFromUDP(buf) - ingressTrafficMeter.Mark(int64(nbytes)) - if netutil.IsTemporaryError(err) { - // Ignore temporary read errors. - log.Debug(fmt.Sprintf("Temporary read error: %v", err)) - continue - } else if err != nil { - // Shut down the loop for permament errors. - log.Debug(fmt.Sprintf("Read error: %v", err)) - return - } - t.handlePacket(from, buf[:nbytes]) - } -} - -func (t *udp) handlePacket(from *net.UDPAddr, buf []byte) error { - pkt := ingressPacket{remoteAddr: from} - if err := decodePacket(buf, &pkt); err != nil { - log.Debug(fmt.Sprintf("Bad packet from %v: %v", from, err)) - //fmt.Println("bad packet", err) - return err - } - t.net.reqReadPacket(pkt) - return nil -} - -func decodePacket(buffer []byte, pkt *ingressPacket) error { - if len(buffer) < headSize+1 { - return errPacketTooSmall - } - buf := make([]byte, len(buffer)) - copy(buf, buffer) - prefix, sig, sigdata := buf[:versionPrefixSize], buf[versionPrefixSize:headSize], buf[headSize:] - if !bytes.Equal(prefix, versionPrefix) { - return errBadPrefix - } - fromID, err := recoverNodeID(crypto.Keccak256(buf[headSize:]), sig) - if err != nil { - return err - } - pkt.rawData = buf - pkt.hash = crypto.Keccak256(buf[versionPrefixSize:]) - pkt.remoteID = fromID - switch pkt.ev = nodeEvent(sigdata[0]); pkt.ev { - case pingPacket: - pkt.data = new(ping) - case pongPacket: - pkt.data = new(pong) - case findnodePacket: - pkt.data = new(findnode) - case neighborsPacket: - pkt.data = new(neighbors) - case findnodeHashPacket: - pkt.data = new(findnodeHash) - case topicRegisterPacket: - pkt.data = new(topicRegister) - case topicQueryPacket: - pkt.data = new(topicQuery) - case topicNodesPacket: - pkt.data = new(topicNodes) - default: - return fmt.Errorf("unknown packet type: %d", sigdata[0]) - } - s := rlp.NewStream(bytes.NewReader(sigdata[1:]), 0) - err = s.Decode(pkt.data) - return err -} From aa799fd0992fcc2e094ed400244003684227d9bc Mon Sep 17 00:00:00 2001 From: Zsolt Felfoldi Date: Tue, 15 Dec 2020 11:01:44 +0100 Subject: [PATCH 05/10] les: simplified setupDiscovery --- les/enr_entry.go | 19 ++++--------------- 1 file changed, 4 insertions(+), 15 deletions(-) diff --git a/les/enr_entry.go b/les/enr_entry.go index eec415712030..16d81c22129b 100644 --- a/les/enr_entry.go +++ b/les/enr_entry.go @@ -38,28 +38,17 @@ func (e lesEntry) ENRKey() string { // setupDiscovery creates the node discovery source for the eth protocol. func (eth *LightEthereum) setupDiscovery(cfg *p2p.Config) (enode.Iterator, error) { - var it enode.Iterator + it := enode.NewFairMix(0) if len(eth.config.EthDiscoveryURLs) != 0 { client := dnsdisc.NewClient(dnsdisc.Config{}) - var err error - it, err = client.NewIterator(eth.config.EthDiscoveryURLs...) + dns, err := client.NewIterator(eth.config.EthDiscoveryURLs...) if err != nil { return nil, err } + it.AddSource(dns) } if cfg.DiscoveryV5 && eth.p2pServer.DiscV5 != nil { - v5 := eth.p2pServer.DiscV5.RandomNodes() - if it == nil { - it = v5 - } else { - mix := enode.NewFairMix(0) - mix.AddSource(it) - mix.AddSource(v5) - it = mix - } - } - if it == nil { - return nil, nil + it.AddSource(eth.p2pServer.DiscV5.RandomNodes()) } forkFilter := forkid.NewFilter(eth.blockchain) return enode.Filter(it, func(n *enode.Node) bool { From 8825627bb15102f41a608c7ee454643df8c69cfa Mon Sep 17 00:00:00 2001 From: Zsolt Felfoldi Date: Fri, 15 Jan 2021 19:18:58 +0100 Subject: [PATCH 06/10] params: add discv5 bootnodes --- cmd/utils/flags.go | 10 +--------- params/bootnodes.go | 18 ++++++++++++++++++ 2 files changed, 19 insertions(+), 9 deletions(-) diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index 3fe84e26764b..dbcce157b871 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -836,7 +836,7 @@ func setBootstrapNodes(ctx *cli.Context, cfg *p2p.Config) { // setBootstrapNodesV5 creates a list of bootstrap nodes from the command line // flags, reverting to pre-configured ones if none have been specified. func setBootstrapNodesV5(ctx *cli.Context, cfg *p2p.Config) { - urls := params.MainnetBootnodes + urls := params.V5Bootnodes switch { case ctx.GlobalIsSet(BootnodesFlag.Name) || ctx.GlobalIsSet(LegacyBootnodesV5Flag.Name): if ctx.GlobalIsSet(LegacyBootnodesV5Flag.Name) { @@ -844,14 +844,6 @@ func setBootstrapNodesV5(ctx *cli.Context, cfg *p2p.Config) { } else { urls = SplitAndTrim(ctx.GlobalString(BootnodesFlag.Name)) } - case ctx.GlobalBool(RopstenFlag.Name): - urls = params.RopstenBootnodes - case ctx.GlobalBool(RinkebyFlag.Name): - urls = params.RinkebyBootnodes - case ctx.GlobalBool(GoerliFlag.Name): - urls = params.GoerliBootnodes - case ctx.GlobalBool(YoloV2Flag.Name): - urls = params.YoloV2Bootnodes case cfg.BootstrapNodesV5 != nil: return // already set, don't apply defaults. } diff --git a/params/bootnodes.go b/params/bootnodes.go index d4512bf789a5..66f917015e85 100644 --- a/params/bootnodes.go +++ b/params/bootnodes.go @@ -73,6 +73,24 @@ var YoloV2Bootnodes = []string{ "enode://9e1096aa59862a6f164994cb5cb16f5124d6c992cdbf4535ff7dea43ea1512afe5448dca9df1b7ab0726129603f1a3336b631e4d7a1a44c94daddd03241587f9@3.9.20.133:30303", } +var V5Bootnodes = []string{ + // Teku team's bootnode + "enr:-KG4QOtcP9X1FbIMOe17QNMKqDxCpm14jcX5tiOE4_TyMrFqbmhPZHK_ZPG2Gxb1GE2xdtodOfx9-cgvNtxnRyHEmC0ghGV0aDKQ9aX9QgAAAAD__________4JpZIJ2NIJpcIQDE8KdiXNlY3AyNTZrMaEDhpehBDbZjM_L9ek699Y7vhUJ-eAdMyQW_Fil522Y0fODdGNwgiMog3VkcIIjKA", + "enr:-KG4QDyytgmE4f7AnvW-ZaUOIi9i79qX4JwjRAiXBZCU65wOfBu-3Nb5I7b_Rmg3KCOcZM_C3y5pg7EBU5XGrcLTduQEhGV0aDKQ9aX9QgAAAAD__________4JpZIJ2NIJpcIQ2_DUbiXNlY3AyNTZrMaEDKnz_-ps3UUOfHWVYaskI5kWYO_vtYMGYCQRAR3gHDouDdGNwgiMog3VkcIIjKA", + // Prylab team's bootnodes + "enr:-Ku4QImhMc1z8yCiNJ1TyUxdcfNucje3BGwEHzodEZUan8PherEo4sF7pPHPSIB1NNuSg5fZy7qFsjmUKs2ea1Whi0EBh2F0dG5ldHOIAAAAAAAAAACEZXRoMpD1pf1CAAAAAP__________gmlkgnY0gmlwhBLf22SJc2VjcDI1NmsxoQOVphkDqal4QzPMksc5wnpuC3gvSC8AfbFOnZY_On34wIN1ZHCCIyg", + "enr:-Ku4QP2xDnEtUXIjzJ_DhlCRN9SN99RYQPJL92TMlSv7U5C1YnYLjwOQHgZIUXw6c-BvRg2Yc2QsZxxoS_pPRVe0yK8Bh2F0dG5ldHOIAAAAAAAAAACEZXRoMpD1pf1CAAAAAP__________gmlkgnY0gmlwhBLf22SJc2VjcDI1NmsxoQMeFF5GrS7UZpAH2Ly84aLK-TyvH-dRo0JM1i8yygH50YN1ZHCCJxA", + "enr:-Ku4QPp9z1W4tAO8Ber_NQierYaOStqhDqQdOPY3bB3jDgkjcbk6YrEnVYIiCBbTxuar3CzS528d2iE7TdJsrL-dEKoBh2F0dG5ldHOIAAAAAAAAAACEZXRoMpD1pf1CAAAAAP__________gmlkgnY0gmlwhBLf22SJc2VjcDI1NmsxoQMw5fqqkw2hHC4F5HZZDPsNmPdB1Gi8JPQK7pRc9XHh-oN1ZHCCKvg", + // Lighthouse team's bootnodes + "enr:-IS4QLkKqDMy_ExrpOEWa59NiClemOnor-krjp4qoeZwIw2QduPC-q7Kz4u1IOWf3DDbdxqQIgC4fejavBOuUPy-HE4BgmlkgnY0gmlwhCLzAHqJc2VjcDI1NmsxoQLQSJfEAHZApkm5edTCZ_4qps_1k_ub2CxHFxi-gr2JMIN1ZHCCIyg", + "enr:-IS4QDAyibHCzYZmIYZCjXwU9BqpotWmv2BsFlIq1V31BwDDMJPFEbox1ijT5c2Ou3kvieOKejxuaCqIcjxBjJ_3j_cBgmlkgnY0gmlwhAMaHiCJc2VjcDI1NmsxoQJIdpj_foZ02MXz4It8xKD7yUHTBx7lVFn3oeRP21KRV4N1ZHCCIyg", + // EF bootnodes + "enr:-Ku4QHqVeJ8PPICcWk1vSn_XcSkjOkNiTg6Fmii5j6vUQgvzMc9L1goFnLKgXqBJspJjIsB91LTOleFmyWWrFVATGngBh2F0dG5ldHOIAAAAAAAAAACEZXRoMpC1MD8qAAAAAP__________gmlkgnY0gmlwhAMRHkWJc2VjcDI1NmsxoQKLVXFOhp2uX6jeT0DvvDpPcU8FWMjQdR4wMuORMhpX24N1ZHCCIyg", + "enr:-Ku4QG-2_Md3sZIAUebGYT6g0SMskIml77l6yR-M_JXc-UdNHCmHQeOiMLbylPejyJsdAPsTHJyjJB2sYGDLe0dn8uYBh2F0dG5ldHOIAAAAAAAAAACEZXRoMpC1MD8qAAAAAP__________gmlkgnY0gmlwhBLY-NyJc2VjcDI1NmsxoQORcM6e19T1T9gi7jxEZjk_sjVLGFscUNqAY9obgZaxbIN1ZHCCIyg", + "enr:-Ku4QPn5eVhcoF1opaFEvg1b6JNFD2rqVkHQ8HApOKK61OIcIXD127bKWgAtbwI7pnxx6cDyk_nI88TrZKQaGMZj0q0Bh2F0dG5ldHOIAAAAAAAAAACEZXRoMpC1MD8qAAAAAP__________gmlkgnY0gmlwhDayLMaJc2VjcDI1NmsxoQK2sBOLGcUb4AwuYzFuAVCaNHA-dy24UuEKkeFNgCVCsIN1ZHCCIyg", + "enr:-Ku4QEWzdnVtXc2Q0ZVigfCGggOVB2Vc1ZCPEc6j21NIFLODSJbvNaef1g4PxhPwl_3kax86YPheFUSLXPRs98vvYsoBh2F0dG5ldHOIAAAAAAAAAACEZXRoMpC1MD8qAAAAAP__________gmlkgnY0gmlwhDZBrP2Jc2VjcDI1NmsxoQM6jr8Rb1ktLEsVcKAPa08wCsKUmvoQ8khiOl_SLozf9IN1ZHCCIyg", +} + const dnsPrefix = "enrtree://AKA3AM6LPBYEUDMVNU3BSVQJ5AD45Y7YPOHJLEF6W26QOE4VTUDPE@" // KnownDNSNetwork returns the address of a public DNS-based node list for the given From ba27381092cae198dfbed50d2fbf713a2353f6c5 Mon Sep 17 00:00:00 2001 From: Zsolt Felfoldi Date: Fri, 15 Jan 2021 21:05:04 +0100 Subject: [PATCH 07/10] les: fixed discovery init order --- les/client.go | 13 ++++++++----- les/serverpool.go | 12 ++++++++---- les/serverpool_test.go | 3 ++- 3 files changed, 18 insertions(+), 10 deletions(-) diff --git a/les/client.go b/les/client.go index c073d97312b1..4ed08e3805d0 100644 --- a/les/client.go +++ b/les/client.go @@ -72,6 +72,7 @@ type LightEthereum struct { netRPCService *ethapi.PublicNetAPI p2pServer *p2p.Server + p2pConfig *p2p.Config } // New creates an instance of the light client. @@ -109,6 +110,7 @@ func New(stack *node.Node, config *eth.Config) (*LightEthereum, error) { bloomIndexer: eth.NewBloomIndexer(chainDb, params.BloomBitsBlocksClient, params.HelperTrieConfirmations), valueTracker: lpc.NewValueTracker(lespayDb, &mclock.System{}, requestList, time.Minute, 1/float64(time.Hour), 1/float64(time.Hour*100), 1/float64(time.Hour*1000)), p2pServer: stack.Server(), + p2pConfig: &stack.Config().P2P, } peers.subscribe((*vtSubscription)(leth.valueTracker)) @@ -132,11 +134,7 @@ func New(stack *node.Node, config *eth.Config) (*LightEthereum, error) { leth.chainReader = leth.blockchain leth.txPool = light.NewTxPool(leth.chainConfig, leth.blockchain, leth.relay) - discovery, err := leth.setupDiscovery(&stack.Config().P2P) - if err != nil { - return nil, err - } - leth.serverPool = newServerPool(lespayDb, []byte("serverpool:"), leth.valueTracker, discovery, time.Second, nil, &mclock.System{}, config.UltraLightServers) + leth.serverPool = newServerPool(lespayDb, []byte("serverpool:"), leth.valueTracker, time.Second, nil, &mclock.System{}, config.UltraLightServers) peers.subscribe(leth.serverPool) leth.dialCandidates = leth.serverPool.dialIterator leth.retriever.softRequestTimeout = leth.serverPool.getTimeout @@ -300,6 +298,11 @@ func (s *LightEthereum) Protocols() []p2p.Protocol { func (s *LightEthereum) Start() error { log.Warn("Light client mode is an experimental feature") + discovery, err := s.setupDiscovery(s.p2pConfig) + if err != nil { + return err + } + s.serverPool.addSource(discovery) s.serverPool.start() // Start bloom request workers. s.wg.Add(bloomServiceThreads) diff --git a/les/serverpool.go b/les/serverpool.go index 6cf4affff097..ac87acf6da6e 100644 --- a/les/serverpool.go +++ b/les/serverpool.go @@ -131,7 +131,7 @@ var ( ) // newServerPool creates a new server pool -func newServerPool(db ethdb.KeyValueStore, dbKey []byte, vt *lpc.ValueTracker, discovery enode.Iterator, mixTimeout time.Duration, query queryFunc, clock mclock.Clock, trustedURLs []string) *serverPool { +func newServerPool(db ethdb.KeyValueStore, dbKey []byte, vt *lpc.ValueTracker, mixTimeout time.Duration, query queryFunc, clock mclock.Clock, trustedURLs []string) *serverPool { s := &serverPool{ db: db, clock: clock, @@ -147,9 +147,6 @@ func newServerPool(db ethdb.KeyValueStore, dbKey []byte, vt *lpc.ValueTracker, d alwaysConnect := lpc.NewQueueIterator(s.ns, sfAlwaysConnect, sfDisableSelection, true, nil) s.mixSources = append(s.mixSources, knownSelector) s.mixSources = append(s.mixSources, alwaysConnect) - if discovery != nil { - s.mixSources = append(s.mixSources, discovery) - } iter := enode.Iterator(s.mixer) if query != nil { @@ -175,6 +172,13 @@ func newServerPool(db ethdb.KeyValueStore, dbKey []byte, vt *lpc.ValueTracker, d return s } +// addSource adds a node discovery source to the server pool (should be called before start) +func (s *serverPool) addSource(source enode.Iterator) { + if source != nil { + s.mixSources = append(s.mixSources, source) + } +} + // addPreNegFilter installs a node filter mechanism that performs a pre-negotiation query. // Nodes that are filtered out and does not appear on the output iterator are put back // into redialWait state. diff --git a/les/serverpool_test.go b/les/serverpool_test.go index 70a41b74c6f8..3b7ae65d5d90 100644 --- a/les/serverpool_test.go +++ b/les/serverpool_test.go @@ -145,7 +145,8 @@ func (s *serverPoolTest) start() { } s.vt = lpc.NewValueTracker(s.db, s.clock, requestList, time.Minute, 1/float64(time.Hour), 1/float64(time.Hour*100), 1/float64(time.Hour*1000)) - s.sp = newServerPool(s.db, []byte("serverpool:"), s.vt, s.input, 0, testQuery, s.clock, s.trusted) + s.sp = newServerPool(s.db, []byte("serverpool:"), s.vt, 0, testQuery, s.clock, s.trusted) + s.sp.addSource(s.input) s.sp.validSchemes = enode.ValidSchemesForTesting s.sp.unixTime = func() int64 { return int64(s.clock.Now()) / int64(time.Second) } s.disconnect = make(map[int][]int) From 58802af1a08fb41b6168ee0860c6a51604af42f4 Mon Sep 17 00:00:00 2001 From: Zsolt Felfoldi Date: Fri, 15 Jan 2021 21:26:47 +0100 Subject: [PATCH 08/10] les: reverted newRetrieveManager params --- les/client.go | 11 +++++------ les/retrieve.go | 9 +++++---- les/test_helper.go | 3 +-- 3 files changed, 11 insertions(+), 12 deletions(-) diff --git a/les/client.go b/les/client.go index 4ed08e3805d0..2d6c850c5d43 100644 --- a/les/client.go +++ b/les/client.go @@ -114,7 +114,11 @@ func New(stack *node.Node, config *eth.Config) (*LightEthereum, error) { } peers.subscribe((*vtSubscription)(leth.valueTracker)) - leth.retriever = newRetrieveManager(peers, leth.reqDist) + leth.serverPool = newServerPool(lespayDb, []byte("serverpool:"), leth.valueTracker, time.Second, nil, &mclock.System{}, config.UltraLightServers) + peers.subscribe(leth.serverPool) + leth.dialCandidates = leth.serverPool.dialIterator + + leth.retriever = newRetrieveManager(peers, leth.reqDist, leth.serverPool.getTimeout) leth.relay = newLesTxRelay(peers, leth.retriever) leth.odr = NewLesOdr(chainDb, light.DefaultClientIndexerConfig, leth.retriever) @@ -134,11 +138,6 @@ func New(stack *node.Node, config *eth.Config) (*LightEthereum, error) { leth.chainReader = leth.blockchain leth.txPool = light.NewTxPool(leth.chainConfig, leth.blockchain, leth.relay) - leth.serverPool = newServerPool(lespayDb, []byte("serverpool:"), leth.valueTracker, time.Second, nil, &mclock.System{}, config.UltraLightServers) - peers.subscribe(leth.serverPool) - leth.dialCandidates = leth.serverPool.dialIterator - leth.retriever.softRequestTimeout = leth.serverPool.getTimeout - // Set up checkpoint oracle. leth.oracle = leth.setupOracle(stack, genesisHash, config) diff --git a/les/retrieve.go b/les/retrieve.go index e6d08312ed7c..ca4f867ea80f 100644 --- a/les/retrieve.go +++ b/les/retrieve.go @@ -92,11 +92,12 @@ const ( ) // newRetrieveManager creates the retrieve manager -func newRetrieveManager(peers *serverPeerSet, dist *requestDistributor) *retrieveManager { +func newRetrieveManager(peers *serverPeerSet, dist *requestDistributor, srto func() time.Duration) *retrieveManager { return &retrieveManager{ - peers: peers, - dist: dist, - sentReqs: make(map[uint64]*sentReq), + peers: peers, + dist: dist, + sentReqs: make(map[uint64]*sentReq), + softRequestTimeout: srto, } } diff --git a/les/test_helper.go b/les/test_helper.go index 80fc34ce6def..d108a8dacefb 100644 --- a/les/test_helper.go +++ b/les/test_helper.go @@ -514,8 +514,7 @@ func newClientServerEnv(t *testing.T, blocks int, protocol int, callback indexer clock = &mclock.Simulated{} } dist := newRequestDistributor(speers, clock) - rm := newRetrieveManager(speers, dist) - rm.softRequestTimeout = func() time.Duration { return time.Millisecond * 500 } + rm := newRetrieveManager(speers, dist, func() time.Duration { return time.Millisecond * 500 }) odr := NewLesOdr(cdb, light.TestClientIndexerConfig, rm) sindexers := testIndexers(sdb, nil, light.TestServerIndexerConfig, true) From e884e99d92c3f6cdf37d903927edb1e062723c28 Mon Sep 17 00:00:00 2001 From: Zsolt Felfoldi Date: Fri, 15 Jan 2021 21:33:24 +0100 Subject: [PATCH 09/10] les: limit discovery speed when no matching nodes are found --- les/enr_entry.go | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/les/enr_entry.go b/les/enr_entry.go index 16d81c22129b..deaa94c78c45 100644 --- a/les/enr_entry.go +++ b/les/enr_entry.go @@ -17,6 +17,8 @@ package les import ( + "time" + "github.com/ethereum/go-ethereum/core/forkid" "github.com/ethereum/go-ethereum/p2p" "github.com/ethereum/go-ethereum/p2p/dnsdisc" @@ -51,6 +53,7 @@ func (eth *LightEthereum) setupDiscovery(cfg *p2p.Config) (enode.Iterator, error it.AddSource(eth.p2pServer.DiscV5.RandomNodes()) } forkFilter := forkid.NewFilter(eth.blockchain) + var delay time.Duration return enode.Filter(it, func(n *enode.Node) bool { var ( les struct { @@ -61,6 +64,12 @@ func (eth *LightEthereum) setupDiscovery(cfg *p2p.Config) (enode.Iterator, error _ []rlp.RawValue `rlp:"tail"` } ) - return n.Load(enr.WithEntry("les", &les)) == nil && n.Load(enr.WithEntry("eth", ð)) == nil && forkFilter(eth.ForkID) == nil + if n.Load(enr.WithEntry("les", &les)) == nil && n.Load(enr.WithEntry("eth", ð)) == nil && forkFilter(eth.ForkID) == nil { + delay /= 2 + return true + } + delay += time.Millisecond + time.Sleep(delay) + return false }), nil } From 398775cf3af94b72295348ff9358f46ec4945382 Mon Sep 17 00:00:00 2001 From: Felix Lange Date: Tue, 26 Jan 2021 14:01:28 +0100 Subject: [PATCH 10/10] les: remove delay in server node filter --- les/enr_entry.go | 49 +++++++++++++++++++++++------------------------- 1 file changed, 23 insertions(+), 26 deletions(-) diff --git a/les/enr_entry.go b/les/enr_entry.go index deaa94c78c45..1e56c1f1754b 100644 --- a/les/enr_entry.go +++ b/les/enr_entry.go @@ -17,30 +17,34 @@ package les import ( - "time" - "github.com/ethereum/go-ethereum/core/forkid" "github.com/ethereum/go-ethereum/p2p" "github.com/ethereum/go-ethereum/p2p/dnsdisc" "github.com/ethereum/go-ethereum/p2p/enode" - "github.com/ethereum/go-ethereum/p2p/enr" "github.com/ethereum/go-ethereum/rlp" ) // lesEntry is the "les" ENR entry. This is set for LES servers only. type lesEntry struct { // Ignore additional fields (for forward compatibility). - Rest []rlp.RawValue `rlp:"tail"` + _ []rlp.RawValue `rlp:"tail"` } -// ENRKey implements enr.Entry. -func (e lesEntry) ENRKey() string { - return "les" +func (lesEntry) ENRKey() string { return "les" } + +// ethEntry is the "eth" ENR entry. This is redeclared here to avoid depending on package eth. +type ethEntry struct { + ForkID forkid.ID + _ []rlp.RawValue `rlp:"tail"` } +func (ethEntry) ENRKey() string { return "eth" } + // setupDiscovery creates the node discovery source for the eth protocol. func (eth *LightEthereum) setupDiscovery(cfg *p2p.Config) (enode.Iterator, error) { it := enode.NewFairMix(0) + + // Enable DNS discovery. if len(eth.config.EthDiscoveryURLs) != 0 { client := dnsdisc.NewClient(dnsdisc.Config{}) dns, err := client.NewIterator(eth.config.EthDiscoveryURLs...) @@ -49,27 +53,20 @@ func (eth *LightEthereum) setupDiscovery(cfg *p2p.Config) (enode.Iterator, error } it.AddSource(dns) } + + // Enable DHT. if cfg.DiscoveryV5 && eth.p2pServer.DiscV5 != nil { it.AddSource(eth.p2pServer.DiscV5.RandomNodes()) } + forkFilter := forkid.NewFilter(eth.blockchain) - var delay time.Duration - return enode.Filter(it, func(n *enode.Node) bool { - var ( - les struct { - _ []rlp.RawValue `rlp:"tail"` - } - eth struct { - ForkID forkid.ID - _ []rlp.RawValue `rlp:"tail"` - } - ) - if n.Load(enr.WithEntry("les", &les)) == nil && n.Load(enr.WithEntry("eth", ð)) == nil && forkFilter(eth.ForkID) == nil { - delay /= 2 - return true - } - delay += time.Millisecond - time.Sleep(delay) - return false - }), nil + iterator := enode.Filter(it, func(n *enode.Node) bool { return nodeIsServer(forkFilter, n) }) + return iterator, nil +} + +// nodeIsServer checks whether n is an LES server node. +func nodeIsServer(forkFilter forkid.Filter, n *enode.Node) bool { + var les lesEntry + var eth ethEntry + return n.Load(&les) == nil && n.Load(ð) == nil && forkFilter(eth.ForkID) == nil }