eth: request id dispatcher and direct req/reply APIs (#23576)
* eth: request ID based message dispatcher * eth: fix dispatcher cancellation, rework fetchers idleness tracker * eth/downloader: drop peers who refuse to serve advertised chains
This commit is contained in:
@ -22,9 +22,7 @@ package downloader
|
||||
import (
|
||||
"errors"
|
||||
"math/big"
|
||||
"sort"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
@ -39,7 +37,6 @@ const (
|
||||
)
|
||||
|
||||
var (
|
||||
errAlreadyFetching = errors.New("already fetching blocks from peer")
|
||||
errAlreadyRegistered = errors.New("peer is already registered")
|
||||
errNotRegistered = errors.New("peer is not registered")
|
||||
)
|
||||
@ -48,16 +45,6 @@ var (
|
||||
type peerConnection struct {
|
||||
id string // Unique identifier of the peer
|
||||
|
||||
headerIdle int32 // Current header activity state of the peer (idle = 0, active = 1)
|
||||
blockIdle int32 // Current block activity state of the peer (idle = 0, active = 1)
|
||||
receiptIdle int32 // Current receipt activity state of the peer (idle = 0, active = 1)
|
||||
stateIdle int32 // Current node data activity state of the peer (idle = 0, active = 1)
|
||||
|
||||
headerStarted time.Time // Time instance when the last header fetch was started
|
||||
blockStarted time.Time // Time instance when the last block (body) fetch was started
|
||||
receiptStarted time.Time // Time instance when the last receipt fetch was started
|
||||
stateStarted time.Time // Time instance when the last node data fetch was started
|
||||
|
||||
rates *msgrate.Tracker // Tracker to hone in on the number of items retrievable per second
|
||||
lacking map[common.Hash]struct{} // Set of hashes not to request (didn't have previously)
|
||||
|
||||
@ -71,16 +58,15 @@ type peerConnection struct {
|
||||
// LightPeer encapsulates the methods required to synchronise with a remote light peer.
|
||||
type LightPeer interface {
|
||||
Head() (common.Hash, *big.Int)
|
||||
RequestHeadersByHash(common.Hash, int, int, bool) error
|
||||
RequestHeadersByNumber(uint64, int, int, bool) error
|
||||
RequestHeadersByHash(common.Hash, int, int, bool, chan *eth.Response) (*eth.Request, error)
|
||||
RequestHeadersByNumber(uint64, int, int, bool, chan *eth.Response) (*eth.Request, error)
|
||||
}
|
||||
|
||||
// Peer encapsulates the methods required to synchronise with a remote full peer.
|
||||
type Peer interface {
|
||||
LightPeer
|
||||
RequestBodies([]common.Hash) error
|
||||
RequestReceipts([]common.Hash) error
|
||||
RequestNodeData([]common.Hash) error
|
||||
RequestBodies([]common.Hash, chan *eth.Response) (*eth.Request, error)
|
||||
RequestReceipts([]common.Hash, chan *eth.Response) (*eth.Request, error)
|
||||
}
|
||||
|
||||
// lightPeerWrapper wraps a LightPeer struct, stubbing out the Peer-only methods.
|
||||
@ -89,21 +75,18 @@ type lightPeerWrapper struct {
|
||||
}
|
||||
|
||||
func (w *lightPeerWrapper) Head() (common.Hash, *big.Int) { return w.peer.Head() }
|
||||
func (w *lightPeerWrapper) RequestHeadersByHash(h common.Hash, amount int, skip int, reverse bool) error {
|
||||
return w.peer.RequestHeadersByHash(h, amount, skip, reverse)
|
||||
func (w *lightPeerWrapper) RequestHeadersByHash(h common.Hash, amount int, skip int, reverse bool, sink chan *eth.Response) (*eth.Request, error) {
|
||||
return w.peer.RequestHeadersByHash(h, amount, skip, reverse, sink)
|
||||
}
|
||||
func (w *lightPeerWrapper) RequestHeadersByNumber(i uint64, amount int, skip int, reverse bool) error {
|
||||
return w.peer.RequestHeadersByNumber(i, amount, skip, reverse)
|
||||
func (w *lightPeerWrapper) RequestHeadersByNumber(i uint64, amount int, skip int, reverse bool, sink chan *eth.Response) (*eth.Request, error) {
|
||||
return w.peer.RequestHeadersByNumber(i, amount, skip, reverse, sink)
|
||||
}
|
||||
func (w *lightPeerWrapper) RequestBodies([]common.Hash) error {
|
||||
func (w *lightPeerWrapper) RequestBodies([]common.Hash, chan *eth.Response) (*eth.Request, error) {
|
||||
panic("RequestBodies not supported in light client mode sync")
|
||||
}
|
||||
func (w *lightPeerWrapper) RequestReceipts([]common.Hash) error {
|
||||
func (w *lightPeerWrapper) RequestReceipts([]common.Hash, chan *eth.Response) (*eth.Request, error) {
|
||||
panic("RequestReceipts not supported in light client mode sync")
|
||||
}
|
||||
func (w *lightPeerWrapper) RequestNodeData([]common.Hash) error {
|
||||
panic("RequestNodeData not supported in light client mode sync")
|
||||
}
|
||||
|
||||
// newPeerConnection creates a new downloader peer.
|
||||
func newPeerConnection(id string, version uint, peer Peer, logger log.Logger) *peerConnection {
|
||||
@ -121,114 +104,28 @@ func (p *peerConnection) Reset() {
|
||||
p.lock.Lock()
|
||||
defer p.lock.Unlock()
|
||||
|
||||
atomic.StoreInt32(&p.headerIdle, 0)
|
||||
atomic.StoreInt32(&p.blockIdle, 0)
|
||||
atomic.StoreInt32(&p.receiptIdle, 0)
|
||||
atomic.StoreInt32(&p.stateIdle, 0)
|
||||
|
||||
p.lacking = make(map[common.Hash]struct{})
|
||||
}
|
||||
|
||||
// FetchHeaders sends a header retrieval request to the remote peer.
|
||||
func (p *peerConnection) FetchHeaders(from uint64, count int) error {
|
||||
// Short circuit if the peer is already fetching
|
||||
if !atomic.CompareAndSwapInt32(&p.headerIdle, 0, 1) {
|
||||
return errAlreadyFetching
|
||||
}
|
||||
p.headerStarted = time.Now()
|
||||
|
||||
// Issue the header retrieval request (absolute upwards without gaps)
|
||||
go p.peer.RequestHeadersByNumber(from, count, 0, false)
|
||||
|
||||
return nil
|
||||
// UpdateHeaderRate updates the peer's estimated header retrieval throughput with
|
||||
// the current measurement.
|
||||
func (p *peerConnection) UpdateHeaderRate(delivered int, elapsed time.Duration) {
|
||||
p.rates.Update(eth.BlockHeadersMsg, elapsed, delivered)
|
||||
}
|
||||
|
||||
// FetchBodies sends a block body retrieval request to the remote peer.
|
||||
func (p *peerConnection) FetchBodies(request *fetchRequest) error {
|
||||
// Short circuit if the peer is already fetching
|
||||
if !atomic.CompareAndSwapInt32(&p.blockIdle, 0, 1) {
|
||||
return errAlreadyFetching
|
||||
}
|
||||
p.blockStarted = time.Now()
|
||||
|
||||
go func() {
|
||||
// Convert the header set to a retrievable slice
|
||||
hashes := make([]common.Hash, 0, len(request.Headers))
|
||||
for _, header := range request.Headers {
|
||||
hashes = append(hashes, header.Hash())
|
||||
}
|
||||
p.peer.RequestBodies(hashes)
|
||||
}()
|
||||
|
||||
return nil
|
||||
// UpdateBodyRate updates the peer's estimated body retrieval throughput with the
|
||||
// current measurement.
|
||||
func (p *peerConnection) UpdateBodyRate(delivered int, elapsed time.Duration) {
|
||||
p.rates.Update(eth.BlockBodiesMsg, elapsed, delivered)
|
||||
}
|
||||
|
||||
// FetchReceipts sends a receipt retrieval request to the remote peer.
|
||||
func (p *peerConnection) FetchReceipts(request *fetchRequest) error {
|
||||
// Short circuit if the peer is already fetching
|
||||
if !atomic.CompareAndSwapInt32(&p.receiptIdle, 0, 1) {
|
||||
return errAlreadyFetching
|
||||
}
|
||||
p.receiptStarted = time.Now()
|
||||
|
||||
go func() {
|
||||
// Convert the header set to a retrievable slice
|
||||
hashes := make([]common.Hash, 0, len(request.Headers))
|
||||
for _, header := range request.Headers {
|
||||
hashes = append(hashes, header.Hash())
|
||||
}
|
||||
p.peer.RequestReceipts(hashes)
|
||||
}()
|
||||
|
||||
return nil
|
||||
// UpdateReceiptRate updates the peer's estimated receipt retrieval throughput
|
||||
// with the current measurement.
|
||||
func (p *peerConnection) UpdateReceiptRate(delivered int, elapsed time.Duration) {
|
||||
p.rates.Update(eth.ReceiptsMsg, elapsed, delivered)
|
||||
}
|
||||
|
||||
// FetchNodeData sends a node state data retrieval request to the remote peer.
|
||||
func (p *peerConnection) FetchNodeData(hashes []common.Hash) error {
|
||||
// Short circuit if the peer is already fetching
|
||||
if !atomic.CompareAndSwapInt32(&p.stateIdle, 0, 1) {
|
||||
return errAlreadyFetching
|
||||
}
|
||||
p.stateStarted = time.Now()
|
||||
|
||||
go p.peer.RequestNodeData(hashes)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// SetHeadersIdle sets the peer to idle, allowing it to execute new header retrieval
|
||||
// requests. Its estimated header retrieval throughput is updated with that measured
|
||||
// just now.
|
||||
func (p *peerConnection) SetHeadersIdle(delivered int, deliveryTime time.Time) {
|
||||
p.rates.Update(eth.BlockHeadersMsg, deliveryTime.Sub(p.headerStarted), delivered)
|
||||
atomic.StoreInt32(&p.headerIdle, 0)
|
||||
}
|
||||
|
||||
// SetBodiesIdle sets the peer to idle, allowing it to execute block body retrieval
|
||||
// requests. Its estimated body retrieval throughput is updated with that measured
|
||||
// just now.
|
||||
func (p *peerConnection) SetBodiesIdle(delivered int, deliveryTime time.Time) {
|
||||
p.rates.Update(eth.BlockBodiesMsg, deliveryTime.Sub(p.blockStarted), delivered)
|
||||
atomic.StoreInt32(&p.blockIdle, 0)
|
||||
}
|
||||
|
||||
// SetReceiptsIdle sets the peer to idle, allowing it to execute new receipt
|
||||
// retrieval requests. Its estimated receipt retrieval throughput is updated
|
||||
// with that measured just now.
|
||||
func (p *peerConnection) SetReceiptsIdle(delivered int, deliveryTime time.Time) {
|
||||
p.rates.Update(eth.ReceiptsMsg, deliveryTime.Sub(p.receiptStarted), delivered)
|
||||
atomic.StoreInt32(&p.receiptIdle, 0)
|
||||
}
|
||||
|
||||
// SetNodeDataIdle sets the peer to idle, allowing it to execute new state trie
|
||||
// data retrieval requests. Its estimated state retrieval throughput is updated
|
||||
// with that measured just now.
|
||||
func (p *peerConnection) SetNodeDataIdle(delivered int, deliveryTime time.Time) {
|
||||
p.rates.Update(eth.NodeDataMsg, deliveryTime.Sub(p.stateStarted), delivered)
|
||||
atomic.StoreInt32(&p.stateIdle, 0)
|
||||
}
|
||||
|
||||
// HeaderCapacity retrieves the peers header download allowance based on its
|
||||
// HeaderCapacity retrieves the peer's header download allowance based on its
|
||||
// previously discovered throughput.
|
||||
func (p *peerConnection) HeaderCapacity(targetRTT time.Duration) int {
|
||||
cap := p.rates.Capacity(eth.BlockHeadersMsg, targetRTT)
|
||||
@ -238,9 +135,9 @@ func (p *peerConnection) HeaderCapacity(targetRTT time.Duration) int {
|
||||
return cap
|
||||
}
|
||||
|
||||
// BlockCapacity retrieves the peers block download allowance based on its
|
||||
// BodyCapacity retrieves the peer's body download allowance based on its
|
||||
// previously discovered throughput.
|
||||
func (p *peerConnection) BlockCapacity(targetRTT time.Duration) int {
|
||||
func (p *peerConnection) BodyCapacity(targetRTT time.Duration) int {
|
||||
cap := p.rates.Capacity(eth.BlockBodiesMsg, targetRTT)
|
||||
if cap > MaxBlockFetch {
|
||||
cap = MaxBlockFetch
|
||||
@ -258,16 +155,6 @@ func (p *peerConnection) ReceiptCapacity(targetRTT time.Duration) int {
|
||||
return cap
|
||||
}
|
||||
|
||||
// NodeDataCapacity retrieves the peers state download allowance based on its
|
||||
// previously discovered throughput.
|
||||
func (p *peerConnection) NodeDataCapacity(targetRTT time.Duration) int {
|
||||
cap := p.rates.Capacity(eth.NodeDataMsg, targetRTT)
|
||||
if cap > MaxStateFetch {
|
||||
cap = MaxStateFetch
|
||||
}
|
||||
return cap
|
||||
}
|
||||
|
||||
// MarkLacking appends a new entity to the set of items (blocks, receipts, states)
|
||||
// that a peer is known not to have (i.e. have been requested before). If the
|
||||
// set reaches its maximum allowed capacity, items are randomly dropped off.
|
||||
@ -294,14 +181,19 @@ func (p *peerConnection) Lacks(hash common.Hash) bool {
|
||||
return ok
|
||||
}
|
||||
|
||||
// peeringEvent is sent on the peer event feed when a remote peer connects or
|
||||
// disconnects.
|
||||
type peeringEvent struct {
|
||||
peer *peerConnection
|
||||
join bool
|
||||
}
|
||||
|
||||
// peerSet represents the collection of active peer participating in the chain
|
||||
// download procedure.
|
||||
type peerSet struct {
|
||||
peers map[string]*peerConnection
|
||||
rates *msgrate.Trackers // Set of rate trackers to give the sync a common beat
|
||||
|
||||
newPeerFeed event.Feed
|
||||
peerDropFeed event.Feed
|
||||
peers map[string]*peerConnection
|
||||
rates *msgrate.Trackers // Set of rate trackers to give the sync a common beat
|
||||
events event.Feed // Feed to publish peer lifecycle events on
|
||||
|
||||
lock sync.RWMutex
|
||||
}
|
||||
@ -314,14 +206,9 @@ func newPeerSet() *peerSet {
|
||||
}
|
||||
}
|
||||
|
||||
// SubscribeNewPeers subscribes to peer arrival events.
|
||||
func (ps *peerSet) SubscribeNewPeers(ch chan<- *peerConnection) event.Subscription {
|
||||
return ps.newPeerFeed.Subscribe(ch)
|
||||
}
|
||||
|
||||
// SubscribePeerDrops subscribes to peer departure events.
|
||||
func (ps *peerSet) SubscribePeerDrops(ch chan<- *peerConnection) event.Subscription {
|
||||
return ps.peerDropFeed.Subscribe(ch)
|
||||
// SubscribeEvents subscribes to peer arrival and departure events.
|
||||
func (ps *peerSet) SubscribeEvents(ch chan<- *peeringEvent) event.Subscription {
|
||||
return ps.events.Subscribe(ch)
|
||||
}
|
||||
|
||||
// Reset iterates over the current peer set, and resets each of the known peers
|
||||
@ -355,7 +242,7 @@ func (ps *peerSet) Register(p *peerConnection) error {
|
||||
ps.peers[p.id] = p
|
||||
ps.lock.Unlock()
|
||||
|
||||
ps.newPeerFeed.Send(p)
|
||||
ps.events.Send(&peeringEvent{peer: p, join: true})
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -372,7 +259,7 @@ func (ps *peerSet) Unregister(id string) error {
|
||||
ps.rates.Untrack(id)
|
||||
ps.lock.Unlock()
|
||||
|
||||
ps.peerDropFeed.Send(p)
|
||||
ps.events.Send(&peeringEvent{peer: p, join: false})
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -404,82 +291,6 @@ func (ps *peerSet) AllPeers() []*peerConnection {
|
||||
return list
|
||||
}
|
||||
|
||||
// HeaderIdlePeers retrieves a flat list of all the currently header-idle peers
|
||||
// within the active peer set, ordered by their reputation.
|
||||
func (ps *peerSet) HeaderIdlePeers() ([]*peerConnection, int) {
|
||||
idle := func(p *peerConnection) bool {
|
||||
return atomic.LoadInt32(&p.headerIdle) == 0
|
||||
}
|
||||
throughput := func(p *peerConnection) int {
|
||||
return p.rates.Capacity(eth.BlockHeadersMsg, time.Second)
|
||||
}
|
||||
return ps.idlePeers(eth.ETH66, eth.ETH66, idle, throughput)
|
||||
}
|
||||
|
||||
// BodyIdlePeers retrieves a flat list of all the currently body-idle peers within
|
||||
// the active peer set, ordered by their reputation.
|
||||
func (ps *peerSet) BodyIdlePeers() ([]*peerConnection, int) {
|
||||
idle := func(p *peerConnection) bool {
|
||||
return atomic.LoadInt32(&p.blockIdle) == 0
|
||||
}
|
||||
throughput := func(p *peerConnection) int {
|
||||
return p.rates.Capacity(eth.BlockBodiesMsg, time.Second)
|
||||
}
|
||||
return ps.idlePeers(eth.ETH66, eth.ETH66, idle, throughput)
|
||||
}
|
||||
|
||||
// ReceiptIdlePeers retrieves a flat list of all the currently receipt-idle peers
|
||||
// within the active peer set, ordered by their reputation.
|
||||
func (ps *peerSet) ReceiptIdlePeers() ([]*peerConnection, int) {
|
||||
idle := func(p *peerConnection) bool {
|
||||
return atomic.LoadInt32(&p.receiptIdle) == 0
|
||||
}
|
||||
throughput := func(p *peerConnection) int {
|
||||
return p.rates.Capacity(eth.ReceiptsMsg, time.Second)
|
||||
}
|
||||
return ps.idlePeers(eth.ETH66, eth.ETH66, idle, throughput)
|
||||
}
|
||||
|
||||
// NodeDataIdlePeers retrieves a flat list of all the currently node-data-idle
|
||||
// peers within the active peer set, ordered by their reputation.
|
||||
func (ps *peerSet) NodeDataIdlePeers() ([]*peerConnection, int) {
|
||||
idle := func(p *peerConnection) bool {
|
||||
return atomic.LoadInt32(&p.stateIdle) == 0
|
||||
}
|
||||
throughput := func(p *peerConnection) int {
|
||||
return p.rates.Capacity(eth.NodeDataMsg, time.Second)
|
||||
}
|
||||
return ps.idlePeers(eth.ETH66, eth.ETH66, idle, throughput)
|
||||
}
|
||||
|
||||
// idlePeers retrieves a flat list of all currently idle peers satisfying the
|
||||
// protocol version constraints, using the provided function to check idleness.
|
||||
// The resulting set of peers are sorted by their capacity.
|
||||
func (ps *peerSet) idlePeers(minProtocol, maxProtocol uint, idleCheck func(*peerConnection) bool, capacity func(*peerConnection) int) ([]*peerConnection, int) {
|
||||
ps.lock.RLock()
|
||||
defer ps.lock.RUnlock()
|
||||
|
||||
var (
|
||||
total = 0
|
||||
idle = make([]*peerConnection, 0, len(ps.peers))
|
||||
tps = make([]int, 0, len(ps.peers))
|
||||
)
|
||||
for _, p := range ps.peers {
|
||||
if p.version >= minProtocol && p.version <= maxProtocol {
|
||||
if idleCheck(p) {
|
||||
idle = append(idle, p)
|
||||
tps = append(tps, capacity(p))
|
||||
}
|
||||
total++
|
||||
}
|
||||
}
|
||||
|
||||
// And sort them
|
||||
sortPeers := &peerCapacitySort{idle, tps}
|
||||
sort.Sort(sortPeers)
|
||||
return sortPeers.p, total
|
||||
}
|
||||
|
||||
// peerCapacitySort implements sort.Interface.
|
||||
// It sorts peer connections by capacity (descending).
|
||||
type peerCapacitySort struct {
|
||||
|
Reference in New Issue
Block a user