queue.go 31 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887
  1. // Copyright 2015 The go-ethereum Authors
  2. // This file is part of the go-ethereum library.
  3. //
  4. // The go-ethereum library is free software: you can redistribute it and/or modify
  5. // it under the terms of the GNU Lesser General Public License as published by
  6. // the Free Software Foundation, either version 3 of the License, or
  7. // (at your option) any later version.
  8. //
  9. // The go-ethereum library is distributed in the hope that it will be useful,
  10. // but WITHOUT ANY WARRANTY; without even the implied warranty of
  11. // MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
  12. // GNU Lesser General Public License for more details.
  13. //
  14. // You should have received a copy of the GNU Lesser General Public License
  15. // along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
  16. // Contains the block download scheduler to collect download tasks and schedule
  17. // them in an ordered, and throttled way.
  18. package downloader
  19. import (
  20. "errors"
  21. "fmt"
  22. "sync"
  23. "time"
  24. "github.com/ethereum/go-ethereum/common"
  25. "github.com/ethereum/go-ethereum/core/types"
  26. "github.com/ethereum/go-ethereum/log"
  27. "github.com/ethereum/go-ethereum/metrics"
  28. "gopkg.in/karalabe/cookiejar.v2/collections/prque"
  29. )
  30. var (
  31. blockCacheItems = 8192 // Maximum number of blocks to cache before throttling the download
  32. blockCacheMemory = 64 * 1024 * 1024 // Maximum amount of memory to use for block caching
  33. blockCacheSizeWeight = 0.1 // Multiplier to approximate the average block size based on past ones
  34. )
  35. var (
  36. errNoFetchesPending = errors.New("no fetches pending")
  37. errStaleDelivery = errors.New("stale delivery")
  38. )
  39. // fetchRequest is a currently running data retrieval operation.
  40. type fetchRequest struct {
  41. Peer *peerConnection // Peer to which the request was sent
  42. From uint64 // [eth/62] Requested chain element index (used for skeleton fills only)
  43. Headers []*types.Header // [eth/62] Requested headers, sorted by request order
  44. Time time.Time // Time when the request was made
  45. }
  46. // fetchResult is a struct collecting partial results from data fetchers until
  47. // all outstanding pieces complete and the result as a whole can be processed.
  48. type fetchResult struct {
  49. Pending int // Number of data fetches still pending
  50. Hash common.Hash // Hash of the header to prevent recalculating
  51. Header *types.Header
  52. Uncles []*types.Header
  53. Transactions types.Transactions
  54. Receipts types.Receipts
  55. }
  56. // queue represents hashes that are either need fetching or are being fetched
  57. type queue struct {
  58. mode SyncMode // Synchronisation mode to decide on the block parts to schedule for fetching
  59. // Headers are "special", they download in batches, supported by a skeleton chain
  60. headerHead common.Hash // [eth/62] Hash of the last queued header to verify order
  61. headerTaskPool map[uint64]*types.Header // [eth/62] Pending header retrieval tasks, mapping starting indexes to skeleton headers
  62. headerTaskQueue *prque.Prque // [eth/62] Priority queue of the skeleton indexes to fetch the filling headers for
  63. headerPeerMiss map[string]map[uint64]struct{} // [eth/62] Set of per-peer header batches known to be unavailable
  64. headerPendPool map[string]*fetchRequest // [eth/62] Currently pending header retrieval operations
  65. headerResults []*types.Header // [eth/62] Result cache accumulating the completed headers
  66. headerProced int // [eth/62] Number of headers already processed from the results
  67. headerOffset uint64 // [eth/62] Number of the first header in the result cache
  68. headerContCh chan bool // [eth/62] Channel to notify when header download finishes
  69. // All data retrievals below are based on an already assembles header chain
  70. blockTaskPool map[common.Hash]*types.Header // [eth/62] Pending block (body) retrieval tasks, mapping hashes to headers
  71. blockTaskQueue *prque.Prque // [eth/62] Priority queue of the headers to fetch the blocks (bodies) for
  72. blockPendPool map[string]*fetchRequest // [eth/62] Currently pending block (body) retrieval operations
  73. blockDonePool map[common.Hash]struct{} // [eth/62] Set of the completed block (body) fetches
  74. receiptTaskPool map[common.Hash]*types.Header // [eth/63] Pending receipt retrieval tasks, mapping hashes to headers
  75. receiptTaskQueue *prque.Prque // [eth/63] Priority queue of the headers to fetch the receipts for
  76. receiptPendPool map[string]*fetchRequest // [eth/63] Currently pending receipt retrieval operations
  77. receiptDonePool map[common.Hash]struct{} // [eth/63] Set of the completed receipt fetches
  78. resultCache []*fetchResult // Downloaded but not yet delivered fetch results
  79. resultOffset uint64 // Offset of the first cached fetch result in the block chain
  80. resultSize common.StorageSize // Approximate size of a block (exponential moving average)
  81. lock *sync.Mutex
  82. active *sync.Cond
  83. closed bool
  84. }
  85. // newQueue creates a new download queue for scheduling block retrieval.
  86. func newQueue() *queue {
  87. lock := new(sync.Mutex)
  88. return &queue{
  89. headerPendPool: make(map[string]*fetchRequest),
  90. headerContCh: make(chan bool),
  91. blockTaskPool: make(map[common.Hash]*types.Header),
  92. blockTaskQueue: prque.New(),
  93. blockPendPool: make(map[string]*fetchRequest),
  94. blockDonePool: make(map[common.Hash]struct{}),
  95. receiptTaskPool: make(map[common.Hash]*types.Header),
  96. receiptTaskQueue: prque.New(),
  97. receiptPendPool: make(map[string]*fetchRequest),
  98. receiptDonePool: make(map[common.Hash]struct{}),
  99. resultCache: make([]*fetchResult, blockCacheItems),
  100. active: sync.NewCond(lock),
  101. lock: lock,
  102. }
  103. }
  104. // Reset clears out the queue contents.
  105. func (q *queue) Reset() {
  106. q.lock.Lock()
  107. defer q.lock.Unlock()
  108. q.closed = false
  109. q.mode = FullSync
  110. q.headerHead = common.Hash{}
  111. q.headerPendPool = make(map[string]*fetchRequest)
  112. q.blockTaskPool = make(map[common.Hash]*types.Header)
  113. q.blockTaskQueue.Reset()
  114. q.blockPendPool = make(map[string]*fetchRequest)
  115. q.blockDonePool = make(map[common.Hash]struct{})
  116. q.receiptTaskPool = make(map[common.Hash]*types.Header)
  117. q.receiptTaskQueue.Reset()
  118. q.receiptPendPool = make(map[string]*fetchRequest)
  119. q.receiptDonePool = make(map[common.Hash]struct{})
  120. q.resultCache = make([]*fetchResult, blockCacheItems)
  121. q.resultOffset = 0
  122. }
  123. // Close marks the end of the sync, unblocking WaitResults.
  124. // It may be called even if the queue is already closed.
  125. func (q *queue) Close() {
  126. q.lock.Lock()
  127. q.closed = true
  128. q.lock.Unlock()
  129. q.active.Broadcast()
  130. }
  131. // PendingHeaders retrieves the number of header requests pending for retrieval.
  132. func (q *queue) PendingHeaders() int {
  133. q.lock.Lock()
  134. defer q.lock.Unlock()
  135. return q.headerTaskQueue.Size()
  136. }
  137. // PendingBlocks retrieves the number of block (body) requests pending for retrieval.
  138. func (q *queue) PendingBlocks() int {
  139. q.lock.Lock()
  140. defer q.lock.Unlock()
  141. return q.blockTaskQueue.Size()
  142. }
  143. // PendingReceipts retrieves the number of block receipts pending for retrieval.
  144. func (q *queue) PendingReceipts() int {
  145. q.lock.Lock()
  146. defer q.lock.Unlock()
  147. return q.receiptTaskQueue.Size()
  148. }
  149. // InFlightHeaders retrieves whether there are header fetch requests currently
  150. // in flight.
  151. func (q *queue) InFlightHeaders() bool {
  152. q.lock.Lock()
  153. defer q.lock.Unlock()
  154. return len(q.headerPendPool) > 0
  155. }
  156. // InFlightBlocks retrieves whether there are block fetch requests currently in
  157. // flight.
  158. func (q *queue) InFlightBlocks() bool {
  159. q.lock.Lock()
  160. defer q.lock.Unlock()
  161. return len(q.blockPendPool) > 0
  162. }
  163. // InFlightReceipts retrieves whether there are receipt fetch requests currently
  164. // in flight.
  165. func (q *queue) InFlightReceipts() bool {
  166. q.lock.Lock()
  167. defer q.lock.Unlock()
  168. return len(q.receiptPendPool) > 0
  169. }
  170. // Idle returns if the queue is fully idle or has some data still inside.
  171. func (q *queue) Idle() bool {
  172. q.lock.Lock()
  173. defer q.lock.Unlock()
  174. queued := q.blockTaskQueue.Size() + q.receiptTaskQueue.Size()
  175. pending := len(q.blockPendPool) + len(q.receiptPendPool)
  176. cached := len(q.blockDonePool) + len(q.receiptDonePool)
  177. return (queued + pending + cached) == 0
  178. }
  179. // ShouldThrottleBlocks checks if the download should be throttled (active block (body)
  180. // fetches exceed block cache).
  181. func (q *queue) ShouldThrottleBlocks() bool {
  182. q.lock.Lock()
  183. defer q.lock.Unlock()
  184. return q.resultSlots(q.blockPendPool, q.blockDonePool) <= 0
  185. }
  186. // ShouldThrottleReceipts checks if the download should be throttled (active receipt
  187. // fetches exceed block cache).
  188. func (q *queue) ShouldThrottleReceipts() bool {
  189. q.lock.Lock()
  190. defer q.lock.Unlock()
  191. return q.resultSlots(q.receiptPendPool, q.receiptDonePool) <= 0
  192. }
  193. // resultSlots calculates the number of results slots available for requests
  194. // whilst adhering to both the item and the memory limit too of the results
  195. // cache.
  196. func (q *queue) resultSlots(pendPool map[string]*fetchRequest, donePool map[common.Hash]struct{}) int {
  197. // Calculate the maximum length capped by the memory limit
  198. limit := len(q.resultCache)
  199. if common.StorageSize(len(q.resultCache))*q.resultSize > common.StorageSize(blockCacheMemory) {
  200. limit = int((common.StorageSize(blockCacheMemory) + q.resultSize - 1) / q.resultSize)
  201. }
  202. // Calculate the number of slots already finished
  203. finished := 0
  204. for _, result := range q.resultCache[:limit] {
  205. if result == nil {
  206. break
  207. }
  208. if _, ok := donePool[result.Hash]; ok {
  209. finished++
  210. }
  211. }
  212. // Calculate the number of slots currently downloading
  213. pending := 0
  214. for _, request := range pendPool {
  215. for _, header := range request.Headers {
  216. if header.Number.Uint64() < q.resultOffset+uint64(limit) {
  217. pending++
  218. }
  219. }
  220. }
  221. // Return the free slots to distribute
  222. return limit - finished - pending
  223. }
  224. // ScheduleSkeleton adds a batch of header retrieval tasks to the queue to fill
  225. // up an already retrieved header skeleton.
  226. func (q *queue) ScheduleSkeleton(from uint64, skeleton []*types.Header) {
  227. q.lock.Lock()
  228. defer q.lock.Unlock()
  229. // No skeleton retrieval can be in progress, fail hard if so (huge implementation bug)
  230. if q.headerResults != nil {
  231. panic("skeleton assembly already in progress")
  232. }
  233. // Schedule all the header retrieval tasks for the skeleton assembly
  234. q.headerTaskPool = make(map[uint64]*types.Header)
  235. q.headerTaskQueue = prque.New()
  236. q.headerPeerMiss = make(map[string]map[uint64]struct{}) // Reset availability to correct invalid chains
  237. q.headerResults = make([]*types.Header, len(skeleton)*MaxHeaderFetch)
  238. q.headerProced = 0
  239. q.headerOffset = from
  240. q.headerContCh = make(chan bool, 1)
  241. for i, header := range skeleton {
  242. index := from + uint64(i*MaxHeaderFetch)
  243. q.headerTaskPool[index] = header
  244. q.headerTaskQueue.Push(index, -float32(index))
  245. }
  246. }
  247. // RetrieveHeaders retrieves the header chain assemble based on the scheduled
  248. // skeleton.
  249. func (q *queue) RetrieveHeaders() ([]*types.Header, int) {
  250. q.lock.Lock()
  251. defer q.lock.Unlock()
  252. headers, proced := q.headerResults, q.headerProced
  253. q.headerResults, q.headerProced = nil, 0
  254. return headers, proced
  255. }
  256. // Schedule adds a set of headers for the download queue for scheduling, returning
  257. // the new headers encountered.
  258. func (q *queue) Schedule(headers []*types.Header, from uint64) []*types.Header {
  259. q.lock.Lock()
  260. defer q.lock.Unlock()
  261. // Insert all the headers prioritised by the contained block number
  262. inserts := make([]*types.Header, 0, len(headers))
  263. for _, header := range headers {
  264. // Make sure chain order is honoured and preserved throughout
  265. hash := header.Hash()
  266. if header.Number == nil || header.Number.Uint64() != from {
  267. log.Warn("Header broke chain ordering", "number", header.Number, "hash", hash, "expected", from)
  268. break
  269. }
  270. if q.headerHead != (common.Hash{}) && q.headerHead != header.ParentHash {
  271. log.Warn("Header broke chain ancestry", "number", header.Number, "hash", hash)
  272. break
  273. }
  274. // Make sure no duplicate requests are executed
  275. if _, ok := q.blockTaskPool[hash]; ok {
  276. log.Warn("Header already scheduled for block fetch", "number", header.Number, "hash", hash)
  277. continue
  278. }
  279. if _, ok := q.receiptTaskPool[hash]; ok {
  280. log.Warn("Header already scheduled for receipt fetch", "number", header.Number, "hash", hash)
  281. continue
  282. }
  283. // Queue the header for content retrieval
  284. q.blockTaskPool[hash] = header
  285. q.blockTaskQueue.Push(header, -float32(header.Number.Uint64()))
  286. if q.mode == FastSync {
  287. q.receiptTaskPool[hash] = header
  288. q.receiptTaskQueue.Push(header, -float32(header.Number.Uint64()))
  289. }
  290. inserts = append(inserts, header)
  291. q.headerHead = hash
  292. from++
  293. }
  294. return inserts
  295. }
  296. // Results retrieves and permanently removes a batch of fetch results from
  297. // the cache. the result slice will be empty if the queue has been closed.
  298. func (q *queue) Results(block bool) []*fetchResult {
  299. q.lock.Lock()
  300. defer q.lock.Unlock()
  301. // Count the number of items available for processing
  302. nproc := q.countProcessableItems()
  303. for nproc == 0 && !q.closed {
  304. if !block {
  305. return nil
  306. }
  307. q.active.Wait()
  308. nproc = q.countProcessableItems()
  309. }
  310. // Since we have a batch limit, don't pull more into "dangling" memory
  311. if nproc > maxResultsProcess {
  312. nproc = maxResultsProcess
  313. }
  314. results := make([]*fetchResult, nproc)
  315. copy(results, q.resultCache[:nproc])
  316. if len(results) > 0 {
  317. // Mark results as done before dropping them from the cache.
  318. for _, result := range results {
  319. hash := result.Header.Hash()
  320. delete(q.blockDonePool, hash)
  321. delete(q.receiptDonePool, hash)
  322. }
  323. // Delete the results from the cache and clear the tail.
  324. copy(q.resultCache, q.resultCache[nproc:])
  325. for i := len(q.resultCache) - nproc; i < len(q.resultCache); i++ {
  326. q.resultCache[i] = nil
  327. }
  328. // Advance the expected block number of the first cache entry.
  329. q.resultOffset += uint64(nproc)
  330. // Recalculate the result item weights to prevent memory exhaustion
  331. for _, result := range results {
  332. size := result.Header.Size()
  333. for _, uncle := range result.Uncles {
  334. size += uncle.Size()
  335. }
  336. for _, receipt := range result.Receipts {
  337. size += receipt.Size()
  338. }
  339. for _, tx := range result.Transactions {
  340. size += tx.Size()
  341. }
  342. q.resultSize = common.StorageSize(blockCacheSizeWeight)*size + (1-common.StorageSize(blockCacheSizeWeight))*q.resultSize
  343. }
  344. }
  345. return results
  346. }
  347. // countProcessableItems counts the processable items.
  348. func (q *queue) countProcessableItems() int {
  349. for i, result := range q.resultCache {
  350. if result == nil || result.Pending > 0 {
  351. return i
  352. }
  353. }
  354. return len(q.resultCache)
  355. }
  356. // ReserveHeaders reserves a set of headers for the given peer, skipping any
  357. // previously failed batches.
  358. func (q *queue) ReserveHeaders(p *peerConnection, count int) *fetchRequest {
  359. q.lock.Lock()
  360. defer q.lock.Unlock()
  361. // Short circuit if the peer's already downloading something (sanity check to
  362. // not corrupt state)
  363. if _, ok := q.headerPendPool[p.id]; ok {
  364. return nil
  365. }
  366. // Retrieve a batch of hashes, skipping previously failed ones
  367. send, skip := uint64(0), []uint64{}
  368. for send == 0 && !q.headerTaskQueue.Empty() {
  369. from, _ := q.headerTaskQueue.Pop()
  370. if q.headerPeerMiss[p.id] != nil {
  371. if _, ok := q.headerPeerMiss[p.id][from.(uint64)]; ok {
  372. skip = append(skip, from.(uint64))
  373. continue
  374. }
  375. }
  376. send = from.(uint64)
  377. }
  378. // Merge all the skipped batches back
  379. for _, from := range skip {
  380. q.headerTaskQueue.Push(from, -float32(from))
  381. }
  382. // Assemble and return the block download request
  383. if send == 0 {
  384. return nil
  385. }
  386. request := &fetchRequest{
  387. Peer: p,
  388. From: send,
  389. Time: time.Now(),
  390. }
  391. q.headerPendPool[p.id] = request
  392. return request
  393. }
  394. // ReserveBodies reserves a set of body fetches for the given peer, skipping any
  395. // previously failed downloads. Beside the next batch of needed fetches, it also
  396. // returns a flag whether empty blocks were queued requiring processing.
  397. func (q *queue) ReserveBodies(p *peerConnection, count int) (*fetchRequest, bool, error) {
  398. isNoop := func(header *types.Header) bool {
  399. return header.TxHash == types.EmptyRootHash && header.UncleHash == types.EmptyUncleHash
  400. }
  401. q.lock.Lock()
  402. defer q.lock.Unlock()
  403. return q.reserveHeaders(p, count, q.blockTaskPool, q.blockTaskQueue, q.blockPendPool, q.blockDonePool, isNoop)
  404. }
  405. // ReserveReceipts reserves a set of receipt fetches for the given peer, skipping
  406. // any previously failed downloads. Beside the next batch of needed fetches, it
  407. // also returns a flag whether empty receipts were queued requiring importing.
  408. func (q *queue) ReserveReceipts(p *peerConnection, count int) (*fetchRequest, bool, error) {
  409. isNoop := func(header *types.Header) bool {
  410. return header.ReceiptHash == types.EmptyRootHash
  411. }
  412. q.lock.Lock()
  413. defer q.lock.Unlock()
  414. return q.reserveHeaders(p, count, q.receiptTaskPool, q.receiptTaskQueue, q.receiptPendPool, q.receiptDonePool, isNoop)
  415. }
  416. // reserveHeaders reserves a set of data download operations for a given peer,
  417. // skipping any previously failed ones. This method is a generic version used
  418. // by the individual special reservation functions.
  419. //
  420. // Note, this method expects the queue lock to be already held for writing. The
  421. // reason the lock is not obtained in here is because the parameters already need
  422. // to access the queue, so they already need a lock anyway.
  423. func (q *queue) reserveHeaders(p *peerConnection, count int, taskPool map[common.Hash]*types.Header, taskQueue *prque.Prque,
  424. pendPool map[string]*fetchRequest, donePool map[common.Hash]struct{}, isNoop func(*types.Header) bool) (*fetchRequest, bool, error) {
  425. // Short circuit if the pool has been depleted, or if the peer's already
  426. // downloading something (sanity check not to corrupt state)
  427. if taskQueue.Empty() {
  428. return nil, false, nil
  429. }
  430. if _, ok := pendPool[p.id]; ok {
  431. return nil, false, nil
  432. }
  433. // Calculate an upper limit on the items we might fetch (i.e. throttling)
  434. space := q.resultSlots(pendPool, donePool)
  435. // Retrieve a batch of tasks, skipping previously failed ones
  436. send := make([]*types.Header, 0, count)
  437. skip := make([]*types.Header, 0)
  438. progress := false
  439. for proc := 0; proc < space && len(send) < count && !taskQueue.Empty(); proc++ {
  440. header := taskQueue.PopItem().(*types.Header)
  441. hash := header.Hash()
  442. // If we're the first to request this task, initialise the result container
  443. index := int(header.Number.Int64() - int64(q.resultOffset))
  444. if index >= len(q.resultCache) || index < 0 {
  445. common.Report("index allocation went beyond available resultCache space")
  446. return nil, false, errInvalidChain
  447. }
  448. if q.resultCache[index] == nil {
  449. components := 1
  450. if q.mode == FastSync {
  451. components = 2
  452. }
  453. q.resultCache[index] = &fetchResult{
  454. Pending: components,
  455. Hash: hash,
  456. Header: header,
  457. }
  458. }
  459. // If this fetch task is a noop, skip this fetch operation
  460. if isNoop(header) {
  461. donePool[hash] = struct{}{}
  462. delete(taskPool, hash)
  463. space, proc = space-1, proc-1
  464. q.resultCache[index].Pending--
  465. progress = true
  466. continue
  467. }
  468. // Otherwise unless the peer is known not to have the data, add to the retrieve list
  469. if p.Lacks(hash) {
  470. skip = append(skip, header)
  471. } else {
  472. send = append(send, header)
  473. }
  474. }
  475. // Merge all the skipped headers back
  476. for _, header := range skip {
  477. taskQueue.Push(header, -float32(header.Number.Uint64()))
  478. }
  479. if progress {
  480. // Wake WaitResults, resultCache was modified
  481. q.active.Signal()
  482. }
  483. // Assemble and return the block download request
  484. if len(send) == 0 {
  485. return nil, progress, nil
  486. }
  487. request := &fetchRequest{
  488. Peer: p,
  489. Headers: send,
  490. Time: time.Now(),
  491. }
  492. pendPool[p.id] = request
  493. return request, progress, nil
  494. }
  495. // CancelHeaders aborts a fetch request, returning all pending skeleton indexes to the queue.
  496. func (q *queue) CancelHeaders(request *fetchRequest) {
  497. q.cancel(request, q.headerTaskQueue, q.headerPendPool)
  498. }
  499. // CancelBodies aborts a body fetch request, returning all pending headers to the
  500. // task queue.
  501. func (q *queue) CancelBodies(request *fetchRequest) {
  502. q.cancel(request, q.blockTaskQueue, q.blockPendPool)
  503. }
  504. // CancelReceipts aborts a body fetch request, returning all pending headers to
  505. // the task queue.
  506. func (q *queue) CancelReceipts(request *fetchRequest) {
  507. q.cancel(request, q.receiptTaskQueue, q.receiptPendPool)
  508. }
  509. // Cancel aborts a fetch request, returning all pending hashes to the task queue.
  510. func (q *queue) cancel(request *fetchRequest, taskQueue *prque.Prque, pendPool map[string]*fetchRequest) {
  511. q.lock.Lock()
  512. defer q.lock.Unlock()
  513. if request.From > 0 {
  514. taskQueue.Push(request.From, -float32(request.From))
  515. }
  516. for _, header := range request.Headers {
  517. taskQueue.Push(header, -float32(header.Number.Uint64()))
  518. }
  519. delete(pendPool, request.Peer.id)
  520. }
  521. // Revoke cancels all pending requests belonging to a given peer. This method is
  522. // meant to be called during a peer drop to quickly reassign owned data fetches
  523. // to remaining nodes.
  524. func (q *queue) Revoke(peerId string) {
  525. q.lock.Lock()
  526. defer q.lock.Unlock()
  527. if request, ok := q.blockPendPool[peerId]; ok {
  528. for _, header := range request.Headers {
  529. q.blockTaskQueue.Push(header, -float32(header.Number.Uint64()))
  530. }
  531. delete(q.blockPendPool, peerId)
  532. }
  533. if request, ok := q.receiptPendPool[peerId]; ok {
  534. for _, header := range request.Headers {
  535. q.receiptTaskQueue.Push(header, -float32(header.Number.Uint64()))
  536. }
  537. delete(q.receiptPendPool, peerId)
  538. }
  539. }
  540. // ExpireHeaders checks for in flight requests that exceeded a timeout allowance,
  541. // canceling them and returning the responsible peers for penalisation.
  542. func (q *queue) ExpireHeaders(timeout time.Duration) map[string]int {
  543. q.lock.Lock()
  544. defer q.lock.Unlock()
  545. return q.expire(timeout, q.headerPendPool, q.headerTaskQueue, headerTimeoutMeter)
  546. }
  547. // ExpireBodies checks for in flight block body requests that exceeded a timeout
  548. // allowance, canceling them and returning the responsible peers for penalisation.
  549. func (q *queue) ExpireBodies(timeout time.Duration) map[string]int {
  550. q.lock.Lock()
  551. defer q.lock.Unlock()
  552. return q.expire(timeout, q.blockPendPool, q.blockTaskQueue, bodyTimeoutMeter)
  553. }
  554. // ExpireReceipts checks for in flight receipt requests that exceeded a timeout
  555. // allowance, canceling them and returning the responsible peers for penalisation.
  556. func (q *queue) ExpireReceipts(timeout time.Duration) map[string]int {
  557. q.lock.Lock()
  558. defer q.lock.Unlock()
  559. return q.expire(timeout, q.receiptPendPool, q.receiptTaskQueue, receiptTimeoutMeter)
  560. }
  561. // expire is the generic check that move expired tasks from a pending pool back
  562. // into a task pool, returning all entities caught with expired tasks.
  563. //
  564. // Note, this method expects the queue lock to be already held. The
  565. // reason the lock is not obtained in here is because the parameters already need
  566. // to access the queue, so they already need a lock anyway.
  567. func (q *queue) expire(timeout time.Duration, pendPool map[string]*fetchRequest, taskQueue *prque.Prque, timeoutMeter metrics.Meter) map[string]int {
  568. // Iterate over the expired requests and return each to the queue
  569. expiries := make(map[string]int)
  570. for id, request := range pendPool {
  571. if time.Since(request.Time) > timeout {
  572. // Update the metrics with the timeout
  573. timeoutMeter.Mark(1)
  574. // Return any non satisfied requests to the pool
  575. if request.From > 0 {
  576. taskQueue.Push(request.From, -float32(request.From))
  577. }
  578. for _, header := range request.Headers {
  579. taskQueue.Push(header, -float32(header.Number.Uint64()))
  580. }
  581. // Add the peer to the expiry report along the the number of failed requests
  582. expiries[id] = len(request.Headers)
  583. }
  584. }
  585. // Remove the expired requests from the pending pool
  586. for id := range expiries {
  587. delete(pendPool, id)
  588. }
  589. return expiries
  590. }
  591. // DeliverHeaders injects a header retrieval response into the header results
  592. // cache. This method either accepts all headers it received, or none of them
  593. // if they do not map correctly to the skeleton.
  594. //
  595. // If the headers are accepted, the method makes an attempt to deliver the set
  596. // of ready headers to the processor to keep the pipeline full. However it will
  597. // not block to prevent stalling other pending deliveries.
  598. func (q *queue) DeliverHeaders(id string, headers []*types.Header, headerProcCh chan []*types.Header) (int, error) {
  599. q.lock.Lock()
  600. defer q.lock.Unlock()
  601. // Short circuit if the data was never requested
  602. request := q.headerPendPool[id]
  603. if request == nil {
  604. return 0, errNoFetchesPending
  605. }
  606. headerReqTimer.UpdateSince(request.Time)
  607. delete(q.headerPendPool, id)
  608. // Ensure headers can be mapped onto the skeleton chain
  609. target := q.headerTaskPool[request.From].Hash()
  610. accepted := len(headers) == MaxHeaderFetch
  611. if accepted {
  612. if headers[0].Number.Uint64() != request.From {
  613. log.Trace("First header broke chain ordering", "peer", id, "number", headers[0].Number, "hash", headers[0].Hash(), request.From)
  614. accepted = false
  615. } else if headers[len(headers)-1].Hash() != target {
  616. log.Trace("Last header broke skeleton structure ", "peer", id, "number", headers[len(headers)-1].Number, "hash", headers[len(headers)-1].Hash(), "expected", target)
  617. accepted = false
  618. }
  619. }
  620. if accepted {
  621. for i, header := range headers[1:] {
  622. hash := header.Hash()
  623. if want := request.From + 1 + uint64(i); header.Number.Uint64() != want {
  624. log.Warn("Header broke chain ordering", "peer", id, "number", header.Number, "hash", hash, "expected", want)
  625. accepted = false
  626. break
  627. }
  628. if headers[i].Hash() != header.ParentHash {
  629. log.Warn("Header broke chain ancestry", "peer", id, "number", header.Number, "hash", hash)
  630. accepted = false
  631. break
  632. }
  633. }
  634. }
  635. // If the batch of headers wasn't accepted, mark as unavailable
  636. if !accepted {
  637. log.Trace("Skeleton filling not accepted", "peer", id, "from", request.From)
  638. miss := q.headerPeerMiss[id]
  639. if miss == nil {
  640. q.headerPeerMiss[id] = make(map[uint64]struct{})
  641. miss = q.headerPeerMiss[id]
  642. }
  643. miss[request.From] = struct{}{}
  644. q.headerTaskQueue.Push(request.From, -float32(request.From))
  645. return 0, errors.New("delivery not accepted")
  646. }
  647. // Clean up a successful fetch and try to deliver any sub-results
  648. copy(q.headerResults[request.From-q.headerOffset:], headers)
  649. delete(q.headerTaskPool, request.From)
  650. ready := 0
  651. for q.headerProced+ready < len(q.headerResults) && q.headerResults[q.headerProced+ready] != nil {
  652. ready += MaxHeaderFetch
  653. }
  654. if ready > 0 {
  655. // Headers are ready for delivery, gather them and push forward (non blocking)
  656. process := make([]*types.Header, ready)
  657. copy(process, q.headerResults[q.headerProced:q.headerProced+ready])
  658. select {
  659. case headerProcCh <- process:
  660. log.Trace("Pre-scheduled new headers", "peer", id, "count", len(process), "from", process[0].Number)
  661. q.headerProced += len(process)
  662. default:
  663. }
  664. }
  665. // Check for termination and return
  666. if len(q.headerTaskPool) == 0 {
  667. q.headerContCh <- false
  668. }
  669. return len(headers), nil
  670. }
  671. // DeliverBodies injects a block body retrieval response into the results queue.
  672. // The method returns the number of blocks bodies accepted from the delivery and
  673. // also wakes any threads waiting for data delivery.
  674. func (q *queue) DeliverBodies(id string, txLists [][]*types.Transaction, uncleLists [][]*types.Header) (int, error) {
  675. q.lock.Lock()
  676. defer q.lock.Unlock()
  677. reconstruct := func(header *types.Header, index int, result *fetchResult) error {
  678. if types.DeriveSha(types.Transactions(txLists[index])) != header.TxHash || types.CalcUncleHash(uncleLists[index]) != header.UncleHash {
  679. return errInvalidBody
  680. }
  681. result.Transactions = txLists[index]
  682. result.Uncles = uncleLists[index]
  683. return nil
  684. }
  685. return q.deliver(id, q.blockTaskPool, q.blockTaskQueue, q.blockPendPool, q.blockDonePool, bodyReqTimer, len(txLists), reconstruct)
  686. }
  687. // DeliverReceipts injects a receipt retrieval response into the results queue.
  688. // The method returns the number of transaction receipts accepted from the delivery
  689. // and also wakes any threads waiting for data delivery.
  690. func (q *queue) DeliverReceipts(id string, receiptList [][]*types.Receipt) (int, error) {
  691. q.lock.Lock()
  692. defer q.lock.Unlock()
  693. reconstruct := func(header *types.Header, index int, result *fetchResult) error {
  694. if types.DeriveSha(types.Receipts(receiptList[index])) != header.ReceiptHash {
  695. return errInvalidReceipt
  696. }
  697. result.Receipts = receiptList[index]
  698. return nil
  699. }
  700. return q.deliver(id, q.receiptTaskPool, q.receiptTaskQueue, q.receiptPendPool, q.receiptDonePool, receiptReqTimer, len(receiptList), reconstruct)
  701. }
  702. // deliver injects a data retrieval response into the results queue.
  703. //
  704. // Note, this method expects the queue lock to be already held for writing. The
  705. // reason the lock is not obtained in here is because the parameters already need
  706. // to access the queue, so they already need a lock anyway.
  707. func (q *queue) deliver(id string, taskPool map[common.Hash]*types.Header, taskQueue *prque.Prque,
  708. pendPool map[string]*fetchRequest, donePool map[common.Hash]struct{}, reqTimer metrics.Timer,
  709. results int, reconstruct func(header *types.Header, index int, result *fetchResult) error) (int, error) {
  710. // Short circuit if the data was never requested
  711. request := pendPool[id]
  712. if request == nil {
  713. return 0, errNoFetchesPending
  714. }
  715. reqTimer.UpdateSince(request.Time)
  716. delete(pendPool, id)
  717. // If no data items were retrieved, mark them as unavailable for the origin peer
  718. if results == 0 {
  719. for _, header := range request.Headers {
  720. request.Peer.MarkLacking(header.Hash())
  721. }
  722. }
  723. // Assemble each of the results with their headers and retrieved data parts
  724. var (
  725. accepted int
  726. failure error
  727. useful bool
  728. )
  729. for i, header := range request.Headers {
  730. // Short circuit assembly if no more fetch results are found
  731. if i >= results {
  732. break
  733. }
  734. // Reconstruct the next result if contents match up
  735. index := int(header.Number.Int64() - int64(q.resultOffset))
  736. if index >= len(q.resultCache) || index < 0 || q.resultCache[index] == nil {
  737. failure = errInvalidChain
  738. break
  739. }
  740. if err := reconstruct(header, i, q.resultCache[index]); err != nil {
  741. failure = err
  742. break
  743. }
  744. hash := header.Hash()
  745. donePool[hash] = struct{}{}
  746. q.resultCache[index].Pending--
  747. useful = true
  748. accepted++
  749. // Clean up a successful fetch
  750. request.Headers[i] = nil
  751. delete(taskPool, hash)
  752. }
  753. // Return all failed or missing fetches to the queue
  754. for _, header := range request.Headers {
  755. if header != nil {
  756. taskQueue.Push(header, -float32(header.Number.Uint64()))
  757. }
  758. }
  759. // Wake up WaitResults
  760. if accepted > 0 {
  761. q.active.Signal()
  762. }
  763. // If none of the data was good, it's a stale delivery
  764. switch {
  765. case failure == nil || failure == errInvalidChain:
  766. return accepted, failure
  767. case useful:
  768. return accepted, fmt.Errorf("partial failure: %v", failure)
  769. default:
  770. return accepted, errStaleDelivery
  771. }
  772. }
  773. // Prepare configures the result cache to allow accepting and caching inbound
  774. // fetch results.
  775. func (q *queue) Prepare(offset uint64, mode SyncMode) {
  776. q.lock.Lock()
  777. defer q.lock.Unlock()
  778. // Prepare the queue for sync results
  779. if q.resultOffset < offset {
  780. q.resultOffset = offset
  781. }
  782. q.mode = mode
  783. }