p2p/discover: move discv4 encoding to new 'v4wire' package (#21147)
This moves all v4 protocol definitions to a new package, p2p/discover/v4wire. The new package will be used for low-level protocol tests.
This commit is contained in:
		
							parent
							
								
									cd57d5cd38
								
							
						
					
					
						commit
						723bd8c17f
					
				| @ -61,17 +61,6 @@ func (e encPubkey) id() enode.ID { | ||||
| 	return enode.ID(crypto.Keccak256Hash(e[:])) | ||||
| } | ||||
| 
 | ||||
| // recoverNodeKey computes the public key used to sign the
 | ||||
| // given hash from the signature.
 | ||||
| func recoverNodeKey(hash, sig []byte) (key encPubkey, err error) { | ||||
| 	pubkey, err := crypto.Ecrecover(hash, sig) | ||||
| 	if err != nil { | ||||
| 		return key, err | ||||
| 	} | ||||
| 	copy(key[:], pubkey[1:]) | ||||
| 	return key, nil | ||||
| } | ||||
| 
 | ||||
| func wrapNode(n *enode.Node) *node { | ||||
| 	return &node{Node: *n} | ||||
| } | ||||
|  | ||||
| @ -24,6 +24,7 @@ import ( | ||||
| 	"testing" | ||||
| 
 | ||||
| 	"github.com/ethereum/go-ethereum/crypto" | ||||
| 	"github.com/ethereum/go-ethereum/p2p/discover/v4wire" | ||||
| 	"github.com/ethereum/go-ethereum/p2p/enode" | ||||
| 	"github.com/ethereum/go-ethereum/p2p/enr" | ||||
| ) | ||||
| @ -135,15 +136,15 @@ func TestUDPv4_LookupIteratorClose(t *testing.T) { | ||||
| 
 | ||||
| func serveTestnet(test *udpTest, testnet *preminedTestnet) { | ||||
| 	for done := false; !done; { | ||||
| 		done = test.waitPacketOut(func(p packetV4, to *net.UDPAddr, hash []byte) { | ||||
| 		done = test.waitPacketOut(func(p v4wire.Packet, to *net.UDPAddr, hash []byte) { | ||||
| 			n, key := testnet.nodeByAddr(to) | ||||
| 			switch p.(type) { | ||||
| 			case *pingV4: | ||||
| 				test.packetInFrom(nil, key, to, &pongV4{Expiration: futureExp, ReplyTok: hash}) | ||||
| 			case *findnodeV4: | ||||
| 			case *v4wire.Ping: | ||||
| 				test.packetInFrom(nil, key, to, &v4wire.Pong{Expiration: futureExp, ReplyTok: hash}) | ||||
| 			case *v4wire.Findnode: | ||||
| 				dist := enode.LogDist(n.ID(), testnet.target.id()) | ||||
| 				nodes := testnet.nodesAtDistance(dist - 1) | ||||
| 				test.packetInFrom(nil, key, to, &neighborsV4{Expiration: futureExp, Nodes: nodes}) | ||||
| 				test.packetInFrom(nil, key, to, &v4wire.Neighbors{Expiration: futureExp, Nodes: nodes}) | ||||
| 			} | ||||
| 		}) | ||||
| 	} | ||||
| @ -270,8 +271,8 @@ func (tn *preminedTestnet) nodeByAddr(addr *net.UDPAddr) (*enode.Node, *ecdsa.Pr | ||||
| 	return tn.node(dist, index), key | ||||
| } | ||||
| 
 | ||||
| func (tn *preminedTestnet) nodesAtDistance(dist int) []rpcNode { | ||||
| 	result := make([]rpcNode, len(tn.dists[dist])) | ||||
| func (tn *preminedTestnet) nodesAtDistance(dist int) []v4wire.Node { | ||||
| 	result := make([]v4wire.Node, len(tn.dists[dist])) | ||||
| 	for i := range result { | ||||
| 		result[i] = nodeToRPC(wrapNode(tn.node(dist, i))) | ||||
| 	} | ||||
|  | ||||
| @ -31,16 +31,14 @@ import ( | ||||
| 
 | ||||
| 	"github.com/ethereum/go-ethereum/crypto" | ||||
| 	"github.com/ethereum/go-ethereum/log" | ||||
| 	"github.com/ethereum/go-ethereum/p2p/discover/v4wire" | ||||
| 	"github.com/ethereum/go-ethereum/p2p/enode" | ||||
| 	"github.com/ethereum/go-ethereum/p2p/enr" | ||||
| 	"github.com/ethereum/go-ethereum/p2p/netutil" | ||||
| 	"github.com/ethereum/go-ethereum/rlp" | ||||
| ) | ||||
| 
 | ||||
| // Errors
 | ||||
| var ( | ||||
| 	errPacketTooSmall   = errors.New("too small") | ||||
| 	errBadHash          = errors.New("bad hash") | ||||
| 	errExpired          = errors.New("expired") | ||||
| 	errUnsolicitedReply = errors.New("unsolicited reply") | ||||
| 	errUnknownNode      = errors.New("unknown node") | ||||
| @ -66,135 +64,6 @@ const ( | ||||
| 	maxPacketSize = 1280 | ||||
| ) | ||||
| 
 | ||||
| // RPC packet types
 | ||||
| const ( | ||||
| 	p_pingV4 = iota + 1 // zero is 'reserved'
 | ||||
| 	p_pongV4 | ||||
| 	p_findnodeV4 | ||||
| 	p_neighborsV4 | ||||
| 	p_enrRequestV4 | ||||
| 	p_enrResponseV4 | ||||
| ) | ||||
| 
 | ||||
| // RPC request structures
 | ||||
| type ( | ||||
| 	pingV4 struct { | ||||
| 		senderKey *ecdsa.PublicKey // filled in by preverify
 | ||||
| 
 | ||||
| 		Version    uint | ||||
| 		From, To   rpcEndpoint | ||||
| 		Expiration uint64 | ||||
| 		// Ignore additional fields (for forward compatibility).
 | ||||
| 		Rest []rlp.RawValue `rlp:"tail"` | ||||
| 	} | ||||
| 
 | ||||
| 	// pongV4 is the reply to pingV4.
 | ||||
| 	pongV4 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.
 | ||||
| 		// Ignore additional fields (for forward compatibility).
 | ||||
| 		Rest []rlp.RawValue `rlp:"tail"` | ||||
| 	} | ||||
| 
 | ||||
| 	// findnodeV4 is a query for nodes close to the given target.
 | ||||
| 	findnodeV4 struct { | ||||
| 		Target     encPubkey | ||||
| 		Expiration uint64 | ||||
| 		// Ignore additional fields (for forward compatibility).
 | ||||
| 		Rest []rlp.RawValue `rlp:"tail"` | ||||
| 	} | ||||
| 
 | ||||
| 	// neighborsV4 is the reply to findnodeV4.
 | ||||
| 	neighborsV4 struct { | ||||
| 		Nodes      []rpcNode | ||||
| 		Expiration uint64 | ||||
| 		// Ignore additional fields (for forward compatibility).
 | ||||
| 		Rest []rlp.RawValue `rlp:"tail"` | ||||
| 	} | ||||
| 
 | ||||
| 	// enrRequestV4 queries for the remote node's record.
 | ||||
| 	enrRequestV4 struct { | ||||
| 		Expiration uint64 | ||||
| 		// Ignore additional fields (for forward compatibility).
 | ||||
| 		Rest []rlp.RawValue `rlp:"tail"` | ||||
| 	} | ||||
| 
 | ||||
| 	// enrResponseV4 is the reply to enrRequestV4.
 | ||||
| 	enrResponseV4 struct { | ||||
| 		ReplyTok []byte // Hash of the enrRequest packet.
 | ||||
| 		Record   enr.Record | ||||
| 		// Ignore additional fields (for forward compatibility).
 | ||||
| 		Rest []rlp.RawValue `rlp:"tail"` | ||||
| 	} | ||||
| 
 | ||||
| 	rpcNode struct { | ||||
| 		IP  net.IP // len 4 for IPv4 or 16 for IPv6
 | ||||
| 		UDP uint16 // for discovery protocol
 | ||||
| 		TCP uint16 // for RLPx protocol
 | ||||
| 		ID  encPubkey | ||||
| 	} | ||||
| 
 | ||||
| 	rpcEndpoint struct { | ||||
| 		IP  net.IP // len 4 for IPv4 or 16 for IPv6
 | ||||
| 		UDP uint16 // for discovery protocol
 | ||||
| 		TCP uint16 // for RLPx protocol
 | ||||
| 	} | ||||
| ) | ||||
| 
 | ||||
| // packetV4 is implemented by all v4 protocol messages.
 | ||||
| type packetV4 interface { | ||||
| 	// preverify checks whether the packet is valid and should be handled at all.
 | ||||
| 	preverify(t *UDPv4, from *net.UDPAddr, fromID enode.ID, fromKey encPubkey) error | ||||
| 	// handle handles the packet.
 | ||||
| 	handle(t *UDPv4, from *net.UDPAddr, fromID enode.ID, mac []byte) | ||||
| 	// packet name and type for logging purposes.
 | ||||
| 	name() string | ||||
| 	kind() byte | ||||
| } | ||||
| 
 | ||||
| func makeEndpoint(addr *net.UDPAddr, tcpPort uint16) rpcEndpoint { | ||||
| 	ip := net.IP{} | ||||
| 	if ip4 := addr.IP.To4(); ip4 != nil { | ||||
| 		ip = ip4 | ||||
| 	} else if ip6 := addr.IP.To16(); ip6 != nil { | ||||
| 		ip = ip6 | ||||
| 	} | ||||
| 	return rpcEndpoint{IP: ip, UDP: uint16(addr.Port), TCP: tcpPort} | ||||
| } | ||||
| 
 | ||||
| func (t *UDPv4) nodeFromRPC(sender *net.UDPAddr, rn rpcNode) (*node, error) { | ||||
| 	if rn.UDP <= 1024 { | ||||
| 		return nil, errLowPort | ||||
| 	} | ||||
| 	if err := netutil.CheckRelayIP(sender.IP, rn.IP); err != nil { | ||||
| 		return nil, err | ||||
| 	} | ||||
| 	if t.netrestrict != nil && !t.netrestrict.Contains(rn.IP) { | ||||
| 		return nil, errors.New("not contained in netrestrict whitelist") | ||||
| 	} | ||||
| 	key, err := decodePubkey(crypto.S256(), rn.ID) | ||||
| 	if err != nil { | ||||
| 		return nil, err | ||||
| 	} | ||||
| 	n := wrapNode(enode.NewV4(key, rn.IP, int(rn.TCP), int(rn.UDP))) | ||||
| 	err = n.ValidateComplete() | ||||
| 	return n, err | ||||
| } | ||||
| 
 | ||||
| func nodeToRPC(n *node) rpcNode { | ||||
| 	var key ecdsa.PublicKey | ||||
| 	var ekey encPubkey | ||||
| 	if err := n.Load((*enode.Secp256k1)(&key)); err == nil { | ||||
| 		ekey = encodePubkey(&key) | ||||
| 	} | ||||
| 	return rpcNode{ID: ekey, IP: n.IP(), UDP: uint16(n.UDP()), TCP: uint16(n.TCP())} | ||||
| } | ||||
| 
 | ||||
| // UDPv4 implements the v4 wire protocol.
 | ||||
| type UDPv4 struct { | ||||
| 	conn        UDPConn | ||||
| @ -243,16 +112,16 @@ type replyMatcher struct { | ||||
| 
 | ||||
| 	// reply contains the most recent reply. This field is safe for reading after errc has
 | ||||
| 	// received a value.
 | ||||
| 	reply packetV4 | ||||
| 	reply v4wire.Packet | ||||
| } | ||||
| 
 | ||||
| type replyMatchFunc func(interface{}) (matched bool, requestDone bool) | ||||
| type replyMatchFunc func(v4wire.Packet) (matched bool, requestDone bool) | ||||
| 
 | ||||
| // reply is a reply packet from a certain node.
 | ||||
| type reply struct { | ||||
| 	from enode.ID | ||||
| 	ip   net.IP | ||||
| 	data packetV4 | ||||
| 	data v4wire.Packet | ||||
| 	// loop indicates whether there was
 | ||||
| 	// a matching request by sending on this channel.
 | ||||
| 	matched chan<- bool | ||||
| @ -332,10 +201,10 @@ func (t *UDPv4) Resolve(n *enode.Node) *enode.Node { | ||||
| 	return n | ||||
| } | ||||
| 
 | ||||
| func (t *UDPv4) ourEndpoint() rpcEndpoint { | ||||
| func (t *UDPv4) ourEndpoint() v4wire.Endpoint { | ||||
| 	n := t.Self() | ||||
| 	a := &net.UDPAddr{IP: n.IP(), Port: n.UDP()} | ||||
| 	return makeEndpoint(a, uint16(n.TCP())) | ||||
| 	return v4wire.NewEndpoint(a, uint16(n.TCP())) | ||||
| } | ||||
| 
 | ||||
| // Ping sends a ping message to the given node.
 | ||||
| @ -348,7 +217,7 @@ func (t *UDPv4) Ping(n *enode.Node) error { | ||||
| func (t *UDPv4) ping(n *enode.Node) (seq uint64, err error) { | ||||
| 	rm := t.sendPing(n.ID(), &net.UDPAddr{IP: n.IP(), Port: n.UDP()}, nil) | ||||
| 	if err = <-rm.errc; err == nil { | ||||
| 		seq = seqFromTail(rm.reply.(*pongV4).Rest) | ||||
| 		seq = rm.reply.(*v4wire.Pong).ENRSeq() | ||||
| 	} | ||||
| 	return seq, err | ||||
| } | ||||
| @ -357,7 +226,7 @@ func (t *UDPv4) ping(n *enode.Node) (seq uint64, err error) { | ||||
| // when the reply arrives.
 | ||||
| func (t *UDPv4) sendPing(toid enode.ID, toaddr *net.UDPAddr, callback func()) *replyMatcher { | ||||
| 	req := t.makePing(toaddr) | ||||
| 	packet, hash, err := t.encode(t.priv, req) | ||||
| 	packet, hash, err := v4wire.Encode(t.priv, req) | ||||
| 	if err != nil { | ||||
| 		errc := make(chan error, 1) | ||||
| 		errc <- err | ||||
| @ -365,8 +234,8 @@ func (t *UDPv4) sendPing(toid enode.ID, toaddr *net.UDPAddr, callback func()) *r | ||||
| 	} | ||||
| 	// Add a matcher for the reply to the pending reply queue. Pongs are matched if they
 | ||||
| 	// reference the ping we're about to send.
 | ||||
| 	rm := t.pending(toid, toaddr.IP, p_pongV4, func(p interface{}) (matched bool, requestDone bool) { | ||||
| 		matched = bytes.Equal(p.(*pongV4).ReplyTok, hash) | ||||
| 	rm := t.pending(toid, toaddr.IP, v4wire.PongPacket, func(p v4wire.Packet) (matched bool, requestDone bool) { | ||||
| 		matched = bytes.Equal(p.(*v4wire.Pong).ReplyTok, hash) | ||||
| 		if matched && callback != nil { | ||||
| 			callback() | ||||
| 		} | ||||
| @ -374,16 +243,16 @@ func (t *UDPv4) sendPing(toid enode.ID, toaddr *net.UDPAddr, callback func()) *r | ||||
| 	}) | ||||
| 	// Send the packet.
 | ||||
| 	t.localNode.UDPContact(toaddr) | ||||
| 	t.write(toaddr, toid, req.name(), packet) | ||||
| 	t.write(toaddr, toid, req.Name(), packet) | ||||
| 	return rm | ||||
| } | ||||
| 
 | ||||
| func (t *UDPv4) makePing(toaddr *net.UDPAddr) *pingV4 { | ||||
| func (t *UDPv4) makePing(toaddr *net.UDPAddr) *v4wire.Ping { | ||||
| 	seq, _ := rlp.EncodeToBytes(t.localNode.Node().Seq()) | ||||
| 	return &pingV4{ | ||||
| 	return &v4wire.Ping{ | ||||
| 		Version:    4, | ||||
| 		From:       t.ourEndpoint(), | ||||
| 		To:         makeEndpoint(toaddr, 0), | ||||
| 		To:         v4wire.NewEndpoint(toaddr, 0), | ||||
| 		Expiration: uint64(time.Now().Add(expiration).Unix()), | ||||
| 		Rest:       []rlp.RawValue{seq}, | ||||
| 	} | ||||
| @ -422,23 +291,24 @@ func (t *UDPv4) newRandomLookup(ctx context.Context) *lookup { | ||||
| 
 | ||||
| func (t *UDPv4) newLookup(ctx context.Context, targetKey encPubkey) *lookup { | ||||
| 	target := enode.ID(crypto.Keccak256Hash(targetKey[:])) | ||||
| 	ekey := v4wire.Pubkey(targetKey) | ||||
| 	it := newLookup(ctx, t.tab, target, func(n *node) ([]*node, error) { | ||||
| 		return t.findnode(n.ID(), n.addr(), targetKey) | ||||
| 		return t.findnode(n.ID(), n.addr(), ekey) | ||||
| 	}) | ||||
| 	return it | ||||
| } | ||||
| 
 | ||||
| // findnode sends a findnode request to the given node and waits until
 | ||||
| // the node has sent up to k neighbors.
 | ||||
| func (t *UDPv4) findnode(toid enode.ID, toaddr *net.UDPAddr, target encPubkey) ([]*node, error) { | ||||
| func (t *UDPv4) findnode(toid enode.ID, toaddr *net.UDPAddr, target v4wire.Pubkey) ([]*node, error) { | ||||
| 	t.ensureBond(toid, toaddr) | ||||
| 
 | ||||
| 	// Add a matcher for 'neighbours' replies to the pending reply queue. The matcher is
 | ||||
| 	// active until enough nodes have been received.
 | ||||
| 	nodes := make([]*node, 0, bucketSize) | ||||
| 	nreceived := 0 | ||||
| 	rm := t.pending(toid, toaddr.IP, p_neighborsV4, func(r interface{}) (matched bool, requestDone bool) { | ||||
| 		reply := r.(*neighborsV4) | ||||
| 	rm := t.pending(toid, toaddr.IP, v4wire.NeighborsPacket, func(r v4wire.Packet) (matched bool, requestDone bool) { | ||||
| 		reply := r.(*v4wire.Neighbors) | ||||
| 		for _, rn := range reply.Nodes { | ||||
| 			nreceived++ | ||||
| 			n, err := t.nodeFromRPC(toaddr, rn) | ||||
| @ -450,7 +320,7 @@ func (t *UDPv4) findnode(toid enode.ID, toaddr *net.UDPAddr, target encPubkey) ( | ||||
| 		} | ||||
| 		return true, nreceived >= bucketSize | ||||
| 	}) | ||||
| 	t.send(toaddr, toid, &findnodeV4{ | ||||
| 	t.send(toaddr, toid, &v4wire.Findnode{ | ||||
| 		Target:     target, | ||||
| 		Expiration: uint64(time.Now().Add(expiration).Unix()), | ||||
| 	}) | ||||
| @ -462,26 +332,27 @@ func (t *UDPv4) RequestENR(n *enode.Node) (*enode.Node, error) { | ||||
| 	addr := &net.UDPAddr{IP: n.IP(), Port: n.UDP()} | ||||
| 	t.ensureBond(n.ID(), addr) | ||||
| 
 | ||||
| 	req := &enrRequestV4{ | ||||
| 	req := &v4wire.ENRRequest{ | ||||
| 		Expiration: uint64(time.Now().Add(expiration).Unix()), | ||||
| 	} | ||||
| 	packet, hash, err := t.encode(t.priv, req) | ||||
| 	packet, hash, err := v4wire.Encode(t.priv, req) | ||||
| 	if err != nil { | ||||
| 		return nil, err | ||||
| 	} | ||||
| 
 | ||||
| 	// Add a matcher for the reply to the pending reply queue. Responses are matched if
 | ||||
| 	// they reference the request we're about to send.
 | ||||
| 	rm := t.pending(n.ID(), addr.IP, p_enrResponseV4, func(r interface{}) (matched bool, requestDone bool) { | ||||
| 		matched = bytes.Equal(r.(*enrResponseV4).ReplyTok, hash) | ||||
| 	rm := t.pending(n.ID(), addr.IP, v4wire.ENRResponsePacket, func(r v4wire.Packet) (matched bool, requestDone bool) { | ||||
| 		matched = bytes.Equal(r.(*v4wire.ENRResponse).ReplyTok, hash) | ||||
| 		return matched, matched | ||||
| 	}) | ||||
| 	// Send the packet and wait for the reply.
 | ||||
| 	t.write(addr, n.ID(), req.name(), packet) | ||||
| 	t.write(addr, n.ID(), req.Name(), packet) | ||||
| 	if err := <-rm.errc; err != nil { | ||||
| 		return nil, err | ||||
| 	} | ||||
| 	// Verify the response record.
 | ||||
| 	respN, err := enode.New(enode.ValidSchemes, &rm.reply.(*enrResponseV4).Record) | ||||
| 	respN, err := enode.New(enode.ValidSchemes, &rm.reply.(*v4wire.ENRResponse).Record) | ||||
| 	if err != nil { | ||||
| 		return nil, err | ||||
| 	} | ||||
| @ -513,7 +384,7 @@ func (t *UDPv4) pending(id enode.ID, ip net.IP, ptype byte, callback replyMatchF | ||||
| 
 | ||||
| // handleReply dispatches a reply packet, invoking reply matchers. It returns
 | ||||
| // whether any matcher considered the packet acceptable.
 | ||||
| func (t *UDPv4) handleReply(from enode.ID, fromIP net.IP, req packetV4) bool { | ||||
| func (t *UDPv4) handleReply(from enode.ID, fromIP net.IP, req v4wire.Packet) bool { | ||||
| 	matched := make(chan bool, 1) | ||||
| 	select { | ||||
| 	case t.gotreply <- reply{from, fromIP, req, matched}: | ||||
| @ -579,7 +450,7 @@ func (t *UDPv4) loop() { | ||||
| 			var matched bool // whether any replyMatcher considered the reply acceptable.
 | ||||
| 			for el := plist.Front(); el != nil; el = el.Next() { | ||||
| 				p := el.Value.(*replyMatcher) | ||||
| 				if p.from == r.from && p.ptype == r.data.kind() && p.ip.Equal(r.ip) { | ||||
| 				if p.from == r.from && p.ptype == r.data.Kind() && p.ip.Equal(r.ip) { | ||||
| 					ok, requestDone := p.callback(r.data) | ||||
| 					matched = matched || ok | ||||
| 					// Remove the matcher if callback indicates that all replies have been received.
 | ||||
| @ -618,44 +489,12 @@ func (t *UDPv4) loop() { | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| const ( | ||||
| 	macSize  = 256 / 8 | ||||
| 	sigSize  = 520 / 8 | ||||
| 	headSize = macSize + sigSize // space of packet frame data
 | ||||
| ) | ||||
| 
 | ||||
| var ( | ||||
| 	headSpace = make([]byte, headSize) | ||||
| 
 | ||||
| 	// Neighbors replies are sent across multiple packets to
 | ||||
| 	// stay below the packet size limit. We compute the maximum number
 | ||||
| 	// of entries by stuffing a packet until it grows too large.
 | ||||
| 	maxNeighbors int | ||||
| ) | ||||
| 
 | ||||
| func init() { | ||||
| 	p := neighborsV4{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 >= maxPacketSize { | ||||
| 			maxNeighbors = n | ||||
| 			break | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| func (t *UDPv4) send(toaddr *net.UDPAddr, toid enode.ID, req packetV4) ([]byte, error) { | ||||
| 	packet, hash, err := t.encode(t.priv, req) | ||||
| func (t *UDPv4) send(toaddr *net.UDPAddr, toid enode.ID, req v4wire.Packet) ([]byte, error) { | ||||
| 	packet, hash, err := v4wire.Encode(t.priv, req) | ||||
| 	if err != nil { | ||||
| 		return hash, err | ||||
| 	} | ||||
| 	return hash, t.write(toaddr, toid, req.name(), packet) | ||||
| 	return hash, t.write(toaddr, toid, req.Name(), packet) | ||||
| } | ||||
| 
 | ||||
| func (t *UDPv4) write(toaddr *net.UDPAddr, toid enode.ID, what string, packet []byte) error { | ||||
| @ -664,30 +503,6 @@ func (t *UDPv4) write(toaddr *net.UDPAddr, toid enode.ID, what string, packet [] | ||||
| 	return err | ||||
| } | ||||
| 
 | ||||
| func (t *UDPv4) encode(priv *ecdsa.PrivateKey, req packetV4) (packet, hash []byte, err error) { | ||||
| 	name := req.name() | ||||
| 	b := new(bytes.Buffer) | ||||
| 	b.Write(headSpace) | ||||
| 	b.WriteByte(req.kind()) | ||||
| 	if err := rlp.Encode(b, req); err != nil { | ||||
| 		t.log.Error(fmt.Sprintf("Can't encode %s packet", name), "err", err) | ||||
| 		return nil, nil, err | ||||
| 	} | ||||
| 	packet = b.Bytes() | ||||
| 	sig, err := crypto.Sign(crypto.Keccak256(packet[headSize:]), priv) | ||||
| 	if err != nil { | ||||
| 		t.log.Error(fmt.Sprintf("Can't sign %s packet", name), "err", err) | ||||
| 		return nil, nil, err | ||||
| 	} | ||||
| 	copy(packet[macSize:], sig) | ||||
| 	// add the hash to the front. Note: this doesn't protect the
 | ||||
| 	// packet in any way. Our public key will be part of this hash in
 | ||||
| 	// The future.
 | ||||
| 	hash = crypto.Keccak256(packet[macSize:]) | ||||
| 	copy(packet, hash) | ||||
| 	return packet, hash, nil | ||||
| } | ||||
| 
 | ||||
| // readLoop runs in its own goroutine. it handles incoming UDP packets.
 | ||||
| func (t *UDPv4) readLoop(unhandled chan<- ReadPacket) { | ||||
| 	defer t.wg.Done() | ||||
| @ -719,58 +534,23 @@ func (t *UDPv4) readLoop(unhandled chan<- ReadPacket) { | ||||
| } | ||||
| 
 | ||||
| func (t *UDPv4) handlePacket(from *net.UDPAddr, buf []byte) error { | ||||
| 	packet, fromKey, hash, err := decodeV4(buf) | ||||
| 	rawpacket, fromKey, hash, err := v4wire.Decode(buf) | ||||
| 	if err != nil { | ||||
| 		t.log.Debug("Bad discv4 packet", "addr", from, "err", err) | ||||
| 		return err | ||||
| 	} | ||||
| 	fromID := fromKey.id() | ||||
| 	if err == nil { | ||||
| 		err = packet.preverify(t, from, fromID, fromKey) | ||||
| 	packet := t.wrapPacket(rawpacket) | ||||
| 	fromID := fromKey.ID() | ||||
| 	if err == nil && packet.preverify != nil { | ||||
| 		err = packet.preverify(packet, from, fromID, fromKey) | ||||
| 	} | ||||
| 	t.log.Trace("<< "+packet.name(), "id", fromID, "addr", from, "err", err) | ||||
| 	if err == nil { | ||||
| 		packet.handle(t, from, fromID, hash) | ||||
| 	t.log.Trace("<< "+packet.Name(), "id", fromID, "addr", from, "err", err) | ||||
| 	if err == nil && packet.handle != nil { | ||||
| 		packet.handle(packet, from, fromID, hash) | ||||
| 	} | ||||
| 	return err | ||||
| } | ||||
| 
 | ||||
| func decodeV4(buf []byte) (packetV4, encPubkey, []byte, error) { | ||||
| 	if len(buf) < headSize+1 { | ||||
| 		return nil, encPubkey{}, nil, errPacketTooSmall | ||||
| 	} | ||||
| 	hash, sig, sigdata := buf[:macSize], buf[macSize:headSize], buf[headSize:] | ||||
| 	shouldhash := crypto.Keccak256(buf[macSize:]) | ||||
| 	if !bytes.Equal(hash, shouldhash) { | ||||
| 		return nil, encPubkey{}, nil, errBadHash | ||||
| 	} | ||||
| 	fromKey, err := recoverNodeKey(crypto.Keccak256(buf[headSize:]), sig) | ||||
| 	if err != nil { | ||||
| 		return nil, fromKey, hash, err | ||||
| 	} | ||||
| 
 | ||||
| 	var req packetV4 | ||||
| 	switch ptype := sigdata[0]; ptype { | ||||
| 	case p_pingV4: | ||||
| 		req = new(pingV4) | ||||
| 	case p_pongV4: | ||||
| 		req = new(pongV4) | ||||
| 	case p_findnodeV4: | ||||
| 		req = new(findnodeV4) | ||||
| 	case p_neighborsV4: | ||||
| 		req = new(neighborsV4) | ||||
| 	case p_enrRequestV4: | ||||
| 		req = new(enrRequestV4) | ||||
| 	case p_enrResponseV4: | ||||
| 		req = new(enrResponseV4) | ||||
| 	default: | ||||
| 		return nil, fromKey, hash, fmt.Errorf("unknown type: %d", ptype) | ||||
| 	} | ||||
| 	s := rlp.NewStream(bytes.NewReader(sigdata[1:]), 0) | ||||
| 	err = s.Decode(req) | ||||
| 	return req, fromKey, hash, err | ||||
| } | ||||
| 
 | ||||
| // checkBond checks if the given node has a recent enough endpoint proof.
 | ||||
| func (t *UDPv4) checkBond(id enode.ID, ip net.IP) bool { | ||||
| 	return time.Since(t.db.LastPongReceived(id, ip)) < bondExpiration | ||||
| @ -788,49 +568,99 @@ func (t *UDPv4) ensureBond(toid enode.ID, toaddr *net.UDPAddr) { | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // expired checks whether the given UNIX time stamp is in the past.
 | ||||
| func expired(ts uint64) bool { | ||||
| 	return time.Unix(int64(ts), 0).Before(time.Now()) | ||||
| func (t *UDPv4) nodeFromRPC(sender *net.UDPAddr, rn v4wire.Node) (*node, error) { | ||||
| 	if rn.UDP <= 1024 { | ||||
| 		return nil, errLowPort | ||||
| 	} | ||||
| 	if err := netutil.CheckRelayIP(sender.IP, rn.IP); err != nil { | ||||
| 		return nil, err | ||||
| 	} | ||||
| 	if t.netrestrict != nil && !t.netrestrict.Contains(rn.IP) { | ||||
| 		return nil, errors.New("not contained in netrestrict whitelist") | ||||
| 	} | ||||
| 	key, err := v4wire.DecodePubkey(crypto.S256(), rn.ID) | ||||
| 	if err != nil { | ||||
| 		return nil, err | ||||
| 	} | ||||
| 	n := wrapNode(enode.NewV4(key, rn.IP, int(rn.TCP), int(rn.UDP))) | ||||
| 	err = n.ValidateComplete() | ||||
| 	return n, err | ||||
| } | ||||
| 
 | ||||
| func seqFromTail(tail []rlp.RawValue) uint64 { | ||||
| 	if len(tail) == 0 { | ||||
| 		return 0 | ||||
| func nodeToRPC(n *node) v4wire.Node { | ||||
| 	var key ecdsa.PublicKey | ||||
| 	var ekey v4wire.Pubkey | ||||
| 	if err := n.Load((*enode.Secp256k1)(&key)); err == nil { | ||||
| 		ekey = v4wire.EncodePubkey(&key) | ||||
| 	} | ||||
| 	var seq uint64 | ||||
| 	rlp.DecodeBytes(tail[0], &seq) | ||||
| 	return seq | ||||
| 	return v4wire.Node{ID: ekey, IP: n.IP(), UDP: uint16(n.UDP()), TCP: uint16(n.TCP())} | ||||
| } | ||||
| 
 | ||||
| // wrapPacket returns the handler functions applicable to a packet.
 | ||||
| func (t *UDPv4) wrapPacket(p v4wire.Packet) *packetHandlerV4 { | ||||
| 	var h packetHandlerV4 | ||||
| 	h.Packet = p | ||||
| 	switch p.(type) { | ||||
| 	case *v4wire.Ping: | ||||
| 		h.preverify = t.verifyPing | ||||
| 		h.handle = t.handlePing | ||||
| 	case *v4wire.Pong: | ||||
| 		h.preverify = t.verifyPong | ||||
| 	case *v4wire.Findnode: | ||||
| 		h.preverify = t.verifyFindnode | ||||
| 		h.handle = t.handleFindnode | ||||
| 	case *v4wire.Neighbors: | ||||
| 		h.preverify = t.verifyNeighbors | ||||
| 	case *v4wire.ENRRequest: | ||||
| 		h.preverify = t.verifyENRRequest | ||||
| 		h.handle = t.handleENRRequest | ||||
| 	case *v4wire.ENRResponse: | ||||
| 		h.preverify = t.verifyENRResponse | ||||
| 	} | ||||
| 	return &h | ||||
| } | ||||
| 
 | ||||
| // packetHandlerV4 wraps a packet with handler functions.
 | ||||
| type packetHandlerV4 struct { | ||||
| 	v4wire.Packet | ||||
| 	senderKey *ecdsa.PublicKey // used for ping
 | ||||
| 
 | ||||
| 	// preverify checks whether the packet is valid and should be handled at all.
 | ||||
| 	preverify func(p *packetHandlerV4, from *net.UDPAddr, fromID enode.ID, fromKey v4wire.Pubkey) error | ||||
| 	// handle handles the packet.
 | ||||
| 	handle func(req *packetHandlerV4, from *net.UDPAddr, fromID enode.ID, mac []byte) | ||||
| } | ||||
| 
 | ||||
| // PING/v4
 | ||||
| 
 | ||||
| func (req *pingV4) name() string { return "PING/v4" } | ||||
| func (req *pingV4) kind() byte   { return p_pingV4 } | ||||
| func (t *UDPv4) verifyPing(h *packetHandlerV4, from *net.UDPAddr, fromID enode.ID, fromKey v4wire.Pubkey) error { | ||||
| 	req := h.Packet.(*v4wire.Ping) | ||||
| 
 | ||||
| func (req *pingV4) preverify(t *UDPv4, from *net.UDPAddr, fromID enode.ID, fromKey encPubkey) error { | ||||
| 	if expired(req.Expiration) { | ||||
| 	senderKey, err := v4wire.DecodePubkey(crypto.S256(), fromKey) | ||||
| 	if err != nil { | ||||
| 		return err | ||||
| 	} | ||||
| 	if v4wire.Expired(req.Expiration) { | ||||
| 		return errExpired | ||||
| 	} | ||||
| 	key, err := decodePubkey(crypto.S256(), fromKey) | ||||
| 	if err != nil { | ||||
| 		return errors.New("invalid public key") | ||||
| 	} | ||||
| 	req.senderKey = key | ||||
| 	h.senderKey = senderKey | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
| func (req *pingV4) handle(t *UDPv4, from *net.UDPAddr, fromID enode.ID, mac []byte) { | ||||
| func (t *UDPv4) handlePing(h *packetHandlerV4, from *net.UDPAddr, fromID enode.ID, mac []byte) { | ||||
| 	req := h.Packet.(*v4wire.Ping) | ||||
| 
 | ||||
| 	// Reply.
 | ||||
| 	seq, _ := rlp.EncodeToBytes(t.localNode.Node().Seq()) | ||||
| 	t.send(from, fromID, &pongV4{ | ||||
| 		To:         makeEndpoint(from, req.From.TCP), | ||||
| 	t.send(from, fromID, &v4wire.Pong{ | ||||
| 		To:         v4wire.NewEndpoint(from, req.From.TCP), | ||||
| 		ReplyTok:   mac, | ||||
| 		Expiration: uint64(time.Now().Add(expiration).Unix()), | ||||
| 		Rest:       []rlp.RawValue{seq}, | ||||
| 	}) | ||||
| 
 | ||||
| 	// Ping back if our last pong on file is too far in the past.
 | ||||
| 	n := wrapNode(enode.NewV4(req.senderKey, from.IP, int(req.From.TCP), from.Port)) | ||||
| 	n := wrapNode(enode.NewV4(h.senderKey, from.IP, int(req.From.TCP), from.Port)) | ||||
| 	if time.Since(t.db.LastPongReceived(n.ID(), from.IP)) > bondExpiration { | ||||
| 		t.sendPing(fromID, from, func() { | ||||
| 			t.tab.addVerifiedNode(n) | ||||
| @ -846,31 +676,26 @@ func (req *pingV4) handle(t *UDPv4, from *net.UDPAddr, fromID enode.ID, mac []by | ||||
| 
 | ||||
| // PONG/v4
 | ||||
| 
 | ||||
| func (req *pongV4) name() string { return "PONG/v4" } | ||||
| func (req *pongV4) kind() byte   { return p_pongV4 } | ||||
| func (t *UDPv4) verifyPong(h *packetHandlerV4, from *net.UDPAddr, fromID enode.ID, fromKey v4wire.Pubkey) error { | ||||
| 	req := h.Packet.(*v4wire.Pong) | ||||
| 
 | ||||
| func (req *pongV4) preverify(t *UDPv4, from *net.UDPAddr, fromID enode.ID, fromKey encPubkey) error { | ||||
| 	if expired(req.Expiration) { | ||||
| 	if v4wire.Expired(req.Expiration) { | ||||
| 		return errExpired | ||||
| 	} | ||||
| 	if !t.handleReply(fromID, from.IP, req) { | ||||
| 		return errUnsolicitedReply | ||||
| 	} | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
| func (req *pongV4) handle(t *UDPv4, from *net.UDPAddr, fromID enode.ID, mac []byte) { | ||||
| 	t.localNode.UDPEndpointStatement(from, &net.UDPAddr{IP: req.To.IP, Port: int(req.To.UDP)}) | ||||
| 	t.db.UpdateLastPongReceived(fromID, from.IP, time.Now()) | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
| // FINDNODE/v4
 | ||||
| 
 | ||||
| func (req *findnodeV4) name() string { return "FINDNODE/v4" } | ||||
| func (req *findnodeV4) kind() byte   { return p_findnodeV4 } | ||||
| func (t *UDPv4) verifyFindnode(h *packetHandlerV4, from *net.UDPAddr, fromID enode.ID, fromKey v4wire.Pubkey) error { | ||||
| 	req := h.Packet.(*v4wire.Findnode) | ||||
| 
 | ||||
| func (req *findnodeV4) preverify(t *UDPv4, from *net.UDPAddr, fromID enode.ID, fromKey encPubkey) error { | ||||
| 	if expired(req.Expiration) { | ||||
| 	if v4wire.Expired(req.Expiration) { | ||||
| 		return errExpired | ||||
| 	} | ||||
| 	if !t.checkBond(fromID, from.IP) { | ||||
| @ -885,7 +710,9 @@ func (req *findnodeV4) preverify(t *UDPv4, from *net.UDPAddr, fromID enode.ID, f | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
| func (req *findnodeV4) handle(t *UDPv4, from *net.UDPAddr, fromID enode.ID, mac []byte) { | ||||
| func (t *UDPv4) handleFindnode(h *packetHandlerV4, from *net.UDPAddr, fromID enode.ID, mac []byte) { | ||||
| 	req := h.Packet.(*v4wire.Findnode) | ||||
| 
 | ||||
| 	// Determine closest nodes.
 | ||||
| 	target := enode.ID(crypto.Keccak256Hash(req.Target[:])) | ||||
| 	t.tab.mutex.Lock() | ||||
| @ -894,13 +721,13 @@ func (req *findnodeV4) handle(t *UDPv4, from *net.UDPAddr, fromID enode.ID, mac | ||||
| 
 | ||||
| 	// Send neighbors in chunks with at most maxNeighbors per packet
 | ||||
| 	// to stay below the packet size limit.
 | ||||
| 	p := neighborsV4{Expiration: uint64(time.Now().Add(expiration).Unix())} | ||||
| 	p := v4wire.Neighbors{Expiration: uint64(time.Now().Add(expiration).Unix())} | ||||
| 	var sent bool | ||||
| 	for _, n := range closest { | ||||
| 		if netutil.CheckRelayIP(from.IP, n.IP()) == nil { | ||||
| 			p.Nodes = append(p.Nodes, nodeToRPC(n)) | ||||
| 		} | ||||
| 		if len(p.Nodes) == maxNeighbors { | ||||
| 		if len(p.Nodes) == v4wire.MaxNeighbors { | ||||
| 			t.send(from, fromID, &p) | ||||
| 			p.Nodes = p.Nodes[:0] | ||||
| 			sent = true | ||||
| @ -913,29 +740,24 @@ func (req *findnodeV4) handle(t *UDPv4, from *net.UDPAddr, fromID enode.ID, mac | ||||
| 
 | ||||
| // NEIGHBORS/v4
 | ||||
| 
 | ||||
| func (req *neighborsV4) name() string { return "NEIGHBORS/v4" } | ||||
| func (req *neighborsV4) kind() byte   { return p_neighborsV4 } | ||||
| func (t *UDPv4) verifyNeighbors(h *packetHandlerV4, from *net.UDPAddr, fromID enode.ID, fromKey v4wire.Pubkey) error { | ||||
| 	req := h.Packet.(*v4wire.Neighbors) | ||||
| 
 | ||||
| func (req *neighborsV4) preverify(t *UDPv4, from *net.UDPAddr, fromID enode.ID, fromKey encPubkey) error { | ||||
| 	if expired(req.Expiration) { | ||||
| 	if v4wire.Expired(req.Expiration) { | ||||
| 		return errExpired | ||||
| 	} | ||||
| 	if !t.handleReply(fromID, from.IP, req) { | ||||
| 	if !t.handleReply(fromID, from.IP, h.Packet) { | ||||
| 		return errUnsolicitedReply | ||||
| 	} | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
| func (req *neighborsV4) handle(t *UDPv4, from *net.UDPAddr, fromID enode.ID, mac []byte) { | ||||
| } | ||||
| 
 | ||||
| // ENRREQUEST/v4
 | ||||
| 
 | ||||
| func (req *enrRequestV4) name() string { return "ENRREQUEST/v4" } | ||||
| func (req *enrRequestV4) kind() byte   { return p_enrRequestV4 } | ||||
| func (t *UDPv4) verifyENRRequest(h *packetHandlerV4, from *net.UDPAddr, fromID enode.ID, fromKey v4wire.Pubkey) error { | ||||
| 	req := h.Packet.(*v4wire.ENRRequest) | ||||
| 
 | ||||
| func (req *enrRequestV4) preverify(t *UDPv4, from *net.UDPAddr, fromID enode.ID, fromKey encPubkey) error { | ||||
| 	if expired(req.Expiration) { | ||||
| 	if v4wire.Expired(req.Expiration) { | ||||
| 		return errExpired | ||||
| 	} | ||||
| 	if !t.checkBond(fromID, from.IP) { | ||||
| @ -944,8 +766,8 @@ func (req *enrRequestV4) preverify(t *UDPv4, from *net.UDPAddr, fromID enode.ID, | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
| func (req *enrRequestV4) handle(t *UDPv4, from *net.UDPAddr, fromID enode.ID, mac []byte) { | ||||
| 	t.send(from, fromID, &enrResponseV4{ | ||||
| func (t *UDPv4) handleENRRequest(h *packetHandlerV4, from *net.UDPAddr, fromID enode.ID, mac []byte) { | ||||
| 	t.send(from, fromID, &v4wire.ENRResponse{ | ||||
| 		ReplyTok: mac, | ||||
| 		Record:   *t.localNode.Node().Record(), | ||||
| 	}) | ||||
| @ -953,15 +775,9 @@ func (req *enrRequestV4) handle(t *UDPv4, from *net.UDPAddr, fromID enode.ID, ma | ||||
| 
 | ||||
| // ENRRESPONSE/v4
 | ||||
| 
 | ||||
| func (req *enrResponseV4) name() string { return "ENRRESPONSE/v4" } | ||||
| func (req *enrResponseV4) kind() byte   { return p_enrResponseV4 } | ||||
| 
 | ||||
| func (req *enrResponseV4) preverify(t *UDPv4, from *net.UDPAddr, fromID enode.ID, fromKey encPubkey) error { | ||||
| 	if !t.handleReply(fromID, from.IP, req) { | ||||
| func (t *UDPv4) verifyENRResponse(h *packetHandlerV4, from *net.UDPAddr, fromID enode.ID, fromKey v4wire.Pubkey) error { | ||||
| 	if !t.handleReply(fromID, from.IP, h.Packet) { | ||||
| 		return errUnsolicitedReply | ||||
| 	} | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
| func (req *enrResponseV4) handle(t *UDPv4, from *net.UDPAddr, fromID enode.ID, mac []byte) { | ||||
| } | ||||
|  | ||||
| @ -21,7 +21,6 @@ import ( | ||||
| 	"crypto/ecdsa" | ||||
| 	crand "crypto/rand" | ||||
| 	"encoding/binary" | ||||
| 	"encoding/hex" | ||||
| 	"errors" | ||||
| 	"io" | ||||
| 	"math/rand" | ||||
| @ -31,23 +30,20 @@ import ( | ||||
| 	"testing" | ||||
| 	"time" | ||||
| 
 | ||||
| 	"github.com/davecgh/go-spew/spew" | ||||
| 	"github.com/ethereum/go-ethereum/common" | ||||
| 	"github.com/ethereum/go-ethereum/crypto" | ||||
| 	"github.com/ethereum/go-ethereum/internal/testlog" | ||||
| 	"github.com/ethereum/go-ethereum/log" | ||||
| 	"github.com/ethereum/go-ethereum/p2p/discover/v4wire" | ||||
| 	"github.com/ethereum/go-ethereum/p2p/enode" | ||||
| 	"github.com/ethereum/go-ethereum/p2p/enr" | ||||
| 	"github.com/ethereum/go-ethereum/rlp" | ||||
| ) | ||||
| 
 | ||||
| // shared test variables
 | ||||
| var ( | ||||
| 	futureExp          = uint64(time.Now().Add(10 * time.Hour).Unix()) | ||||
| 	testTarget         = encPubkey{0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1} | ||||
| 	testRemote         = rpcEndpoint{IP: net.ParseIP("1.1.1.1").To4(), UDP: 1, TCP: 2} | ||||
| 	testLocalAnnounced = rpcEndpoint{IP: net.ParseIP("2.2.2.2").To4(), UDP: 3, TCP: 4} | ||||
| 	testLocal          = rpcEndpoint{IP: net.ParseIP("3.3.3.3").To4(), UDP: 5, TCP: 6} | ||||
| 	testTarget         = v4wire.Pubkey{0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1} | ||||
| 	testRemote         = v4wire.Endpoint{IP: net.ParseIP("1.1.1.1").To4(), UDP: 1, TCP: 2} | ||||
| 	testLocalAnnounced = v4wire.Endpoint{IP: net.ParseIP("2.2.2.2").To4(), UDP: 3, TCP: 4} | ||||
| 	testLocal          = v4wire.Endpoint{IP: net.ParseIP("3.3.3.3").To4(), UDP: 5, TCP: 6} | ||||
| ) | ||||
| 
 | ||||
| type udpTest struct { | ||||
| @ -88,19 +84,19 @@ func (test *udpTest) close() { | ||||
| } | ||||
| 
 | ||||
| // handles a packet as if it had been sent to the transport.
 | ||||
| func (test *udpTest) packetIn(wantError error, data packetV4) { | ||||
| func (test *udpTest) packetIn(wantError error, data v4wire.Packet) { | ||||
| 	test.t.Helper() | ||||
| 
 | ||||
| 	test.packetInFrom(wantError, test.remotekey, test.remoteaddr, data) | ||||
| } | ||||
| 
 | ||||
| // handles a packet as if it had been sent to the transport by the key/endpoint.
 | ||||
| func (test *udpTest) packetInFrom(wantError error, key *ecdsa.PrivateKey, addr *net.UDPAddr, data packetV4) { | ||||
| func (test *udpTest) packetInFrom(wantError error, key *ecdsa.PrivateKey, addr *net.UDPAddr, data v4wire.Packet) { | ||||
| 	test.t.Helper() | ||||
| 
 | ||||
| 	enc, _, err := test.udp.encode(key, data) | ||||
| 	enc, _, err := v4wire.Encode(key, data) | ||||
| 	if err != nil { | ||||
| 		test.t.Errorf("%s encode error: %v", data.name(), err) | ||||
| 		test.t.Errorf("%s encode error: %v", data.Name(), err) | ||||
| 	} | ||||
| 	test.sent = append(test.sent, enc) | ||||
| 	if err = test.udp.handlePacket(addr, enc); err != wantError { | ||||
| @ -120,7 +116,7 @@ func (test *udpTest) waitPacketOut(validate interface{}) (closed bool) { | ||||
| 		test.t.Error("packet receive error:", err) | ||||
| 		return false | ||||
| 	} | ||||
| 	p, _, hash, err := decodeV4(dgram.data) | ||||
| 	p, _, hash, err := v4wire.Decode(dgram.data) | ||||
| 	if err != nil { | ||||
| 		test.t.Errorf("sent packet decode error: %v", err) | ||||
| 		return false | ||||
| @ -139,10 +135,10 @@ func TestUDPv4_packetErrors(t *testing.T) { | ||||
| 	test := newUDPTest(t) | ||||
| 	defer test.close() | ||||
| 
 | ||||
| 	test.packetIn(errExpired, &pingV4{From: testRemote, To: testLocalAnnounced, Version: 4}) | ||||
| 	test.packetIn(errUnsolicitedReply, &pongV4{ReplyTok: []byte{}, Expiration: futureExp}) | ||||
| 	test.packetIn(errUnknownNode, &findnodeV4{Expiration: futureExp}) | ||||
| 	test.packetIn(errUnsolicitedReply, &neighborsV4{Expiration: futureExp}) | ||||
| 	test.packetIn(errExpired, &v4wire.Ping{From: testRemote, To: testLocalAnnounced, Version: 4}) | ||||
| 	test.packetIn(errUnsolicitedReply, &v4wire.Pong{ReplyTok: []byte{}, Expiration: futureExp}) | ||||
| 	test.packetIn(errUnknownNode, &v4wire.Findnode{Expiration: futureExp}) | ||||
| 	test.packetIn(errUnsolicitedReply, &v4wire.Neighbors{Expiration: futureExp}) | ||||
| } | ||||
| 
 | ||||
| func TestUDPv4_pingTimeout(t *testing.T) { | ||||
| @ -160,13 +156,8 @@ func TestUDPv4_pingTimeout(t *testing.T) { | ||||
| 
 | ||||
| type testPacket byte | ||||
| 
 | ||||
| func (req testPacket) kind() byte   { return byte(req) } | ||||
| func (req testPacket) name() string { return "" } | ||||
| func (req testPacket) preverify(*UDPv4, *net.UDPAddr, enode.ID, encPubkey) error { | ||||
| 	return nil | ||||
| } | ||||
| func (req testPacket) handle(*UDPv4, *net.UDPAddr, enode.ID, []byte) { | ||||
| } | ||||
| func (req testPacket) Kind() byte   { return byte(req) } | ||||
| func (req testPacket) Name() string { return "" } | ||||
| 
 | ||||
| func TestUDPv4_responseTimeouts(t *testing.T) { | ||||
| 	t.Parallel() | ||||
| @ -191,7 +182,7 @@ func TestUDPv4_responseTimeouts(t *testing.T) { | ||||
| 		// within the timeout window.
 | ||||
| 		p := &replyMatcher{ | ||||
| 			ptype:    byte(rand.Intn(255)), | ||||
| 			callback: func(interface{}) (bool, bool) { return true, true }, | ||||
| 			callback: func(v4wire.Packet) (bool, bool) { return true, true }, | ||||
| 		} | ||||
| 		binary.BigEndian.PutUint64(p.from[:], uint64(i)) | ||||
| 		if p.ptype <= 128 { | ||||
| @ -247,7 +238,7 @@ func TestUDPv4_findnodeTimeout(t *testing.T) { | ||||
| 
 | ||||
| 	toaddr := &net.UDPAddr{IP: net.ParseIP("1.2.3.4"), Port: 2222} | ||||
| 	toid := enode.ID{1, 2, 3, 4} | ||||
| 	target := encPubkey{4, 5, 6, 7} | ||||
| 	target := v4wire.Pubkey{4, 5, 6, 7} | ||||
| 	result, err := test.udp.findnode(toid, toaddr, target) | ||||
| 	if err != errTimeout { | ||||
| 		t.Error("expected timeout error, got", err) | ||||
| @ -264,7 +255,7 @@ func TestUDPv4_findnode(t *testing.T) { | ||||
| 	// put a few nodes into the table. their exact
 | ||||
| 	// distribution shouldn't matter much, although we need to
 | ||||
| 	// take care not to overflow any bucket.
 | ||||
| 	nodes := &nodesByDistance{target: testTarget.id()} | ||||
| 	nodes := &nodesByDistance{target: testTarget.ID()} | ||||
| 	live := make(map[enode.ID]bool) | ||||
| 	numCandidates := 2 * bucketSize | ||||
| 	for i := 0; i < numCandidates; i++ { | ||||
| @ -282,32 +273,32 @@ func TestUDPv4_findnode(t *testing.T) { | ||||
| 
 | ||||
| 	// ensure there's a bond with the test node,
 | ||||
| 	// findnode won't be accepted otherwise.
 | ||||
| 	remoteID := encodePubkey(&test.remotekey.PublicKey).id() | ||||
| 	remoteID := v4wire.EncodePubkey(&test.remotekey.PublicKey).ID() | ||||
| 	test.table.db.UpdateLastPongReceived(remoteID, test.remoteaddr.IP, time.Now()) | ||||
| 
 | ||||
| 	// check that closest neighbors are returned.
 | ||||
| 	expected := test.table.closest(testTarget.id(), bucketSize, true) | ||||
| 	test.packetIn(nil, &findnodeV4{Target: testTarget, Expiration: futureExp}) | ||||
| 	expected := test.table.closest(testTarget.ID(), bucketSize, true) | ||||
| 	test.packetIn(nil, &v4wire.Findnode{Target: testTarget, Expiration: futureExp}) | ||||
| 	waitNeighbors := func(want []*node) { | ||||
| 		test.waitPacketOut(func(p *neighborsV4, to *net.UDPAddr, hash []byte) { | ||||
| 		test.waitPacketOut(func(p *v4wire.Neighbors, to *net.UDPAddr, hash []byte) { | ||||
| 			if len(p.Nodes) != len(want) { | ||||
| 				t.Errorf("wrong number of results: got %d, want %d", len(p.Nodes), bucketSize) | ||||
| 			} | ||||
| 			for i, n := range p.Nodes { | ||||
| 				if n.ID.id() != want[i].ID() { | ||||
| 				if n.ID.ID() != want[i].ID() { | ||||
| 					t.Errorf("result mismatch at %d:\n  got:  %v\n  want: %v", i, n, expected.entries[i]) | ||||
| 				} | ||||
| 				if !live[n.ID.id()] { | ||||
| 					t.Errorf("result includes dead node %v", n.ID.id()) | ||||
| 				if !live[n.ID.ID()] { | ||||
| 					t.Errorf("result includes dead node %v", n.ID.ID()) | ||||
| 				} | ||||
| 			} | ||||
| 		}) | ||||
| 	} | ||||
| 	// Receive replies.
 | ||||
| 	want := expected.entries | ||||
| 	if len(want) > maxNeighbors { | ||||
| 		waitNeighbors(want[:maxNeighbors]) | ||||
| 		want = want[maxNeighbors:] | ||||
| 	if len(want) > v4wire.MaxNeighbors { | ||||
| 		waitNeighbors(want[:v4wire.MaxNeighbors]) | ||||
| 		want = want[v4wire.MaxNeighbors:] | ||||
| 	} | ||||
| 	waitNeighbors(want) | ||||
| } | ||||
| @ -333,7 +324,7 @@ func TestUDPv4_findnodeMultiReply(t *testing.T) { | ||||
| 
 | ||||
| 	// wait for the findnode to be sent.
 | ||||
| 	// after it is sent, the transport is waiting for a reply
 | ||||
| 	test.waitPacketOut(func(p *findnodeV4, to *net.UDPAddr, hash []byte) { | ||||
| 	test.waitPacketOut(func(p *v4wire.Findnode, to *net.UDPAddr, hash []byte) { | ||||
| 		if p.Target != testTarget { | ||||
| 			t.Errorf("wrong target: got %v, want %v", p.Target, testTarget) | ||||
| 		} | ||||
| @ -346,12 +337,12 @@ func TestUDPv4_findnodeMultiReply(t *testing.T) { | ||||
| 		wrapNode(enode.MustParse("enode://9bffefd833d53fac8e652415f4973bee289e8b1a5c6c4cbe70abf817ce8a64cee11b823b66a987f51aaa9fba0d6a91b3e6bf0d5a5d1042de8e9eeea057b217f8@10.0.1.36:30301?discport=17")), | ||||
| 		wrapNode(enode.MustParse("enode://1b5b4aa662d7cb44a7221bfba67302590b643028197a7d5214790f3bac7aaa4a3241be9e83c09cf1f6c69d007c634faae3dc1b1221793e8446c0b3a09de65960@10.0.1.16:30303")), | ||||
| 	} | ||||
| 	rpclist := make([]rpcNode, len(list)) | ||||
| 	rpclist := make([]v4wire.Node, len(list)) | ||||
| 	for i := range list { | ||||
| 		rpclist[i] = nodeToRPC(list[i]) | ||||
| 	} | ||||
| 	test.packetIn(nil, &neighborsV4{Expiration: futureExp, Nodes: rpclist[:2]}) | ||||
| 	test.packetIn(nil, &neighborsV4{Expiration: futureExp, Nodes: rpclist[2:]}) | ||||
| 	test.packetIn(nil, &v4wire.Neighbors{Expiration: futureExp, Nodes: rpclist[:2]}) | ||||
| 	test.packetIn(nil, &v4wire.Neighbors{Expiration: futureExp, Nodes: rpclist[2:]}) | ||||
| 
 | ||||
| 	// check that the sent neighbors are all returned by findnode
 | ||||
| 	select { | ||||
| @ -375,10 +366,10 @@ func TestUDPv4_pingMatch(t *testing.T) { | ||||
| 	randToken := make([]byte, 32) | ||||
| 	crand.Read(randToken) | ||||
| 
 | ||||
| 	test.packetIn(nil, &pingV4{From: testRemote, To: testLocalAnnounced, Version: 4, Expiration: futureExp}) | ||||
| 	test.waitPacketOut(func(*pongV4, *net.UDPAddr, []byte) {}) | ||||
| 	test.waitPacketOut(func(*pingV4, *net.UDPAddr, []byte) {}) | ||||
| 	test.packetIn(errUnsolicitedReply, &pongV4{ReplyTok: randToken, To: testLocalAnnounced, Expiration: futureExp}) | ||||
| 	test.packetIn(nil, &v4wire.Ping{From: testRemote, To: testLocalAnnounced, Version: 4, Expiration: futureExp}) | ||||
| 	test.waitPacketOut(func(*v4wire.Pong, *net.UDPAddr, []byte) {}) | ||||
| 	test.waitPacketOut(func(*v4wire.Ping, *net.UDPAddr, []byte) {}) | ||||
| 	test.packetIn(errUnsolicitedReply, &v4wire.Pong{ReplyTok: randToken, To: testLocalAnnounced, Expiration: futureExp}) | ||||
| } | ||||
| 
 | ||||
| // This test checks that reply matching of pong verifies the sender IP address.
 | ||||
| @ -386,12 +377,12 @@ func TestUDPv4_pingMatchIP(t *testing.T) { | ||||
| 	test := newUDPTest(t) | ||||
| 	defer test.close() | ||||
| 
 | ||||
| 	test.packetIn(nil, &pingV4{From: testRemote, To: testLocalAnnounced, Version: 4, Expiration: futureExp}) | ||||
| 	test.waitPacketOut(func(*pongV4, *net.UDPAddr, []byte) {}) | ||||
| 	test.packetIn(nil, &v4wire.Ping{From: testRemote, To: testLocalAnnounced, Version: 4, Expiration: futureExp}) | ||||
| 	test.waitPacketOut(func(*v4wire.Pong, *net.UDPAddr, []byte) {}) | ||||
| 
 | ||||
| 	test.waitPacketOut(func(p *pingV4, to *net.UDPAddr, hash []byte) { | ||||
| 	test.waitPacketOut(func(p *v4wire.Ping, to *net.UDPAddr, hash []byte) { | ||||
| 		wrongAddr := &net.UDPAddr{IP: net.IP{33, 44, 1, 2}, Port: 30000} | ||||
| 		test.packetInFrom(errUnsolicitedReply, test.remotekey, wrongAddr, &pongV4{ | ||||
| 		test.packetInFrom(errUnsolicitedReply, test.remotekey, wrongAddr, &v4wire.Pong{ | ||||
| 			ReplyTok:   hash, | ||||
| 			To:         testLocalAnnounced, | ||||
| 			Expiration: futureExp, | ||||
| @ -406,15 +397,15 @@ func TestUDPv4_successfulPing(t *testing.T) { | ||||
| 	defer test.close() | ||||
| 
 | ||||
| 	// The remote side sends a ping packet to initiate the exchange.
 | ||||
| 	go test.packetIn(nil, &pingV4{From: testRemote, To: testLocalAnnounced, Version: 4, Expiration: futureExp}) | ||||
| 	go test.packetIn(nil, &v4wire.Ping{From: testRemote, To: testLocalAnnounced, Version: 4, Expiration: futureExp}) | ||||
| 
 | ||||
| 	// The ping is replied to.
 | ||||
| 	test.waitPacketOut(func(p *pongV4, to *net.UDPAddr, hash []byte) { | ||||
| 		pinghash := test.sent[0][:macSize] | ||||
| 	test.waitPacketOut(func(p *v4wire.Pong, to *net.UDPAddr, hash []byte) { | ||||
| 		pinghash := test.sent[0][:32] | ||||
| 		if !bytes.Equal(p.ReplyTok, pinghash) { | ||||
| 			t.Errorf("got pong.ReplyTok %x, want %x", p.ReplyTok, pinghash) | ||||
| 		} | ||||
| 		wantTo := rpcEndpoint{ | ||||
| 		wantTo := v4wire.Endpoint{ | ||||
| 			// The mirrored UDP address is the UDP packet sender
 | ||||
| 			IP: test.remoteaddr.IP, UDP: uint16(test.remoteaddr.Port), | ||||
| 			// The mirrored TCP port is the one from the ping packet
 | ||||
| @ -426,11 +417,11 @@ func TestUDPv4_successfulPing(t *testing.T) { | ||||
| 	}) | ||||
| 
 | ||||
| 	// Remote is unknown, the table pings back.
 | ||||
| 	test.waitPacketOut(func(p *pingV4, to *net.UDPAddr, hash []byte) { | ||||
| 	test.waitPacketOut(func(p *v4wire.Ping, to *net.UDPAddr, hash []byte) { | ||||
| 		if !reflect.DeepEqual(p.From, test.udp.ourEndpoint()) { | ||||
| 			t.Errorf("got ping.From %#v, want %#v", p.From, test.udp.ourEndpoint()) | ||||
| 		} | ||||
| 		wantTo := rpcEndpoint{ | ||||
| 		wantTo := v4wire.Endpoint{ | ||||
| 			// The mirrored UDP address is the UDP packet sender.
 | ||||
| 			IP:  test.remoteaddr.IP, | ||||
| 			UDP: uint16(test.remoteaddr.Port), | ||||
| @ -439,7 +430,7 @@ func TestUDPv4_successfulPing(t *testing.T) { | ||||
| 		if !reflect.DeepEqual(p.To, wantTo) { | ||||
| 			t.Errorf("got ping.To %v, want %v", p.To, wantTo) | ||||
| 		} | ||||
| 		test.packetIn(nil, &pongV4{ReplyTok: hash, Expiration: futureExp}) | ||||
| 		test.packetIn(nil, &v4wire.Pong{ReplyTok: hash, Expiration: futureExp}) | ||||
| 	}) | ||||
| 
 | ||||
| 	// The node should be added to the table shortly after getting the
 | ||||
| @ -473,25 +464,25 @@ func TestUDPv4_EIP868(t *testing.T) { | ||||
| 	wantNode := test.udp.localNode.Node() | ||||
| 
 | ||||
| 	// ENR requests aren't allowed before endpoint proof.
 | ||||
| 	test.packetIn(errUnknownNode, &enrRequestV4{Expiration: futureExp}) | ||||
| 	test.packetIn(errUnknownNode, &v4wire.ENRRequest{Expiration: futureExp}) | ||||
| 
 | ||||
| 	// Perform endpoint proof and check for sequence number in packet tail.
 | ||||
| 	test.packetIn(nil, &pingV4{Expiration: futureExp}) | ||||
| 	test.waitPacketOut(func(p *pongV4, addr *net.UDPAddr, hash []byte) { | ||||
| 		if seq := seqFromTail(p.Rest); seq != wantNode.Seq() { | ||||
| 			t.Errorf("wrong sequence number in pong: %d, want %d", seq, wantNode.Seq()) | ||||
| 	test.packetIn(nil, &v4wire.Ping{Expiration: futureExp}) | ||||
| 	test.waitPacketOut(func(p *v4wire.Pong, addr *net.UDPAddr, hash []byte) { | ||||
| 		if p.ENRSeq() != wantNode.Seq() { | ||||
| 			t.Errorf("wrong sequence number in pong: %d, want %d", p.ENRSeq(), wantNode.Seq()) | ||||
| 		} | ||||
| 	}) | ||||
| 	test.waitPacketOut(func(p *pingV4, addr *net.UDPAddr, hash []byte) { | ||||
| 		if seq := seqFromTail(p.Rest); seq != wantNode.Seq() { | ||||
| 			t.Errorf("wrong sequence number in ping: %d, want %d", seq, wantNode.Seq()) | ||||
| 	test.waitPacketOut(func(p *v4wire.Ping, addr *net.UDPAddr, hash []byte) { | ||||
| 		if p.ENRSeq() != wantNode.Seq() { | ||||
| 			t.Errorf("wrong sequence number in ping: %d, want %d", p.ENRSeq(), wantNode.Seq()) | ||||
| 		} | ||||
| 		test.packetIn(nil, &pongV4{Expiration: futureExp, ReplyTok: hash}) | ||||
| 		test.packetIn(nil, &v4wire.Pong{Expiration: futureExp, ReplyTok: hash}) | ||||
| 	}) | ||||
| 
 | ||||
| 	// Request should work now.
 | ||||
| 	test.packetIn(nil, &enrRequestV4{Expiration: futureExp}) | ||||
| 	test.waitPacketOut(func(p *enrResponseV4, addr *net.UDPAddr, hash []byte) { | ||||
| 	test.packetIn(nil, &v4wire.ENRRequest{Expiration: futureExp}) | ||||
| 	test.waitPacketOut(func(p *v4wire.ENRResponse, addr *net.UDPAddr, hash []byte) { | ||||
| 		n, err := enode.New(enode.ValidSchemes, &p.Record) | ||||
| 		if err != nil { | ||||
| 			t.Fatalf("invalid record: %v", err) | ||||
| @ -502,116 +493,6 @@ func TestUDPv4_EIP868(t *testing.T) { | ||||
| 	}) | ||||
| } | ||||
| 
 | ||||
| // EIP-8 test vectors.
 | ||||
| var testPackets = []struct { | ||||
| 	input      string | ||||
| 	wantPacket interface{} | ||||
| }{ | ||||
| 	{ | ||||
| 		input: "71dbda3a79554728d4f94411e42ee1f8b0d561c10e1e5f5893367948c6a7d70bb87b235fa28a77070271b6c164a2dce8c7e13a5739b53b5e96f2e5acb0e458a02902f5965d55ecbeb2ebb6cabb8b2b232896a36b737666c55265ad0a68412f250001ea04cb847f000001820cfa8215a8d790000000000000000000000000000000018208ae820d058443b9a355", | ||||
| 		wantPacket: &pingV4{ | ||||
| 			Version:    4, | ||||
| 			From:       rpcEndpoint{net.ParseIP("127.0.0.1").To4(), 3322, 5544}, | ||||
| 			To:         rpcEndpoint{net.ParseIP("::1"), 2222, 3333}, | ||||
| 			Expiration: 1136239445, | ||||
| 			Rest:       []rlp.RawValue{}, | ||||
| 		}, | ||||
| 	}, | ||||
| 	{ | ||||
| 		input: "e9614ccfd9fc3e74360018522d30e1419a143407ffcce748de3e22116b7e8dc92ff74788c0b6663aaa3d67d641936511c8f8d6ad8698b820a7cf9e1be7155e9a241f556658c55428ec0563514365799a4be2be5a685a80971ddcfa80cb422cdd0101ec04cb847f000001820cfa8215a8d790000000000000000000000000000000018208ae820d058443b9a3550102", | ||||
| 		wantPacket: &pingV4{ | ||||
| 			Version:    4, | ||||
| 			From:       rpcEndpoint{net.ParseIP("127.0.0.1").To4(), 3322, 5544}, | ||||
| 			To:         rpcEndpoint{net.ParseIP("::1"), 2222, 3333}, | ||||
| 			Expiration: 1136239445, | ||||
| 			Rest:       []rlp.RawValue{{0x01}, {0x02}}, | ||||
| 		}, | ||||
| 	}, | ||||
| 	{ | ||||
| 		input: "577be4349c4dd26768081f58de4c6f375a7a22f3f7adda654d1428637412c3d7fe917cadc56d4e5e7ffae1dbe3efffb9849feb71b262de37977e7c7a44e677295680e9e38ab26bee2fcbae207fba3ff3d74069a50b902a82c9903ed37cc993c50001f83e82022bd79020010db83c4d001500000000abcdef12820cfa8215a8d79020010db885a308d313198a2e037073488208ae82823a8443b9a355c5010203040531b9019afde696e582a78fa8d95ea13ce3297d4afb8ba6433e4154caa5ac6431af1b80ba76023fa4090c408f6b4bc3701562c031041d4702971d102c9ab7fa5eed4cd6bab8f7af956f7d565ee1917084a95398b6a21eac920fe3dd1345ec0a7ef39367ee69ddf092cbfe5b93e5e568ebc491983c09c76d922dc3", | ||||
| 		wantPacket: &pingV4{ | ||||
| 			Version:    555, | ||||
| 			From:       rpcEndpoint{net.ParseIP("2001:db8:3c4d:15::abcd:ef12"), 3322, 5544}, | ||||
| 			To:         rpcEndpoint{net.ParseIP("2001:db8:85a3:8d3:1319:8a2e:370:7348"), 2222, 33338}, | ||||
| 			Expiration: 1136239445, | ||||
| 			Rest:       []rlp.RawValue{{0xC5, 0x01, 0x02, 0x03, 0x04, 0x05}}, | ||||
| 		}, | ||||
| 	}, | ||||
| 	{ | ||||
| 		input: "09b2428d83348d27cdf7064ad9024f526cebc19e4958f0fdad87c15eb598dd61d08423e0bf66b2069869e1724125f820d851c136684082774f870e614d95a2855d000f05d1648b2d5945470bc187c2d2216fbe870f43ed0909009882e176a46b0102f846d79020010db885a308d313198a2e037073488208ae82823aa0fbc914b16819237dcd8801d7e53f69e9719adecb3cc0e790c57e91ca4461c9548443b9a355c6010203c2040506a0c969a58f6f9095004c0177a6b47f451530cab38966a25cca5cb58f055542124e", | ||||
| 		wantPacket: &pongV4{ | ||||
| 			To:         rpcEndpoint{net.ParseIP("2001:db8:85a3:8d3:1319:8a2e:370:7348"), 2222, 33338}, | ||||
| 			ReplyTok:   common.Hex2Bytes("fbc914b16819237dcd8801d7e53f69e9719adecb3cc0e790c57e91ca4461c954"), | ||||
| 			Expiration: 1136239445, | ||||
| 			Rest:       []rlp.RawValue{{0xC6, 0x01, 0x02, 0x03, 0xC2, 0x04, 0x05}, {0x06}}, | ||||
| 		}, | ||||
| 	}, | ||||
| 	{ | ||||
| 		input: "c7c44041b9f7c7e41934417ebac9a8e1a4c6298f74553f2fcfdcae6ed6fe53163eb3d2b52e39fe91831b8a927bf4fc222c3902202027e5e9eb812195f95d20061ef5cd31d502e47ecb61183f74a504fe04c51e73df81f25c4d506b26db4517490103f84eb840ca634cae0d49acb401d8a4c6b6fe8c55b70d115bf400769cc1400f3258cd31387574077f301b421bc84df7266c44e9e6d569fc56be00812904767bf5ccd1fc7f8443b9a35582999983999999280dc62cc8255c73471e0a61da0c89acdc0e035e260add7fc0c04ad9ebf3919644c91cb247affc82b69bd2ca235c71eab8e49737c937a2c396", | ||||
| 		wantPacket: &findnodeV4{ | ||||
| 			Target:     hexEncPubkey("ca634cae0d49acb401d8a4c6b6fe8c55b70d115bf400769cc1400f3258cd31387574077f301b421bc84df7266c44e9e6d569fc56be00812904767bf5ccd1fc7f"), | ||||
| 			Expiration: 1136239445, | ||||
| 			Rest:       []rlp.RawValue{{0x82, 0x99, 0x99}, {0x83, 0x99, 0x99, 0x99}}, | ||||
| 		}, | ||||
| 	}, | ||||
| 	{ | ||||
| 		input: "c679fc8fe0b8b12f06577f2e802d34f6fa257e6137a995f6f4cbfc9ee50ed3710faf6e66f932c4c8d81d64343f429651328758b47d3dbc02c4042f0fff6946a50f4a49037a72bb550f3a7872363a83e1b9ee6469856c24eb4ef80b7535bcf99c0004f9015bf90150f84d846321163782115c82115db8403155e1427f85f10a5c9a7755877748041af1bcd8d474ec065eb33df57a97babf54bfd2103575fa829115d224c523596b401065a97f74010610fce76382c0bf32f84984010203040101b840312c55512422cf9b8a4097e9a6ad79402e87a15ae909a4bfefa22398f03d20951933beea1e4dfa6f968212385e829f04c2d314fc2d4e255e0d3bc08792b069dbf8599020010db83c4d001500000000abcdef12820d05820d05b84038643200b172dcfef857492156971f0e6aa2c538d8b74010f8e140811d53b98c765dd2d96126051913f44582e8c199ad7c6d6819e9a56483f637feaac9448aacf8599020010db885a308d313198a2e037073488203e78203e8b8408dcab8618c3253b558d459da53bd8fa68935a719aff8b811197101a4b2b47dd2d47295286fc00cc081bb542d760717d1bdd6bec2c37cd72eca367d6dd3b9df738443b9a355010203b525a138aa34383fec3d2719a0", | ||||
| 		wantPacket: &neighborsV4{ | ||||
| 			Nodes: []rpcNode{ | ||||
| 				{ | ||||
| 					ID:  hexEncPubkey("3155e1427f85f10a5c9a7755877748041af1bcd8d474ec065eb33df57a97babf54bfd2103575fa829115d224c523596b401065a97f74010610fce76382c0bf32"), | ||||
| 					IP:  net.ParseIP("99.33.22.55").To4(), | ||||
| 					UDP: 4444, | ||||
| 					TCP: 4445, | ||||
| 				}, | ||||
| 				{ | ||||
| 					ID:  hexEncPubkey("312c55512422cf9b8a4097e9a6ad79402e87a15ae909a4bfefa22398f03d20951933beea1e4dfa6f968212385e829f04c2d314fc2d4e255e0d3bc08792b069db"), | ||||
| 					IP:  net.ParseIP("1.2.3.4").To4(), | ||||
| 					UDP: 1, | ||||
| 					TCP: 1, | ||||
| 				}, | ||||
| 				{ | ||||
| 					ID:  hexEncPubkey("38643200b172dcfef857492156971f0e6aa2c538d8b74010f8e140811d53b98c765dd2d96126051913f44582e8c199ad7c6d6819e9a56483f637feaac9448aac"), | ||||
| 					IP:  net.ParseIP("2001:db8:3c4d:15::abcd:ef12"), | ||||
| 					UDP: 3333, | ||||
| 					TCP: 3333, | ||||
| 				}, | ||||
| 				{ | ||||
| 					ID:  hexEncPubkey("8dcab8618c3253b558d459da53bd8fa68935a719aff8b811197101a4b2b47dd2d47295286fc00cc081bb542d760717d1bdd6bec2c37cd72eca367d6dd3b9df73"), | ||||
| 					IP:  net.ParseIP("2001:db8:85a3:8d3:1319:8a2e:370:7348"), | ||||
| 					UDP: 999, | ||||
| 					TCP: 1000, | ||||
| 				}, | ||||
| 			}, | ||||
| 			Expiration: 1136239445, | ||||
| 			Rest:       []rlp.RawValue{{0x01}, {0x02}, {0x03}}, | ||||
| 		}, | ||||
| 	}, | ||||
| } | ||||
| 
 | ||||
| func TestUDPv4_forwardCompatibility(t *testing.T) { | ||||
| 	testkey, _ := crypto.HexToECDSA("b71c71a67e1177ad4e901695e1b4b9ee17ae16c6668d313eac2f96dbcda3f291") | ||||
| 	wantNodeKey := encodePubkey(&testkey.PublicKey) | ||||
| 
 | ||||
| 	for _, test := range testPackets { | ||||
| 		input, err := hex.DecodeString(test.input) | ||||
| 		if err != nil { | ||||
| 			t.Fatalf("invalid hex: %s", test.input) | ||||
| 		} | ||||
| 		packet, nodekey, _, err := decodeV4(input) | ||||
| 		if err != nil { | ||||
| 			t.Errorf("did not accept packet %s\n%v", test.input, err) | ||||
| 			continue | ||||
| 		} | ||||
| 		if !reflect.DeepEqual(packet, test.wantPacket) { | ||||
| 			t.Errorf("got %s\nwant %s", spew.Sdump(packet), spew.Sdump(test.wantPacket)) | ||||
| 		} | ||||
| 		if nodekey != wantNodeKey { | ||||
| 			t.Errorf("got id %v\nwant id %v", nodekey, wantNodeKey) | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // dgramPipe is a fake UDP socket. It queues all sent datagrams.
 | ||||
| type dgramPipe struct { | ||||
| 	mu      *sync.Mutex | ||||
|  | ||||
							
								
								
									
										300
									
								
								p2p/discover/v4wire/v4wire.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										300
									
								
								p2p/discover/v4wire/v4wire.go
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,300 @@ | ||||
| // Copyright 2019 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 <http://www.gnu.org/licenses/>.
 | ||||
| 
 | ||||
| // Package v4wire implements the Discovery v4 Wire Protocol.
 | ||||
| package v4wire | ||||
| 
 | ||||
| import ( | ||||
| 	"bytes" | ||||
| 	"crypto/ecdsa" | ||||
| 	"crypto/elliptic" | ||||
| 	"errors" | ||||
| 	"fmt" | ||||
| 	"math/big" | ||||
| 	"net" | ||||
| 	"time" | ||||
| 
 | ||||
| 	"github.com/ethereum/go-ethereum/common/math" | ||||
| 	"github.com/ethereum/go-ethereum/crypto" | ||||
| 	"github.com/ethereum/go-ethereum/p2p/enode" | ||||
| 	"github.com/ethereum/go-ethereum/p2p/enr" | ||||
| 	"github.com/ethereum/go-ethereum/rlp" | ||||
| ) | ||||
| 
 | ||||
| // RPC packet types
 | ||||
| const ( | ||||
| 	PingPacket = iota + 1 // zero is 'reserved'
 | ||||
| 	PongPacket | ||||
| 	FindnodePacket | ||||
| 	NeighborsPacket | ||||
| 	ENRRequestPacket | ||||
| 	ENRResponsePacket | ||||
| ) | ||||
| 
 | ||||
| // RPC request structures
 | ||||
| type ( | ||||
| 	Ping struct { | ||||
| 		Version    uint | ||||
| 		From, To   Endpoint | ||||
| 		Expiration uint64 | ||||
| 		// 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         Endpoint | ||||
| 		ReplyTok   []byte // This contains the hash of the ping packet.
 | ||||
| 		Expiration uint64 // Absolute timestamp at which the packet becomes invalid.
 | ||||
| 		// 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     Pubkey | ||||
| 		Expiration uint64 | ||||
| 		// Ignore additional fields (for forward compatibility).
 | ||||
| 		Rest []rlp.RawValue `rlp:"tail"` | ||||
| 	} | ||||
| 
 | ||||
| 	// Neighbors is the reply to findnode.
 | ||||
| 	Neighbors struct { | ||||
| 		Nodes      []Node | ||||
| 		Expiration uint64 | ||||
| 		// Ignore additional fields (for forward compatibility).
 | ||||
| 		Rest []rlp.RawValue `rlp:"tail"` | ||||
| 	} | ||||
| 
 | ||||
| 	// enrRequest queries for the remote node's record.
 | ||||
| 	ENRRequest struct { | ||||
| 		Expiration uint64 | ||||
| 		// Ignore additional fields (for forward compatibility).
 | ||||
| 		Rest []rlp.RawValue `rlp:"tail"` | ||||
| 	} | ||||
| 
 | ||||
| 	// enrResponse is the reply to enrRequest.
 | ||||
| 	ENRResponse struct { | ||||
| 		ReplyTok []byte // Hash of the enrRequest packet.
 | ||||
| 		Record   enr.Record | ||||
| 		// Ignore additional fields (for forward compatibility).
 | ||||
| 		Rest []rlp.RawValue `rlp:"tail"` | ||||
| 	} | ||||
| ) | ||||
| 
 | ||||
| // This number is the maximum number of neighbor nodes in a Neigbors packet.
 | ||||
| const MaxNeighbors = 12 | ||||
| 
 | ||||
| // This code computes the MaxNeighbors constant value.
 | ||||
| 
 | ||||
| // func init() {
 | ||||
| // 	var maxNeighbors int
 | ||||
| // 	p := Neighbors{Expiration: ^uint64(0)}
 | ||||
| // 	maxSizeNode := Node{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 {
 | ||||
| // 			maxNeighbors = n
 | ||||
| // 			break
 | ||||
| // 		}
 | ||||
| // 	}
 | ||||
| // 	fmt.Println("maxNeighbors", maxNeighbors)
 | ||||
| // }
 | ||||
| 
 | ||||
| // Pubkey represents an encoded 64-byte secp256k1 public key.
 | ||||
| type Pubkey [64]byte | ||||
| 
 | ||||
| // ID returns the node ID corresponding to the public key.
 | ||||
| func (e Pubkey) ID() enode.ID { | ||||
| 	return enode.ID(crypto.Keccak256Hash(e[:])) | ||||
| } | ||||
| 
 | ||||
| // Node represents information about a node.
 | ||||
| type Node struct { | ||||
| 	IP  net.IP // len 4 for IPv4 or 16 for IPv6
 | ||||
| 	UDP uint16 // for discovery protocol
 | ||||
| 	TCP uint16 // for RLPx protocol
 | ||||
| 	ID  Pubkey | ||||
| } | ||||
| 
 | ||||
| // Endpoint represents a network endpoint.
 | ||||
| type Endpoint struct { | ||||
| 	IP  net.IP // len 4 for IPv4 or 16 for IPv6
 | ||||
| 	UDP uint16 // for discovery protocol
 | ||||
| 	TCP uint16 // for RLPx protocol
 | ||||
| } | ||||
| 
 | ||||
| // NewEndpoint creates an endpoint.
 | ||||
| func NewEndpoint(addr *net.UDPAddr, tcpPort uint16) Endpoint { | ||||
| 	ip := net.IP{} | ||||
| 	if ip4 := addr.IP.To4(); ip4 != nil { | ||||
| 		ip = ip4 | ||||
| 	} else if ip6 := addr.IP.To16(); ip6 != nil { | ||||
| 		ip = ip6 | ||||
| 	} | ||||
| 	return Endpoint{IP: ip, UDP: uint16(addr.Port), TCP: tcpPort} | ||||
| } | ||||
| 
 | ||||
| type Packet interface { | ||||
| 	// packet name and type for logging purposes.
 | ||||
| 	Name() string | ||||
| 	Kind() byte | ||||
| } | ||||
| 
 | ||||
| func (req *Ping) Name() string   { return "PING/v4" } | ||||
| func (req *Ping) Kind() byte     { return PingPacket } | ||||
| func (req *Ping) ENRSeq() uint64 { return seqFromTail(req.Rest) } | ||||
| 
 | ||||
| func (req *Pong) Name() string   { return "PONG/v4" } | ||||
| func (req *Pong) Kind() byte     { return PongPacket } | ||||
| func (req *Pong) ENRSeq() uint64 { return seqFromTail(req.Rest) } | ||||
| 
 | ||||
| func (req *Findnode) Name() string { return "FINDNODE/v4" } | ||||
| func (req *Findnode) Kind() byte   { return FindnodePacket } | ||||
| 
 | ||||
| func (req *Neighbors) Name() string { return "NEIGHBORS/v4" } | ||||
| func (req *Neighbors) Kind() byte   { return NeighborsPacket } | ||||
| 
 | ||||
| func (req *ENRRequest) Name() string { return "ENRREQUEST/v4" } | ||||
| func (req *ENRRequest) Kind() byte   { return ENRRequestPacket } | ||||
| 
 | ||||
| func (req *ENRResponse) Name() string { return "ENRRESPONSE/v4" } | ||||
| func (req *ENRResponse) Kind() byte   { return ENRResponsePacket } | ||||
| 
 | ||||
| // Expired checks whether the given UNIX time stamp is in the past.
 | ||||
| func Expired(ts uint64) bool { | ||||
| 	return time.Unix(int64(ts), 0).Before(time.Now()) | ||||
| } | ||||
| 
 | ||||
| func seqFromTail(tail []rlp.RawValue) uint64 { | ||||
| 	if len(tail) == 0 { | ||||
| 		return 0 | ||||
| 	} | ||||
| 	var seq uint64 | ||||
| 	rlp.DecodeBytes(tail[0], &seq) | ||||
| 	return seq | ||||
| } | ||||
| 
 | ||||
| // Encoder/decoder.
 | ||||
| 
 | ||||
| const ( | ||||
| 	macSize  = 32 | ||||
| 	sigSize  = crypto.SignatureLength | ||||
| 	headSize = macSize + sigSize // space of packet frame data
 | ||||
| ) | ||||
| 
 | ||||
| var ( | ||||
| 	ErrPacketTooSmall = errors.New("too small") | ||||
| 	ErrBadHash        = errors.New("bad hash") | ||||
| 	ErrBadPoint       = errors.New("invalid curve point") | ||||
| ) | ||||
| 
 | ||||
| var headSpace = make([]byte, headSize) | ||||
| 
 | ||||
| // Decode reads a discovery v4 packet.
 | ||||
| func Decode(input []byte) (Packet, Pubkey, []byte, error) { | ||||
| 	if len(input) < headSize+1 { | ||||
| 		return nil, Pubkey{}, nil, ErrPacketTooSmall | ||||
| 	} | ||||
| 	hash, sig, sigdata := input[:macSize], input[macSize:headSize], input[headSize:] | ||||
| 	shouldhash := crypto.Keccak256(input[macSize:]) | ||||
| 	if !bytes.Equal(hash, shouldhash) { | ||||
| 		return nil, Pubkey{}, nil, ErrBadHash | ||||
| 	} | ||||
| 	fromKey, err := recoverNodeKey(crypto.Keccak256(input[headSize:]), sig) | ||||
| 	if err != nil { | ||||
| 		return nil, fromKey, hash, err | ||||
| 	} | ||||
| 
 | ||||
| 	var req Packet | ||||
| 	switch ptype := sigdata[0]; ptype { | ||||
| 	case PingPacket: | ||||
| 		req = new(Ping) | ||||
| 	case PongPacket: | ||||
| 		req = new(Pong) | ||||
| 	case FindnodePacket: | ||||
| 		req = new(Findnode) | ||||
| 	case NeighborsPacket: | ||||
| 		req = new(Neighbors) | ||||
| 	case ENRRequestPacket: | ||||
| 		req = new(ENRRequest) | ||||
| 	case ENRResponsePacket: | ||||
| 		req = new(ENRResponse) | ||||
| 	default: | ||||
| 		return nil, fromKey, hash, fmt.Errorf("unknown type: %d", ptype) | ||||
| 	} | ||||
| 	s := rlp.NewStream(bytes.NewReader(sigdata[1:]), 0) | ||||
| 	err = s.Decode(req) | ||||
| 	return req, fromKey, hash, err | ||||
| } | ||||
| 
 | ||||
| // Encode encodes a discovery packet.
 | ||||
| func Encode(priv *ecdsa.PrivateKey, req Packet) (packet, hash []byte, err error) { | ||||
| 	b := new(bytes.Buffer) | ||||
| 	b.Write(headSpace) | ||||
| 	b.WriteByte(req.Kind()) | ||||
| 	if err := rlp.Encode(b, req); err != nil { | ||||
| 		return nil, nil, err | ||||
| 	} | ||||
| 	packet = b.Bytes() | ||||
| 	sig, err := crypto.Sign(crypto.Keccak256(packet[headSize:]), priv) | ||||
| 	if err != nil { | ||||
| 		return nil, nil, err | ||||
| 	} | ||||
| 	copy(packet[macSize:], sig) | ||||
| 	// Add the hash to the front. Note: this doesn't protect the packet in any way.
 | ||||
| 	hash = crypto.Keccak256(packet[macSize:]) | ||||
| 	copy(packet, hash) | ||||
| 	return packet, hash, nil | ||||
| } | ||||
| 
 | ||||
| // recoverNodeKey computes the public key used to sign the given hash from the signature.
 | ||||
| func recoverNodeKey(hash, sig []byte) (key Pubkey, err error) { | ||||
| 	pubkey, err := crypto.Ecrecover(hash, sig) | ||||
| 	if err != nil { | ||||
| 		return key, err | ||||
| 	} | ||||
| 	copy(key[:], pubkey[1:]) | ||||
| 	return key, nil | ||||
| } | ||||
| 
 | ||||
| // EncodePubkey encodes a secp256k1 public key.
 | ||||
| func EncodePubkey(key *ecdsa.PublicKey) Pubkey { | ||||
| 	var e Pubkey | ||||
| 	math.ReadBits(key.X, e[:len(e)/2]) | ||||
| 	math.ReadBits(key.Y, e[len(e)/2:]) | ||||
| 	return e | ||||
| } | ||||
| 
 | ||||
| // DecodePubkey reads an encoded secp256k1 public key.
 | ||||
| func DecodePubkey(curve elliptic.Curve, e Pubkey) (*ecdsa.PublicKey, error) { | ||||
| 	p := &ecdsa.PublicKey{Curve: curve, X: new(big.Int), Y: new(big.Int)} | ||||
| 	half := len(e) / 2 | ||||
| 	p.X.SetBytes(e[:half]) | ||||
| 	p.Y.SetBytes(e[half:]) | ||||
| 	if !p.Curve.IsOnCurve(p.X, p.Y) { | ||||
| 		return nil, ErrBadPoint | ||||
| 	} | ||||
| 	return p, nil | ||||
| } | ||||
							
								
								
									
										152
									
								
								p2p/discover/v4wire/v4wire_test.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										152
									
								
								p2p/discover/v4wire/v4wire_test.go
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,152 @@ | ||||
| // Copyright 2019 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 <http://www.gnu.org/licenses/>.
 | ||||
| 
 | ||||
| package v4wire | ||||
| 
 | ||||
| import ( | ||||
| 	"encoding/hex" | ||||
| 	"net" | ||||
| 	"reflect" | ||||
| 	"testing" | ||||
| 
 | ||||
| 	"github.com/davecgh/go-spew/spew" | ||||
| 	"github.com/ethereum/go-ethereum/common" | ||||
| 	"github.com/ethereum/go-ethereum/crypto" | ||||
| 	"github.com/ethereum/go-ethereum/rlp" | ||||
| ) | ||||
| 
 | ||||
| // EIP-8 test vectors.
 | ||||
| var testPackets = []struct { | ||||
| 	input      string | ||||
| 	wantPacket interface{} | ||||
| }{ | ||||
| 	{ | ||||
| 		input: "71dbda3a79554728d4f94411e42ee1f8b0d561c10e1e5f5893367948c6a7d70bb87b235fa28a77070271b6c164a2dce8c7e13a5739b53b5e96f2e5acb0e458a02902f5965d55ecbeb2ebb6cabb8b2b232896a36b737666c55265ad0a68412f250001ea04cb847f000001820cfa8215a8d790000000000000000000000000000000018208ae820d058443b9a355", | ||||
| 		wantPacket: &Ping{ | ||||
| 			Version:    4, | ||||
| 			From:       Endpoint{net.ParseIP("127.0.0.1").To4(), 3322, 5544}, | ||||
| 			To:         Endpoint{net.ParseIP("::1"), 2222, 3333}, | ||||
| 			Expiration: 1136239445, | ||||
| 			Rest:       []rlp.RawValue{}, | ||||
| 		}, | ||||
| 	}, | ||||
| 	{ | ||||
| 		input: "e9614ccfd9fc3e74360018522d30e1419a143407ffcce748de3e22116b7e8dc92ff74788c0b6663aaa3d67d641936511c8f8d6ad8698b820a7cf9e1be7155e9a241f556658c55428ec0563514365799a4be2be5a685a80971ddcfa80cb422cdd0101ec04cb847f000001820cfa8215a8d790000000000000000000000000000000018208ae820d058443b9a3550102", | ||||
| 		wantPacket: &Ping{ | ||||
| 			Version:    4, | ||||
| 			From:       Endpoint{net.ParseIP("127.0.0.1").To4(), 3322, 5544}, | ||||
| 			To:         Endpoint{net.ParseIP("::1"), 2222, 3333}, | ||||
| 			Expiration: 1136239445, | ||||
| 			Rest:       []rlp.RawValue{{0x01}, {0x02}}, | ||||
| 		}, | ||||
| 	}, | ||||
| 	{ | ||||
| 		input: "577be4349c4dd26768081f58de4c6f375a7a22f3f7adda654d1428637412c3d7fe917cadc56d4e5e7ffae1dbe3efffb9849feb71b262de37977e7c7a44e677295680e9e38ab26bee2fcbae207fba3ff3d74069a50b902a82c9903ed37cc993c50001f83e82022bd79020010db83c4d001500000000abcdef12820cfa8215a8d79020010db885a308d313198a2e037073488208ae82823a8443b9a355c5010203040531b9019afde696e582a78fa8d95ea13ce3297d4afb8ba6433e4154caa5ac6431af1b80ba76023fa4090c408f6b4bc3701562c031041d4702971d102c9ab7fa5eed4cd6bab8f7af956f7d565ee1917084a95398b6a21eac920fe3dd1345ec0a7ef39367ee69ddf092cbfe5b93e5e568ebc491983c09c76d922dc3", | ||||
| 		wantPacket: &Ping{ | ||||
| 			Version:    555, | ||||
| 			From:       Endpoint{net.ParseIP("2001:db8:3c4d:15::abcd:ef12"), 3322, 5544}, | ||||
| 			To:         Endpoint{net.ParseIP("2001:db8:85a3:8d3:1319:8a2e:370:7348"), 2222, 33338}, | ||||
| 			Expiration: 1136239445, | ||||
| 			Rest:       []rlp.RawValue{{0xC5, 0x01, 0x02, 0x03, 0x04, 0x05}}, | ||||
| 		}, | ||||
| 	}, | ||||
| 	{ | ||||
| 		input: "09b2428d83348d27cdf7064ad9024f526cebc19e4958f0fdad87c15eb598dd61d08423e0bf66b2069869e1724125f820d851c136684082774f870e614d95a2855d000f05d1648b2d5945470bc187c2d2216fbe870f43ed0909009882e176a46b0102f846d79020010db885a308d313198a2e037073488208ae82823aa0fbc914b16819237dcd8801d7e53f69e9719adecb3cc0e790c57e91ca4461c9548443b9a355c6010203c2040506a0c969a58f6f9095004c0177a6b47f451530cab38966a25cca5cb58f055542124e", | ||||
| 		wantPacket: &Pong{ | ||||
| 			To:         Endpoint{net.ParseIP("2001:db8:85a3:8d3:1319:8a2e:370:7348"), 2222, 33338}, | ||||
| 			ReplyTok:   common.Hex2Bytes("fbc914b16819237dcd8801d7e53f69e9719adecb3cc0e790c57e91ca4461c954"), | ||||
| 			Expiration: 1136239445, | ||||
| 			Rest:       []rlp.RawValue{{0xC6, 0x01, 0x02, 0x03, 0xC2, 0x04, 0x05}, {0x06}}, | ||||
| 		}, | ||||
| 	}, | ||||
| 	{ | ||||
| 		input: "c7c44041b9f7c7e41934417ebac9a8e1a4c6298f74553f2fcfdcae6ed6fe53163eb3d2b52e39fe91831b8a927bf4fc222c3902202027e5e9eb812195f95d20061ef5cd31d502e47ecb61183f74a504fe04c51e73df81f25c4d506b26db4517490103f84eb840ca634cae0d49acb401d8a4c6b6fe8c55b70d115bf400769cc1400f3258cd31387574077f301b421bc84df7266c44e9e6d569fc56be00812904767bf5ccd1fc7f8443b9a35582999983999999280dc62cc8255c73471e0a61da0c89acdc0e035e260add7fc0c04ad9ebf3919644c91cb247affc82b69bd2ca235c71eab8e49737c937a2c396", | ||||
| 		wantPacket: &Findnode{ | ||||
| 			Target:     hexPubkey("ca634cae0d49acb401d8a4c6b6fe8c55b70d115bf400769cc1400f3258cd31387574077f301b421bc84df7266c44e9e6d569fc56be00812904767bf5ccd1fc7f"), | ||||
| 			Expiration: 1136239445, | ||||
| 			Rest:       []rlp.RawValue{{0x82, 0x99, 0x99}, {0x83, 0x99, 0x99, 0x99}}, | ||||
| 		}, | ||||
| 	}, | ||||
| 	{ | ||||
| 		input: "c679fc8fe0b8b12f06577f2e802d34f6fa257e6137a995f6f4cbfc9ee50ed3710faf6e66f932c4c8d81d64343f429651328758b47d3dbc02c4042f0fff6946a50f4a49037a72bb550f3a7872363a83e1b9ee6469856c24eb4ef80b7535bcf99c0004f9015bf90150f84d846321163782115c82115db8403155e1427f85f10a5c9a7755877748041af1bcd8d474ec065eb33df57a97babf54bfd2103575fa829115d224c523596b401065a97f74010610fce76382c0bf32f84984010203040101b840312c55512422cf9b8a4097e9a6ad79402e87a15ae909a4bfefa22398f03d20951933beea1e4dfa6f968212385e829f04c2d314fc2d4e255e0d3bc08792b069dbf8599020010db83c4d001500000000abcdef12820d05820d05b84038643200b172dcfef857492156971f0e6aa2c538d8b74010f8e140811d53b98c765dd2d96126051913f44582e8c199ad7c6d6819e9a56483f637feaac9448aacf8599020010db885a308d313198a2e037073488203e78203e8b8408dcab8618c3253b558d459da53bd8fa68935a719aff8b811197101a4b2b47dd2d47295286fc00cc081bb542d760717d1bdd6bec2c37cd72eca367d6dd3b9df738443b9a355010203b525a138aa34383fec3d2719a0", | ||||
| 		wantPacket: &Neighbors{ | ||||
| 			Nodes: []Node{ | ||||
| 				{ | ||||
| 					ID:  hexPubkey("3155e1427f85f10a5c9a7755877748041af1bcd8d474ec065eb33df57a97babf54bfd2103575fa829115d224c523596b401065a97f74010610fce76382c0bf32"), | ||||
| 					IP:  net.ParseIP("99.33.22.55").To4(), | ||||
| 					UDP: 4444, | ||||
| 					TCP: 4445, | ||||
| 				}, | ||||
| 				{ | ||||
| 					ID:  hexPubkey("312c55512422cf9b8a4097e9a6ad79402e87a15ae909a4bfefa22398f03d20951933beea1e4dfa6f968212385e829f04c2d314fc2d4e255e0d3bc08792b069db"), | ||||
| 					IP:  net.ParseIP("1.2.3.4").To4(), | ||||
| 					UDP: 1, | ||||
| 					TCP: 1, | ||||
| 				}, | ||||
| 				{ | ||||
| 					ID:  hexPubkey("38643200b172dcfef857492156971f0e6aa2c538d8b74010f8e140811d53b98c765dd2d96126051913f44582e8c199ad7c6d6819e9a56483f637feaac9448aac"), | ||||
| 					IP:  net.ParseIP("2001:db8:3c4d:15::abcd:ef12"), | ||||
| 					UDP: 3333, | ||||
| 					TCP: 3333, | ||||
| 				}, | ||||
| 				{ | ||||
| 					ID:  hexPubkey("8dcab8618c3253b558d459da53bd8fa68935a719aff8b811197101a4b2b47dd2d47295286fc00cc081bb542d760717d1bdd6bec2c37cd72eca367d6dd3b9df73"), | ||||
| 					IP:  net.ParseIP("2001:db8:85a3:8d3:1319:8a2e:370:7348"), | ||||
| 					UDP: 999, | ||||
| 					TCP: 1000, | ||||
| 				}, | ||||
| 			}, | ||||
| 			Expiration: 1136239445, | ||||
| 			Rest:       []rlp.RawValue{{0x01}, {0x02}, {0x03}}, | ||||
| 		}, | ||||
| 	}, | ||||
| } | ||||
| 
 | ||||
| // This test checks that the decoder accepts packets according to EIP-8.
 | ||||
| func TestForwardCompatibility(t *testing.T) { | ||||
| 	testkey, _ := crypto.HexToECDSA("b71c71a67e1177ad4e901695e1b4b9ee17ae16c6668d313eac2f96dbcda3f291") | ||||
| 	wantNodeKey := EncodePubkey(&testkey.PublicKey) | ||||
| 
 | ||||
| 	for _, test := range testPackets { | ||||
| 		input, err := hex.DecodeString(test.input) | ||||
| 		if err != nil { | ||||
| 			t.Fatalf("invalid hex: %s", test.input) | ||||
| 		} | ||||
| 		packet, nodekey, _, err := Decode(input) | ||||
| 		if err != nil { | ||||
| 			t.Errorf("did not accept packet %s\n%v", test.input, err) | ||||
| 			continue | ||||
| 		} | ||||
| 		if !reflect.DeepEqual(packet, test.wantPacket) { | ||||
| 			t.Errorf("got %s\nwant %s", spew.Sdump(packet), spew.Sdump(test.wantPacket)) | ||||
| 		} | ||||
| 		if nodekey != wantNodeKey { | ||||
| 			t.Errorf("got id %v\nwant id %v", nodekey, wantNodeKey) | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| func hexPubkey(h string) (ret Pubkey) { | ||||
| 	b, err := hex.DecodeString(h) | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 	if len(b) != len(ret) { | ||||
| 		panic("invalid length") | ||||
| 	} | ||||
| 	copy(ret[:], b) | ||||
| 	return ret | ||||
| } | ||||
		Loading…
	
		Reference in New Issue
	
	Block a user