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

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() {