les: separate peer into clientPeer and serverPeer (#19991)
* les: separate peer into clientPeer and serverPeer * les: address comments
This commit is contained in:
		@@ -42,7 +42,7 @@ type requestBenchmark interface {
 | 
				
			|||||||
	// init initializes the generator for generating the given number of randomized requests
 | 
						// init initializes the generator for generating the given number of randomized requests
 | 
				
			||||||
	init(h *serverHandler, count int) error
 | 
						init(h *serverHandler, count int) error
 | 
				
			||||||
	// request initiates sending a single request to the given peer
 | 
						// request initiates sending a single request to the given peer
 | 
				
			||||||
	request(peer *peer, index int) error
 | 
						request(peer *serverPeer, index int) error
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// benchmarkBlockHeaders implements requestBenchmark
 | 
					// benchmarkBlockHeaders implements requestBenchmark
 | 
				
			||||||
@@ -72,11 +72,11 @@ func (b *benchmarkBlockHeaders) init(h *serverHandler, count int) error {
 | 
				
			|||||||
	return nil
 | 
						return nil
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
func (b *benchmarkBlockHeaders) request(peer *peer, index int) error {
 | 
					func (b *benchmarkBlockHeaders) request(peer *serverPeer, index int) error {
 | 
				
			||||||
	if b.byHash {
 | 
						if b.byHash {
 | 
				
			||||||
		return peer.RequestHeadersByHash(0, 0, b.hashes[index], b.amount, b.skip, b.reverse)
 | 
							return peer.requestHeadersByHash(0, b.hashes[index], b.amount, b.skip, b.reverse)
 | 
				
			||||||
	} else {
 | 
						} else {
 | 
				
			||||||
		return peer.RequestHeadersByNumber(0, 0, uint64(b.offset+rand.Int63n(b.randMax)), b.amount, b.skip, b.reverse)
 | 
							return peer.requestHeadersByNumber(0, uint64(b.offset+rand.Int63n(b.randMax)), b.amount, b.skip, b.reverse)
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -95,11 +95,11 @@ func (b *benchmarkBodiesOrReceipts) init(h *serverHandler, count int) error {
 | 
				
			|||||||
	return nil
 | 
						return nil
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
func (b *benchmarkBodiesOrReceipts) request(peer *peer, index int) error {
 | 
					func (b *benchmarkBodiesOrReceipts) request(peer *serverPeer, index int) error {
 | 
				
			||||||
	if b.receipts {
 | 
						if b.receipts {
 | 
				
			||||||
		return peer.RequestReceipts(0, 0, []common.Hash{b.hashes[index]})
 | 
							return peer.requestReceipts(0, []common.Hash{b.hashes[index]})
 | 
				
			||||||
	} else {
 | 
						} else {
 | 
				
			||||||
		return peer.RequestBodies(0, 0, []common.Hash{b.hashes[index]})
 | 
							return peer.requestBodies(0, []common.Hash{b.hashes[index]})
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -114,13 +114,13 @@ func (b *benchmarkProofsOrCode) init(h *serverHandler, count int) error {
 | 
				
			|||||||
	return nil
 | 
						return nil
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
func (b *benchmarkProofsOrCode) request(peer *peer, index int) error {
 | 
					func (b *benchmarkProofsOrCode) request(peer *serverPeer, index int) error {
 | 
				
			||||||
	key := make([]byte, 32)
 | 
						key := make([]byte, 32)
 | 
				
			||||||
	rand.Read(key)
 | 
						rand.Read(key)
 | 
				
			||||||
	if b.code {
 | 
						if b.code {
 | 
				
			||||||
		return peer.RequestCode(0, 0, []CodeReq{{BHash: b.headHash, AccKey: key}})
 | 
							return peer.requestCode(0, []CodeReq{{BHash: b.headHash, AccKey: key}})
 | 
				
			||||||
	} else {
 | 
						} else {
 | 
				
			||||||
		return peer.RequestProofs(0, 0, []ProofReq{{BHash: b.headHash, Key: key}})
 | 
							return peer.requestProofs(0, []ProofReq{{BHash: b.headHash, Key: key}})
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -144,7 +144,7 @@ func (b *benchmarkHelperTrie) init(h *serverHandler, count int) error {
 | 
				
			|||||||
	return nil
 | 
						return nil
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
func (b *benchmarkHelperTrie) request(peer *peer, index int) error {
 | 
					func (b *benchmarkHelperTrie) request(peer *serverPeer, index int) error {
 | 
				
			||||||
	reqs := make([]HelperTrieReq, b.reqCount)
 | 
						reqs := make([]HelperTrieReq, b.reqCount)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	if b.bloom {
 | 
						if b.bloom {
 | 
				
			||||||
@@ -163,7 +163,7 @@ func (b *benchmarkHelperTrie) request(peer *peer, index int) error {
 | 
				
			|||||||
		}
 | 
							}
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	return peer.RequestHelperTrieProofs(0, 0, reqs)
 | 
						return peer.requestHelperTrieProofs(0, reqs)
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// benchmarkTxSend implements requestBenchmark
 | 
					// benchmarkTxSend implements requestBenchmark
 | 
				
			||||||
@@ -189,9 +189,9 @@ func (b *benchmarkTxSend) init(h *serverHandler, count int) error {
 | 
				
			|||||||
	return nil
 | 
						return nil
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
func (b *benchmarkTxSend) request(peer *peer, index int) error {
 | 
					func (b *benchmarkTxSend) request(peer *serverPeer, index int) error {
 | 
				
			||||||
	enc, _ := rlp.EncodeToBytes(types.Transactions{b.txs[index]})
 | 
						enc, _ := rlp.EncodeToBytes(types.Transactions{b.txs[index]})
 | 
				
			||||||
	return peer.SendTxs(0, 0, enc)
 | 
						return peer.sendTxs(0, enc)
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// benchmarkTxStatus implements requestBenchmark
 | 
					// benchmarkTxStatus implements requestBenchmark
 | 
				
			||||||
@@ -201,10 +201,10 @@ func (b *benchmarkTxStatus) init(h *serverHandler, count int) error {
 | 
				
			|||||||
	return nil
 | 
						return nil
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
func (b *benchmarkTxStatus) request(peer *peer, index int) error {
 | 
					func (b *benchmarkTxStatus) request(peer *serverPeer, index int) error {
 | 
				
			||||||
	var hash common.Hash
 | 
						var hash common.Hash
 | 
				
			||||||
	rand.Read(hash[:])
 | 
						rand.Read(hash[:])
 | 
				
			||||||
	return peer.RequestTxStatus(0, 0, []common.Hash{hash})
 | 
						return peer.requestTxStatus(0, []common.Hash{hash})
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// benchmarkSetup stores measurement data for a single benchmark type
 | 
					// benchmarkSetup stores measurement data for a single benchmark type
 | 
				
			||||||
@@ -283,18 +283,17 @@ func (h *serverHandler) measure(setup *benchmarkSetup, count int) error {
 | 
				
			|||||||
	var id enode.ID
 | 
						var id enode.ID
 | 
				
			||||||
	rand.Read(id[:])
 | 
						rand.Read(id[:])
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	clientPeer := newPeer(lpv2, NetworkId, false, p2p.NewPeer(id, "client", nil), clientMeteredPipe)
 | 
						peer1 := newServerPeer(lpv2, NetworkId, false, p2p.NewPeer(id, "client", nil), clientMeteredPipe)
 | 
				
			||||||
	serverPeer := newPeer(lpv2, NetworkId, false, p2p.NewPeer(id, "server", nil), serverMeteredPipe)
 | 
						peer2 := newClientPeer(lpv2, NetworkId, p2p.NewPeer(id, "server", nil), serverMeteredPipe)
 | 
				
			||||||
	serverPeer.sendQueue = newExecQueue(count)
 | 
						peer2.announceType = announceTypeNone
 | 
				
			||||||
	serverPeer.announceType = announceTypeNone
 | 
						peer2.fcCosts = make(requestCostTable)
 | 
				
			||||||
	serverPeer.fcCosts = make(requestCostTable)
 | 
					 | 
				
			||||||
	c := &requestCosts{}
 | 
						c := &requestCosts{}
 | 
				
			||||||
	for code := range requests {
 | 
						for code := range requests {
 | 
				
			||||||
		serverPeer.fcCosts[code] = c
 | 
							peer2.fcCosts[code] = c
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	serverPeer.fcParams = flowcontrol.ServerParams{BufLimit: 1, MinRecharge: 1}
 | 
						peer2.fcParams = flowcontrol.ServerParams{BufLimit: 1, MinRecharge: 1}
 | 
				
			||||||
	serverPeer.fcClient = flowcontrol.NewClientNode(h.server.fcManager, serverPeer.fcParams)
 | 
						peer2.fcClient = flowcontrol.NewClientNode(h.server.fcManager, peer2.fcParams)
 | 
				
			||||||
	defer serverPeer.fcClient.Disconnect()
 | 
						defer peer2.fcClient.Disconnect()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	if err := setup.req.init(h, count); err != nil {
 | 
						if err := setup.req.init(h, count); err != nil {
 | 
				
			||||||
		return err
 | 
							return err
 | 
				
			||||||
@@ -305,7 +304,7 @@ func (h *serverHandler) measure(setup *benchmarkSetup, count int) error {
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
	go func() {
 | 
						go func() {
 | 
				
			||||||
		for i := 0; i < count; i++ {
 | 
							for i := 0; i < count; i++ {
 | 
				
			||||||
			if err := setup.req.request(clientPeer, i); err != nil {
 | 
								if err := setup.req.request(peer1, i); err != nil {
 | 
				
			||||||
				errCh <- err
 | 
									errCh <- err
 | 
				
			||||||
				return
 | 
									return
 | 
				
			||||||
			}
 | 
								}
 | 
				
			||||||
@@ -313,7 +312,7 @@ func (h *serverHandler) measure(setup *benchmarkSetup, count int) error {
 | 
				
			|||||||
	}()
 | 
						}()
 | 
				
			||||||
	go func() {
 | 
						go func() {
 | 
				
			||||||
		for i := 0; i < count; i++ {
 | 
							for i := 0; i < count; i++ {
 | 
				
			||||||
			if err := h.handleMsg(serverPeer, &sync.WaitGroup{}); err != nil {
 | 
								if err := h.handleMsg(peer2, &sync.WaitGroup{}); err != nil {
 | 
				
			||||||
				errCh <- err
 | 
									errCh <- err
 | 
				
			||||||
				return
 | 
									return
 | 
				
			||||||
			}
 | 
								}
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -49,6 +49,7 @@ import (
 | 
				
			|||||||
type LightEthereum struct {
 | 
					type LightEthereum struct {
 | 
				
			||||||
	lesCommons
 | 
						lesCommons
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
						peers      *serverPeerSet
 | 
				
			||||||
	reqDist    *requestDistributor
 | 
						reqDist    *requestDistributor
 | 
				
			||||||
	retriever  *retrieveManager
 | 
						retriever  *retrieveManager
 | 
				
			||||||
	odr        *LesOdr
 | 
						odr        *LesOdr
 | 
				
			||||||
@@ -80,7 +81,7 @@ func New(ctx *node.ServiceContext, config *eth.Config) (*LightEthereum, error) {
 | 
				
			|||||||
	}
 | 
						}
 | 
				
			||||||
	log.Info("Initialised chain configuration", "config", chainConfig)
 | 
						log.Info("Initialised chain configuration", "config", chainConfig)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	peers := newPeerSet()
 | 
						peers := newServerPeerSet()
 | 
				
			||||||
	leth := &LightEthereum{
 | 
						leth := &LightEthereum{
 | 
				
			||||||
		lesCommons: lesCommons{
 | 
							lesCommons: lesCommons{
 | 
				
			||||||
			genesis:     genesisHash,
 | 
								genesis:     genesisHash,
 | 
				
			||||||
@@ -88,9 +89,9 @@ func New(ctx *node.ServiceContext, config *eth.Config) (*LightEthereum, error) {
 | 
				
			|||||||
			chainConfig: chainConfig,
 | 
								chainConfig: chainConfig,
 | 
				
			||||||
			iConfig:     light.DefaultClientIndexerConfig,
 | 
								iConfig:     light.DefaultClientIndexerConfig,
 | 
				
			||||||
			chainDb:     chainDb,
 | 
								chainDb:     chainDb,
 | 
				
			||||||
			peers:       peers,
 | 
					 | 
				
			||||||
			closeCh:     make(chan struct{}),
 | 
								closeCh:     make(chan struct{}),
 | 
				
			||||||
		},
 | 
							},
 | 
				
			||||||
 | 
							peers:          peers,
 | 
				
			||||||
		eventMux:       ctx.EventMux,
 | 
							eventMux:       ctx.EventMux,
 | 
				
			||||||
		reqDist:        newRequestDistributor(peers, &mclock.System{}),
 | 
							reqDist:        newRequestDistributor(peers, &mclock.System{}),
 | 
				
			||||||
		accountManager: ctx.AccountManager,
 | 
							accountManager: ctx.AccountManager,
 | 
				
			||||||
@@ -225,7 +226,7 @@ func (s *LightEthereum) EventMux() *event.TypeMux           { return s.eventMux
 | 
				
			|||||||
// network protocols to start.
 | 
					// network protocols to start.
 | 
				
			||||||
func (s *LightEthereum) Protocols() []p2p.Protocol {
 | 
					func (s *LightEthereum) Protocols() []p2p.Protocol {
 | 
				
			||||||
	return s.makeProtocols(ClientProtocolVersions, s.handler.runPeer, func(id enode.ID) interface{} {
 | 
						return s.makeProtocols(ClientProtocolVersions, s.handler.runPeer, func(id enode.ID) interface{} {
 | 
				
			||||||
		if p := s.peers.Peer(peerIdToString(id)); p != nil {
 | 
							if p := s.peers.peer(peerIdToString(id)); p != nil {
 | 
				
			||||||
			return p.Info()
 | 
								return p.Info()
 | 
				
			||||||
		}
 | 
							}
 | 
				
			||||||
		return nil
 | 
							return nil
 | 
				
			||||||
@@ -253,7 +254,7 @@ func (s *LightEthereum) Start(srvr *p2p.Server) error {
 | 
				
			|||||||
// Ethereum protocol.
 | 
					// Ethereum protocol.
 | 
				
			||||||
func (s *LightEthereum) Stop() error {
 | 
					func (s *LightEthereum) Stop() error {
 | 
				
			||||||
	close(s.closeCh)
 | 
						close(s.closeCh)
 | 
				
			||||||
	s.peers.Close()
 | 
						s.peers.close()
 | 
				
			||||||
	s.reqDist.close()
 | 
						s.reqDist.close()
 | 
				
			||||||
	s.odr.Stop()
 | 
						s.odr.Stop()
 | 
				
			||||||
	s.relay.Stop()
 | 
						s.relay.Stop()
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -65,7 +65,7 @@ func newClientHandler(ulcServers []string, ulcFraction int, checkpoint *params.T
 | 
				
			|||||||
	}
 | 
						}
 | 
				
			||||||
	handler.fetcher = newLightFetcher(handler)
 | 
						handler.fetcher = newLightFetcher(handler)
 | 
				
			||||||
	handler.downloader = downloader.New(height, backend.chainDb, nil, backend.eventMux, nil, backend.blockchain, handler.removePeer)
 | 
						handler.downloader = downloader.New(height, backend.chainDb, nil, backend.eventMux, nil, backend.blockchain, handler.removePeer)
 | 
				
			||||||
	handler.backend.peers.notify((*downloaderPeerNotify)(handler))
 | 
						handler.backend.peers.subscribe((*downloaderPeerNotify)(handler))
 | 
				
			||||||
	return handler
 | 
						return handler
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -82,7 +82,8 @@ func (h *clientHandler) runPeer(version uint, p *p2p.Peer, rw p2p.MsgReadWriter)
 | 
				
			|||||||
	if h.ulc != nil {
 | 
						if h.ulc != nil {
 | 
				
			||||||
		trusted = h.ulc.trusted(p.ID())
 | 
							trusted = h.ulc.trusted(p.ID())
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	peer := newPeer(int(version), h.backend.config.NetworkId, trusted, p, newMeteredMsgWriter(rw, int(version)))
 | 
						peer := newServerPeer(int(version), h.backend.config.NetworkId, trusted, p, newMeteredMsgWriter(rw, int(version)))
 | 
				
			||||||
 | 
						defer peer.close()
 | 
				
			||||||
	peer.poolEntry = h.backend.serverPool.connect(peer, peer.Node())
 | 
						peer.poolEntry = h.backend.serverPool.connect(peer, peer.Node())
 | 
				
			||||||
	if peer.poolEntry == nil {
 | 
						if peer.poolEntry == nil {
 | 
				
			||||||
		return p2p.DiscRequested
 | 
							return p2p.DiscRequested
 | 
				
			||||||
@@ -94,8 +95,8 @@ func (h *clientHandler) runPeer(version uint, p *p2p.Peer, rw p2p.MsgReadWriter)
 | 
				
			|||||||
	return err
 | 
						return err
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
func (h *clientHandler) handle(p *peer) error {
 | 
					func (h *clientHandler) handle(p *serverPeer) error {
 | 
				
			||||||
	if h.backend.peers.Len() >= h.backend.config.LightPeers && !p.Peer.Info().Network.Trusted {
 | 
						if h.backend.peers.len() >= h.backend.config.LightPeers && !p.Peer.Info().Network.Trusted {
 | 
				
			||||||
		return p2p.DiscTooManyPeers
 | 
							return p2p.DiscTooManyPeers
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	p.Log().Debug("Light Ethereum peer connected", "name", p.Name())
 | 
						p.Log().Debug("Light Ethereum peer connected", "name", p.Name())
 | 
				
			||||||
@@ -112,20 +113,20 @@ func (h *clientHandler) handle(p *peer) error {
 | 
				
			|||||||
		return err
 | 
							return err
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	// Register the peer locally
 | 
						// Register the peer locally
 | 
				
			||||||
	if err := h.backend.peers.Register(p); err != nil {
 | 
						if err := h.backend.peers.register(p); err != nil {
 | 
				
			||||||
		p.Log().Error("Light Ethereum peer registration failed", "err", err)
 | 
							p.Log().Error("Light Ethereum peer registration failed", "err", err)
 | 
				
			||||||
		return err
 | 
							return err
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	serverConnectionGauge.Update(int64(h.backend.peers.Len()))
 | 
						serverConnectionGauge.Update(int64(h.backend.peers.len()))
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	connectedAt := mclock.Now()
 | 
						connectedAt := mclock.Now()
 | 
				
			||||||
	defer func() {
 | 
						defer func() {
 | 
				
			||||||
		h.backend.peers.Unregister(p.id)
 | 
							h.backend.peers.unregister(p.id)
 | 
				
			||||||
		connectionTimer.Update(time.Duration(mclock.Now() - connectedAt))
 | 
							connectionTimer.Update(time.Duration(mclock.Now() - connectedAt))
 | 
				
			||||||
		serverConnectionGauge.Update(int64(h.backend.peers.Len()))
 | 
							serverConnectionGauge.Update(int64(h.backend.peers.len()))
 | 
				
			||||||
	}()
 | 
						}()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	h.fetcher.announce(p, p.headInfo)
 | 
						h.fetcher.announce(p, &announceData{Hash: p.headInfo.Hash, Number: p.headInfo.Number, Td: p.headInfo.Td})
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	// pool entry can be nil during the unit test.
 | 
						// pool entry can be nil during the unit test.
 | 
				
			||||||
	if p.poolEntry != nil {
 | 
						if p.poolEntry != nil {
 | 
				
			||||||
@@ -143,7 +144,7 @@ func (h *clientHandler) handle(p *peer) error {
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
// handleMsg is invoked whenever an inbound message is received from a remote
 | 
					// handleMsg is invoked whenever an inbound message is received from a remote
 | 
				
			||||||
// peer. The remote connection is torn down upon returning any error.
 | 
					// peer. The remote connection is torn down upon returning any error.
 | 
				
			||||||
func (h *clientHandler) handleMsg(p *peer) error {
 | 
					func (h *clientHandler) handleMsg(p *serverPeer) error {
 | 
				
			||||||
	// Read the next message from the remote peer, and ensure it's fully consumed
 | 
						// Read the next message from the remote peer, and ensure it's fully consumed
 | 
				
			||||||
	msg, err := p.rw.ReadMsg()
 | 
						msg, err := p.rw.ReadMsg()
 | 
				
			||||||
	if err != nil {
 | 
						if err != nil {
 | 
				
			||||||
@@ -297,7 +298,7 @@ func (h *clientHandler) handleMsg(p *peer) error {
 | 
				
			|||||||
			Obj:     resp.Status,
 | 
								Obj:     resp.Status,
 | 
				
			||||||
		}
 | 
							}
 | 
				
			||||||
	case StopMsg:
 | 
						case StopMsg:
 | 
				
			||||||
		p.freezeServer(true)
 | 
							p.freeze()
 | 
				
			||||||
		h.backend.retriever.frozen(p)
 | 
							h.backend.retriever.frozen(p)
 | 
				
			||||||
		p.Log().Debug("Service stopped")
 | 
							p.Log().Debug("Service stopped")
 | 
				
			||||||
	case ResumeMsg:
 | 
						case ResumeMsg:
 | 
				
			||||||
@@ -306,7 +307,7 @@ func (h *clientHandler) handleMsg(p *peer) error {
 | 
				
			|||||||
			return errResp(ErrDecode, "msg %v: %v", msg, err)
 | 
								return errResp(ErrDecode, "msg %v: %v", msg, err)
 | 
				
			||||||
		}
 | 
							}
 | 
				
			||||||
		p.fcServer.ResumeFreeze(bv)
 | 
							p.fcServer.ResumeFreeze(bv)
 | 
				
			||||||
		p.freezeServer(false)
 | 
							p.unfreeze()
 | 
				
			||||||
		p.Log().Debug("Service resumed")
 | 
							p.Log().Debug("Service resumed")
 | 
				
			||||||
	default:
 | 
						default:
 | 
				
			||||||
		p.Log().Trace("Received invalid message", "code", msg.Code)
 | 
							p.Log().Trace("Received invalid message", "code", msg.Code)
 | 
				
			||||||
@@ -315,8 +316,8 @@ func (h *clientHandler) handleMsg(p *peer) error {
 | 
				
			|||||||
	// Deliver the received response to retriever.
 | 
						// Deliver the received response to retriever.
 | 
				
			||||||
	if deliverMsg != nil {
 | 
						if deliverMsg != nil {
 | 
				
			||||||
		if err := h.backend.retriever.deliver(p, deliverMsg); err != nil {
 | 
							if err := h.backend.retriever.deliver(p, deliverMsg); err != nil {
 | 
				
			||||||
			p.responseErrors++
 | 
								p.errCount++
 | 
				
			||||||
			if p.responseErrors > maxResponseErrors {
 | 
								if p.errCount > maxResponseErrors {
 | 
				
			||||||
				return err
 | 
									return err
 | 
				
			||||||
			}
 | 
								}
 | 
				
			||||||
		}
 | 
							}
 | 
				
			||||||
@@ -325,12 +326,12 @@ func (h *clientHandler) handleMsg(p *peer) error {
 | 
				
			|||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
func (h *clientHandler) removePeer(id string) {
 | 
					func (h *clientHandler) removePeer(id string) {
 | 
				
			||||||
	h.backend.peers.Unregister(id)
 | 
						h.backend.peers.unregister(id)
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
type peerConnection struct {
 | 
					type peerConnection struct {
 | 
				
			||||||
	handler *clientHandler
 | 
						handler *clientHandler
 | 
				
			||||||
	peer    *peer
 | 
						peer    *serverPeer
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
func (pc *peerConnection) Head() (common.Hash, *big.Int) {
 | 
					func (pc *peerConnection) Head() (common.Hash, *big.Int) {
 | 
				
			||||||
@@ -340,18 +341,18 @@ func (pc *peerConnection) Head() (common.Hash, *big.Int) {
 | 
				
			|||||||
func (pc *peerConnection) RequestHeadersByHash(origin common.Hash, amount int, skip int, reverse bool) error {
 | 
					func (pc *peerConnection) RequestHeadersByHash(origin common.Hash, amount int, skip int, reverse bool) error {
 | 
				
			||||||
	rq := &distReq{
 | 
						rq := &distReq{
 | 
				
			||||||
		getCost: func(dp distPeer) uint64 {
 | 
							getCost: func(dp distPeer) uint64 {
 | 
				
			||||||
			peer := dp.(*peer)
 | 
								peer := dp.(*serverPeer)
 | 
				
			||||||
			return peer.GetRequestCost(GetBlockHeadersMsg, amount)
 | 
								return peer.getRequestCost(GetBlockHeadersMsg, amount)
 | 
				
			||||||
		},
 | 
							},
 | 
				
			||||||
		canSend: func(dp distPeer) bool {
 | 
							canSend: func(dp distPeer) bool {
 | 
				
			||||||
			return dp.(*peer) == pc.peer
 | 
								return dp.(*serverPeer) == pc.peer
 | 
				
			||||||
		},
 | 
							},
 | 
				
			||||||
		request: func(dp distPeer) func() {
 | 
							request: func(dp distPeer) func() {
 | 
				
			||||||
			reqID := genReqID()
 | 
								reqID := genReqID()
 | 
				
			||||||
			peer := dp.(*peer)
 | 
								peer := dp.(*serverPeer)
 | 
				
			||||||
			cost := peer.GetRequestCost(GetBlockHeadersMsg, amount)
 | 
								cost := peer.getRequestCost(GetBlockHeadersMsg, amount)
 | 
				
			||||||
			peer.fcServer.QueuedRequest(reqID, cost)
 | 
								peer.fcServer.QueuedRequest(reqID, cost)
 | 
				
			||||||
			return func() { peer.RequestHeadersByHash(reqID, cost, origin, amount, skip, reverse) }
 | 
								return func() { peer.requestHeadersByHash(reqID, origin, amount, skip, reverse) }
 | 
				
			||||||
		},
 | 
							},
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	_, ok := <-pc.handler.backend.reqDist.queue(rq)
 | 
						_, ok := <-pc.handler.backend.reqDist.queue(rq)
 | 
				
			||||||
@@ -364,18 +365,18 @@ func (pc *peerConnection) RequestHeadersByHash(origin common.Hash, amount int, s
 | 
				
			|||||||
func (pc *peerConnection) RequestHeadersByNumber(origin uint64, amount int, skip int, reverse bool) error {
 | 
					func (pc *peerConnection) RequestHeadersByNumber(origin uint64, amount int, skip int, reverse bool) error {
 | 
				
			||||||
	rq := &distReq{
 | 
						rq := &distReq{
 | 
				
			||||||
		getCost: func(dp distPeer) uint64 {
 | 
							getCost: func(dp distPeer) uint64 {
 | 
				
			||||||
			peer := dp.(*peer)
 | 
								peer := dp.(*serverPeer)
 | 
				
			||||||
			return peer.GetRequestCost(GetBlockHeadersMsg, amount)
 | 
								return peer.getRequestCost(GetBlockHeadersMsg, amount)
 | 
				
			||||||
		},
 | 
							},
 | 
				
			||||||
		canSend: func(dp distPeer) bool {
 | 
							canSend: func(dp distPeer) bool {
 | 
				
			||||||
			return dp.(*peer) == pc.peer
 | 
								return dp.(*serverPeer) == pc.peer
 | 
				
			||||||
		},
 | 
							},
 | 
				
			||||||
		request: func(dp distPeer) func() {
 | 
							request: func(dp distPeer) func() {
 | 
				
			||||||
			reqID := genReqID()
 | 
								reqID := genReqID()
 | 
				
			||||||
			peer := dp.(*peer)
 | 
								peer := dp.(*serverPeer)
 | 
				
			||||||
			cost := peer.GetRequestCost(GetBlockHeadersMsg, amount)
 | 
								cost := peer.getRequestCost(GetBlockHeadersMsg, amount)
 | 
				
			||||||
			peer.fcServer.QueuedRequest(reqID, cost)
 | 
								peer.fcServer.QueuedRequest(reqID, cost)
 | 
				
			||||||
			return func() { peer.RequestHeadersByNumber(reqID, cost, origin, amount, skip, reverse) }
 | 
								return func() { peer.requestHeadersByNumber(reqID, origin, amount, skip, reverse) }
 | 
				
			||||||
		},
 | 
							},
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	_, ok := <-pc.handler.backend.reqDist.queue(rq)
 | 
						_, ok := <-pc.handler.backend.reqDist.queue(rq)
 | 
				
			||||||
@@ -388,7 +389,7 @@ func (pc *peerConnection) RequestHeadersByNumber(origin uint64, amount int, skip
 | 
				
			|||||||
// downloaderPeerNotify implements peerSetNotify
 | 
					// downloaderPeerNotify implements peerSetNotify
 | 
				
			||||||
type downloaderPeerNotify clientHandler
 | 
					type downloaderPeerNotify clientHandler
 | 
				
			||||||
 | 
					
 | 
				
			||||||
func (d *downloaderPeerNotify) registerPeer(p *peer) {
 | 
					func (d *downloaderPeerNotify) registerPeer(p *serverPeer) {
 | 
				
			||||||
	h := (*clientHandler)(d)
 | 
						h := (*clientHandler)(d)
 | 
				
			||||||
	pc := &peerConnection{
 | 
						pc := &peerConnection{
 | 
				
			||||||
		handler: h,
 | 
							handler: h,
 | 
				
			||||||
@@ -397,7 +398,7 @@ func (d *downloaderPeerNotify) registerPeer(p *peer) {
 | 
				
			|||||||
	h.downloader.RegisterLightPeer(p.id, ethVersion, pc)
 | 
						h.downloader.RegisterLightPeer(p.id, ethVersion, pc)
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
func (d *downloaderPeerNotify) unregisterPeer(p *peer) {
 | 
					func (d *downloaderPeerNotify) unregisterPeer(p *serverPeer) {
 | 
				
			||||||
	h := (*clientHandler)(d)
 | 
						h := (*clientHandler)(d)
 | 
				
			||||||
	h.downloader.UnregisterPeer(p.id)
 | 
						h.downloader.UnregisterPeer(p.id)
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -97,12 +97,12 @@ type clientPool struct {
 | 
				
			|||||||
	disableBias       bool           // Disable connection bias(used in testing)
 | 
						disableBias       bool           // Disable connection bias(used in testing)
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// clientPeer represents a client in the pool.
 | 
					// clientPoolPeer represents a client peer in the pool.
 | 
				
			||||||
// Positive balances are assigned to node key while negative balances are assigned
 | 
					// Positive balances are assigned to node key while negative balances are assigned
 | 
				
			||||||
// to freeClientId. Currently network IP address without port is used because
 | 
					// to freeClientId. Currently network IP address without port is used because
 | 
				
			||||||
// clients have a limited access to IP addresses while new node keys can be easily
 | 
					// clients have a limited access to IP addresses while new node keys can be easily
 | 
				
			||||||
// generated so it would be useless to assign a negative value to them.
 | 
					// generated so it would be useless to assign a negative value to them.
 | 
				
			||||||
type clientPeer interface {
 | 
					type clientPoolPeer interface {
 | 
				
			||||||
	ID() enode.ID
 | 
						ID() enode.ID
 | 
				
			||||||
	freeClientId() string
 | 
						freeClientId() string
 | 
				
			||||||
	updateCapacity(uint64)
 | 
						updateCapacity(uint64)
 | 
				
			||||||
@@ -117,7 +117,7 @@ type clientInfo struct {
 | 
				
			|||||||
	capacity               uint64
 | 
						capacity               uint64
 | 
				
			||||||
	priority               bool
 | 
						priority               bool
 | 
				
			||||||
	pool                   *clientPool
 | 
						pool                   *clientPool
 | 
				
			||||||
	peer                   clientPeer
 | 
						peer                   clientPoolPeer
 | 
				
			||||||
	queueIndex             int // position in connectedQueue
 | 
						queueIndex             int // position in connectedQueue
 | 
				
			||||||
	balanceTracker         balanceTracker
 | 
						balanceTracker         balanceTracker
 | 
				
			||||||
	posFactors, negFactors priceFactors
 | 
						posFactors, negFactors priceFactors
 | 
				
			||||||
@@ -207,7 +207,7 @@ func (f *clientPool) stop() {
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
// connect should be called after a successful handshake. If the connection was
 | 
					// connect should be called after a successful handshake. If the connection was
 | 
				
			||||||
// rejected, there is no need to call disconnect.
 | 
					// rejected, there is no need to call disconnect.
 | 
				
			||||||
func (f *clientPool) connect(peer clientPeer, capacity uint64) bool {
 | 
					func (f *clientPool) connect(peer clientPoolPeer, capacity uint64) bool {
 | 
				
			||||||
	f.lock.Lock()
 | 
						f.lock.Lock()
 | 
				
			||||||
	defer f.lock.Unlock()
 | 
						defer f.lock.Unlock()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -322,7 +322,7 @@ func (f *clientPool) connect(peer clientPeer, capacity uint64) bool {
 | 
				
			|||||||
// disconnect should be called when a connection is terminated. If the disconnection
 | 
					// disconnect should be called when a connection is terminated. If the disconnection
 | 
				
			||||||
// was initiated by the pool itself using disconnectFn then calling disconnect is
 | 
					// was initiated by the pool itself using disconnectFn then calling disconnect is
 | 
				
			||||||
// not necessary but permitted.
 | 
					// not necessary but permitted.
 | 
				
			||||||
func (f *clientPool) disconnect(p clientPeer) {
 | 
					func (f *clientPool) disconnect(p clientPoolPeer) {
 | 
				
			||||||
	f.lock.Lock()
 | 
						f.lock.Lock()
 | 
				
			||||||
	defer f.lock.Unlock()
 | 
						defer f.lock.Unlock()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -516,7 +516,7 @@ func (f *clientPool) setCapacity(c *clientInfo, capacity uint64) error {
 | 
				
			|||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// requestCost feeds request cost after serving a request from the given peer.
 | 
					// requestCost feeds request cost after serving a request from the given peer.
 | 
				
			||||||
func (f *clientPool) requestCost(p *peer, cost uint64) {
 | 
					func (f *clientPool) requestCost(p *clientPeer, cost uint64) {
 | 
				
			||||||
	f.lock.Lock()
 | 
						f.lock.Lock()
 | 
				
			||||||
	defer f.lock.Unlock()
 | 
						defer f.lock.Unlock()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -61,7 +61,6 @@ type lesCommons struct {
 | 
				
			|||||||
	chainConfig                  *params.ChainConfig
 | 
						chainConfig                  *params.ChainConfig
 | 
				
			||||||
	iConfig                      *light.IndexerConfig
 | 
						iConfig                      *light.IndexerConfig
 | 
				
			||||||
	chainDb                      ethdb.Database
 | 
						chainDb                      ethdb.Database
 | 
				
			||||||
	peers                        *peerSet
 | 
					 | 
				
			||||||
	chainReader                  chainReader
 | 
						chainReader                  chainReader
 | 
				
			||||||
	chtIndexer, bloomTrieIndexer *core.ChainIndexer
 | 
						chtIndexer, bloomTrieIndexer *core.ChainIndexer
 | 
				
			||||||
	oracle                       *checkpointoracle.CheckpointOracle
 | 
						oracle                       *checkpointoracle.CheckpointOracle
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -49,7 +49,7 @@ type requestDistributor struct {
 | 
				
			|||||||
type distPeer interface {
 | 
					type distPeer interface {
 | 
				
			||||||
	waitBefore(uint64) (time.Duration, float64)
 | 
						waitBefore(uint64) (time.Duration, float64)
 | 
				
			||||||
	canQueue() bool
 | 
						canQueue() bool
 | 
				
			||||||
	queueSend(f func())
 | 
						queueSend(f func()) bool
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// distReq is the request abstraction used by the distributor. It is based on
 | 
					// distReq is the request abstraction used by the distributor. It is based on
 | 
				
			||||||
@@ -73,7 +73,7 @@ type distReq struct {
 | 
				
			|||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// newRequestDistributor creates a new request distributor
 | 
					// newRequestDistributor creates a new request distributor
 | 
				
			||||||
func newRequestDistributor(peers *peerSet, clock mclock.Clock) *requestDistributor {
 | 
					func newRequestDistributor(peers *serverPeerSet, clock mclock.Clock) *requestDistributor {
 | 
				
			||||||
	d := &requestDistributor{
 | 
						d := &requestDistributor{
 | 
				
			||||||
		clock:    clock,
 | 
							clock:    clock,
 | 
				
			||||||
		reqQueue: list.New(),
 | 
							reqQueue: list.New(),
 | 
				
			||||||
@@ -82,7 +82,7 @@ func newRequestDistributor(peers *peerSet, clock mclock.Clock) *requestDistribut
 | 
				
			|||||||
		peers:    make(map[distPeer]struct{}),
 | 
							peers:    make(map[distPeer]struct{}),
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	if peers != nil {
 | 
						if peers != nil {
 | 
				
			||||||
		peers.notify(d)
 | 
							peers.subscribe(d)
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	d.wg.Add(1)
 | 
						d.wg.Add(1)
 | 
				
			||||||
	go d.loop()
 | 
						go d.loop()
 | 
				
			||||||
@@ -90,14 +90,14 @@ func newRequestDistributor(peers *peerSet, clock mclock.Clock) *requestDistribut
 | 
				
			|||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// registerPeer implements peerSetNotify
 | 
					// registerPeer implements peerSetNotify
 | 
				
			||||||
func (d *requestDistributor) registerPeer(p *peer) {
 | 
					func (d *requestDistributor) registerPeer(p *serverPeer) {
 | 
				
			||||||
	d.peerLock.Lock()
 | 
						d.peerLock.Lock()
 | 
				
			||||||
	d.peers[p] = struct{}{}
 | 
						d.peers[p] = struct{}{}
 | 
				
			||||||
	d.peerLock.Unlock()
 | 
						d.peerLock.Unlock()
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// unregisterPeer implements peerSetNotify
 | 
					// unregisterPeer implements peerSetNotify
 | 
				
			||||||
func (d *requestDistributor) unregisterPeer(p *peer) {
 | 
					func (d *requestDistributor) unregisterPeer(p *serverPeer) {
 | 
				
			||||||
	d.peerLock.Lock()
 | 
						d.peerLock.Lock()
 | 
				
			||||||
	delete(d.peers, p)
 | 
						delete(d.peers, p)
 | 
				
			||||||
	d.peerLock.Unlock()
 | 
						d.peerLock.Unlock()
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -105,8 +105,9 @@ func (p *testDistPeer) canQueue() bool {
 | 
				
			|||||||
	return true
 | 
						return true
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
func (p *testDistPeer) queueSend(f func()) {
 | 
					func (p *testDistPeer) queueSend(f func()) bool {
 | 
				
			||||||
	f()
 | 
						f()
 | 
				
			||||||
 | 
						return true
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
func TestRequestDistributor(t *testing.T) {
 | 
					func TestRequestDistributor(t *testing.T) {
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -45,10 +45,10 @@ type lightFetcher struct {
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
	lock            sync.Mutex // lock protects access to the fetcher's internal state variables except sent requests
 | 
						lock            sync.Mutex // lock protects access to the fetcher's internal state variables except sent requests
 | 
				
			||||||
	maxConfirmedTd  *big.Int
 | 
						maxConfirmedTd  *big.Int
 | 
				
			||||||
	peers           map[*peer]*fetcherPeerInfo
 | 
						peers           map[*serverPeer]*fetcherPeerInfo
 | 
				
			||||||
	lastUpdateStats *updateStatsEntry
 | 
						lastUpdateStats *updateStatsEntry
 | 
				
			||||||
	syncing         bool
 | 
						syncing         bool
 | 
				
			||||||
	syncDone        chan *peer
 | 
						syncDone        chan *serverPeer
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	reqMu             sync.RWMutex // reqMu protects access to sent header fetch requests
 | 
						reqMu             sync.RWMutex // reqMu protects access to sent header fetch requests
 | 
				
			||||||
	requested         map[uint64]fetchRequest
 | 
						requested         map[uint64]fetchRequest
 | 
				
			||||||
@@ -96,7 +96,7 @@ type fetcherTreeNode struct {
 | 
				
			|||||||
type fetchRequest struct {
 | 
					type fetchRequest struct {
 | 
				
			||||||
	hash    common.Hash
 | 
						hash    common.Hash
 | 
				
			||||||
	amount  uint64
 | 
						amount  uint64
 | 
				
			||||||
	peer    *peer
 | 
						peer    *serverPeer
 | 
				
			||||||
	sent    mclock.AbsTime
 | 
						sent    mclock.AbsTime
 | 
				
			||||||
	timeout bool
 | 
						timeout bool
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
@@ -105,7 +105,7 @@ type fetchRequest struct {
 | 
				
			|||||||
type fetchResponse struct {
 | 
					type fetchResponse struct {
 | 
				
			||||||
	reqID   uint64
 | 
						reqID   uint64
 | 
				
			||||||
	headers []*types.Header
 | 
						headers []*types.Header
 | 
				
			||||||
	peer    *peer
 | 
						peer    *serverPeer
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// newLightFetcher creates a new light fetcher
 | 
					// newLightFetcher creates a new light fetcher
 | 
				
			||||||
@@ -113,16 +113,16 @@ func newLightFetcher(h *clientHandler) *lightFetcher {
 | 
				
			|||||||
	f := &lightFetcher{
 | 
						f := &lightFetcher{
 | 
				
			||||||
		handler:        h,
 | 
							handler:        h,
 | 
				
			||||||
		chain:          h.backend.blockchain,
 | 
							chain:          h.backend.blockchain,
 | 
				
			||||||
		peers:          make(map[*peer]*fetcherPeerInfo),
 | 
							peers:          make(map[*serverPeer]*fetcherPeerInfo),
 | 
				
			||||||
		deliverChn:     make(chan fetchResponse, 100),
 | 
							deliverChn:     make(chan fetchResponse, 100),
 | 
				
			||||||
		requested:      make(map[uint64]fetchRequest),
 | 
							requested:      make(map[uint64]fetchRequest),
 | 
				
			||||||
		timeoutChn:     make(chan uint64),
 | 
							timeoutChn:     make(chan uint64),
 | 
				
			||||||
		requestTrigger: make(chan struct{}, 1),
 | 
							requestTrigger: make(chan struct{}, 1),
 | 
				
			||||||
		syncDone:       make(chan *peer),
 | 
							syncDone:       make(chan *serverPeer),
 | 
				
			||||||
		closeCh:        make(chan struct{}),
 | 
							closeCh:        make(chan struct{}),
 | 
				
			||||||
		maxConfirmedTd: big.NewInt(0),
 | 
							maxConfirmedTd: big.NewInt(0),
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	h.backend.peers.notify(f)
 | 
						h.backend.peers.subscribe(f)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	f.wg.Add(1)
 | 
						f.wg.Add(1)
 | 
				
			||||||
	go f.syncLoop()
 | 
						go f.syncLoop()
 | 
				
			||||||
@@ -222,7 +222,7 @@ func (f *lightFetcher) syncLoop() {
 | 
				
			|||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// registerPeer adds a new peer to the fetcher's peer set
 | 
					// registerPeer adds a new peer to the fetcher's peer set
 | 
				
			||||||
func (f *lightFetcher) registerPeer(p *peer) {
 | 
					func (f *lightFetcher) registerPeer(p *serverPeer) {
 | 
				
			||||||
	p.lock.Lock()
 | 
						p.lock.Lock()
 | 
				
			||||||
	p.hasBlock = func(hash common.Hash, number uint64, hasState bool) bool {
 | 
						p.hasBlock = func(hash common.Hash, number uint64, hasState bool) bool {
 | 
				
			||||||
		return f.peerHasBlock(p, hash, number, hasState)
 | 
							return f.peerHasBlock(p, hash, number, hasState)
 | 
				
			||||||
@@ -235,7 +235,7 @@ func (f *lightFetcher) registerPeer(p *peer) {
 | 
				
			|||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// unregisterPeer removes a new peer from the fetcher's peer set
 | 
					// unregisterPeer removes a new peer from the fetcher's peer set
 | 
				
			||||||
func (f *lightFetcher) unregisterPeer(p *peer) {
 | 
					func (f *lightFetcher) unregisterPeer(p *serverPeer) {
 | 
				
			||||||
	p.lock.Lock()
 | 
						p.lock.Lock()
 | 
				
			||||||
	p.hasBlock = nil
 | 
						p.hasBlock = nil
 | 
				
			||||||
	p.lock.Unlock()
 | 
						p.lock.Unlock()
 | 
				
			||||||
@@ -250,7 +250,7 @@ func (f *lightFetcher) unregisterPeer(p *peer) {
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
// announce processes a new announcement message received from a peer, adding new
 | 
					// announce processes a new announcement message received from a peer, adding new
 | 
				
			||||||
// nodes to the peer's block tree and removing old nodes if necessary
 | 
					// nodes to the peer's block tree and removing old nodes if necessary
 | 
				
			||||||
func (f *lightFetcher) announce(p *peer, head *announceData) {
 | 
					func (f *lightFetcher) announce(p *serverPeer, head *announceData) {
 | 
				
			||||||
	f.lock.Lock()
 | 
						f.lock.Lock()
 | 
				
			||||||
	defer f.lock.Unlock()
 | 
						defer f.lock.Unlock()
 | 
				
			||||||
	p.Log().Debug("Received new announcement", "number", head.Number, "hash", head.Hash, "reorg", head.ReorgDepth)
 | 
						p.Log().Debug("Received new announcement", "number", head.Number, "hash", head.Hash, "reorg", head.ReorgDepth)
 | 
				
			||||||
@@ -346,7 +346,7 @@ func (f *lightFetcher) announce(p *peer, head *announceData) {
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
	f.checkKnownNode(p, n)
 | 
						f.checkKnownNode(p, n)
 | 
				
			||||||
	p.lock.Lock()
 | 
						p.lock.Lock()
 | 
				
			||||||
	p.headInfo = head
 | 
						p.headInfo = blockInfo{Number: head.Number, Hash: head.Hash, Td: head.Td}
 | 
				
			||||||
	fp.lastAnnounced = n
 | 
						fp.lastAnnounced = n
 | 
				
			||||||
	p.lock.Unlock()
 | 
						p.lock.Unlock()
 | 
				
			||||||
	f.checkUpdateStats(p, nil)
 | 
						f.checkUpdateStats(p, nil)
 | 
				
			||||||
@@ -358,7 +358,7 @@ func (f *lightFetcher) announce(p *peer, head *announceData) {
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
// peerHasBlock returns true if we can assume the peer knows the given block
 | 
					// peerHasBlock returns true if we can assume the peer knows the given block
 | 
				
			||||||
// based on its announcements
 | 
					// based on its announcements
 | 
				
			||||||
func (f *lightFetcher) peerHasBlock(p *peer, hash common.Hash, number uint64, hasState bool) bool {
 | 
					func (f *lightFetcher) peerHasBlock(p *serverPeer, hash common.Hash, number uint64, hasState bool) bool {
 | 
				
			||||||
	f.lock.Lock()
 | 
						f.lock.Lock()
 | 
				
			||||||
	defer f.lock.Unlock()
 | 
						defer f.lock.Unlock()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -395,7 +395,7 @@ func (f *lightFetcher) peerHasBlock(p *peer, hash common.Hash, number uint64, ha
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
// requestAmount calculates the amount of headers to be downloaded starting
 | 
					// requestAmount calculates the amount of headers to be downloaded starting
 | 
				
			||||||
// from a certain head backwards
 | 
					// from a certain head backwards
 | 
				
			||||||
func (f *lightFetcher) requestAmount(p *peer, n *fetcherTreeNode) uint64 {
 | 
					func (f *lightFetcher) requestAmount(p *serverPeer, n *fetcherTreeNode) uint64 {
 | 
				
			||||||
	amount := uint64(0)
 | 
						amount := uint64(0)
 | 
				
			||||||
	nn := n
 | 
						nn := n
 | 
				
			||||||
	for nn != nil && !f.checkKnownNode(p, nn) {
 | 
						for nn != nil && !f.checkKnownNode(p, nn) {
 | 
				
			||||||
@@ -488,7 +488,7 @@ func (f *lightFetcher) newFetcherDistReqForSync(bestHash common.Hash) *distReq {
 | 
				
			|||||||
			return 0
 | 
								return 0
 | 
				
			||||||
		},
 | 
							},
 | 
				
			||||||
		canSend: func(dp distPeer) bool {
 | 
							canSend: func(dp distPeer) bool {
 | 
				
			||||||
			p := dp.(*peer)
 | 
								p := dp.(*serverPeer)
 | 
				
			||||||
			f.lock.Lock()
 | 
								f.lock.Lock()
 | 
				
			||||||
			defer f.lock.Unlock()
 | 
								defer f.lock.Unlock()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -504,7 +504,7 @@ func (f *lightFetcher) newFetcherDistReqForSync(bestHash common.Hash) *distReq {
 | 
				
			|||||||
				f.setLastTrustedHeader(f.chain.CurrentHeader())
 | 
									f.setLastTrustedHeader(f.chain.CurrentHeader())
 | 
				
			||||||
			}
 | 
								}
 | 
				
			||||||
			go func() {
 | 
								go func() {
 | 
				
			||||||
				p := dp.(*peer)
 | 
									p := dp.(*serverPeer)
 | 
				
			||||||
				p.Log().Debug("Synchronisation started")
 | 
									p.Log().Debug("Synchronisation started")
 | 
				
			||||||
				f.handler.synchronise(p)
 | 
									f.handler.synchronise(p)
 | 
				
			||||||
				f.syncDone <- p
 | 
									f.syncDone <- p
 | 
				
			||||||
@@ -518,11 +518,11 @@ func (f *lightFetcher) newFetcherDistReqForSync(bestHash common.Hash) *distReq {
 | 
				
			|||||||
func (f *lightFetcher) newFetcherDistReq(bestHash common.Hash, reqID uint64, bestAmount uint64) *distReq {
 | 
					func (f *lightFetcher) newFetcherDistReq(bestHash common.Hash, reqID uint64, bestAmount uint64) *distReq {
 | 
				
			||||||
	return &distReq{
 | 
						return &distReq{
 | 
				
			||||||
		getCost: func(dp distPeer) uint64 {
 | 
							getCost: func(dp distPeer) uint64 {
 | 
				
			||||||
			p := dp.(*peer)
 | 
								p := dp.(*serverPeer)
 | 
				
			||||||
			return p.GetRequestCost(GetBlockHeadersMsg, int(bestAmount))
 | 
								return p.getRequestCost(GetBlockHeadersMsg, int(bestAmount))
 | 
				
			||||||
		},
 | 
							},
 | 
				
			||||||
		canSend: func(dp distPeer) bool {
 | 
							canSend: func(dp distPeer) bool {
 | 
				
			||||||
			p := dp.(*peer)
 | 
								p := dp.(*serverPeer)
 | 
				
			||||||
			f.lock.Lock()
 | 
								f.lock.Lock()
 | 
				
			||||||
			defer f.lock.Unlock()
 | 
								defer f.lock.Unlock()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -537,7 +537,7 @@ func (f *lightFetcher) newFetcherDistReq(bestHash common.Hash, reqID uint64, bes
 | 
				
			|||||||
			return n != nil && !n.requested
 | 
								return n != nil && !n.requested
 | 
				
			||||||
		},
 | 
							},
 | 
				
			||||||
		request: func(dp distPeer) func() {
 | 
							request: func(dp distPeer) func() {
 | 
				
			||||||
			p := dp.(*peer)
 | 
								p := dp.(*serverPeer)
 | 
				
			||||||
			f.lock.Lock()
 | 
								f.lock.Lock()
 | 
				
			||||||
			fp := f.peers[p]
 | 
								fp := f.peers[p]
 | 
				
			||||||
			if fp != nil {
 | 
								if fp != nil {
 | 
				
			||||||
@@ -548,7 +548,7 @@ func (f *lightFetcher) newFetcherDistReq(bestHash common.Hash, reqID uint64, bes
 | 
				
			|||||||
			}
 | 
								}
 | 
				
			||||||
			f.lock.Unlock()
 | 
								f.lock.Unlock()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
			cost := p.GetRequestCost(GetBlockHeadersMsg, int(bestAmount))
 | 
								cost := p.getRequestCost(GetBlockHeadersMsg, int(bestAmount))
 | 
				
			||||||
			p.fcServer.QueuedRequest(reqID, cost)
 | 
								p.fcServer.QueuedRequest(reqID, cost)
 | 
				
			||||||
			f.reqMu.Lock()
 | 
								f.reqMu.Lock()
 | 
				
			||||||
			f.requested[reqID] = fetchRequest{hash: bestHash, amount: bestAmount, peer: p, sent: mclock.Now()}
 | 
								f.requested[reqID] = fetchRequest{hash: bestHash, amount: bestAmount, peer: p, sent: mclock.Now()}
 | 
				
			||||||
@@ -557,13 +557,13 @@ func (f *lightFetcher) newFetcherDistReq(bestHash common.Hash, reqID uint64, bes
 | 
				
			|||||||
				time.Sleep(hardRequestTimeout)
 | 
									time.Sleep(hardRequestTimeout)
 | 
				
			||||||
				f.timeoutChn <- reqID
 | 
									f.timeoutChn <- reqID
 | 
				
			||||||
			}()
 | 
								}()
 | 
				
			||||||
			return func() { p.RequestHeadersByHash(reqID, cost, bestHash, int(bestAmount), 0, true) }
 | 
								return func() { p.requestHeadersByHash(reqID, bestHash, int(bestAmount), 0, true) }
 | 
				
			||||||
		},
 | 
							},
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// deliverHeaders delivers header download request responses for processing
 | 
					// deliverHeaders delivers header download request responses for processing
 | 
				
			||||||
func (f *lightFetcher) deliverHeaders(peer *peer, reqID uint64, headers []*types.Header) {
 | 
					func (f *lightFetcher) deliverHeaders(peer *serverPeer, reqID uint64, headers []*types.Header) {
 | 
				
			||||||
	f.deliverChn <- fetchResponse{reqID: reqID, headers: headers, peer: peer}
 | 
						f.deliverChn <- fetchResponse{reqID: reqID, headers: headers, peer: peer}
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -694,7 +694,7 @@ func (f *lightFetcher) checkAnnouncedHeaders(fp *fetcherPeerInfo, headers []*typ
 | 
				
			|||||||
// checkSyncedHeaders updates peer's block tree after synchronisation by marking
 | 
					// checkSyncedHeaders updates peer's block tree after synchronisation by marking
 | 
				
			||||||
// downloaded headers as known. If none of the announced headers are found after
 | 
					// downloaded headers as known. If none of the announced headers are found after
 | 
				
			||||||
// syncing, the peer is dropped.
 | 
					// syncing, the peer is dropped.
 | 
				
			||||||
func (f *lightFetcher) checkSyncedHeaders(p *peer) {
 | 
					func (f *lightFetcher) checkSyncedHeaders(p *serverPeer) {
 | 
				
			||||||
	fp := f.peers[p]
 | 
						fp := f.peers[p]
 | 
				
			||||||
	if fp == nil {
 | 
						if fp == nil {
 | 
				
			||||||
		p.Log().Debug("Unknown peer to check sync headers")
 | 
							p.Log().Debug("Unknown peer to check sync headers")
 | 
				
			||||||
@@ -728,7 +728,7 @@ func (f *lightFetcher) checkSyncedHeaders(p *peer) {
 | 
				
			|||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// lastTrustedTreeNode return last approved treeNode and a list of unapproved hashes
 | 
					// lastTrustedTreeNode return last approved treeNode and a list of unapproved hashes
 | 
				
			||||||
func (f *lightFetcher) lastTrustedTreeNode(p *peer) (*types.Header, []common.Hash) {
 | 
					func (f *lightFetcher) lastTrustedTreeNode(p *serverPeer) (*types.Header, []common.Hash) {
 | 
				
			||||||
	unapprovedHashes := make([]common.Hash, 0)
 | 
						unapprovedHashes := make([]common.Hash, 0)
 | 
				
			||||||
	current := f.chain.CurrentHeader()
 | 
						current := f.chain.CurrentHeader()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -764,7 +764,7 @@ func (f *lightFetcher) setLastTrustedHeader(h *types.Header) {
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
// checkKnownNode checks if a block tree node is known (downloaded and validated)
 | 
					// checkKnownNode checks if a block tree node is known (downloaded and validated)
 | 
				
			||||||
// If it was not known previously but found in the database, sets its known flag
 | 
					// If it was not known previously but found in the database, sets its known flag
 | 
				
			||||||
func (f *lightFetcher) checkKnownNode(p *peer, n *fetcherTreeNode) bool {
 | 
					func (f *lightFetcher) checkKnownNode(p *serverPeer, n *fetcherTreeNode) bool {
 | 
				
			||||||
	if n.known {
 | 
						if n.known {
 | 
				
			||||||
		return true
 | 
							return true
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
@@ -867,7 +867,7 @@ func (f *lightFetcher) updateMaxConfirmedTd(td *big.Int) {
 | 
				
			|||||||
// If a new entry has been added to the global tail, it is passed as a parameter here even though this function
 | 
					// If a new entry has been added to the global tail, it is passed as a parameter here even though this function
 | 
				
			||||||
// assumes that it has already been added, so that if the peer's list is empty (all heads confirmed, head is nil),
 | 
					// assumes that it has already been added, so that if the peer's list is empty (all heads confirmed, head is nil),
 | 
				
			||||||
// it can set the new head to newEntry.
 | 
					// it can set the new head to newEntry.
 | 
				
			||||||
func (f *lightFetcher) checkUpdateStats(p *peer, newEntry *updateStatsEntry) {
 | 
					func (f *lightFetcher) checkUpdateStats(p *serverPeer, newEntry *updateStatsEntry) {
 | 
				
			||||||
	now := mclock.Now()
 | 
						now := mclock.Now()
 | 
				
			||||||
	fp := f.peers[p]
 | 
						fp := f.peers[p]
 | 
				
			||||||
	if fp == nil {
 | 
						if fp == nil {
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -168,8 +168,7 @@ func testGetBlockHeaders(t *testing.T, protocol int) {
 | 
				
			|||||||
		// Send the hash request and verify the response
 | 
							// Send the hash request and verify the response
 | 
				
			||||||
		reqID++
 | 
							reqID++
 | 
				
			||||||
 | 
					
 | 
				
			||||||
		cost := server.peer.peer.GetRequestCost(GetBlockHeadersMsg, int(tt.query.Amount))
 | 
							sendRequest(server.peer.app, GetBlockHeadersMsg, reqID, tt.query)
 | 
				
			||||||
		sendRequest(server.peer.app, GetBlockHeadersMsg, reqID, cost, tt.query)
 | 
					 | 
				
			||||||
		if err := expectResponse(server.peer.app, BlockHeadersMsg, reqID, testBufLimit, headers); err != nil {
 | 
							if err := expectResponse(server.peer.app, BlockHeadersMsg, reqID, testBufLimit, headers); err != nil {
 | 
				
			||||||
			t.Errorf("test %d: headers mismatch: %v", i, err)
 | 
								t.Errorf("test %d: headers mismatch: %v", i, err)
 | 
				
			||||||
		}
 | 
							}
 | 
				
			||||||
@@ -246,8 +245,7 @@ func testGetBlockBodies(t *testing.T, protocol int) {
 | 
				
			|||||||
		reqID++
 | 
							reqID++
 | 
				
			||||||
 | 
					
 | 
				
			||||||
		// Send the hash request and verify the response
 | 
							// Send the hash request and verify the response
 | 
				
			||||||
		cost := server.peer.peer.GetRequestCost(GetBlockBodiesMsg, len(hashes))
 | 
							sendRequest(server.peer.app, GetBlockBodiesMsg, reqID, hashes)
 | 
				
			||||||
		sendRequest(server.peer.app, GetBlockBodiesMsg, reqID, cost, hashes)
 | 
					 | 
				
			||||||
		if err := expectResponse(server.peer.app, BlockBodiesMsg, reqID, testBufLimit, bodies); err != nil {
 | 
							if err := expectResponse(server.peer.app, BlockBodiesMsg, reqID, testBufLimit, bodies); err != nil {
 | 
				
			||||||
			t.Errorf("test %d: bodies mismatch: %v", i, err)
 | 
								t.Errorf("test %d: bodies mismatch: %v", i, err)
 | 
				
			||||||
		}
 | 
							}
 | 
				
			||||||
@@ -278,8 +276,7 @@ func testGetCode(t *testing.T, protocol int) {
 | 
				
			|||||||
		}
 | 
							}
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	cost := server.peer.peer.GetRequestCost(GetCodeMsg, len(codereqs))
 | 
						sendRequest(server.peer.app, GetCodeMsg, 42, codereqs)
 | 
				
			||||||
	sendRequest(server.peer.app, GetCodeMsg, 42, cost, codereqs)
 | 
					 | 
				
			||||||
	if err := expectResponse(server.peer.app, CodeMsg, 42, testBufLimit, codes); err != nil {
 | 
						if err := expectResponse(server.peer.app, CodeMsg, 42, testBufLimit, codes); err != nil {
 | 
				
			||||||
		t.Errorf("codes mismatch: %v", err)
 | 
							t.Errorf("codes mismatch: %v", err)
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
@@ -299,8 +296,7 @@ func testGetStaleCode(t *testing.T, protocol int) {
 | 
				
			|||||||
			BHash:  bc.GetHeaderByNumber(number).Hash(),
 | 
								BHash:  bc.GetHeaderByNumber(number).Hash(),
 | 
				
			||||||
			AccKey: crypto.Keccak256(testContractAddr[:]),
 | 
								AccKey: crypto.Keccak256(testContractAddr[:]),
 | 
				
			||||||
		}
 | 
							}
 | 
				
			||||||
		cost := server.peer.peer.GetRequestCost(GetCodeMsg, 1)
 | 
							sendRequest(server.peer.app, GetCodeMsg, 42, []*CodeReq{req})
 | 
				
			||||||
		sendRequest(server.peer.app, GetCodeMsg, 42, cost, []*CodeReq{req})
 | 
					 | 
				
			||||||
		if err := expectResponse(server.peer.app, CodeMsg, 42, testBufLimit, expected); err != nil {
 | 
							if err := expectResponse(server.peer.app, CodeMsg, 42, testBufLimit, expected); err != nil {
 | 
				
			||||||
			t.Errorf("codes mismatch: %v", err)
 | 
								t.Errorf("codes mismatch: %v", err)
 | 
				
			||||||
		}
 | 
							}
 | 
				
			||||||
@@ -331,8 +327,7 @@ func testGetReceipt(t *testing.T, protocol int) {
 | 
				
			|||||||
		receipts = append(receipts, rawdb.ReadRawReceipts(server.db, block.Hash(), block.NumberU64()))
 | 
							receipts = append(receipts, rawdb.ReadRawReceipts(server.db, block.Hash(), block.NumberU64()))
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	// Send the hash request and verify the response
 | 
						// Send the hash request and verify the response
 | 
				
			||||||
	cost := server.peer.peer.GetRequestCost(GetReceiptsMsg, len(hashes))
 | 
						sendRequest(server.peer.app, GetReceiptsMsg, 42, hashes)
 | 
				
			||||||
	sendRequest(server.peer.app, GetReceiptsMsg, 42, cost, hashes)
 | 
					 | 
				
			||||||
	if err := expectResponse(server.peer.app, ReceiptsMsg, 42, testBufLimit, receipts); err != nil {
 | 
						if err := expectResponse(server.peer.app, ReceiptsMsg, 42, testBufLimit, receipts); err != nil {
 | 
				
			||||||
		t.Errorf("receipts mismatch: %v", err)
 | 
							t.Errorf("receipts mismatch: %v", err)
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
@@ -367,8 +362,7 @@ func testGetProofs(t *testing.T, protocol int) {
 | 
				
			|||||||
		}
 | 
							}
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	// Send the proof request and verify the response
 | 
						// Send the proof request and verify the response
 | 
				
			||||||
	cost := server.peer.peer.GetRequestCost(GetProofsV2Msg, len(proofreqs))
 | 
						sendRequest(server.peer.app, GetProofsV2Msg, 42, proofreqs)
 | 
				
			||||||
	sendRequest(server.peer.app, GetProofsV2Msg, 42, cost, proofreqs)
 | 
					 | 
				
			||||||
	if err := expectResponse(server.peer.app, ProofsV2Msg, 42, testBufLimit, proofsV2.NodeList()); err != nil {
 | 
						if err := expectResponse(server.peer.app, ProofsV2Msg, 42, testBufLimit, proofsV2.NodeList()); err != nil {
 | 
				
			||||||
		t.Errorf("proofs mismatch: %v", err)
 | 
							t.Errorf("proofs mismatch: %v", err)
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
@@ -392,8 +386,7 @@ func testGetStaleProof(t *testing.T, protocol int) {
 | 
				
			|||||||
			BHash: header.Hash(),
 | 
								BHash: header.Hash(),
 | 
				
			||||||
			Key:   account,
 | 
								Key:   account,
 | 
				
			||||||
		}
 | 
							}
 | 
				
			||||||
		cost := server.peer.peer.GetRequestCost(GetProofsV2Msg, 1)
 | 
							sendRequest(server.peer.app, GetProofsV2Msg, 42, []*ProofReq{req})
 | 
				
			||||||
		sendRequest(server.peer.app, GetProofsV2Msg, 42, cost, []*ProofReq{req})
 | 
					 | 
				
			||||||
 | 
					
 | 
				
			||||||
		var expected []rlp.RawValue
 | 
							var expected []rlp.RawValue
 | 
				
			||||||
		if wantOK {
 | 
							if wantOK {
 | 
				
			||||||
@@ -453,8 +446,7 @@ func testGetCHTProofs(t *testing.T, protocol int) {
 | 
				
			|||||||
		AuxReq:  auxHeader,
 | 
							AuxReq:  auxHeader,
 | 
				
			||||||
	}}
 | 
						}}
 | 
				
			||||||
	// Send the proof request and verify the response
 | 
						// Send the proof request and verify the response
 | 
				
			||||||
	cost := server.peer.peer.GetRequestCost(GetHelperTrieProofsMsg, len(requestsV2))
 | 
						sendRequest(server.peer.app, GetHelperTrieProofsMsg, 42, requestsV2)
 | 
				
			||||||
	sendRequest(server.peer.app, GetHelperTrieProofsMsg, 42, cost, requestsV2)
 | 
					 | 
				
			||||||
	if err := expectResponse(server.peer.app, HelperTrieProofsMsg, 42, testBufLimit, proofsV2); err != nil {
 | 
						if err := expectResponse(server.peer.app, HelperTrieProofsMsg, 42, testBufLimit, proofsV2); err != nil {
 | 
				
			||||||
		t.Errorf("proofs mismatch: %v", err)
 | 
							t.Errorf("proofs mismatch: %v", err)
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
@@ -502,8 +494,7 @@ func testGetBloombitsProofs(t *testing.T, protocol int) {
 | 
				
			|||||||
		trie.Prove(key, 0, &proofs.Proofs)
 | 
							trie.Prove(key, 0, &proofs.Proofs)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
		// Send the proof request and verify the response
 | 
							// Send the proof request and verify the response
 | 
				
			||||||
		cost := server.peer.peer.GetRequestCost(GetHelperTrieProofsMsg, len(requests))
 | 
							sendRequest(server.peer.app, GetHelperTrieProofsMsg, 42, requests)
 | 
				
			||||||
		sendRequest(server.peer.app, GetHelperTrieProofsMsg, 42, cost, requests)
 | 
					 | 
				
			||||||
		if err := expectResponse(server.peer.app, HelperTrieProofsMsg, 42, testBufLimit, proofs); err != nil {
 | 
							if err := expectResponse(server.peer.app, HelperTrieProofsMsg, 42, testBufLimit, proofs); err != nil {
 | 
				
			||||||
			t.Errorf("bit %d: proofs mismatch: %v", bit, err)
 | 
								t.Errorf("bit %d: proofs mismatch: %v", bit, err)
 | 
				
			||||||
		}
 | 
							}
 | 
				
			||||||
@@ -525,11 +516,9 @@ func testTransactionStatus(t *testing.T, protocol int) {
 | 
				
			|||||||
	test := func(tx *types.Transaction, send bool, expStatus light.TxStatus) {
 | 
						test := func(tx *types.Transaction, send bool, expStatus light.TxStatus) {
 | 
				
			||||||
		reqID++
 | 
							reqID++
 | 
				
			||||||
		if send {
 | 
							if send {
 | 
				
			||||||
			cost := server.peer.peer.GetRequestCost(SendTxV2Msg, 1)
 | 
								sendRequest(server.peer.app, SendTxV2Msg, reqID, types.Transactions{tx})
 | 
				
			||||||
			sendRequest(server.peer.app, SendTxV2Msg, reqID, cost, types.Transactions{tx})
 | 
					 | 
				
			||||||
		} else {
 | 
							} else {
 | 
				
			||||||
			cost := server.peer.peer.GetRequestCost(GetTxStatusMsg, 1)
 | 
								sendRequest(server.peer.app, GetTxStatusMsg, reqID, []common.Hash{tx.Hash()})
 | 
				
			||||||
			sendRequest(server.peer.app, GetTxStatusMsg, reqID, cost, []common.Hash{tx.Hash()})
 | 
					 | 
				
			||||||
		}
 | 
							}
 | 
				
			||||||
		if err := expectResponse(server.peer.app, TxStatusMsg, reqID, testBufLimit, []light.TxStatus{expStatus}); err != nil {
 | 
							if err := expectResponse(server.peer.app, TxStatusMsg, reqID, testBufLimit, []light.TxStatus{expStatus}); err != nil {
 | 
				
			||||||
			t.Errorf("transaction status mismatch")
 | 
								t.Errorf("transaction status mismatch")
 | 
				
			||||||
@@ -620,7 +609,7 @@ func TestStopResumeLes3(t *testing.T) {
 | 
				
			|||||||
	header := server.handler.blockchain.CurrentHeader()
 | 
						header := server.handler.blockchain.CurrentHeader()
 | 
				
			||||||
	req := func() {
 | 
						req := func() {
 | 
				
			||||||
		reqID++
 | 
							reqID++
 | 
				
			||||||
		sendRequest(server.peer.app, GetBlockHeadersMsg, reqID, testCost, &getBlockHeadersData{Origin: hashOrNumber{Hash: header.Hash()}, Amount: 1})
 | 
							sendRequest(server.peer.app, GetBlockHeadersMsg, reqID, &getBlockHeadersData{Origin: hashOrNumber{Hash: header.Hash()}, Amount: 1})
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	for i := 1; i <= 5; i++ {
 | 
						for i := 1; i <= 5; i++ {
 | 
				
			||||||
		// send requests while we still have enough buffer and expect a response
 | 
							// send requests while we still have enough buffer and expect a response
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -106,17 +106,17 @@ func (odr *LesOdr) Retrieve(ctx context.Context, req light.OdrRequest) (err erro
 | 
				
			|||||||
	reqID := genReqID()
 | 
						reqID := genReqID()
 | 
				
			||||||
	rq := &distReq{
 | 
						rq := &distReq{
 | 
				
			||||||
		getCost: func(dp distPeer) uint64 {
 | 
							getCost: func(dp distPeer) uint64 {
 | 
				
			||||||
			return lreq.GetCost(dp.(*peer))
 | 
								return lreq.GetCost(dp.(*serverPeer))
 | 
				
			||||||
		},
 | 
							},
 | 
				
			||||||
		canSend: func(dp distPeer) bool {
 | 
							canSend: func(dp distPeer) bool {
 | 
				
			||||||
			p := dp.(*peer)
 | 
								p := dp.(*serverPeer)
 | 
				
			||||||
			if !p.onlyAnnounce {
 | 
								if !p.onlyAnnounce {
 | 
				
			||||||
				return lreq.CanSend(p)
 | 
									return lreq.CanSend(p)
 | 
				
			||||||
			}
 | 
								}
 | 
				
			||||||
			return false
 | 
								return false
 | 
				
			||||||
		},
 | 
							},
 | 
				
			||||||
		request: func(dp distPeer) func() {
 | 
							request: func(dp distPeer) func() {
 | 
				
			||||||
			p := dp.(*peer)
 | 
								p := dp.(*serverPeer)
 | 
				
			||||||
			cost := lreq.GetCost(p)
 | 
								cost := lreq.GetCost(p)
 | 
				
			||||||
			p.fcServer.QueuedRequest(reqID, cost)
 | 
								p.fcServer.QueuedRequest(reqID, cost)
 | 
				
			||||||
			return func() { lreq.Request(reqID, p) }
 | 
								return func() { lreq.Request(reqID, p) }
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -46,9 +46,9 @@ var (
 | 
				
			|||||||
)
 | 
					)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
type LesOdrRequest interface {
 | 
					type LesOdrRequest interface {
 | 
				
			||||||
	GetCost(*peer) uint64
 | 
						GetCost(*serverPeer) uint64
 | 
				
			||||||
	CanSend(*peer) bool
 | 
						CanSend(*serverPeer) bool
 | 
				
			||||||
	Request(uint64, *peer) error
 | 
						Request(uint64, *serverPeer) error
 | 
				
			||||||
	Validate(ethdb.Database, *Msg) error
 | 
						Validate(ethdb.Database, *Msg) error
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -78,19 +78,19 @@ type BlockRequest light.BlockRequest
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
// GetCost returns the cost of the given ODR request according to the serving
 | 
					// GetCost returns the cost of the given ODR request according to the serving
 | 
				
			||||||
// peer's cost table (implementation of LesOdrRequest)
 | 
					// peer's cost table (implementation of LesOdrRequest)
 | 
				
			||||||
func (r *BlockRequest) GetCost(peer *peer) uint64 {
 | 
					func (r *BlockRequest) GetCost(peer *serverPeer) uint64 {
 | 
				
			||||||
	return peer.GetRequestCost(GetBlockBodiesMsg, 1)
 | 
						return peer.getRequestCost(GetBlockBodiesMsg, 1)
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// CanSend tells if a certain peer is suitable for serving the given request
 | 
					// CanSend tells if a certain peer is suitable for serving the given request
 | 
				
			||||||
func (r *BlockRequest) CanSend(peer *peer) bool {
 | 
					func (r *BlockRequest) CanSend(peer *serverPeer) bool {
 | 
				
			||||||
	return peer.HasBlock(r.Hash, r.Number, false)
 | 
						return peer.HasBlock(r.Hash, r.Number, false)
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// Request sends an ODR request to the LES network (implementation of LesOdrRequest)
 | 
					// Request sends an ODR request to the LES network (implementation of LesOdrRequest)
 | 
				
			||||||
func (r *BlockRequest) Request(reqID uint64, peer *peer) error {
 | 
					func (r *BlockRequest) Request(reqID uint64, peer *serverPeer) error {
 | 
				
			||||||
	peer.Log().Debug("Requesting block body", "hash", r.Hash)
 | 
						peer.Log().Debug("Requesting block body", "hash", r.Hash)
 | 
				
			||||||
	return peer.RequestBodies(reqID, r.GetCost(peer), []common.Hash{r.Hash})
 | 
						return peer.requestBodies(reqID, []common.Hash{r.Hash})
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// Valid processes an ODR request reply message from the LES network
 | 
					// Valid processes an ODR request reply message from the LES network
 | 
				
			||||||
@@ -134,19 +134,19 @@ type ReceiptsRequest light.ReceiptsRequest
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
// GetCost returns the cost of the given ODR request according to the serving
 | 
					// GetCost returns the cost of the given ODR request according to the serving
 | 
				
			||||||
// peer's cost table (implementation of LesOdrRequest)
 | 
					// peer's cost table (implementation of LesOdrRequest)
 | 
				
			||||||
func (r *ReceiptsRequest) GetCost(peer *peer) uint64 {
 | 
					func (r *ReceiptsRequest) GetCost(peer *serverPeer) uint64 {
 | 
				
			||||||
	return peer.GetRequestCost(GetReceiptsMsg, 1)
 | 
						return peer.getRequestCost(GetReceiptsMsg, 1)
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// CanSend tells if a certain peer is suitable for serving the given request
 | 
					// CanSend tells if a certain peer is suitable for serving the given request
 | 
				
			||||||
func (r *ReceiptsRequest) CanSend(peer *peer) bool {
 | 
					func (r *ReceiptsRequest) CanSend(peer *serverPeer) bool {
 | 
				
			||||||
	return peer.HasBlock(r.Hash, r.Number, false)
 | 
						return peer.HasBlock(r.Hash, r.Number, false)
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// Request sends an ODR request to the LES network (implementation of LesOdrRequest)
 | 
					// Request sends an ODR request to the LES network (implementation of LesOdrRequest)
 | 
				
			||||||
func (r *ReceiptsRequest) Request(reqID uint64, peer *peer) error {
 | 
					func (r *ReceiptsRequest) Request(reqID uint64, peer *serverPeer) error {
 | 
				
			||||||
	peer.Log().Debug("Requesting block receipts", "hash", r.Hash)
 | 
						peer.Log().Debug("Requesting block receipts", "hash", r.Hash)
 | 
				
			||||||
	return peer.RequestReceipts(reqID, r.GetCost(peer), []common.Hash{r.Hash})
 | 
						return peer.requestReceipts(reqID, []common.Hash{r.Hash})
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// Valid processes an ODR request reply message from the LES network
 | 
					// Valid processes an ODR request reply message from the LES network
 | 
				
			||||||
@@ -191,24 +191,24 @@ type TrieRequest light.TrieRequest
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
// GetCost returns the cost of the given ODR request according to the serving
 | 
					// GetCost returns the cost of the given ODR request according to the serving
 | 
				
			||||||
// peer's cost table (implementation of LesOdrRequest)
 | 
					// peer's cost table (implementation of LesOdrRequest)
 | 
				
			||||||
func (r *TrieRequest) GetCost(peer *peer) uint64 {
 | 
					func (r *TrieRequest) GetCost(peer *serverPeer) uint64 {
 | 
				
			||||||
	return peer.GetRequestCost(GetProofsV2Msg, 1)
 | 
						return peer.getRequestCost(GetProofsV2Msg, 1)
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// CanSend tells if a certain peer is suitable for serving the given request
 | 
					// CanSend tells if a certain peer is suitable for serving the given request
 | 
				
			||||||
func (r *TrieRequest) CanSend(peer *peer) bool {
 | 
					func (r *TrieRequest) CanSend(peer *serverPeer) bool {
 | 
				
			||||||
	return peer.HasBlock(r.Id.BlockHash, r.Id.BlockNumber, true)
 | 
						return peer.HasBlock(r.Id.BlockHash, r.Id.BlockNumber, true)
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// Request sends an ODR request to the LES network (implementation of LesOdrRequest)
 | 
					// Request sends an ODR request to the LES network (implementation of LesOdrRequest)
 | 
				
			||||||
func (r *TrieRequest) Request(reqID uint64, peer *peer) error {
 | 
					func (r *TrieRequest) Request(reqID uint64, peer *serverPeer) error {
 | 
				
			||||||
	peer.Log().Debug("Requesting trie proof", "root", r.Id.Root, "key", r.Key)
 | 
						peer.Log().Debug("Requesting trie proof", "root", r.Id.Root, "key", r.Key)
 | 
				
			||||||
	req := ProofReq{
 | 
						req := ProofReq{
 | 
				
			||||||
		BHash:  r.Id.BlockHash,
 | 
							BHash:  r.Id.BlockHash,
 | 
				
			||||||
		AccKey: r.Id.AccKey,
 | 
							AccKey: r.Id.AccKey,
 | 
				
			||||||
		Key:    r.Key,
 | 
							Key:    r.Key,
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	return peer.RequestProofs(reqID, r.GetCost(peer), []ProofReq{req})
 | 
						return peer.requestProofs(reqID, []ProofReq{req})
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// Valid processes an ODR request reply message from the LES network
 | 
					// Valid processes an ODR request reply message from the LES network
 | 
				
			||||||
@@ -245,23 +245,23 @@ type CodeRequest light.CodeRequest
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
// GetCost returns the cost of the given ODR request according to the serving
 | 
					// GetCost returns the cost of the given ODR request according to the serving
 | 
				
			||||||
// peer's cost table (implementation of LesOdrRequest)
 | 
					// peer's cost table (implementation of LesOdrRequest)
 | 
				
			||||||
func (r *CodeRequest) GetCost(peer *peer) uint64 {
 | 
					func (r *CodeRequest) GetCost(peer *serverPeer) uint64 {
 | 
				
			||||||
	return peer.GetRequestCost(GetCodeMsg, 1)
 | 
						return peer.getRequestCost(GetCodeMsg, 1)
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// CanSend tells if a certain peer is suitable for serving the given request
 | 
					// CanSend tells if a certain peer is suitable for serving the given request
 | 
				
			||||||
func (r *CodeRequest) CanSend(peer *peer) bool {
 | 
					func (r *CodeRequest) CanSend(peer *serverPeer) bool {
 | 
				
			||||||
	return peer.HasBlock(r.Id.BlockHash, r.Id.BlockNumber, true)
 | 
						return peer.HasBlock(r.Id.BlockHash, r.Id.BlockNumber, true)
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// Request sends an ODR request to the LES network (implementation of LesOdrRequest)
 | 
					// Request sends an ODR request to the LES network (implementation of LesOdrRequest)
 | 
				
			||||||
func (r *CodeRequest) Request(reqID uint64, peer *peer) error {
 | 
					func (r *CodeRequest) Request(reqID uint64, peer *serverPeer) error {
 | 
				
			||||||
	peer.Log().Debug("Requesting code data", "hash", r.Hash)
 | 
						peer.Log().Debug("Requesting code data", "hash", r.Hash)
 | 
				
			||||||
	req := CodeReq{
 | 
						req := CodeReq{
 | 
				
			||||||
		BHash:  r.Id.BlockHash,
 | 
							BHash:  r.Id.BlockHash,
 | 
				
			||||||
		AccKey: r.Id.AccKey,
 | 
							AccKey: r.Id.AccKey,
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	return peer.RequestCode(reqID, r.GetCost(peer), []CodeReq{req})
 | 
						return peer.requestCode(reqID, []CodeReq{req})
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// Valid processes an ODR request reply message from the LES network
 | 
					// Valid processes an ODR request reply message from the LES network
 | 
				
			||||||
@@ -316,12 +316,12 @@ type ChtRequest light.ChtRequest
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
// GetCost returns the cost of the given ODR request according to the serving
 | 
					// GetCost returns the cost of the given ODR request according to the serving
 | 
				
			||||||
// peer's cost table (implementation of LesOdrRequest)
 | 
					// peer's cost table (implementation of LesOdrRequest)
 | 
				
			||||||
func (r *ChtRequest) GetCost(peer *peer) uint64 {
 | 
					func (r *ChtRequest) GetCost(peer *serverPeer) uint64 {
 | 
				
			||||||
	return peer.GetRequestCost(GetHelperTrieProofsMsg, 1)
 | 
						return peer.getRequestCost(GetHelperTrieProofsMsg, 1)
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// CanSend tells if a certain peer is suitable for serving the given request
 | 
					// CanSend tells if a certain peer is suitable for serving the given request
 | 
				
			||||||
func (r *ChtRequest) CanSend(peer *peer) bool {
 | 
					func (r *ChtRequest) CanSend(peer *serverPeer) bool {
 | 
				
			||||||
	peer.lock.RLock()
 | 
						peer.lock.RLock()
 | 
				
			||||||
	defer peer.lock.RUnlock()
 | 
						defer peer.lock.RUnlock()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -333,7 +333,7 @@ func (r *ChtRequest) CanSend(peer *peer) bool {
 | 
				
			|||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// Request sends an ODR request to the LES network (implementation of LesOdrRequest)
 | 
					// Request sends an ODR request to the LES network (implementation of LesOdrRequest)
 | 
				
			||||||
func (r *ChtRequest) Request(reqID uint64, peer *peer) error {
 | 
					func (r *ChtRequest) Request(reqID uint64, peer *serverPeer) error {
 | 
				
			||||||
	peer.Log().Debug("Requesting CHT", "cht", r.ChtNum, "block", r.BlockNum)
 | 
						peer.Log().Debug("Requesting CHT", "cht", r.ChtNum, "block", r.BlockNum)
 | 
				
			||||||
	var encNum [8]byte
 | 
						var encNum [8]byte
 | 
				
			||||||
	binary.BigEndian.PutUint64(encNum[:], r.BlockNum)
 | 
						binary.BigEndian.PutUint64(encNum[:], r.BlockNum)
 | 
				
			||||||
@@ -343,7 +343,7 @@ func (r *ChtRequest) Request(reqID uint64, peer *peer) error {
 | 
				
			|||||||
		Key:     encNum[:],
 | 
							Key:     encNum[:],
 | 
				
			||||||
		AuxReq:  auxHeader,
 | 
							AuxReq:  auxHeader,
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	return peer.RequestHelperTrieProofs(reqID, r.GetCost(peer), []HelperTrieReq{req})
 | 
						return peer.requestHelperTrieProofs(reqID, []HelperTrieReq{req})
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// Valid processes an ODR request reply message from the LES network
 | 
					// Valid processes an ODR request reply message from the LES network
 | 
				
			||||||
@@ -413,12 +413,12 @@ type BloomRequest light.BloomRequest
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
// GetCost returns the cost of the given ODR request according to the serving
 | 
					// GetCost returns the cost of the given ODR request according to the serving
 | 
				
			||||||
// peer's cost table (implementation of LesOdrRequest)
 | 
					// peer's cost table (implementation of LesOdrRequest)
 | 
				
			||||||
func (r *BloomRequest) GetCost(peer *peer) uint64 {
 | 
					func (r *BloomRequest) GetCost(peer *serverPeer) uint64 {
 | 
				
			||||||
	return peer.GetRequestCost(GetHelperTrieProofsMsg, len(r.SectionIndexList))
 | 
						return peer.getRequestCost(GetHelperTrieProofsMsg, len(r.SectionIndexList))
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// CanSend tells if a certain peer is suitable for serving the given request
 | 
					// CanSend tells if a certain peer is suitable for serving the given request
 | 
				
			||||||
func (r *BloomRequest) CanSend(peer *peer) bool {
 | 
					func (r *BloomRequest) CanSend(peer *serverPeer) bool {
 | 
				
			||||||
	peer.lock.RLock()
 | 
						peer.lock.RLock()
 | 
				
			||||||
	defer peer.lock.RUnlock()
 | 
						defer peer.lock.RUnlock()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -429,7 +429,7 @@ func (r *BloomRequest) CanSend(peer *peer) bool {
 | 
				
			|||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// Request sends an ODR request to the LES network (implementation of LesOdrRequest)
 | 
					// Request sends an ODR request to the LES network (implementation of LesOdrRequest)
 | 
				
			||||||
func (r *BloomRequest) Request(reqID uint64, peer *peer) error {
 | 
					func (r *BloomRequest) Request(reqID uint64, peer *serverPeer) error {
 | 
				
			||||||
	peer.Log().Debug("Requesting BloomBits", "bloomTrie", r.BloomTrieNum, "bitIdx", r.BitIdx, "sections", r.SectionIndexList)
 | 
						peer.Log().Debug("Requesting BloomBits", "bloomTrie", r.BloomTrieNum, "bitIdx", r.BitIdx, "sections", r.SectionIndexList)
 | 
				
			||||||
	reqs := make([]HelperTrieReq, len(r.SectionIndexList))
 | 
						reqs := make([]HelperTrieReq, len(r.SectionIndexList))
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -444,7 +444,7 @@ func (r *BloomRequest) Request(reqID uint64, peer *peer) error {
 | 
				
			|||||||
			Key:     common.CopyBytes(encNumber[:]),
 | 
								Key:     common.CopyBytes(encNumber[:]),
 | 
				
			||||||
		}
 | 
							}
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	return peer.RequestHelperTrieProofs(reqID, r.GetCost(peer), reqs)
 | 
						return peer.requestHelperTrieProofs(reqID, reqs)
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// Valid processes an ODR request reply message from the LES network
 | 
					// Valid processes an ODR request reply message from the LES network
 | 
				
			||||||
@@ -489,19 +489,19 @@ type TxStatusRequest light.TxStatusRequest
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
// GetCost returns the cost of the given ODR request according to the serving
 | 
					// GetCost returns the cost of the given ODR request according to the serving
 | 
				
			||||||
// peer's cost table (implementation of LesOdrRequest)
 | 
					// peer's cost table (implementation of LesOdrRequest)
 | 
				
			||||||
func (r *TxStatusRequest) GetCost(peer *peer) uint64 {
 | 
					func (r *TxStatusRequest) GetCost(peer *serverPeer) uint64 {
 | 
				
			||||||
	return peer.GetRequestCost(GetTxStatusMsg, len(r.Hashes))
 | 
						return peer.getRequestCost(GetTxStatusMsg, len(r.Hashes))
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// CanSend tells if a certain peer is suitable for serving the given request
 | 
					// CanSend tells if a certain peer is suitable for serving the given request
 | 
				
			||||||
func (r *TxStatusRequest) CanSend(peer *peer) bool {
 | 
					func (r *TxStatusRequest) CanSend(peer *serverPeer) bool {
 | 
				
			||||||
	return peer.version >= lpv2
 | 
						return peer.version >= lpv2
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// Request sends an ODR request to the LES network (implementation of LesOdrRequest)
 | 
					// Request sends an ODR request to the LES network (implementation of LesOdrRequest)
 | 
				
			||||||
func (r *TxStatusRequest) Request(reqID uint64, peer *peer) error {
 | 
					func (r *TxStatusRequest) Request(reqID uint64, peer *serverPeer) error {
 | 
				
			||||||
	peer.Log().Debug("Requesting transaction status", "count", len(r.Hashes))
 | 
						peer.Log().Debug("Requesting transaction status", "count", len(r.Hashes))
 | 
				
			||||||
	return peer.RequestTxStatus(reqID, r.GetCost(peer), r.Hashes)
 | 
						return peer.requestTxStatus(reqID, r.Hashes)
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// Valid processes an ODR request reply message from the LES network
 | 
					// Valid processes an ODR request reply message from the LES network
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -186,7 +186,7 @@ func testOdr(t *testing.T, protocol int, expFail uint64, checkCached bool, fn od
 | 
				
			|||||||
	server, client, tearDown := newClientServerEnv(t, 4, protocol, nil, nil, 0, false, true)
 | 
						server, client, tearDown := newClientServerEnv(t, 4, protocol, nil, nil, 0, false, true)
 | 
				
			||||||
	defer tearDown()
 | 
						defer tearDown()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	client.handler.synchronise(client.peer.peer)
 | 
						client.handler.synchronise(client.peer.speer)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	// Ensure the client has synced all necessary data.
 | 
						// Ensure the client has synced all necessary data.
 | 
				
			||||||
	clientHead := client.handler.backend.blockchain.CurrentHeader()
 | 
						clientHead := client.handler.backend.blockchain.CurrentHeader()
 | 
				
			||||||
@@ -224,19 +224,19 @@ func testOdr(t *testing.T, protocol int, expFail uint64, checkCached bool, fn od
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
	// expect retrievals to fail (except genesis block) without a les peer
 | 
						// expect retrievals to fail (except genesis block) without a les peer
 | 
				
			||||||
	client.handler.backend.peers.lock.Lock()
 | 
						client.handler.backend.peers.lock.Lock()
 | 
				
			||||||
	client.peer.peer.hasBlock = func(common.Hash, uint64, bool) bool { return false }
 | 
						client.peer.speer.hasBlock = func(common.Hash, uint64, bool) bool { return false }
 | 
				
			||||||
	client.handler.backend.peers.lock.Unlock()
 | 
						client.handler.backend.peers.lock.Unlock()
 | 
				
			||||||
	test(expFail)
 | 
						test(expFail)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	// expect all retrievals to pass
 | 
						// expect all retrievals to pass
 | 
				
			||||||
	client.handler.backend.peers.lock.Lock()
 | 
						client.handler.backend.peers.lock.Lock()
 | 
				
			||||||
	client.peer.peer.hasBlock = func(common.Hash, uint64, bool) bool { return true }
 | 
						client.peer.speer.hasBlock = func(common.Hash, uint64, bool) bool { return true }
 | 
				
			||||||
	client.handler.backend.peers.lock.Unlock()
 | 
						client.handler.backend.peers.lock.Unlock()
 | 
				
			||||||
	test(5)
 | 
						test(5)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	// still expect all retrievals to pass, now data should be cached locally
 | 
						// still expect all retrievals to pass, now data should be cached locally
 | 
				
			||||||
	if checkCached {
 | 
						if checkCached {
 | 
				
			||||||
		client.handler.backend.peers.Unregister(client.peer.peer.id)
 | 
							client.handler.backend.peers.unregister(client.peer.speer.id)
 | 
				
			||||||
		time.Sleep(time.Millisecond * 10) // ensure that all peerSetNotify callbacks are executed
 | 
							time.Sleep(time.Millisecond * 10) // ensure that all peerSetNotify callbacks are executed
 | 
				
			||||||
		test(5)
 | 
							test(5)
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
 
 | 
				
			|||||||
							
								
								
									
										1520
									
								
								les/peer.go
									
									
									
									
									
								
							
							
						
						
									
										1520
									
								
								les/peer.go
									
									
									
									
									
								
							
										
											
												File diff suppressed because it is too large
												Load Diff
											
										
									
								
							
							
								
								
									
										369
									
								
								les/peer_test.go
									
									
									
									
									
								
							
							
						
						
									
										369
									
								
								les/peer_test.go
									
									
									
									
									
								
							@@ -17,50 +17,107 @@
 | 
				
			|||||||
package les
 | 
					package les
 | 
				
			||||||
 | 
					
 | 
				
			||||||
import (
 | 
					import (
 | 
				
			||||||
 | 
						"crypto/rand"
 | 
				
			||||||
	"math/big"
 | 
						"math/big"
 | 
				
			||||||
	"net"
 | 
						"reflect"
 | 
				
			||||||
 | 
						"sort"
 | 
				
			||||||
	"testing"
 | 
						"testing"
 | 
				
			||||||
 | 
						"time"
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	"github.com/ethereum/go-ethereum/common"
 | 
						"github.com/ethereum/go-ethereum/common"
 | 
				
			||||||
	"github.com/ethereum/go-ethereum/common/mclock"
 | 
					 | 
				
			||||||
	"github.com/ethereum/go-ethereum/core/rawdb"
 | 
					 | 
				
			||||||
	"github.com/ethereum/go-ethereum/crypto"
 | 
					 | 
				
			||||||
	"github.com/ethereum/go-ethereum/eth"
 | 
					 | 
				
			||||||
	"github.com/ethereum/go-ethereum/les/flowcontrol"
 | 
					 | 
				
			||||||
	"github.com/ethereum/go-ethereum/p2p"
 | 
						"github.com/ethereum/go-ethereum/p2p"
 | 
				
			||||||
	"github.com/ethereum/go-ethereum/p2p/enode"
 | 
						"github.com/ethereum/go-ethereum/p2p/enode"
 | 
				
			||||||
	"github.com/ethereum/go-ethereum/rlp"
 | 
					 | 
				
			||||||
)
 | 
					)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
const protocolVersion = lpv2
 | 
					type testServerPeerSub struct {
 | 
				
			||||||
 | 
						regCh   chan *serverPeer
 | 
				
			||||||
var (
 | 
						unregCh chan *serverPeer
 | 
				
			||||||
	hash    = common.HexToHash("deadbeef")
 | 
					 | 
				
			||||||
	genesis = common.HexToHash("cafebabe")
 | 
					 | 
				
			||||||
	headNum = uint64(1234)
 | 
					 | 
				
			||||||
	td      = big.NewInt(123)
 | 
					 | 
				
			||||||
)
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
func newNodeID(t *testing.T) *enode.Node {
 | 
					 | 
				
			||||||
	key, err := crypto.GenerateKey()
 | 
					 | 
				
			||||||
	if err != nil {
 | 
					 | 
				
			||||||
		t.Fatal("generate key err:", err)
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
	return enode.NewV4(&key.PublicKey, net.IP{}, 35000, 35000)
 | 
					 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// ulc connects to trusted peer and send announceType=announceTypeSigned
 | 
					func newTestServerPeerSub() *testServerPeerSub {
 | 
				
			||||||
func TestPeerHandshakeSetAnnounceTypeToAnnounceTypeSignedForTrustedPeer(t *testing.T) {
 | 
						return &testServerPeerSub{
 | 
				
			||||||
	id := newNodeID(t).ID()
 | 
							regCh:   make(chan *serverPeer, 1),
 | 
				
			||||||
 | 
							unregCh: make(chan *serverPeer, 1),
 | 
				
			||||||
 | 
						}
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	// peer to connect(on ulc side)
 | 
					func (t *testServerPeerSub) registerPeer(p *serverPeer)   { t.regCh <- p }
 | 
				
			||||||
	p := peer{
 | 
					func (t *testServerPeerSub) unregisterPeer(p *serverPeer) { t.unregCh <- p }
 | 
				
			||||||
		Peer:    p2p.NewPeer(id, "test peer", []p2p.Cap{}),
 | 
					
 | 
				
			||||||
		version: protocolVersion,
 | 
					func TestPeerSubscription(t *testing.T) {
 | 
				
			||||||
		trusted: true,
 | 
						peers := newServerPeerSet()
 | 
				
			||||||
		rw: &rwStub{
 | 
						defer peers.close()
 | 
				
			||||||
			WriteHook: func(recvList keyValueList) {
 | 
					
 | 
				
			||||||
				recv, _ := recvList.decode()
 | 
						checkIds := func(expect []string) {
 | 
				
			||||||
 | 
							given := peers.ids()
 | 
				
			||||||
 | 
							if len(given) == 0 && len(expect) == 0 {
 | 
				
			||||||
 | 
								return
 | 
				
			||||||
 | 
							}
 | 
				
			||||||
 | 
							sort.Strings(given)
 | 
				
			||||||
 | 
							sort.Strings(expect)
 | 
				
			||||||
 | 
							if !reflect.DeepEqual(given, expect) {
 | 
				
			||||||
 | 
								t.Fatalf("all peer ids mismatch, want %v, given %v", expect, given)
 | 
				
			||||||
 | 
							}
 | 
				
			||||||
 | 
						}
 | 
				
			||||||
 | 
						checkPeers := func(peerCh chan *serverPeer) {
 | 
				
			||||||
 | 
							select {
 | 
				
			||||||
 | 
							case <-peerCh:
 | 
				
			||||||
 | 
							case <-time.NewTimer(100 * time.Millisecond).C:
 | 
				
			||||||
 | 
								t.Fatalf("timeout, no event received")
 | 
				
			||||||
 | 
							}
 | 
				
			||||||
 | 
							select {
 | 
				
			||||||
 | 
							case <-peerCh:
 | 
				
			||||||
 | 
								t.Fatalf("unexpected event received")
 | 
				
			||||||
 | 
							case <-time.NewTimer(10 * time.Millisecond).C:
 | 
				
			||||||
 | 
							}
 | 
				
			||||||
 | 
						}
 | 
				
			||||||
 | 
						checkIds([]string{})
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
						sub := newTestServerPeerSub()
 | 
				
			||||||
 | 
						peers.subscribe(sub)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
						// Generate a random id and create the peer
 | 
				
			||||||
 | 
						var id enode.ID
 | 
				
			||||||
 | 
						rand.Read(id[:])
 | 
				
			||||||
 | 
						peer := newServerPeer(2, NetworkId, false, p2p.NewPeer(id, "name", nil), nil)
 | 
				
			||||||
 | 
						peers.register(peer)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
						checkIds([]string{peer.id})
 | 
				
			||||||
 | 
						checkPeers(sub.regCh)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
						peers.unregister(peer.id)
 | 
				
			||||||
 | 
						checkIds([]string{})
 | 
				
			||||||
 | 
						checkPeers(sub.unregCh)
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					func TestHandshake(t *testing.T) {
 | 
				
			||||||
 | 
						// Create a message pipe to communicate through
 | 
				
			||||||
 | 
						app, net := p2p.MsgPipe()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
						// Generate a random id and create the peer
 | 
				
			||||||
 | 
						var id enode.ID
 | 
				
			||||||
 | 
						rand.Read(id[:])
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
						peer1 := newClientPeer(2, NetworkId, p2p.NewPeer(id, "name", nil), net)
 | 
				
			||||||
 | 
						peer2 := newServerPeer(2, NetworkId, true, p2p.NewPeer(id, "name", nil), app)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
						var (
 | 
				
			||||||
 | 
							errCh1 = make(chan error, 1)
 | 
				
			||||||
 | 
							errCh2 = make(chan error, 1)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
							td      = big.NewInt(100)
 | 
				
			||||||
 | 
							head    = common.HexToHash("deadbeef")
 | 
				
			||||||
 | 
							headNum = uint64(10)
 | 
				
			||||||
 | 
							genesis = common.HexToHash("cafebabe")
 | 
				
			||||||
 | 
						)
 | 
				
			||||||
 | 
						go func() {
 | 
				
			||||||
 | 
							errCh1 <- peer1.handshake(td, head, headNum, genesis, func(list *keyValueList) {
 | 
				
			||||||
 | 
								var announceType uint64 = announceTypeSigned
 | 
				
			||||||
 | 
								*list = (*list).add("announceType", announceType)
 | 
				
			||||||
 | 
							}, nil)
 | 
				
			||||||
 | 
						}()
 | 
				
			||||||
 | 
						go func() {
 | 
				
			||||||
 | 
							errCh2 <- peer2.handshake(td, head, headNum, genesis, nil, func(recv keyValueMap) error {
 | 
				
			||||||
			var reqType uint64
 | 
								var reqType uint64
 | 
				
			||||||
			err := recv.get("announceType", &reqType)
 | 
								err := recv.get("announceType", &reqType)
 | 
				
			||||||
			if err != nil {
 | 
								if err != nil {
 | 
				
			||||||
@@ -69,234 +126,22 @@ func TestPeerHandshakeSetAnnounceTypeToAnnounceTypeSignedForTrustedPeer(t *testi
 | 
				
			|||||||
			if reqType != announceTypeSigned {
 | 
								if reqType != announceTypeSigned {
 | 
				
			||||||
				t.Fatal("Expected announceTypeSigned")
 | 
									t.Fatal("Expected announceTypeSigned")
 | 
				
			||||||
			}
 | 
								}
 | 
				
			||||||
			},
 | 
					 | 
				
			||||||
			ReadHook: func(l keyValueList) keyValueList {
 | 
					 | 
				
			||||||
				l = l.add("serveHeaders", nil)
 | 
					 | 
				
			||||||
				l = l.add("serveChainSince", uint64(0))
 | 
					 | 
				
			||||||
				l = l.add("serveStateSince", uint64(0))
 | 
					 | 
				
			||||||
				l = l.add("txRelay", nil)
 | 
					 | 
				
			||||||
				l = l.add("flowControl/BL", uint64(0))
 | 
					 | 
				
			||||||
				l = l.add("flowControl/MRR", uint64(0))
 | 
					 | 
				
			||||||
				l = l.add("flowControl/MRC", testCostList(0))
 | 
					 | 
				
			||||||
				return l
 | 
					 | 
				
			||||||
			},
 | 
					 | 
				
			||||||
		},
 | 
					 | 
				
			||||||
		network: NetworkId,
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
	err := p.Handshake(td, hash, headNum, genesis, nil)
 | 
					 | 
				
			||||||
	if err != nil {
 | 
					 | 
				
			||||||
		t.Fatalf("Handshake error: %s", err)
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
	if p.announceType != announceTypeSigned {
 | 
					 | 
				
			||||||
		t.Fatal("Incorrect announceType")
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
func TestPeerHandshakeAnnounceTypeSignedForTrustedPeersPeerNotInTrusted(t *testing.T) {
 | 
					 | 
				
			||||||
	id := newNodeID(t).ID()
 | 
					 | 
				
			||||||
	p := peer{
 | 
					 | 
				
			||||||
		Peer:    p2p.NewPeer(id, "test peer", []p2p.Cap{}),
 | 
					 | 
				
			||||||
		version: protocolVersion,
 | 
					 | 
				
			||||||
		rw: &rwStub{
 | 
					 | 
				
			||||||
			WriteHook: func(recvList keyValueList) {
 | 
					 | 
				
			||||||
				// checking that ulc sends to peer allowedRequests=noRequests and announceType != announceTypeSigned
 | 
					 | 
				
			||||||
				recv, _ := recvList.decode()
 | 
					 | 
				
			||||||
				var reqType uint64
 | 
					 | 
				
			||||||
				err := recv.get("announceType", &reqType)
 | 
					 | 
				
			||||||
				if err != nil {
 | 
					 | 
				
			||||||
					t.Fatal(err)
 | 
					 | 
				
			||||||
				}
 | 
					 | 
				
			||||||
				if reqType == announceTypeSigned {
 | 
					 | 
				
			||||||
					t.Fatal("Expected not announceTypeSigned")
 | 
					 | 
				
			||||||
				}
 | 
					 | 
				
			||||||
			},
 | 
					 | 
				
			||||||
			ReadHook: func(l keyValueList) keyValueList {
 | 
					 | 
				
			||||||
				l = l.add("serveHeaders", nil)
 | 
					 | 
				
			||||||
				l = l.add("serveChainSince", uint64(0))
 | 
					 | 
				
			||||||
				l = l.add("serveStateSince", uint64(0))
 | 
					 | 
				
			||||||
				l = l.add("txRelay", nil)
 | 
					 | 
				
			||||||
				l = l.add("flowControl/BL", uint64(0))
 | 
					 | 
				
			||||||
				l = l.add("flowControl/MRR", uint64(0))
 | 
					 | 
				
			||||||
				l = l.add("flowControl/MRC", testCostList(0))
 | 
					 | 
				
			||||||
				return l
 | 
					 | 
				
			||||||
			},
 | 
					 | 
				
			||||||
		},
 | 
					 | 
				
			||||||
		network: NetworkId,
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
	err := p.Handshake(td, hash, headNum, genesis, nil)
 | 
					 | 
				
			||||||
	if err != nil {
 | 
					 | 
				
			||||||
		t.Fatal(err)
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
	if p.announceType == announceTypeSigned {
 | 
					 | 
				
			||||||
		t.Fatal("Incorrect announceType")
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
func TestPeerHandshakeDefaultAllRequests(t *testing.T) {
 | 
					 | 
				
			||||||
	id := newNodeID(t).ID()
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
	s := generateLesServer()
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
	p := peer{
 | 
					 | 
				
			||||||
		Peer:    p2p.NewPeer(id, "test peer", []p2p.Cap{}),
 | 
					 | 
				
			||||||
		version: protocolVersion,
 | 
					 | 
				
			||||||
		rw: &rwStub{
 | 
					 | 
				
			||||||
			ReadHook: func(l keyValueList) keyValueList {
 | 
					 | 
				
			||||||
				l = l.add("announceType", uint64(announceTypeSigned))
 | 
					 | 
				
			||||||
				l = l.add("allowedRequests", uint64(0))
 | 
					 | 
				
			||||||
				return l
 | 
					 | 
				
			||||||
			},
 | 
					 | 
				
			||||||
		},
 | 
					 | 
				
			||||||
		network: NetworkId,
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
	err := p.Handshake(td, hash, headNum, genesis, s)
 | 
					 | 
				
			||||||
	if err != nil {
 | 
					 | 
				
			||||||
		t.Fatal(err)
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
	if p.onlyAnnounce {
 | 
					 | 
				
			||||||
		t.Fatal("Incorrect announceType")
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
func TestPeerHandshakeServerSendOnlyAnnounceRequestsHeaders(t *testing.T) {
 | 
					 | 
				
			||||||
	id := newNodeID(t).ID()
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
	s := generateLesServer()
 | 
					 | 
				
			||||||
	s.config.UltraLightOnlyAnnounce = true
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
	p := peer{
 | 
					 | 
				
			||||||
		Peer:    p2p.NewPeer(id, "test peer", []p2p.Cap{}),
 | 
					 | 
				
			||||||
		version: protocolVersion,
 | 
					 | 
				
			||||||
		rw: &rwStub{
 | 
					 | 
				
			||||||
			ReadHook: func(l keyValueList) keyValueList {
 | 
					 | 
				
			||||||
				l = l.add("announceType", uint64(announceTypeSigned))
 | 
					 | 
				
			||||||
				return l
 | 
					 | 
				
			||||||
			},
 | 
					 | 
				
			||||||
			WriteHook: func(l keyValueList) {
 | 
					 | 
				
			||||||
				for _, v := range l {
 | 
					 | 
				
			||||||
					if v.Key == "serveHeaders" ||
 | 
					 | 
				
			||||||
						v.Key == "serveChainSince" ||
 | 
					 | 
				
			||||||
						v.Key == "serveStateSince" ||
 | 
					 | 
				
			||||||
						v.Key == "txRelay" {
 | 
					 | 
				
			||||||
						t.Fatalf("%v exists", v.Key)
 | 
					 | 
				
			||||||
					}
 | 
					 | 
				
			||||||
				}
 | 
					 | 
				
			||||||
			},
 | 
					 | 
				
			||||||
		},
 | 
					 | 
				
			||||||
		network: NetworkId,
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
	err := p.Handshake(td, hash, headNum, genesis, s)
 | 
					 | 
				
			||||||
	if err != nil {
 | 
					 | 
				
			||||||
		t.Fatal(err)
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
}
 | 
					 | 
				
			||||||
func TestPeerHandshakeClientReceiveOnlyAnnounceRequestsHeaders(t *testing.T) {
 | 
					 | 
				
			||||||
	id := newNodeID(t).ID()
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
	p := peer{
 | 
					 | 
				
			||||||
		Peer:    p2p.NewPeer(id, "test peer", []p2p.Cap{}),
 | 
					 | 
				
			||||||
		version: protocolVersion,
 | 
					 | 
				
			||||||
		rw: &rwStub{
 | 
					 | 
				
			||||||
			ReadHook: func(l keyValueList) keyValueList {
 | 
					 | 
				
			||||||
				l = l.add("flowControl/BL", uint64(0))
 | 
					 | 
				
			||||||
				l = l.add("flowControl/MRR", uint64(0))
 | 
					 | 
				
			||||||
				l = l.add("flowControl/MRC", RequestCostList{})
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
				l = l.add("announceType", uint64(announceTypeSigned))
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
				return l
 | 
					 | 
				
			||||||
			},
 | 
					 | 
				
			||||||
		},
 | 
					 | 
				
			||||||
		network: NetworkId,
 | 
					 | 
				
			||||||
		trusted: true,
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
	err := p.Handshake(td, hash, headNum, genesis, nil)
 | 
					 | 
				
			||||||
	if err != nil {
 | 
					 | 
				
			||||||
		t.Fatal(err)
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
	if !p.onlyAnnounce {
 | 
					 | 
				
			||||||
		t.Fatal("onlyAnnounce must be true")
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
func TestPeerHandshakeClientReturnErrorOnUselessPeer(t *testing.T) {
 | 
					 | 
				
			||||||
	id := newNodeID(t).ID()
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
	p := peer{
 | 
					 | 
				
			||||||
		Peer:    p2p.NewPeer(id, "test peer", []p2p.Cap{}),
 | 
					 | 
				
			||||||
		version: protocolVersion,
 | 
					 | 
				
			||||||
		rw: &rwStub{
 | 
					 | 
				
			||||||
			ReadHook: func(l keyValueList) keyValueList {
 | 
					 | 
				
			||||||
				l = l.add("flowControl/BL", uint64(0))
 | 
					 | 
				
			||||||
				l = l.add("flowControl/MRR", uint64(0))
 | 
					 | 
				
			||||||
				l = l.add("flowControl/MRC", RequestCostList{})
 | 
					 | 
				
			||||||
				l = l.add("announceType", uint64(announceTypeSigned))
 | 
					 | 
				
			||||||
				return l
 | 
					 | 
				
			||||||
			},
 | 
					 | 
				
			||||||
		},
 | 
					 | 
				
			||||||
		network: NetworkId,
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
	err := p.Handshake(td, hash, headNum, genesis, nil)
 | 
					 | 
				
			||||||
	if err == nil {
 | 
					 | 
				
			||||||
		t.FailNow()
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
func generateLesServer() *LesServer {
 | 
					 | 
				
			||||||
	s := &LesServer{
 | 
					 | 
				
			||||||
		lesCommons: lesCommons{
 | 
					 | 
				
			||||||
			config: ð.Config{UltraLightOnlyAnnounce: true},
 | 
					 | 
				
			||||||
		},
 | 
					 | 
				
			||||||
		defParams: flowcontrol.ServerParams{
 | 
					 | 
				
			||||||
			BufLimit:    uint64(300000000),
 | 
					 | 
				
			||||||
			MinRecharge: uint64(50000),
 | 
					 | 
				
			||||||
		},
 | 
					 | 
				
			||||||
		fcManager: flowcontrol.NewClientManager(nil, &mclock.System{}),
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
	s.costTracker, _ = newCostTracker(rawdb.NewMemoryDatabase(), s.config)
 | 
					 | 
				
			||||||
	return s
 | 
					 | 
				
			||||||
}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
type rwStub struct {
 | 
					 | 
				
			||||||
	ReadHook  func(l keyValueList) keyValueList
 | 
					 | 
				
			||||||
	WriteHook func(l keyValueList)
 | 
					 | 
				
			||||||
}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
func (s *rwStub) ReadMsg() (p2p.Msg, error) {
 | 
					 | 
				
			||||||
	payload := keyValueList{}
 | 
					 | 
				
			||||||
	payload = payload.add("protocolVersion", uint64(protocolVersion))
 | 
					 | 
				
			||||||
	payload = payload.add("networkId", uint64(NetworkId))
 | 
					 | 
				
			||||||
	payload = payload.add("headTd", td)
 | 
					 | 
				
			||||||
	payload = payload.add("headHash", hash)
 | 
					 | 
				
			||||||
	payload = payload.add("headNum", headNum)
 | 
					 | 
				
			||||||
	payload = payload.add("genesisHash", genesis)
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
	if s.ReadHook != nil {
 | 
					 | 
				
			||||||
		payload = s.ReadHook(payload)
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
	size, p, err := rlp.EncodeToReader(payload)
 | 
					 | 
				
			||||||
	if err != nil {
 | 
					 | 
				
			||||||
		return p2p.Msg{}, err
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
	return p2p.Msg{
 | 
					 | 
				
			||||||
		Size:    uint32(size),
 | 
					 | 
				
			||||||
		Payload: p,
 | 
					 | 
				
			||||||
	}, nil
 | 
					 | 
				
			||||||
}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
func (s *rwStub) WriteMsg(m p2p.Msg) error {
 | 
					 | 
				
			||||||
	recvList := keyValueList{}
 | 
					 | 
				
			||||||
	if err := m.Decode(&recvList); err != nil {
 | 
					 | 
				
			||||||
		return err
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
	if s.WriteHook != nil {
 | 
					 | 
				
			||||||
		s.WriteHook(recvList)
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
			return nil
 | 
								return nil
 | 
				
			||||||
 | 
							})
 | 
				
			||||||
 | 
						}()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
						for i := 0; i < 2; i++ {
 | 
				
			||||||
 | 
							select {
 | 
				
			||||||
 | 
							case err := <-errCh1:
 | 
				
			||||||
 | 
								if err != nil {
 | 
				
			||||||
 | 
									t.Fatalf("handshake failed, %v", err)
 | 
				
			||||||
 | 
								}
 | 
				
			||||||
 | 
							case err := <-errCh2:
 | 
				
			||||||
 | 
								if err != nil {
 | 
				
			||||||
 | 
									t.Fatalf("handshake failed, %v", err)
 | 
				
			||||||
 | 
								}
 | 
				
			||||||
 | 
							case <-time.NewTimer(100 * time.Millisecond).C:
 | 
				
			||||||
 | 
								t.Fatalf("timeout")
 | 
				
			||||||
 | 
							}
 | 
				
			||||||
 | 
						}
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -81,8 +81,7 @@ func testAccess(t *testing.T, protocol int, fn accessTestFn) {
 | 
				
			|||||||
	// Assemble the test environment
 | 
						// Assemble the test environment
 | 
				
			||||||
	server, client, tearDown := newClientServerEnv(t, 4, protocol, nil, nil, 0, false, true)
 | 
						server, client, tearDown := newClientServerEnv(t, 4, protocol, nil, nil, 0, false, true)
 | 
				
			||||||
	defer tearDown()
 | 
						defer tearDown()
 | 
				
			||||||
 | 
						client.handler.synchronise(client.peer.speer)
 | 
				
			||||||
	client.handler.synchronise(client.peer.peer)
 | 
					 | 
				
			||||||
 | 
					
 | 
				
			||||||
	// Ensure the client has synced all necessary data.
 | 
						// Ensure the client has synced all necessary data.
 | 
				
			||||||
	clientHead := client.handler.backend.blockchain.CurrentHeader()
 | 
						clientHead := client.handler.backend.blockchain.CurrentHeader()
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -38,7 +38,7 @@ var (
 | 
				
			|||||||
// matching replies by request ID and handles timeouts and resends if necessary.
 | 
					// matching replies by request ID and handles timeouts and resends if necessary.
 | 
				
			||||||
type retrieveManager struct {
 | 
					type retrieveManager struct {
 | 
				
			||||||
	dist       *requestDistributor
 | 
						dist       *requestDistributor
 | 
				
			||||||
	peers      *peerSet
 | 
						peers      *serverPeerSet
 | 
				
			||||||
	serverPool peerSelector
 | 
						serverPool peerSelector
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	lock     sync.RWMutex
 | 
						lock     sync.RWMutex
 | 
				
			||||||
@@ -99,7 +99,7 @@ const (
 | 
				
			|||||||
)
 | 
					)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// newRetrieveManager creates the retrieve manager
 | 
					// newRetrieveManager creates the retrieve manager
 | 
				
			||||||
func newRetrieveManager(peers *peerSet, dist *requestDistributor, serverPool peerSelector) *retrieveManager {
 | 
					func newRetrieveManager(peers *serverPeerSet, dist *requestDistributor, serverPool peerSelector) *retrieveManager {
 | 
				
			||||||
	return &retrieveManager{
 | 
						return &retrieveManager{
 | 
				
			||||||
		peers:      peers,
 | 
							peers:      peers,
 | 
				
			||||||
		dist:       dist,
 | 
							dist:       dist,
 | 
				
			||||||
@@ -337,7 +337,7 @@ func (r *sentReq) tryRequest() {
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
	defer func() {
 | 
						defer func() {
 | 
				
			||||||
		// send feedback to server pool and remove peer if hard timeout happened
 | 
							// send feedback to server pool and remove peer if hard timeout happened
 | 
				
			||||||
		pp, ok := p.(*peer)
 | 
							pp, ok := p.(*serverPeer)
 | 
				
			||||||
		if ok && r.rm.serverPool != nil {
 | 
							if ok && r.rm.serverPool != nil {
 | 
				
			||||||
			respTime := time.Duration(mclock.Now() - reqSent)
 | 
								respTime := time.Duration(mclock.Now() - reqSent)
 | 
				
			||||||
			r.rm.serverPool.adjustResponseTime(pp.poolEntry, respTime, srto)
 | 
								r.rm.serverPool.adjustResponseTime(pp.poolEntry, respTime, srto)
 | 
				
			||||||
@@ -345,7 +345,7 @@ func (r *sentReq) tryRequest() {
 | 
				
			|||||||
		if hrto {
 | 
							if hrto {
 | 
				
			||||||
			pp.Log().Debug("Request timed out hard")
 | 
								pp.Log().Debug("Request timed out hard")
 | 
				
			||||||
			if r.rm.peers != nil {
 | 
								if r.rm.peers != nil {
 | 
				
			||||||
				r.rm.peers.Unregister(pp.id)
 | 
									r.rm.peers.unregister(pp.id)
 | 
				
			||||||
			}
 | 
								}
 | 
				
			||||||
		}
 | 
							}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -40,6 +40,7 @@ type LesServer struct {
 | 
				
			|||||||
	lesCommons
 | 
						lesCommons
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	archiveMode bool // Flag whether the ethereum node runs in archive mode.
 | 
						archiveMode bool // Flag whether the ethereum node runs in archive mode.
 | 
				
			||||||
 | 
						peers       *clientPeerSet
 | 
				
			||||||
	handler     *serverHandler
 | 
						handler     *serverHandler
 | 
				
			||||||
	lesTopics   []discv5.Topic
 | 
						lesTopics   []discv5.Topic
 | 
				
			||||||
	privateKey  *ecdsa.PrivateKey
 | 
						privateKey  *ecdsa.PrivateKey
 | 
				
			||||||
@@ -75,13 +76,13 @@ func NewLesServer(e *eth.Ethereum, config *eth.Config) (*LesServer, error) {
 | 
				
			|||||||
			chainConfig:      e.BlockChain().Config(),
 | 
								chainConfig:      e.BlockChain().Config(),
 | 
				
			||||||
			iConfig:          light.DefaultServerIndexerConfig,
 | 
								iConfig:          light.DefaultServerIndexerConfig,
 | 
				
			||||||
			chainDb:          e.ChainDb(),
 | 
								chainDb:          e.ChainDb(),
 | 
				
			||||||
			peers:            newPeerSet(),
 | 
					 | 
				
			||||||
			chainReader:      e.BlockChain(),
 | 
								chainReader:      e.BlockChain(),
 | 
				
			||||||
			chtIndexer:       light.NewChtIndexer(e.ChainDb(), nil, params.CHTFrequency, params.HelperTrieProcessConfirmations),
 | 
								chtIndexer:       light.NewChtIndexer(e.ChainDb(), nil, params.CHTFrequency, params.HelperTrieProcessConfirmations),
 | 
				
			||||||
			bloomTrieIndexer: light.NewBloomTrieIndexer(e.ChainDb(), nil, params.BloomBitsBlocks, params.BloomTrieFrequency),
 | 
								bloomTrieIndexer: light.NewBloomTrieIndexer(e.ChainDb(), nil, params.BloomBitsBlocks, params.BloomTrieFrequency),
 | 
				
			||||||
			closeCh:          make(chan struct{}),
 | 
								closeCh:          make(chan struct{}),
 | 
				
			||||||
		},
 | 
							},
 | 
				
			||||||
		archiveMode:  e.ArchiveMode(),
 | 
							archiveMode:  e.ArchiveMode(),
 | 
				
			||||||
 | 
							peers:        newClientPeerSet(),
 | 
				
			||||||
		lesTopics:    lesTopics,
 | 
							lesTopics:    lesTopics,
 | 
				
			||||||
		fcManager:    flowcontrol.NewClientManager(nil, &mclock.System{}),
 | 
							fcManager:    flowcontrol.NewClientManager(nil, &mclock.System{}),
 | 
				
			||||||
		servingQueue: newServingQueue(int64(time.Millisecond*10), float64(config.LightServ)/100),
 | 
							servingQueue: newServingQueue(int64(time.Millisecond*10), float64(config.LightServ)/100),
 | 
				
			||||||
@@ -115,7 +116,7 @@ func NewLesServer(e *eth.Ethereum, config *eth.Config) (*LesServer, error) {
 | 
				
			|||||||
		srv.maxCapacity = totalRecharge
 | 
							srv.maxCapacity = totalRecharge
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	srv.fcManager.SetCapacityLimits(srv.freeCapacity, srv.maxCapacity, srv.freeCapacity*2)
 | 
						srv.fcManager.SetCapacityLimits(srv.freeCapacity, srv.maxCapacity, srv.freeCapacity*2)
 | 
				
			||||||
	srv.clientPool = newClientPool(srv.chainDb, srv.freeCapacity, mclock.System{}, func(id enode.ID) { go srv.peers.Unregister(peerIdToString(id)) })
 | 
						srv.clientPool = newClientPool(srv.chainDb, srv.freeCapacity, mclock.System{}, func(id enode.ID) { go srv.peers.unregister(peerIdToString(id)) })
 | 
				
			||||||
	srv.clientPool.setDefaultFactors(priceFactors{0, 1, 1}, priceFactors{0, 1, 1})
 | 
						srv.clientPool.setDefaultFactors(priceFactors{0, 1, 1}, priceFactors{0, 1, 1})
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	checkpoint := srv.latestLocalCheckpoint()
 | 
						checkpoint := srv.latestLocalCheckpoint()
 | 
				
			||||||
@@ -152,7 +153,7 @@ func (s *LesServer) APIs() []rpc.API {
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
func (s *LesServer) Protocols() []p2p.Protocol {
 | 
					func (s *LesServer) Protocols() []p2p.Protocol {
 | 
				
			||||||
	ps := s.makeProtocols(ServerProtocolVersions, s.handler.runPeer, func(id enode.ID) interface{} {
 | 
						ps := s.makeProtocols(ServerProtocolVersions, s.handler.runPeer, func(id enode.ID) interface{} {
 | 
				
			||||||
		if p := s.peers.Peer(peerIdToString(id)); p != nil {
 | 
							if p := s.peers.peer(peerIdToString(id)); p != nil {
 | 
				
			||||||
			return p.Info()
 | 
								return p.Info()
 | 
				
			||||||
		}
 | 
							}
 | 
				
			||||||
		return nil
 | 
							return nil
 | 
				
			||||||
@@ -194,7 +195,7 @@ func (s *LesServer) Stop() {
 | 
				
			|||||||
	// This also closes the gate for any new registrations on the peer set.
 | 
						// This also closes the gate for any new registrations on the peer set.
 | 
				
			||||||
	// sessions which are already established but not added to pm.peers yet
 | 
						// sessions which are already established but not added to pm.peers yet
 | 
				
			||||||
	// will exit when they try to register.
 | 
						// will exit when they try to register.
 | 
				
			||||||
	s.peers.Close()
 | 
						s.peers.close()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	s.fcManager.Stop()
 | 
						s.fcManager.Stop()
 | 
				
			||||||
	s.costTracker.stop()
 | 
						s.costTracker.stop()
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -101,13 +101,14 @@ func (h *serverHandler) stop() {
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
// runPeer is the p2p protocol run function for the given version.
 | 
					// runPeer is the p2p protocol run function for the given version.
 | 
				
			||||||
func (h *serverHandler) runPeer(version uint, p *p2p.Peer, rw p2p.MsgReadWriter) error {
 | 
					func (h *serverHandler) runPeer(version uint, p *p2p.Peer, rw p2p.MsgReadWriter) error {
 | 
				
			||||||
	peer := newPeer(int(version), h.server.config.NetworkId, false, p, newMeteredMsgWriter(rw, int(version)))
 | 
						peer := newClientPeer(int(version), h.server.config.NetworkId, p, newMeteredMsgWriter(rw, int(version)))
 | 
				
			||||||
 | 
						defer peer.close()
 | 
				
			||||||
	h.wg.Add(1)
 | 
						h.wg.Add(1)
 | 
				
			||||||
	defer h.wg.Done()
 | 
						defer h.wg.Done()
 | 
				
			||||||
	return h.handle(peer)
 | 
						return h.handle(peer)
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
func (h *serverHandler) handle(p *peer) error {
 | 
					func (h *serverHandler) handle(p *clientPeer) error {
 | 
				
			||||||
	p.Log().Debug("Light Ethereum peer connected", "name", p.Name())
 | 
						p.Log().Debug("Light Ethereum peer connected", "name", p.Name())
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	// Execute the LES handshake
 | 
						// Execute the LES handshake
 | 
				
			||||||
@@ -139,21 +140,21 @@ func (h *serverHandler) handle(p *peer) error {
 | 
				
			|||||||
		return errFullClientPool
 | 
							return errFullClientPool
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	// Register the peer locally
 | 
						// Register the peer locally
 | 
				
			||||||
	if err := h.server.peers.Register(p); err != nil {
 | 
						if err := h.server.peers.register(p); err != nil {
 | 
				
			||||||
		h.server.clientPool.disconnect(p)
 | 
							h.server.clientPool.disconnect(p)
 | 
				
			||||||
		p.Log().Error("Light Ethereum peer registration failed", "err", err)
 | 
							p.Log().Error("Light Ethereum peer registration failed", "err", err)
 | 
				
			||||||
		return err
 | 
							return err
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	clientConnectionGauge.Update(int64(h.server.peers.Len()))
 | 
						clientConnectionGauge.Update(int64(h.server.peers.len()))
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	var wg sync.WaitGroup // Wait group used to track all in-flight task routines.
 | 
						var wg sync.WaitGroup // Wait group used to track all in-flight task routines.
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	connectedAt := mclock.Now()
 | 
						connectedAt := mclock.Now()
 | 
				
			||||||
	defer func() {
 | 
						defer func() {
 | 
				
			||||||
		wg.Wait() // Ensure all background task routines have exited.
 | 
							wg.Wait() // Ensure all background task routines have exited.
 | 
				
			||||||
		h.server.peers.Unregister(p.id)
 | 
							h.server.peers.unregister(p.id)
 | 
				
			||||||
		h.server.clientPool.disconnect(p)
 | 
							h.server.clientPool.disconnect(p)
 | 
				
			||||||
		clientConnectionGauge.Update(int64(h.server.peers.Len()))
 | 
							clientConnectionGauge.Update(int64(h.server.peers.len()))
 | 
				
			||||||
		connectionTimer.Update(time.Duration(mclock.Now() - connectedAt))
 | 
							connectionTimer.Update(time.Duration(mclock.Now() - connectedAt))
 | 
				
			||||||
	}()
 | 
						}()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -174,7 +175,7 @@ func (h *serverHandler) handle(p *peer) error {
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
// handleMsg is invoked whenever an inbound message is received from a remote
 | 
					// handleMsg is invoked whenever an inbound message is received from a remote
 | 
				
			||||||
// peer. The remote connection is torn down upon returning any error.
 | 
					// peer. The remote connection is torn down upon returning any error.
 | 
				
			||||||
func (h *serverHandler) handleMsg(p *peer, wg *sync.WaitGroup) error {
 | 
					func (h *serverHandler) handleMsg(p *clientPeer, wg *sync.WaitGroup) error {
 | 
				
			||||||
	// Read the next message from the remote peer, and ensure it's fully consumed
 | 
						// Read the next message from the remote peer, and ensure it's fully consumed
 | 
				
			||||||
	msg, err := p.rw.ReadMsg()
 | 
						msg, err := p.rw.ReadMsg()
 | 
				
			||||||
	if err != nil {
 | 
						if err != nil {
 | 
				
			||||||
@@ -208,7 +209,7 @@ func (h *serverHandler) handleMsg(p *peer, wg *sync.WaitGroup) error {
 | 
				
			|||||||
		maxCost = p.fcCosts.getMaxCost(msg.Code, reqCnt)
 | 
							maxCost = p.fcCosts.getMaxCost(msg.Code, reqCnt)
 | 
				
			||||||
		accepted, bufShort, priority := p.fcClient.AcceptRequest(reqID, responseCount, maxCost)
 | 
							accepted, bufShort, priority := p.fcClient.AcceptRequest(reqID, responseCount, maxCost)
 | 
				
			||||||
		if !accepted {
 | 
							if !accepted {
 | 
				
			||||||
			p.freezeClient()
 | 
								p.freeze()
 | 
				
			||||||
			p.Log().Error("Request came too early", "remaining", common.PrettyDuration(time.Duration(bufShort*1000000/p.fcParams.MinRecharge)))
 | 
								p.Log().Error("Request came too early", "remaining", common.PrettyDuration(time.Duration(bufShort*1000000/p.fcParams.MinRecharge)))
 | 
				
			||||||
			p.fcClient.OneTimeCost(inSizeCost)
 | 
								p.fcClient.OneTimeCost(inSizeCost)
 | 
				
			||||||
			return false
 | 
								return false
 | 
				
			||||||
@@ -258,7 +259,7 @@ func (h *serverHandler) handleMsg(p *peer, wg *sync.WaitGroup) error {
 | 
				
			|||||||
			h.server.clientPool.requestCost(p, realCost)
 | 
								h.server.clientPool.requestCost(p, realCost)
 | 
				
			||||||
		}
 | 
							}
 | 
				
			||||||
		if reply != nil {
 | 
							if reply != nil {
 | 
				
			||||||
			p.queueSend(func() {
 | 
								p.mustQueueSend(func() {
 | 
				
			||||||
				if err := reply.send(bv); err != nil {
 | 
									if err := reply.send(bv); err != nil {
 | 
				
			||||||
					select {
 | 
										select {
 | 
				
			||||||
					case p.errCh <- err:
 | 
										case p.errCh <- err:
 | 
				
			||||||
@@ -372,8 +373,8 @@ func (h *serverHandler) handleMsg(p *peer, wg *sync.WaitGroup) error {
 | 
				
			|||||||
					}
 | 
										}
 | 
				
			||||||
					first = false
 | 
										first = false
 | 
				
			||||||
				}
 | 
									}
 | 
				
			||||||
				reply := p.ReplyBlockHeaders(req.ReqID, headers)
 | 
									reply := p.replyBlockHeaders(req.ReqID, headers)
 | 
				
			||||||
				sendResponse(req.ReqID, query.Amount, p.ReplyBlockHeaders(req.ReqID, headers), task.done())
 | 
									sendResponse(req.ReqID, query.Amount, p.replyBlockHeaders(req.ReqID, headers), task.done())
 | 
				
			||||||
				if metrics.EnabledExpensive {
 | 
									if metrics.EnabledExpensive {
 | 
				
			||||||
					miscOutHeaderPacketsMeter.Mark(1)
 | 
										miscOutHeaderPacketsMeter.Mark(1)
 | 
				
			||||||
					miscOutHeaderTrafficMeter.Mark(int64(reply.size()))
 | 
										miscOutHeaderTrafficMeter.Mark(int64(reply.size()))
 | 
				
			||||||
@@ -421,7 +422,7 @@ func (h *serverHandler) handleMsg(p *peer, wg *sync.WaitGroup) error {
 | 
				
			|||||||
					bodies = append(bodies, body)
 | 
										bodies = append(bodies, body)
 | 
				
			||||||
					bytes += len(body)
 | 
										bytes += len(body)
 | 
				
			||||||
				}
 | 
									}
 | 
				
			||||||
				reply := p.ReplyBlockBodiesRLP(req.ReqID, bodies)
 | 
									reply := p.replyBlockBodiesRLP(req.ReqID, bodies)
 | 
				
			||||||
				sendResponse(req.ReqID, uint64(reqCnt), reply, task.done())
 | 
									sendResponse(req.ReqID, uint64(reqCnt), reply, task.done())
 | 
				
			||||||
				if metrics.EnabledExpensive {
 | 
									if metrics.EnabledExpensive {
 | 
				
			||||||
					miscOutBodyPacketsMeter.Mark(1)
 | 
										miscOutBodyPacketsMeter.Mark(1)
 | 
				
			||||||
@@ -493,7 +494,7 @@ func (h *serverHandler) handleMsg(p *peer, wg *sync.WaitGroup) error {
 | 
				
			|||||||
						break
 | 
											break
 | 
				
			||||||
					}
 | 
										}
 | 
				
			||||||
				}
 | 
									}
 | 
				
			||||||
				reply := p.ReplyCode(req.ReqID, data)
 | 
									reply := p.replyCode(req.ReqID, data)
 | 
				
			||||||
				sendResponse(req.ReqID, uint64(reqCnt), reply, task.done())
 | 
									sendResponse(req.ReqID, uint64(reqCnt), reply, task.done())
 | 
				
			||||||
				if metrics.EnabledExpensive {
 | 
									if metrics.EnabledExpensive {
 | 
				
			||||||
					miscOutCodePacketsMeter.Mark(1)
 | 
										miscOutCodePacketsMeter.Mark(1)
 | 
				
			||||||
@@ -550,7 +551,7 @@ func (h *serverHandler) handleMsg(p *peer, wg *sync.WaitGroup) error {
 | 
				
			|||||||
						bytes += len(encoded)
 | 
											bytes += len(encoded)
 | 
				
			||||||
					}
 | 
										}
 | 
				
			||||||
				}
 | 
									}
 | 
				
			||||||
				reply := p.ReplyReceiptsRLP(req.ReqID, receipts)
 | 
									reply := p.replyReceiptsRLP(req.ReqID, receipts)
 | 
				
			||||||
				sendResponse(req.ReqID, uint64(reqCnt), reply, task.done())
 | 
									sendResponse(req.ReqID, uint64(reqCnt), reply, task.done())
 | 
				
			||||||
				if metrics.EnabledExpensive {
 | 
									if metrics.EnabledExpensive {
 | 
				
			||||||
					miscOutReceiptPacketsMeter.Mark(1)
 | 
										miscOutReceiptPacketsMeter.Mark(1)
 | 
				
			||||||
@@ -653,7 +654,7 @@ func (h *serverHandler) handleMsg(p *peer, wg *sync.WaitGroup) error {
 | 
				
			|||||||
						break
 | 
											break
 | 
				
			||||||
					}
 | 
										}
 | 
				
			||||||
				}
 | 
									}
 | 
				
			||||||
				reply := p.ReplyProofsV2(req.ReqID, nodes.NodeList())
 | 
									reply := p.replyProofsV2(req.ReqID, nodes.NodeList())
 | 
				
			||||||
				sendResponse(req.ReqID, uint64(reqCnt), reply, task.done())
 | 
									sendResponse(req.ReqID, uint64(reqCnt), reply, task.done())
 | 
				
			||||||
				if metrics.EnabledExpensive {
 | 
									if metrics.EnabledExpensive {
 | 
				
			||||||
					miscOutTrieProofPacketsMeter.Mark(1)
 | 
										miscOutTrieProofPacketsMeter.Mark(1)
 | 
				
			||||||
@@ -728,7 +729,7 @@ func (h *serverHandler) handleMsg(p *peer, wg *sync.WaitGroup) error {
 | 
				
			|||||||
						break
 | 
											break
 | 
				
			||||||
					}
 | 
										}
 | 
				
			||||||
				}
 | 
									}
 | 
				
			||||||
				reply := p.ReplyHelperTrieProofs(req.ReqID, HelperTrieResps{Proofs: nodes.NodeList(), AuxData: auxData})
 | 
									reply := p.replyHelperTrieProofs(req.ReqID, HelperTrieResps{Proofs: nodes.NodeList(), AuxData: auxData})
 | 
				
			||||||
				sendResponse(req.ReqID, uint64(reqCnt), reply, task.done())
 | 
									sendResponse(req.ReqID, uint64(reqCnt), reply, task.done())
 | 
				
			||||||
				if metrics.EnabledExpensive {
 | 
									if metrics.EnabledExpensive {
 | 
				
			||||||
					miscOutHelperTriePacketsMeter.Mark(1)
 | 
										miscOutHelperTriePacketsMeter.Mark(1)
 | 
				
			||||||
@@ -777,7 +778,7 @@ func (h *serverHandler) handleMsg(p *peer, wg *sync.WaitGroup) error {
 | 
				
			|||||||
						stats[i] = h.txStatus(hash)
 | 
											stats[i] = h.txStatus(hash)
 | 
				
			||||||
					}
 | 
										}
 | 
				
			||||||
				}
 | 
									}
 | 
				
			||||||
				reply := p.ReplyTxStatus(req.ReqID, stats)
 | 
									reply := p.replyTxStatus(req.ReqID, stats)
 | 
				
			||||||
				sendResponse(req.ReqID, uint64(reqCnt), reply, task.done())
 | 
									sendResponse(req.ReqID, uint64(reqCnt), reply, task.done())
 | 
				
			||||||
				if metrics.EnabledExpensive {
 | 
									if metrics.EnabledExpensive {
 | 
				
			||||||
					miscOutTxsPacketsMeter.Mark(1)
 | 
										miscOutTxsPacketsMeter.Mark(1)
 | 
				
			||||||
@@ -814,7 +815,7 @@ func (h *serverHandler) handleMsg(p *peer, wg *sync.WaitGroup) error {
 | 
				
			|||||||
					}
 | 
										}
 | 
				
			||||||
					stats[i] = h.txStatus(hash)
 | 
										stats[i] = h.txStatus(hash)
 | 
				
			||||||
				}
 | 
									}
 | 
				
			||||||
				reply := p.ReplyTxStatus(req.ReqID, stats)
 | 
									reply := p.replyTxStatus(req.ReqID, stats)
 | 
				
			||||||
				sendResponse(req.ReqID, uint64(reqCnt), reply, task.done())
 | 
									sendResponse(req.ReqID, uint64(reqCnt), reply, task.done())
 | 
				
			||||||
				if metrics.EnabledExpensive {
 | 
									if metrics.EnabledExpensive {
 | 
				
			||||||
					miscOutTxStatusPacketsMeter.Mark(1)
 | 
										miscOutTxStatusPacketsMeter.Mark(1)
 | 
				
			||||||
@@ -913,7 +914,7 @@ func (h *serverHandler) broadcastHeaders() {
 | 
				
			|||||||
	for {
 | 
						for {
 | 
				
			||||||
		select {
 | 
							select {
 | 
				
			||||||
		case ev := <-headCh:
 | 
							case ev := <-headCh:
 | 
				
			||||||
			peers := h.server.peers.AllPeers()
 | 
								peers := h.server.peers.allPeers()
 | 
				
			||||||
			if len(peers) == 0 {
 | 
								if len(peers) == 0 {
 | 
				
			||||||
				continue
 | 
									continue
 | 
				
			||||||
			}
 | 
								}
 | 
				
			||||||
@@ -939,14 +940,18 @@ func (h *serverHandler) broadcastHeaders() {
 | 
				
			|||||||
				p := p
 | 
									p := p
 | 
				
			||||||
				switch p.announceType {
 | 
									switch p.announceType {
 | 
				
			||||||
				case announceTypeSimple:
 | 
									case announceTypeSimple:
 | 
				
			||||||
					p.queueSend(func() { p.SendAnnounce(announce) })
 | 
										if !p.queueSend(func() { p.sendAnnounce(announce) }) {
 | 
				
			||||||
 | 
											log.Debug("Drop announcement because queue is full", "number", number, "hash", hash)
 | 
				
			||||||
 | 
										}
 | 
				
			||||||
				case announceTypeSigned:
 | 
									case announceTypeSigned:
 | 
				
			||||||
					if !signed {
 | 
										if !signed {
 | 
				
			||||||
						signedAnnounce = announce
 | 
											signedAnnounce = announce
 | 
				
			||||||
						signedAnnounce.sign(h.server.privateKey)
 | 
											signedAnnounce.sign(h.server.privateKey)
 | 
				
			||||||
						signed = true
 | 
											signed = true
 | 
				
			||||||
					}
 | 
										}
 | 
				
			||||||
					p.queueSend(func() { p.SendAnnounce(signedAnnounce) })
 | 
										if !p.queueSend(func() { p.sendAnnounce(signedAnnounce) }) {
 | 
				
			||||||
 | 
											log.Debug("Drop announcement because queue is full", "number", number, "hash", hash)
 | 
				
			||||||
 | 
										}
 | 
				
			||||||
				}
 | 
									}
 | 
				
			||||||
			}
 | 
								}
 | 
				
			||||||
		case <-h.closeCh:
 | 
							case <-h.closeCh:
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -90,7 +90,7 @@ const (
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
// connReq represents a request for peer connection.
 | 
					// connReq represents a request for peer connection.
 | 
				
			||||||
type connReq struct {
 | 
					type connReq struct {
 | 
				
			||||||
	p      *peer
 | 
						p      *serverPeer
 | 
				
			||||||
	node   *enode.Node
 | 
						node   *enode.Node
 | 
				
			||||||
	result chan *poolEntry
 | 
						result chan *poolEntry
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
@@ -220,7 +220,7 @@ func (pool *serverPool) discoverNodes() {
 | 
				
			|||||||
// Otherwise, the connection should be rejected.
 | 
					// Otherwise, the connection should be rejected.
 | 
				
			||||||
// Note that whenever a connection has been accepted and a pool entry has been returned,
 | 
					// Note that whenever a connection has been accepted and a pool entry has been returned,
 | 
				
			||||||
// disconnect should also always be called.
 | 
					// disconnect should also always be called.
 | 
				
			||||||
func (pool *serverPool) connect(p *peer, node *enode.Node) *poolEntry {
 | 
					func (pool *serverPool) connect(p *serverPeer, node *enode.Node) *poolEntry {
 | 
				
			||||||
	log.Debug("Connect new entry", "enode", p.id)
 | 
						log.Debug("Connect new entry", "enode", p.id)
 | 
				
			||||||
	req := &connReq{p: p, node: node, result: make(chan *poolEntry, 1)}
 | 
						req := &connReq{p: p, node: node, result: make(chan *poolEntry, 1)}
 | 
				
			||||||
	select {
 | 
						select {
 | 
				
			||||||
@@ -679,7 +679,7 @@ const (
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
// poolEntry represents a server node and stores its current state and statistics.
 | 
					// poolEntry represents a server node and stores its current state and statistics.
 | 
				
			||||||
type poolEntry struct {
 | 
					type poolEntry struct {
 | 
				
			||||||
	peer                  *peer
 | 
						peer                  *serverPeer
 | 
				
			||||||
	pubkey                [64]byte // secp256k1 key of the node
 | 
						pubkey                [64]byte // secp256k1 key of the node
 | 
				
			||||||
	addr                  map[string]*poolEntryAddress
 | 
						addr                  map[string]*poolEntryAddress
 | 
				
			||||||
	node                  *enode.Node
 | 
						node                  *enode.Node
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -55,7 +55,7 @@ type servingQueue struct {
 | 
				
			|||||||
type servingTask struct {
 | 
					type servingTask struct {
 | 
				
			||||||
	sq                                       *servingQueue
 | 
						sq                                       *servingQueue
 | 
				
			||||||
	servingTime, timeAdded, maxTime, expTime uint64
 | 
						servingTime, timeAdded, maxTime, expTime uint64
 | 
				
			||||||
	peer                                     *peer
 | 
						peer                                     *clientPeer
 | 
				
			||||||
	priority                                 int64
 | 
						priority                                 int64
 | 
				
			||||||
	biasAdded                                bool
 | 
						biasAdded                                bool
 | 
				
			||||||
	token                                    runToken
 | 
						token                                    runToken
 | 
				
			||||||
@@ -142,7 +142,7 @@ func newServingQueue(suspendBias int64, utilTarget float64) *servingQueue {
 | 
				
			|||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// newTask creates a new task with the given priority
 | 
					// newTask creates a new task with the given priority
 | 
				
			||||||
func (sq *servingQueue) newTask(peer *peer, maxTime uint64, priority int64) *servingTask {
 | 
					func (sq *servingQueue) newTask(peer *clientPeer, maxTime uint64, priority int64) *servingTask {
 | 
				
			||||||
	return &servingTask{
 | 
						return &servingTask{
 | 
				
			||||||
		sq:       sq,
 | 
							sq:       sq,
 | 
				
			||||||
		peer:     peer,
 | 
							peer:     peer,
 | 
				
			||||||
@@ -187,7 +187,7 @@ func (sq *servingQueue) threadController() {
 | 
				
			|||||||
type (
 | 
					type (
 | 
				
			||||||
	// peerTasks lists the tasks received from a given peer when selecting peers to freeze
 | 
						// peerTasks lists the tasks received from a given peer when selecting peers to freeze
 | 
				
			||||||
	peerTasks struct {
 | 
						peerTasks struct {
 | 
				
			||||||
		peer     *peer
 | 
							peer     *clientPeer
 | 
				
			||||||
		list     []*servingTask
 | 
							list     []*servingTask
 | 
				
			||||||
		sumTime  uint64
 | 
							sumTime  uint64
 | 
				
			||||||
		priority float64
 | 
							priority float64
 | 
				
			||||||
@@ -211,7 +211,7 @@ func (l peerList) Swap(i, j int) {
 | 
				
			|||||||
// freezePeers selects the peers with the worst priority queued tasks and freezes
 | 
					// freezePeers selects the peers with the worst priority queued tasks and freezes
 | 
				
			||||||
// them until burstTime goes under burstDropLimit or all peers are frozen
 | 
					// them until burstTime goes under burstDropLimit or all peers are frozen
 | 
				
			||||||
func (sq *servingQueue) freezePeers() {
 | 
					func (sq *servingQueue) freezePeers() {
 | 
				
			||||||
	peerMap := make(map[*peer]*peerTasks)
 | 
						peerMap := make(map[*clientPeer]*peerTasks)
 | 
				
			||||||
	var peerList peerList
 | 
						var peerList peerList
 | 
				
			||||||
	if sq.best != nil {
 | 
						if sq.best != nil {
 | 
				
			||||||
		sq.queue.Push(sq.best, sq.best.priority)
 | 
							sq.queue.Push(sq.best, sq.best.priority)
 | 
				
			||||||
@@ -239,7 +239,7 @@ func (sq *servingQueue) freezePeers() {
 | 
				
			|||||||
	drop := true
 | 
						drop := true
 | 
				
			||||||
	for _, tasks := range peerList {
 | 
						for _, tasks := range peerList {
 | 
				
			||||||
		if drop {
 | 
							if drop {
 | 
				
			||||||
			tasks.peer.freezeClient()
 | 
								tasks.peer.freeze()
 | 
				
			||||||
			tasks.peer.fcClient.Freeze()
 | 
								tasks.peer.fcClient.Freeze()
 | 
				
			||||||
			sq.queuedTime -= tasks.sumTime
 | 
								sq.queuedTime -= tasks.sumTime
 | 
				
			||||||
			sqQueuedGauge.Update(int64(sq.queuedTime))
 | 
								sqQueuedGauge.Update(int64(sq.queuedTime))
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -51,7 +51,7 @@ const (
 | 
				
			|||||||
// In addition to the checkpoint registered in the registrar contract, there are
 | 
					// In addition to the checkpoint registered in the registrar contract, there are
 | 
				
			||||||
// several legacy hardcoded checkpoints in our codebase. These checkpoints are
 | 
					// several legacy hardcoded checkpoints in our codebase. These checkpoints are
 | 
				
			||||||
// also considered as valid.
 | 
					// also considered as valid.
 | 
				
			||||||
func (h *clientHandler) validateCheckpoint(peer *peer) error {
 | 
					func (h *clientHandler) validateCheckpoint(peer *serverPeer) error {
 | 
				
			||||||
	ctx, cancel := context.WithTimeout(context.Background(), time.Second*10)
 | 
						ctx, cancel := context.WithTimeout(context.Background(), time.Second*10)
 | 
				
			||||||
	defer cancel()
 | 
						defer cancel()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -87,7 +87,7 @@ func (h *clientHandler) validateCheckpoint(peer *peer) error {
 | 
				
			|||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// synchronise tries to sync up our local chain with a remote peer.
 | 
					// synchronise tries to sync up our local chain with a remote peer.
 | 
				
			||||||
func (h *clientHandler) synchronise(peer *peer) {
 | 
					func (h *clientHandler) synchronise(peer *serverPeer) {
 | 
				
			||||||
	// Short circuit if the peer is nil.
 | 
						// Short circuit if the peer is nil.
 | 
				
			||||||
	if peer == nil {
 | 
						if peer == nil {
 | 
				
			||||||
		return
 | 
							return
 | 
				
			||||||
@@ -95,7 +95,7 @@ func (h *clientHandler) synchronise(peer *peer) {
 | 
				
			|||||||
	// Make sure the peer's TD is higher than our own.
 | 
						// Make sure the peer's TD is higher than our own.
 | 
				
			||||||
	latest := h.backend.blockchain.CurrentHeader()
 | 
						latest := h.backend.blockchain.CurrentHeader()
 | 
				
			||||||
	currentTd := rawdb.ReadTd(h.backend.chainDb, latest.Hash(), latest.Number.Uint64())
 | 
						currentTd := rawdb.ReadTd(h.backend.chainDb, latest.Hash(), latest.Number.Uint64())
 | 
				
			||||||
	if currentTd != nil && peer.headBlockInfo().Td.Cmp(currentTd) < 0 {
 | 
						if currentTd != nil && peer.Td().Cmp(currentTd) < 0 {
 | 
				
			||||||
		return
 | 
							return
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	// Recap the checkpoint.
 | 
						// Recap the checkpoint.
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -109,7 +109,9 @@ func testCheckpointSyncing(t *testing.T, protocol int, syncMode int) {
 | 
				
			|||||||
	}
 | 
						}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	// Create connected peer pair.
 | 
						// Create connected peer pair.
 | 
				
			||||||
	_, err1, _, err2 := newTestPeerPair("peer", protocol, server.handler, client.handler)
 | 
						peer1, err1, peer2, err2 := newTestPeerPair("peer", protocol, server.handler, client.handler)
 | 
				
			||||||
 | 
						defer peer1.close()
 | 
				
			||||||
 | 
						defer peer2.close()
 | 
				
			||||||
	select {
 | 
						select {
 | 
				
			||||||
	case <-time.After(time.Millisecond * 100):
 | 
						case <-time.After(time.Millisecond * 100):
 | 
				
			||||||
	case err := <-err1:
 | 
						case err := <-err1:
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -166,7 +166,7 @@ func testIndexers(db ethdb.Database, odr light.OdrBackend, config *light.Indexer
 | 
				
			|||||||
	return indexers[:]
 | 
						return indexers[:]
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
func newTestClientHandler(backend *backends.SimulatedBackend, odr *LesOdr, indexers []*core.ChainIndexer, db ethdb.Database, peers *peerSet, ulcServers []string, ulcFraction int) *clientHandler {
 | 
					func newTestClientHandler(backend *backends.SimulatedBackend, odr *LesOdr, indexers []*core.ChainIndexer, db ethdb.Database, peers *serverPeerSet, ulcServers []string, ulcFraction int) *clientHandler {
 | 
				
			||||||
	var (
 | 
						var (
 | 
				
			||||||
		evmux  = new(event.TypeMux)
 | 
							evmux  = new(event.TypeMux)
 | 
				
			||||||
		engine = ethash.NewFaker()
 | 
							engine = ethash.NewFaker()
 | 
				
			||||||
@@ -206,9 +206,9 @@ func newTestClientHandler(backend *backends.SimulatedBackend, odr *LesOdr, index
 | 
				
			|||||||
			chainDb:     db,
 | 
								chainDb:     db,
 | 
				
			||||||
			oracle:      oracle,
 | 
								oracle:      oracle,
 | 
				
			||||||
			chainReader: chain,
 | 
								chainReader: chain,
 | 
				
			||||||
			peers:       peers,
 | 
					 | 
				
			||||||
			closeCh:     make(chan struct{}),
 | 
								closeCh:     make(chan struct{}),
 | 
				
			||||||
		},
 | 
							},
 | 
				
			||||||
 | 
							peers:      peers,
 | 
				
			||||||
		reqDist:    odr.retriever.dist,
 | 
							reqDist:    odr.retriever.dist,
 | 
				
			||||||
		retriever:  odr.retriever,
 | 
							retriever:  odr.retriever,
 | 
				
			||||||
		odr:        odr,
 | 
							odr:        odr,
 | 
				
			||||||
@@ -224,7 +224,7 @@ func newTestClientHandler(backend *backends.SimulatedBackend, odr *LesOdr, index
 | 
				
			|||||||
	return client.handler
 | 
						return client.handler
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
func newTestServerHandler(blocks int, indexers []*core.ChainIndexer, db ethdb.Database, peers *peerSet, clock mclock.Clock) (*serverHandler, *backends.SimulatedBackend) {
 | 
					func newTestServerHandler(blocks int, indexers []*core.ChainIndexer, db ethdb.Database, peers *clientPeerSet, clock mclock.Clock) (*serverHandler, *backends.SimulatedBackend) {
 | 
				
			||||||
	var (
 | 
						var (
 | 
				
			||||||
		gspec = core.Genesis{
 | 
							gspec = core.Genesis{
 | 
				
			||||||
			Config:   params.AllEthashProtocolChanges,
 | 
								Config:   params.AllEthashProtocolChanges,
 | 
				
			||||||
@@ -269,9 +269,9 @@ func newTestServerHandler(blocks int, indexers []*core.ChainIndexer, db ethdb.Da
 | 
				
			|||||||
			chainDb:     db,
 | 
								chainDb:     db,
 | 
				
			||||||
			chainReader: simulation.Blockchain(),
 | 
								chainReader: simulation.Blockchain(),
 | 
				
			||||||
			oracle:      oracle,
 | 
								oracle:      oracle,
 | 
				
			||||||
			peers:       peers,
 | 
					 | 
				
			||||||
			closeCh:     make(chan struct{}),
 | 
								closeCh:     make(chan struct{}),
 | 
				
			||||||
		},
 | 
							},
 | 
				
			||||||
 | 
							peers:        peers,
 | 
				
			||||||
		servingQueue: newServingQueue(int64(time.Millisecond*10), 1),
 | 
							servingQueue: newServingQueue(int64(time.Millisecond*10), 1),
 | 
				
			||||||
		defParams: flowcontrol.ServerParams{
 | 
							defParams: flowcontrol.ServerParams{
 | 
				
			||||||
			BufLimit:    testBufLimit,
 | 
								BufLimit:    testBufLimit,
 | 
				
			||||||
@@ -294,7 +294,8 @@ func newTestServerHandler(blocks int, indexers []*core.ChainIndexer, db ethdb.Da
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
// testPeer is a simulated peer to allow testing direct network calls.
 | 
					// testPeer is a simulated peer to allow testing direct network calls.
 | 
				
			||||||
type testPeer struct {
 | 
					type testPeer struct {
 | 
				
			||||||
	peer *peer
 | 
						cpeer *clientPeer
 | 
				
			||||||
 | 
						speer *serverPeer
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	net p2p.MsgReadWriter // Network layer reader/writer to simulate remote messaging
 | 
						net p2p.MsgReadWriter // Network layer reader/writer to simulate remote messaging
 | 
				
			||||||
	app *p2p.MsgPipeRW    // Application layer reader/writer to simulate the local side
 | 
						app *p2p.MsgPipeRW    // Application layer reader/writer to simulate the local side
 | 
				
			||||||
@@ -308,7 +309,7 @@ func newTestPeer(t *testing.T, name string, version int, handler *serverHandler,
 | 
				
			|||||||
	// Generate a random id and create the peer
 | 
						// Generate a random id and create the peer
 | 
				
			||||||
	var id enode.ID
 | 
						var id enode.ID
 | 
				
			||||||
	rand.Read(id[:])
 | 
						rand.Read(id[:])
 | 
				
			||||||
	peer := newPeer(version, NetworkId, false, p2p.NewPeer(id, name, nil), net)
 | 
						peer := newClientPeer(version, NetworkId, p2p.NewPeer(id, name, nil), net)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	// Start the peer on a new thread
 | 
						// Start the peer on a new thread
 | 
				
			||||||
	errCh := make(chan error, 1)
 | 
						errCh := make(chan error, 1)
 | 
				
			||||||
@@ -322,7 +323,7 @@ func newTestPeer(t *testing.T, name string, version int, handler *serverHandler,
 | 
				
			|||||||
	tp := &testPeer{
 | 
						tp := &testPeer{
 | 
				
			||||||
		app:   app,
 | 
							app:   app,
 | 
				
			||||||
		net:   net,
 | 
							net:   net,
 | 
				
			||||||
		peer: peer,
 | 
							cpeer: peer,
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	// Execute any implicitly requested handshakes and return
 | 
						// Execute any implicitly requested handshakes and return
 | 
				
			||||||
	if shake {
 | 
						if shake {
 | 
				
			||||||
@@ -354,8 +355,8 @@ func newTestPeerPair(name string, version int, server *serverHandler, client *cl
 | 
				
			|||||||
	var id enode.ID
 | 
						var id enode.ID
 | 
				
			||||||
	rand.Read(id[:])
 | 
						rand.Read(id[:])
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	peer1 := newPeer(version, NetworkId, false, p2p.NewPeer(id, name, nil), net)
 | 
						peer1 := newClientPeer(version, NetworkId, p2p.NewPeer(id, name, nil), net)
 | 
				
			||||||
	peer2 := newPeer(version, NetworkId, false, p2p.NewPeer(id, name, nil), app)
 | 
						peer2 := newServerPeer(version, NetworkId, false, p2p.NewPeer(id, name, nil), app)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	// Start the peer on a new thread
 | 
						// Start the peer on a new thread
 | 
				
			||||||
	errc1 := make(chan error, 1)
 | 
						errc1 := make(chan error, 1)
 | 
				
			||||||
@@ -374,14 +375,14 @@ func newTestPeerPair(name string, version int, server *serverHandler, client *cl
 | 
				
			|||||||
		case errc1 <- client.handle(peer2):
 | 
							case errc1 <- client.handle(peer2):
 | 
				
			||||||
		}
 | 
							}
 | 
				
			||||||
	}()
 | 
						}()
 | 
				
			||||||
	return &testPeer{peer: peer1, net: net, app: app}, errc1, &testPeer{peer: peer2, net: app, app: net}, errc2
 | 
						return &testPeer{cpeer: peer1, net: net, app: app}, errc1, &testPeer{speer: peer2, net: app, app: net}, errc2
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// handshake simulates a trivial handshake that expects the same state from the
 | 
					// handshake simulates a trivial handshake that expects the same state from the
 | 
				
			||||||
// remote side as we are simulating locally.
 | 
					// remote side as we are simulating locally.
 | 
				
			||||||
func (p *testPeer) handshake(t *testing.T, td *big.Int, head common.Hash, headNum uint64, genesis common.Hash, costList RequestCostList) {
 | 
					func (p *testPeer) handshake(t *testing.T, td *big.Int, head common.Hash, headNum uint64, genesis common.Hash, costList RequestCostList) {
 | 
				
			||||||
	var expList keyValueList
 | 
						var expList keyValueList
 | 
				
			||||||
	expList = expList.add("protocolVersion", uint64(p.peer.version))
 | 
						expList = expList.add("protocolVersion", uint64(p.cpeer.version))
 | 
				
			||||||
	expList = expList.add("networkId", uint64(NetworkId))
 | 
						expList = expList.add("networkId", uint64(NetworkId))
 | 
				
			||||||
	expList = expList.add("headTd", td)
 | 
						expList = expList.add("headTd", td)
 | 
				
			||||||
	expList = expList.add("headHash", head)
 | 
						expList = expList.add("headHash", head)
 | 
				
			||||||
@@ -404,7 +405,7 @@ func (p *testPeer) handshake(t *testing.T, td *big.Int, head common.Hash, headNu
 | 
				
			|||||||
	if err := p2p.Send(p.app, StatusMsg, sendList); err != nil {
 | 
						if err := p2p.Send(p.app, StatusMsg, sendList); err != nil {
 | 
				
			||||||
		t.Fatalf("status send: %v", err)
 | 
							t.Fatalf("status send: %v", err)
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	p.peer.fcParams = flowcontrol.ServerParams{
 | 
						p.cpeer.fcParams = flowcontrol.ServerParams{
 | 
				
			||||||
		BufLimit:    testBufLimit,
 | 
							BufLimit:    testBufLimit,
 | 
				
			||||||
		MinRecharge: testBufRecharge,
 | 
							MinRecharge: testBufRecharge,
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
@@ -445,7 +446,7 @@ func newServerEnv(t *testing.T, blocks int, protocol int, callback indexerCallba
 | 
				
			|||||||
	if simClock {
 | 
						if simClock {
 | 
				
			||||||
		clock = &mclock.Simulated{}
 | 
							clock = &mclock.Simulated{}
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	handler, b := newTestServerHandler(blocks, indexers, db, newPeerSet(), clock)
 | 
						handler, b := newTestServerHandler(blocks, indexers, db, newClientPeerSet(), clock)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	var peer *testPeer
 | 
						var peer *testPeer
 | 
				
			||||||
	if newPeer {
 | 
						if newPeer {
 | 
				
			||||||
@@ -473,6 +474,7 @@ func newServerEnv(t *testing.T, blocks int, protocol int, callback indexerCallba
 | 
				
			|||||||
	teardown := func() {
 | 
						teardown := func() {
 | 
				
			||||||
		if newPeer {
 | 
							if newPeer {
 | 
				
			||||||
			peer.close()
 | 
								peer.close()
 | 
				
			||||||
 | 
								peer.cpeer.close()
 | 
				
			||||||
			b.Close()
 | 
								b.Close()
 | 
				
			||||||
		}
 | 
							}
 | 
				
			||||||
		cIndexer.Close()
 | 
							cIndexer.Close()
 | 
				
			||||||
@@ -483,14 +485,14 @@ func newServerEnv(t *testing.T, blocks int, protocol int, callback indexerCallba
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
func newClientServerEnv(t *testing.T, blocks int, protocol int, callback indexerCallback, ulcServers []string, ulcFraction int, simClock bool, connect bool) (*testServer, *testClient, func()) {
 | 
					func newClientServerEnv(t *testing.T, blocks int, protocol int, callback indexerCallback, ulcServers []string, ulcFraction int, simClock bool, connect bool) (*testServer, *testClient, func()) {
 | 
				
			||||||
	sdb, cdb := rawdb.NewMemoryDatabase(), rawdb.NewMemoryDatabase()
 | 
						sdb, cdb := rawdb.NewMemoryDatabase(), rawdb.NewMemoryDatabase()
 | 
				
			||||||
	speers, cPeers := newPeerSet(), newPeerSet()
 | 
						speers, cpeers := newServerPeerSet(), newClientPeerSet()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	var clock mclock.Clock = &mclock.System{}
 | 
						var clock mclock.Clock = &mclock.System{}
 | 
				
			||||||
	if simClock {
 | 
						if simClock {
 | 
				
			||||||
		clock = &mclock.Simulated{}
 | 
							clock = &mclock.Simulated{}
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	dist := newRequestDistributor(cPeers, clock)
 | 
						dist := newRequestDistributor(speers, clock)
 | 
				
			||||||
	rm := newRetrieveManager(cPeers, dist, nil)
 | 
						rm := newRetrieveManager(speers, dist, nil)
 | 
				
			||||||
	odr := NewLesOdr(cdb, light.TestClientIndexerConfig, rm)
 | 
						odr := NewLesOdr(cdb, light.TestClientIndexerConfig, rm)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	sindexers := testIndexers(sdb, nil, light.TestServerIndexerConfig)
 | 
						sindexers := testIndexers(sdb, nil, light.TestServerIndexerConfig)
 | 
				
			||||||
@@ -500,8 +502,8 @@ func newClientServerEnv(t *testing.T, blocks int, protocol int, callback indexer
 | 
				
			|||||||
	ccIndexer, cbIndexer, cbtIndexer := cIndexers[0], cIndexers[1], cIndexers[2]
 | 
						ccIndexer, cbIndexer, cbtIndexer := cIndexers[0], cIndexers[1], cIndexers[2]
 | 
				
			||||||
	odr.SetIndexers(ccIndexer, cbIndexer, cbtIndexer)
 | 
						odr.SetIndexers(ccIndexer, cbIndexer, cbtIndexer)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	server, b := newTestServerHandler(blocks, sindexers, sdb, speers, clock)
 | 
						server, b := newTestServerHandler(blocks, sindexers, sdb, cpeers, clock)
 | 
				
			||||||
	client := newTestClientHandler(b, odr, cIndexers, cdb, cPeers, ulcServers, ulcFraction)
 | 
						client := newTestClientHandler(b, odr, cIndexers, cdb, speers, ulcServers, ulcFraction)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	scIndexer.Start(server.blockchain)
 | 
						scIndexer.Start(server.blockchain)
 | 
				
			||||||
	sbIndexer.Start(server.blockchain)
 | 
						sbIndexer.Start(server.blockchain)
 | 
				
			||||||
@@ -548,6 +550,8 @@ func newClientServerEnv(t *testing.T, blocks int, protocol int, callback indexer
 | 
				
			|||||||
		if connect {
 | 
							if connect {
 | 
				
			||||||
			speer.close()
 | 
								speer.close()
 | 
				
			||||||
			cpeer.close()
 | 
								cpeer.close()
 | 
				
			||||||
 | 
								cpeer.cpeer.close()
 | 
				
			||||||
 | 
								speer.speer.close()
 | 
				
			||||||
		}
 | 
							}
 | 
				
			||||||
		ccIndexer.Close()
 | 
							ccIndexer.Close()
 | 
				
			||||||
		cbIndexer.Close()
 | 
							cbIndexer.Close()
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -27,14 +27,13 @@ import (
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
type ltrInfo struct {
 | 
					type ltrInfo struct {
 | 
				
			||||||
	tx     *types.Transaction
 | 
						tx     *types.Transaction
 | 
				
			||||||
	sentTo map[*peer]struct{}
 | 
						sentTo map[*serverPeer]struct{}
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
type lesTxRelay struct {
 | 
					type lesTxRelay struct {
 | 
				
			||||||
	txSent       map[common.Hash]*ltrInfo
 | 
						txSent       map[common.Hash]*ltrInfo
 | 
				
			||||||
	txPending    map[common.Hash]struct{}
 | 
						txPending    map[common.Hash]struct{}
 | 
				
			||||||
	ps           *peerSet
 | 
						peerList     []*serverPeer
 | 
				
			||||||
	peerList     []*peer
 | 
					 | 
				
			||||||
	peerStartPos int
 | 
						peerStartPos int
 | 
				
			||||||
	lock         sync.RWMutex
 | 
						lock         sync.RWMutex
 | 
				
			||||||
	stop         chan struct{}
 | 
						stop         chan struct{}
 | 
				
			||||||
@@ -42,15 +41,14 @@ type lesTxRelay struct {
 | 
				
			|||||||
	retriever *retrieveManager
 | 
						retriever *retrieveManager
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
func newLesTxRelay(ps *peerSet, retriever *retrieveManager) *lesTxRelay {
 | 
					func newLesTxRelay(ps *serverPeerSet, retriever *retrieveManager) *lesTxRelay {
 | 
				
			||||||
	r := &lesTxRelay{
 | 
						r := &lesTxRelay{
 | 
				
			||||||
		txSent:    make(map[common.Hash]*ltrInfo),
 | 
							txSent:    make(map[common.Hash]*ltrInfo),
 | 
				
			||||||
		txPending: make(map[common.Hash]struct{}),
 | 
							txPending: make(map[common.Hash]struct{}),
 | 
				
			||||||
		ps:        ps,
 | 
					 | 
				
			||||||
		retriever: retriever,
 | 
							retriever: retriever,
 | 
				
			||||||
		stop:      make(chan struct{}),
 | 
							stop:      make(chan struct{}),
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
	ps.notify(r)
 | 
						ps.subscribe(r)
 | 
				
			||||||
	return r
 | 
						return r
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -58,24 +56,34 @@ func (ltrx *lesTxRelay) Stop() {
 | 
				
			|||||||
	close(ltrx.stop)
 | 
						close(ltrx.stop)
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
func (ltrx *lesTxRelay) registerPeer(p *peer) {
 | 
					func (ltrx *lesTxRelay) registerPeer(p *serverPeer) {
 | 
				
			||||||
	ltrx.lock.Lock()
 | 
						ltrx.lock.Lock()
 | 
				
			||||||
	defer ltrx.lock.Unlock()
 | 
						defer ltrx.lock.Unlock()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	ltrx.peerList = ltrx.ps.AllPeers()
 | 
						// Short circuit if the peer is announce only.
 | 
				
			||||||
 | 
						if p.onlyAnnounce {
 | 
				
			||||||
 | 
							return
 | 
				
			||||||
 | 
						}
 | 
				
			||||||
 | 
						ltrx.peerList = append(ltrx.peerList, p)
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
func (ltrx *lesTxRelay) unregisterPeer(p *peer) {
 | 
					func (ltrx *lesTxRelay) unregisterPeer(p *serverPeer) {
 | 
				
			||||||
	ltrx.lock.Lock()
 | 
						ltrx.lock.Lock()
 | 
				
			||||||
	defer ltrx.lock.Unlock()
 | 
						defer ltrx.lock.Unlock()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	ltrx.peerList = ltrx.ps.AllPeers()
 | 
						for i, peer := range ltrx.peerList {
 | 
				
			||||||
 | 
							if peer == p {
 | 
				
			||||||
 | 
								// Remove from the peer list
 | 
				
			||||||
 | 
								ltrx.peerList = append(ltrx.peerList[:i], ltrx.peerList[i+1:]...)
 | 
				
			||||||
 | 
								return
 | 
				
			||||||
 | 
							}
 | 
				
			||||||
 | 
						}
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// send sends a list of transactions to at most a given number of peers at
 | 
					// send sends a list of transactions to at most a given number of peers at
 | 
				
			||||||
// once, never resending any particular transaction to the same peer twice
 | 
					// once, never resending any particular transaction to the same peer twice
 | 
				
			||||||
func (ltrx *lesTxRelay) send(txs types.Transactions, count int) {
 | 
					func (ltrx *lesTxRelay) send(txs types.Transactions, count int) {
 | 
				
			||||||
	sendTo := make(map[*peer]types.Transactions)
 | 
						sendTo := make(map[*serverPeer]types.Transactions)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	ltrx.peerStartPos++ // rotate the starting position of the peer list
 | 
						ltrx.peerStartPos++ // rotate the starting position of the peer list
 | 
				
			||||||
	if ltrx.peerStartPos >= len(ltrx.peerList) {
 | 
						if ltrx.peerStartPos >= len(ltrx.peerList) {
 | 
				
			||||||
@@ -88,7 +96,7 @@ func (ltrx *lesTxRelay) send(txs types.Transactions, count int) {
 | 
				
			|||||||
		if !ok {
 | 
							if !ok {
 | 
				
			||||||
			ltr = <rInfo{
 | 
								ltr = <rInfo{
 | 
				
			||||||
				tx:     tx,
 | 
									tx:     tx,
 | 
				
			||||||
				sentTo: make(map[*peer]struct{}),
 | 
									sentTo: make(map[*serverPeer]struct{}),
 | 
				
			||||||
			}
 | 
								}
 | 
				
			||||||
			ltrx.txSent[hash] = ltr
 | 
								ltrx.txSent[hash] = ltr
 | 
				
			||||||
			ltrx.txPending[hash] = struct{}{}
 | 
								ltrx.txPending[hash] = struct{}{}
 | 
				
			||||||
@@ -126,17 +134,17 @@ func (ltrx *lesTxRelay) send(txs types.Transactions, count int) {
 | 
				
			|||||||
		reqID := genReqID()
 | 
							reqID := genReqID()
 | 
				
			||||||
		rq := &distReq{
 | 
							rq := &distReq{
 | 
				
			||||||
			getCost: func(dp distPeer) uint64 {
 | 
								getCost: func(dp distPeer) uint64 {
 | 
				
			||||||
				peer := dp.(*peer)
 | 
									peer := dp.(*serverPeer)
 | 
				
			||||||
				return peer.GetTxRelayCost(len(ll), len(enc))
 | 
									return peer.getTxRelayCost(len(ll), len(enc))
 | 
				
			||||||
			},
 | 
								},
 | 
				
			||||||
			canSend: func(dp distPeer) bool {
 | 
								canSend: func(dp distPeer) bool {
 | 
				
			||||||
				return !dp.(*peer).onlyAnnounce && dp.(*peer) == pp
 | 
									return !dp.(*serverPeer).onlyAnnounce && dp.(*serverPeer) == pp
 | 
				
			||||||
			},
 | 
								},
 | 
				
			||||||
			request: func(dp distPeer) func() {
 | 
								request: func(dp distPeer) func() {
 | 
				
			||||||
				peer := dp.(*peer)
 | 
									peer := dp.(*serverPeer)
 | 
				
			||||||
				cost := peer.GetTxRelayCost(len(ll), len(enc))
 | 
									cost := peer.getTxRelayCost(len(ll), len(enc))
 | 
				
			||||||
				peer.fcServer.QueuedRequest(reqID, cost)
 | 
									peer.fcServer.QueuedRequest(reqID, cost)
 | 
				
			||||||
				return func() { peer.SendTxs(reqID, cost, enc) }
 | 
									return func() { peer.sendTxs(reqID, enc) }
 | 
				
			||||||
			},
 | 
								},
 | 
				
			||||||
		}
 | 
							}
 | 
				
			||||||
		go ltrx.retriever.retrieve(context.Background(), reqID, rq, func(p distPeer, msg *Msg) error { return nil }, ltrx.stop)
 | 
							go ltrx.retriever.retrieve(context.Background(), reqID, rq, func(p distPeer, msg *Msg) error { return nil }, ltrx.stop)
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -54,14 +54,14 @@ func testULCAnnounceThreshold(t *testing.T, protocol int) {
 | 
				
			|||||||
			ids       []string
 | 
								ids       []string
 | 
				
			||||||
		)
 | 
							)
 | 
				
			||||||
		for i := 0; i < len(testcase.height); i++ {
 | 
							for i := 0; i < len(testcase.height); i++ {
 | 
				
			||||||
			s, n, teardown := newServerPeer(t, 0, protocol)
 | 
								s, n, teardown := newTestServerPeer(t, 0, protocol)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
			servers = append(servers, s)
 | 
								servers = append(servers, s)
 | 
				
			||||||
			nodes = append(nodes, n)
 | 
								nodes = append(nodes, n)
 | 
				
			||||||
			teardowns = append(teardowns, teardown)
 | 
								teardowns = append(teardowns, teardown)
 | 
				
			||||||
			ids = append(ids, n.String())
 | 
								ids = append(ids, n.String())
 | 
				
			||||||
		}
 | 
							}
 | 
				
			||||||
		c, teardown := newLightPeer(t, protocol, ids, testcase.threshold)
 | 
							c, teardown := newTestLightPeer(t, protocol, ids, testcase.threshold)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
		// Connect all servers.
 | 
							// Connect all servers.
 | 
				
			||||||
		for i := 0; i < len(servers); i++ {
 | 
							for i := 0; i < len(servers); i++ {
 | 
				
			||||||
@@ -86,15 +86,15 @@ func testULCAnnounceThreshold(t *testing.T, protocol int) {
 | 
				
			|||||||
	}
 | 
						}
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
func connect(server *serverHandler, serverId enode.ID, client *clientHandler, protocol int) (*peer, *peer, error) {
 | 
					func connect(server *serverHandler, serverId enode.ID, client *clientHandler, protocol int) (*serverPeer, *clientPeer, error) {
 | 
				
			||||||
	// Create a message pipe to communicate through
 | 
						// Create a message pipe to communicate through
 | 
				
			||||||
	app, net := p2p.MsgPipe()
 | 
						app, net := p2p.MsgPipe()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	var id enode.ID
 | 
						var id enode.ID
 | 
				
			||||||
	rand.Read(id[:])
 | 
						rand.Read(id[:])
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	peer1 := newPeer(protocol, NetworkId, true, p2p.NewPeer(serverId, "", nil), net) // Mark server as trusted
 | 
						peer1 := newServerPeer(protocol, NetworkId, true, p2p.NewPeer(serverId, "", nil), net) // Mark server as trusted
 | 
				
			||||||
	peer2 := newPeer(protocol, NetworkId, false, p2p.NewPeer(id, "", nil), app)
 | 
						peer2 := newClientPeer(protocol, NetworkId, p2p.NewPeer(id, "", nil), app)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	// Start the peerLight on a new thread
 | 
						// Start the peerLight on a new thread
 | 
				
			||||||
	errc1 := make(chan error, 1)
 | 
						errc1 := make(chan error, 1)
 | 
				
			||||||
@@ -124,8 +124,8 @@ func connect(server *serverHandler, serverId enode.ID, client *clientHandler, pr
 | 
				
			|||||||
	return peer1, peer2, nil
 | 
						return peer1, peer2, nil
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// newServerPeer creates server peer.
 | 
					// newTestServerPeer creates server peer.
 | 
				
			||||||
func newServerPeer(t *testing.T, blocks int, protocol int) (*testServer, *enode.Node, func()) {
 | 
					func newTestServerPeer(t *testing.T, blocks int, protocol int) (*testServer, *enode.Node, func()) {
 | 
				
			||||||
	s, teardown := newServerEnv(t, blocks, protocol, nil, false, false, 0)
 | 
						s, teardown := newServerEnv(t, blocks, protocol, nil, false, false, 0)
 | 
				
			||||||
	key, err := crypto.GenerateKey()
 | 
						key, err := crypto.GenerateKey()
 | 
				
			||||||
	if err != nil {
 | 
						if err != nil {
 | 
				
			||||||
@@ -136,8 +136,8 @@ func newServerPeer(t *testing.T, blocks int, protocol int) (*testServer, *enode.
 | 
				
			|||||||
	return s, n, teardown
 | 
						return s, n, teardown
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// newLightPeer creates node with light sync mode
 | 
					// newTestLightPeer creates node with light sync mode
 | 
				
			||||||
func newLightPeer(t *testing.T, protocol int, ulcServers []string, ulcFraction int) (*testClient, func()) {
 | 
					func newTestLightPeer(t *testing.T, protocol int, ulcServers []string, ulcFraction int) (*testClient, func()) {
 | 
				
			||||||
	_, c, teardown := newClientServerEnv(t, 0, protocol, nil, ulcServers, ulcFraction, false, false)
 | 
						_, c, teardown := newClientServerEnv(t, 0, protocol, nil, ulcServers, ulcFraction, false, false)
 | 
				
			||||||
	return c, teardown
 | 
						return c, teardown
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 
 | 
				
			|||||||
		Reference in New Issue
	
	Block a user