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:
gary rong
2020-07-28 23:02:35 +08:00
committed by GitHub
parent 93da0cf8a1
commit 28c5a8a54b
22 changed files with 1366 additions and 1062 deletions

View File

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