eth: request id dispatcher and direct req/reply APIs (#23576)

* eth: request ID based message dispatcher

* eth: fix dispatcher cancellation, rework fetchers idleness tracker

* eth/downloader: drop peers who refuse to serve advertised chains
This commit is contained in:
Péter Szilágyi
2021-11-26 13:26:03 +02:00
committed by GitHub
parent 3038e480f5
commit c10a0a62c3
52 changed files with 3213 additions and 3400 deletions

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

115
eth/downloader/fetchers.go Normal file
View File

@ -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
}
}

View File

@ -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
}
}
}
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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
}

View File

@ -38,8 +38,5 @@ var (
receiptDropMeter = metrics.NewRegisteredMeter("eth/downloader/receipts/drop", nil)
receiptTimeoutMeter = metrics.NewRegisteredMeter("eth/downloader/receipts/timeout", nil)
stateInMeter = metrics.NewRegisteredMeter("eth/downloader/states/in", nil)
stateDropMeter = metrics.NewRegisteredMeter("eth/downloader/states/drop", nil)
throttleCounter = metrics.NewRegisteredCounter("eth/downloader/throttle", nil)
)

View File

@ -24,7 +24,6 @@ type SyncMode uint32
const (
FullSync SyncMode = iota // Synchronise the entire blockchain history from full blocks
FastSync // Quickly download the headers, full sync only at the chain
SnapSync // Download the chain and the state via compact snapshots
LightSync // Download only the headers and terminate afterwards
)
@ -38,8 +37,6 @@ func (mode SyncMode) String() string {
switch mode {
case FullSync:
return "full"
case FastSync:
return "fast"
case SnapSync:
return "snap"
case LightSync:
@ -53,8 +50,6 @@ func (mode SyncMode) MarshalText() ([]byte, error) {
switch mode {
case FullSync:
return []byte("full"), nil
case FastSync:
return []byte("fast"), nil
case SnapSync:
return []byte("snap"), nil
case LightSync:
@ -68,14 +63,12 @@ func (mode *SyncMode) UnmarshalText(text []byte) error {
switch string(text) {
case "full":
*mode = FullSync
case "fast":
*mode = FastSync
case "snap":
*mode = SnapSync
case "light":
*mode = LightSync
default:
return fmt.Errorf(`unknown sync mode %q, want "full", "fast" or "light"`, text)
return fmt.Errorf(`unknown sync mode %q, want "full", "snap" or "light"`, text)
}
return nil
}

View File

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

View File

@ -54,8 +54,8 @@ var (
// fetchRequest is a currently running data retrieval operation.
type fetchRequest struct {
Peer *peerConnection // Peer to which the request was sent
From uint64 // [eth/62] Requested chain element index (used for skeleton fills only)
Headers []*types.Header // [eth/62] Requested headers, sorted by request order
From uint64 // Requested chain element index (used for skeleton fills only)
Headers []*types.Header // Requested headers, sorted by request order
Time time.Time // Time when the request was made
}
@ -127,10 +127,12 @@ type queue struct {
blockTaskPool map[common.Hash]*types.Header // Pending block (body) retrieval tasks, mapping hashes to headers
blockTaskQueue *prque.Prque // Priority queue of the headers to fetch the blocks (bodies) for
blockPendPool map[string]*fetchRequest // Currently pending block (body) retrieval operations
blockWakeCh chan bool // Channel to notify the block fetcher of new tasks
receiptTaskPool map[common.Hash]*types.Header // Pending receipt retrieval tasks, mapping hashes to headers
receiptTaskQueue *prque.Prque // Priority queue of the headers to fetch the receipts for
receiptPendPool map[string]*fetchRequest // Currently pending receipt retrieval operations
receiptWakeCh chan bool // Channel to notify when receipt fetcher of new tasks
resultCache *resultStore // Downloaded but not yet delivered fetch results
resultSize common.StorageSize // Approximate size of a block (exponential moving average)
@ -146,9 +148,11 @@ type queue struct {
func newQueue(blockCacheLimit int, thresholdInitialSize int) *queue {
lock := new(sync.RWMutex)
q := &queue{
headerContCh: make(chan bool),
headerContCh: make(chan bool, 1),
blockTaskQueue: prque.New(nil),
blockWakeCh: make(chan bool, 1),
receiptTaskQueue: prque.New(nil),
receiptWakeCh: make(chan bool, 1),
active: sync.NewCond(lock),
lock: lock,
}
@ -196,8 +200,8 @@ func (q *queue) PendingHeaders() int {
return q.headerTaskQueue.Size()
}
// PendingBlocks retrieves the number of block (body) requests pending for retrieval.
func (q *queue) PendingBlocks() int {
// PendingBodies retrieves the number of block body requests pending for retrieval.
func (q *queue) PendingBodies() int {
q.lock.Lock()
defer q.lock.Unlock()
@ -212,15 +216,6 @@ func (q *queue) PendingReceipts() int {
return q.receiptTaskQueue.Size()
}
// InFlightHeaders retrieves whether there are header fetch requests currently
// in flight.
func (q *queue) InFlightHeaders() bool {
q.lock.Lock()
defer q.lock.Unlock()
return len(q.headerPendPool) > 0
}
// InFlightBlocks retrieves whether there are block fetch requests currently in
// flight.
func (q *queue) InFlightBlocks() bool {
@ -318,7 +313,7 @@ func (q *queue) Schedule(headers []*types.Header, from uint64) []*types.Header {
q.blockTaskQueue.Push(header, -int64(header.Number.Uint64()))
}
// Queue for receipt retrieval
if q.mode == FastSync && !header.EmptyReceipts() {
if q.mode == SnapSync && !header.EmptyReceipts() {
if _, ok := q.receiptTaskPool[hash]; ok {
log.Warn("Header already scheduled for receipt fetch", "number", header.Number, "hash", hash)
} else {
@ -383,6 +378,13 @@ func (q *queue) Results(block bool) []*fetchResult {
throttleThreshold := uint64((common.StorageSize(blockCacheMemory) + q.resultSize - 1) / q.resultSize)
throttleThreshold = q.resultCache.SetThrottleThreshold(throttleThreshold)
// With results removed from the cache, wake throttled fetchers
for _, ch := range []chan bool{q.blockWakeCh, q.receiptWakeCh} {
select {
case ch <- true:
default:
}
}
// Log some info at certain times
if time.Since(q.lastStatLog) > 60*time.Second {
q.lastStatLog = time.Now()
@ -503,7 +505,7 @@ func (q *queue) reserveHeaders(p *peerConnection, count int, taskPool map[common
// we can ask the resultcache if this header is within the
// "prioritized" segment of blocks. If it is not, we need to throttle
stale, throttle, item, err := q.resultCache.AddFetch(header, q.mode == FastSync)
stale, throttle, item, err := q.resultCache.AddFetch(header, q.mode == SnapSync)
if stale {
// Don't put back in the task queue, this item has already been
// delivered upstream
@ -566,40 +568,6 @@ func (q *queue) reserveHeaders(p *peerConnection, count int, taskPool map[common
return request, progress, throttled
}
// CancelHeaders aborts a fetch request, returning all pending skeleton indexes to the queue.
func (q *queue) CancelHeaders(request *fetchRequest) {
q.lock.Lock()
defer q.lock.Unlock()
q.cancel(request, q.headerTaskQueue, q.headerPendPool)
}
// CancelBodies aborts a body fetch request, returning all pending headers to the
// task queue.
func (q *queue) CancelBodies(request *fetchRequest) {
q.lock.Lock()
defer q.lock.Unlock()
q.cancel(request, q.blockTaskQueue, q.blockPendPool)
}
// CancelReceipts aborts a body fetch request, returning all pending headers to
// the task queue.
func (q *queue) CancelReceipts(request *fetchRequest) {
q.lock.Lock()
defer q.lock.Unlock()
q.cancel(request, q.receiptTaskQueue, q.receiptPendPool)
}
// Cancel aborts a fetch request, returning all pending hashes to the task queue.
func (q *queue) cancel(request *fetchRequest, taskQueue *prque.Prque, pendPool map[string]*fetchRequest) {
if request.From > 0 {
taskQueue.Push(request.From, -int64(request.From))
}
for _, header := range request.Headers {
taskQueue.Push(header, -int64(header.Number.Uint64()))
}
delete(pendPool, request.Peer.id)
}
// Revoke cancels all pending requests belonging to a given peer. This method is
// meant to be called during a peer drop to quickly reassign owned data fetches
// to remaining nodes.
@ -607,6 +575,10 @@ func (q *queue) Revoke(peerID string) {
q.lock.Lock()
defer q.lock.Unlock()
if request, ok := q.headerPendPool[peerID]; ok {
q.headerTaskQueue.Push(request.From, -int64(request.From))
delete(q.headerPendPool, peerID)
}
if request, ok := q.blockPendPool[peerID]; ok {
for _, header := range request.Headers {
q.blockTaskQueue.Push(header, -int64(header.Number.Uint64()))
@ -621,62 +593,60 @@ func (q *queue) Revoke(peerID string) {
}
}
// ExpireHeaders checks for in flight requests that exceeded a timeout allowance,
// canceling them and returning the responsible peers for penalisation.
func (q *queue) ExpireHeaders(timeout time.Duration) map[string]int {
// ExpireHeaders cancels a request that timed out and moves the pending fetch
// task back into the queue for rescheduling.
func (q *queue) ExpireHeaders(peer string) int {
q.lock.Lock()
defer q.lock.Unlock()
return q.expire(timeout, q.headerPendPool, q.headerTaskQueue, headerTimeoutMeter)
headerTimeoutMeter.Mark(1)
return q.expire(peer, q.headerPendPool, q.headerTaskQueue)
}
// ExpireBodies checks for in flight block body requests that exceeded a timeout
// allowance, canceling them and returning the responsible peers for penalisation.
func (q *queue) ExpireBodies(timeout time.Duration) map[string]int {
func (q *queue) ExpireBodies(peer string) int {
q.lock.Lock()
defer q.lock.Unlock()
return q.expire(timeout, q.blockPendPool, q.blockTaskQueue, bodyTimeoutMeter)
bodyTimeoutMeter.Mark(1)
return q.expire(peer, q.blockPendPool, q.blockTaskQueue)
}
// ExpireReceipts checks for in flight receipt requests that exceeded a timeout
// allowance, canceling them and returning the responsible peers for penalisation.
func (q *queue) ExpireReceipts(timeout time.Duration) map[string]int {
func (q *queue) ExpireReceipts(peer string) int {
q.lock.Lock()
defer q.lock.Unlock()
return q.expire(timeout, q.receiptPendPool, q.receiptTaskQueue, receiptTimeoutMeter)
receiptTimeoutMeter.Mark(1)
return q.expire(peer, q.receiptPendPool, q.receiptTaskQueue)
}
// expire is the generic check that move expired tasks from a pending pool back
// into a task pool, returning all entities caught with expired tasks.
// expire is the generic check that moves a specific expired task from a pending
// pool back into a task pool.
//
// Note, this method expects the queue lock to be already held. 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.
func (q *queue) expire(timeout time.Duration, pendPool map[string]*fetchRequest, taskQueue *prque.Prque, timeoutMeter metrics.Meter) map[string]int {
// Iterate over the expired requests and return each to the queue
expiries := make(map[string]int)
for id, request := range pendPool {
if time.Since(request.Time) > timeout {
// Update the metrics with the timeout
timeoutMeter.Mark(1)
// Return any non satisfied requests to the pool
if request.From > 0 {
taskQueue.Push(request.From, -int64(request.From))
}
for _, header := range request.Headers {
taskQueue.Push(header, -int64(header.Number.Uint64()))
}
// Add the peer to the expiry report along the number of failed requests
expiries[id] = len(request.Headers)
// Remove the expired requests from the pending pool directly
delete(pendPool, id)
}
// Note, this method expects the queue lock to be already held. The reason the
// lock is not obtained in here is that the parameters already need to access
// the queue, so they already need a lock anyway.
func (q *queue) expire(peer string, pendPool map[string]*fetchRequest, taskQueue *prque.Prque) int {
// Retrieve the request being expired and log an error if it's non-existnet,
// as there's no order of events that should lead to such expirations.
req := pendPool[peer]
if req == nil {
log.Error("Expired request does not exist", "peer", peer)
return 0
}
return expiries
delete(pendPool, peer)
// Return any non-satisfied requests to the pool
if req.From > 0 {
taskQueue.Push(req.From, -int64(req.From))
}
for _, header := range req.Headers {
taskQueue.Push(header, -int64(header.Number.Uint64()))
}
return len(req.Headers)
}
// DeliverHeaders injects a header retrieval response into the header results
@ -684,7 +654,7 @@ func (q *queue) expire(timeout time.Duration, pendPool map[string]*fetchRequest,
// if they do not map correctly to the skeleton.
//
// If the headers are accepted, the method makes an attempt to deliver the set
// of ready headers to the processor to keep the pipeline full. However it will
// of ready headers to the processor to keep the pipeline full. However, it will
// not block to prevent stalling other pending deliveries.
func (q *queue) DeliverHeaders(id string, headers []*types.Header, headerProcCh chan []*types.Header) (int, error) {
q.lock.Lock()
@ -700,11 +670,14 @@ func (q *queue) DeliverHeaders(id string, headers []*types.Header, headerProcCh
// Short circuit if the data was never requested
request := q.headerPendPool[id]
if request == nil {
headerDropMeter.Mark(int64(len(headers)))
return 0, errNoFetchesPending
}
headerReqTimer.UpdateSince(request.Time)
delete(q.headerPendPool, id)
headerReqTimer.UpdateSince(request.Time)
headerInMeter.Mark(int64(len(headers)))
// Ensure headers can be mapped onto the skeleton chain
target := q.headerTaskPool[request.From].Hash()
@ -739,6 +712,7 @@ func (q *queue) DeliverHeaders(id string, headers []*types.Header, headerProcCh
// If the batch of headers wasn't accepted, mark as unavailable
if !accepted {
logger.Trace("Skeleton filling not accepted", "from", request.From)
headerDropMeter.Mark(int64(len(headers)))
miss := q.headerPeerMiss[id]
if miss == nil {
@ -783,6 +757,7 @@ 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()
trieHasher := trie.NewStackTrie(nil)
validate := func(index int, header *types.Header) error {
if types.DeriveSha(types.Transactions(txLists[index]), trieHasher) != header.TxHash {
@ -800,7 +775,7 @@ func (q *queue) DeliverBodies(id string, txLists [][]*types.Transaction, uncleLi
result.SetBodyDone()
}
return q.deliver(id, q.blockTaskPool, q.blockTaskQueue, q.blockPendPool,
bodyReqTimer, len(txLists), validate, reconstruct)
bodyReqTimer, bodyInMeter, bodyDropMeter, len(txLists), validate, reconstruct)
}
// DeliverReceipts injects a receipt retrieval response into the results queue.
@ -809,6 +784,7 @@ 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()
trieHasher := trie.NewStackTrie(nil)
validate := func(index int, header *types.Header) error {
if types.DeriveSha(types.Receipts(receiptList[index]), trieHasher) != header.ReceiptHash {
@ -821,7 +797,7 @@ func (q *queue) DeliverReceipts(id string, receiptList [][]*types.Receipt) (int,
result.SetReceiptsDone()
}
return q.deliver(id, q.receiptTaskPool, q.receiptTaskQueue, q.receiptPendPool,
receiptReqTimer, len(receiptList), validate, reconstruct)
receiptReqTimer, receiptInMeter, receiptDropMeter, len(receiptList), validate, reconstruct)
}
// deliver injects a data retrieval response into the results queue.
@ -830,18 +806,22 @@ func (q *queue) DeliverReceipts(id string, receiptList [][]*types.Receipt) (int,
// 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, reqTimer metrics.Timer,
taskQueue *prque.Prque, pendPool map[string]*fetchRequest,
reqTimer metrics.Timer, resInMeter metrics.Meter, resDropMeter metrics.Meter,
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]
if request == nil {
resDropMeter.Mark(int64(results))
return 0, errNoFetchesPending
}
reqTimer.UpdateSince(request.Time)
delete(pendPool, id)
reqTimer.UpdateSince(request.Time)
resInMeter.Mark(int64(results))
// If no data items were retrieved, mark them as unavailable for the origin peer
if results == 0 {
for _, header := range request.Headers {
@ -883,6 +863,8 @@ func (q *queue) deliver(id string, taskPool map[common.Hash]*types.Header,
delete(taskPool, hashes[accepted])
accepted++
}
resDropMeter.Mark(int64(results - accepted))
// Return all failed or missing fetches to the queue
for _, header := range request.Headers[accepted:] {
taskQueue.Push(header, -int64(header.Number.Uint64()))

View File

@ -104,7 +104,7 @@ func TestBasics(t *testing.T) {
if !q.Idle() {
t.Errorf("new queue should be idle")
}
q.Prepare(1, FastSync)
q.Prepare(1, SnapSync)
if res := q.Results(false); len(res) != 0 {
t.Fatal("new queue should have 0 results")
}
@ -114,7 +114,7 @@ func TestBasics(t *testing.T) {
if q.Idle() {
t.Errorf("queue should not be idle")
}
if got, exp := q.PendingBlocks(), chain.Len(); got != exp {
if got, exp := q.PendingBodies(), chain.Len(); got != exp {
t.Errorf("wrong pending block count, got %d, exp %d", got, exp)
}
// Only non-empty receipts get added to task-queue
@ -197,13 +197,13 @@ func TestEmptyBlocks(t *testing.T) {
q := newQueue(10, 10)
q.Prepare(1, FastSync)
q.Prepare(1, SnapSync)
// Schedule a batch of headers
q.Schedule(emptyChain.headers(), 1)
if q.Idle() {
t.Errorf("queue should not be idle")
}
if got, exp := q.PendingBlocks(), len(emptyChain.blocks); got != exp {
if got, exp := q.PendingBodies(), len(emptyChain.blocks); got != exp {
t.Errorf("wrong pending block count, got %d, exp %d", got, exp)
}
if got, exp := q.PendingReceipts(), 0; got != exp {
@ -272,7 +272,7 @@ func XTestDelivery(t *testing.T) {
}
q := newQueue(10, 10)
var wg sync.WaitGroup
q.Prepare(1, FastSync)
q.Prepare(1, SnapSync)
wg.Add(1)
go func() {
// deliver headers

View File

@ -17,48 +17,12 @@
package downloader
import (
"fmt"
"sync"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/rawdb"
"github.com/ethereum/go-ethereum/core/state"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/trie"
"golang.org/x/crypto/sha3"
)
// stateReq represents a batch of state fetch requests grouped together into
// a single data retrieval network packet.
type stateReq struct {
nItems uint16 // Number of items requested for download (max is 384, so uint16 is sufficient)
trieTasks map[common.Hash]*trieTask // Trie node download tasks to track previous attempts
codeTasks map[common.Hash]*codeTask // Byte code download tasks to track previous attempts
timeout time.Duration // Maximum round trip time for this to complete
timer *time.Timer // Timer to fire when the RTT timeout expires
peer *peerConnection // Peer that we're requesting from
delivered time.Time // Time when the packet was delivered (independent when we process it)
response [][]byte // Response data of the peer (nil for timeouts)
dropped bool // Flag whether the peer dropped off early
}
// timedOut returns if this request timed out.
func (req *stateReq) timedOut() bool {
return req.response == nil
}
// stateSyncStats is a collection of progress stats to report during a state trie
// sync to RPC requests as well as to display in user logs.
type stateSyncStats struct {
processed uint64 // Number of state entries processed
duplicate uint64 // Number of state entries downloaded twice
unexpected uint64 // Number of non-requested state entries received
pending uint64 // Number of still pending state entries
}
// syncState starts downloading state with the given root hash.
func (d *Downloader) syncState(root common.Hash) *stateSync {
// Create the state sync
@ -85,8 +49,6 @@ func (d *Downloader) stateFetcher() {
for next := s; next != nil; {
next = d.runStateSync(next)
}
case <-d.stateCh:
// Ignore state responses while no sync is running.
case <-d.quitCh:
return
}
@ -96,216 +58,44 @@ func (d *Downloader) stateFetcher() {
// runStateSync runs a state synchronisation until it completes or another root
// hash is requested to be switched over to.
func (d *Downloader) runStateSync(s *stateSync) *stateSync {
var (
active = make(map[string]*stateReq) // Currently in-flight requests
finished []*stateReq // Completed or failed requests
timeout = make(chan *stateReq) // Timed out active requests
)
log.Trace("State sync starting", "root", s.root)
defer func() {
// Cancel active request timers on exit. Also set peers to idle so they're
// available for the next sync.
for _, req := range active {
req.timer.Stop()
req.peer.SetNodeDataIdle(int(req.nItems), time.Now())
}
}()
go s.run()
defer s.Cancel()
// Listen for peer departure events to cancel assigned tasks
peerDrop := make(chan *peerConnection, 1024)
peerSub := s.d.peers.SubscribePeerDrops(peerDrop)
defer peerSub.Unsubscribe()
for {
// Enable sending of the first buffered element if there is one.
var (
deliverReq *stateReq
deliverReqCh chan *stateReq
)
if len(finished) > 0 {
deliverReq = finished[0]
deliverReqCh = s.deliver
}
select {
// The stateSync lifecycle:
case next := <-d.stateSyncStart:
d.spindownStateSync(active, finished, timeout, peerDrop)
return next
case <-s.done:
d.spindownStateSync(active, finished, timeout, peerDrop)
return nil
// Send the next finished request to the current sync:
case deliverReqCh <- deliverReq:
// Shift out the first request, but also set the emptied slot to nil for GC
copy(finished, finished[1:])
finished[len(finished)-1] = nil
finished = finished[:len(finished)-1]
// Handle incoming state packs:
case pack := <-d.stateCh:
// Discard any data not requested (or previously timed out)
req := active[pack.PeerId()]
if req == nil {
log.Debug("Unrequested node data", "peer", pack.PeerId(), "len", pack.Items())
continue
}
// Finalize the request and queue up for processing
req.timer.Stop()
req.response = pack.(*statePack).states
req.delivered = time.Now()
finished = append(finished, req)
delete(active, pack.PeerId())
// Handle dropped peer connections:
case p := <-peerDrop:
// Skip if no request is currently pending
req := active[p.id]
if req == nil {
continue
}
// Finalize the request and queue up for processing
req.timer.Stop()
req.dropped = true
req.delivered = time.Now()
finished = append(finished, req)
delete(active, p.id)
// Handle timed-out requests:
case req := <-timeout:
// If the peer is already requesting something else, ignore the stale timeout.
// This can happen when the timeout and the delivery happens simultaneously,
// causing both pathways to trigger.
if active[req.peer.id] != req {
continue
}
req.delivered = time.Now()
// Move the timed out data back into the download queue
finished = append(finished, req)
delete(active, req.peer.id)
// Track outgoing state requests:
case req := <-d.trackStateReq:
// If an active request already exists for this peer, we have a problem. In
// theory the trie node schedule must never assign two requests to the same
// peer. In practice however, a peer might receive a request, disconnect and
// immediately reconnect before the previous times out. In this case the first
// request is never honored, alas we must not silently overwrite it, as that
// causes valid requests to go missing and sync to get stuck.
if old := active[req.peer.id]; old != nil {
log.Warn("Busy peer assigned new state fetch", "peer", old.peer.id)
// Move the previous request to the finished set
old.timer.Stop()
old.dropped = true
old.delivered = time.Now()
finished = append(finished, old)
}
// Start a timer to notify the sync loop if the peer stalled.
req.timer = time.AfterFunc(req.timeout, func() {
timeout <- req
})
active[req.peer.id] = req
}
}
}
// spindownStateSync 'drains' the outstanding requests; some will be delivered and other
// will time out. This is to ensure that when the next stateSync starts working, all peers
// are marked as idle and de facto _are_ idle.
func (d *Downloader) spindownStateSync(active map[string]*stateReq, finished []*stateReq, timeout chan *stateReq, peerDrop chan *peerConnection) {
log.Trace("State sync spinning down", "active", len(active), "finished", len(finished))
for len(active) > 0 {
var (
req *stateReq
reason string
)
select {
// Handle (drop) incoming state packs:
case pack := <-d.stateCh:
req = active[pack.PeerId()]
reason = "delivered"
// Handle dropped peer connections:
case p := <-peerDrop:
req = active[p.id]
reason = "peerdrop"
// Handle timed-out requests:
case req = <-timeout:
reason = "timeout"
}
if req == nil {
continue
}
req.peer.log.Trace("State peer marked idle (spindown)", "req.items", int(req.nItems), "reason", reason)
req.timer.Stop()
delete(active, req.peer.id)
req.peer.SetNodeDataIdle(int(req.nItems), time.Now())
}
// The 'finished' set contains deliveries that we were going to pass to processing.
// Those are now moot, but we still need to set those peers as idle, which would
// otherwise have been done after processing
for _, req := range finished {
req.peer.SetNodeDataIdle(int(req.nItems), time.Now())
}
}
// stateSync schedules requests for downloading a particular state trie defined
// by a given state root.
type stateSync struct {
d *Downloader // Downloader instance to access and manage current peerset
d *Downloader // Downloader instance to access and manage current peerset
root common.Hash // State root currently being synced
root common.Hash // State root currently being synced
sched *trie.Sync // State trie sync scheduler defining the tasks
keccak crypto.KeccakState // Keccak256 hasher to verify deliveries with
trieTasks map[common.Hash]*trieTask // Set of trie node tasks currently queued for retrieval
codeTasks map[common.Hash]*codeTask // Set of byte code tasks currently queued for retrieval
numUncommitted int
bytesUncommitted int
started chan struct{} // Started is signalled once the sync loop starts
deliver chan *stateReq // Delivery channel multiplexing peer responses
cancel chan struct{} // Channel to signal a termination request
cancelOnce sync.Once // Ensures cancel only ever gets called once
done chan struct{} // Channel to signal termination completion
err error // Any error hit during sync (set before completion)
}
// trieTask represents a single trie node download task, containing a set of
// peers already attempted retrieval from to detect stalled syncs and abort.
type trieTask struct {
path [][]byte
attempts map[string]struct{}
}
// codeTask represents a single byte code download task, containing a set of
// peers already attempted retrieval from to detect stalled syncs and abort.
type codeTask struct {
attempts map[string]struct{}
started chan struct{} // Started is signalled once the sync loop starts
cancel chan struct{} // Channel to signal a termination request
cancelOnce sync.Once // Ensures cancel only ever gets called once
done chan struct{} // Channel to signal termination completion
err error // Any error hit during sync (set before completion)
}
// newStateSync creates a new state trie download scheduler. This method does not
// yet start the sync. The user needs to call run to initiate.
func newStateSync(d *Downloader, root common.Hash) *stateSync {
return &stateSync{
d: d,
root: root,
sched: state.NewStateSync(root, d.stateDB, d.stateBloom, nil),
keccak: sha3.NewLegacyKeccak256().(crypto.KeccakState),
trieTasks: make(map[common.Hash]*trieTask),
codeTasks: make(map[common.Hash]*codeTask),
deliver: make(chan *stateReq),
cancel: make(chan struct{}),
done: make(chan struct{}),
started: make(chan struct{}),
d: d,
root: root,
cancel: make(chan struct{}),
done: make(chan struct{}),
started: make(chan struct{}),
}
}
@ -314,11 +104,7 @@ func newStateSync(d *Downloader, root common.Hash) *stateSync {
// finish.
func (s *stateSync) run() {
close(s.started)
if s.d.snapSync {
s.err = s.d.SnapSyncer.Sync(s.root, s.cancel)
} else {
s.err = s.loop()
}
s.err = s.d.SnapSyncer.Sync(s.root, s.cancel)
close(s.done)
}
@ -335,281 +121,3 @@ func (s *stateSync) Cancel() error {
})
return s.Wait()
}
// loop is the main event loop of a state trie sync. It it responsible for the
// assignment of new tasks to peers (including sending it to them) as well as
// for the processing of inbound data. Note, that the loop does not directly
// receive data from peers, rather those are buffered up in the downloader and
// pushed here async. The reason is to decouple processing from data receipt
// and timeouts.
func (s *stateSync) loop() (err error) {
// Listen for new peer events to assign tasks to them
newPeer := make(chan *peerConnection, 1024)
peerSub := s.d.peers.SubscribeNewPeers(newPeer)
defer peerSub.Unsubscribe()
defer func() {
cerr := s.commit(true)
if err == nil {
err = cerr
}
}()
// Keep assigning new tasks until the sync completes or aborts
for s.sched.Pending() > 0 {
if err = s.commit(false); err != nil {
return err
}
s.assignTasks()
// Tasks assigned, wait for something to happen
select {
case <-newPeer:
// New peer arrived, try to assign it download tasks
case <-s.cancel:
return errCancelStateFetch
case <-s.d.cancelCh:
return errCanceled
case req := <-s.deliver:
// Response, disconnect or timeout triggered, drop the peer if stalling
log.Trace("Received node data response", "peer", req.peer.id, "count", len(req.response), "dropped", req.dropped, "timeout", !req.dropped && req.timedOut())
if req.nItems <= 2 && !req.dropped && req.timedOut() {
// 2 items are the minimum requested, if even that times out, we've no use of
// this peer at the moment.
log.Warn("Stalling state sync, dropping peer", "peer", req.peer.id)
if s.d.dropPeer == nil {
// The dropPeer method is nil when `--copydb` is used for a local copy.
// Timeouts can occur if e.g. compaction hits at the wrong time, and can be ignored
req.peer.log.Warn("Downloader wants to drop peer, but peerdrop-function is not set", "peer", req.peer.id)
} else {
s.d.dropPeer(req.peer.id)
// If this peer was the master peer, abort sync immediately
s.d.cancelLock.RLock()
master := req.peer.id == s.d.cancelPeer
s.d.cancelLock.RUnlock()
if master {
s.d.cancel()
return errTimeout
}
}
}
// Process all the received blobs and check for stale delivery
delivered, err := s.process(req)
req.peer.SetNodeDataIdle(delivered, req.delivered)
if err != nil {
log.Warn("Node data write error", "err", err)
return err
}
}
}
return nil
}
func (s *stateSync) commit(force bool) error {
if !force && s.bytesUncommitted < ethdb.IdealBatchSize {
return nil
}
start := time.Now()
b := s.d.stateDB.NewBatch()
if err := s.sched.Commit(b); err != nil {
return err
}
if err := b.Write(); err != nil {
return fmt.Errorf("DB write error: %v", err)
}
s.updateStats(s.numUncommitted, 0, 0, time.Since(start))
s.numUncommitted = 0
s.bytesUncommitted = 0
return nil
}
// assignTasks attempts to assign new tasks to all idle peers, either from the
// batch currently being retried, or fetching new data from the trie sync itself.
func (s *stateSync) assignTasks() {
// Iterate over all idle peers and try to assign them state fetches
peers, _ := s.d.peers.NodeDataIdlePeers()
for _, p := range peers {
// Assign a batch of fetches proportional to the estimated latency/bandwidth
cap := p.NodeDataCapacity(s.d.peers.rates.TargetRoundTrip())
req := &stateReq{peer: p, timeout: s.d.peers.rates.TargetTimeout()}
nodes, _, codes := s.fillTasks(cap, req)
// If the peer was assigned tasks to fetch, send the network request
if len(nodes)+len(codes) > 0 {
req.peer.log.Trace("Requesting batch of state data", "nodes", len(nodes), "codes", len(codes), "root", s.root)
select {
case s.d.trackStateReq <- req:
req.peer.FetchNodeData(append(nodes, codes...)) // Unified retrieval under eth/6x
case <-s.cancel:
case <-s.d.cancelCh:
}
}
}
}
// fillTasks fills the given request object with a maximum of n state download
// tasks to send to the remote peer.
func (s *stateSync) fillTasks(n int, req *stateReq) (nodes []common.Hash, paths []trie.SyncPath, codes []common.Hash) {
// Refill available tasks from the scheduler.
if fill := n - (len(s.trieTasks) + len(s.codeTasks)); fill > 0 {
nodes, paths, codes := s.sched.Missing(fill)
for i, hash := range nodes {
s.trieTasks[hash] = &trieTask{
path: paths[i],
attempts: make(map[string]struct{}),
}
}
for _, hash := range codes {
s.codeTasks[hash] = &codeTask{
attempts: make(map[string]struct{}),
}
}
}
// Find tasks that haven't been tried with the request's peer. Prefer code
// over trie nodes as those can be written to disk and forgotten about.
nodes = make([]common.Hash, 0, n)
paths = make([]trie.SyncPath, 0, n)
codes = make([]common.Hash, 0, n)
req.trieTasks = make(map[common.Hash]*trieTask, n)
req.codeTasks = make(map[common.Hash]*codeTask, n)
for hash, t := range s.codeTasks {
// Stop when we've gathered enough requests
if len(nodes)+len(codes) == n {
break
}
// Skip any requests we've already tried from this peer
if _, ok := t.attempts[req.peer.id]; ok {
continue
}
// Assign the request to this peer
t.attempts[req.peer.id] = struct{}{}
codes = append(codes, hash)
req.codeTasks[hash] = t
delete(s.codeTasks, hash)
}
for hash, t := range s.trieTasks {
// Stop when we've gathered enough requests
if len(nodes)+len(codes) == n {
break
}
// Skip any requests we've already tried from this peer
if _, ok := t.attempts[req.peer.id]; ok {
continue
}
// Assign the request to this peer
t.attempts[req.peer.id] = struct{}{}
nodes = append(nodes, hash)
paths = append(paths, t.path)
req.trieTasks[hash] = t
delete(s.trieTasks, hash)
}
req.nItems = uint16(len(nodes) + len(codes))
return nodes, paths, codes
}
// process iterates over a batch of delivered state data, injecting each item
// into a running state sync, re-queuing any items that were requested but not
// delivered. Returns whether the peer actually managed to deliver anything of
// value, and any error that occurred.
func (s *stateSync) process(req *stateReq) (int, error) {
// Collect processing stats and update progress if valid data was received
duplicate, unexpected, successful := 0, 0, 0
defer func(start time.Time) {
if duplicate > 0 || unexpected > 0 {
s.updateStats(0, duplicate, unexpected, time.Since(start))
}
}(time.Now())
// Iterate over all the delivered data and inject one-by-one into the trie
for _, blob := range req.response {
hash, err := s.processNodeData(blob)
switch err {
case nil:
s.numUncommitted++
s.bytesUncommitted += len(blob)
successful++
case trie.ErrNotRequested:
unexpected++
case trie.ErrAlreadyProcessed:
duplicate++
default:
return successful, fmt.Errorf("invalid state node %s: %v", hash.TerminalString(), err)
}
// Delete from both queues (one delivery is enough for the syncer)
delete(req.trieTasks, hash)
delete(req.codeTasks, hash)
}
// Put unfulfilled tasks back into the retry queue
npeers := s.d.peers.Len()
for hash, task := range req.trieTasks {
// If the node did deliver something, missing items may be due to a protocol
// limit or a previous timeout + delayed delivery. Both cases should permit
// the node to retry the missing items (to avoid single-peer stalls).
if len(req.response) > 0 || req.timedOut() {
delete(task.attempts, req.peer.id)
}
// If we've requested the node too many times already, it may be a malicious
// sync where nobody has the right data. Abort.
if len(task.attempts) >= npeers {
return successful, fmt.Errorf("trie node %s failed with all peers (%d tries, %d peers)", hash.TerminalString(), len(task.attempts), npeers)
}
// Missing item, place into the retry queue.
s.trieTasks[hash] = task
}
for hash, task := range req.codeTasks {
// If the node did deliver something, missing items may be due to a protocol
// limit or a previous timeout + delayed delivery. Both cases should permit
// the node to retry the missing items (to avoid single-peer stalls).
if len(req.response) > 0 || req.timedOut() {
delete(task.attempts, req.peer.id)
}
// If we've requested the node too many times already, it may be a malicious
// sync where nobody has the right data. Abort.
if len(task.attempts) >= npeers {
return successful, fmt.Errorf("byte code %s failed with all peers (%d tries, %d peers)", hash.TerminalString(), len(task.attempts), npeers)
}
// Missing item, place into the retry queue.
s.codeTasks[hash] = task
}
return successful, nil
}
// processNodeData tries to inject a trie node data blob delivered from a remote
// peer into the state trie, returning whether anything useful was written or any
// error occurred.
func (s *stateSync) processNodeData(blob []byte) (common.Hash, error) {
res := trie.SyncResult{Data: blob}
s.keccak.Reset()
s.keccak.Write(blob)
s.keccak.Read(res.Hash[:])
err := s.sched.Process(res)
return res.Hash, err
}
// updateStats bumps the various state sync progress counters and displays a log
// message for the user to see.
func (s *stateSync) updateStats(written, duplicate, unexpected int, duration time.Duration) {
s.d.syncStatsLock.Lock()
defer s.d.syncStatsLock.Unlock()
s.d.syncStatsState.pending = uint64(s.sched.Pending())
s.d.syncStatsState.processed += uint64(written)
s.d.syncStatsState.duplicate += uint64(duplicate)
s.d.syncStatsState.unexpected += uint64(unexpected)
if written > 0 || duplicate > 0 || unexpected > 0 {
log.Info("Imported new state entries", "count", written, "elapsed", common.PrettyDuration(duration), "processed", s.d.syncStatsState.processed, "pending", s.d.syncStatsState.pending, "trieretry", len(s.trieTasks), "coderetry", len(s.codeTasks), "duplicate", s.d.syncStatsState.duplicate, "unexpected", s.d.syncStatsState.unexpected)
}
if written > 0 {
rawdb.WriteFastTrieProgress(s.d.stateDB, s.d.syncStatsState.processed)
}
}

View File

@ -20,12 +20,14 @@ import (
"fmt"
"math/big"
"sync"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/consensus/ethash"
"github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/rawdb"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/core/vm"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/params"
)
@ -39,73 +41,110 @@ var (
)
// The common prefix of all test chains:
var testChainBase = newTestChain(blockCacheMaxItems+200, testGenesis)
var testChainBase *testChain
// Different forks on top of the base chain:
var testChainForkLightA, testChainForkLightB, testChainForkHeavy *testChain
var pregenerated bool
func init() {
// Reduce some of the parameters to make the tester faster
fullMaxForkAncestry = 10000
lightMaxForkAncestry = 10000
blockCacheMaxItems = 1024
fsHeaderSafetyNet = 256
fsHeaderContCheck = 500 * time.Millisecond
testChainBase = newTestChain(blockCacheMaxItems+200, testGenesis)
var forkLen = int(fullMaxForkAncestry + 50)
var wg sync.WaitGroup
// Generate the test chains to seed the peers with
wg.Add(3)
go func() { testChainForkLightA = testChainBase.makeFork(forkLen, false, 1); wg.Done() }()
go func() { testChainForkLightB = testChainBase.makeFork(forkLen, false, 2); wg.Done() }()
go func() { testChainForkHeavy = testChainBase.makeFork(forkLen, true, 3); wg.Done() }()
wg.Wait()
// Generate the test peers used by the tests to avoid overloading during testing.
// These seemingly random chains are used in various downloader tests. We're just
// pre-generating them here.
chains := []*testChain{
testChainBase,
testChainForkLightA,
testChainForkLightB,
testChainForkHeavy,
testChainBase.shorten(1),
testChainBase.shorten(blockCacheMaxItems - 15),
testChainBase.shorten((blockCacheMaxItems - 15) / 2),
testChainBase.shorten(blockCacheMaxItems - 15 - 5),
testChainBase.shorten(MaxHeaderFetch),
testChainBase.shorten(800),
testChainBase.shorten(800 / 2),
testChainBase.shorten(800 / 3),
testChainBase.shorten(800 / 4),
testChainBase.shorten(800 / 5),
testChainBase.shorten(800 / 6),
testChainBase.shorten(800 / 7),
testChainBase.shorten(800 / 8),
testChainBase.shorten(3*fsHeaderSafetyNet + 256 + fsMinFullBlocks),
testChainBase.shorten(fsMinFullBlocks + 256 - 1),
testChainForkLightA.shorten(len(testChainBase.blocks) + 80),
testChainForkLightB.shorten(len(testChainBase.blocks) + 81),
testChainForkLightA.shorten(len(testChainBase.blocks) + MaxHeaderFetch),
testChainForkLightB.shorten(len(testChainBase.blocks) + MaxHeaderFetch),
testChainForkHeavy.shorten(len(testChainBase.blocks) + 79),
}
wg.Add(len(chains))
for _, chain := range chains {
go func(blocks []*types.Block) {
newTestBlockchain(blocks)
wg.Done()
}(chain.blocks[1:])
}
wg.Wait()
// Mark the chains pregenerated. Generating a new one will lead to a panic.
pregenerated = true
}
type testChain struct {
genesis *types.Block
chain []common.Hash
headerm map[common.Hash]*types.Header
blockm map[common.Hash]*types.Block
receiptm map[common.Hash][]*types.Receipt
tdm map[common.Hash]*big.Int
blocks []*types.Block
}
// newTestChain creates a blockchain of the given length.
func newTestChain(length int, genesis *types.Block) *testChain {
tc := new(testChain).copy(length)
tc.genesis = genesis
tc.chain = append(tc.chain, genesis.Hash())
tc.headerm[tc.genesis.Hash()] = tc.genesis.Header()
tc.tdm[tc.genesis.Hash()] = tc.genesis.Difficulty()
tc.blockm[tc.genesis.Hash()] = tc.genesis
tc := &testChain{
blocks: []*types.Block{genesis},
}
tc.generate(length-1, 0, genesis, false)
return tc
}
// makeFork creates a fork on top of the test chain.
func (tc *testChain) makeFork(length int, heavy bool, seed byte) *testChain {
fork := tc.copy(tc.len() + length)
fork.generate(length, seed, tc.headBlock(), heavy)
fork := tc.copy(len(tc.blocks) + length)
fork.generate(length, seed, tc.blocks[len(tc.blocks)-1], heavy)
return fork
}
// shorten creates a copy of the chain with the given length. It panics if the
// length is longer than the number of available blocks.
func (tc *testChain) shorten(length int) *testChain {
if length > tc.len() {
panic(fmt.Errorf("can't shorten test chain to %d blocks, it's only %d blocks long", length, tc.len()))
if length > len(tc.blocks) {
panic(fmt.Errorf("can't shorten test chain to %d blocks, it's only %d blocks long", length, len(tc.blocks)))
}
return tc.copy(length)
}
func (tc *testChain) copy(newlen int) *testChain {
cpy := &testChain{
genesis: tc.genesis,
headerm: make(map[common.Hash]*types.Header, newlen),
blockm: make(map[common.Hash]*types.Block, newlen),
receiptm: make(map[common.Hash][]*types.Receipt, newlen),
tdm: make(map[common.Hash]*big.Int, newlen),
if newlen > len(tc.blocks) {
newlen = len(tc.blocks)
}
for i := 0; i < len(tc.chain) && i < newlen; i++ {
hash := tc.chain[i]
cpy.chain = append(cpy.chain, tc.chain[i])
cpy.tdm[hash] = tc.tdm[hash]
cpy.blockm[hash] = tc.blockm[hash]
cpy.headerm[hash] = tc.headerm[hash]
cpy.receiptm[hash] = tc.receiptm[hash]
cpy := &testChain{
blocks: append([]*types.Block{}, tc.blocks[:newlen]...),
}
return cpy
}
@ -115,17 +154,14 @@ func (tc *testChain) copy(newlen int) *testChain {
// contains a transaction and every 5th an uncle to allow testing correct block
// reassembly.
func (tc *testChain) generate(n int, seed byte, parent *types.Block, heavy bool) {
// start := time.Now()
// defer func() { fmt.Printf("test chain generated in %v\n", time.Since(start)) }()
blocks, receipts := core.GenerateChain(params.TestChainConfig, parent, ethash.NewFaker(), testDB, n, func(i int, block *core.BlockGen) {
blocks, _ := core.GenerateChain(params.TestChainConfig, parent, ethash.NewFaker(), testDB, n, func(i int, block *core.BlockGen) {
block.SetCoinbase(common.Address{seed})
// If a heavy chain is requested, delay blocks to raise difficulty
if heavy {
block.OffsetTime(-1)
block.OffsetTime(-9)
}
// Include transactions to the miner to make blocks more interesting.
if parent == tc.genesis && i%22 == 0 {
if parent == tc.blocks[0] && i%22 == 0 {
signer := types.MakeSigner(params.TestChainConfig, block.Number())
tx, err := types.SignTx(types.NewTransaction(block.TxNonce(testAddress), common.Address{seed}, big.NewInt(1000), params.TxGas, block.BaseFee(), nil), signer, testKey)
if err != nil {
@ -136,95 +172,56 @@ func (tc *testChain) generate(n int, seed byte, parent *types.Block, heavy bool)
// if the block number is a multiple of 5, add a bonus uncle to the block
if i > 0 && i%5 == 0 {
block.AddUncle(&types.Header{
ParentHash: block.PrevBlock(i - 1).Hash(),
ParentHash: block.PrevBlock(i - 2).Hash(),
Number: big.NewInt(block.Number().Int64() - 1),
})
}
})
tc.blocks = append(tc.blocks, blocks...)
}
// Convert the block-chain into a hash-chain and header/block maps
td := new(big.Int).Set(tc.td(parent.Hash()))
for i, b := range blocks {
td := td.Add(td, b.Difficulty())
hash := b.Hash()
tc.chain = append(tc.chain, hash)
tc.blockm[hash] = b
tc.headerm[hash] = b.Header()
tc.receiptm[hash] = receipts[i]
tc.tdm[hash] = new(big.Int).Set(td)
var (
testBlockchains = make(map[common.Hash]*testBlockchain)
testBlockchainsLock sync.Mutex
)
type testBlockchain struct {
chain *core.BlockChain
gen sync.Once
}
// newTestBlockchain creates a blockchain database built by running the given blocks,
// either actually running them, or reusing a previously created one. The returned
// chains are *shared*, so *do not* mutate them.
func newTestBlockchain(blocks []*types.Block) *core.BlockChain {
// Retrieve an existing database, or create a new one
head := testGenesis.Hash()
if len(blocks) > 0 {
head = blocks[len(blocks)-1].Hash()
}
}
// len returns the total number of blocks in the chain.
func (tc *testChain) len() int {
return len(tc.chain)
}
// headBlock returns the head of the chain.
func (tc *testChain) headBlock() *types.Block {
return tc.blockm[tc.chain[len(tc.chain)-1]]
}
// td returns the total difficulty of the given block.
func (tc *testChain) td(hash common.Hash) *big.Int {
return tc.tdm[hash]
}
// headersByHash returns headers in order from the given hash.
func (tc *testChain) headersByHash(origin common.Hash, amount int, skip int, reverse bool) []*types.Header {
num, _ := tc.hashToNumber(origin)
return tc.headersByNumber(num, amount, skip, reverse)
}
// headersByNumber returns headers from the given number.
func (tc *testChain) headersByNumber(origin uint64, amount int, skip int, reverse bool) []*types.Header {
result := make([]*types.Header, 0, amount)
if !reverse {
for num := origin; num < uint64(len(tc.chain)) && len(result) < amount; num += uint64(skip) + 1 {
if header, ok := tc.headerm[tc.chain[int(num)]]; ok {
result = append(result, header)
}
}
} else {
for num := int64(origin); num >= 0 && len(result) < amount; num -= int64(skip) + 1 {
if header, ok := tc.headerm[tc.chain[int(num)]]; ok {
result = append(result, header)
}
}
testBlockchainsLock.Lock()
if _, ok := testBlockchains[head]; !ok {
testBlockchains[head] = new(testBlockchain)
}
return result
}
tbc := testBlockchains[head]
testBlockchainsLock.Unlock()
// receipts returns the receipts of the given block hashes.
func (tc *testChain) receipts(hashes []common.Hash) [][]*types.Receipt {
results := make([][]*types.Receipt, 0, len(hashes))
for _, hash := range hashes {
if receipt, ok := tc.receiptm[hash]; ok {
results = append(results, receipt)
// Ensure that the database is generated
tbc.gen.Do(func() {
if pregenerated {
panic("Requested chain generation outside of init")
}
}
return results
}
db := rawdb.NewMemoryDatabase()
core.GenesisBlockForTesting(db, testAddress, big.NewInt(1000000000000000))
// bodies returns the block bodies of the given block hashes.
func (tc *testChain) bodies(hashes []common.Hash) ([][]*types.Transaction, [][]*types.Header) {
transactions := make([][]*types.Transaction, 0, len(hashes))
uncles := make([][]*types.Header, 0, len(hashes))
for _, hash := range hashes {
if block, ok := tc.blockm[hash]; ok {
transactions = append(transactions, block.Transactions())
uncles = append(uncles, block.Uncles())
chain, err := core.NewBlockChain(db, nil, params.TestChainConfig, ethash.NewFaker(), vm.Config{}, nil, nil)
if err != nil {
panic(err)
}
}
return transactions, uncles
}
func (tc *testChain) hashToNumber(target common.Hash) (uint64, bool) {
for num, hash := range tc.chain {
if hash == target {
return uint64(num), true
if n, err := chain.InsertChain(blocks); err != nil {
panic(fmt.Sprintf("block %d: %v", n, err))
}
}
return 0, false
tbc.chain = chain
})
return tbc.chain
}

View File

@ -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)) }

View File

@ -26,6 +26,7 @@ import (
"github.com/ethereum/go-ethereum/common/prque"
"github.com/ethereum/go-ethereum/consensus"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/eth/protocols/eth"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/metrics"
"github.com/ethereum/go-ethereum/trie"
@ -74,10 +75,10 @@ type HeaderRetrievalFn func(common.Hash) *types.Header
type blockRetrievalFn func(common.Hash) *types.Block
// headerRequesterFn is a callback type for sending a header retrieval request.
type headerRequesterFn func(common.Hash) error
type headerRequesterFn func(common.Hash, chan *eth.Response) (*eth.Request, error)
// bodyRequesterFn is a callback type for sending a body retrieval request.
type bodyRequesterFn func([]common.Hash) error
type bodyRequesterFn func([]common.Hash, chan *eth.Response) (*eth.Request, error)
// headerVerifierFn is a callback type to verify a block's header for fast propagation.
type headerVerifierFn func(header *types.Header) error
@ -461,15 +462,28 @@ func (f *BlockFetcher) loop() {
// Create a closure of the fetch and schedule in on a new thread
fetchHeader, hashes := f.fetching[hashes[0]].fetchHeader, hashes
go func() {
go func(peer string) {
if f.fetchingHook != nil {
f.fetchingHook(hashes)
}
for _, hash := range hashes {
headerFetchMeter.Mark(1)
fetchHeader(hash) // Suboptimal, but protocol doesn't allow batch header retrievals
go func(hash common.Hash) {
resCh := make(chan *eth.Response)
req, err := fetchHeader(hash, resCh)
if err != nil {
return // Legacy code, yolo
}
defer req.Close()
res := <-resCh
res.Done <- nil
f.FilterHeaders(peer, *res.Res.(*eth.BlockHeadersPacket), time.Now().Add(res.Time))
}(hash)
}
}()
}(peer)
}
// Schedule the next fetch if blocks are still pending
f.rescheduleFetch(fetchTimer)
@ -497,8 +511,24 @@ func (f *BlockFetcher) loop() {
if f.completingHook != nil {
f.completingHook(hashes)
}
fetchBodies := f.completing[hashes[0]].fetchBodies
bodyFetchMeter.Mark(int64(len(hashes)))
go f.completing[hashes[0]].fetchBodies(hashes)
go func(peer string, hashes []common.Hash) {
resCh := make(chan *eth.Response)
req, err := fetchBodies(hashes, resCh)
if err != nil {
return // Legacy code, yolo
}
defer req.Close()
res := <-resCh
res.Done <- nil
txs, uncles := res.Res.(*eth.BlockBodiesPacket).Unpack()
f.FilterBodies(peer, txs, uncles, time.Now())
}(peer, hashes)
}
// Schedule the next fetch if blocks are still pending
f.rescheduleComplete(completeTimer)

View File

@ -30,6 +30,7 @@ import (
"github.com/ethereum/go-ethereum/core/rawdb"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/eth/protocols/eth"
"github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/trie"
)
@ -60,8 +61,8 @@ func makeChain(n int, seed byte, parent *types.Block) ([]common.Hash, map[common
block.AddTx(tx)
}
// If the block number is a multiple of 5, add a bonus uncle to the block
if i%5 == 0 {
block.AddUncle(&types.Header{ParentHash: block.PrevBlock(i - 1).Hash(), Number: big.NewInt(int64(i - 1))})
if i > 0 && i%5 == 0 {
block.AddUncle(&types.Header{ParentHash: block.PrevBlock(i - 2).Hash(), Number: big.NewInt(int64(i - 1))})
}
})
hashes := make([]common.Hash, n+1)
@ -195,16 +196,26 @@ func (f *fetcherTester) makeHeaderFetcher(peer string, blocks map[common.Hash]*t
closure[hash] = block
}
// Create a function that return a header from the closure
return func(hash common.Hash) error {
return func(hash common.Hash, sink chan *eth.Response) (*eth.Request, error) {
// Gather the blocks to return
headers := make([]*types.Header, 0, 1)
if block, ok := closure[hash]; ok {
headers = append(headers, block.Header())
}
// Return on a new thread
go f.fetcher.FilterHeaders(peer, headers, time.Now().Add(drift))
return nil
req := &eth.Request{
Peer: peer,
}
res := &eth.Response{
Req: req,
Res: (*eth.BlockHeadersPacket)(&headers),
Time: drift,
Done: make(chan error, 1), // Ignore the returned status
}
go func() {
sink <- res
}()
return req, nil
}
}
@ -215,7 +226,7 @@ func (f *fetcherTester) makeBodyFetcher(peer string, blocks map[common.Hash]*typ
closure[hash] = block
}
// Create a function that returns blocks from the closure
return func(hashes []common.Hash) error {
return func(hashes []common.Hash, sink chan *eth.Response) (*eth.Request, error) {
// Gather the block bodies to return
transactions := make([][]*types.Transaction, 0, len(hashes))
uncles := make([][]*types.Header, 0, len(hashes))
@ -227,14 +238,33 @@ func (f *fetcherTester) makeBodyFetcher(peer string, blocks map[common.Hash]*typ
}
}
// Return on a new thread
go f.fetcher.FilterBodies(peer, transactions, uncles, time.Now().Add(drift))
return nil
bodies := make([]*eth.BlockBody, len(transactions))
for i, txs := range transactions {
bodies[i] = &eth.BlockBody{
Transactions: txs,
Uncles: uncles[i],
}
}
req := &eth.Request{
Peer: peer,
}
res := &eth.Response{
Req: req,
Res: (*eth.BlockBodiesPacket)(&bodies),
Time: drift,
Done: make(chan error, 1), // Ignore the returned status
}
go func() {
sink <- res
}()
return req, nil
}
}
// verifyFetchingEvent verifies that one single event arrive on a fetching channel.
func verifyFetchingEvent(t *testing.T, fetching chan []common.Hash, arrive bool) {
t.Helper()
if arrive {
select {
case <-fetching:
@ -252,6 +282,8 @@ func verifyFetchingEvent(t *testing.T, fetching chan []common.Hash, arrive bool)
// verifyCompletingEvent verifies that one single event arrive on an completing channel.
func verifyCompletingEvent(t *testing.T, completing chan []common.Hash, arrive bool) {
t.Helper()
if arrive {
select {
case <-completing:
@ -269,6 +301,8 @@ func verifyCompletingEvent(t *testing.T, completing chan []common.Hash, arrive b
// verifyImportEvent verifies that one single event arrive on an import channel.
func verifyImportEvent(t *testing.T, imported chan interface{}, arrive bool) {
t.Helper()
if arrive {
select {
case <-imported:
@ -287,6 +321,8 @@ func verifyImportEvent(t *testing.T, imported chan interface{}, arrive bool) {
// verifyImportCount verifies that exactly count number of events arrive on an
// import hook channel.
func verifyImportCount(t *testing.T, imported chan interface{}, count int) {
t.Helper()
for i := 0; i < count; i++ {
select {
case <-imported:
@ -299,6 +335,8 @@ func verifyImportCount(t *testing.T, imported chan interface{}, count int) {
// verifyImportDone verifies that no more events are arriving on an import channel.
func verifyImportDone(t *testing.T, imported chan interface{}) {
t.Helper()
select {
case <-imported:
t.Fatalf("extra block imported")
@ -308,6 +346,8 @@ func verifyImportDone(t *testing.T, imported chan interface{}) {
// verifyChainHeight verifies the chain height is as expected.
func verifyChainHeight(t *testing.T, fetcher *fetcherTester, height uint64) {
t.Helper()
if fetcher.chainHeight() != height {
t.Fatalf("chain height mismatch, got %d, want %d", fetcher.chainHeight(), height)
}
@ -368,13 +408,13 @@ func testConcurrentAnnouncements(t *testing.T, light bool) {
secondBodyFetcher := tester.makeBodyFetcher("second", blocks, 0)
counter := uint32(0)
firstHeaderWrapper := func(hash common.Hash) error {
firstHeaderWrapper := func(hash common.Hash, sink chan *eth.Response) (*eth.Request, error) {
atomic.AddUint32(&counter, 1)
return firstHeaderFetcher(hash)
return firstHeaderFetcher(hash, sink)
}
secondHeaderWrapper := func(hash common.Hash) error {
secondHeaderWrapper := func(hash common.Hash, sink chan *eth.Response) (*eth.Request, error) {
atomic.AddUint32(&counter, 1)
return secondHeaderFetcher(hash)
return secondHeaderFetcher(hash, sink)
}
// Iteratively announce blocks until all are imported
imported := make(chan interface{})
@ -468,15 +508,20 @@ func testPendingDeduplication(t *testing.T, light bool) {
delay := 50 * time.Millisecond
counter := uint32(0)
headerWrapper := func(hash common.Hash) error {
headerWrapper := func(hash common.Hash, sink chan *eth.Response) (*eth.Request, error) {
atomic.AddUint32(&counter, 1)
// Simulate a long running fetch
go func() {
time.Sleep(delay)
headerFetcher(hash)
}()
return nil
resink := make(chan *eth.Response)
req, err := headerFetcher(hash, resink)
if err == nil {
go func() {
res := <-resink
time.Sleep(delay)
sink <- res
}()
}
return req, err
}
checkNonExist := func() bool {
return tester.getBlock(hashes[0]) == nil

View File

@ -83,8 +83,8 @@ type handlerConfig struct {
TxPool txPool // Transaction pool to propagate from
Merger *consensus.Merger // The manager for eth1/2 transition
Network uint64 // Network identifier to adfvertise
Sync downloader.SyncMode // Whether to fast or full sync
BloomCache uint64 // Megabytes to alloc for fast sync bloom
Sync downloader.SyncMode // Whether to snap or full sync
BloomCache uint64 // Megabytes to alloc for snap sync bloom
EventMux *event.TypeMux // Legacy event mux, deprecate for `feed`
Checkpoint *params.TrustedCheckpoint // Hard coded checkpoint for sync challenges
Whitelist map[uint64]common.Hash // Hard coded whitelist for sync challenged
@ -94,8 +94,7 @@ type handler struct {
networkID uint64
forkFilter forkid.Filter // Fork ID filter, constant across the lifetime of the node
fastSync uint32 // Flag whether fast sync is enabled (gets disabled if we already have blocks)
snapSync uint32 // Flag whether fast sync should operate on top of the snap protocol
snapSync uint32 // Flag whether snap sync is enabled (gets disabled if we already have blocks)
acceptTxs uint32 // Flag whether we're considered synchronised (enables transaction processing)
checkpointNumber uint64 // Block number for the sync progress validator to cross reference
@ -147,29 +146,26 @@ func newHandler(config *handlerConfig) (*handler, error) {
quitSync: make(chan struct{}),
}
if config.Sync == downloader.FullSync {
// The database seems empty as the current block is the genesis. Yet the fast
// block is ahead, so fast sync was enabled for this node at a certain point.
// The database seems empty as the current block is the genesis. Yet the snap
// block is ahead, so snap sync was enabled for this node at a certain point.
// The scenarios where this can happen is
// * if the user manually (or via a bad block) rolled back a fast sync node
// * if the user manually (or via a bad block) rolled back a snap sync node
// below the sync point.
// * the last fast sync is not finished while user specifies a full sync this
// * the last snap sync is not finished while user specifies a full sync this
// time. But we don't have any recent state for full sync.
// In these cases however it's safe to reenable fast sync.
// In these cases however it's safe to reenable snap sync.
fullBlock, fastBlock := h.chain.CurrentBlock(), h.chain.CurrentFastBlock()
if fullBlock.NumberU64() == 0 && fastBlock.NumberU64() > 0 {
h.fastSync = uint32(1)
log.Warn("Switch sync mode from full sync to fast sync")
h.snapSync = uint32(1)
log.Warn("Switch sync mode from full sync to snap sync")
}
} else {
if h.chain.CurrentBlock().NumberU64() > 0 {
// Print warning log if database is not empty to run fast sync.
log.Warn("Switch sync mode from fast sync to full sync")
// Print warning log if database is not empty to run snap sync.
log.Warn("Switch sync mode from snap sync to full sync")
} else {
// If fast sync was requested and our database is empty, grant it
h.fastSync = uint32(1)
if config.Sync == downloader.SnapSync {
h.snapSync = uint32(1)
}
// If snap sync was requested and our database is empty, grant it
h.snapSync = uint32(1)
}
}
// If we have trusted checkpoints, enforce them on the chain
@ -177,14 +173,14 @@ func newHandler(config *handlerConfig) (*handler, error) {
h.checkpointNumber = (config.Checkpoint.SectionIndex+1)*params.CHTFrequency - 1
h.checkpointHash = config.Checkpoint.SectionHead
}
// Construct the downloader (long sync) and its backing state bloom if fast
// Construct the downloader (long sync) and its backing state bloom if snap
// sync is requested. The downloader is responsible for deallocating the state
// bloom when it's done.
// Note: we don't enable it if snap-sync is performed, since it's very heavy
// and the heal-portion of the snap sync is much lighter than fast. What we particularly
// and the heal-portion of the snap sync is much lighter than snap. What we particularly
// want to avoid, is a 90%-finished (but restarted) snap-sync to begin
// indexing the entire trie
if atomic.LoadUint32(&h.fastSync) == 1 && atomic.LoadUint32(&h.snapSync) == 0 {
if atomic.LoadUint32(&h.snapSync) == 1 && atomic.LoadUint32(&h.snapSync) == 0 {
h.stateBloom = trie.NewSyncBloom(config.BloomCache, config.Database)
}
h.downloader = downloader.New(h.checkpointNumber, config.Database, h.stateBloom, h.eventMux, h.chain, nil, h.removePeer)
@ -236,12 +232,12 @@ func newHandler(config *handlerConfig) (*handler, error) {
log.Warn("Unsynced yet, discarded propagated block", "number", blocks[0].Number(), "hash", blocks[0].Hash())
return 0, nil
}
// If fast sync is running, deny importing weird blocks. This is a problematic
// clause when starting up a new network, because fast-syncing miners might not
// If snap sync is running, deny importing weird blocks. This is a problematic
// clause when starting up a new network, because snap-syncing miners might not
// accept each others' blocks until a restart. Unfortunately we haven't figured
// out a way yet where nodes can decide unilaterally whether the network is new
// or not. This should be fixed if we figure out a solution.
if atomic.LoadUint32(&h.fastSync) == 1 {
if atomic.LoadUint32(&h.snapSync) == 1 {
log.Warn("Fast syncing, discarded propagated block", "number", blocks[0].Number(), "hash", blocks[0].Hash())
return 0, nil
}
@ -365,30 +361,93 @@ func (h *handler) runEthPeer(peer *eth.Peer, handler eth.Handler) error {
// after this will be sent via broadcasts.
h.syncTransactions(peer)
// Create a notification channel for pending requests if the peer goes down
dead := make(chan struct{})
defer close(dead)
// If we have a trusted CHT, reject all peers below that (avoid fast sync eclipse)
if h.checkpointHash != (common.Hash{}) {
// Request the peer's checkpoint header for chain height/weight validation
if err := peer.RequestHeadersByNumber(h.checkpointNumber, 1, 0, false); err != nil {
resCh := make(chan *eth.Response)
if _, err := peer.RequestHeadersByNumber(h.checkpointNumber, 1, 0, false, resCh); err != nil {
return err
}
// Start a timer to disconnect if the peer doesn't reply in time
p.syncDrop = time.AfterFunc(syncChallengeTimeout, func() {
peer.Log().Warn("Checkpoint challenge timed out, dropping", "addr", peer.RemoteAddr(), "type", peer.Name())
h.removePeer(peer.ID())
})
// Make sure it's cleaned up if the peer dies off
defer func() {
if p.syncDrop != nil {
p.syncDrop.Stop()
p.syncDrop = nil
go func() {
timeout := time.NewTimer(syncChallengeTimeout)
defer timeout.Stop()
select {
case res := <-resCh:
headers := ([]*types.Header)(*res.Res.(*eth.BlockHeadersPacket))
if len(headers) == 0 {
// If we're doing a snap sync, we must enforce the checkpoint
// block to avoid eclipse attacks. Unsynced nodes are welcome
// to connect after we're done joining the network.
if atomic.LoadUint32(&h.snapSync) == 1 {
peer.Log().Warn("Dropping unsynced node during sync", "addr", peer.RemoteAddr(), "type", peer.Name())
res.Done <- errors.New("unsynced node cannot serve sync")
return
}
res.Done <- nil
return
}
// Validate the header and either drop the peer or continue
if len(headers) > 1 {
res.Done <- errors.New("too many headers in checkpoint response")
return
}
if headers[0].Hash() != h.checkpointHash {
res.Done <- errors.New("checkpoint hash mismatch")
return
}
res.Done <- nil
case <-timeout.C:
peer.Log().Warn("Checkpoint challenge timed out, dropping", "addr", peer.RemoteAddr(), "type", peer.Name())
h.removePeer(peer.ID())
case <-dead:
// Peer handler terminated, abort all goroutines
}
}()
}
// If we have any explicit whitelist block hashes, request them
for number := range h.whitelist {
if err := peer.RequestHeadersByNumber(number, 1, 0, false); err != nil {
for number, hash := range h.whitelist {
resCh := make(chan *eth.Response)
if _, err := peer.RequestHeadersByNumber(number, 1, 0, false, resCh); err != nil {
return err
}
go func(number uint64, hash common.Hash) {
timeout := time.NewTimer(syncChallengeTimeout)
defer timeout.Stop()
select {
case res := <-resCh:
headers := ([]*types.Header)(*res.Res.(*eth.BlockHeadersPacket))
if len(headers) == 0 {
// Whitelisted blocks are allowed to be missing if the remote
// node is not yet synced
res.Done <- nil
return
}
// Validate the header and either drop the peer or continue
if len(headers) > 1 {
res.Done <- errors.New("too many headers in whitelist response")
return
}
if headers[0].Number.Uint64() != number || headers[0].Hash() != hash {
peer.Log().Info("Whitelist mismatch, dropping peer", "number", number, "hash", headers[0].Hash(), "want", hash)
res.Done <- errors.New("whitelist block mismatch")
return
}
peer.Log().Debug("Whitelist block verified", "number", number, "hash", hash)
case <-timeout.C:
peer.Log().Warn("Whitelist challenge timed out, dropping", "addr", peer.RemoteAddr(), "type", peer.Name())
h.removePeer(peer.ID())
}
}(number, hash)
}
// Handle incoming messages until the connection is torn down
return handler(peer)

View File

@ -17,7 +17,6 @@
package eth
import (
"errors"
"fmt"
"math/big"
"sync/atomic"
@ -27,7 +26,6 @@ import (
"github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/eth/protocols/eth"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/p2p/enode"
"github.com/ethereum/go-ethereum/trie"
)
@ -64,25 +62,6 @@ func (h *ethHandler) AcceptTxs() bool {
func (h *ethHandler) Handle(peer *eth.Peer, packet eth.Packet) error {
// Consume any broadcasts and announces, forwarding the rest to the downloader
switch packet := packet.(type) {
case *eth.BlockHeadersPacket:
return h.handleHeaders(peer, *packet)
case *eth.BlockBodiesPacket:
txset, uncleset := packet.Unpack()
return h.handleBodies(peer, txset, uncleset)
case *eth.NodeDataPacket:
if err := h.downloader.DeliverNodeData(peer.ID(), *packet); err != nil {
log.Debug("Failed to deliver node state data", "err", err)
}
return nil
case *eth.ReceiptsPacket:
if err := h.downloader.DeliverReceipts(peer.ID(), *packet); err != nil {
log.Debug("Failed to deliver receipts", "err", err)
}
return nil
case *eth.NewBlockHashesPacket:
hashes, numbers := packet.Unpack()
return h.handleBlockAnnounces(peer, hashes, numbers)
@ -104,79 +83,6 @@ func (h *ethHandler) Handle(peer *eth.Peer, packet eth.Packet) error {
}
}
// handleHeaders is invoked from a peer's message handler when it transmits a batch
// of headers for the local node to process.
func (h *ethHandler) handleHeaders(peer *eth.Peer, headers []*types.Header) error {
p := h.peers.peer(peer.ID())
if p == nil {
return errors.New("unregistered during callback")
}
// If no headers were received, but we're expencting a checkpoint header, consider it that
if len(headers) == 0 && p.syncDrop != nil {
// Stop the timer either way, decide later to drop or not
p.syncDrop.Stop()
p.syncDrop = nil
// If we're doing a fast (or snap) sync, we must enforce the checkpoint block to avoid
// eclipse attacks. Unsynced nodes are welcome to connect after we're done
// joining the network
if atomic.LoadUint32(&h.fastSync) == 1 {
peer.Log().Warn("Dropping unsynced node during sync", "addr", peer.RemoteAddr(), "type", peer.Name())
return errors.New("unsynced node cannot serve sync")
}
}
// Filter out any explicitly requested headers, deliver the rest to the downloader
filter := len(headers) == 1
if filter {
// If it's a potential sync progress check, validate the content and advertised chain weight
if p.syncDrop != nil && headers[0].Number.Uint64() == h.checkpointNumber {
// Disable the sync drop timer
p.syncDrop.Stop()
p.syncDrop = nil
// Validate the header and either drop the peer or continue
if headers[0].Hash() != h.checkpointHash {
return errors.New("checkpoint hash mismatch")
}
return nil
}
// Otherwise if it's a whitelisted block, validate against the set
if want, ok := h.whitelist[headers[0].Number.Uint64()]; ok {
if hash := headers[0].Hash(); want != hash {
peer.Log().Info("Whitelist mismatch, dropping peer", "number", headers[0].Number.Uint64(), "hash", hash, "want", want)
return errors.New("whitelist block mismatch")
}
peer.Log().Debug("Whitelist block verified", "number", headers[0].Number.Uint64(), "hash", want)
}
// Irrelevant of the fork checks, send the header to the fetcher just in case
headers = h.blockFetcher.FilterHeaders(peer.ID(), headers, time.Now())
}
if len(headers) > 0 || !filter {
err := h.downloader.DeliverHeaders(peer.ID(), headers)
if err != nil {
log.Debug("Failed to deliver headers", "err", err)
}
}
return nil
}
// handleBodies is invoked from a peer's message handler when it transmits a batch
// of block bodies for the local node to process.
func (h *ethHandler) handleBodies(peer *eth.Peer, txs [][]*types.Transaction, uncles [][]*types.Header) error {
// Filter out any explicitly requested bodies, deliver the rest to the downloader
filter := len(txs) > 0 || len(uncles) > 0
if filter {
txs, uncles = h.blockFetcher.FilterBodies(peer.ID(), txs, uncles, time.Now())
}
if len(txs) > 0 || len(uncles) > 0 || !filter {
err := h.downloader.DeliverBodies(peer.ID(), txs, uncles)
if err != nil {
log.Debug("Failed to deliver bodies", "err", err)
}
}
return nil
}
// handleBlockAnnounces is invoked from a peer's message handler when it transmits a
// batch of block announcements for the local node to process.
func (h *ethHandler) handleBlockAnnounces(peer *eth.Peer, hashes []common.Hash, numbers []uint64) error {

View File

@ -354,7 +354,7 @@ func testSendTransactions(t *testing.T, protocol uint) {
seen := make(map[common.Hash]struct{})
for len(seen) < len(insert) {
switch protocol {
case 65, 66:
case 66:
select {
case hashes := <-anns:
for _, hash := range hashes {
@ -364,7 +364,7 @@ func testSendTransactions(t *testing.T, protocol uint) {
seen[hash] = struct{}{}
}
case <-bcasts:
t.Errorf("initial tx broadcast received on post eth/65")
t.Errorf("initial tx broadcast received on post eth/66")
}
default:
@ -389,6 +389,7 @@ func testTransactionPropagation(t *testing.T, protocol uint) {
// to receive them. We need multiple sinks since a one-to-one peering would
// broadcast all transactions without announcement.
source := newTestHandler()
source.handler.snapSync = 0 // Avoid requiring snap, otherwise some will be dropped below
defer source.close()
sinks := make([]*testHandler, 10)
@ -406,7 +407,7 @@ func testTransactionPropagation(t *testing.T, protocol uint) {
defer sourcePipe.Close()
defer sinkPipe.Close()
sourcePeer := eth.NewPeer(protocol, p2p.NewPeerPipe(enode.ID{byte(i)}, "", nil, sourcePipe), sourcePipe, source.txpool)
sourcePeer := eth.NewPeer(protocol, p2p.NewPeerPipe(enode.ID{byte(i + 1)}, "", nil, sourcePipe), sourcePipe, source.txpool)
sinkPeer := eth.NewPeer(protocol, p2p.NewPeerPipe(enode.ID{0}, "", nil, sinkPipe), sinkPipe, sink.txpool)
defer sourcePeer.Close()
defer sinkPeer.Close()
@ -438,12 +439,13 @@ func testTransactionPropagation(t *testing.T, protocol uint) {
// Iterate through all the sinks and ensure they all got the transactions
for i := range sinks {
for arrived := 0; arrived < len(txs); {
for arrived, timeout := 0, false; arrived < len(txs) && !timeout; {
select {
case event := <-txChs[i]:
arrived += len(event.Txs)
case <-time.NewTimer(time.Second).C:
case <-time.After(time.Second):
t.Errorf("sink %d: transaction propagation timed out: have %d, want %d", i, arrived, len(txs))
timeout = true
}
}
}
@ -463,23 +465,23 @@ func TestCheckpointChallenge(t *testing.T) {
}{
// If checkpointing is not enabled locally, don't challenge and don't drop
{downloader.FullSync, false, false, false, false, false},
{downloader.FastSync, false, false, false, false, false},
{downloader.SnapSync, false, false, false, false, false},
// If checkpointing is enabled locally and remote response is empty, only drop during fast sync
{downloader.FullSync, true, false, true, false, false},
{downloader.FastSync, true, false, true, false, true}, // Special case, fast sync, unsynced peer
{downloader.SnapSync, true, false, true, false, true}, // Special case, fast sync, unsynced peer
// If checkpointing is enabled locally and remote response mismatches, always drop
{downloader.FullSync, true, false, false, false, true},
{downloader.FastSync, true, false, false, false, true},
{downloader.SnapSync, true, false, false, false, true},
// If checkpointing is enabled locally and remote response matches, never drop
{downloader.FullSync, true, false, false, true, false},
{downloader.FastSync, true, false, false, true, false},
{downloader.SnapSync, true, false, false, true, false},
// If checkpointing is enabled locally and remote times out, always drop
{downloader.FullSync, true, true, false, true, true},
{downloader.FastSync, true, true, false, true, true},
{downloader.SnapSync, true, true, false, true, true},
}
for _, tt := range tests {
t.Run(fmt.Sprintf("sync %v checkpoint %v timeout %v empty %v match %v", tt.syncmode, tt.checkpoint, tt.timeout, tt.empty, tt.match), func(t *testing.T) {
@ -500,10 +502,10 @@ func testCheckpointChallenge(t *testing.T, syncmode downloader.SyncMode, checkpo
handler := newTestHandler()
defer handler.close()
if syncmode == downloader.FastSync {
atomic.StoreUint32(&handler.handler.fastSync, 1)
if syncmode == downloader.SnapSync {
atomic.StoreUint32(&handler.handler.snapSync, 1)
} else {
atomic.StoreUint32(&handler.handler.fastSync, 0)
atomic.StoreUint32(&handler.handler.snapSync, 0)
}
var response *types.Header
if checkpoint {

View File

@ -152,7 +152,7 @@ func newTestHandlerWithBlocks(blocks int) *testHandler {
TxPool: txpool,
Merger: consensus.NewMerger(rawdb.NewMemoryDatabase()),
Network: 1,
Sync: downloader.FastSync,
Sync: downloader.SnapSync,
BloomCache: 1,
})
handler.Start(1000)

View File

@ -18,8 +18,6 @@ package eth
import (
"math/big"
"sync"
"time"
"github.com/ethereum/go-ethereum/eth/protocols/eth"
"github.com/ethereum/go-ethereum/eth/protocols/snap"
@ -36,11 +34,8 @@ type ethPeerInfo struct {
// ethPeer is a wrapper around eth.Peer to maintain a few extra metadata.
type ethPeer struct {
*eth.Peer
snapExt *snapPeer // Satellite `snap` connection
syncDrop *time.Timer // Connection dropper if `eth` sync progress isn't validated in time
snapExt *snapPeer // Satellite `snap` connection
snapWait chan struct{} // Notification channel for snap connections
lock sync.RWMutex // Mutex protecting the internal fields
}
// info gathers and returns some `eth` protocol metadata known about a peer.

View File

@ -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
}
}
}

View File

@ -21,6 +21,7 @@ import (
"fmt"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/rlp"
@ -34,11 +35,13 @@ func handleGetBlockHeaders66(backend Backend, msg Decoder, peer *Peer) error {
if err := msg.Decode(&query); err != nil {
return fmt.Errorf("%w: message %v: %v", errDecode, msg, err)
}
response := answerGetBlockHeadersQuery(backend, query.GetBlockHeadersPacket, peer)
response := ServiceGetBlockHeadersQuery(backend.Chain(), query.GetBlockHeadersPacket, peer)
return peer.ReplyBlockHeaders(query.RequestId, response)
}
func answerGetBlockHeadersQuery(backend Backend, query *GetBlockHeadersPacket, peer *Peer) []*types.Header {
// ServiceGetBlockHeadersQuery assembles the response to a header query. It is
// exposed to allow external packages to test protocol behavior.
func ServiceGetBlockHeadersQuery(chain *core.BlockChain, query *GetBlockHeadersPacket, peer *Peer) []*types.Header {
hashMode := query.Origin.Hash != (common.Hash{})
first := true
maxNonCanonical := uint64(100)
@ -58,15 +61,15 @@ func answerGetBlockHeadersQuery(backend Backend, query *GetBlockHeadersPacket, p
if hashMode {
if first {
first = false
origin = backend.Chain().GetHeaderByHash(query.Origin.Hash)
origin = chain.GetHeaderByHash(query.Origin.Hash)
if origin != nil {
query.Origin.Number = origin.Number.Uint64()
}
} else {
origin = backend.Chain().GetHeader(query.Origin.Hash, query.Origin.Number)
origin = chain.GetHeader(query.Origin.Hash, query.Origin.Number)
}
} else {
origin = backend.Chain().GetHeaderByNumber(query.Origin.Number)
origin = chain.GetHeaderByNumber(query.Origin.Number)
}
if origin == nil {
break
@ -82,7 +85,7 @@ func answerGetBlockHeadersQuery(backend Backend, query *GetBlockHeadersPacket, p
if ancestor == 0 {
unknown = true
} else {
query.Origin.Hash, query.Origin.Number = backend.Chain().GetAncestor(query.Origin.Hash, query.Origin.Number, ancestor, &maxNonCanonical)
query.Origin.Hash, query.Origin.Number = chain.GetAncestor(query.Origin.Hash, query.Origin.Number, ancestor, &maxNonCanonical)
unknown = (query.Origin.Hash == common.Hash{})
}
case hashMode && !query.Reverse:
@ -96,9 +99,9 @@ func answerGetBlockHeadersQuery(backend Backend, query *GetBlockHeadersPacket, p
peer.Log().Warn("GetBlockHeaders skip overflow attack", "current", current, "skip", query.Skip, "next", next, "attacker", infos)
unknown = true
} else {
if header := backend.Chain().GetHeaderByNumber(next); header != nil {
if header := chain.GetHeaderByNumber(next); header != nil {
nextHash := header.Hash()
expOldHash, _ := backend.Chain().GetAncestor(nextHash, next, query.Skip+1, &maxNonCanonical)
expOldHash, _ := chain.GetAncestor(nextHash, next, query.Skip+1, &maxNonCanonical)
if expOldHash == query.Origin.Hash {
query.Origin.Hash, query.Origin.Number = nextHash, next
} else {
@ -130,11 +133,13 @@ func handleGetBlockBodies66(backend Backend, msg Decoder, peer *Peer) error {
if err := msg.Decode(&query); err != nil {
return fmt.Errorf("%w: message %v: %v", errDecode, msg, err)
}
response := answerGetBlockBodiesQuery(backend, query.GetBlockBodiesPacket, peer)
response := ServiceGetBlockBodiesQuery(backend.Chain(), query.GetBlockBodiesPacket)
return peer.ReplyBlockBodiesRLP(query.RequestId, response)
}
func answerGetBlockBodiesQuery(backend Backend, query GetBlockBodiesPacket, peer *Peer) []rlp.RawValue {
// ServiceGetBlockBodiesQuery assembles the response to a body query. It is
// exposed to allow external packages to test protocol behavior.
func ServiceGetBlockBodiesQuery(chain *core.BlockChain, query GetBlockBodiesPacket) []rlp.RawValue {
// Gather blocks until the fetch or network limits is reached
var (
bytes int
@ -145,7 +150,7 @@ func answerGetBlockBodiesQuery(backend Backend, query GetBlockBodiesPacket, peer
lookups >= 2*maxBodiesServe {
break
}
if data := backend.Chain().GetBodyRLP(hash); len(data) != 0 {
if data := chain.GetBodyRLP(hash); len(data) != 0 {
bodies = append(bodies, data)
bytes += len(data)
}
@ -159,11 +164,13 @@ func handleGetNodeData66(backend Backend, msg Decoder, peer *Peer) error {
if err := msg.Decode(&query); err != nil {
return fmt.Errorf("%w: message %v: %v", errDecode, msg, err)
}
response := answerGetNodeDataQuery(backend, query.GetNodeDataPacket, peer)
response := ServiceGetNodeDataQuery(backend.Chain(), backend.StateBloom(), query.GetNodeDataPacket)
return peer.ReplyNodeData(query.RequestId, response)
}
func answerGetNodeDataQuery(backend Backend, query GetNodeDataPacket, peer *Peer) [][]byte {
// ServiceGetNodeDataQuery assembles the response to a node data query. It is
// exposed to allow external packages to test protocol behavior.
func ServiceGetNodeDataQuery(chain *core.BlockChain, bloom *trie.SyncBloom, query GetNodeDataPacket) [][]byte {
// Gather state data until the fetch or network limits is reached
var (
bytes int
@ -175,14 +182,14 @@ func answerGetNodeDataQuery(backend Backend, query GetNodeDataPacket, peer *Peer
break
}
// Retrieve the requested state entry
if bloom := backend.StateBloom(); bloom != nil && !bloom.Contains(hash[:]) {
if bloom != nil && !bloom.Contains(hash[:]) {
// Only lookup the trie node if there's chance that we actually have it
continue
}
entry, err := backend.Chain().TrieNode(hash)
entry, err := chain.TrieNode(hash)
if len(entry) == 0 || err != nil {
// Read the contract code with prefix only to save unnecessary lookups.
entry, err = backend.Chain().ContractCodeWithPrefix(hash)
entry, err = chain.ContractCodeWithPrefix(hash)
}
if err == nil && len(entry) > 0 {
nodes = append(nodes, entry)
@ -198,11 +205,13 @@ func handleGetReceipts66(backend Backend, msg Decoder, peer *Peer) error {
if err := msg.Decode(&query); err != nil {
return fmt.Errorf("%w: message %v: %v", errDecode, msg, err)
}
response := answerGetReceiptsQuery(backend, query.GetReceiptsPacket, peer)
response := ServiceGetReceiptsQuery(backend.Chain(), query.GetReceiptsPacket)
return peer.ReplyReceiptsRLP(query.RequestId, response)
}
func answerGetReceiptsQuery(backend Backend, query GetReceiptsPacket, peer *Peer) []rlp.RawValue {
// ServiceGetReceiptsQuery assembles the response to a receipt query. It is
// exposed to allow external packages to test protocol behavior.
func ServiceGetReceiptsQuery(chain *core.BlockChain, query GetReceiptsPacket) []rlp.RawValue {
// Gather state data until the fetch or network limits is reached
var (
bytes int
@ -214,9 +223,9 @@ func answerGetReceiptsQuery(backend Backend, query GetReceiptsPacket, peer *Peer
break
}
// Retrieve the requested block's receipts
results := backend.Chain().GetReceiptsByHash(hash)
results := chain.GetReceiptsByHash(hash)
if results == nil {
if header := backend.Chain().GetHeaderByHash(hash); header == nil || header.ReceiptHash != types.EmptyRootHash {
if header := chain.GetHeaderByHash(hash); header == nil || header.ReceiptHash != types.EmptyRootHash {
continue
}
}
@ -277,9 +286,11 @@ func handleBlockHeaders66(backend Backend, msg Decoder, peer *Peer) error {
if err := msg.Decode(res); err != nil {
return fmt.Errorf("%w: message %v: %v", errDecode, msg, err)
}
requestTracker.Fulfil(peer.id, peer.version, BlockHeadersMsg, res.RequestId)
return backend.Handle(peer, &res.BlockHeadersPacket)
return peer.dispatchResponse(&Response{
id: res.RequestId,
code: BlockHeadersMsg,
Res: &res.BlockHeadersPacket,
})
}
func handleBlockBodies66(backend Backend, msg Decoder, peer *Peer) error {
@ -288,9 +299,11 @@ func handleBlockBodies66(backend Backend, msg Decoder, peer *Peer) error {
if err := msg.Decode(res); err != nil {
return fmt.Errorf("%w: message %v: %v", errDecode, msg, err)
}
requestTracker.Fulfil(peer.id, peer.version, BlockBodiesMsg, res.RequestId)
return backend.Handle(peer, &res.BlockBodiesPacket)
return peer.dispatchResponse(&Response{
id: res.RequestId,
code: BlockBodiesMsg,
Res: &res.BlockBodiesPacket,
})
}
func handleNodeData66(backend Backend, msg Decoder, peer *Peer) error {
@ -299,9 +312,11 @@ func handleNodeData66(backend Backend, msg Decoder, peer *Peer) error {
if err := msg.Decode(res); err != nil {
return fmt.Errorf("%w: message %v: %v", errDecode, msg, err)
}
requestTracker.Fulfil(peer.id, peer.version, NodeDataMsg, res.RequestId)
return backend.Handle(peer, &res.NodeDataPacket)
return peer.dispatchResponse(&Response{
id: res.RequestId,
code: NodeDataMsg,
Res: &res.NodeDataPacket,
})
}
func handleReceipts66(backend Backend, msg Decoder, peer *Peer) error {
@ -310,9 +325,11 @@ func handleReceipts66(backend Backend, msg Decoder, peer *Peer) error {
if err := msg.Decode(res); err != nil {
return fmt.Errorf("%w: message %v: %v", errDecode, msg, err)
}
requestTracker.Fulfil(peer.id, peer.version, ReceiptsMsg, res.RequestId)
return backend.Handle(peer, &res.ReceiptsPacket)
return peer.dispatchResponse(&Response{
id: res.RequestId,
code: ReceiptsMsg,
Res: &res.ReceiptsPacket,
})
}
func handleNewPooledTransactionHashes(backend Backend, msg Decoder, peer *Peer) error {

View File

@ -84,6 +84,10 @@ type Peer struct {
txBroadcast chan []common.Hash // Channel used to queue transaction propagation requests
txAnnounce chan []common.Hash // Channel used to queue transaction announcement requests
reqDispatch chan *request // Dispatch channel to send requests and track then until fulfilment
reqCancel chan *cancel // Dispatch channel to cancel pending requests and untrack them
resDispatch chan *response // Dispatch channel to fulfil pending requests and untrack them
term chan struct{} // Termination channel to stop the broadcasters
lock sync.RWMutex // Mutex protecting the internal fields
}
@ -102,6 +106,9 @@ func NewPeer(version uint, p *p2p.Peer, rw p2p.MsgReadWriter, txpool TxPool) *Pe
queuedBlockAnns: make(chan *types.Block, maxQueuedBlockAnns),
txBroadcast: make(chan []common.Hash),
txAnnounce: make(chan []common.Hash),
reqDispatch: make(chan *request),
reqCancel: make(chan *cancel),
resDispatch: make(chan *response),
txpool: txpool,
term: make(chan struct{}),
}
@ -109,6 +116,7 @@ func NewPeer(version uint, p *p2p.Peer, rw p2p.MsgReadWriter, txpool TxPool) *Pe
go peer.broadcastBlocks()
go peer.broadcastTransactions()
go peer.announceTransactions()
go peer.dispatcher()
return peer
}
@ -323,94 +331,148 @@ func (p *Peer) ReplyReceiptsRLP(id uint64, receipts []rlp.RawValue) error {
// RequestOneHeader is a wrapper around the header query functions to fetch a
// single header. It is used solely by the fetcher.
func (p *Peer) RequestOneHeader(hash common.Hash) error {
func (p *Peer) RequestOneHeader(hash common.Hash, sink chan *Response) (*Request, error) {
p.Log().Debug("Fetching single header", "hash", hash)
id := rand.Uint64()
requestTracker.Track(p.id, p.version, GetBlockHeadersMsg, BlockHeadersMsg, id)
return p2p.Send(p.rw, GetBlockHeadersMsg, &GetBlockHeadersPacket66{
RequestId: id,
GetBlockHeadersPacket: &GetBlockHeadersPacket{
Origin: HashOrNumber{Hash: hash},
Amount: uint64(1),
Skip: uint64(0),
Reverse: false,
req := &Request{
id: id,
sink: sink,
code: GetBlockHeadersMsg,
want: BlockHeadersMsg,
data: &GetBlockHeadersPacket66{
RequestId: id,
GetBlockHeadersPacket: &GetBlockHeadersPacket{
Origin: HashOrNumber{Hash: hash},
Amount: uint64(1),
Skip: uint64(0),
Reverse: false,
},
},
})
}
if err := p.dispatchRequest(req); err != nil {
return nil, err
}
return req, nil
}
// RequestHeadersByHash fetches a batch of blocks' headers corresponding to the
// specified header query, based on the hash of an origin block.
func (p *Peer) RequestHeadersByHash(origin common.Hash, amount int, skip int, reverse bool) error {
func (p *Peer) RequestHeadersByHash(origin common.Hash, amount int, skip int, reverse bool, sink chan *Response) (*Request, error) {
p.Log().Debug("Fetching batch of headers", "count", amount, "fromhash", origin, "skip", skip, "reverse", reverse)
id := rand.Uint64()
requestTracker.Track(p.id, p.version, GetBlockHeadersMsg, BlockHeadersMsg, id)
return p2p.Send(p.rw, GetBlockHeadersMsg, &GetBlockHeadersPacket66{
RequestId: id,
GetBlockHeadersPacket: &GetBlockHeadersPacket{
Origin: HashOrNumber{Hash: origin},
Amount: uint64(amount),
Skip: uint64(skip),
Reverse: reverse,
req := &Request{
id: id,
sink: sink,
code: GetBlockHeadersMsg,
want: BlockHeadersMsg,
data: &GetBlockHeadersPacket66{
RequestId: id,
GetBlockHeadersPacket: &GetBlockHeadersPacket{
Origin: HashOrNumber{Hash: origin},
Amount: uint64(amount),
Skip: uint64(skip),
Reverse: reverse,
},
},
})
}
if err := p.dispatchRequest(req); err != nil {
return nil, err
}
return req, nil
}
// RequestHeadersByNumber fetches a batch of blocks' headers corresponding to the
// specified header query, based on the number of an origin block.
func (p *Peer) RequestHeadersByNumber(origin uint64, amount int, skip int, reverse bool) error {
func (p *Peer) RequestHeadersByNumber(origin uint64, amount int, skip int, reverse bool, sink chan *Response) (*Request, error) {
p.Log().Debug("Fetching batch of headers", "count", amount, "fromnum", origin, "skip", skip, "reverse", reverse)
id := rand.Uint64()
requestTracker.Track(p.id, p.version, GetBlockHeadersMsg, BlockHeadersMsg, id)
return p2p.Send(p.rw, GetBlockHeadersMsg, &GetBlockHeadersPacket66{
RequestId: id,
GetBlockHeadersPacket: &GetBlockHeadersPacket{
Origin: HashOrNumber{Number: origin},
Amount: uint64(amount),
Skip: uint64(skip),
Reverse: reverse,
req := &Request{
id: id,
sink: sink,
code: GetBlockHeadersMsg,
want: BlockHeadersMsg,
data: &GetBlockHeadersPacket66{
RequestId: id,
GetBlockHeadersPacket: &GetBlockHeadersPacket{
Origin: HashOrNumber{Number: origin},
Amount: uint64(amount),
Skip: uint64(skip),
Reverse: reverse,
},
},
})
}
if err := p.dispatchRequest(req); err != nil {
return nil, err
}
return req, nil
}
// RequestBodies fetches a batch of blocks' bodies corresponding to the hashes
// specified.
func (p *Peer) RequestBodies(hashes []common.Hash) error {
func (p *Peer) RequestBodies(hashes []common.Hash, sink chan *Response) (*Request, error) {
p.Log().Debug("Fetching batch of block bodies", "count", len(hashes))
id := rand.Uint64()
requestTracker.Track(p.id, p.version, GetBlockBodiesMsg, BlockBodiesMsg, id)
return p2p.Send(p.rw, GetBlockBodiesMsg, &GetBlockBodiesPacket66{
RequestId: id,
GetBlockBodiesPacket: hashes,
})
req := &Request{
id: id,
sink: sink,
code: GetBlockBodiesMsg,
want: BlockBodiesMsg,
data: &GetBlockBodiesPacket66{
RequestId: id,
GetBlockBodiesPacket: hashes,
},
}
if err := p.dispatchRequest(req); err != nil {
return nil, err
}
return req, nil
}
// RequestNodeData fetches a batch of arbitrary data from a node's known state
// data, corresponding to the specified hashes.
func (p *Peer) RequestNodeData(hashes []common.Hash) error {
func (p *Peer) RequestNodeData(hashes []common.Hash, sink chan *Response) (*Request, error) {
p.Log().Debug("Fetching batch of state data", "count", len(hashes))
id := rand.Uint64()
requestTracker.Track(p.id, p.version, GetNodeDataMsg, NodeDataMsg, id)
return p2p.Send(p.rw, GetNodeDataMsg, &GetNodeDataPacket66{
RequestId: id,
GetNodeDataPacket: hashes,
})
req := &Request{
id: id,
sink: sink,
code: GetNodeDataMsg,
want: NodeDataMsg,
data: &GetNodeDataPacket66{
RequestId: id,
GetNodeDataPacket: hashes,
},
}
if err := p.dispatchRequest(req); err != nil {
return nil, err
}
return req, nil
}
// RequestReceipts fetches a batch of transaction receipts from a remote node.
func (p *Peer) RequestReceipts(hashes []common.Hash) error {
func (p *Peer) RequestReceipts(hashes []common.Hash, sink chan *Response) (*Request, error) {
p.Log().Debug("Fetching batch of receipts", "count", len(hashes))
id := rand.Uint64()
requestTracker.Track(p.id, p.version, GetReceiptsMsg, ReceiptsMsg, id)
return p2p.Send(p.rw, GetReceiptsMsg, &GetReceiptsPacket66{
RequestId: id,
GetReceiptsPacket: hashes,
})
req := &Request{
id: id,
sink: sink,
code: GetReceiptsMsg,
want: ReceiptsMsg,
data: &GetReceiptsPacket66{
RequestId: id,
GetReceiptsPacket: hashes,
},
}
if err := p.dispatchRequest(req); err != nil {
return nil, err
}
return req, nil
}
// RequestTxs fetches a batch of transactions from a remote node.

View File

@ -99,8 +99,8 @@ func MakeProtocols(backend Backend, dnsdisc enode.Iterator) []p2p.Protocol {
Version: version,
Length: protocolLengths[version],
Run: func(p *p2p.Peer, rw p2p.MsgReadWriter) error {
return backend.RunPeer(newPeer(version, p, rw), func(peer *Peer) error {
return handle(backend, peer)
return backend.RunPeer(NewPeer(version, p, rw), func(peer *Peer) error {
return Handle(backend, peer)
})
},
NodeInfo: func() interface{} {
@ -116,9 +116,9 @@ func MakeProtocols(backend Backend, dnsdisc enode.Iterator) []p2p.Protocol {
return protocols
}
// handle is the callback invoked to manage the life cycle of a `snap` peer.
// Handle is the callback invoked to manage the life cycle of a `snap` peer.
// When this function terminates, the peer is disconnected.
func handle(backend Backend, peer *Peer) error {
func Handle(backend Backend, peer *Peer) error {
for {
if err := handleMessage(backend, peer); err != nil {
peer.Log().Debug("Message handling failed in `snap`", "err", err)
@ -161,60 +161,10 @@ func handleMessage(backend Backend, peer *Peer) error {
if err := msg.Decode(&req); err != nil {
return fmt.Errorf("%w: message %v: %v", errDecode, msg, err)
}
if req.Bytes > softResponseLimit {
req.Bytes = softResponseLimit
}
// Retrieve the requested state and bail out if non existent
tr, err := trie.New(req.Root, backend.Chain().StateCache().TrieDB())
if err != nil {
return p2p.Send(peer.rw, AccountRangeMsg, &AccountRangePacket{ID: req.ID})
}
it, err := backend.Chain().Snapshots().AccountIterator(req.Root, req.Origin)
if err != nil {
return p2p.Send(peer.rw, AccountRangeMsg, &AccountRangePacket{ID: req.ID})
}
// Iterate over the requested range and pile accounts up
var (
accounts []*AccountData
size uint64
last common.Hash
)
for it.Next() && size < req.Bytes {
hash, account := it.Hash(), common.CopyBytes(it.Account())
// Service the request, potentially returning nothing in case of errors
accounts, proofs := ServiceGetAccountRangeQuery(backend.Chain(), &req)
// Track the returned interval for the Merkle proofs
last = hash
// Assemble the reply item
size += uint64(common.HashLength + len(account))
accounts = append(accounts, &AccountData{
Hash: hash,
Body: account,
})
// If we've exceeded the request threshold, abort
if bytes.Compare(hash[:], req.Limit[:]) >= 0 {
break
}
}
it.Release()
// Generate the Merkle proofs for the first and last account
proof := light.NewNodeSet()
if err := tr.Prove(req.Origin[:], 0, proof); err != nil {
log.Warn("Failed to prove account range", "origin", req.Origin, "err", err)
return p2p.Send(peer.rw, AccountRangeMsg, &AccountRangePacket{ID: req.ID})
}
if last != (common.Hash{}) {
if err := tr.Prove(last[:], 0, proof); err != nil {
log.Warn("Failed to prove account range", "last", last, "err", err)
return p2p.Send(peer.rw, AccountRangeMsg, &AccountRangePacket{ID: req.ID})
}
}
var proofs [][]byte
for _, blob := range proof.NodeList() {
proofs = append(proofs, blob)
}
// Send back anything accumulated
// Send back anything accumulated (or empty in case of errors)
return p2p.Send(peer.rw, AccountRangeMsg, &AccountRangePacket{
ID: req.ID,
Accounts: accounts,
@ -243,111 +193,10 @@ func handleMessage(backend Backend, peer *Peer) error {
if err := msg.Decode(&req); err != nil {
return fmt.Errorf("%w: message %v: %v", errDecode, msg, err)
}
if req.Bytes > softResponseLimit {
req.Bytes = softResponseLimit
}
// TODO(karalabe): Do we want to enforce > 0 accounts and 1 account if origin is set?
// TODO(karalabe): - Logging locally is not ideal as remote faulst annoy the local user
// TODO(karalabe): - Dropping the remote peer is less flexible wrt client bugs (slow is better than non-functional)
// Service the request, potentially returning nothing in case of errors
slots, proofs := ServiceGetStorageRangesQuery(backend.Chain(), &req)
// Calculate the hard limit at which to abort, even if mid storage trie
hardLimit := uint64(float64(req.Bytes) * (1 + stateLookupSlack))
// Retrieve storage ranges until the packet limit is reached
var (
slots [][]*StorageData
proofs [][]byte
size uint64
)
for _, account := range req.Accounts {
// If we've exceeded the requested data limit, abort without opening
// a new storage range (that we'd need to prove due to exceeded size)
if size >= req.Bytes {
break
}
// The first account might start from a different origin and end sooner
var origin common.Hash
if len(req.Origin) > 0 {
origin, req.Origin = common.BytesToHash(req.Origin), nil
}
var limit = common.HexToHash("0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff")
if len(req.Limit) > 0 {
limit, req.Limit = common.BytesToHash(req.Limit), nil
}
// Retrieve the requested state and bail out if non existent
it, err := backend.Chain().Snapshots().StorageIterator(req.Root, account, origin)
if err != nil {
return p2p.Send(peer.rw, StorageRangesMsg, &StorageRangesPacket{ID: req.ID})
}
// Iterate over the requested range and pile slots up
var (
storage []*StorageData
last common.Hash
abort bool
)
for it.Next() {
if size >= hardLimit {
abort = true
break
}
hash, slot := it.Hash(), common.CopyBytes(it.Slot())
// Track the returned interval for the Merkle proofs
last = hash
// Assemble the reply item
size += uint64(common.HashLength + len(slot))
storage = append(storage, &StorageData{
Hash: hash,
Body: slot,
})
// If we've exceeded the request threshold, abort
if bytes.Compare(hash[:], limit[:]) >= 0 {
break
}
}
slots = append(slots, storage)
it.Release()
// Generate the Merkle proofs for the first and last storage slot, but
// only if the response was capped. If the entire storage trie included
// in the response, no need for any proofs.
if origin != (common.Hash{}) || abort {
// Request started at a non-zero hash or was capped prematurely, add
// the endpoint Merkle proofs
accTrie, err := trie.New(req.Root, backend.Chain().StateCache().TrieDB())
if err != nil {
return p2p.Send(peer.rw, StorageRangesMsg, &StorageRangesPacket{ID: req.ID})
}
var acc types.StateAccount
if err := rlp.DecodeBytes(accTrie.Get(account[:]), &acc); err != nil {
return p2p.Send(peer.rw, StorageRangesMsg, &StorageRangesPacket{ID: req.ID})
}
stTrie, err := trie.New(acc.Root, backend.Chain().StateCache().TrieDB())
if err != nil {
return p2p.Send(peer.rw, StorageRangesMsg, &StorageRangesPacket{ID: req.ID})
}
proof := light.NewNodeSet()
if err := stTrie.Prove(origin[:], 0, proof); err != nil {
log.Warn("Failed to prove storage range", "origin", req.Origin, "err", err)
return p2p.Send(peer.rw, StorageRangesMsg, &StorageRangesPacket{ID: req.ID})
}
if last != (common.Hash{}) {
if err := stTrie.Prove(last[:], 0, proof); err != nil {
log.Warn("Failed to prove storage range", "last", last, "err", err)
return p2p.Send(peer.rw, StorageRangesMsg, &StorageRangesPacket{ID: req.ID})
}
}
for _, blob := range proof.NodeList() {
proofs = append(proofs, blob)
}
// Proof terminates the reply as proofs are only added if a node
// refuses to serve more data (exception when a contract fetch is
// finishing, but that's that).
break
}
}
// Send back anything accumulated
// Send back anything accumulated (or empty in case of errors)
return p2p.Send(peer.rw, StorageRangesMsg, &StorageRangesPacket{
ID: req.ID,
Slots: slots,
@ -378,31 +227,10 @@ func handleMessage(backend Backend, peer *Peer) error {
if err := msg.Decode(&req); err != nil {
return fmt.Errorf("%w: message %v: %v", errDecode, msg, err)
}
if req.Bytes > softResponseLimit {
req.Bytes = softResponseLimit
}
if len(req.Hashes) > maxCodeLookups {
req.Hashes = req.Hashes[:maxCodeLookups]
}
// Retrieve bytecodes until the packet size limit is reached
var (
codes [][]byte
bytes uint64
)
for _, hash := range req.Hashes {
if hash == emptyCode {
// Peers should not request the empty code, but if they do, at
// least sent them back a correct response without db lookups
codes = append(codes, []byte{})
} else if blob, err := backend.Chain().ContractCode(hash); err == nil {
codes = append(codes, blob)
bytes += uint64(len(blob))
}
if bytes > req.Bytes {
break
}
}
// Send back anything accumulated
// Service the request, potentially returning nothing in case of errors
codes := ServiceGetByteCodesQuery(backend.Chain(), &req)
// Send back anything accumulated (or empty in case of errors)
return p2p.Send(peer.rw, ByteCodesMsg, &ByteCodesPacket{
ID: req.ID,
Codes: codes,
@ -424,80 +252,12 @@ func handleMessage(backend Backend, peer *Peer) error {
if err := msg.Decode(&req); err != nil {
return fmt.Errorf("%w: message %v: %v", errDecode, msg, err)
}
if req.Bytes > softResponseLimit {
req.Bytes = softResponseLimit
}
// Make sure we have the state associated with the request
triedb := backend.Chain().StateCache().TrieDB()
accTrie, err := trie.NewSecure(req.Root, triedb)
// Service the request, potentially returning nothing in case of errors
nodes, err := ServiceGetTrieNodesQuery(backend.Chain(), &req, start)
if err != nil {
// We don't have the requested state available, bail out
return p2p.Send(peer.rw, TrieNodesMsg, &TrieNodesPacket{ID: req.ID})
return err
}
snap := backend.Chain().Snapshots().Snapshot(req.Root)
if snap == nil {
// We don't have the requested state snapshotted yet, bail out.
// In reality we could still serve using the account and storage
// tries only, but let's protect the node a bit while it's doing
// snapshot generation.
return p2p.Send(peer.rw, TrieNodesMsg, &TrieNodesPacket{ID: req.ID})
}
// Retrieve trie nodes until the packet size limit is reached
var (
nodes [][]byte
bytes uint64
loads int // Trie hash expansions to cound database reads
)
for _, pathset := range req.Paths {
switch len(pathset) {
case 0:
// Ensure we penalize invalid requests
return fmt.Errorf("%w: zero-item pathset requested", errBadRequest)
case 1:
// If we're only retrieving an account trie node, fetch it directly
blob, resolved, err := accTrie.TryGetNode(pathset[0])
loads += resolved // always account database reads, even for failures
if err != nil {
break
}
nodes = append(nodes, blob)
bytes += uint64(len(blob))
default:
// Storage slots requested, open the storage trie and retrieve from there
account, err := snap.Account(common.BytesToHash(pathset[0]))
loads++ // always account database reads, even for failures
if err != nil || account == nil {
break
}
stTrie, err := trie.NewSecure(common.BytesToHash(account.Root), triedb)
loads++ // always account database reads, even for failures
if err != nil {
break
}
for _, path := range pathset[1:] {
blob, resolved, err := stTrie.TryGetNode(path)
loads += resolved // always account database reads, even for failures
if err != nil {
break
}
nodes = append(nodes, blob)
bytes += uint64(len(blob))
// Sanity check limits to avoid DoS on the store trie loads
if bytes > req.Bytes || loads > maxTrieNodeLookups || time.Since(start) > maxTrieNodeTimeSpent {
break
}
}
}
// Abort request processing if we've exceeded our limits
if bytes > req.Bytes || loads > maxTrieNodeLookups || time.Since(start) > maxTrieNodeTimeSpent {
break
}
}
// Send back anything accumulated
// Send back anything accumulated (or empty in case of errors)
return p2p.Send(peer.rw, TrieNodesMsg, &TrieNodesPacket{
ID: req.ID,
Nodes: nodes,
@ -518,6 +278,282 @@ func handleMessage(backend Backend, peer *Peer) error {
}
}
// ServiceGetAccountRangeQuery assembles the response to an account range query.
// It is exposed to allow external packages to test protocol behavior.
func ServiceGetAccountRangeQuery(chain *core.BlockChain, req *GetAccountRangePacket) ([]*AccountData, [][]byte) {
if req.Bytes > softResponseLimit {
req.Bytes = softResponseLimit
}
// Retrieve the requested state and bail out if non existent
tr, err := trie.New(req.Root, chain.StateCache().TrieDB())
if err != nil {
return nil, nil
}
it, err := chain.Snapshots().AccountIterator(req.Root, req.Origin)
if err != nil {
return nil, nil
}
// Iterate over the requested range and pile accounts up
var (
accounts []*AccountData
size uint64
last common.Hash
)
for it.Next() && size < req.Bytes {
hash, account := it.Hash(), common.CopyBytes(it.Account())
// Track the returned interval for the Merkle proofs
last = hash
// Assemble the reply item
size += uint64(common.HashLength + len(account))
accounts = append(accounts, &AccountData{
Hash: hash,
Body: account,
})
// If we've exceeded the request threshold, abort
if bytes.Compare(hash[:], req.Limit[:]) >= 0 {
break
}
}
it.Release()
// Generate the Merkle proofs for the first and last account
proof := light.NewNodeSet()
if err := tr.Prove(req.Origin[:], 0, proof); err != nil {
log.Warn("Failed to prove account range", "origin", req.Origin, "err", err)
return nil, nil
}
if last != (common.Hash{}) {
if err := tr.Prove(last[:], 0, proof); err != nil {
log.Warn("Failed to prove account range", "last", last, "err", err)
return nil, nil
}
}
var proofs [][]byte
for _, blob := range proof.NodeList() {
proofs = append(proofs, blob)
}
return accounts, proofs
}
func ServiceGetStorageRangesQuery(chain *core.BlockChain, req *GetStorageRangesPacket) ([][]*StorageData, [][]byte) {
if req.Bytes > softResponseLimit {
req.Bytes = softResponseLimit
}
// TODO(karalabe): Do we want to enforce > 0 accounts and 1 account if origin is set?
// TODO(karalabe): - Logging locally is not ideal as remote faulst annoy the local user
// TODO(karalabe): - Dropping the remote peer is less flexible wrt client bugs (slow is better than non-functional)
// Calculate the hard limit at which to abort, even if mid storage trie
hardLimit := uint64(float64(req.Bytes) * (1 + stateLookupSlack))
// Retrieve storage ranges until the packet limit is reached
var (
slots [][]*StorageData
proofs [][]byte
size uint64
)
for _, account := range req.Accounts {
// If we've exceeded the requested data limit, abort without opening
// a new storage range (that we'd need to prove due to exceeded size)
if size >= req.Bytes {
break
}
// The first account might start from a different origin and end sooner
var origin common.Hash
if len(req.Origin) > 0 {
origin, req.Origin = common.BytesToHash(req.Origin), nil
}
var limit = common.HexToHash("0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff")
if len(req.Limit) > 0 {
limit, req.Limit = common.BytesToHash(req.Limit), nil
}
// Retrieve the requested state and bail out if non existent
it, err := chain.Snapshots().StorageIterator(req.Root, account, origin)
if err != nil {
return nil, nil
}
// Iterate over the requested range and pile slots up
var (
storage []*StorageData
last common.Hash
abort bool
)
for it.Next() {
if size >= hardLimit {
abort = true
break
}
hash, slot := it.Hash(), common.CopyBytes(it.Slot())
// Track the returned interval for the Merkle proofs
last = hash
// Assemble the reply item
size += uint64(common.HashLength + len(slot))
storage = append(storage, &StorageData{
Hash: hash,
Body: slot,
})
// If we've exceeded the request threshold, abort
if bytes.Compare(hash[:], limit[:]) >= 0 {
break
}
}
slots = append(slots, storage)
it.Release()
// Generate the Merkle proofs for the first and last storage slot, but
// only if the response was capped. If the entire storage trie included
// in the response, no need for any proofs.
if origin != (common.Hash{}) || abort {
// Request started at a non-zero hash or was capped prematurely, add
// the endpoint Merkle proofs
accTrie, err := trie.New(req.Root, chain.StateCache().TrieDB())
if err != nil {
return nil, nil
}
var acc types.StateAccount
if err := rlp.DecodeBytes(accTrie.Get(account[:]), &acc); err != nil {
return nil, nil
}
stTrie, err := trie.New(acc.Root, chain.StateCache().TrieDB())
if err != nil {
return nil, nil
}
proof := light.NewNodeSet()
if err := stTrie.Prove(origin[:], 0, proof); err != nil {
log.Warn("Failed to prove storage range", "origin", req.Origin, "err", err)
return nil, nil
}
if last != (common.Hash{}) {
if err := stTrie.Prove(last[:], 0, proof); err != nil {
log.Warn("Failed to prove storage range", "last", last, "err", err)
return nil, nil
}
}
for _, blob := range proof.NodeList() {
proofs = append(proofs, blob)
}
// Proof terminates the reply as proofs are only added if a node
// refuses to serve more data (exception when a contract fetch is
// finishing, but that's that).
break
}
}
return slots, proofs
}
// ServiceGetByteCodesQuery assembles the response to a byte codes query.
// It is exposed to allow external packages to test protocol behavior.
func ServiceGetByteCodesQuery(chain *core.BlockChain, req *GetByteCodesPacket) [][]byte {
if req.Bytes > softResponseLimit {
req.Bytes = softResponseLimit
}
if len(req.Hashes) > maxCodeLookups {
req.Hashes = req.Hashes[:maxCodeLookups]
}
// Retrieve bytecodes until the packet size limit is reached
var (
codes [][]byte
bytes uint64
)
for _, hash := range req.Hashes {
if hash == emptyCode {
// Peers should not request the empty code, but if they do, at
// least sent them back a correct response without db lookups
codes = append(codes, []byte{})
} else if blob, err := chain.ContractCode(hash); err == nil {
codes = append(codes, blob)
bytes += uint64(len(blob))
}
if bytes > req.Bytes {
break
}
}
return codes
}
// ServiceGetTrieNodesQuery assembles the response to a trie nodes query.
// It is exposed to allow external packages to test protocol behavior.
func ServiceGetTrieNodesQuery(chain *core.BlockChain, req *GetTrieNodesPacket, start time.Time) ([][]byte, error) {
if req.Bytes > softResponseLimit {
req.Bytes = softResponseLimit
}
// Make sure we have the state associated with the request
triedb := chain.StateCache().TrieDB()
accTrie, err := trie.NewSecure(req.Root, triedb)
if err != nil {
// We don't have the requested state available, bail out
return nil, nil
}
snap := chain.Snapshots().Snapshot(req.Root)
if snap == nil {
// We don't have the requested state snapshotted yet, bail out.
// In reality we could still serve using the account and storage
// tries only, but let's protect the node a bit while it's doing
// snapshot generation.
return nil, nil
}
// Retrieve trie nodes until the packet size limit is reached
var (
nodes [][]byte
bytes uint64
loads int // Trie hash expansions to cound database reads
)
for _, pathset := range req.Paths {
switch len(pathset) {
case 0:
// Ensure we penalize invalid requests
return nil, fmt.Errorf("%w: zero-item pathset requested", errBadRequest)
case 1:
// If we're only retrieving an account trie node, fetch it directly
blob, resolved, err := accTrie.TryGetNode(pathset[0])
loads += resolved // always account database reads, even for failures
if err != nil {
break
}
nodes = append(nodes, blob)
bytes += uint64(len(blob))
default:
// Storage slots requested, open the storage trie and retrieve from there
account, err := snap.Account(common.BytesToHash(pathset[0]))
loads++ // always account database reads, even for failures
if err != nil || account == nil {
break
}
stTrie, err := trie.NewSecure(common.BytesToHash(account.Root), triedb)
loads++ // always account database reads, even for failures
if err != nil {
break
}
for _, path := range pathset[1:] {
blob, resolved, err := stTrie.TryGetNode(path)
loads += resolved // always account database reads, even for failures
if err != nil {
break
}
nodes = append(nodes, blob)
bytes += uint64(len(blob))
// Sanity check limits to avoid DoS on the store trie loads
if bytes > req.Bytes || loads > maxTrieNodeLookups || time.Since(start) > maxTrieNodeTimeSpent {
break
}
}
}
// Abort request processing if we've exceeded our limits
if bytes > req.Bytes || loads > maxTrieNodeLookups || time.Since(start) > maxTrieNodeTimeSpent {
break
}
}
return nodes, nil
}
// NodeInfo represents a short summary of the `snap` sub-protocol metadata
// known about the host peer.
type NodeInfo struct{}

View File

@ -33,9 +33,9 @@ type Peer struct {
logger log.Logger // Contextual logger with the peer id injected
}
// newPeer create a wrapper for a network connection and negotiated protocol
// NewPeer create a wrapper for a network connection and negotiated protocol
// version.
func newPeer(version uint, p *p2p.Peer, rw p2p.MsgReadWriter) *Peer {
func NewPeer(version uint, p *p2p.Peer, rw p2p.MsgReadWriter) *Peer {
id := p.ID().String()
return &Peer{
id: id,

View File

@ -27,7 +27,7 @@ import (
// Constants to match up protocol versions and messages
const (
snap1 = 1
SNAP1 = 1
)
// ProtocolName is the official short name of the `snap` protocol used during
@ -36,11 +36,11 @@ const ProtocolName = "snap"
// ProtocolVersions are the supported versions of the `snap` protocol (first
// is primary).
var ProtocolVersions = []uint{snap1}
var ProtocolVersions = []uint{SNAP1}
// protocolLengths are the number of implemented message corresponding to
// different protocol versions.
var protocolLengths = map[uint]uint64{snap1: 8}
var protocolLengths = map[uint]uint64{SNAP1: 8}
// maxMessageSize is the maximum cap on the size of a protocol message.
const maxMessageSize = 10 * 1024 * 1024

View File

@ -325,10 +325,10 @@ type healTask struct {
codeTasks map[common.Hash]struct{} // Set of byte code tasks currently queued for retrieval
}
// syncProgress is a database entry to allow suspending and resuming a snapshot state
// SyncProgress is a database entry to allow suspending and resuming a snapshot state
// sync. Opposed to full and fast sync, there is no way to restart a suspended
// snap sync without prior knowledge of the suspension point.
type syncProgress struct {
type SyncProgress struct {
Tasks []*accountTask // The suspended account tasks (contract tasks within)
// Status report during syncing phase
@ -342,12 +342,15 @@ type syncProgress struct {
// Status report during healing phase
TrienodeHealSynced uint64 // Number of state trie nodes downloaded
TrienodeHealBytes common.StorageSize // Number of state trie bytes persisted to disk
TrienodeHealDups uint64 // Number of state trie nodes already processed
TrienodeHealNops uint64 // Number of state trie nodes not requested
BytecodeHealSynced uint64 // Number of bytecodes downloaded
BytecodeHealBytes common.StorageSize // Number of bytecodes persisted to disk
BytecodeHealDups uint64 // Number of bytecodes already processed
BytecodeHealNops uint64 // Number of bytecodes not requested
}
// SyncPending is analogous to SyncProgress, but it's used to report on pending
// ephemeral sync progress that doesn't get persisted into the database.
type SyncPending struct {
TrienodeHeal uint64 // Number of state trie nodes pending
BytecodeHeal uint64 // Number of bytecodes pending
}
// SyncPeer abstracts out the methods required for a peer to be synced against
@ -671,7 +674,7 @@ func (s *Syncer) Sync(root common.Hash, cancel chan struct{}) error {
// loadSyncStatus retrieves a previously aborted sync status from the database,
// or generates a fresh one if none is available.
func (s *Syncer) loadSyncStatus() {
var progress syncProgress
var progress SyncProgress
if status := rawdb.ReadSnapshotSyncStatus(s.db); status != nil {
if err := json.Unmarshal(status, &progress); err != nil {
@ -775,7 +778,7 @@ func (s *Syncer) saveSyncStatus() {
}
}
// Store the actual progress markers
progress := &syncProgress{
progress := &SyncProgress{
Tasks: s.tasks,
AccountSynced: s.accountSynced,
AccountBytes: s.accountBytes,
@ -795,6 +798,31 @@ func (s *Syncer) saveSyncStatus() {
rawdb.WriteSnapshotSyncStatus(s.db, status)
}
// Progress returns the snap sync status statistics.
func (s *Syncer) Progress() (*SyncProgress, *SyncPending) {
s.lock.Lock()
defer s.lock.Unlock()
progress := &SyncProgress{
AccountSynced: s.accountSynced,
AccountBytes: s.accountBytes,
BytecodeSynced: s.bytecodeSynced,
BytecodeBytes: s.bytecodeBytes,
StorageSynced: s.storageSynced,
StorageBytes: s.storageBytes,
TrienodeHealSynced: s.trienodeHealSynced,
TrienodeHealBytes: s.trienodeHealBytes,
BytecodeHealSynced: s.bytecodeHealSynced,
BytecodeHealBytes: s.bytecodeHealBytes,
}
pending := new(SyncPending)
if s.healer != nil {
pending.TrienodeHeal = uint64(len(s.healer.trieTasks))
pending.BytecodeHeal = uint64(len(s.healer.codeTasks))
}
return progress, pending
}
// cleanAccountTasks removes account range retrieval tasks that have already been
// completed.
func (s *Syncer) cleanAccountTasks() {

View File

@ -165,10 +165,7 @@ func (cs *chainSyncer) nextSyncOp() *chainSyncOp {
return nil
}
mode, ourTD := cs.modeAndLocalHead()
if mode == downloader.FastSync && atomic.LoadUint32(&cs.handler.snapSync) == 1 {
// Fast sync via the snap protocol
mode = downloader.SnapSync
}
op := peerToSyncOp(mode, peer)
if op.td.Cmp(ourTD) <= 0 {
return nil // We're in sync.
@ -182,19 +179,19 @@ func peerToSyncOp(mode downloader.SyncMode, p *eth.Peer) *chainSyncOp {
}
func (cs *chainSyncer) modeAndLocalHead() (downloader.SyncMode, *big.Int) {
// If we're in fast sync mode, return that directly
if atomic.LoadUint32(&cs.handler.fastSync) == 1 {
// If we're in snap sync mode, return that directly
if atomic.LoadUint32(&cs.handler.snapSync) == 1 {
block := cs.handler.chain.CurrentFastBlock()
td := cs.handler.chain.GetTd(block.Hash(), block.NumberU64())
return downloader.FastSync, td
return downloader.SnapSync, td
}
// We are probably in full sync, but we might have rewound to before the
// fast sync pivot, check if we should reenable
// snap sync pivot, check if we should reenable
if pivot := rawdb.ReadLastPivotNumber(cs.handler.database); pivot != nil {
if head := cs.handler.chain.CurrentBlock(); head.NumberU64() < *pivot {
block := cs.handler.chain.CurrentFastBlock()
td := cs.handler.chain.GetTd(block.Hash(), block.NumberU64())
return downloader.FastSync, td
return downloader.SnapSync, td
}
}
// Nope, we're really full syncing
@ -211,15 +208,15 @@ func (cs *chainSyncer) startSync(op *chainSyncOp) {
// doSync synchronizes the local blockchain with a remote peer.
func (h *handler) doSync(op *chainSyncOp) error {
if op.mode == downloader.FastSync || op.mode == downloader.SnapSync {
// Before launch the fast sync, we have to ensure user uses the same
if op.mode == downloader.SnapSync {
// Before launch the snap sync, we have to ensure user uses the same
// txlookup limit.
// The main concern here is: during the fast sync Geth won't index the
// The main concern here is: during the snap sync Geth won't index the
// block(generate tx indices) before the HEAD-limit. But if user changes
// the limit in the next fast sync(e.g. user kill Geth manually and
// the limit in the next snap sync(e.g. user kill Geth manually and
// restart) then it will be hard for Geth to figure out the oldest block
// has been indexed. So here for the user-experience wise, it's non-optimal
// that user can't change limit during the fast sync. If changed, Geth
// that user can't change limit during the snap sync. If changed, Geth
// will just blindly use the original one.
limit := h.chain.TxLookupLimit()
if stored := rawdb.ReadFastTxLookupLimit(h.database); stored == nil {
@ -229,15 +226,11 @@ func (h *handler) doSync(op *chainSyncOp) error {
log.Warn("Update txLookup limit", "provided", limit, "updated", *stored)
}
}
// Run the sync cycle, and disable fast sync if we're past the pivot block
// Run the sync cycle, and disable snap sync if we're past the pivot block
err := h.downloader.Synchronise(op.peer.ID(), op.head, op.td, op.mode)
if err != nil {
return err
}
if atomic.LoadUint32(&h.fastSync) == 1 {
log.Info("Fast sync complete, auto disabling")
atomic.StoreUint32(&h.fastSync, 0)
}
if atomic.LoadUint32(&h.snapSync) == 1 {
log.Info("Snap sync complete, auto disabling")
atomic.StoreUint32(&h.snapSync, 0)

View File

@ -23,57 +23,74 @@ import (
"github.com/ethereum/go-ethereum/eth/downloader"
"github.com/ethereum/go-ethereum/eth/protocols/eth"
"github.com/ethereum/go-ethereum/eth/protocols/snap"
"github.com/ethereum/go-ethereum/p2p"
"github.com/ethereum/go-ethereum/p2p/enode"
)
// Tests that fast sync is disabled after a successful sync cycle.
func TestFastSyncDisabling66(t *testing.T) { testFastSyncDisabling(t, eth.ETH66) }
// Tests that snap sync is disabled after a successful sync cycle.
func TestSnapSyncDisabling66(t *testing.T) { testSnapSyncDisabling(t, eth.ETH66, snap.SNAP1) }
// Tests that fast sync gets disabled as soon as a real block is successfully
// Tests that snap sync gets disabled as soon as a real block is successfully
// imported into the blockchain.
func testFastSyncDisabling(t *testing.T, protocol uint) {
func testSnapSyncDisabling(t *testing.T, ethVer uint, snapVer uint) {
t.Parallel()
// Create an empty handler and ensure it's in fast sync mode
// Create an empty handler and ensure it's in snap sync mode
empty := newTestHandler()
if atomic.LoadUint32(&empty.handler.fastSync) == 0 {
t.Fatalf("fast sync disabled on pristine blockchain")
if atomic.LoadUint32(&empty.handler.snapSync) == 0 {
t.Fatalf("snap sync disabled on pristine blockchain")
}
defer empty.close()
// Create a full handler and ensure fast sync ends up disabled
// Create a full handler and ensure snap sync ends up disabled
full := newTestHandlerWithBlocks(1024)
if atomic.LoadUint32(&full.handler.fastSync) == 1 {
t.Fatalf("fast sync not disabled on non-empty blockchain")
if atomic.LoadUint32(&full.handler.snapSync) == 1 {
t.Fatalf("snap sync not disabled on non-empty blockchain")
}
defer full.close()
// Sync up the two handlers
emptyPipe, fullPipe := p2p.MsgPipe()
defer emptyPipe.Close()
defer fullPipe.Close()
// Sync up the two handlers via both `eth` and `snap`
caps := []p2p.Cap{{Name: "eth", Version: ethVer}, {Name: "snap", Version: snapVer}}
emptyPeer := eth.NewPeer(protocol, p2p.NewPeer(enode.ID{1}, "", nil), emptyPipe, empty.txpool)
fullPeer := eth.NewPeer(protocol, p2p.NewPeer(enode.ID{2}, "", nil), fullPipe, full.txpool)
defer emptyPeer.Close()
defer fullPeer.Close()
emptyPipeEth, fullPipeEth := p2p.MsgPipe()
defer emptyPipeEth.Close()
defer fullPipeEth.Close()
go empty.handler.runEthPeer(emptyPeer, func(peer *eth.Peer) error {
emptyPeerEth := eth.NewPeer(ethVer, p2p.NewPeer(enode.ID{1}, "", caps), emptyPipeEth, empty.txpool)
fullPeerEth := eth.NewPeer(ethVer, p2p.NewPeer(enode.ID{2}, "", caps), fullPipeEth, full.txpool)
defer emptyPeerEth.Close()
defer fullPeerEth.Close()
go empty.handler.runEthPeer(emptyPeerEth, func(peer *eth.Peer) error {
return eth.Handle((*ethHandler)(empty.handler), peer)
})
go full.handler.runEthPeer(fullPeer, func(peer *eth.Peer) error {
go full.handler.runEthPeer(fullPeerEth, func(peer *eth.Peer) error {
return eth.Handle((*ethHandler)(full.handler), peer)
})
emptyPipeSnap, fullPipeSnap := p2p.MsgPipe()
defer emptyPipeSnap.Close()
defer fullPipeSnap.Close()
emptyPeerSnap := snap.NewPeer(snapVer, p2p.NewPeer(enode.ID{1}, "", caps), emptyPipeSnap)
fullPeerSnap := snap.NewPeer(snapVer, p2p.NewPeer(enode.ID{2}, "", caps), fullPipeSnap)
go empty.handler.runSnapExtension(emptyPeerSnap, func(peer *snap.Peer) error {
return snap.Handle((*snapHandler)(empty.handler), peer)
})
go full.handler.runSnapExtension(fullPeerSnap, func(peer *snap.Peer) error {
return snap.Handle((*snapHandler)(full.handler), peer)
})
// Wait a bit for the above handlers to start
time.Sleep(250 * time.Millisecond)
// Check that fast sync was disabled
op := peerToSyncOp(downloader.FastSync, empty.handler.peers.peerWithHighestTD())
// Check that snap sync was disabled
op := peerToSyncOp(downloader.SnapSync, empty.handler.peers.peerWithHighestTD())
if err := empty.handler.doSync(op); err != nil {
t.Fatal("sync failed:", err)
}
if atomic.LoadUint32(&empty.handler.fastSync) == 1 {
t.Fatalf("fast sync not disabled after successful synchronisation")
if atomic.LoadUint32(&empty.handler.snapSync) == 1 {
t.Fatalf("snap sync not disabled after successful synchronisation")
}
}