Quarantine and reassembly of gossiped blobs and blocks (#4808)

This commit is contained in:
henridf 2023-04-13 21:11:40 +02:00 committed by GitHub
parent fa3655527c
commit 021de18e06
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 267 additions and 42 deletions

View File

@ -0,0 +1,70 @@
# beacon_chain
# Copyright (c) 2018-2023 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.push raises: [].}
import
std/tables,
../spec/datatypes/deneb
const
MaxBlobs = SLOTS_PER_EPOCH * MAX_BLOBS_PER_BLOCK
type
BlobQuarantine* = object
blobs*: Table[(Eth2Digest, BlobIndex), ref BlobSidecar]
func put*(quarantine: var BlobQuarantine, blobSidecar: ref BlobSidecar) =
if quarantine.blobs.lenu64 > MaxBlobs:
return
discard quarantine.blobs.hasKeyOrPut((blobSidecar.block_root,
blobSidecar.index), blobSidecar)
func blobIndices*(quarantine: BlobQuarantine, digest: Eth2Digest):
seq[BlobIndex] =
var r: seq[BlobIndex] = @[]
for i in 0..MAX_BLOBS_PER_BLOCK-1:
if quarantine.blobs.hasKey((digest, i)):
r.add(i)
r
func hasBlob*(quarantine: BlobQuarantine, blobSidecar: BlobSidecar) : bool =
quarantine.blobs.hasKey((blobSidecar.block_root, blobSidecar.index))
func popBlobs*(quarantine: var BlobQuarantine, digest: Eth2Digest):
seq[ref BlobSidecar] =
var r: seq[ref BlobSidecar] = @[]
for i in 0..MAX_BLOBS_PER_BLOCK-1:
var b: ref BlobSidecar
if quarantine.blobs.pop((digest, i), b):
r.add(b)
r
func peekBlobs*(quarantine: var BlobQuarantine, digest: Eth2Digest):
seq[ref BlobSidecar] =
var r: seq[ref BlobSidecar] = @[]
for i in 0..MAX_BLOBS_PER_BLOCK-1:
quarantine.blobs.withValue((digest, i), value):
r.add(value[])
r
func removeBlobs*(quarantine: var BlobQuarantine, digest: Eth2Digest) =
for i in 0..MAX_BLOBS_PER_BLOCK-1:
quarantine.blobs.del((digest, i))
func hasBlobs*(quarantine: BlobQuarantine, blck: deneb.SignedBeaconBlock):
bool =
let idxs = quarantine.blobIndices(blck.root)
if len(blck.message.body.blob_kzg_commitments) != len(idxs):
return false
for i in 0..len(idxs):
if idxs[i] != uint64(i):
return false
true

View File

@ -19,6 +19,8 @@ const
## Arbitrary ## Arbitrary
MaxOrphans = SLOTS_PER_EPOCH * 3 MaxOrphans = SLOTS_PER_EPOCH * 3
## Enough for finalization in an alternative fork ## Enough for finalization in an alternative fork
MaxBlobless = SLOTS_PER_EPOCH
## Arbitrary
MaxUnviables = 16 * 1024 MaxUnviables = 16 * 1024
## About a day of blocks - most likely not needed but it's quite cheap.. ## About a day of blocks - most likely not needed but it's quite cheap..
@ -39,10 +41,19 @@ type
## Trivially invalid blocks may be dropped before reaching this stage. ## Trivially invalid blocks may be dropped before reaching this stage.
orphans*: Table[(Eth2Digest, ValidatorSig), ForkedSignedBeaconBlock] orphans*: Table[(Eth2Digest, ValidatorSig), ForkedSignedBeaconBlock]
## Blocks that we don't have a parent for - when we resolve the parent, we ## Blocks that we don't have a parent for - when we resolve the
## can proceed to resolving the block as well - we index this by root and ## parent, we can proceed to resolving the block as well - we
## signature such that a block with invalid signature won't cause a block ## index this by root and signature such that a block with
## with a valid signature to be dropped ## invalid signature won't cause a block with a valid signature
## to be dropped. An orphan block may also be "blobless" (see
## below) - if so, upon resolving the parent, it should be
## stored in the blobless table.
blobless*: Table[(Eth2Digest, ValidatorSig), deneb.SignedBeaconBlock]
## Blocks that we don't have blobs for. When we have received
## all blobs for this block, we can proceed to resolving the
## block as well. A blobless block inserted into this table must
## have a resolved parent (i.e., it is not an orphan).
unviable*: OrderedTable[Eth2Digest, tuple[]] unviable*: OrderedTable[Eth2Digest, tuple[]]
## Unviable blocks are those that come from a history that does not ## Unviable blocks are those that come from a history that does not
@ -115,12 +126,14 @@ func removeOrphan*(
quarantine: var Quarantine, signedBlock: ForkySignedBeaconBlock) = quarantine: var Quarantine, signedBlock: ForkySignedBeaconBlock) =
quarantine.orphans.del((signedBlock.root, signedBlock.signature)) quarantine.orphans.del((signedBlock.root, signedBlock.signature))
func isViableOrphan( func removeBlobless*(
finalizedSlot: Slot, signedBlock: ForkedSignedBeaconBlock): bool = quarantine: var Quarantine, signedBlock: ForkySignedBeaconBlock) =
quarantine.blobless.del((signedBlock.root, signedBlock.signature))
func isViable(
finalizedSlot: Slot, slot: Slot): bool =
# The orphan must be newer than the finalization point so that its parent # The orphan must be newer than the finalization point so that its parent
# either is the finalized block or more recent # either is the finalized block or more recent
let
slot = getForkedBlockField(signedBlock, slot)
slot > finalizedSlot slot > finalizedSlot
func cleanupUnviable(quarantine: var Quarantine) = func cleanupUnviable(quarantine: var Quarantine) =
@ -131,47 +144,76 @@ func cleanupUnviable(quarantine: var Quarantine) =
break # Cannot modify while for-looping break # Cannot modify while for-looping
quarantine.unviable.del(toDel) quarantine.unviable.del(toDel)
func addUnviable*(quarantine: var Quarantine, root: Eth2Digest) = func removeUnviableTree[T](quarantine: var Quarantine,
if root in quarantine.unviable: toCheck: var seq[Eth2Digest],
return tbl: var Table[(Eth2Digest, ValidatorSig),
T]): seq[Eth2Digest] =
quarantine.cleanupUnviable()
# Remove the tree of orphans whose ancestor is unviable - they are now also # Remove the tree of orphans whose ancestor is unviable - they are now also
# unviable! This helps avoiding junk in the quarantine, because we don't keep # unviable! This helps avoiding junk in the quarantine, because we don't keep
# unviable parents in the DAG and there's no way to tell an orphan from an # unviable parents in the DAG and there's no way to tell an orphan from an
# unviable block without the parent. # unviable block without the parent.
var var
toRemove: seq[(Eth2Digest, ValidatorSig)] # Can't modify while iterating toRemove: seq[(Eth2Digest, ValidatorSig)] # Can't modify while iterating
toCheck = @[root] checked: seq[Eth2Digest]
while toCheck.len > 0: while toCheck.len > 0:
let root = toCheck.pop() let root = toCheck.pop()
for k, v in quarantine.orphans.mpairs(): if root notin checked:
if getForkedBlockField(v, parent_root) == root: checked.add(root)
for k, v in tbl.mpairs():
when T is ForkedSignedBeaconBlock:
let blockRoot = getForkedBlockField(v, parent_root)
elif T is ForkySignedBeaconBlock:
let blockRoot = v.message.parent_root
else:
static: doAssert false
if blockRoot == root:
toCheck.add(k[0]) toCheck.add(k[0])
toRemove.add(k) toRemove.add(k)
elif k[0] == root: elif k[0] == root:
toRemove.add(k) toRemove.add(k)
for k in toRemove: for k in toRemove:
quarantine.orphans.del k tbl.del k
quarantine.unviable[k[0]] = () quarantine.unviable[k[0]] = ()
toRemove.setLen(0) toRemove.setLen(0)
checked
func addUnviable*(quarantine: var Quarantine, root: Eth2Digest) =
if root in quarantine.unviable:
return
quarantine.cleanupUnviable()
var toCheck = @[root]
var checked = quarantine.removeUnviableTree(toCheck, quarantine.orphans)
discard quarantine.removeUnviableTree(checked, quarantine.blobless)
quarantine.unviable[root] = () quarantine.unviable[root] = ()
func cleanupOrphans(quarantine: var Quarantine, finalizedSlot: Slot) = func cleanupOrphans(quarantine: var Quarantine, finalizedSlot: Slot) =
var toDel: seq[(Eth2Digest, ValidatorSig)] var toDel: seq[(Eth2Digest, ValidatorSig)]
for k, v in quarantine.orphans: for k, v in quarantine.orphans:
if not isViableOrphan(finalizedSlot, v): if not isViable(finalizedSlot, getForkedBlockField(v, slot)):
toDel.add k toDel.add k
for k in toDel: for k in toDel:
quarantine.addUnviable k[0] quarantine.addUnviable k[0]
quarantine.orphans.del k quarantine.orphans.del k
func cleanupBlobless(quarantine: var Quarantine, finalizedSlot: Slot) =
var toDel: seq[(Eth2Digest, ValidatorSig)]
for k, v in quarantine.blobless:
if not isViable(finalizedSlot, v.message.slot):
toDel.add k
for k in toDel:
quarantine.addUnviable k[0]
quarantine.blobless.del k
func clearAfterReorg*(quarantine: var Quarantine) = func clearAfterReorg*(quarantine: var Quarantine) =
## Clear missing and orphans to start with a fresh slate in case of a reorg ## Clear missing and orphans to start with a fresh slate in case of a reorg
## Unviables remain unviable and are not cleared. ## Unviables remain unviable and are not cleared.
@ -193,7 +235,7 @@ func addOrphan*(
quarantine: var Quarantine, finalizedSlot: Slot, quarantine: var Quarantine, finalizedSlot: Slot,
signedBlock: ForkedSignedBeaconBlock): bool = signedBlock: ForkedSignedBeaconBlock): bool =
## Adds block to quarantine's `orphans` and `missing` lists. ## Adds block to quarantine's `orphans` and `missing` lists.
if not isViableOrphan(finalizedSlot, signedBlock): if not isViable(finalizedSlot, getForkedBlockField(signedBlock, slot)):
quarantine.addUnviable(signedBlock.root) quarantine.addUnviable(signedBlock.root)
return false return false
@ -230,3 +272,26 @@ iterator pop*(quarantine: var Quarantine, root: Eth2Digest):
if getForkedBlockField(v, parent_root) == root: if getForkedBlockField(v, parent_root) == root:
toRemove.add(k) toRemove.add(k)
yield v yield v
proc addBlobless*(
quarantine: var Quarantine, finalizedSlot: Slot,
signedBlock: deneb.SignedBeaconBlock): bool =
if not isViable(finalizedSlot, signedBlock.message.slot):
quarantine.addUnviable(signedBlock.root)
return false
quarantine.cleanupBlobless(finalizedSlot)
if quarantine.blobless.lenu64 >= MaxBlobless:
return false
quarantine.blobless[(signedBlock.root, signedBlock.signature)] = signedBlock
quarantine.missing.del(signedBlock.root)
true
iterator peekBlobless*(quarantine: var Quarantine, root: Eth2Digest):
deneb.SignedBeaconBlock =
for k, v in quarantine.blobless.mpairs():
if k[0] == root:
yield v

View File

@ -22,7 +22,9 @@ from ../consensus_object_pools/block_dag import BlockRef, root, shortLog, slot
from ../consensus_object_pools/block_pools_types import from ../consensus_object_pools/block_pools_types import
EpochRef, VerifierError EpochRef, VerifierError
from ../consensus_object_pools/block_quarantine import from ../consensus_object_pools/block_quarantine import
addOrphan, addUnviable, pop, removeOrphan addBlobless, addOrphan, addUnviable, pop, removeOrphan
from ../consensus_object_pools/blob_quarantine import
BlobQuarantine, hasBlobs, popBlobs
from ../validators/validator_monitor import from ../validators/validator_monitor import
MsgSource, ValidatorMonitor, registerAttestationInBlock, registerBeaconBlock, MsgSource, ValidatorMonitor, registerAttestationInBlock, registerBeaconBlock,
registerSyncAggregateInBlock registerSyncAggregateInBlock
@ -91,6 +93,7 @@ type
validatorMonitor: ref ValidatorMonitor validatorMonitor: ref ValidatorMonitor
getBeaconTime: GetBeaconTimeFn getBeaconTime: GetBeaconTimeFn
blobQuarantine: ref BlobQuarantine
verifier: BatchVerifier verifier: BatchVerifier
lastPayload: Slot lastPayload: Slot
@ -123,6 +126,7 @@ proc new*(T: type BlockProcessor,
rng: ref HmacDrbgContext, taskpool: TaskPoolPtr, rng: ref HmacDrbgContext, taskpool: TaskPoolPtr,
consensusManager: ref ConsensusManager, consensusManager: ref ConsensusManager,
validatorMonitor: ref ValidatorMonitor, validatorMonitor: ref ValidatorMonitor,
blobQuarantine: ref BlobQuarantine,
getBeaconTime: GetBeaconTimeFn): ref BlockProcessor = getBeaconTime: GetBeaconTimeFn): ref BlockProcessor =
(ref BlockProcessor)( (ref BlockProcessor)(
dumpEnabled: dumpEnabled, dumpEnabled: dumpEnabled,
@ -131,6 +135,7 @@ proc new*(T: type BlockProcessor,
blockQueue: newAsyncQueue[BlockEntry](), blockQueue: newAsyncQueue[BlockEntry](),
consensusManager: consensusManager, consensusManager: consensusManager,
validatorMonitor: validatorMonitor, validatorMonitor: validatorMonitor,
blobQuarantine: blobQuarantine,
getBeaconTime: getBeaconTime, getBeaconTime: getBeaconTime,
verifier: BatchVerifier(rng: rng, taskpool: taskpool) verifier: BatchVerifier(rng: rng, taskpool: taskpool)
) )
@ -564,7 +569,22 @@ proc storeBlock*(
for quarantined in self.consensusManager.quarantine[].pop(blck.get().root): for quarantined in self.consensusManager.quarantine[].pop(blck.get().root):
# Process the blocks that had the newly accepted block as parent # Process the blocks that had the newly accepted block as parent
self[].addBlock(MsgSource.gossip, quarantined, BlobSidecars @[]) withBlck(quarantined):
when typeof(blck).toFork() < ConsensusFork.Deneb:
self[].addBlock(MsgSource.gossip, quarantined, BlobSidecars @[])
else:
if len(blck.message.body.blob_kzg_commitments) == 0:
self[].addBlock(MsgSource.gossip, quarantined, BlobSidecars @[])
else:
if self.blobQuarantine[].hasBlobs(blck):
let blobs = self.blobQuarantine[].popBlobs(blck.root)
self[].addBlock(MsgSource.gossip, quarantined, blobs)
else:
if not self.consensusManager.quarantine[].addBlobless(
dag.finalizedHead.slot, blck):
notice "Block quarantine full (blobless)",
blockRoot = shortLog(quarantined.root),
signature = shortLog(quarantined.signature)
return Result[BlockRef, (VerifierError, ProcessingStatus)].ok blck.get return Result[BlockRef, (VerifierError, ProcessingStatus)].ok blck.get

View File

@ -14,8 +14,8 @@ import
../spec/[helpers, forks], ../spec/[helpers, forks],
../spec/datatypes/[altair, phase0, deneb], ../spec/datatypes/[altair, phase0, deneb],
../consensus_object_pools/[ ../consensus_object_pools/[
block_clearance, block_quarantine, blockchain_dag, exit_pool, attestation_pool, blob_quarantine, block_clearance, block_quarantine, blockchain_dag,
light_client_pool, sync_committee_msg_pool], exit_pool, attestation_pool, light_client_pool, sync_committee_msg_pool],
../validators/validator_pool, ../validators/validator_pool,
../beacon_clock, ../beacon_clock,
"."/[gossip_validation, block_processor, batch_validation], "."/[gossip_validation, block_processor, batch_validation],
@ -145,6 +145,8 @@ type
# ---------------------------------------------------------------- # ----------------------------------------------------------------
quarantine*: ref Quarantine quarantine*: ref Quarantine
blobQuarantine*: ref BlobQuarantine
# Application-provided current time provider (to facilitate testing) # Application-provided current time provider (to facilitate testing)
getCurrentBeaconTime*: GetBeaconTimeFn getCurrentBeaconTime*: GetBeaconTimeFn
@ -167,6 +169,7 @@ proc new*(T: type Eth2Processor,
syncCommitteeMsgPool: ref SyncCommitteeMsgPool, syncCommitteeMsgPool: ref SyncCommitteeMsgPool,
lightClientPool: ref LightClientPool, lightClientPool: ref LightClientPool,
quarantine: ref Quarantine, quarantine: ref Quarantine,
blobQuarantine: ref BlobQuarantine,
rng: ref HmacDrbgContext, rng: ref HmacDrbgContext,
getBeaconTime: GetBeaconTimeFn, getBeaconTime: GetBeaconTimeFn,
taskpool: TaskPoolPtr taskpool: TaskPoolPtr
@ -184,6 +187,7 @@ proc new*(T: type Eth2Processor,
syncCommitteeMsgPool: syncCommitteeMsgPool, syncCommitteeMsgPool: syncCommitteeMsgPool,
lightClientPool: lightClientPool, lightClientPool: lightClientPool,
quarantine: quarantine, quarantine: quarantine,
blobQuarantine: blobQuarantine,
getCurrentBeaconTime: getBeaconTime, getCurrentBeaconTime: getBeaconTime,
batchCrypto: BatchCrypto.new( batchCrypto: BatchCrypto.new(
rng = rng, rng = rng,
@ -234,9 +238,21 @@ proc processSignedBeaconBlock*(
# propagation of seemingly good blocks # propagation of seemingly good blocks
trace "Block validated" trace "Block validated"
var blobs: BlobSidecars
when typeof(signedBlock).toFork() >= ConsensusFork.Deneb:
if self.blobQuarantine[].hasBlobs(signedBlock):
blobs = self.blobQuarantine[].popBlobs(signedBlock.root)
else:
if not self.quarantine[].addBlobless(self.dag.finalizedHead.slot,
signedBlock):
notice "Block quarantine full (blobless)",
blockRoot = shortLog(signedBlock.root),
blck = shortLog(signedBlock.message),
signature = shortLog(signedBlock.signature)
self.blockProcessor[].addBlock( self.blockProcessor[].addBlock(
src, ForkedSignedBeaconBlock.init(signedBlock), src, ForkedSignedBeaconBlock.init(signedBlock),
BlobSidecars @[], blobs,
maybeFinalized = maybeFinalized, maybeFinalized = maybeFinalized,
validationDur = nanoseconds( validationDur = nanoseconds(
(self.getCurrentBeaconTime() - wallTime).nanoseconds)) (self.getCurrentBeaconTime() - wallTime).nanoseconds))
@ -268,23 +284,50 @@ proc processSignedBlobSidecar*(
# Potential under/overflows are fine; would just create odd metrics and logs # Potential under/overflows are fine; would just create odd metrics and logs
let delay = wallTime - signedBlobSidecar.message.slot.start_beacon_time let delay = wallTime - signedBlobSidecar.message.slot.start_beacon_time
debug "Blob received", delay if self.blobQuarantine[].hasBlob(signedBlobSidecar.message):
debug "Blob received, already in quarantine", delay
return ValidationRes.ok
else:
debug "Blob received", delay
let v = let v =
self.dag.validateBlobSidecar(self.quarantine, signedBlobSidecar, wallTime, idx) self.dag.validateBlobSidecar(self.quarantine,
signedBlobSidecar, wallTime, idx)
if v.isOk(): if v.isErr():
trace "Blob validated"
# TODO
# hand blob off to blob quarantine
blob_sidecars_received.inc()
blob_sidecar_delay.observe(delay.toFloatSeconds())
else:
debug "Dropping blob", error = v.error() debug "Dropping blob", error = v.error()
blob_sidecars_dropped.inc(1, [$v.error[0]]) blob_sidecars_dropped.inc(1, [$v.error[0]])
return v
debug "Blob validated, putting in blob quarantine"
self.blobQuarantine[].put(newClone(signedBlobSidecar.message))
var toAdd: seq[deneb.SignedBeaconBlock]
var skippedBlocks = false
for blobless in self.quarantine[].peekBlobless(
signedBlobSidecar.message.block_root):
if self.blobQuarantine[].hasBlobs(blobless):
toAdd.add(blobless)
else:
skippedBlocks = true
if len(toAdd) > 0:
let blobs = self.blobQuarantine[].peekBlobs(
signedBlobSidecar.message.block_root)
for blobless in toAdd:
self.blockProcessor[].addBlock(
MsgSource.gossip,
ForkedSignedBeaconBlock.init(blobless), blobs)
self.quarantine[].removeBlobless(blobless)
if not skippedBlocks:
# no blobless blocks remain in quarantine that need these blobs,
# so we can remove them.
self.blobQuarantine[].removeBlobs(toAdd[0].root)
blob_sidecars_received.inc()
blob_sidecar_delay.observe(delay.toFloatSeconds())
v v

View File

@ -14,6 +14,7 @@ import
stew/[byteutils, io2], stew/[byteutils, io2],
eth/p2p/discoveryv5/[enr, random2], eth/p2p/discoveryv5/[enr, random2],
eth/keys, eth/keys,
./consensus_object_pools/blob_quarantine,
./consensus_object_pools/vanity_logs/vanity_logs, ./consensus_object_pools/vanity_logs/vanity_logs,
./networking/topic_params, ./networking/topic_params,
./rpc/[rest_api, state_ttl_cache], ./rpc/[rest_api, state_ttl_cache],
@ -319,6 +320,7 @@ proc initFullNode(
LightClientPool()) LightClientPool())
validatorChangePool = newClone( validatorChangePool = newClone(
ValidatorChangePool.init(dag, attestationPool, onVoluntaryExitAdded)) ValidatorChangePool.init(dag, attestationPool, onVoluntaryExitAdded))
blobQuarantine = newClone(BlobQuarantine())
consensusManager = ConsensusManager.new( consensusManager = ConsensusManager.new(
dag, attestationPool, quarantine, node.elManager, dag, attestationPool, quarantine, node.elManager,
ActionTracker.init(rng, config.subscribeAllSubnets), ActionTracker.init(rng, config.subscribeAllSubnets),
@ -326,7 +328,8 @@ proc initFullNode(
config.defaultFeeRecipient, config.suggestedGasLimit) config.defaultFeeRecipient, config.suggestedGasLimit)
blockProcessor = BlockProcessor.new( blockProcessor = BlockProcessor.new(
config.dumpEnabled, config.dumpDirInvalid, config.dumpDirIncoming, config.dumpEnabled, config.dumpDirInvalid, config.dumpDirIncoming,
rng, taskpool, consensusManager, node.validatorMonitor, getBeaconTime) rng, taskpool, consensusManager, node.validatorMonitor,
blobQuarantine, getBeaconTime)
blockVerifier = blockVerifier =
proc(signedBlock: ForkedSignedBeaconBlock, maybeFinalized: bool): proc(signedBlock: ForkedSignedBeaconBlock, maybeFinalized: bool):
Future[Result[void, VerifierError]] = Future[Result[void, VerifierError]] =
@ -356,7 +359,7 @@ proc initFullNode(
config.doppelgangerDetection, config.doppelgangerDetection,
blockProcessor, node.validatorMonitor, dag, attestationPool, blockProcessor, node.validatorMonitor, dag, attestationPool,
validatorChangePool, node.attachedValidators, syncCommitteeMsgPool, validatorChangePool, node.attachedValidators, syncCommitteeMsgPool,
lightClientPool, quarantine, rng, getBeaconTime, taskpool) lightClientPool, quarantine, blobQuarantine, rng, getBeaconTime, taskpool)
syncManager = newSyncManager[Peer, PeerId]( syncManager = newSyncManager[Peer, PeerId](
node.network.peerPool, dag.cfg.DENEB_FORK_EPOCH, SyncQueueKind.Forward, getLocalHeadSlot, node.network.peerPool, dag.cfg.DENEB_FORK_EPOCH, SyncQueueKind.Forward, getLocalHeadSlot,
getLocalWallSlot, getFirstSlotAtFinalizedEpoch, getBackfillSlot, getLocalWallSlot, getFirstSlotAtFinalizedEpoch, getBackfillSlot,
@ -1348,6 +1351,9 @@ proc onSecond(node: BeaconNode, time: Moment) =
notice "Shutting down after having reached the target synced epoch" notice "Shutting down after having reached the target synced epoch"
bnStatus = BeaconNodeStatus.Stopping bnStatus = BeaconNodeStatus.Stopping
# TODO
# onSecond timer to handle missing blobs, similar to above for blocks
proc runOnSecondLoop(node: BeaconNode) {.async.} = proc runOnSecondLoop(node: BeaconNode) {.async.} =
const const
sleepTime = chronos.seconds(1) sleepTime = chronos.seconds(1)

View File

@ -17,8 +17,8 @@ import
../beacon_chain/spec/datatypes/deneb, ../beacon_chain/spec/datatypes/deneb,
../beacon_chain/gossip_processing/block_processor, ../beacon_chain/gossip_processing/block_processor,
../beacon_chain/consensus_object_pools/[ ../beacon_chain/consensus_object_pools/[
attestation_pool, blockchain_dag, block_quarantine, block_clearance, attestation_pool, blockchain_dag, blob_quarantine, block_quarantine,
consensus_manager], block_clearance, consensus_manager],
../beacon_chain/eth1/eth1_monitor, ../beacon_chain/eth1/eth1_monitor,
./testutil, ./testdbutil, ./testblockutil ./testutil, ./testdbutil, ./testblockutil
@ -41,6 +41,7 @@ suite "Block processor" & preset():
taskpool = Taskpool.new() taskpool = Taskpool.new()
verifier = BatchVerifier(rng: keys.newRng(), taskpool: taskpool) verifier = BatchVerifier(rng: keys.newRng(), taskpool: taskpool)
quarantine = newClone(Quarantine.init()) quarantine = newClone(Quarantine.init())
blobQuarantine = newClone(BlobQuarantine())
attestationPool = newClone(AttestationPool.init(dag, quarantine)) attestationPool = newClone(AttestationPool.init(dag, quarantine))
elManager = new ELManager # TODO: initialise this properly elManager = new ELManager # TODO: initialise this properly
actionTracker: ActionTracker actionTracker: ActionTracker
@ -56,7 +57,7 @@ suite "Block processor" & preset():
getTimeFn = proc(): BeaconTime = b2.message.slot.start_beacon_time() getTimeFn = proc(): BeaconTime = b2.message.slot.start_beacon_time()
processor = BlockProcessor.new( processor = BlockProcessor.new(
false, "", "", keys.newRng(), taskpool, consensusManager, false, "", "", keys.newRng(), taskpool, consensusManager,
validatorMonitor, getTimeFn) validatorMonitor, blobQuarantine, getTimeFn)
asyncTest "Reverse order block add & get" & preset(): asyncTest "Reverse order block add & get" & preset():
let missing = await processor.storeBlock( let missing = await processor.storeBlock(

View File

@ -20,6 +20,13 @@ func makeBlock(slot: Slot, parent: Eth2Digest): ForkedSignedBeaconBlock =
b.root = hash_tree_root(b.message) b.root = hash_tree_root(b.message)
ForkedSignedBeaconBlock.init(b) ForkedSignedBeaconBlock.init(b)
func makeBlobbyBlock(slot: Slot, parent: Eth2Digest): deneb.SignedBeaconBlock =
var
b = deneb.SignedBeaconBlock(
message: deneb.BeaconBlock(slot: slot, parent_root: parent))
b.root = hash_tree_root(b.message)
b
suite "Block quarantine": suite "Block quarantine":
test "Unviable smoke test": test "Unviable smoke test":
let let
@ -28,6 +35,8 @@ suite "Block quarantine":
b2 = makeBlock(Slot 2, b1.root) b2 = makeBlock(Slot 2, b1.root)
b3 = makeBlock(Slot 3, b2.root) b3 = makeBlock(Slot 3, b2.root)
b4 = makeBlock(Slot 4, b2.root) b4 = makeBlock(Slot 4, b2.root)
b5 = makeBlobbyBlock(Slot 4, b3.root)
b6 = makeBlobbyBlock(Slot 4, b4.root)
var quarantine: Quarantine var quarantine: Quarantine
@ -43,16 +52,27 @@ suite "Block quarantine":
quarantine.addOrphan(Slot 0, b3) quarantine.addOrphan(Slot 0, b3)
quarantine.addOrphan(Slot 0, b4) quarantine.addOrphan(Slot 0, b4)
quarantine.addBlobless(Slot 0, b5)
quarantine.addBlobless(Slot 0, b6)
(b4.root, ValidatorSig()) in quarantine.orphans (b4.root, ValidatorSig()) in quarantine.orphans
(b5.root, ValidatorSig()) in quarantine.blobless
(b6.root, ValidatorSig()) in quarantine.blobless
quarantine.addUnviable(b4.root) quarantine.addUnviable(b4.root)
check: check:
(b4.root, ValidatorSig()) notin quarantine.orphans (b4.root, ValidatorSig()) notin quarantine.orphans
(b5.root, ValidatorSig()) in quarantine.blobless
(b6.root, ValidatorSig()) notin quarantine.blobless
quarantine.addUnviable(b1.root) quarantine.addUnviable(b1.root)
check: check:
(b1.root, ValidatorSig()) notin quarantine.orphans (b1.root, ValidatorSig()) notin quarantine.orphans
(b2.root, ValidatorSig()) notin quarantine.orphans (b2.root, ValidatorSig()) notin quarantine.orphans
(b3.root, ValidatorSig()) notin quarantine.orphans (b3.root, ValidatorSig()) notin quarantine.orphans
(b5.root, ValidatorSig()) notin quarantine.blobless
(b6.root, ValidatorSig()) notin quarantine.blobless