chaindag: don't keep backfill block table in memory (#3429)

This PR names and documents the concept of the archive: a range of slots
for which we have degraded functionality in terms of historical access -
in particular:

* we don't support rewinding to states in this range
* we don't keep an in-memory representation of the block dag

The archive de-facto exists in a trusted-node-synced node, but this PR
gives it a name and drops the in-memory digest index.

In order to satisfy `GetBlocksByRange` requests, we ensure that we have
blocks for the entire archive period via backfill. Future versions may
relax this further, adding a "pre-archive" period that is fully pruned.

During by-slot searches in the archive (both for libp2p and rest
requests), an extra database lookup is used to covert the given `slot`
to a `root` - future versions will avoid this using era files which
natively are indexed by `slot`. That said, the lookup is quite
fast compared to the actual block loading given how trivial the table
is - it's hard to measure, even.

A collateral benefit of this PR is that checkpoint-synced nodes will see
100-200MB memory usage savings, thanks to the dropped in-memory cache -
future pruning work will bring this benefit to full nodes as well.

* document chaindag storage architecture and assumptions
* look up parent using block id instead of full block in clearance
(future-proofing the code against a future in which blocks come from era
files)
* simplify finalized block init, always writing the backfill portion to
db at startup (to ensure lookups work as expected)
* preallocate some extra memory for finalized blocks, to avoid immediate
realloc
This commit is contained in:
Jacek Sieka 2022-02-26 19:16:19 +01:00 committed by GitHub
parent 92e7e288e7
commit 40a4c01086
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 215 additions and 138 deletions

View File

@ -119,10 +119,28 @@ type
## are stored in a mod-increment pattern across fixed-sized arrays, which
## addresses most of the rest of the BeaconState sizes.
summaries: KvStoreRef # BlockRoot -> BeaconBlockSummary
summaries: KvStoreRef
## BlockRoot -> BeaconBlockSummary - permits looking up basic block
## information via block root - contains only summaries that were valid
## at some point in history - it is however possible that entries exist
## that are no longer part of the finalized chain history, thus the
## cache should not be used to answer fork choice questions - see
## `getHeadBlock` and `finalizedBlocks` instead.
##
## May contain entries for blocks that are not stored in the database.
##
## See `finalizedBlocks` for an index in the other direction.
finalizedBlocks*: FinalizedBlocks
## Blocks that are known to be finalized, per the latest head (v1.7.0+)
## Only blocks that have passed verification, either via state transition
## or backfilling are indexed here - thus, similar to `head`, it is part
## of the inner security ring and is used to answer security questions
## in the chaindag.
##
## May contain entries for blocks that are not stored in the database.
##
## See `summaries` for an index in the other direction.
DbKeyKind = enum
kHashToState

View File

@ -153,6 +153,7 @@ proc addHeadBlock*(
logScope:
blockRoot = shortLog(signedBlock.root)
blck = shortLog(signedBlock.message)
signature = shortLog(signedBlock.signature)
template blck(): untyped = signedBlock.message # shortcuts without copy
template blockRoot(): untyped = signedBlock.root
@ -162,11 +163,10 @@ proc addHeadBlock*(
# happens in the meantime - the block we requested will then be stale
# by the time it gets here.
if blck.slot <= dag.finalizedHead.slot:
let previous = dag.getBlockIdAtSlot(blck.slot)
if previous.isProposed() and blockRoot == previous.bid.root:
# We should not call the block added callback for blocks that already
# existed in the pool, as that may confuse consumers such as the fork
# choice.
let existing = dag.getBlockIdAtSlot(blck.slot)
# The exact slot match ensures we reject blocks that were orphaned in
# the finalized chain
if existing.bid.slot == blck.slot and blockRoot == existing.bid.root:
debug "Duplicate block"
return err(BlockError.Duplicate)
@ -188,18 +188,20 @@ proc addHeadBlock*(
# is on is no longer a viable fork candidate - we can't tell which is which
# at this stage, but we can check if we've seen the parent block previously
# and thus prevent requests for it to be downloaded again.
if dag.db.containsBlock(blck.parent_root):
debug "Block unviable due to pre-finalized-checkpoint parent"
let parentId = dag.getBlockId(blck.parent_root)
if parentId.isSome():
debug "Block unviable due to pre-finalized-checkpoint parent",
parentId = parentId.get()
return err(BlockError.UnviableFork)
debug "Block parent unknown or finalized already"
debug "Block parent unknown or finalized already", parentId
return err(BlockError.MissingParent)
if parent.slot >= signedBlock.message.slot:
if parent.slot >= blck.slot:
# A block whose parent is newer than the block itself is clearly invalid -
# discard it immediately
debug "Block with invalid parent",
parentBlock = shortLog(parent)
debug "Block older than parent",
parent = shortLog(parent)
return err(BlockError.Invalid)
@ -237,8 +239,8 @@ proc addHeadBlock*(
# A PublicKey or Signature isn't on the BLS12-381 curve
info "Unable to load signature sets",
err = e.error()
return err(BlockError.Invalid)
if not verifier.batchVerify(sigs):
info "Block signature verification failed",
signature = shortLog(signedBlock.signature)
@ -274,6 +276,7 @@ proc addBackfillBlock*(
logScope:
blockRoot = shortLog(signedBlock.root)
blck = shortLog(signedBlock.message)
signature = shortLog(signedBlock.signature)
backfill = (dag.backfill.slot, shortLog(dag.backfill.parent_root))
template blck(): untyped = signedBlock.message # shortcuts without copy
@ -282,19 +285,18 @@ proc addBackfillBlock*(
let startTick = Moment.now()
if blck.slot >= dag.backfill.slot:
let previous = dag.getBlockIdAtSlot(blck.slot)
if previous.isProposed() and blockRoot == previous.bid.root:
let existing = dag.getBlockIdAtSlot(blck.slot)
if existing.bid.slot == blck.slot and blockRoot == existing.bid.root:
# We should not call the block added callback for blocks that already
# existed in the pool, as that may confuse consumers such as the fork
# choice. While the validation result won't be accessed, it's IGNORE,
# according to the spec.
# choice.
debug "Duplicate block"
return err(BlockError.Duplicate)
# Block is older than finalized, but different from the block in our
# canonical history: it must be from an unviable branch
debug "Block from unviable fork",
finalizedHead = shortLog(dag.finalizedHead),
backfill = shortLog(dag.backfill)
finalizedHead = shortLog(dag.finalizedHead)
return err(BlockError.UnviableFork)
@ -306,16 +308,18 @@ proc addBackfillBlock*(
# can happen is when an invalid `--network` parameter is given during
# startup (though in theory, we check that - maybe the database was
# swapped or something?).
fatal "Checkpoint given during initial startup inconsistent with genesis - wrong network used when starting the node?"
fatal "Checkpoint given during initial startup inconsistent with genesis block - wrong network used when starting the node?",
genesis = shortLog(dag.genesis), tail = shortLog(dag.tail),
head = shortLog(dag.head)
quit 1
dag.backfillBlocks[blck.slot.int] = blockRoot
dag.backfill = blck.toBeaconBlockSummary()
dag.db.finalizedBlocks.insert(blck.slot, blockRoot)
notice "Received matching genesis block during backfill, backfill complete"
notice "Received final block during backfill, backfill complete"
# Backfill done - dag.backfill.slot now points to genesis block just like
# it would if we loaded a fully backfilled database - returning duplicate
# it would if we loaded a fully synced database - returning duplicate
# here is appropriate, though one could also call it ... ok?
return err(BlockError.Duplicate)
@ -325,14 +329,18 @@ proc addBackfillBlock*(
# If the hash is correct, the block itself must be correct, but the root does
# not cover the signature, which we check next
let proposerKey = dag.validatorKey(blck.proposer_index)
if proposerKey.isNone():
# This cannot happen, in theory, unless the checkpoint state is broken or
# there is a bug in our validator key caching scheme - in order not to
# send invalid attestations, we'll shut down defensively here - this might
# need revisiting in the future.
fatal "Invalid proposer in backfill block - checkpoint state corrupt?"
# We've verified that the block root matches our expectations by following
# the chain of parents all the way from checkpoint. If all those blocks
# were valid, the proposer_index in this block must also be valid, and we
# should have a key for it but we don't: this is either a bug on our from
# which we cannot recover, or an invalid checkpoint state was given in which
# case we're in trouble.
fatal "Invalid proposer in backfill block - checkpoint state corrupt?",
head = shortLog(dag.head), tail = shortLog(dag.tail),
genesis = shortLog(dag.genesis)
quit 1
if not verify_block_signature(
@ -342,21 +350,16 @@ proc addBackfillBlock*(
signedBlock.root,
proposerKey.get(),
signedBlock.signature):
info "Block signature verification failed",
signature = shortLog(signedBlock.signature)
info "Block signature verification failed"
return err(BlockError.Invalid)
let sigVerifyTick = Moment.now
dag.putBlock(signedBlock.asTrusted())
# Invariants maintained on startup
doAssert dag.backfillBlocks.lenu64 == dag.tail.slot.uint64
doAssert dag.backfillBlocks.lenu64 > blck.slot.uint64
dag.backfillBlocks[blck.slot.int] = blockRoot
dag.backfill = blck.toBeaconBlockSummary()
dag.db.finalizedBlocks.insert(blck.slot, blockRoot)
dag.backfill = blck.toBeaconBlockSummary()
let putBlockTick = Moment.now
debug "Block backfilled",
sigVerifyDur = sigVerifyTick - startTick,

View File

@ -9,11 +9,9 @@
import
chronicles,
../spec/forks
../spec/datatypes/[phase0, altair],
../spec/[helpers]
export chronicles, phase0, altair, helpers
export chronicles, forks
type
BlockId* = object

View File

@ -61,80 +61,110 @@ type
data*: BlockRef
ChainDAGRef* = ref object
## Pool of blocks responsible for keeping a DAG of resolved blocks.
## ChainDAG validates, stores and serves chain history of valid blocks
## according to the beacon chain state transtion. From genesis to the
## finalization point, block history is linear - from there, it branches out
## into a dag with several heads, one of which is considered canonical.
##
## It is responsible for the following
## As new blocks are added, new segments of the chain may finalize,
## discarding now unviable candidate histories.
##
## - Handle requests and updates to the "ColdDB" which
## holds the canonical chain.
## - Maintain a direct acyclic graph (DAG) of
## candidate chains from the last
## finalized block.
## In addition to storing blocks, the chaindag also is responsible for
## storing intermediate states in the database that are used to recreate
## chain history at any point in time through a rewinding process that loads
## a snapshots and applies blocks until the desired point in history is
## reached.
##
## When a chain becomes finalized, it is saved in the ColdDB,
## the rejected candidates are discarded and this pool
## is pruned, only keeping the last finalized block.
## Several indices are kept in memory to enable fast lookups - their shape
## and contents somewhat depend on how the chain was instantiated: sync
## from genesis or checkpoint, and therefore, what features we can offer in
## terms of historical replay.
##
## The last finalized block is called the tail block.
# -----------------------------------
# ColdDB - Canonical chain
## Beacuse the state transition is forwards-only, checkpoint sync generally
## allows replaying states from that point onwards - anything earlier
## would require a backfill of blocks and a subsequent replay from genesis.
##
## Era files contain state snapshots along the way, providing arbitrary
## starting points for replay and can be used to frontfill the archive -
## however, they are not used until the contents have been verified via
## parent_root ancestry.
##
## The chain and the various pointers and indices we keep can be seen in
## the following graph: depending on how the chain was instantiated, some
## pointers may overlap and some indices might be empty as a result.
##
## / heads
## /-------* |
## *--------*---------*---------------*--------------*
## | | | | |
## genesis backfill tail finalizedHead head
## | | |
## archive finalizedBlocks forkBlocks
##
## The archive is the the part of finalized history for which we no longer
## recreate states quickly because we don't have a reasonable state to
## start replay from - when starting from a checkpoint, this is the typical
## case - recreating history requires either replaying from genesis or
## providing an earlier checkpoint state.
##
## We do not keep an in-memory index for the archive - instead, lookups are
## made via `BeaconChainDB.finalizedBlocks` which covers the full range from
## `backfill` to `finalizedHead`.
db*: BeaconChainDB
## ColdDB - Stores the canonical chain
## Database of recent chain history as well as the state and metadata
## needed to pick up where we left off after a restart - in particular,
## the DAG and the canonical head are stored here, as well as several
## caches.
validatorMonitor*: ref ValidatorMonitor
# -----------------------------------
# ChainDAGRef - DAG of candidate chains
forkBlocks*: HashSet[KeyedBlockRef]
## root -> BlockRef mapping of blocks still relevant to fork choice, ie
## root -> BlockRef mapping of blocks relevant to fork choice, ie
## those that have not yet been finalized - covers the slots
## `finalizedHead.slot..head.slot` (inclusive)
## `finalizedHead.slot..head.slot` (inclusive) - dag.heads keeps track
## of each potential head block in this table.
finalizedBlocks*: seq[BlockRef]
## Slot -> BlockRef mapping for the canonical chain - use getBlockAtSlot
## to access, generally - covers the slots
## `tail.slot..finalizedHead.slot` (including the finalized head block) -
## indices are thus offset by tail.slot
backfillBlocks*: seq[Eth2Digest]
## Slot -> Eth2Digest, covers genesis.slot..tail.slot - 1 (inclusive)
## Slot -> BlockRef mapping for the finalized portion of the canonical
## chain - use getBlockAtSlot to access
## Covers the slots `tail.slot..finalizedHead.slot` (including the
## finalized head block). Indices offset by `tail.slot`.
genesis*: BlockRef
## The genesis block of the network
## The genesis block of the network
tail*: BlockRef
## The earliest finalized block for which we have a corresponding state -
## when making a replay of chain history, this is as far back as we can
## go - the tail block is unique in that its parent is set to `nil`, even
## in the case where an earlier genesis block exists.
## The earliest finalized block for which we have a corresponding state -
## when making a replay of chain history, this is as far back as we can
## go - the tail block is unique in that its parent is set to `nil`, even
## in the case where an earlier genesis block exists.
backfill*: BeaconBlockSummary
## The backfill points to the oldest block that we have in the database -
## when backfilling, the first block to download is the parent of this block
## The backfill points to the oldest block with an unbroken ancestry from
## dag.tail - when backfilling, we'll move backwards in time starting
## with the parent of this block until we reach `genesis`.
heads*: seq[BlockRef]
## Candidate heads of candidate chains
finalizedHead*: BlockSlot
## The latest block that was finalized according to the block in head
## Ancestors of this block are guaranteed to have 1 child only.
## The latest block that was finalized according to the block in head
## Ancestors of this block are guaranteed to have 1 child only.
# -----------------------------------
# Pruning metadata
lastPrunePoint*: BlockSlot
## The last prune point
## We can prune up to finalizedHead
## The last prune point
## We can prune up to finalizedHead
# -----------------------------------
# Rewinder - Mutable state processing
headState*: StateData
## State given by the head block - must only be updated in `updateHead` -
## always matches dag.head
## State given by the head block - must only be updated in `updateHead` -
## always matches dag.head
epochRefState*: StateData
## State used to produce epochRef instances - must only be used in

View File

@ -106,25 +106,16 @@ proc updateValidatorKeys*(dag: ChainDAGRef, validators: openArray[Validator]) =
dag.db.updateImmutableValidators(validators)
proc updateFinalizedBlocks*(dag: ChainDAGRef) =
template update(s: Slot) =
if s < dag.tail.slot:
if not dag.backfillBlocks[s.int].isZero:
dag.db.finalizedBlocks.insert(s, dag.backfillBlocks[s.int])
else:
let dagIndex = int(s - dag.tail.slot)
if not isNil(dag.finalizedBlocks[dagIndex]):
dag.db.finalizedBlocks.insert(s, dag.finalizedBlocks[dagIndex].root)
if dag.db.db.readOnly: return # TODO abstraction leak - where to put this?
if not dag.db.db.readOnly: # TODO abstraction leak - where to put this?
dag.db.withManyWrites:
if dag.db.finalizedBlocks.low.isNone():
for s in dag.backfill.slot .. dag.finalizedHead.slot:
update(s)
else:
for s in dag.backfill.slot ..< dag.db.finalizedBlocks.low.get():
update(s)
for s in dag.db.finalizedBlocks.high.get() + 1 .. dag.finalizedHead.slot:
update(s)
dag.db.withManyWrites:
let high = dag.db.finalizedBlocks.high.expect(
"wrote at least tailRef during init")
for s in high + 1 .. dag.finalizedHead.slot:
let tailIdx = int(s - dag.tail.slot)
if not isNil(dag.finalizedBlocks[tailIdx]):
dag.db.finalizedBlocks.insert(s, dag.finalizedBlocks[tailIdx].root)
func validatorKey*(
dag: ChainDAGRef, index: ValidatorIndex or uint64): Option[CookedPubKey] =
@ -219,7 +210,6 @@ func getBlockAtSlot*(dag: ChainDAGRef, slot: Slot): BlockSlot =
## May return an empty BlockSlot (where blck is nil!)
if slot == dag.genesis.slot:
# There may be gaps in the
return dag.genesis.atSlot(slot)
if slot > dag.finalizedHead.slot:
@ -232,45 +222,52 @@ func getBlockAtSlot*(dag: ChainDAGRef, slot: Slot): BlockSlot =
if slot >= dag.tail.slot:
var pos = int(slot - dag.tail.slot)
while true:
if dag.finalizedBlocks[pos] != nil:
return dag.finalizedBlocks[pos].atSlot(slot)
if pos == 0:
break
doAssert pos > 0, "We should have returned the tail"
pos -= 1
if dag.tail.slot == 0:
raiseAssert "Genesis missing"
pos = pos - 1
BlockSlot() # nil blck!
func getBlockIdAtSlot*(dag: ChainDAGRef, slot: Slot): BlockSlotId =
## Retrieve the canonical block at the given slot, or the last block that
## comes before - similar to atSlot, but without the linear scan
## comes before - similar to atSlot, but without the linear scan - may hit
## the database to look up early indices.
if slot == dag.genesis.slot:
return dag.genesis.bid.atSlot(slot)
if slot >= dag.tail.slot:
return dag.getBlockAtSlot(slot).toBlockSlotId()
var pos = slot.int
while pos >= dag.backfill.slot.int:
if not dag.backfillBlocks[pos].isZero:
return BlockId(root: dag.backfillBlocks[pos], slot: Slot(pos)).atSlot(slot)
pos -= 1
let finlow = dag.db.finalizedBlocks.low.expect("at least tailRef written")
if slot >= finlow:
var pos = slot
while true:
let root = dag.db.finalizedBlocks.get(pos)
if root.isSome():
return BlockId(root: root.get(), slot: pos).atSlot(slot)
doAssert pos > finlow, "We should have returned the finlow"
pos = pos - 1
BlockSlotId() # not backfilled yet, and not genesis
proc getBlockId*(dag: ChainDAGRef, root: Eth2Digest): Opt[BlockId] =
## Look up block id by root in history - useful for turning a root into a
## slot - may hit the database, may return blocks that have since become
## unviable - use `getBlockIdAtSlot` to check that the block is still viable
## if used in a sensitive context
block: # If we have a BlockRef, this is the fastest way to get a block id
let blck = dag.getBlockRef(root)
if blck.isOk():
return ok(blck.get().bid)
block: # Otherwise, we might have a summary in the database
block: # We might have a summary in the database
let summary = dag.db.getBeaconBlockSummary(root)
if summary.isOk():
return ok(BlockId(root: root, slot: summary.get().slot))
@ -498,25 +495,18 @@ proc init*(T: type ChainDAGRef, cfg: RuntimeConfig, db: BeaconChainDB,
# Backfills are blocks that we have in the database, but can't construct a
# state for without replaying from genesis
var
backfillBlocks = newSeq[Eth2Digest](tailRef.slot.int)
# This is where we'll start backfilling, worst case - we might refine this
# while loading blocks, in case backfilling has happened already
backfill = withBlck(tailBlock): blck.message.toBeaconBlockSummary()
# The most recent block that we load from the finalized blocks table
midRef: BlockRef
backRoot: Option[Eth2Digest]
# Start by loading basic block information about finalized blocks - this
# generally goes from genesis (or the earliest backfilled block) all the way
# to the latest block finalized in the `head` history - we have to be careful
# though, versions prior to 1.7.0 did not store finalized blocks in the
# database, and / or the application might have crashed between the head and
# finalized blocks updates
# finalized blocks updates.
for slot, root in db.finalizedBlocks:
if slot < tailRef.slot:
backfillBlocks[slot.int] = root
if backRoot.isNone():
backRoot = some(root)
discard # TODO don't load this range at all from the database
elif slot == tailRef.slot:
if root != tailRef.root:
fatal "Finalized blocks do not meet with tail, database corrupt?",
@ -541,10 +531,14 @@ proc init*(T: type ChainDAGRef, cfg: RuntimeConfig, db: BeaconChainDB,
var
headRef: BlockRef
curRef: BlockRef
finalizedBlocks = newSeq[Eth2Digest](
if midRef == nil: tailRef.slot.int + 1
else: 0
)
# Now load the part from head to finalized (going backwards) - if we loaded
# any finalized blocks, we should hit the last of them while loading this
# history
# Load the part from head going backwards - if we found any entries in the
# finalized block table, we'll stop at `midRef`, otherwise we'll keep going
# as far as we can find headers and fill in the finalized blocks from tail
for blck in db.getAncestorSummaries(headRoot):
if midRef != nil and blck.summary.slot <= midRef.slot:
if midRef.slot != blck.summary.slot or midRef.root != blck.root:
@ -567,9 +561,9 @@ proc init*(T: type ChainDAGRef, cfg: RuntimeConfig, db: BeaconChainDB,
break
if blck.summary.slot < tailRef.slot:
backfillBlocks[blck.summary.slot.int] = blck.root
if backRoot.isNone():
backfill = blck.summary
doAssert midRef == nil,
"If we loaded any blocks from the finalized slot table, they should have included tailRef"
finalizedBlocks[blck.summary.slot.int] = blck.root
elif blck.summary.slot == tailRef.slot:
if curRef == nil:
curRef = tailRef
@ -577,6 +571,9 @@ proc init*(T: type ChainDAGRef, cfg: RuntimeConfig, db: BeaconChainDB,
else:
link(tailRef, curRef)
curRef = curRef.parent
if midRef == nil:
finalizedBlocks[blck.summary.slot.int] = blck.root
else:
let newRef = BlockRef.init(blck.root, blck.summary.slot)
if curRef == nil:
@ -588,9 +585,23 @@ proc init*(T: type ChainDAGRef, cfg: RuntimeConfig, db: BeaconChainDB,
trace "Populating block dag", key = curRef.root, val = curRef
if backRoot.isSome():
backfill = db.getBeaconBlockSummary(backRoot.get()).expect(
"Backfill block must have a summary")
if finalizedBlocks.len > 0 and not db.db.readOnly: # TODO abstraction leak
db.withManyWrites:
for i, root in finalizedBlocks.mpairs:
if root.isZero: continue
db.finalizedBlocks.insert(Slot(i), root)
let backfill = block:
let backfillSlot = db.finalizedBlocks.low.expect("tail at least")
if backfillSlot < tailRef.slot:
let backfillRoot = db.finalizedBlocks.get(backfillSlot).expect(
"low to be loadable")
db.getBeaconBlockSummary(backfillRoot).expect(
"Backfill block must have a summary")
else:
withBlck(tailBlock): blck.message.toBeaconBlockSummary()
let summariesTick = Moment.now()
@ -683,8 +694,6 @@ proc init*(T: type ChainDAGRef, cfg: RuntimeConfig, db: BeaconChainDB,
cfg,
getStateField(dag.headState.data, genesis_validators_root))
swap(dag.backfillBlocks, backfillBlocks) # avoid allocating a full copy
let forkVersions =
[cfg.GENESIS_FORK_VERSION, cfg.ALTAIR_FORK_VERSION,
cfg.BELLATRIX_FORK_VERSION, cfg.SHARDING_FORK_VERSION]
@ -732,7 +741,12 @@ proc init*(T: type ChainDAGRef, cfg: RuntimeConfig, db: BeaconChainDB,
dag.finalizedHead = tmp.atSlot(finalizedSlot)
block: # Set up tail -> finalizedHead
dag.finalizedBlocks.setLen((dag.finalizedHead.slot - dag.tail.slot).int + 1)
# Room for all finalized blocks from the tail onwards
let n = (dag.finalizedHead.slot - dag.tail.slot).int + 1
# Make room for some more blocks to avoid an instant reallocation
dag.finalizedBlocks = newSeqOfCap[BlockRef](int(n * 3 / 2))
dag.finalizedBlocks.setLen(n)
var tmp = dag.finalizedHead.blck
while not isNil(tmp):

View File

@ -374,6 +374,12 @@ suite "chain DAG finalization tests" & preset():
assign(tmpState[], dag.headState.data)
# skip slots so we can test gappy getBlockAtSlot
check process_slots(
defaultRuntimeConfig, tmpState[],
getStateField(tmpState[], slot) + 2.uint64,
cache, info, {}).isOk()
for i in 0 ..< (SLOTS_PER_EPOCH * 6):
if i == 1:
# There are 2 heads now because of the fork at slot 1
@ -392,12 +398,15 @@ suite "chain DAG finalization tests" & preset():
check:
dag.heads.len() == 1
dag.getBlockAtSlot(0.Slot) == BlockSlot(blck: dag.genesis, slot: 0.Slot)
dag.getBlockAtSlot(2.Slot) ==
BlockSlot(blck: dag.getBlockAtSlot(1.Slot).blck, slot: 2.Slot)
dag.getBlockAtSlot(dag.head.slot) == BlockSlot(
blck: dag.head, slot: dag.head.slot.Slot)
dag.getBlockAtSlot(dag.head.slot + 1) == BlockSlot(
blck: dag.head, slot: dag.head.slot.Slot + 1)
not dag.containsForkBlock(dag.getBlockAtSlot(1.Slot).blck.root)
not dag.containsForkBlock(dag.getBlockAtSlot(5.Slot).blck.root)
dag.containsForkBlock(dag.finalizedHead.blck.root)
check:
@ -713,6 +722,8 @@ suite "Backfill":
# No epochref for pre-tail epochs
dag.getEpochRef(dag.tail, dag.tail.slot.epoch - 1, true).isErr()
dag.backfill == tailBlock.phase0Data.message.toBeaconBlockSummary()
var
badBlock = blocks[^2].phase0Data
badBlock.signature = blocks[^3].phase0Data.signature
@ -738,6 +749,8 @@ suite "Backfill":
blocks[^2].toBlockId().atSlot()
dag.getBlockIdAtSlot(dag.tail.slot - 2) == BlockSlotId()
dag.backfill == blocks[^2].phase0Data.message.toBeaconBlockSummary()
check:
dag.addBackfillBlock(blocks[^3].phase0Data).isOk()
@ -766,6 +779,7 @@ suite "Backfill":
check:
dag.addBackfillBlock(blocks[^2].phase0Data).isOk()
dag.backfill == blocks[^2].phase0Data.message.toBeaconBlockSummary()
let
validatorMonitor2 = newClone(ValidatorMonitor.init())
@ -782,4 +796,4 @@ suite "Backfill":
dag2.getBlockIdAtSlot(dag.tail.slot - 1) ==
blocks[^2].toBlockId().atSlot()
dag2.getBlockIdAtSlot(dag.tail.slot - 2) == BlockSlotId()
dag2.backfill.slot == blocks[^2].toBlockId().slot
dag2.backfill == blocks[^2].phase0Data.message.toBeaconBlockSummary()