eth/downloader: refactor downloader + queue (#21263)
* eth/downloader: refactor downloader + queue downloader, fetcher: throttle-metrics, fetcher filter improvements, standalone resultcache downloader: more accurate deliverytime calculation, less mem overhead in state requests downloader/queue: increase underlying buffer of results, new throttle mechanism eth/downloader: updates to tests eth/downloader: fix up some review concerns eth/downloader/queue: minor fixes eth/downloader: minor fixes after review call eth/downloader: testcases for queue.go eth/downloader: minor change, don't set progress unless progress... eth/downloader: fix flaw which prevented useless peers from being dropped eth/downloader: try to fix tests eth/downloader: verify non-deliveries against advertised remote head eth/downloader: fix flaw with checking closed-status causing hang eth/downloader: hashing avoidance eth/downloader: review concerns + simplify resultcache and queue eth/downloader: add back some locks, address review concerns downloader/queue: fix remaining lock flaw * eth/downloader: nitpick fixes * eth/downloader: remove the *2*3/4 throttling threshold dance * eth/downloader: print correct throttle threshold in stats Co-authored-by: Péter Szilágyi <peterke@gmail.com>
This commit is contained in:
committed by
GitHub
parent
3a57eecc69
commit
105922180f
@@ -23,6 +23,7 @@ import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
@@ -32,6 +33,11 @@ import (
|
||||
"github.com/ethereum/go-ethereum/metrics"
|
||||
)
|
||||
|
||||
const (
|
||||
bodyType = uint(0)
|
||||
receiptType = uint(1)
|
||||
)
|
||||
|
||||
var (
|
||||
blockCacheItems = 8192 // Maximum number of blocks to cache before throttling the download
|
||||
blockCacheMemory = 64 * 1024 * 1024 // Maximum amount of memory to use for block caching
|
||||
@@ -54,8 +60,7 @@ type fetchRequest struct {
|
||||
// fetchResult is a struct collecting partial results from data fetchers until
|
||||
// all outstanding pieces complete and the result as a whole can be processed.
|
||||
type fetchResult struct {
|
||||
Pending int // Number of data fetches still pending
|
||||
Hash common.Hash // Hash of the header to prevent recalculating
|
||||
pending int32 // Flag telling what deliveries are outstanding
|
||||
|
||||
Header *types.Header
|
||||
Uncles []*types.Header
|
||||
@@ -63,6 +68,44 @@ type fetchResult struct {
|
||||
Receipts types.Receipts
|
||||
}
|
||||
|
||||
func newFetchResult(header *types.Header, fastSync bool) *fetchResult {
|
||||
item := &fetchResult{
|
||||
Header: header,
|
||||
}
|
||||
if !header.EmptyBody() {
|
||||
item.pending |= (1 << bodyType)
|
||||
}
|
||||
if fastSync && !header.EmptyReceipts() {
|
||||
item.pending |= (1 << receiptType)
|
||||
}
|
||||
return item
|
||||
}
|
||||
|
||||
// SetBodyDone flags the body as finished.
|
||||
func (f *fetchResult) SetBodyDone() {
|
||||
if v := atomic.LoadInt32(&f.pending); (v & (1 << bodyType)) != 0 {
|
||||
atomic.AddInt32(&f.pending, -1)
|
||||
}
|
||||
}
|
||||
|
||||
// AllDone checks if item is done.
|
||||
func (f *fetchResult) AllDone() bool {
|
||||
return atomic.LoadInt32(&f.pending) == 0
|
||||
}
|
||||
|
||||
// SetReceiptsDone flags the receipts as finished.
|
||||
func (f *fetchResult) SetReceiptsDone() {
|
||||
if v := atomic.LoadInt32(&f.pending); (v & (1 << receiptType)) != 0 {
|
||||
atomic.AddInt32(&f.pending, -2)
|
||||
}
|
||||
}
|
||||
|
||||
// Done checks if the given type is done already
|
||||
func (f *fetchResult) Done(kind uint) bool {
|
||||
v := atomic.LoadInt32(&f.pending)
|
||||
return v&(1<<kind) == 0
|
||||
}
|
||||
|
||||
// queue represents hashes that are either need fetching or are being fetched
|
||||
type queue struct {
|
||||
mode SyncMode // Synchronisation mode to decide on the block parts to schedule for fetching
|
||||
@@ -82,44 +125,37 @@ type queue struct {
|
||||
blockTaskPool map[common.Hash]*types.Header // [eth/62] Pending block (body) retrieval tasks, mapping hashes to headers
|
||||
blockTaskQueue *prque.Prque // [eth/62] Priority queue of the headers to fetch the blocks (bodies) for
|
||||
blockPendPool map[string]*fetchRequest // [eth/62] Currently pending block (body) retrieval operations
|
||||
blockDonePool map[common.Hash]struct{} // [eth/62] Set of the completed block (body) fetches
|
||||
|
||||
receiptTaskPool map[common.Hash]*types.Header // [eth/63] Pending receipt retrieval tasks, mapping hashes to headers
|
||||
receiptTaskQueue *prque.Prque // [eth/63] Priority queue of the headers to fetch the receipts for
|
||||
receiptPendPool map[string]*fetchRequest // [eth/63] Currently pending receipt retrieval operations
|
||||
receiptDonePool map[common.Hash]struct{} // [eth/63] Set of the completed receipt fetches
|
||||
|
||||
resultCache []*fetchResult // Downloaded but not yet delivered fetch results
|
||||
resultOffset uint64 // Offset of the first cached fetch result in the block chain
|
||||
resultSize common.StorageSize // Approximate size of a block (exponential moving average)
|
||||
resultCache *resultStore // Downloaded but not yet delivered fetch results
|
||||
resultSize common.StorageSize // Approximate size of a block (exponential moving average)
|
||||
|
||||
lock *sync.Mutex
|
||||
lock *sync.RWMutex
|
||||
active *sync.Cond
|
||||
closed bool
|
||||
|
||||
lastStatLog time.Time
|
||||
}
|
||||
|
||||
// newQueue creates a new download queue for scheduling block retrieval.
|
||||
func newQueue() *queue {
|
||||
lock := new(sync.Mutex)
|
||||
return &queue{
|
||||
headerPendPool: make(map[string]*fetchRequest),
|
||||
func newQueue(blockCacheLimit int) *queue {
|
||||
lock := new(sync.RWMutex)
|
||||
q := &queue{
|
||||
headerContCh: make(chan bool),
|
||||
blockTaskPool: make(map[common.Hash]*types.Header),
|
||||
blockTaskQueue: prque.New(nil),
|
||||
blockPendPool: make(map[string]*fetchRequest),
|
||||
blockDonePool: make(map[common.Hash]struct{}),
|
||||
receiptTaskPool: make(map[common.Hash]*types.Header),
|
||||
receiptTaskQueue: prque.New(nil),
|
||||
receiptPendPool: make(map[string]*fetchRequest),
|
||||
receiptDonePool: make(map[common.Hash]struct{}),
|
||||
resultCache: make([]*fetchResult, blockCacheItems),
|
||||
active: sync.NewCond(lock),
|
||||
lock: lock,
|
||||
}
|
||||
q.Reset(blockCacheLimit)
|
||||
return q
|
||||
}
|
||||
|
||||
// Reset clears out the queue contents.
|
||||
func (q *queue) Reset() {
|
||||
func (q *queue) Reset(blockCacheLimit int) {
|
||||
q.lock.Lock()
|
||||
defer q.lock.Unlock()
|
||||
|
||||
@@ -132,15 +168,12 @@ func (q *queue) Reset() {
|
||||
q.blockTaskPool = make(map[common.Hash]*types.Header)
|
||||
q.blockTaskQueue.Reset()
|
||||
q.blockPendPool = make(map[string]*fetchRequest)
|
||||
q.blockDonePool = make(map[common.Hash]struct{})
|
||||
|
||||
q.receiptTaskPool = make(map[common.Hash]*types.Header)
|
||||
q.receiptTaskQueue.Reset()
|
||||
q.receiptPendPool = make(map[string]*fetchRequest)
|
||||
q.receiptDonePool = make(map[common.Hash]struct{})
|
||||
|
||||
q.resultCache = make([]*fetchResult, blockCacheItems)
|
||||
q.resultOffset = 0
|
||||
q.resultCache = newResultStore(blockCacheLimit)
|
||||
}
|
||||
|
||||
// Close marks the end of the sync, unblocking Results.
|
||||
@@ -148,8 +181,8 @@ func (q *queue) Reset() {
|
||||
func (q *queue) Close() {
|
||||
q.lock.Lock()
|
||||
q.closed = true
|
||||
q.active.Signal()
|
||||
q.lock.Unlock()
|
||||
q.active.Broadcast()
|
||||
}
|
||||
|
||||
// PendingHeaders retrieves the number of header requests pending for retrieval.
|
||||
@@ -210,58 +243,8 @@ func (q *queue) Idle() bool {
|
||||
|
||||
queued := q.blockTaskQueue.Size() + q.receiptTaskQueue.Size()
|
||||
pending := len(q.blockPendPool) + len(q.receiptPendPool)
|
||||
cached := len(q.blockDonePool) + len(q.receiptDonePool)
|
||||
|
||||
return (queued + pending + cached) == 0
|
||||
}
|
||||
|
||||
// ShouldThrottleBlocks checks if the download should be throttled (active block (body)
|
||||
// fetches exceed block cache).
|
||||
func (q *queue) ShouldThrottleBlocks() bool {
|
||||
q.lock.Lock()
|
||||
defer q.lock.Unlock()
|
||||
|
||||
return q.resultSlots(q.blockPendPool, q.blockDonePool) <= 0
|
||||
}
|
||||
|
||||
// ShouldThrottleReceipts checks if the download should be throttled (active receipt
|
||||
// fetches exceed block cache).
|
||||
func (q *queue) ShouldThrottleReceipts() bool {
|
||||
q.lock.Lock()
|
||||
defer q.lock.Unlock()
|
||||
|
||||
return q.resultSlots(q.receiptPendPool, q.receiptDonePool) <= 0
|
||||
}
|
||||
|
||||
// resultSlots calculates the number of results slots available for requests
|
||||
// whilst adhering to both the item and the memory limits of the result cache.
|
||||
func (q *queue) resultSlots(pendPool map[string]*fetchRequest, donePool map[common.Hash]struct{}) int {
|
||||
// Calculate the maximum length capped by the memory limit
|
||||
limit := len(q.resultCache)
|
||||
if common.StorageSize(len(q.resultCache))*q.resultSize > common.StorageSize(blockCacheMemory) {
|
||||
limit = int((common.StorageSize(blockCacheMemory) + q.resultSize - 1) / q.resultSize)
|
||||
}
|
||||
// Calculate the number of slots already finished
|
||||
finished := 0
|
||||
for _, result := range q.resultCache[:limit] {
|
||||
if result == nil {
|
||||
break
|
||||
}
|
||||
if _, ok := donePool[result.Hash]; ok {
|
||||
finished++
|
||||
}
|
||||
}
|
||||
// Calculate the number of slots currently downloading
|
||||
pending := 0
|
||||
for _, request := range pendPool {
|
||||
for _, header := range request.Headers {
|
||||
if header.Number.Uint64() < q.resultOffset+uint64(limit) {
|
||||
pending++
|
||||
}
|
||||
}
|
||||
}
|
||||
// Return the free slots to distribute
|
||||
return limit - finished - pending
|
||||
return (queued + pending) == 0
|
||||
}
|
||||
|
||||
// ScheduleSkeleton adds a batch of header retrieval tasks to the queue to fill
|
||||
@@ -323,21 +306,22 @@ func (q *queue) Schedule(headers []*types.Header, from uint64) []*types.Header {
|
||||
break
|
||||
}
|
||||
// Make sure no duplicate requests are executed
|
||||
// We cannot skip this, even if the block is empty, since this is
|
||||
// what triggers the fetchResult creation.
|
||||
if _, ok := q.blockTaskPool[hash]; ok {
|
||||
log.Warn("Header already scheduled for block fetch", "number", header.Number, "hash", hash)
|
||||
continue
|
||||
} else {
|
||||
q.blockTaskPool[hash] = header
|
||||
q.blockTaskQueue.Push(header, -int64(header.Number.Uint64()))
|
||||
}
|
||||
if _, ok := q.receiptTaskPool[hash]; ok {
|
||||
log.Warn("Header already scheduled for receipt fetch", "number", header.Number, "hash", hash)
|
||||
continue
|
||||
}
|
||||
// Queue the header for content retrieval
|
||||
q.blockTaskPool[hash] = header
|
||||
q.blockTaskQueue.Push(header, -int64(header.Number.Uint64()))
|
||||
|
||||
if q.mode == FastSync {
|
||||
q.receiptTaskPool[hash] = header
|
||||
q.receiptTaskQueue.Push(header, -int64(header.Number.Uint64()))
|
||||
// Queue for receipt retrieval
|
||||
if q.mode == FastSync && !header.EmptyReceipts() {
|
||||
if _, ok := q.receiptTaskPool[hash]; ok {
|
||||
log.Warn("Header already scheduled for receipt fetch", "number", header.Number, "hash", hash)
|
||||
} else {
|
||||
q.receiptTaskPool[hash] = header
|
||||
q.receiptTaskQueue.Push(header, -int64(header.Number.Uint64()))
|
||||
}
|
||||
}
|
||||
inserts = append(inserts, header)
|
||||
q.headerHead = hash
|
||||
@@ -347,67 +331,78 @@ func (q *queue) Schedule(headers []*types.Header, from uint64) []*types.Header {
|
||||
}
|
||||
|
||||
// Results retrieves and permanently removes a batch of fetch results from
|
||||
// the cache. The result slice will be empty if the queue has been closed.
|
||||
// the cache. the result slice will be empty if the queue has been closed.
|
||||
// Results can be called concurrently with Deliver and Schedule,
|
||||
// but assumes that there are not two simultaneous callers to Results
|
||||
func (q *queue) Results(block bool) []*fetchResult {
|
||||
q.lock.Lock()
|
||||
defer q.lock.Unlock()
|
||||
|
||||
// Count the number of items available for processing
|
||||
nproc := q.countProcessableItems()
|
||||
for nproc == 0 && !q.closed {
|
||||
if !block {
|
||||
return nil
|
||||
// Abort early if there are no items and non-blocking requested
|
||||
if !block && !q.resultCache.HasCompletedItems() {
|
||||
return nil
|
||||
}
|
||||
closed := false
|
||||
for !closed && !q.resultCache.HasCompletedItems() {
|
||||
// In order to wait on 'active', we need to obtain the lock.
|
||||
// That may take a while, if someone is delivering at the same
|
||||
// time, so after obtaining the lock, we check again if there
|
||||
// are any results to fetch.
|
||||
// Also, in-between we ask for the lock and the lock is obtained,
|
||||
// someone can have closed the queue. In that case, we should
|
||||
// return the available results and stop blocking
|
||||
q.lock.Lock()
|
||||
if q.resultCache.HasCompletedItems() || q.closed {
|
||||
q.lock.Unlock()
|
||||
break
|
||||
}
|
||||
// No items available, and not closed
|
||||
q.active.Wait()
|
||||
nproc = q.countProcessableItems()
|
||||
closed = q.closed
|
||||
q.lock.Unlock()
|
||||
}
|
||||
// Since we have a batch limit, don't pull more into "dangling" memory
|
||||
if nproc > maxResultsProcess {
|
||||
nproc = maxResultsProcess
|
||||
}
|
||||
results := make([]*fetchResult, nproc)
|
||||
copy(results, q.resultCache[:nproc])
|
||||
if len(results) > 0 {
|
||||
// Mark results as done before dropping them from the cache.
|
||||
for _, result := range results {
|
||||
hash := result.Header.Hash()
|
||||
delete(q.blockDonePool, hash)
|
||||
delete(q.receiptDonePool, hash)
|
||||
}
|
||||
// Delete the results from the cache and clear the tail.
|
||||
copy(q.resultCache, q.resultCache[nproc:])
|
||||
for i := len(q.resultCache) - nproc; i < len(q.resultCache); i++ {
|
||||
q.resultCache[i] = nil
|
||||
}
|
||||
// Advance the expected block number of the first cache entry.
|
||||
q.resultOffset += uint64(nproc)
|
||||
|
||||
// Regardless if closed or not, we can still deliver whatever we have
|
||||
results := q.resultCache.GetCompleted(maxResultsProcess)
|
||||
for _, result := range results {
|
||||
// Recalculate the result item weights to prevent memory exhaustion
|
||||
for _, result := range results {
|
||||
size := result.Header.Size()
|
||||
for _, uncle := range result.Uncles {
|
||||
size += uncle.Size()
|
||||
}
|
||||
for _, receipt := range result.Receipts {
|
||||
size += receipt.Size()
|
||||
}
|
||||
for _, tx := range result.Transactions {
|
||||
size += tx.Size()
|
||||
}
|
||||
q.resultSize = common.StorageSize(blockCacheSizeWeight)*size + (1-common.StorageSize(blockCacheSizeWeight))*q.resultSize
|
||||
size := result.Header.Size()
|
||||
for _, uncle := range result.Uncles {
|
||||
size += uncle.Size()
|
||||
}
|
||||
for _, receipt := range result.Receipts {
|
||||
size += receipt.Size()
|
||||
}
|
||||
for _, tx := range result.Transactions {
|
||||
size += tx.Size()
|
||||
}
|
||||
q.resultSize = common.StorageSize(blockCacheSizeWeight)*size +
|
||||
(1-common.StorageSize(blockCacheSizeWeight))*q.resultSize
|
||||
}
|
||||
// Using the newly calibrated resultsize, figure out the new throttle limit
|
||||
// on the result cache
|
||||
throttleThreshold := uint64((common.StorageSize(blockCacheMemory) + q.resultSize - 1) / q.resultSize)
|
||||
throttleThreshold = q.resultCache.SetThrottleThreshold(throttleThreshold)
|
||||
|
||||
// Log some info at certain times
|
||||
if time.Since(q.lastStatLog) > 10*time.Second {
|
||||
q.lastStatLog = time.Now()
|
||||
info := q.Stats()
|
||||
info = append(info, "throttle", throttleThreshold)
|
||||
log.Info("Downloader queue stats", info...)
|
||||
}
|
||||
return results
|
||||
}
|
||||
|
||||
// countProcessableItems counts the processable items.
|
||||
func (q *queue) countProcessableItems() int {
|
||||
for i, result := range q.resultCache {
|
||||
if result == nil || result.Pending > 0 {
|
||||
return i
|
||||
}
|
||||
func (q *queue) Stats() []interface{} {
|
||||
q.lock.RLock()
|
||||
defer q.lock.RUnlock()
|
||||
|
||||
return q.stats()
|
||||
}
|
||||
|
||||
func (q *queue) stats() []interface{} {
|
||||
return []interface{}{
|
||||
"receiptTasks", q.receiptTaskQueue.Size(),
|
||||
"blockTasks", q.blockTaskQueue.Size(),
|
||||
"itemSize", q.resultSize,
|
||||
}
|
||||
return len(q.resultCache)
|
||||
}
|
||||
|
||||
// ReserveHeaders reserves a set of headers for the given peer, skipping any
|
||||
@@ -453,27 +448,21 @@ func (q *queue) ReserveHeaders(p *peerConnection, count int) *fetchRequest {
|
||||
// ReserveBodies reserves a set of body fetches for the given peer, skipping any
|
||||
// previously failed downloads. Beside the next batch of needed fetches, it also
|
||||
// returns a flag whether empty blocks were queued requiring processing.
|
||||
func (q *queue) ReserveBodies(p *peerConnection, count int) (*fetchRequest, bool, error) {
|
||||
isNoop := func(header *types.Header) bool {
|
||||
return header.TxHash == types.EmptyRootHash && header.UncleHash == types.EmptyUncleHash
|
||||
}
|
||||
func (q *queue) ReserveBodies(p *peerConnection, count int) (*fetchRequest, bool, bool) {
|
||||
q.lock.Lock()
|
||||
defer q.lock.Unlock()
|
||||
|
||||
return q.reserveHeaders(p, count, q.blockTaskPool, q.blockTaskQueue, q.blockPendPool, q.blockDonePool, isNoop)
|
||||
return q.reserveHeaders(p, count, q.blockTaskPool, q.blockTaskQueue, q.blockPendPool, bodyType)
|
||||
}
|
||||
|
||||
// ReserveReceipts reserves a set of receipt fetches for the given peer, skipping
|
||||
// any previously failed downloads. Beside the next batch of needed fetches, it
|
||||
// also returns a flag whether empty receipts were queued requiring importing.
|
||||
func (q *queue) ReserveReceipts(p *peerConnection, count int) (*fetchRequest, bool, error) {
|
||||
isNoop := func(header *types.Header) bool {
|
||||
return header.ReceiptHash == types.EmptyRootHash
|
||||
}
|
||||
func (q *queue) ReserveReceipts(p *peerConnection, count int) (*fetchRequest, bool, bool) {
|
||||
q.lock.Lock()
|
||||
defer q.lock.Unlock()
|
||||
|
||||
return q.reserveHeaders(p, count, q.receiptTaskPool, q.receiptTaskQueue, q.receiptPendPool, q.receiptDonePool, isNoop)
|
||||
return q.reserveHeaders(p, count, q.receiptTaskPool, q.receiptTaskQueue, q.receiptPendPool, receiptType)
|
||||
}
|
||||
|
||||
// reserveHeaders reserves a set of data download operations for a given peer,
|
||||
@@ -483,57 +472,71 @@ func (q *queue) ReserveReceipts(p *peerConnection, count int) (*fetchRequest, bo
|
||||
// Note, this method expects the queue lock to be already held for writing. The
|
||||
// reason the lock is not obtained in here is because the parameters already need
|
||||
// to access the queue, so they already need a lock anyway.
|
||||
//
|
||||
// Returns:
|
||||
// item - the fetchRequest
|
||||
// progress - whether any progress was made
|
||||
// throttle - if the caller should throttle for a while
|
||||
func (q *queue) reserveHeaders(p *peerConnection, count int, taskPool map[common.Hash]*types.Header, taskQueue *prque.Prque,
|
||||
pendPool map[string]*fetchRequest, donePool map[common.Hash]struct{}, isNoop func(*types.Header) bool) (*fetchRequest, bool, error) {
|
||||
pendPool map[string]*fetchRequest, kind uint) (*fetchRequest, bool, bool) {
|
||||
// Short circuit if the pool has been depleted, or if the peer's already
|
||||
// downloading something (sanity check not to corrupt state)
|
||||
if taskQueue.Empty() {
|
||||
return nil, false, nil
|
||||
return nil, false, true
|
||||
}
|
||||
if _, ok := pendPool[p.id]; ok {
|
||||
return nil, false, nil
|
||||
return nil, false, false
|
||||
}
|
||||
// Calculate an upper limit on the items we might fetch (i.e. throttling)
|
||||
space := q.resultSlots(pendPool, donePool)
|
||||
|
||||
// Retrieve a batch of tasks, skipping previously failed ones
|
||||
send := make([]*types.Header, 0, count)
|
||||
skip := make([]*types.Header, 0)
|
||||
|
||||
progress := false
|
||||
for proc := 0; proc < space && len(send) < count && !taskQueue.Empty(); proc++ {
|
||||
header := taskQueue.PopItem().(*types.Header)
|
||||
hash := header.Hash()
|
||||
throttled := false
|
||||
for proc := 0; len(send) < count && !taskQueue.Empty(); proc++ {
|
||||
// the task queue will pop items in order, so the highest prio block
|
||||
// is also the lowest block number.
|
||||
h, _ := taskQueue.Peek()
|
||||
header := h.(*types.Header)
|
||||
// we can ask the resultcache if this header is within the
|
||||
// "prioritized" segment of blocks. If it is not, we need to throttle
|
||||
|
||||
// If we're the first to request this task, initialise the result container
|
||||
index := int(header.Number.Int64() - int64(q.resultOffset))
|
||||
if index >= len(q.resultCache) || index < 0 {
|
||||
common.Report("index allocation went beyond available resultCache space")
|
||||
return nil, false, fmt.Errorf("%w: index allocation went beyond available resultCache space", errInvalidChain)
|
||||
stale, throttle, item, err := q.resultCache.AddFetch(header, q.mode == FastSync)
|
||||
if stale {
|
||||
// Don't put back in the task queue, this item has already been
|
||||
// delivered upstream
|
||||
taskQueue.PopItem()
|
||||
progress = true
|
||||
delete(taskPool, header.Hash())
|
||||
proc = proc - 1
|
||||
log.Error("Fetch reservation already delivered", "number", header.Number.Uint64())
|
||||
continue
|
||||
}
|
||||
if q.resultCache[index] == nil {
|
||||
components := 1
|
||||
if q.mode == FastSync {
|
||||
components = 2
|
||||
}
|
||||
q.resultCache[index] = &fetchResult{
|
||||
Pending: components,
|
||||
Hash: hash,
|
||||
Header: header,
|
||||
}
|
||||
if throttle {
|
||||
// There are no resultslots available. Leave it in the task queue
|
||||
// However, if there are any left as 'skipped', we should not tell
|
||||
// the caller to throttle, since we still want some other
|
||||
// peer to fetch those for us
|
||||
throttled = len(skip) == 0
|
||||
break
|
||||
}
|
||||
// If this fetch task is a noop, skip this fetch operation
|
||||
if isNoop(header) {
|
||||
donePool[hash] = struct{}{}
|
||||
delete(taskPool, hash)
|
||||
|
||||
space, proc = space-1, proc-1
|
||||
q.resultCache[index].Pending--
|
||||
if err != nil {
|
||||
// this most definitely should _not_ happen
|
||||
log.Warn("Failed to reserve headers", "err", err)
|
||||
// There are no resultslots available. Leave it in the task queue
|
||||
break
|
||||
}
|
||||
if item.Done(kind) {
|
||||
// If it's a noop, we can skip this task
|
||||
delete(taskPool, header.Hash())
|
||||
taskQueue.PopItem()
|
||||
proc = proc - 1
|
||||
progress = true
|
||||
continue
|
||||
}
|
||||
// Remove it from the task queue
|
||||
taskQueue.PopItem()
|
||||
// Otherwise unless the peer is known not to have the data, add to the retrieve list
|
||||
if p.Lacks(hash) {
|
||||
if p.Lacks(header.Hash()) {
|
||||
skip = append(skip, header)
|
||||
} else {
|
||||
send = append(send, header)
|
||||
@@ -543,13 +546,13 @@ func (q *queue) reserveHeaders(p *peerConnection, count int, taskPool map[common
|
||||
for _, header := range skip {
|
||||
taskQueue.Push(header, -int64(header.Number.Uint64()))
|
||||
}
|
||||
if progress {
|
||||
if q.resultCache.HasCompletedItems() {
|
||||
// Wake Results, resultCache was modified
|
||||
q.active.Signal()
|
||||
}
|
||||
// Assemble and return the block download request
|
||||
if len(send) == 0 {
|
||||
return nil, progress, nil
|
||||
return nil, progress, throttled
|
||||
}
|
||||
request := &fetchRequest{
|
||||
Peer: p,
|
||||
@@ -557,8 +560,7 @@ func (q *queue) reserveHeaders(p *peerConnection, count int, taskPool map[common
|
||||
Time: time.Now(),
|
||||
}
|
||||
pendPool[p.id] = request
|
||||
|
||||
return request, progress, nil
|
||||
return request, progress, throttled
|
||||
}
|
||||
|
||||
// CancelHeaders aborts a fetch request, returning all pending skeleton indexes to the queue.
|
||||
@@ -768,16 +770,23 @@ func (q *queue) DeliverHeaders(id string, headers []*types.Header, headerProcCh
|
||||
func (q *queue) DeliverBodies(id string, txLists [][]*types.Transaction, uncleLists [][]*types.Header) (int, error) {
|
||||
q.lock.Lock()
|
||||
defer q.lock.Unlock()
|
||||
|
||||
reconstruct := func(header *types.Header, index int, result *fetchResult) error {
|
||||
if types.DeriveSha(types.Transactions(txLists[index])) != header.TxHash || types.CalcUncleHash(uncleLists[index]) != header.UncleHash {
|
||||
validate := func(index int, header *types.Header) error {
|
||||
if types.DeriveSha(types.Transactions(txLists[index])) != header.TxHash {
|
||||
return errInvalidBody
|
||||
}
|
||||
if types.CalcUncleHash(uncleLists[index]) != header.UncleHash {
|
||||
return errInvalidBody
|
||||
}
|
||||
result.Transactions = txLists[index]
|
||||
result.Uncles = uncleLists[index]
|
||||
return nil
|
||||
}
|
||||
return q.deliver(id, q.blockTaskPool, q.blockTaskQueue, q.blockPendPool, q.blockDonePool, bodyReqTimer, len(txLists), reconstruct)
|
||||
|
||||
reconstruct := func(index int, result *fetchResult) {
|
||||
result.Transactions = txLists[index]
|
||||
result.Uncles = uncleLists[index]
|
||||
result.SetBodyDone()
|
||||
}
|
||||
return q.deliver(id, q.blockTaskPool, q.blockTaskQueue, q.blockPendPool,
|
||||
bodyReqTimer, len(txLists), validate, reconstruct)
|
||||
}
|
||||
|
||||
// DeliverReceipts injects a receipt retrieval response into the results queue.
|
||||
@@ -786,25 +795,29 @@ func (q *queue) DeliverBodies(id string, txLists [][]*types.Transaction, uncleLi
|
||||
func (q *queue) DeliverReceipts(id string, receiptList [][]*types.Receipt) (int, error) {
|
||||
q.lock.Lock()
|
||||
defer q.lock.Unlock()
|
||||
|
||||
reconstruct := func(header *types.Header, index int, result *fetchResult) error {
|
||||
validate := func(index int, header *types.Header) error {
|
||||
if types.DeriveSha(types.Receipts(receiptList[index])) != header.ReceiptHash {
|
||||
return errInvalidReceipt
|
||||
}
|
||||
result.Receipts = receiptList[index]
|
||||
return nil
|
||||
}
|
||||
return q.deliver(id, q.receiptTaskPool, q.receiptTaskQueue, q.receiptPendPool, q.receiptDonePool, receiptReqTimer, len(receiptList), reconstruct)
|
||||
reconstruct := func(index int, result *fetchResult) {
|
||||
result.Receipts = receiptList[index]
|
||||
result.SetReceiptsDone()
|
||||
}
|
||||
return q.deliver(id, q.receiptTaskPool, q.receiptTaskQueue, q.receiptPendPool,
|
||||
receiptReqTimer, len(receiptList), validate, reconstruct)
|
||||
}
|
||||
|
||||
// deliver injects a data retrieval response into the results queue.
|
||||
//
|
||||
// Note, this method expects the queue lock to be already held for writing. The
|
||||
// reason the lock is not obtained in here is because the parameters already need
|
||||
// reason this lock is not obtained in here is because the parameters already need
|
||||
// to access the queue, so they already need a lock anyway.
|
||||
func (q *queue) deliver(id string, taskPool map[common.Hash]*types.Header, taskQueue *prque.Prque,
|
||||
pendPool map[string]*fetchRequest, donePool map[common.Hash]struct{}, reqTimer metrics.Timer,
|
||||
results int, reconstruct func(header *types.Header, index int, result *fetchResult) error) (int, error) {
|
||||
func (q *queue) deliver(id string, taskPool map[common.Hash]*types.Header,
|
||||
taskQueue *prque.Prque, pendPool map[string]*fetchRequest, reqTimer metrics.Timer,
|
||||
results int, validate func(index int, header *types.Header) error,
|
||||
reconstruct func(index int, result *fetchResult)) (int, error) {
|
||||
|
||||
// Short circuit if the data was never requested
|
||||
request := pendPool[id]
|
||||
@@ -824,52 +837,53 @@ func (q *queue) deliver(id string, taskPool map[common.Hash]*types.Header, taskQ
|
||||
var (
|
||||
accepted int
|
||||
failure error
|
||||
useful bool
|
||||
i int
|
||||
hashes []common.Hash
|
||||
)
|
||||
for i, header := range request.Headers {
|
||||
for _, header := range request.Headers {
|
||||
// Short circuit assembly if no more fetch results are found
|
||||
if i >= results {
|
||||
break
|
||||
}
|
||||
// Reconstruct the next result if contents match up
|
||||
index := int(header.Number.Int64() - int64(q.resultOffset))
|
||||
if index >= len(q.resultCache) || index < 0 || q.resultCache[index] == nil {
|
||||
failure = errInvalidChain
|
||||
break
|
||||
}
|
||||
if err := reconstruct(header, i, q.resultCache[index]); err != nil {
|
||||
// Validate the fields
|
||||
if err := validate(i, header); err != nil {
|
||||
failure = err
|
||||
break
|
||||
}
|
||||
hash := header.Hash()
|
||||
|
||||
donePool[hash] = struct{}{}
|
||||
q.resultCache[index].Pending--
|
||||
useful = true
|
||||
accepted++
|
||||
hashes = append(hashes, header.Hash())
|
||||
i++
|
||||
}
|
||||
|
||||
for _, header := range request.Headers[:i] {
|
||||
if res, stale, err := q.resultCache.GetDeliverySlot(header.Number.Uint64()); err == nil {
|
||||
reconstruct(accepted, res)
|
||||
} else {
|
||||
// else: betweeen here and above, some other peer filled this result,
|
||||
// or it was indeed a no-op. This should not happen, but if it does it's
|
||||
// not something to panic about
|
||||
log.Error("Delivery stale", "stale", stale, "number", header.Number.Uint64(), "err", err)
|
||||
failure = errStaleDelivery
|
||||
}
|
||||
// Clean up a successful fetch
|
||||
request.Headers[i] = nil
|
||||
delete(taskPool, hash)
|
||||
delete(taskPool, hashes[accepted])
|
||||
accepted++
|
||||
}
|
||||
// Return all failed or missing fetches to the queue
|
||||
for _, header := range request.Headers {
|
||||
if header != nil {
|
||||
taskQueue.Push(header, -int64(header.Number.Uint64()))
|
||||
}
|
||||
for _, header := range request.Headers[accepted:] {
|
||||
taskQueue.Push(header, -int64(header.Number.Uint64()))
|
||||
}
|
||||
// Wake up Results
|
||||
if accepted > 0 {
|
||||
q.active.Signal()
|
||||
}
|
||||
// If none of the data was good, it's a stale delivery
|
||||
if failure == nil {
|
||||
return accepted, nil
|
||||
}
|
||||
// If none of the data was good, it's a stale delivery
|
||||
if errors.Is(failure, errInvalidChain) {
|
||||
return accepted, failure
|
||||
}
|
||||
if useful {
|
||||
if accepted > 0 {
|
||||
return accepted, fmt.Errorf("partial failure: %v", failure)
|
||||
}
|
||||
return accepted, fmt.Errorf("%w: %v", failure, errStaleDelivery)
|
||||
@@ -882,8 +896,6 @@ func (q *queue) Prepare(offset uint64, mode SyncMode) {
|
||||
defer q.lock.Unlock()
|
||||
|
||||
// Prepare the queue for sync results
|
||||
if q.resultOffset < offset {
|
||||
q.resultOffset = offset
|
||||
}
|
||||
q.resultCache.Prepare(offset)
|
||||
q.mode = mode
|
||||
}
|
||||
|
Reference in New Issue
Block a user