les: implement new les fetcher (#20692)
* cmd, consensus, eth, les: implement light fetcher * les: address comment * les: address comment * les: address comments * les: check td after delivery * les: add linearExpiredValue for error counter * les: fix import * les: fix dead lock * les: order announces by td * les: encapsulate invalid counter * les: address comment * les: add more checks during the delivery * les: fix log * eth, les: fix lint * eth/fetcher: address comment
This commit is contained in:
@ -14,7 +14,7 @@
|
||||
// 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 fetcher contains the announcement based blocks or transaction synchronisation.
|
||||
// Package fetcher contains the announcement based header, blocks or transaction synchronisation.
|
||||
package fetcher
|
||||
|
||||
import (
|
||||
@ -31,6 +31,7 @@ import (
|
||||
)
|
||||
|
||||
const (
|
||||
lightTimeout = time.Millisecond // Time allowance before an announced header is explicitly requested
|
||||
arriveTimeout = 500 * time.Millisecond // Time allowance before an announced block/transaction is explicitly requested
|
||||
gatherSlack = 100 * time.Millisecond // Interval used to collate almost-expired announces with fetches
|
||||
fetchTimeout = 5 * time.Second // Maximum allotted time to return an explicitly requested block/transaction
|
||||
@ -39,7 +40,7 @@ const (
|
||||
const (
|
||||
maxUncleDist = 7 // Maximum allowed backward distance from the chain head
|
||||
maxQueueDist = 32 // Maximum allowed distance from the chain head to queue
|
||||
hashLimit = 256 // Maximum number of unique blocks a peer may have announced
|
||||
hashLimit = 256 // Maximum number of unique blocks or headers a peer may have announced
|
||||
blockLimit = 64 // Maximum number of unique blocks a peer may have delivered
|
||||
)
|
||||
|
||||
@ -63,9 +64,10 @@ var (
|
||||
bodyFilterOutMeter = metrics.NewRegisteredMeter("eth/fetcher/block/filter/bodies/out", nil)
|
||||
)
|
||||
|
||||
var (
|
||||
errTerminated = errors.New("terminated")
|
||||
)
|
||||
var errTerminated = errors.New("terminated")
|
||||
|
||||
// HeaderRetrievalFn is a callback type for retrieving a header from the local chain.
|
||||
type HeaderRetrievalFn func(common.Hash) *types.Header
|
||||
|
||||
// blockRetrievalFn is a callback type for retrieving a block from the local chain.
|
||||
type blockRetrievalFn func(common.Hash) *types.Block
|
||||
@ -85,6 +87,9 @@ type blockBroadcasterFn func(block *types.Block, propagate bool)
|
||||
// chainHeightFn is a callback type to retrieve the current chain height.
|
||||
type chainHeightFn func() uint64
|
||||
|
||||
// headersInsertFn is a callback type to insert a batch of headers into the local chain.
|
||||
type headersInsertFn func(headers []*types.Header) (int, error)
|
||||
|
||||
// chainInsertFn is a callback type to insert a batch of blocks into the local chain.
|
||||
type chainInsertFn func(types.Blocks) (int, error)
|
||||
|
||||
@ -121,18 +126,38 @@ type bodyFilterTask struct {
|
||||
time time.Time // Arrival time of the blocks' contents
|
||||
}
|
||||
|
||||
// blockInject represents a schedules import operation.
|
||||
type blockInject struct {
|
||||
// blockOrHeaderInject represents a schedules import operation.
|
||||
type blockOrHeaderInject struct {
|
||||
origin string
|
||||
block *types.Block
|
||||
|
||||
header *types.Header // Used for light mode fetcher which only cares about header.
|
||||
block *types.Block // Used for normal mode fetcher which imports full block.
|
||||
}
|
||||
|
||||
// number returns the block number of the injected object.
|
||||
func (inject *blockOrHeaderInject) number() uint64 {
|
||||
if inject.header != nil {
|
||||
return inject.header.Number.Uint64()
|
||||
}
|
||||
return inject.block.NumberU64()
|
||||
}
|
||||
|
||||
// number returns the block hash of the injected object.
|
||||
func (inject *blockOrHeaderInject) hash() common.Hash {
|
||||
if inject.header != nil {
|
||||
return inject.header.Hash()
|
||||
}
|
||||
return inject.block.Hash()
|
||||
}
|
||||
|
||||
// BlockFetcher is responsible for accumulating block announcements from various peers
|
||||
// and scheduling them for retrieval.
|
||||
type BlockFetcher struct {
|
||||
light bool // The indicator whether it's a light fetcher or normal one.
|
||||
|
||||
// Various event channels
|
||||
notify chan *blockAnnounce
|
||||
inject chan *blockInject
|
||||
inject chan *blockOrHeaderInject
|
||||
|
||||
headerFilter chan chan *headerFilterTask
|
||||
bodyFilter chan chan *bodyFilterTask
|
||||
@ -148,31 +173,34 @@ type BlockFetcher struct {
|
||||
completing map[common.Hash]*blockAnnounce // Blocks with headers, currently body-completing
|
||||
|
||||
// Block cache
|
||||
queue *prque.Prque // Queue containing the import operations (block number sorted)
|
||||
queues map[string]int // Per peer block counts to prevent memory exhaustion
|
||||
queued map[common.Hash]*blockInject // Set of already queued blocks (to dedupe imports)
|
||||
queue *prque.Prque // Queue containing the import operations (block number sorted)
|
||||
queues map[string]int // Per peer block counts to prevent memory exhaustion
|
||||
queued map[common.Hash]*blockOrHeaderInject // Set of already queued blocks (to dedup imports)
|
||||
|
||||
// Callbacks
|
||||
getHeader HeaderRetrievalFn // Retrieves a header from the local chain
|
||||
getBlock blockRetrievalFn // Retrieves a block from the local chain
|
||||
verifyHeader headerVerifierFn // Checks if a block's headers have a valid proof of work
|
||||
broadcastBlock blockBroadcasterFn // Broadcasts a block to connected peers
|
||||
chainHeight chainHeightFn // Retrieves the current chain's height
|
||||
insertHeaders headersInsertFn // Injects a batch of headers into the chain
|
||||
insertChain chainInsertFn // Injects a batch of blocks into the chain
|
||||
dropPeer peerDropFn // Drops a peer for misbehaving
|
||||
|
||||
// Testing hooks
|
||||
announceChangeHook func(common.Hash, bool) // Method to call upon adding or deleting a hash from the blockAnnounce list
|
||||
queueChangeHook func(common.Hash, bool) // Method to call upon adding or deleting a block from the import queue
|
||||
fetchingHook func([]common.Hash) // Method to call upon starting a block (eth/61) or header (eth/62) fetch
|
||||
completingHook func([]common.Hash) // Method to call upon starting a block body fetch (eth/62)
|
||||
importedHook func(*types.Block) // Method to call upon successful block import (both eth/61 and eth/62)
|
||||
announceChangeHook func(common.Hash, bool) // Method to call upon adding or deleting a hash from the blockAnnounce list
|
||||
queueChangeHook func(common.Hash, bool) // Method to call upon adding or deleting a block from the import queue
|
||||
fetchingHook func([]common.Hash) // Method to call upon starting a block (eth/61) or header (eth/62) fetch
|
||||
completingHook func([]common.Hash) // Method to call upon starting a block body fetch (eth/62)
|
||||
importedHook func(*types.Header, *types.Block) // Method to call upon successful header or block import (both eth/61 and eth/62)
|
||||
}
|
||||
|
||||
// NewBlockFetcher creates a block fetcher to retrieve blocks based on hash announcements.
|
||||
func NewBlockFetcher(getBlock blockRetrievalFn, verifyHeader headerVerifierFn, broadcastBlock blockBroadcasterFn, chainHeight chainHeightFn, insertChain chainInsertFn, dropPeer peerDropFn) *BlockFetcher {
|
||||
func NewBlockFetcher(light bool, getHeader HeaderRetrievalFn, getBlock blockRetrievalFn, verifyHeader headerVerifierFn, broadcastBlock blockBroadcasterFn, chainHeight chainHeightFn, insertHeaders headersInsertFn, insertChain chainInsertFn, dropPeer peerDropFn) *BlockFetcher {
|
||||
return &BlockFetcher{
|
||||
light: light,
|
||||
notify: make(chan *blockAnnounce),
|
||||
inject: make(chan *blockInject),
|
||||
inject: make(chan *blockOrHeaderInject),
|
||||
headerFilter: make(chan chan *headerFilterTask),
|
||||
bodyFilter: make(chan chan *bodyFilterTask),
|
||||
done: make(chan common.Hash),
|
||||
@ -184,11 +212,13 @@ func NewBlockFetcher(getBlock blockRetrievalFn, verifyHeader headerVerifierFn, b
|
||||
completing: make(map[common.Hash]*blockAnnounce),
|
||||
queue: prque.New(nil),
|
||||
queues: make(map[string]int),
|
||||
queued: make(map[common.Hash]*blockInject),
|
||||
queued: make(map[common.Hash]*blockOrHeaderInject),
|
||||
getHeader: getHeader,
|
||||
getBlock: getBlock,
|
||||
verifyHeader: verifyHeader,
|
||||
broadcastBlock: broadcastBlock,
|
||||
chainHeight: chainHeight,
|
||||
insertHeaders: insertHeaders,
|
||||
insertChain: insertChain,
|
||||
dropPeer: dropPeer,
|
||||
}
|
||||
@ -228,7 +258,7 @@ func (f *BlockFetcher) Notify(peer string, hash common.Hash, number uint64, time
|
||||
|
||||
// Enqueue tries to fill gaps the fetcher's future import queue.
|
||||
func (f *BlockFetcher) Enqueue(peer string, block *types.Block) error {
|
||||
op := &blockInject{
|
||||
op := &blockOrHeaderInject{
|
||||
origin: peer,
|
||||
block: block,
|
||||
}
|
||||
@ -315,13 +345,13 @@ func (f *BlockFetcher) loop() {
|
||||
// Import any queued blocks that could potentially fit
|
||||
height := f.chainHeight()
|
||||
for !f.queue.Empty() {
|
||||
op := f.queue.PopItem().(*blockInject)
|
||||
hash := op.block.Hash()
|
||||
op := f.queue.PopItem().(*blockOrHeaderInject)
|
||||
hash := op.hash()
|
||||
if f.queueChangeHook != nil {
|
||||
f.queueChangeHook(hash, false)
|
||||
}
|
||||
// If too high up the chain or phase, continue later
|
||||
number := op.block.NumberU64()
|
||||
number := op.number()
|
||||
if number > height+1 {
|
||||
f.queue.Push(op, -int64(number))
|
||||
if f.queueChangeHook != nil {
|
||||
@ -330,11 +360,15 @@ func (f *BlockFetcher) loop() {
|
||||
break
|
||||
}
|
||||
// Otherwise if fresh and still unknown, try and import
|
||||
if number+maxUncleDist < height || f.getBlock(hash) != nil {
|
||||
if (number+maxUncleDist < height) || (f.light && f.getHeader(hash) != nil) || (!f.light && f.getBlock(hash) != nil) {
|
||||
f.forgetBlock(hash)
|
||||
continue
|
||||
}
|
||||
f.insert(op.origin, op.block)
|
||||
if f.light {
|
||||
f.importHeaders(op.origin, op.header)
|
||||
} else {
|
||||
f.importBlocks(op.origin, op.block)
|
||||
}
|
||||
}
|
||||
// Wait for an outside event to occur
|
||||
select {
|
||||
@ -379,7 +413,13 @@ func (f *BlockFetcher) loop() {
|
||||
case op := <-f.inject:
|
||||
// A direct block insertion was requested, try and fill any pending gaps
|
||||
blockBroadcastInMeter.Mark(1)
|
||||
f.enqueue(op.origin, op.block)
|
||||
|
||||
// Now only direct block injection is allowed, drop the header injection
|
||||
// here silently if we receive.
|
||||
if f.light {
|
||||
continue
|
||||
}
|
||||
f.enqueue(op.origin, nil, op.block)
|
||||
|
||||
case hash := <-f.done:
|
||||
// A pending import finished, remove all traces of the notification
|
||||
@ -391,13 +431,19 @@ func (f *BlockFetcher) loop() {
|
||||
request := make(map[string][]common.Hash)
|
||||
|
||||
for hash, announces := range f.announced {
|
||||
if time.Since(announces[0].time) > arriveTimeout-gatherSlack {
|
||||
// In current LES protocol(les2/les3), only header announce is
|
||||
// available, no need to wait too much time for header broadcast.
|
||||
timeout := arriveTimeout - gatherSlack
|
||||
if f.light {
|
||||
timeout = 0
|
||||
}
|
||||
if time.Since(announces[0].time) > timeout {
|
||||
// Pick a random peer to retrieve from, reset all others
|
||||
announce := announces[rand.Intn(len(announces))]
|
||||
f.forgetHash(hash)
|
||||
|
||||
// If the block still didn't arrive, queue for fetching
|
||||
if f.getBlock(hash) == nil {
|
||||
if (f.light && f.getHeader(hash) == nil) || (!f.light && f.getBlock(hash) == nil) {
|
||||
request[announce.origin] = append(request[announce.origin], hash)
|
||||
f.fetching[hash] = announce
|
||||
}
|
||||
@ -465,7 +511,7 @@ func (f *BlockFetcher) loop() {
|
||||
|
||||
// Split the batch of headers into unknown ones (to return to the caller),
|
||||
// known incomplete ones (requiring body retrievals) and completed blocks.
|
||||
unknown, incomplete, complete := []*types.Header{}, []*blockAnnounce{}, []*types.Block{}
|
||||
unknown, incomplete, complete, lightHeaders := []*types.Header{}, []*blockAnnounce{}, []*types.Block{}, []*blockAnnounce{}
|
||||
for _, header := range task.headers {
|
||||
hash := header.Hash()
|
||||
|
||||
@ -478,6 +524,16 @@ func (f *BlockFetcher) loop() {
|
||||
f.forgetHash(hash)
|
||||
continue
|
||||
}
|
||||
// Collect all headers only if we are running in light
|
||||
// mode and the headers are not imported by other means.
|
||||
if f.light {
|
||||
if f.getHeader(hash) == nil {
|
||||
announce.header = header
|
||||
lightHeaders = append(lightHeaders, announce)
|
||||
}
|
||||
f.forgetHash(hash)
|
||||
continue
|
||||
}
|
||||
// Only keep if not imported by other means
|
||||
if f.getBlock(hash) == nil {
|
||||
announce.header = header
|
||||
@ -522,10 +578,14 @@ func (f *BlockFetcher) loop() {
|
||||
f.rescheduleComplete(completeTimer)
|
||||
}
|
||||
}
|
||||
// Schedule the header for light fetcher import
|
||||
for _, announce := range lightHeaders {
|
||||
f.enqueue(announce.origin, announce.header, nil)
|
||||
}
|
||||
// Schedule the header-only blocks for import
|
||||
for _, block := range complete {
|
||||
if announce := f.completing[block.Hash()]; announce != nil {
|
||||
f.enqueue(announce.origin, block)
|
||||
f.enqueue(announce.origin, nil, block)
|
||||
}
|
||||
}
|
||||
|
||||
@ -592,7 +652,7 @@ func (f *BlockFetcher) loop() {
|
||||
// Schedule the retrieved blocks for ordered import
|
||||
for _, block := range blocks {
|
||||
if announce := f.completing[block.Hash()]; announce != nil {
|
||||
f.enqueue(announce.origin, block)
|
||||
f.enqueue(announce.origin, nil, block)
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -605,6 +665,12 @@ func (f *BlockFetcher) rescheduleFetch(fetch *time.Timer) {
|
||||
if len(f.announced) == 0 {
|
||||
return
|
||||
}
|
||||
// Schedule announcement retrieval quickly for light mode
|
||||
// since server won't send any headers to client.
|
||||
if f.light {
|
||||
fetch.Reset(lightTimeout)
|
||||
return
|
||||
}
|
||||
// Otherwise find the earliest expiring announcement
|
||||
earliest := time.Now()
|
||||
for _, announces := range f.announced {
|
||||
@ -631,46 +697,88 @@ func (f *BlockFetcher) rescheduleComplete(complete *time.Timer) {
|
||||
complete.Reset(gatherSlack - time.Since(earliest))
|
||||
}
|
||||
|
||||
// enqueue schedules a new future import operation, if the block to be imported
|
||||
// has not yet been seen.
|
||||
func (f *BlockFetcher) enqueue(peer string, block *types.Block) {
|
||||
hash := block.Hash()
|
||||
|
||||
// enqueue schedules a new header or block import operation, if the component
|
||||
// to be imported has not yet been seen.
|
||||
func (f *BlockFetcher) enqueue(peer string, header *types.Header, block *types.Block) {
|
||||
var (
|
||||
hash common.Hash
|
||||
number uint64
|
||||
)
|
||||
if header != nil {
|
||||
hash, number = header.Hash(), header.Number.Uint64()
|
||||
} else {
|
||||
hash, number = block.Hash(), block.NumberU64()
|
||||
}
|
||||
// Ensure the peer isn't DOSing us
|
||||
count := f.queues[peer] + 1
|
||||
if count > blockLimit {
|
||||
log.Debug("Discarded propagated block, exceeded allowance", "peer", peer, "number", block.Number(), "hash", hash, "limit", blockLimit)
|
||||
log.Debug("Discarded delivered header or block, exceeded allowance", "peer", peer, "number", number, "hash", hash, "limit", blockLimit)
|
||||
blockBroadcastDOSMeter.Mark(1)
|
||||
f.forgetHash(hash)
|
||||
return
|
||||
}
|
||||
// Discard any past or too distant blocks
|
||||
if dist := int64(block.NumberU64()) - int64(f.chainHeight()); dist < -maxUncleDist || dist > maxQueueDist {
|
||||
log.Debug("Discarded propagated block, too far away", "peer", peer, "number", block.Number(), "hash", hash, "distance", dist)
|
||||
if dist := int64(number) - int64(f.chainHeight()); dist < -maxUncleDist || dist > maxQueueDist {
|
||||
log.Debug("Discarded delivered header or block, too far away", "peer", peer, "number", number, "hash", hash, "distance", dist)
|
||||
blockBroadcastDropMeter.Mark(1)
|
||||
f.forgetHash(hash)
|
||||
return
|
||||
}
|
||||
// Schedule the block for future importing
|
||||
if _, ok := f.queued[hash]; !ok {
|
||||
op := &blockInject{
|
||||
origin: peer,
|
||||
block: block,
|
||||
op := &blockOrHeaderInject{origin: peer}
|
||||
if header != nil {
|
||||
op.header = header
|
||||
} else {
|
||||
op.block = block
|
||||
}
|
||||
f.queues[peer] = count
|
||||
f.queued[hash] = op
|
||||
f.queue.Push(op, -int64(block.NumberU64()))
|
||||
f.queue.Push(op, -int64(number))
|
||||
if f.queueChangeHook != nil {
|
||||
f.queueChangeHook(op.block.Hash(), true)
|
||||
f.queueChangeHook(hash, true)
|
||||
}
|
||||
log.Debug("Queued propagated block", "peer", peer, "number", block.Number(), "hash", hash, "queued", f.queue.Size())
|
||||
log.Debug("Queued delivered header or block", "peer", peer, "number", number, "hash", hash, "queued", f.queue.Size())
|
||||
}
|
||||
}
|
||||
|
||||
// insert spawns a new goroutine to run a block insertion into the chain. If the
|
||||
// importHeaders spawns a new goroutine to run a header insertion into the chain.
|
||||
// If the header's number is at the same height as the current import phase, it
|
||||
// updates the phase states accordingly.
|
||||
func (f *BlockFetcher) importHeaders(peer string, header *types.Header) {
|
||||
hash := header.Hash()
|
||||
log.Debug("Importing propagated header", "peer", peer, "number", header.Number, "hash", hash)
|
||||
|
||||
go func() {
|
||||
defer func() { f.done <- hash }()
|
||||
// If the parent's unknown, abort insertion
|
||||
parent := f.getHeader(header.ParentHash)
|
||||
if parent == nil {
|
||||
log.Debug("Unknown parent of propagated header", "peer", peer, "number", header.Number, "hash", hash, "parent", header.ParentHash)
|
||||
return
|
||||
}
|
||||
// Validate the header and if something went wrong, drop the peer
|
||||
if err := f.verifyHeader(header); err != nil && err != consensus.ErrFutureBlock {
|
||||
log.Debug("Propagated header verification failed", "peer", peer, "number", header.Number, "hash", hash, "err", err)
|
||||
f.dropPeer(peer)
|
||||
return
|
||||
}
|
||||
// Run the actual import and log any issues
|
||||
if _, err := f.insertHeaders([]*types.Header{header}); err != nil {
|
||||
log.Debug("Propagated header import failed", "peer", peer, "number", header.Number, "hash", hash, "err", err)
|
||||
return
|
||||
}
|
||||
// Invoke the testing hook if needed
|
||||
if f.importedHook != nil {
|
||||
f.importedHook(header, nil)
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
// importBlocks spawns a new goroutine to run a block insertion into the chain. If the
|
||||
// block's number is at the same height as the current import phase, it updates
|
||||
// the phase states accordingly.
|
||||
func (f *BlockFetcher) insert(peer string, block *types.Block) {
|
||||
func (f *BlockFetcher) importBlocks(peer string, block *types.Block) {
|
||||
hash := block.Hash()
|
||||
|
||||
// Run the import on a new thread
|
||||
@ -711,7 +819,7 @@ func (f *BlockFetcher) insert(peer string, block *types.Block) {
|
||||
|
||||
// Invoke the testing hook if needed
|
||||
if f.importedHook != nil {
|
||||
f.importedHook(block)
|
||||
f.importedHook(nil, block)
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
Reference in New Issue
Block a user