forked from mirror/go-ethereum
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 chainsverkle/onleaf
parent
3038e480f5
commit
c10a0a62c3
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
@ -0,0 +1,115 @@ |
|||||||
|
// Copyright 2021 The go-ethereum Authors
|
||||||
|
// This file is part of the go-ethereum library.
|
||||||
|
//
|
||||||
|
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||||
|
// it under the terms of the GNU Lesser General Public License as published by
|
||||||
|
// the Free Software Foundation, either version 3 of the License, or
|
||||||
|
// (at your option) any later version.
|
||||||
|
//
|
||||||
|
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||||
|
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||||
|
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||||
|
// GNU Lesser General Public License for more details.
|
||||||
|
//
|
||||||
|
// You should have received a copy of the GNU Lesser General Public License
|
||||||
|
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||||
|
|
||||||
|
package downloader |
||||||
|
|
||||||
|
import ( |
||||||
|
"time" |
||||||
|
|
||||||
|
"github.com/ethereum/go-ethereum/common" |
||||||
|
"github.com/ethereum/go-ethereum/core/types" |
||||||
|
"github.com/ethereum/go-ethereum/eth/protocols/eth" |
||||||
|
) |
||||||
|
|
||||||
|
// fetchHeadersByHash is a blocking version of Peer.RequestHeadersByHash which
|
||||||
|
// handles all the cancellation, interruption and timeout mechanisms of a data
|
||||||
|
// retrieval to allow blocking API calls.
|
||||||
|
func (d *Downloader) fetchHeadersByHash(p *peerConnection, hash common.Hash, amount int, skip int, reverse bool) ([]*types.Header, error) { |
||||||
|
// Create the response sink and send the network request
|
||||||
|
start := time.Now() |
||||||
|
resCh := make(chan *eth.Response) |
||||||
|
|
||||||
|
req, err := p.peer.RequestHeadersByHash(hash, amount, skip, reverse, resCh) |
||||||
|
if err != nil { |
||||||
|
return nil, err |
||||||
|
} |
||||||
|
defer req.Close() |
||||||
|
|
||||||
|
// Wait until the response arrives, the request is cancelled or times out
|
||||||
|
ttl := d.peers.rates.TargetTimeout() |
||||||
|
|
||||||
|
timeoutTimer := time.NewTimer(ttl) |
||||||
|
defer timeoutTimer.Stop() |
||||||
|
|
||||||
|
select { |
||||||
|
case <-d.cancelCh: |
||||||
|
return nil, errCanceled |
||||||
|
|
||||||
|
case <-timeoutTimer.C: |
||||||
|
// Header retrieval timed out, update the metrics
|
||||||
|
p.log.Debug("Header request timed out", "elapsed", ttl) |
||||||
|
headerTimeoutMeter.Mark(1) |
||||||
|
|
||||||
|
return nil, errTimeout |
||||||
|
|
||||||
|
case res := <-resCh: |
||||||
|
// Headers successfully retrieved, update the metrics
|
||||||
|
headerReqTimer.Update(time.Since(start)) |
||||||
|
headerInMeter.Mark(int64(len(*res.Res.(*eth.BlockHeadersPacket)))) |
||||||
|
|
||||||
|
// Don't reject the packet even if it turns out to be bad, downloader will
|
||||||
|
// disconnect the peer on its own terms. Simply delivery the headers to
|
||||||
|
// be processed by the caller
|
||||||
|
res.Done <- nil |
||||||
|
|
||||||
|
return *res.Res.(*eth.BlockHeadersPacket), nil |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// fetchHeadersByNumber is a blocking version of Peer.RequestHeadersByNumber which
|
||||||
|
// handles all the cancellation, interruption and timeout mechanisms of a data
|
||||||
|
// retrieval to allow blocking API calls.
|
||||||
|
func (d *Downloader) fetchHeadersByNumber(p *peerConnection, number uint64, amount int, skip int, reverse bool) ([]*types.Header, error) { |
||||||
|
// Create the response sink and send the network request
|
||||||
|
start := time.Now() |
||||||
|
resCh := make(chan *eth.Response) |
||||||
|
|
||||||
|
req, err := p.peer.RequestHeadersByNumber(number, amount, skip, reverse, resCh) |
||||||
|
if err != nil { |
||||||
|
return nil, err |
||||||
|
} |
||||||
|
defer req.Close() |
||||||
|
|
||||||
|
// Wait until the response arrives, the request is cancelled or times out
|
||||||
|
ttl := d.peers.rates.TargetTimeout() |
||||||
|
|
||||||
|
timeoutTimer := time.NewTimer(ttl) |
||||||
|
defer timeoutTimer.Stop() |
||||||
|
|
||||||
|
select { |
||||||
|
case <-d.cancelCh: |
||||||
|
return nil, errCanceled |
||||||
|
|
||||||
|
case <-timeoutTimer.C: |
||||||
|
// Header retrieval timed out, update the metrics
|
||||||
|
p.log.Debug("Header request timed out", "elapsed", ttl) |
||||||
|
headerTimeoutMeter.Mark(1) |
||||||
|
|
||||||
|
return nil, errTimeout |
||||||
|
|
||||||
|
case res := <-resCh: |
||||||
|
// Headers successfully retrieved, update the metrics
|
||||||
|
headerReqTimer.Update(time.Since(start)) |
||||||
|
headerInMeter.Mark(int64(len(*res.Res.(*eth.BlockHeadersPacket)))) |
||||||
|
|
||||||
|
// Don't reject the packet even if it turns out to be bad, downloader will
|
||||||
|
// disconnect the peer on its own terms. Simply delivery the headers to
|
||||||
|
// be processed by the caller
|
||||||
|
res.Done <- nil |
||||||
|
|
||||||
|
return *res.Res.(*eth.BlockHeadersPacket), nil |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,381 @@ |
|||||||
|
// Copyright 2021 The go-ethereum Authors
|
||||||
|
// This file is part of the go-ethereum library.
|
||||||
|
//
|
||||||
|
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||||
|
// it under the terms of the GNU Lesser General Public License as published by
|
||||||
|
// the Free Software Foundation, either version 3 of the License, or
|
||||||
|
// (at your option) any later version.
|
||||||
|
//
|
||||||
|
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||||
|
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||||
|
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||||
|
// GNU Lesser General Public License for more details.
|
||||||
|
//
|
||||||
|
// You should have received a copy of the GNU Lesser General Public License
|
||||||
|
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||||
|
|
||||||
|
package downloader |
||||||
|
|
||||||
|
import ( |
||||||
|
"errors" |
||||||
|
"sort" |
||||||
|
"time" |
||||||
|
|
||||||
|
"github.com/ethereum/go-ethereum/common" |
||||||
|
"github.com/ethereum/go-ethereum/common/prque" |
||||||
|
"github.com/ethereum/go-ethereum/eth/protocols/eth" |
||||||
|
"github.com/ethereum/go-ethereum/log" |
||||||
|
) |
||||||
|
|
||||||
|
// timeoutGracePeriod is the amount of time to allow for a peer to deliver a
|
||||||
|
// response to a locally already timed out request. Timeouts are not penalized
|
||||||
|
// as a peer might be temporarily overloaded, however, they still must reply
|
||||||
|
// to each request. Failing to do so is considered a protocol violation.
|
||||||
|
var timeoutGracePeriod = 2 * time.Minute |
||||||
|
|
||||||
|
// typedQueue is an interface defining the adaptor needed to translate the type
|
||||||
|
// specific downloader/queue schedulers into the type-agnostic general concurrent
|
||||||
|
// fetcher algorithm calls.
|
||||||
|
type typedQueue interface { |
||||||
|
// waker returns a notification channel that gets pinged in case more fetches
|
||||||
|
// have been queued up, so the fetcher might assign it to idle peers.
|
||||||
|
waker() chan bool |
||||||
|
|
||||||
|
// pending returns the number of wrapped items that are currently queued for
|
||||||
|
// fetching by the concurrent downloader.
|
||||||
|
pending() int |
||||||
|
|
||||||
|
// capacity is responsible for calculating how many items of the abstracted
|
||||||
|
// type a particular peer is estimated to be able to retrieve within the
|
||||||
|
// alloted round trip time.
|
||||||
|
capacity(peer *peerConnection, rtt time.Duration) int |
||||||
|
|
||||||
|
// updateCapacity is responsible for updating how many items of the abstracted
|
||||||
|
// type a particular peer is estimated to be able to retrieve in a unit time.
|
||||||
|
updateCapacity(peer *peerConnection, items int, elapsed time.Duration) |
||||||
|
|
||||||
|
// reserve is responsible for allocating a requested number of pending items
|
||||||
|
// from the download queue to the specified peer.
|
||||||
|
reserve(peer *peerConnection, items int) (*fetchRequest, bool, bool) |
||||||
|
|
||||||
|
// unreserve is resposible for removing the current retrieval allocation
|
||||||
|
// assigned to a specific peer and placing it back into the pool to allow
|
||||||
|
// reassigning to some other peer.
|
||||||
|
unreserve(peer string) int |
||||||
|
|
||||||
|
// request is responsible for converting a generic fetch request into a typed
|
||||||
|
// one and sending it to the remote peer for fulfillment.
|
||||||
|
request(peer *peerConnection, req *fetchRequest, resCh chan *eth.Response) (*eth.Request, error) |
||||||
|
|
||||||
|
// deliver is responsible for taking a generic response packet from the
|
||||||
|
// concurrent fetcher, unpacking the type specific data and delivering
|
||||||
|
// it to the downloader's queue.
|
||||||
|
deliver(peer *peerConnection, packet *eth.Response) (int, error) |
||||||
|
} |
||||||
|
|
||||||
|
// concurrentFetch iteratively downloads scheduled block parts, taking available
|
||||||
|
// peers, reserving a chunk of fetch requests for each and waiting for delivery
|
||||||
|
// or timeouts.
|
||||||
|
func (d *Downloader) concurrentFetch(queue typedQueue) error { |
||||||
|
// Create a delivery channel to accept responses from all peers
|
||||||
|
responses := make(chan *eth.Response) |
||||||
|
|
||||||
|
// Track the currently active requests and their timeout order
|
||||||
|
pending := make(map[string]*eth.Request) |
||||||
|
defer func() { |
||||||
|
// Abort all requests on sync cycle cancellation. The requests may still
|
||||||
|
// be fulfilled by the remote side, but the dispatcher will not wait to
|
||||||
|
// deliver them since nobody's going to be listening.
|
||||||
|
for _, req := range pending { |
||||||
|
req.Close() |
||||||
|
} |
||||||
|
}() |
||||||
|
ordering := make(map[*eth.Request]int) |
||||||
|
timeouts := prque.New(func(data interface{}, index int) { |
||||||
|
ordering[data.(*eth.Request)] = index |
||||||
|
}) |
||||||
|
|
||||||
|
timeout := time.NewTimer(0) |
||||||
|
if !timeout.Stop() { |
||||||
|
<-timeout.C |
||||||
|
} |
||||||
|
defer timeout.Stop() |
||||||
|
|
||||||
|
// Track the timed-out but not-yet-answered requests separately. We want to
|
||||||
|
// keep tracking which peers are busy (potentially overloaded), so removing
|
||||||
|
// all trace of a timed out request is not good. We also can't just cancel
|
||||||
|
// the pending request altogether as that would prevent a late response from
|
||||||
|
// being delivered, thus never unblocking the peer.
|
||||||
|
stales := make(map[string]*eth.Request) |
||||||
|
defer func() { |
||||||
|
// Abort all requests on sync cycle cancellation. The requests may still
|
||||||
|
// be fulfilled by the remote side, but the dispatcher will not wait to
|
||||||
|
// deliver them since nobody's going to be listening.
|
||||||
|
for _, req := range stales { |
||||||
|
req.Close() |
||||||
|
} |
||||||
|
}() |
||||||
|
// Subscribe to peer lifecycle events to schedule tasks to new joiners and
|
||||||
|
// reschedule tasks upon disconnections. We don't care which event happened
|
||||||
|
// for simplicity, so just use a single channel.
|
||||||
|
peering := make(chan *peeringEvent, 64) // arbitrary buffer, just some burst protection
|
||||||
|
|
||||||
|
peeringSub := d.peers.SubscribeEvents(peering) |
||||||
|
defer peeringSub.Unsubscribe() |
||||||
|
|
||||||
|
// Prepare the queue and fetch block parts until the block header fetcher's done
|
||||||
|
finished := false |
||||||
|
for { |
||||||
|
// Short circuit if we lost all our peers
|
||||||
|
if d.peers.Len() == 0 { |
||||||
|
return errNoPeers |
||||||
|
} |
||||||
|
// If there's nothing more to fetch, wait or terminate
|
||||||
|
if queue.pending() == 0 { |
||||||
|
if len(pending) == 0 && finished { |
||||||
|
return nil |
||||||
|
} |
||||||
|
} else { |
||||||
|
// Send a download request to all idle peers, until throttled
|
||||||
|
var ( |
||||||
|
idles []*peerConnection |
||||||
|
caps []int |
||||||
|
) |
||||||
|
for _, peer := range d.peers.AllPeers() { |
||||||
|
pending, stale := pending[peer.id], stales[peer.id] |
||||||
|
if pending == nil && stale == nil { |
||||||
|
idles = append(idles, peer) |
||||||
|
caps = append(caps, queue.capacity(peer, time.Second)) |
||||||
|
} else if stale != nil { |
||||||
|
if waited := time.Since(stale.Sent); waited > timeoutGracePeriod { |
||||||
|
// Request has been in flight longer than the grace period
|
||||||
|
// permitted it, consider the peer malicious attempting to
|
||||||
|
// stall the sync.
|
||||||
|
peer.log.Warn("Peer stalling, dropping", "waited", common.PrettyDuration(waited)) |
||||||
|
d.dropPeer(peer.id) |
||||||
|
} |
||||||
|
} |
||||||
|
} |
||||||
|
sort.Sort(&peerCapacitySort{idles, caps}) |
||||||
|
|
||||||
|
var ( |
||||||
|
progressed bool |
||||||
|
throttled bool |
||||||
|
queued = queue.pending() |
||||||
|
) |
||||||
|
for _, peer := range idles { |
||||||
|
// Short circuit if throttling activated or there are no more
|
||||||
|
// queued tasks to be retrieved
|
||||||
|
if throttled { |
||||||
|
break |
||||||
|
} |
||||||
|
if queued = queue.pending(); queued == 0 { |
||||||
|
break |
||||||
|
} |
||||||
|
// Reserve a chunk of fetches for a peer. A nil can mean either that
|
||||||
|
// no more headers are available, or that the peer is known not to
|
||||||
|
// have them.
|
||||||
|
request, progress, throttle := queue.reserve(peer, queue.capacity(peer, d.peers.rates.TargetRoundTrip())) |
||||||
|
if progress { |
||||||
|
progressed = true |
||||||
|
} |
||||||
|
if throttle { |
||||||
|
throttled = true |
||||||
|
throttleCounter.Inc(1) |
||||||
|
} |
||||||
|
if request == nil { |
||||||
|
continue |
||||||
|
} |
||||||
|
// Fetch the chunk and make sure any errors return the hashes to the queue
|
||||||
|
req, err := queue.request(peer, request, responses) |
||||||
|
if err != nil { |
||||||
|
// Sending the request failed, which generally means the peer
|
||||||
|
// was diconnected in between assignment and network send.
|
||||||
|
// Although all peer removal operations return allocated tasks
|
||||||
|
// to the queue, that is async, and we can do better here by
|
||||||
|
// immediately pushing the unfulfilled requests.
|
||||||
|
queue.unreserve(peer.id) // TODO(karalabe): This needs a non-expiration method
|
||||||
|
continue |
||||||
|
} |
||||||
|
pending[peer.id] = req |
||||||
|
|
||||||
|
ttl := d.peers.rates.TargetTimeout() |
||||||
|
ordering[req] = timeouts.Size() |
||||||
|
|
||||||
|
timeouts.Push(req, -time.Now().Add(ttl).UnixNano()) |
||||||
|
if timeouts.Size() == 1 { |
||||||
|
timeout.Reset(ttl) |
||||||
|
} |
||||||
|
} |
||||||
|
// Make sure that we have peers available for fetching. If all peers have been tried
|
||||||
|
// and all failed throw an error
|
||||||
|
if !progressed && !throttled && len(pending) == 0 && len(idles) == d.peers.Len() && queued > 0 { |
||||||
|
return errPeersUnavailable |
||||||
|
} |
||||||
|
} |
||||||
|
// Wait for something to happen
|
||||||
|
select { |
||||||
|
case <-d.cancelCh: |
||||||
|
// If sync was cancelled, tear down the parallel retriever. Pending
|
||||||
|
// requests will be cancelled locally, and the remote responses will
|
||||||
|
// be dropped when they arrive
|
||||||
|
return errCanceled |
||||||
|
|
||||||
|
case event := <-peering: |
||||||
|
// A peer joined or left, the tasks queue and allocations need to be
|
||||||
|
// checked for potential assignment or reassignment
|
||||||
|
peerid := event.peer.id |
||||||
|
|
||||||
|
if event.join { |
||||||
|
// Sanity check the internal state; this can be dropped later
|
||||||
|
if _, ok := pending[peerid]; ok { |
||||||
|
event.peer.log.Error("Pending request exists for joining peer") |
||||||
|
} |
||||||
|
if _, ok := stales[peerid]; ok { |
||||||
|
event.peer.log.Error("Stale request exists for joining peer") |
||||||
|
} |
||||||
|
// Loop back to the entry point for task assignment
|
||||||
|
continue |
||||||
|
} |
||||||
|
// A peer left, any existing requests need to be untracked, pending
|
||||||
|
// tasks returned and possible reassignment checked
|
||||||
|
if req, ok := pending[peerid]; ok { |
||||||
|
queue.unreserve(peerid) // TODO(karalabe): This needs a non-expiration method
|
||||||
|
delete(pending, peerid) |
||||||
|
req.Close() |
||||||
|
|
||||||
|
if index, live := ordering[req]; live { |
||||||
|
timeouts.Remove(index) |
||||||
|
if index == 0 { |
||||||
|
if !timeout.Stop() { |
||||||
|
<-timeout.C |
||||||
|
} |
||||||
|
if timeouts.Size() > 0 { |
||||||
|
_, exp := timeouts.Peek() |
||||||
|
timeout.Reset(time.Until(time.Unix(0, -exp))) |
||||||
|
} |
||||||
|
} |
||||||
|
delete(ordering, req) |
||||||
|
} |
||||||
|
} |
||||||
|
if req, ok := stales[peerid]; ok { |
||||||
|
delete(stales, peerid) |
||||||
|
req.Close() |
||||||
|
} |
||||||
|
|
||||||
|
case <-timeout.C: |
||||||
|
// Retrieve the next request which should have timed out. The check
|
||||||
|
// below is purely for to catch programming errors, given the correct
|
||||||
|
// code, there's no possible order of events that should result in a
|
||||||
|
// timeout firing for a non-existent event.
|
||||||
|
item, exp := timeouts.Peek() |
||||||
|
if now, at := time.Now(), time.Unix(0, -exp); now.Before(at) { |
||||||
|
log.Error("Timeout triggered but not reached", "left", at.Sub(now)) |
||||||
|
timeout.Reset(at.Sub(now)) |
||||||
|
continue |
||||||
|
} |
||||||
|
req := item.(*eth.Request) |
||||||
|
|
||||||
|
// Stop tracking the timed out request from a timing perspective,
|
||||||
|
// cancel it, so it's not considered in-flight anymore, but keep
|
||||||
|
// the peer marked busy to prevent assigning a second request and
|
||||||
|
// overloading it further.
|
||||||
|
delete(pending, req.Peer) |
||||||
|
stales[req.Peer] = req |
||||||
|
delete(ordering, req) |
||||||
|
|
||||||
|
timeouts.Pop() |
||||||
|
if timeouts.Size() > 0 { |
||||||
|
_, exp := timeouts.Peek() |
||||||
|
timeout.Reset(time.Until(time.Unix(0, -exp))) |
||||||
|
} |
||||||
|
// New timeout potentially set if there are more requests pending,
|
||||||
|
// reschedule the failed one to a free peer
|
||||||
|
fails := queue.unreserve(req.Peer) |
||||||
|
|
||||||
|
// Finally, update the peer's retrieval capacity, or if it's already
|
||||||
|
// below the minimum allowance, drop the peer. If a lot of retrieval
|
||||||
|
// elements expired, we might have overestimated the remote peer or
|
||||||
|
// perhaps ourselves. Only reset to minimal throughput but don't drop
|
||||||
|
// just yet.
|
||||||
|
//
|
||||||
|
// The reason the minimum threshold is 2 is that the downloader tries
|
||||||
|
// to estimate the bandwidth and latency of a peer separately, which
|
||||||
|
// requires pushing the measured capacity a bit and seeing how response
|
||||||
|
// times reacts, to it always requests one more than the minimum (i.e.
|
||||||
|
// min 2).
|
||||||
|
peer := d.peers.Peer(req.Peer) |
||||||
|
if peer == nil { |
||||||
|
// If the peer got disconnected in between, we should really have
|
||||||
|
// short-circuited it already. Just in case there's some strange
|
||||||
|
// codepath, leave this check in not to crash.
|
||||||
|
log.Error("Delivery timeout from unknown peer", "peer", req.Peer) |
||||||
|
continue |
||||||
|
} |
||||||
|
if fails > 2 { |
||||||
|
queue.updateCapacity(peer, 0, 0) |
||||||
|
} else { |
||||||
|
d.dropPeer(peer.id) |
||||||
|
|
||||||
|
// If this peer was the master peer, abort sync immediately
|
||||||
|
d.cancelLock.RLock() |
||||||
|
master := peer.id == d.cancelPeer |
||||||
|
d.cancelLock.RUnlock() |
||||||
|
|
||||||
|
if master { |
||||||
|
d.cancel() |
||||||
|
return errTimeout |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
case res := <-responses: |
||||||
|
// Response arrived, it may be for an existing or an already timed
|
||||||
|
// out request. If the former, update the timeout heap and perhaps
|
||||||
|
// reschedule the timeout timer.
|
||||||
|
index, live := ordering[res.Req] |
||||||
|
if live { |
||||||
|
timeouts.Remove(index) |
||||||
|
if index == 0 { |
||||||
|
if !timeout.Stop() { |
||||||
|
<-timeout.C |
||||||
|
} |
||||||
|
if timeouts.Size() > 0 { |
||||||
|
_, exp := timeouts.Peek() |
||||||
|
timeout.Reset(time.Until(time.Unix(0, -exp))) |
||||||
|
} |
||||||
|
} |
||||||
|
delete(ordering, res.Req) |
||||||
|
} |
||||||
|
// Delete the pending request (if it still exists) and mark the peer idle
|
||||||
|
delete(pending, res.Req.Peer) |
||||||
|
delete(stales, res.Req.Peer) |
||||||
|
|
||||||
|
// Signal the dispatcher that the round trip is done. We'll drop the
|
||||||
|
// peer if the data turns out to be junk.
|
||||||
|
res.Done <- nil |
||||||
|
res.Req.Close() |
||||||
|
|
||||||
|
// If the peer was previously banned and failed to deliver its pack
|
||||||
|
// in a reasonable time frame, ignore its message.
|
||||||
|
if peer := d.peers.Peer(res.Req.Peer); peer != nil { |
||||||
|
// Deliver the received chunk of data and check chain validity
|
||||||
|
accepted, err := queue.deliver(peer, res) |
||||||
|
if errors.Is(err, errInvalidChain) { |
||||||
|
return err |
||||||
|
} |
||||||
|
// Unless a peer delivered something completely else than requested (usually
|
||||||
|
// caused by a timed out request which came through in the end), set it to
|
||||||
|
// idle. If the delivery's stale, the peer should have already been idled.
|
||||||
|
if !errors.Is(err, errStaleDelivery) { |
||||||
|
queue.updateCapacity(peer, accepted, res.Time) |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
case cont := <-queue.waker(): |
||||||
|
// The header fetcher sent a continuation flag, check if it's done
|
||||||
|
if !cont { |
||||||
|
finished = true |
||||||
|
} |
||||||
|
} |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,104 @@ |
|||||||
|
// Copyright 2021 The go-ethereum Authors
|
||||||
|
// This file is part of the go-ethereum library.
|
||||||
|
//
|
||||||
|
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||||
|
// it under the terms of the GNU Lesser General Public License as published by
|
||||||
|
// the Free Software Foundation, either version 3 of the License, or
|
||||||
|
// (at your option) any later version.
|
||||||
|
//
|
||||||
|
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||||
|
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||||
|
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||||
|
// GNU Lesser General Public License for more details.
|
||||||
|
//
|
||||||
|
// You should have received a copy of the GNU Lesser General Public License
|
||||||
|
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||||
|
|
||||||
|
package downloader |
||||||
|
|
||||||
|
import ( |
||||||
|
"time" |
||||||
|
|
||||||
|
"github.com/ethereum/go-ethereum/common" |
||||||
|
"github.com/ethereum/go-ethereum/eth/protocols/eth" |
||||||
|
"github.com/ethereum/go-ethereum/log" |
||||||
|
) |
||||||
|
|
||||||
|
// bodyQueue implements typedQueue and is a type adapter between the generic
|
||||||
|
// concurrent fetcher and the downloader.
|
||||||
|
type bodyQueue Downloader |
||||||
|
|
||||||
|
// waker returns a notification channel that gets pinged in case more body
|
||||||
|
// fetches have been queued up, so the fetcher might assign it to idle peers.
|
||||||
|
func (q *bodyQueue) waker() chan bool { |
||||||
|
return q.queue.blockWakeCh |
||||||
|
} |
||||||
|
|
||||||
|
// pending returns the number of bodies that are currently queued for fetching
|
||||||
|
// by the concurrent downloader.
|
||||||
|
func (q *bodyQueue) pending() int { |
||||||
|
return q.queue.PendingBodies() |
||||||
|
} |
||||||
|
|
||||||
|
// capacity is responsible for calculating how many bodies a particular peer is
|
||||||
|
// estimated to be able to retrieve within the alloted round trip time.
|
||||||
|
func (q *bodyQueue) capacity(peer *peerConnection, rtt time.Duration) int { |
||||||
|
return peer.BodyCapacity(rtt) |
||||||
|
} |
||||||
|
|
||||||
|
// updateCapacity is responsible for updating how many bodies a particular peer
|
||||||
|
// is estimated to be able to retrieve in a unit time.
|
||||||
|
func (q *bodyQueue) updateCapacity(peer *peerConnection, items int, span time.Duration) { |
||||||
|
peer.UpdateBodyRate(items, span) |
||||||
|
} |
||||||
|
|
||||||
|
// reserve is responsible for allocating a requested number of pending bodies
|
||||||
|
// from the download queue to the specified peer.
|
||||||
|
func (q *bodyQueue) reserve(peer *peerConnection, items int) (*fetchRequest, bool, bool) { |
||||||
|
return q.queue.ReserveBodies(peer, items) |
||||||
|
} |
||||||
|
|
||||||
|
// unreserve is resposible for removing the current body retrieval allocation
|
||||||
|
// assigned to a specific peer and placing it back into the pool to allow
|
||||||
|
// reassigning to some other peer.
|
||||||
|
func (q *bodyQueue) unreserve(peer string) int { |
||||||
|
fails := q.queue.ExpireBodies(peer) |
||||||
|
if fails > 2 { |
||||||
|
log.Trace("Body delivery timed out", "peer", peer) |
||||||
|
} else { |
||||||
|
log.Debug("Body delivery stalling", "peer", peer) |
||||||
|
} |
||||||
|
return fails |
||||||
|
} |
||||||
|
|
||||||
|
// request is responsible for converting a generic fetch request into a body
|
||||||
|
// one and sending it to the remote peer for fulfillment.
|
||||||
|
func (q *bodyQueue) request(peer *peerConnection, req *fetchRequest, resCh chan *eth.Response) (*eth.Request, error) { |
||||||
|
peer.log.Trace("Requesting new batch of bodies", "count", len(req.Headers), "from", req.Headers[0].Number) |
||||||
|
if q.bodyFetchHook != nil { |
||||||
|
q.bodyFetchHook(req.Headers) |
||||||
|
} |
||||||
|
|
||||||
|
hashes := make([]common.Hash, 0, len(req.Headers)) |
||||||
|
for _, header := range req.Headers { |
||||||
|
hashes = append(hashes, header.Hash()) |
||||||
|
} |
||||||
|
return peer.peer.RequestBodies(hashes, resCh) |
||||||
|
} |
||||||
|
|
||||||
|
// deliver is responsible for taking a generic response packet from the concurrent
|
||||||
|
// fetcher, unpacking the body data and delivering it to the downloader's queue.
|
||||||
|
func (q *bodyQueue) deliver(peer *peerConnection, packet *eth.Response) (int, error) { |
||||||
|
txs, uncles := packet.Res.(*eth.BlockBodiesPacket).Unpack() |
||||||
|
|
||||||
|
accepted, err := q.queue.DeliverBodies(peer.id, txs, uncles) |
||||||
|
switch { |
||||||
|
case err == nil && len(txs) == 0: |
||||||
|
peer.log.Trace("Requested bodies delivered") |
||||||
|
case err == nil: |
||||||
|
peer.log.Trace("Delivered new batch of bodies", "count", len(txs), "accepted", accepted) |
||||||
|
default: |
||||||
|
peer.log.Debug("Failed to deliver retrieved bodies", "err", err) |
||||||
|
} |
||||||
|
return accepted, err |
||||||
|
} |
@ -0,0 +1,95 @@ |
|||||||
|
// Copyright 2021 The go-ethereum Authors
|
||||||
|
// This file is part of the go-ethereum library.
|
||||||
|
//
|
||||||
|
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||||
|
// it under the terms of the GNU Lesser General Public License as published by
|
||||||
|
// the Free Software Foundation, either version 3 of the License, or
|
||||||
|
// (at your option) any later version.
|
||||||
|
//
|
||||||
|
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||||
|
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||||
|
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||||
|
// GNU Lesser General Public License for more details.
|
||||||
|
//
|
||||||
|
// You should have received a copy of the GNU Lesser General Public License
|
||||||
|
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||||
|
|
||||||
|
package downloader |
||||||
|
|
||||||
|
import ( |
||||||
|
"time" |
||||||
|
|
||||||
|
"github.com/ethereum/go-ethereum/eth/protocols/eth" |
||||||
|
"github.com/ethereum/go-ethereum/log" |
||||||
|
) |
||||||
|
|
||||||
|
// headerQueue implements typedQueue and is a type adapter between the generic
|
||||||
|
// concurrent fetcher and the downloader.
|
||||||
|
type headerQueue Downloader |
||||||
|
|
||||||
|
// waker returns a notification channel that gets pinged in case more header
|
||||||
|
// fetches have been queued up, so the fetcher might assign it to idle peers.
|
||||||
|
func (q *headerQueue) waker() chan bool { |
||||||
|
return q.queue.headerContCh |
||||||
|
} |
||||||
|
|
||||||
|
// pending returns the number of headers that are currently queued for fetching
|
||||||
|
// by the concurrent downloader.
|
||||||
|
func (q *headerQueue) pending() int { |
||||||
|
return q.queue.PendingHeaders() |
||||||
|
} |
||||||
|
|
||||||
|
// capacity is responsible for calculating how many headers a particular peer is
|
||||||
|
// estimated to be able to retrieve within the alloted round trip time.
|
||||||
|
func (q *headerQueue) capacity(peer *peerConnection, rtt time.Duration) int { |
||||||
|
return peer.HeaderCapacity(rtt) |
||||||
|
} |
||||||
|
|
||||||
|
// updateCapacity is responsible for updating how many headers a particular peer
|
||||||
|
// is estimated to be able to retrieve in a unit time.
|
||||||
|
func (q *headerQueue) updateCapacity(peer *peerConnection, items int, span time.Duration) { |
||||||
|
peer.UpdateHeaderRate(items, span) |
||||||
|
} |
||||||
|
|
||||||
|
// reserve is responsible for allocating a requested number of pending headers
|
||||||
|
// from the download queue to the specified peer.
|
||||||
|
func (q *headerQueue) reserve(peer *peerConnection, items int) (*fetchRequest, bool, bool) { |
||||||
|
return q.queue.ReserveHeaders(peer, items), false, false |
||||||
|
} |
||||||
|
|
||||||
|
// unreserve is resposible for removing the current header retrieval allocation
|
||||||
|
// assigned to a specific peer and placing it back into the pool to allow
|
||||||
|
// reassigning to some other peer.
|
||||||
|
func (q *headerQueue) unreserve(peer string) int { |
||||||
|
fails := q.queue.ExpireHeaders(peer) |
||||||
|
if fails > 2 { |
||||||
|
log.Trace("Header delivery timed out", "peer", peer) |
||||||
|
} else { |
||||||
|
log.Debug("Header delivery stalling", "peer", peer) |
||||||
|
} |
||||||
|
return fails |
||||||
|
} |
||||||
|
|
||||||
|
// request is responsible for converting a generic fetch request into a header
|
||||||
|
// one and sending it to the remote peer for fulfillment.
|
||||||
|
func (q *headerQueue) request(peer *peerConnection, req *fetchRequest, resCh chan *eth.Response) (*eth.Request, error) { |
||||||
|
peer.log.Trace("Requesting new batch of headers", "from", req.From) |
||||||
|
return peer.peer.RequestHeadersByNumber(req.From, MaxHeaderFetch, 0, false, resCh) |
||||||
|
} |
||||||
|
|
||||||
|
// deliver is responsible for taking a generic response packet from the concurrent
|
||||||
|
// fetcher, unpacking the header data and delivering it to the downloader's queue.
|
||||||
|
func (q *headerQueue) deliver(peer *peerConnection, packet *eth.Response) (int, error) { |
||||||
|
headers := *packet.Res.(*eth.BlockHeadersPacket) |
||||||
|
|
||||||
|
accepted, err := q.queue.DeliverHeaders(peer.id, headers, q.headerProcCh) |
||||||
|
switch { |
||||||
|
case err == nil && len(headers) == 0: |
||||||
|
peer.log.Trace("Requested headers delivered") |
||||||
|
case err == nil: |
||||||
|
peer.log.Trace("Delivered new batch of headers", "count", len(headers), "accepted", accepted) |
||||||
|
default: |
||||||
|
peer.log.Debug("Failed to deliver retrieved headers", "err", err) |
||||||
|
} |
||||||
|
return accepted, err |
||||||
|
} |
@ -0,0 +1,103 @@ |
|||||||
|
// Copyright 2021 The go-ethereum Authors
|
||||||
|
// This file is part of the go-ethereum library.
|
||||||
|
//
|
||||||
|
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||||
|
// it under the terms of the GNU Lesser General Public License as published by
|
||||||
|
// the Free Software Foundation, either version 3 of the License, or
|
||||||
|
// (at your option) any later version.
|
||||||
|
//
|
||||||
|
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||||
|
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||||
|
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||||
|
// GNU Lesser General Public License for more details.
|
||||||
|
//
|
||||||
|
// You should have received a copy of the GNU Lesser General Public License
|
||||||
|
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||||
|
|
||||||
|
package downloader |
||||||
|
|
||||||
|
import ( |
||||||
|
"time" |
||||||
|
|
||||||
|
"github.com/ethereum/go-ethereum/common" |
||||||
|
"github.com/ethereum/go-ethereum/eth/protocols/eth" |
||||||
|
"github.com/ethereum/go-ethereum/log" |
||||||
|
) |
||||||
|
|
||||||
|
// receiptQueue implements typedQueue and is a type adapter between the generic
|
||||||
|
// concurrent fetcher and the downloader.
|
||||||
|
type receiptQueue Downloader |
||||||
|
|
||||||
|
// waker returns a notification channel that gets pinged in case more reecipt
|
||||||
|
// fetches have been queued up, so the fetcher might assign it to idle peers.
|
||||||
|
func (q *receiptQueue) waker() chan bool { |
||||||
|
return q.queue.receiptWakeCh |
||||||
|
} |
||||||
|
|
||||||
|
// pending returns the number of receipt that are currently queued for fetching
|
||||||
|
// by the concurrent downloader.
|
||||||
|
func (q *receiptQueue) pending() int { |
||||||
|
return q.queue.PendingReceipts() |
||||||
|
} |
||||||
|
|
||||||
|
// capacity is responsible for calculating how many receipts a particular peer is
|
||||||
|
// estimated to be able to retrieve within the alloted round trip time.
|
||||||
|
func (q *receiptQueue) capacity(peer *peerConnection, rtt time.Duration) int { |
||||||
|
return peer.ReceiptCapacity(rtt) |
||||||
|
} |
||||||
|
|
||||||
|
// updateCapacity is responsible for updating how many receipts a particular peer
|
||||||
|
// is estimated to be able to retrieve in a unit time.
|
||||||
|
func (q *receiptQueue) updateCapacity(peer *peerConnection, items int, span time.Duration) { |
||||||
|
peer.UpdateReceiptRate(items, span) |
||||||
|
} |
||||||
|
|
||||||
|
// reserve is responsible for allocating a requested number of pending receipts
|
||||||
|
// from the download queue to the specified peer.
|
||||||
|
func (q *receiptQueue) reserve(peer *peerConnection, items int) (*fetchRequest, bool, bool) { |
||||||
|
return q.queue.ReserveReceipts(peer, items) |
||||||
|
} |
||||||
|
|
||||||
|
// unreserve is resposible for removing the current receipt retrieval allocation
|
||||||
|
// assigned to a specific peer and placing it back into the pool to allow
|
||||||
|
// reassigning to some other peer.
|
||||||
|
func (q *receiptQueue) unreserve(peer string) int { |
||||||
|
fails := q.queue.ExpireReceipts(peer) |
||||||
|
if fails > 2 { |
||||||
|
log.Trace("Receipt delivery timed out", "peer", peer) |
||||||
|
} else { |
||||||
|
log.Debug("Receipt delivery stalling", "peer", peer) |
||||||
|
} |
||||||
|
return fails |
||||||
|
} |
||||||
|
|
||||||
|
// request is responsible for converting a generic fetch request into a receipt
|
||||||
|
// one and sending it to the remote peer for fulfillment.
|
||||||
|
func (q *receiptQueue) request(peer *peerConnection, req *fetchRequest, resCh chan *eth.Response) (*eth.Request, error) { |
||||||
|
peer.log.Trace("Requesting new batch of receipts", "count", len(req.Headers), "from", req.Headers[0].Number) |
||||||
|
if q.receiptFetchHook != nil { |
||||||
|
q.receiptFetchHook(req.Headers) |
||||||
|
} |
||||||
|
hashes := make([]common.Hash, 0, len(req.Headers)) |
||||||
|
for _, header := range req.Headers { |
||||||
|
hashes = append(hashes, header.Hash()) |
||||||
|
} |
||||||
|
return peer.peer.RequestReceipts(hashes, resCh) |
||||||
|
} |
||||||
|
|
||||||
|
// deliver is responsible for taking a generic response packet from the concurrent
|
||||||
|
// fetcher, unpacking the receipt data and delivering it to the downloader's queue.
|
||||||
|
func (q *receiptQueue) deliver(peer *peerConnection, packet *eth.Response) (int, error) { |
||||||
|
receipts := *packet.Res.(*eth.ReceiptsPacket) |
||||||
|
|
||||||
|
accepted, err := q.queue.DeliverReceipts(peer.id, receipts) |
||||||
|
switch { |
||||||
|
case err == nil && len(receipts) == 0: |
||||||
|
peer.log.Trace("Requested receipts delivered") |
||||||
|
case err == nil: |
||||||
|
peer.log.Trace("Delivered new batch of receipts", "count", len(receipts), "accepted", accepted) |
||||||
|
default: |
||||||
|
peer.log.Debug("Failed to deliver retrieved receipts", "err", err) |
||||||
|
} |
||||||
|
return accepted, err |
||||||
|
} |
@ -1,79 +0,0 @@ |
|||||||
// Copyright 2015 The go-ethereum Authors
|
|
||||||
// This file is part of the go-ethereum library.
|
|
||||||
//
|
|
||||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
|
||||||
// it under the terms of the GNU Lesser General Public License as published by
|
|
||||||
// the Free Software Foundation, either version 3 of the License, or
|
|
||||||
// (at your option) any later version.
|
|
||||||
//
|
|
||||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
|
||||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
|
||||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
|
||||||
// GNU Lesser General Public License for more details.
|
|
||||||
//
|
|
||||||
// You should have received a copy of the GNU Lesser General Public License
|
|
||||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
|
||||||
|
|
||||||
package downloader |
|
||||||
|
|
||||||
import ( |
|
||||||
"fmt" |
|
||||||
|
|
||||||
"github.com/ethereum/go-ethereum/core/types" |
|
||||||
) |
|
||||||
|
|
||||||
// peerDropFn is a callback type for dropping a peer detected as malicious.
|
|
||||||
type peerDropFn func(id string) |
|
||||||
|
|
||||||
// dataPack is a data message returned by a peer for some query.
|
|
||||||
type dataPack interface { |
|
||||||
PeerId() string |
|
||||||
Items() int |
|
||||||
Stats() string |
|
||||||
} |
|
||||||
|
|
||||||
// headerPack is a batch of block headers returned by a peer.
|
|
||||||
type headerPack struct { |
|
||||||
peerID string |
|
||||||
headers []*types.Header |
|
||||||
} |
|
||||||
|
|
||||||
func (p *headerPack) PeerId() string { return p.peerID } |
|
||||||
func (p *headerPack) Items() int { return len(p.headers) } |
|
||||||
func (p *headerPack) Stats() string { return fmt.Sprintf("%d", len(p.headers)) } |
|
||||||
|
|
||||||
// bodyPack is a batch of block bodies returned by a peer.
|
|
||||||
type bodyPack struct { |
|
||||||
peerID string |
|
||||||
transactions [][]*types.Transaction |
|
||||||
uncles [][]*types.Header |
|
||||||
} |
|
||||||
|
|
||||||
func (p *bodyPack) PeerId() string { return p.peerID } |
|
||||||
func (p *bodyPack) Items() int { |
|
||||||
if len(p.transactions) <= len(p.uncles) { |
|
||||||
return len(p.transactions) |
|
||||||
} |
|
||||||
return len(p.uncles) |
|
||||||
} |
|
||||||
func (p *bodyPack) Stats() string { return fmt.Sprintf("%d:%d", len(p.transactions), len(p.uncles)) } |
|
||||||
|
|
||||||
// receiptPack is a batch of receipts returned by a peer.
|
|
||||||
type receiptPack struct { |
|
||||||
peerID string |
|
||||||
receipts [][]*types.Receipt |
|
||||||
} |
|
||||||
|
|
||||||
func (p *receiptPack) PeerId() string { return p.peerID } |
|
||||||
func (p *receiptPack) Items() int { return len(p.receipts) } |
|
||||||
func (p *receiptPack) Stats() string { return fmt.Sprintf("%d", len(p.receipts)) } |
|
||||||
|
|
||||||
// statePack is a batch of states returned by a peer.
|
|
||||||
type statePack struct { |
|
||||||
peerID string |
|
||||||
states [][]byte |
|
||||||
} |
|
||||||
|
|
||||||
func (p *statePack) PeerId() string { return p.peerID } |
|
||||||
func (p *statePack) Items() int { return len(p.states) } |
|
||||||
func (p *statePack) Stats() string { return fmt.Sprintf("%d", len(p.states)) } |
|
@ -0,0 +1,247 @@ |
|||||||
|
// Copyright 2021 The go-ethereum Authors
|
||||||
|
// This file is part of the go-ethereum library.
|
||||||
|
//
|
||||||
|
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||||
|
// it under the terms of the GNU Lesser General Public License as published by
|
||||||
|
// the Free Software Foundation, either version 3 of the License, or
|
||||||
|
// (at your option) any later version.
|
||||||
|
//
|
||||||
|
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||||
|
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||||
|
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||||
|
// GNU Lesser General Public License for more details.
|
||||||
|
//
|
||||||
|
// You should have received a copy of the GNU Lesser General Public License
|
||||||
|
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||||
|
|
||||||
|
package eth |
||||||
|
|
||||||
|
import ( |
||||||
|
"errors" |
||||||
|
"fmt" |
||||||
|
"time" |
||||||
|
|
||||||
|
"github.com/ethereum/go-ethereum/p2p" |
||||||
|
) |
||||||
|
|
||||||
|
var ( |
||||||
|
// errDisconnected is returned if a request is attempted to be made to a peer
|
||||||
|
// that was already closed.
|
||||||
|
errDisconnected = errors.New("disconnected") |
||||||
|
|
||||||
|
// errDanglingResponse is returned if a response arrives with a request id
|
||||||
|
// which does not match to any existing pending requests.
|
||||||
|
errDanglingResponse = errors.New("response to non-existent request") |
||||||
|
|
||||||
|
// errMismatchingResponseType is returned if the remote peer sent a different
|
||||||
|
// packet type as a response to a request than what the local node expected.
|
||||||
|
errMismatchingResponseType = errors.New("mismatching response type") |
||||||
|
) |
||||||
|
|
||||||
|
// Request is a pending request to allow tracking it and delivering a response
|
||||||
|
// back to the requester on their chosen channel.
|
||||||
|
type Request struct { |
||||||
|
peer *Peer // Peer to which this request belogs for untracking
|
||||||
|
id uint64 // Request ID to match up replies to
|
||||||
|
|
||||||
|
sink chan *Response // Channel to deliver the response on
|
||||||
|
cancel chan struct{} // Channel to cancel requests ahead of time
|
||||||
|
|
||||||
|
code uint64 // Message code of the request packet
|
||||||
|
want uint64 // Message code of the response packet
|
||||||
|
data interface{} // Data content of the request packet
|
||||||
|
|
||||||
|
Peer string // Demultiplexer if cross-peer requests are batched together
|
||||||
|
Sent time.Time // Timestamp when the request was sent
|
||||||
|
} |
||||||
|
|
||||||
|
// Close aborts an in-flight request. Although there's no way to notify the
|
||||||
|
// remote peer about the cancellation, this method notifies the dispatcher to
|
||||||
|
// discard any late responses.
|
||||||
|
func (r *Request) Close() error { |
||||||
|
if r.peer == nil { // Tests mock out the dispatcher, skip internal cancellation
|
||||||
|
return nil |
||||||
|
} |
||||||
|
cancelOp := &cancel{ |
||||||
|
id: r.id, |
||||||
|
fail: make(chan error), |
||||||
|
} |
||||||
|
select { |
||||||
|
case r.peer.reqCancel <- cancelOp: |
||||||
|
if err := <-cancelOp.fail; err != nil { |
||||||
|
return err |
||||||
|
} |
||||||
|
close(r.cancel) |
||||||
|
return nil |
||||||
|
case <-r.peer.term: |
||||||
|
return errDisconnected |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// request is a wrapper around a client Request that has an error channel to
|
||||||
|
// signal on if sending the request already failed on a network level.
|
||||||
|
type request struct { |
||||||
|
req *Request |
||||||
|
fail chan error |
||||||
|
} |
||||||
|
|
||||||
|
// cancel is a maintenance type on the dispatcher to stop tracking a pending
|
||||||
|
// request.
|
||||||
|
type cancel struct { |
||||||
|
id uint64 // Request ID to stop tracking
|
||||||
|
fail chan error |
||||||
|
} |
||||||
|
|
||||||
|
// Response is a reply packet to a previously created request. It is delivered
|
||||||
|
// on the channel assigned by the requester subsystem and contains the original
|
||||||
|
// request embedded to allow uniquely matching it caller side.
|
||||||
|
type Response struct { |
||||||
|
id uint64 // Request ID to match up this reply to
|
||||||
|
recv time.Time // Timestamp when the request was received
|
||||||
|
code uint64 // Response packet type to cross validate with request
|
||||||
|
|
||||||
|
Req *Request // Original request to cross-reference with
|
||||||
|
Res interface{} // Remote response for the request query
|
||||||
|
Time time.Duration // Time it took for the request to be served
|
||||||
|
Done chan error // Channel to signal message handling to the reader
|
||||||
|
} |
||||||
|
|
||||||
|
// response is a wrapper around a remote Response that has an error channel to
|
||||||
|
// signal on if processing the response failed.
|
||||||
|
type response struct { |
||||||
|
res *Response |
||||||
|
fail chan error |
||||||
|
} |
||||||
|
|
||||||
|
// dispatchRequest schedules the request to the dispatcher for tracking and
|
||||||
|
// network serialization, blocking until it's successfully sent.
|
||||||
|
//
|
||||||
|
// The returned Request must either be closed before discarding it, or the reply
|
||||||
|
// must be waited for and the Response's Done channel signalled.
|
||||||
|
func (p *Peer) dispatchRequest(req *Request) error { |
||||||
|
reqOp := &request{ |
||||||
|
req: req, |
||||||
|
fail: make(chan error), |
||||||
|
} |
||||||
|
req.cancel = make(chan struct{}) |
||||||
|
req.peer = p |
||||||
|
req.Peer = p.id |
||||||
|
|
||||||
|
select { |
||||||
|
case p.reqDispatch <- reqOp: |
||||||
|
return <-reqOp.fail |
||||||
|
case <-p.term: |
||||||
|
return errDisconnected |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// dispatchRequest fulfils a pending request and delivers it to the requested
|
||||||
|
// sink.
|
||||||
|
func (p *Peer) dispatchResponse(res *Response) error { |
||||||
|
resOp := &response{ |
||||||
|
res: res, |
||||||
|
fail: make(chan error), |
||||||
|
} |
||||||
|
res.recv = time.Now() |
||||||
|
res.Done = make(chan error) |
||||||
|
|
||||||
|
select { |
||||||
|
case p.resDispatch <- resOp: |
||||||
|
// Ensure the response is accepted by the dispatcher
|
||||||
|
if err := <-resOp.fail; err != nil { |
||||||
|
return nil |
||||||
|
} |
||||||
|
// Deliver the filled out response and wait until it's handled. This
|
||||||
|
// path is a bit funky as Go's select has no order, so if a response
|
||||||
|
// arrives to an already cancelled request, there's a 50-50% changes
|
||||||
|
// of picking on channel or the other. To avoid such cases delivering
|
||||||
|
// the packet upstream, check for cancellation first and only after
|
||||||
|
// block on delivery.
|
||||||
|
select { |
||||||
|
case <-res.Req.cancel: |
||||||
|
return nil // Request cancelled, silently discard response
|
||||||
|
default: |
||||||
|
// Request not yet cancelled, attempt to deliver it, but do watch
|
||||||
|
// for fresh cancellations too
|
||||||
|
select { |
||||||
|
case res.Req.sink <- res: |
||||||
|
return <-res.Done // Response delivered, return any errors
|
||||||
|
case <-res.Req.cancel: |
||||||
|
return nil // Request cancelled, silently discard response
|
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
case <-p.term: |
||||||
|
return errDisconnected |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// dispatcher is a loop that accepts requests from higher layer packages, pushes
|
||||||
|
// it to the network and tracks and dispatches the responses back to the original
|
||||||
|
// requester.
|
||||||
|
func (p *Peer) dispatcher() { |
||||||
|
pending := make(map[uint64]*Request) |
||||||
|
|
||||||
|
for { |
||||||
|
select { |
||||||
|
case reqOp := <-p.reqDispatch: |
||||||
|
req := reqOp.req |
||||||
|
req.Sent = time.Now() |
||||||
|
|
||||||
|
requestTracker.Track(p.id, p.version, req.code, req.want, req.id) |
||||||
|
err := p2p.Send(p.rw, req.code, req.data) |
||||||
|
reqOp.fail <- err |
||||||
|
|
||||||
|
if err == nil { |
||||||
|
pending[req.id] = req |
||||||
|
} |
||||||
|
|
||||||
|
case cancelOp := <-p.reqCancel: |
||||||
|
// Retrieve the pendign request to cancel and short circuit if it
|
||||||
|
// has already been serviced and is not available anymore
|
||||||
|
req := pending[cancelOp.id] |
||||||
|
if req == nil { |
||||||
|
cancelOp.fail <- nil |
||||||
|
continue |
||||||
|
} |
||||||
|
// Stop tracking the request
|
||||||
|
delete(pending, cancelOp.id) |
||||||
|
cancelOp.fail <- nil |
||||||
|
|
||||||
|
case resOp := <-p.resDispatch: |
||||||
|
res := resOp.res |
||||||
|
res.Req = pending[res.id] |
||||||
|
|
||||||
|
// Independent if the request exists or not, track this packet
|
||||||
|
requestTracker.Fulfil(p.id, p.version, res.code, res.id) |
||||||
|
|
||||||
|
switch { |
||||||
|
case res.Req == nil: |
||||||
|
// Response arrived with an untracked ID. Since even cancelled
|
||||||
|
// requests are tracked until fulfilment, a dangling repsponse
|
||||||
|
// means the remote peer implements the protocol badly.
|
||||||
|
resOp.fail <- errDanglingResponse |
||||||
|
|
||||||
|
case res.Req.want != res.code: |
||||||
|
// Response arrived, but it's a different packet type than the
|
||||||
|
// one expected by the requester. Either the local code is bad,
|
||||||
|
// or the remote peer send junk. In neither cases can we handle
|
||||||
|
// the packet.
|
||||||
|
resOp.fail <- fmt.Errorf("%w: have %d, want %d", errMismatchingResponseType, res.code, res.Req.want) |
||||||
|
|
||||||
|
default: |
||||||
|
// All dispatcher checks passed and the response was initialized
|
||||||
|
// with the matching request. Signal to the delivery routine that
|
||||||
|
// it can wait for a handler response and dispatch the data.
|
||||||
|
res.Time = res.recv.Sub(res.Req.Sent) |
||||||
|
resOp.fail <- nil |
||||||
|
|
||||||
|
// Stop tracking the request, the response dispatcher will deliver
|
||||||
|
delete(pending, res.id) |
||||||
|
} |
||||||
|
|
||||||
|
case <-p.term: |
||||||
|
return |
||||||
|
} |
||||||
|
} |
||||||
|
} |
File diff suppressed because one or more lines are too long
Loading…
Reference in new issue