State-only checkpoint state startup (#4251)

Currently, we require genesis and a checkpoint block and state to start
from an arbitrary slot - this PR relaxes this requirement so that we can
start with a state alone.

The current trusted-node-sync algorithm works by first downloading
blocks until we find an epoch aligned non-empty slot, then downloads the
state via slot.

However, current
[proposals](https://github.com/ethereum/beacon-APIs/pull/226) for
checkpointing prefer finalized state as
the main reference - this allows more simple access control and caching
on the server side - in particular, this should help checkpoint-syncing
from sources that have a fast `finalized` state download (like infura
and teku) but are slow when accessing state via slot.

Earlier versions of Nimbus will not be able to read databases created
without a checkpoint block and genesis. In most cases, backfilling makes
the database compatible except where genesis is also missing (custom
networks).

* backfill checkpoint block from libp2p instead of checkpoint source,
when doing trusted node sync
* allow starting the client without genesis / checkpoint block
* perform epoch start slot lookahead when loading tail state, so as to
deal with the case where the epoch start slot does not have a block
* replace `--blockId` with `--state-id` in TNS command line
* when replaying, also look at the parent of the last-known-block (even
if we don't have the parent block data, we can still replay from a
"parent" state) - in particular, this clears the way for implementing
state pruning
* deprecate `--finalized-checkpoint-block` option (no longer needed)
This commit is contained in:
Jacek Sieka 2022-11-02 11:02:38 +01:00 committed by GitHub
parent aff9147c31
commit d839b9d07e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 553 additions and 528 deletions

View File

@ -449,6 +449,11 @@ OK: 2/2 Fail: 0/2 Skip: 0/2
+ roundtrip OK
```
OK: 2/2 Fail: 0/2 Skip: 0/2
## Starting states
```diff
+ Starting state without block OK
```
OK: 1/1 Fail: 0/1 Skip: 0/1
## Sync committee pool
```diff
+ Aggregating votes OK
@ -598,4 +603,4 @@ OK: 1/1 Fail: 0/1 Skip: 0/1
OK: 9/9 Fail: 0/9 Skip: 0/9
---TOTAL---
OK: 331/336 Fail: 0/336 Skip: 5/336
OK: 332/337 Fail: 0/337 Skip: 5/337

View File

@ -1101,6 +1101,21 @@ proc getState*(
db.immutableValidators, db.statesNoVal[T.toFork], key.data, output,
rollback)
proc getState*(
db: BeaconChainDB, fork: BeaconStateFork, state_root: Eth2Digest,
state: var ForkedHashedBeaconState, rollback: RollbackProc): bool =
if state.kind != fork:
# Avoid temporary (!)
state = (ref ForkedHashedBeaconState)(kind: fork)[]
withState(state):
if not db.getState(state_root, forkyState.data, rollback):
return false
forkyState.root = state_root
true
proc getStateRoot(db: BeaconChainDBV0,
root: Eth2Digest,
slot: Slot): Opt[Eth2Digest] =

View File

@ -312,6 +312,7 @@ type
name: "finalized-checkpoint-state" .}: Option[InputFile]
finalizedCheckpointBlock* {.
hidden
desc: "SSZ file specifying a recent finalized block"
name: "finalized-checkpoint-block" .}: Option[InputFile]
@ -763,11 +764,17 @@ type
name: "trusted-node-url"
.}: string
blockId* {.
desc: "Block id to sync to - this can be a block root, slot number, \"finalized\" or \"head\""
defaultValue: "finalized"
stateId* {.
desc: "State id to sync to - this can be \"finalized\", a slot number or state hash or \"head\""
defaultValue: "finalized",
name: "state-id"
.}: string
blockId* {.
hidden
desc: "Block id to sync to - this can be a block root, slot number, \"finalized\" or \"head\" (deprecated)"
.}: Option[string]
backfillBlocks* {.
desc: "Backfill blocks directly from REST server instead of fetching via API"
defaultValue: true

View File

@ -201,8 +201,81 @@ func getBlockIdAtSlot*(dag: ChainDAGRef, slot: Slot): Opt[BlockSlotId] =
err() # not backfilled yet
proc containsBlock(
cfg: RuntimeConfig, db: BeaconChainDB, slot: Slot, root: Eth2Digest): bool =
db.containsBlock(root, cfg.blockForkAtEpoch(slot.epoch))
proc getForkedBlock*(db: BeaconChainDB, root: Eth2Digest):
Opt[ForkedTrustedSignedBeaconBlock] {.gcsafe.}
Opt[ForkedTrustedSignedBeaconBlock] =
# When we only have a digest, we don't know which fork it's from so we try
# them one by one - this should be used sparingly
if (let blck = db.getBlock(root, bellatrix.TrustedSignedBeaconBlock);
blck.isSome()):
ok(ForkedTrustedSignedBeaconBlock.init(blck.get()))
elif (let blck = db.getBlock(root, altair.TrustedSignedBeaconBlock);
blck.isSome()):
ok(ForkedTrustedSignedBeaconBlock.init(blck.get()))
elif (let blck = db.getBlock(root, phase0.TrustedSignedBeaconBlock);
blck.isSome()):
ok(ForkedTrustedSignedBeaconBlock.init(blck.get()))
else:
err()
proc containsBlock(dag: ChainDAGRef, bid: BlockId): bool =
let fork = dag.cfg.blockForkAtEpoch(bid.slot.epoch)
if dag.db.containsBlock(bid.root, fork):
return true
# TODO avoid loading bytes from era
var bytes: seq[byte]
(bid.slot <= dag.finalizedHead.slot and
getBlockSZ(
dag.era, getStateField(dag.headState, historical_roots).asSeq,
bid.slot, bytes).isOk and bytes.len > 0)
proc getBlock*(
dag: ChainDAGRef, bid: BlockId,
T: type ForkyTrustedSignedBeaconBlock): Opt[T] =
dag.db.getBlock(bid.root, T) or
getBlock(
dag.era, getStateField(dag.headState, historical_roots).asSeq,
bid.slot, Opt[Eth2Digest].ok(bid.root), T)
proc getBlockSSZ*(dag: ChainDAGRef, bid: BlockId, bytes: var seq[byte]): bool =
# Load the SSZ-encoded data of a block into `bytes`, overwriting the existing
# content
let fork = dag.cfg.blockForkAtEpoch(bid.slot.epoch)
dag.db.getBlockSSZ(bid.root, bytes, fork) or
(bid.slot <= dag.finalizedHead.slot and
getBlockSSZ(
dag.era, getStateField(dag.headState, historical_roots).asSeq,
bid.slot, bytes).isOk)
proc getBlockSZ*(dag: ChainDAGRef, bid: BlockId, bytes: var seq[byte]): bool =
# Load the snappy-frame-compressed ("SZ") SSZ-encoded data of a block into
# `bytes`, overwriting the existing content
# careful: there are two snappy encodings in use, with and without framing!
# Returns true if the block is found, false if not
let fork = dag.cfg.blockForkAtEpoch(bid.slot.epoch)
dag.db.getBlockSZ(bid.root, bytes, fork) or
(bid.slot <= dag.finalizedHead.slot and
getBlockSZ(
dag.era, getStateField(dag.headState, historical_roots).asSeq,
bid.slot, bytes).isOk)
proc getForkedBlock*(
dag: ChainDAGRef, bid: BlockId): Opt[ForkedTrustedSignedBeaconBlock] =
let fork = dag.cfg.blockForkAtEpoch(bid.slot.epoch)
result.ok(ForkedTrustedSignedBeaconBlock(kind: fork))
withBlck(result.get()):
type T = type(blck)
blck = getBlock(dag, bid, T).valueOr:
getBlock(
dag.era, getStateField(dag.headState, historical_roots).asSeq,
bid.slot, Opt[Eth2Digest].ok(bid.root), T).valueOr:
result.err()
return
proc getBlockId(db: BeaconChainDB, root: Eth2Digest): Opt[BlockId] =
block: # We might have a summary in the database
@ -238,6 +311,15 @@ proc getBlockId*(dag: ChainDAGRef, root: Eth2Digest): Opt[BlockId] =
dag.db.getBlockId(root)
proc getForkedBlock*(
dag: ChainDAGRef, root: Eth2Digest): Opt[ForkedTrustedSignedBeaconBlock] =
let bid = dag.getBlockId(root)
if bid.isSome():
dag.getForkedBlock(bid.get())
else:
# In case we didn't have a summary - should be rare, but ..
dag.db.getForkedBlock(root)
func isCanonical*(dag: ChainDAGRef, bid: BlockId): bool =
## Return true iff the given `bid` is part of the history selected by `dag.head`
let current = dag.getBlockIdAtSlot(bid.slot).valueOr:
@ -513,10 +595,6 @@ func containsForkBlock*(dag: ChainDAGRef, root: Eth2Digest): bool =
## Checks for blocks at the finalized checkpoint or newer
KeyedBlockRef.asLookupKey(root) in dag.forkBlocks
proc containsBlock(
cfg: RuntimeConfig, db: BeaconChainDB, slot: Slot, root: Eth2Digest): bool =
db.containsBlock(root, cfg.blockForkAtEpoch(slot.epoch))
func isFinalizedStateSnapshot(slot: Slot): bool =
slot.is_epoch and slot.epoch mod EPOCHS_PER_STATE_SNAPSHOT == 0
@ -535,21 +613,6 @@ func isStateCheckpoint(dag: ChainDAGRef, bsi: BlockSlotId): bool =
(bsi.isProposed and bsi.bid == dag.tail) or
(bsi.slot.is_epoch and bsi.slot.epoch == (bsi.bid.slot.epoch + 1))
proc getState(
db: BeaconChainDB, fork: BeaconStateFork, state_root: Eth2Digest,
state: var ForkedHashedBeaconState, rollback: RollbackProc): bool =
if state.kind != fork:
# Avoid temporary (!)
state = (ref ForkedHashedBeaconState)(kind: fork)[]
withState(state):
if not db.getState(state_root, forkyState.data, rollback):
return false
forkyState.root = state_root
true
proc getState(
db: BeaconChainDB, cfg: RuntimeConfig, block_root: Eth2Digest, slot: Slot,
state: var ForkedHashedBeaconState, rollback: RollbackProc): bool =
@ -579,74 +642,48 @@ proc getState(
dag.db.getState(dag.cfg, bsi.bid.root, bsi.slot, state, rollback)
proc getForkedBlock*(db: BeaconChainDB, root: Eth2Digest):
Opt[ForkedTrustedSignedBeaconBlock] =
# When we only have a digest, we don't know which fork it's from so we try
# them one by one - this should be used sparingly
if (let blck = db.getBlock(root, bellatrix.TrustedSignedBeaconBlock);
blck.isSome()):
ok(ForkedTrustedSignedBeaconBlock.init(blck.get()))
elif (let blck = db.getBlock(root, altair.TrustedSignedBeaconBlock);
blck.isSome()):
ok(ForkedTrustedSignedBeaconBlock.init(blck.get()))
elif (let blck = db.getBlock(root, phase0.TrustedSignedBeaconBlock);
blck.isSome()):
ok(ForkedTrustedSignedBeaconBlock.init(blck.get()))
else:
err()
proc getStateByParent(
dag: ChainDAGRef, bid: BlockId, state: var ForkedHashedBeaconState): bool =
## Try to load the state referenced by the parent of the given `bid` - this
## state can be used to advance to the `bid` state itself.
var slot = bid.slot
proc getBlock*(
dag: ChainDAGRef, bid: BlockId,
T: type ForkyTrustedSignedBeaconBlock): Opt[T] =
dag.db.getBlock(bid.root, T) or
getBlock(
dag.era, getStateField(dag.headState, historical_roots).asSeq,
bid.slot, Opt[Eth2Digest].ok(bid.root), T)
let
summary = dag.db.getBeaconBlockSummary(bid.root).valueOr:
return false
parentMinSlot =
dag.db.getBeaconBlockSummary(summary.parent_root).
map(proc(x: auto): auto = x.slot).valueOr:
# in the cases that we don't have slot information, we'll search for the
# state for a few back from the `bid` slot - if there are gaps of empty
# slots larger than this, we will not be able to load the state using this
# trick
if slot.uint64 >= (EPOCHS_PER_STATE_SNAPSHOT * 2) * SLOTS_PER_EPOCH:
slot - (EPOCHS_PER_STATE_SNAPSHOT * 2) * SLOTS_PER_EPOCH
else:
Slot(0)
proc getBlockSSZ*(dag: ChainDAGRef, bid: BlockId, bytes: var seq[byte]): bool =
# Load the SSZ-encoded data of a block into `bytes`, overwriting the existing
# content
let fork = dag.cfg.blockForkAtEpoch(bid.slot.epoch)
dag.db.getBlockSSZ(bid.root, bytes, fork) or
(bid.slot <= dag.finalizedHead.slot and
getBlockSSZ(
dag.era, getStateField(dag.headState, historical_roots).asSeq,
bid.slot, bytes).isOk)
let rollbackAddr =
# Any restore point will do as long as it's not the object being updated
if unsafeAddr(state) == unsafeAddr(dag.headState):
unsafeAddr dag.clearanceState
else:
unsafeAddr dag.headState
proc getBlockSZ*(dag: ChainDAGRef, bid: BlockId, bytes: var seq[byte]): bool =
# Load the snappy-frame-compressed ("SZ") SSZ-encoded data of a block into
# `bytes`, overwriting the existing content
# careful: there are two snappy encodings in use, with and without framing!
# Returns true if the block is found, false if not
let fork = dag.cfg.blockForkAtEpoch(bid.slot.epoch)
dag.db.getBlockSZ(bid.root, bytes, fork) or
(bid.slot <= dag.finalizedHead.slot and
getBlockSZ(
dag.era, getStateField(dag.headState, historical_roots).asSeq,
bid.slot, bytes).isOk)
let v = addr state
func rollback() =
assign(v[], rollbackAddr[])
proc getForkedBlock*(
dag: ChainDAGRef, bid: BlockId): Opt[ForkedTrustedSignedBeaconBlock] =
while true:
if dag.db.getState(dag.cfg, summary.parent_root, slot, state, rollback):
return true
let fork = dag.cfg.blockForkAtEpoch(bid.slot.epoch)
result.ok(ForkedTrustedSignedBeaconBlock(kind: fork))
withBlck(result.get()):
type T = type(blck)
blck = getBlock(dag, bid, T).valueOr:
getBlock(
dag.era, getStateField(dag.headState, historical_roots).asSeq,
bid.slot, Opt[Eth2Digest].ok(bid.root), T).valueOr:
result.err()
return
if slot == parentMinSlot:
return false
proc getForkedBlock*(
dag: ChainDAGRef, root: Eth2Digest): Opt[ForkedTrustedSignedBeaconBlock] =
let bid = dag.getBlockId(root)
if bid.isSome():
dag.getForkedBlock(bid.get())
else:
# In case we didn't have a summary - should be rare, but ..
dag.db.getForkedBlock(root)
slot -= 1
return false
proc currentSyncCommitteeForPeriod*(
dag: ChainDAGRef,
@ -864,7 +901,10 @@ proc init*(T: type ChainDAGRef, cfg: RuntimeConfig, db: BeaconChainDB,
var
headRef, curRef: BlockRef
slot = head.slot
# When starting from a checkpoint with an empty block, we'll store the state
# "ahead" of the head slot - this slot would be considered finalized
slot = max(head.slot, (tail.slot.epoch + 1).start_slot)
# To know the finalized checkpoint of the head, we need to recreate its
# state - the tail is implicitly finalized, and if we have a finalized block
# table, that provides another hint
@ -894,23 +934,6 @@ proc init*(T: type ChainDAGRef, cfg: RuntimeConfig, db: BeaconChainDB,
# Try loading state from database - we need the head state early on to
# establish the (real) finalized checkpoint
if db.getState(cfg, blck.root, slot, dag.headState, noRollback):
# EpochRef needs an epoch boundary state
assign(dag.epochRefState, dag.headState)
var info: ForkedEpochInfo
while headBlocks.len > 0:
dag.applyBlock(
dag.headState, headBlocks.pop().bid, cache,
info).expect("head blocks should apply")
dag.head = headRef
assign(dag.clearanceState, dag.headState)
finalizedSlot =
max(finalizedSlot,
getStateField(dag.headState, finalized_checkpoint).epoch.start_slot)
foundHeadState = true
break
slot -= 1
@ -941,9 +964,34 @@ proc init*(T: type ChainDAGRef, cfg: RuntimeConfig, db: BeaconChainDB,
let summariesTick = Moment.now()
if not foundHeadState:
fatal "Could not load head state, database corrupt?",
head = shortLog(head), tail = shortLog(dag.tail)
quit 1
if not dag.getStateByParent(curRef.bid, dag.headState):
fatal "Could not load head state, database corrupt?",
head = shortLog(head), tail = shortLog(dag.tail)
quit 1
block:
# EpochRef needs an epoch boundary state
assign(dag.epochRefState, dag.headState)
var info: ForkedEpochInfo
while headBlocks.len > 0:
dag.applyBlock(
dag.headState, headBlocks.pop().bid, cache,
info).expect("head blocks should apply")
dag.head = headRef
assign(dag.clearanceState, dag.headState)
if dag.headState.latest_block_root == tail.root:
# In case we started from a checkpoint with an empty slot
finalizedSlot = getStateField(dag.headState, slot)
finalizedSlot =
max(
finalizedSlot,
getStateField(dag.headState, finalized_checkpoint).epoch.start_slot)
let
configFork = case dag.headState.kind
@ -1404,10 +1452,12 @@ proc updateState*(
# Move slot by slot to capture epoch boundary states
# TODO https://github.com/nim-lang/Nim/issues/19613
cur = dag.parentOrSlot(cur).valueOr:
notice "Request for pruned historical state",
request = shortLog(bsi), tail = shortLog(dag.tail),
cur = shortLog(cur)
return false
if not dag.getStateByParent(cur.bid, state):
notice "Request for pruned historical state",
request = shortLog(bsi), tail = shortLog(dag.tail),
cur = shortLog(cur)
return false
break
beacon_state_rewinds.inc()
@ -1743,8 +1793,10 @@ proc updateHead*(
## now fall from grace, or no longer be considered resolved.
doAssert not newHead.isNil()
# Could happen if enough blocks get invalidated and would corrupt database
doAssert newHead.slot >= dag.finalizedHead.slot
# Could happen if enough blocks get invalidated and would corrupt database -
# When finalized checkpoint is empty, the slot may also be smaller
doAssert newHead.slot >= dag.finalizedHead.slot or
newHead == dag.finalizedHead.blck
let
lastHead = dag.head
@ -1922,42 +1974,17 @@ proc updateHead*(
dag.onFinHappened(dag, data)
proc isInitialized*(T: type ChainDAGRef, db: BeaconChainDB): Result[void, cstring] =
# Lightweight check to see if we have the minimal information needed to
# load up a database - we don't check head here - if something is wrong with
# head, it's likely an initialized, but corrupt database - init will detect
# that
let
genesisBlockRoot = db.getGenesisBlock()
if not genesisBlockRoot.isSome():
return err("Genesis block root missing")
let
genesisBlock = db.getForkedBlock(genesisBlockRoot.get())
if not genesisBlock.isSome():
return err("Genesis block missing")
let
genesisStateRoot = withBlck(genesisBlock.get()): blck.message.state_root
if not db.containsState(genesisStateRoot):
return err("Genesis state missing")
## Lightweight check to see if it is likely that the given database has been
## initialized
let
tailBlockRoot = db.getTailBlock()
if not tailBlockRoot.isSome():
return err("Tail block root missing")
let
tailBlock = db.getForkedBlock(tailBlockRoot.get())
tailBlock = db.getBlockId(tailBlockRoot.get())
if not tailBlock.isSome():
return err("Tail block missing")
let
tailStateRoot = withBlck(tailBlock.get()): blck.message.state_root
if not db.containsState(tailStateRoot):
return err("Tail state missing")
return err("Tail block information missing")
ok()
@ -1966,15 +1993,18 @@ proc preInit*(
## Initialize a database using the given state, which potentially may be a
## non-genesis state.
##
## Databases created like this are incompatible with versions prior to
## 22.11.0.
## When used with a non-genesis state, the resulting database will not be
## compatible with pre-22.11 versions.
logScope:
stateRoot = getStateRoot(state)
stateRoot = $getStateRoot(state)
stateSlot = getStateField(state, slot)
doAssert getStateField(state, slot).is_epoch,
"Can only initialize database from epoch states"
withState(state):
db.putState(forkyState)
if forkyState.data.slot == GENESIS_SLOT:
let blck = get_initial_beacon_block(forkyState)
db.putBlock(blck)
@ -1982,11 +2012,12 @@ proc preInit*(
db.putHeadBlock(blck.root)
db.putTailBlock(blck.root)
notice "New genesis database initialized",
genesisBlockRoot = $blck.root
notice "Database initialized from genesis",
blockRoot = $blck.root
else:
let blockRoot = forkyState.latest_block_root()
# We write a summary but not the block contents - these will have to be
# backfilled from the network
db.putBeaconBlockSummary(blockRoot, BeaconBlockSummary(
slot: forkyState.data.latest_block_header.slot,
parent_root: forkyState.data.latest_block_header.parent_root
@ -1994,75 +2025,10 @@ proc preInit*(
db.putHeadBlock(blockRoot)
db.putTailBlock(blockRoot)
notice "New snapshot database initialized",
blockRoot = $blockRoot
proc preInit*(
T: type ChainDAGRef, db: BeaconChainDB,
genesisState, tailState: ForkedHashedBeaconState,
tailBlock: ForkedTrustedSignedBeaconBlock) =
# write a genesis state, the way the ChainDAGRef expects it to be stored in
# database
logScope:
genesisStateRoot = getStateRoot(genesisState)
genesisStateSlot = getStateField(genesisState, slot)
tailStateRoot = getStateRoot(tailState)
tailStateSlot = getStateField(tailState, slot)
let genesisBlockRoot = withState(genesisState):
if forkyState.root != getStateRoot(tailState):
# Different tail and genesis
if forkyState.data.slot >= getStateField(tailState, slot):
fatal "Tail state must be newer or the same as genesis state"
quit 1
let tail_genesis_validators_root =
getStateField(tailState, genesis_validators_root)
if forkyState.data.genesis_validators_root !=
tail_genesis_validators_root:
fatal "Tail state doesn't match genesis validators root, it is likely from a different network!",
genesis_validators_root = shortLog(forkyState.data.genesis_validators_root),
tail_genesis_validators_root = shortLog(tail_genesis_validators_root)
quit 1
let blck = get_initial_beacon_block(forkyState)
db.putBlock(blck)
db.putState(forkyState)
db.putGenesisBlock(blck.root)
blck.root
else: # tail and genesis are the same
withBlck(tailBlock):
db.putGenesisBlock(blck.root)
blck.root
withState(tailState):
withBlck(tailBlock):
# When looking up the state root of the tail block, we don't use the
# BlockSlot->state_root map, so the only way the init code can find the
# state is through the state root in the block - this could be relaxed
# down the line
if blck.message.state_root != forkyState.root:
fatal "State must match the given block",
tailBlck = shortLog(blck)
quit 1
db.putBlock(blck)
db.putState(forkyState)
db.putTailBlock(blck.root)
db.putHeadBlock(blck.root)
notice "New database from snapshot",
genesisBlockRoot = shortLog(genesisBlockRoot),
genesisStateRoot = shortLog(getStateRoot(genesisState)),
tailBlockRoot = shortLog(blck.root),
tailStateRoot = shortLog(state.root),
fork = forkyState.data.fork,
validators = forkyState.data.validators.len()
if db.getGenesisBlock().isSome():
notice "Checkpoint written to database", blockRoot = $blockRoot
else:
notice "Database initialized from checkpoint", blockRoot = $blockRoot
proc getProposer*(
dag: ChainDAGRef, head: BlockRef, slot: Slot): Option[ValidatorIndex] =

View File

@ -17,7 +17,6 @@ import
web3/[ethtypes, conversions],
chronicles,
eth/common/eth_types_json_serialization,
ssz_serialization/navigator,
../spec/eth2_ssz_serialization,
../spec/datatypes/phase0
@ -264,7 +263,3 @@ proc getRuntimeConfig*(
if eth2Network.isSome:
return getMetadataForNetwork(eth2Network.get).cfg
defaultRuntimeConfig
proc extractGenesisValidatorRootFromSnapshot*(
snapshot: string): Eth2Digest {.raises: [Defect, IOError, SszError].} =
sszMount(snapshot, phase0.BeaconState).genesis_validators_root[]

View File

@ -143,7 +143,7 @@ proc loadChainDag(
db: BeaconChainDB,
eventBus: EventBus,
validatorMonitor: ref ValidatorMonitor,
networkGenesisValidatorsRoot: Option[Eth2Digest],
networkGenesisValidatorsRoot: Opt[Eth2Digest],
shouldEnableTestFeatures: bool): ChainDAGRef =
info "Loading block DAG from database", path = config.databaseDir
@ -435,13 +435,12 @@ proc init*(T: type BeaconNode,
)
db = BeaconChainDB.new(config.databaseDir, inMemory = false)
var
genesisState, checkpointState: ref ForkedHashedBeaconState
checkpointBlock: ForkedTrustedSignedBeaconBlock
if config.finalizedCheckpointBlock.isSome:
warn "--finalized-checkpoint-block has been deprecated, ignoring"
if config.finalizedCheckpointState.isSome:
let checkpointState = if config.finalizedCheckpointState.isSome:
let checkpointStatePath = config.finalizedCheckpointState.get.string
checkpointState = try:
let tmp = try:
newClone(readSszForkedHashedBeaconState(
cfg, readAllBytes(checkpointStatePath).tryGet()))
except SszError as err:
@ -452,40 +451,13 @@ proc init*(T: type BeaconNode,
fatal "Failed to read checkpoint state file", err = err.msg
quit 1
if not getStateField(checkpointState[], slot).is_epoch:
if not getStateField(tmp[], slot).is_epoch:
fatal "--finalized-checkpoint-state must point to a state for an epoch slot",
slot = getStateField(checkpointState[], slot)
slot = getStateField(tmp[], slot)
quit 1
if config.finalizedCheckpointBlock.isNone:
if getStateField(checkpointState[], slot) > 0:
fatal "Specifying a non-genesis --finalized-checkpoint-state requires specifying --finalized-checkpoint-block as well"
quit 1
else:
let checkpointBlockPath = config.finalizedCheckpointBlock.get.string
try:
# Checkpoint block might come from an earlier fork than the state with
# the state having empty slots processed past the fork epoch.
let tmp = readSszForkedSignedBeaconBlock(
cfg, readAllBytes(checkpointBlockPath).tryGet())
checkpointBlock = tmp.asTrusted()
except SszError as err:
fatal "Invalid checkpoint block", err = err.formatMsg(checkpointBlockPath)
quit 1
except IOError as err:
fatal "Failed to load the checkpoint block", err = err.msg
quit 1
if not checkpointBlock.slot.is_epoch:
fatal "--finalized-checkpoint-block must point to a block for an epoch slot",
slot = checkpointBlock.slot
quit 1
elif config.finalizedCheckpointBlock.isSome:
# TODO We can download the state from somewhere in the future relying
# on the trusted `state_root` appearing in the checkpoint block.
fatal "--finalized-checkpoint-block cannot be specified without --finalized-checkpoint-state"
quit 1
tmp
else:
nil
let optJwtSecret = rng[].loadJwtSecret(config, allowCreate = false)
@ -516,12 +488,20 @@ proc init*(T: type BeaconNode,
notice "Running without execution client - validator features disabled (see https://nimbus.guide/eth1.html)"
var eth1Monitor: Eth1Monitor
if not ChainDAGRef.isInitialized(db).isOk():
var
tailState: ref ForkedHashedBeaconState
tailBlock: ForkedTrustedSignedBeaconBlock
if genesisStateContents.len == 0 and checkpointState == nil:
var genesisState =
if genesisStateContents.len > 0:
try:
newClone(readSszForkedHashedBeaconState(
cfg,
genesisStateContents.toOpenArrayByte(0, genesisStateContents.high())))
except CatchableError as err:
raiseAssert "Invalid baked-in state: " & err.msg
else:
nil
if not ChainDAGRef.isInitialized(db).isOk():
if genesisState == nil and checkpointState == nil:
when hasGenesisDetection:
if depositContractSnapshotContents.len > 0:
fatal "A deposits snapshot cannot be provided without also providing a matching beacon state snapshot"
@ -560,9 +540,6 @@ proc init*(T: type BeaconNode,
if bnStatus == BeaconNodeStatus.Stopping:
return nil
tailState = genesisState
tailBlock = get_initial_beacon_block(genesisState[])
notice "Eth2 genesis state detected",
genesisTime = phase0Genesis.genesisTime,
eth1Block = phase0Genesis.eth1_data.block_hash,
@ -574,31 +551,27 @@ proc init*(T: type BeaconNode,
"in order to support monitoring for genesis events"
quit 1
elif genesisStateContents.len == 0:
if getStateField(checkpointState[], slot) == GENESIS_SLOT:
genesisState = checkpointState
tailState = checkpointState
tailBlock = get_initial_beacon_block(genesisState[])
else:
fatal "State checkpoints cannot be provided for a network without a known genesis state"
if not genesisState.isNil and not checkpointState.isNil:
if getStateField(genesisState[], genesis_validators_root) !=
getStateField(checkpointState[], genesis_validators_root):
fatal "Checkpoint state does not match genesis - check the --network parameter",
rootFromGenesis = getStateField(
genesisState[], genesis_validators_root),
rootFromCheckpoint = getStateField(
checkpointState[], genesis_validators_root)
quit 1
else:
try:
genesisState = newClone(readSszForkedHashedBeaconState(
cfg,
genesisStateContents.toOpenArrayByte(0, genesisStateContents.high())))
except CatchableError as err:
raiseAssert "Invalid baked-in state: " & err.msg
if not checkpointState.isNil:
tailState = checkpointState
tailBlock = checkpointBlock
else:
tailState = genesisState
tailBlock = get_initial_beacon_block(genesisState[])
try:
ChainDAGRef.preInit(db, genesisState[], tailState[], tailBlock)
# Always store genesis state if we have it - this allows reindexing and
# answering genesis queries
if not genesisState.isNil:
ChainDAGRef.preInit(db, genesisState[])
if not checkpointState.isNil:
if genesisState.isNil or
getStateField(checkpointState[], slot) != GENESIS_SLOT:
ChainDAGRef.preInit(db, checkpointState[])
doAssert ChainDAGRef.isInitialized(db).isOk(), "preInit should have initialized db"
except CatchableError as exc:
error "Failed to initialize database", err = exc.msg
@ -620,11 +593,12 @@ proc init*(T: type BeaconNode,
validatorMonitor[].addMonitor(key, Opt.none(ValidatorIndex))
let
networkGenesisValidatorsRoot: Option[Eth2Digest] =
if genesisStateContents.len != 0:
some(extractGenesisValidatorRootFromSnapshot(genesisStateContents))
networkGenesisValidatorsRoot =
if not genesisState.isNil:
Opt.some(getStateField(genesisState[], genesis_validators_root))
else:
none(Eth2Digest)
Opt.none(Eth2Digest)
dag = loadChainDag(
config, cfg, db, eventBus,
validatorMonitor, networkGenesisValidatorsRoot,
@ -2044,11 +2018,15 @@ proc handleStartUpCmd(config: var BeaconNodeConf) {.raises: [Defect, CatchableEr
network.genesisData.toOpenArrayByte(0, network.genesisData.high())))
else: nil
if config.blockId.isSome():
error "--blockId option has been removed - use --state-id instead!"
quit 1
waitFor doTrustedNodeSync(
cfg,
config.databaseDir,
config.trustedNodeUrl,
config.blockId,
config.stateId,
config.backfillBlocks,
config.reindex,
genesis)

View File

@ -23,13 +23,16 @@ type
summaries: Table[Eth2Digest, BeaconBlockSummary]
slots: seq[Option[Eth2Digest]]
proc updateSlots(cache: var DbCache, slot: Slot) =
if cache.slots.lenu64() < slot:
cache.slots.setLen(slot.int + 1)
proc updateSlots(cache: var DbCache, root: Eth2Digest, slot: Slot) =
# The slots mapping stores one linear block history - we construct it by
# starting from a given root/slot and walking the known parents as far back
# as possible which ensures that all blocks belong to the same history
if cache.slots.len() < slot.int + 1:
cache.slots.setLen(slot.int + 1)
cache.updateSlots(slot)
var
root = root
@ -53,9 +56,6 @@ proc updateSlots(cache: var DbCache, root: Eth2Digest, slot: Slot) =
return
proc update(cache: var DbCache, blck: ForkySignedBeaconBlock) =
let
slot = blck.message.slot
if blck.root notin cache.summaries:
cache.summaries[blck.root] = blck.message.toBeaconBlockSummary()
@ -66,10 +66,14 @@ proc isKnown(cache: DbCache, slot: Slot): bool =
proc doTrustedNodeSync*(
cfg: RuntimeConfig, databaseDir: string, restUrl: string,
blockId: string, backfill: bool, reindex: bool,
stateId: string, backfill: bool, reindex: bool,
genesisState: ref ForkedHashedBeaconState = nil) {.async.} =
logScope:
restUrl
stateId
notice "Starting trusted node sync",
databaseDir, restUrl, blockId, backfill, reindex
databaseDir, backfill, reindex
let
db = BeaconChainDB.new(databaseDir, inMemory = false)
@ -96,7 +100,7 @@ proc doTrustedNodeSync*(
FAR_FUTURE_SLOT
var client = RestClientRef.new(restUrl).valueOr:
error "Cannot connect to server", url = restUrl, error = error
error "Cannot connect to server", error = error
quit 1
proc downloadBlock(slot: Slot):
@ -122,167 +126,104 @@ proc doTrustedNodeSync*(
raise lastError
let
localGenesisRoot = db.getGenesisBlock().valueOr:
let genesisState =
if genesisState != nil:
genesisState
else:
notice "Downloading genesis state", restUrl
let state = try:
await client.getStateV2(
StateIdent.init(StateIdentType.Genesis), cfg)
except CatchableError as exc:
error "Unable to download genesis state",
error = exc.msg, restUrl
quit 1
# If possible, we'll store the genesis state in the database - this is not
# strictly necessary but renders the resulting database compatible with
# versions prior to 22.11 and makes reindexing possible
let genesisState =
if (let genesisRoot = db.getGenesisBlock(); genesisRoot.isSome()):
let
genesisBlock = db.getForkedBlock(genesisRoot.get()).valueOr:
error "Cannot load genesis block from database",
genesisRoot = genesisRoot.get()
quit 1
genesisStateRoot = getForkedBlockField(genesisBlock, state_root)
stateFork = cfg.stateForkAtEpoch(GENESIS_EPOCH)
if isNil(state):
error "Server is missing genesis state",
restUrl
quit 1
state
tmp = (ref ForkedHashedBeaconState)(kind: stateFork)
if not db.getState(stateFork, genesisStateRoot, tmp[], noRollback):
error "Cannot load genesis state from database",
genesisStateRoot
quit 1
withState(genesisState[]):
info "Writing genesis state",
stateRoot = shortLog(forkyState.root),
genesis_validators_root =
shortLog(forkyState.data.genesis_validators_root)
if (genesisState != nil) and
(getStateRoot(tmp[]) != getStateRoot(genesisState[])):
error "Unexpected genesis state in database, is this the same network?",
databaseRoot = getStateRoot(tmp[]),
genesisRoot = getStateRoot(genesisState[])
quit 1
tmp
else:
let tmp = if genesisState != nil:
genesisState
else:
notice "Downloading genesis state", restUrl
try:
await client.getStateV2(
StateIdent.init(StateIdentType.Genesis), cfg)
except CatchableError as exc:
info "Unable to download genesis state",
error = exc.msg, restUrl
nil
db.putState(forkyState)
let blck = get_initial_beacon_block(forkyState)
info "Writing genesis block",
blockRoot = shortLog(blck.root),
blck = shortLog(blck.message)
db.putBlock(blck)
db.putGenesisBlock(blck.root)
dbCache.update(blck.asSigned())
blck.root
remoteGenesisRoot = try:
(await client.getBlockRoot(
BlockIdent.init(BlockIdentType.Genesis))).data.data.root
except CatchableError as exc:
error "Unable to download genesis block root",
error = exc.msg, restUrl
quit 1
if remoteGenesisRoot != localGenesisRoot:
error "Server genesis block root does not match local genesis, is the server serving the same chain?",
localGenesisRoot = shortLog(localGenesisRoot),
remoteGenesisRoot = shortLog(remoteGenesisRoot)
quit 1
if isNil(tmp):
notice "Server is missing genesis state, node will not be able to reindex history",
restUrl
tmp
let (checkpointSlot, checkpointRoot) = if dbHead.isNone:
notice "Downloading checkpoint block", restUrl, blockId
notice "Downloading checkpoint state"
let checkpointBlock = block:
# Finding a checkpoint block is tricky: we need the block to fall on an
# epoch boundary and when making the first request, we don't know exactly
# what slot we'll get - to find it, we'll keep walking backwards for a
# reasonable number of tries
var
checkpointBlock: ref ForkedSignedBeaconBlock
id = BlockIdent.decodeString(blockId).valueOr:
error "Cannot decode checkpoint block id, must be a slot, hash, 'finalized' or 'head'",
blockId
quit 1
found = false
for i in 0..<10:
let blck = try:
await client.getBlockV2(id, cfg)
except CatchableError as exc:
error "Unable to download checkpoint block",
error = exc.msg, restUrl
quit 1
if blck.isNone():
# Server returned 404 - no block was found at the given id, so we need
# to try an earlier slot - assuming we know of one!
if id.kind == BlockQueryKind.Slot:
let slot = id.slot
id = BlockIdent.init((id.slot.epoch() - 1).start_slot)
info "No block found at given slot, trying an earlier epoch",
slot, id
continue
else:
error "Cannot find a block at given block id, and cannot compute an earlier slot",
id, blockId
let
state = try:
let id = block:
let tmp = StateIdent.decodeString(stateId).valueOr:
error "Cannot decode checkpoint state id, must be a slot, hash, 'finalized' or 'head'"
quit 1
checkpointBlock = blck.get()
let checkpointSlot = getForkedBlockField(checkpointBlock[], slot)
if checkpointSlot.is_epoch():
found = true
break
id = BlockIdent.init((checkpointSlot.epoch() - 1).start_slot)
info "Downloaded checkpoint block does not fall on epoch boundary, trying an earlier epoch",
checkpointSlot, id
if not found:
# The ChainDAG requires that the tail falls on an epoch boundary, or it
# will be unable to load the corresponding state - this could be fixed, but
# for now, we ask the user to fix it instead
error "A checkpoint block from the first slot of an epoch could not be found with the given block id - pass an epoch slot with a block using the --block-id parameter",
blockId
if tmp.kind == StateQueryKind.Slot and not tmp.slot.is_epoch():
notice "Rounding given slot to epoch"
StateIdent.init(tmp.slot.epoch().start_slot)
else:
tmp
await client.getStateV2(id, cfg)
except CatchableError as exc:
error "Unable to download checkpoint state",
error = exc.msg
quit 1
checkpointBlock
let checkpointSlot = getForkedBlockField(checkpointBlock[], slot)
if checkpointBlock[].root in dbCache.summaries:
notice "Checkpoint block is already known, skipping checkpoint state download"
if state == nil:
error "No state found a given checkpoint",
stateId
quit 1
withBlck(checkpointBlock[]):
dbCache.updateSlots(blck.root, blck.message.slot)
if not getStateField(state[], slot).is_epoch():
error "State slot must fall on an epoch boundary",
slot = getStateField(state[], slot),
offset = getStateField(state[], slot) -
getStateField(state[], slot).epoch.start_slot
quit 1
if genesisState != nil:
if getStateField(state[], genesis_validators_root) !=
getStateField(genesisState[], genesis_validators_root):
error "Checkpoint state does not match genesis",
rootInCheckpoint = getStateField(state[], genesis_validators_root),
rootInGenesis = getStateField(genesisState[], genesis_validators_root)
quit 1
withState(genesisState[]):
let blck = get_initial_beacon_block(forkyState)
dbCache.update(blck.asSigned())
ChainDAGRef.preInit(db, genesisState[])
if getStateField(genesisState[], slot) != getStateField(state[], slot):
ChainDAGRef.preInit(db, state[])
else:
notice "Downloading checkpoint state", restUrl, checkpointSlot
ChainDAGRef.preInit(db, state[])
let
state = try:
await client.getStateV2(StateIdent.init(checkpointSlot), cfg)
except CatchableError as exc:
error "Unable to download checkpoint state",
error = exc.msg, restUrl, checkpointSlot
quit 1
let latest_bid = state[].latest_block_id()
if isNil(state):
notice "No state found at given checkpoint", checkpointSlot
quit 1
withState(state[]):
let latest_block_root = forkyState.latest_block_root
if latest_block_root != checkpointBlock[].root:
error "Checkpoint state does not match checkpoint block, server error?",
blockRoot = shortLog(checkpointBlock[].root),
blck = shortLog(checkpointBlock[]),
stateBlockRoot = shortLog(latest_block_root)
quit 1
info "Writing checkpoint state",
stateRoot = shortLog(forkyState.root)
db.putState(forkyState)
withBlck(checkpointBlock[]):
info "Writing checkpoint block",
blockRoot = shortLog(blck.root),
blck = shortLog(blck.message)
db.putBlock(blck.asTrusted())
db.putHeadBlock(blck.root)
db.putTailBlock(blck.root)
dbCache.update(blck)
(checkpointSlot, checkpointBlock[].root)
(latest_bid.slot, latest_bid.root)
else:
notice "Skipping checkpoint download, database already exists (remove db directory to get a fresh snapshot)",
databaseDir, head = shortLog(dbHead.get())
@ -295,15 +236,18 @@ proc doTrustedNodeSync*(
err = v.error()
quit 1
let missingSlots = block:
var total = 0
for i in 0..<checkpointSlot.int:
if dbCache.slots[i].isNone():
total += 1
total
dbCache.updateSlots(checkpointSlot)
let
missingSlots = block:
var total = 0
for slot in Slot(0)..<checkpointSlot:
if not dbCache.isKnown(slot):
total += 1
total
let canReindex = if missingSlots == 0:
info "Database fully backfilled"
info "Database backfilled"
true
elif backfill:
notice "Downloading historical blocks - you can interrupt this process at any time and it automatically be completed when you start the beacon node",
@ -331,35 +275,43 @@ proc doTrustedNodeSync*(
blck = shortLog(blck.message),
blockRoot = shortLog(blck.root)
var childSlot = blck.message.slot + 1
while true:
if childSlot >= dbCache.slots.lenu64():
error "Downloaded block does not match checkpoint history"
if blck.message.slot == checkpointSlot:
if blck.root != checkpointRoot:
error "Downloaded block does not match checkpoint history",
blck = shortLog(blck),
expectedRoot = shortLog(checkpointRoot)
quit 1
if not dbCache.slots[childSlot.int].isSome():
# Should never happen - we download slots backwards
error "Downloaded block does not match checkpoint history"
quit 1
let knownRoot = dbCache.slots[childSlot.int].get()
if knownRoot == ZERO_HASH:
childSlot += 1
continue
dbCache.summaries.withValue(knownRoot, summary):
if summary[].parent_root != blck.root:
error "Downloaded block does not match checkpoint history",
blockRoot = shortLog(blck.root),
expectedRoot = shortLog(summary[].parent_root)
else:
var childSlot = blck.message.slot + 1
while true:
if childSlot >= dbCache.slots.lenu64():
error "Downloaded block does not match checkpoint history"
quit 1
break
if not dbCache.slots[childSlot.int].isSome():
# Should never happen - we download slots backwards
error "Downloaded block does not match checkpoint history"
quit 1
# This shouldn't happen - we should have downloaded the child and
# updated knownBlocks before here
error "Expected child block not found in checkpoint history"
quit 1
let knownRoot = dbCache.slots[childSlot.int].get()
if knownRoot == ZERO_HASH:
childSlot += 1
continue
dbCache.summaries.withValue(knownRoot, summary):
if summary[].parent_root != blck.root:
error "Downloaded block does not match checkpoint history",
blockRoot = shortLog(blck.root),
expectedRoot = shortLog(summary[].parent_root)
quit 1
break
# This shouldn't happen - we should have downloaded the child and
# updated knownBlocks before here
error "Expected child block not found in checkpoint history"
quit 1
if blck.root notin dbCache.summaries:
db.putBlock(blck.asTrusted())
@ -435,5 +387,5 @@ when isMainModule:
let backfill = os.paramCount() > 4 and os.paramStr(5) == "true"
waitFor doTrustedNodeSync(
getRuntimeConfig(some os.paramStr(1)), os.paramStr(2), os.paramStr(3), os.paramStr(4),
backfill)
getRuntimeConfig(some os.paramStr(1)), os.paramStr(2), os.paramStr(3),
os.paramStr(4), backfill, false)

View File

@ -60,7 +60,6 @@ The following options are available:
this functionality [=false].
--weak-subjectivity-checkpoint Weak subjectivity checkpoint in the format block_root:epoch_number.
--finalized-checkpoint-state SSZ file specifying a recent finalized state.
--finalized-checkpoint-block SSZ file specifying a recent finalized block.
--node-name A name for this node that will appear in the logs. If you set this to 'auto', a
persistent automatically generated ID will be selected for each --data-dir
folder.

View File

@ -93,31 +93,27 @@ You can also resume the trusted node backfill at any time by simply running the
### Modify sync point
By default, the node will sync up to the latest finalized checkpoint of the node that you're syncing with. While you can choose a different sync point using a block hash or a slot number, this block must fall on an epoch boundary:
By default, the node will sync up to the latest finalized checkpoint of the node that you're syncing with. While you can choose a different sync point using a state hash or a slot number, this state must fall on an epoch boundary:
```sh
build/nimbus_beacon_node trustedNodeSync \
--network:mainnet \
--data-dir=build/data/shared_mainnet_0 \
--blockId:0x239940f2537f5bbee1a3829f9058f4c04f49897e4d325145153ca89838dfc9e2
--state-id:1024
```
### Sync from checkpoint files
If you have a state and a block file available, you can start the node using the finalized checkpoint options:
If you have a state file available, you can start the node using the `--finalized-checkpoint-state`:
```sh
# Obtain a state and a block from a Beacon API - these must be in SSZ format:
curl -o state.32000.ssz \
-H 'Accept: application/octet-stream' \
http://localhost:5052/eth/v2/debug/beacon/states/32000
curl -o block.32000.ssz \
-H 'Accept: application/octet-stream' \
http://localhost:5052/eth/v2/beacon/blocks/32000
# Start the beacon node using the downloaded state and block as starting point
# Start the beacon node using the downloaded state as starting point
./run-mainnet-beacon-node.sh \
--finalized-checkpoint-block=block.32000.ssz \
--finalized-checkpoint-state=state.32000.ssz
```

View File

@ -65,7 +65,6 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
replay = true):
let
(genesisState, depositContractSnapshot) = loadGenesis(validators, false)
genesisBlock = get_initial_beacon_block(genesisState[])
genesisTime = float getStateField(genesisState[], genesis_time)
var
@ -79,7 +78,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
let db = BeaconChainDB.new("block_sim_db")
defer: db.close()
ChainDAGRef.preInit(db, genesisState[], genesisState[], genesisBlock)
ChainDAGRef.preInit(db, genesisState[])
putInitialDepositContractSnapshot(db, depositContractSnapshot)
var

View File

@ -95,10 +95,7 @@ proc initialLoad(
))
else: {.error: "Unknown block fork: " & name(BlockType).}
ChainDAGRef.preInit(
db,
forkedState[], forkedState[],
asTrusted(signedBlock))
ChainDAGRef.preInit(db, forkedState[])
let
validatorMonitor = newClone(ValidatorMonitor.init())

View File

@ -28,6 +28,10 @@ proc pruneAtFinalization(dag: ChainDAGRef) =
if dag.needStateCachesAndForkChoicePruning():
dag.pruneStateCachesDAG()
type
AddHeadRes = Result[BlockRef, BlockError]
AddBackRes = Result[void, BlockError]
suite "Block pool processing" & preset():
setup:
var
@ -269,7 +273,7 @@ suite "Block pool processing" & preset():
b11 = dag.addHeadBlock(verifier, b1, nilPhase0Callback)
check:
b11.error == BlockError.Duplicate
b11 == AddHeadRes.err BlockError.Duplicate
not b10[].isNil
test "updateHead updates head and headState" & preset():
@ -378,7 +382,7 @@ suite "Block pool altair processing" & preset():
let
bAdd = dag.addHeadBlock(verifier, b, nilAltairCallback)
check:
bAdd.error() == BlockError.Invalid
bAdd == AddHeadRes.err BlockError.Invalid
block: # Randao reveal
var b = b2
@ -386,7 +390,7 @@ suite "Block pool altair processing" & preset():
let
bAdd = dag.addHeadBlock(verifier, b, nilAltairCallback)
check:
bAdd.error() == BlockError.Invalid
bAdd == AddHeadRes.err BlockError.Invalid
block: # Attestations
var b = b2
@ -394,7 +398,7 @@ suite "Block pool altair processing" & preset():
let
bAdd = dag.addHeadBlock(verifier, b, nilAltairCallback)
check:
bAdd.error() == BlockError.Invalid
bAdd == AddHeadRes.err BlockError.Invalid
block: # SyncAggregate empty
var b = b2
@ -402,7 +406,7 @@ suite "Block pool altair processing" & preset():
let
bAdd = dag.addHeadBlock(verifier, b, nilAltairCallback)
check:
bAdd.error() == BlockError.Invalid
bAdd == AddHeadRes.err BlockError.Invalid
block: # SyncAggregate junk
var b = b2
@ -412,7 +416,7 @@ suite "Block pool altair processing" & preset():
let
bAdd = dag.addHeadBlock(verifier, b, nilAltairCallback)
check:
bAdd.error() == BlockError.Invalid
bAdd == AddHeadRes.err BlockError.Invalid
suite "chain DAG finalization tests" & preset():
setup:
@ -552,7 +556,7 @@ suite "chain DAG finalization tests" & preset():
let status = dag.addHeadBlock(verifier, lateBlock, nilPhase0Callback)
# This _should_ be Unviable, but we can't tell, from the data that we have
# so MissingParent is the least wrong thing to reply
check: status.error == BlockError.UnviableFork
check: status == AddHeadRes.err BlockError.UnviableFork
block:
let
@ -819,8 +823,8 @@ suite "Backfill":
tailBlock = blocks[^1]
genBlock = get_initial_beacon_block(genState[])
ChainDAGRef.preInit(
db, genState[], tailState[], tailBlock.asTrusted())
ChainDAGRef.preInit(db, genState[])
ChainDAGRef.preInit(db, tailState[])
let
validatorMonitor = newClone(ValidatorMonitor.init())
@ -867,12 +871,15 @@ suite "Backfill":
badBlock.signature = blocks[^3].phase0Data.signature
check:
dag.addBackfillBlock(badBlock).error == BlockError.Invalid
dag.addBackfillBlock(badBlock) == AddBackRes.err BlockError.Invalid
check:
dag.addBackfillBlock(blocks[^3].phase0Data).error == BlockError.MissingParent
dag.addBackfillBlock(tailBlock.phase0Data).error == BlockError.Duplicate
dag.addBackfillBlock(genBlock.phase0Data.asSigned()).error == BlockError.MissingParent
dag.addBackfillBlock(blocks[^3].phase0Data) ==
AddBackRes.err BlockError.MissingParent
dag.addBackfillBlock(genBlock.phase0Data.asSigned()) ==
AddBackRes.err BlockError.MissingParent
dag.addBackfillBlock(tailBlock.phase0Data).isOk()
check:
dag.addBackfillBlock(blocks[^2].phase0Data).isOk()
@ -901,7 +908,8 @@ suite "Backfill":
check: dag.addBackfillBlock(blocks[blocks.len - i - 1].phase0Data).isOk()
check:
dag.addBackfillBlock(genBlock.phase0Data.asSigned).error == BlockError.Duplicate
dag.addBackfillBlock(genBlock.phase0Data.asSigned) ==
AddBackRes.err BlockError.Duplicate
dag.backfill.slot == GENESIS_SLOT
@ -914,8 +922,8 @@ suite "Backfill":
let
tailBlock = blocks[^1]
ChainDAGRef.preInit(
db, genState[], tailState[], tailBlock.asTrusted())
ChainDAGRef.preInit(db, genState[])
ChainDAGRef.preInit(db, tailState[])
let
validatorMonitor = newClone(ValidatorMonitor.init())
@ -964,7 +972,7 @@ suite "Backfill":
check:
dag.addBackfillBlock(genBlock.phase0Data.asSigned).isOk()
dag.addBackfillBlock(
genBlock.phase0Data.asSigned).error == BlockError.Duplicate
genBlock.phase0Data.asSigned) == AddBackRes.err BlockError.Duplicate
var
cache: StateCache
@ -983,6 +991,120 @@ suite "Backfill":
check:
dag2.head.root == next.root
suite "Starting states":
setup:
let
genState = (ref ForkedHashedBeaconState)(
kind: BeaconStateFork.Phase0,
phase0Data: initialize_hashed_beacon_state_from_eth1(
defaultRuntimeConfig, ZERO_HASH, 0,
makeInitialDeposits(SLOTS_PER_EPOCH.uint64, flags = {skipBlsValidation}),
{skipBlsValidation}))
tailState = assignClone(genState[])
db = BeaconChainDB.new("", inMemory = true)
quarantine = newClone(Quarantine.init())
test "Starting state without block":
var
cache: StateCache
info: ForkedEpochInfo
let
genBlock = get_initial_beacon_block(genState[])
blocks = block:
var blocks: seq[ForkedSignedBeaconBlock]
while getStateField(tailState[], slot).uint64 + 1 < SLOTS_PER_EPOCH:
blocks.add addTestBlock(tailState[], cache)
blocks
tailBlock = blocks[^1]
check process_slots(
defaultRuntimeConfig, tailState[], Slot(SLOTS_PER_EPOCH), cache, info,
{}).isOk()
ChainDAGRef.preInit(db, tailState[])
let
validatorMonitor = newClone(ValidatorMonitor.init())
dag = init(ChainDAGRef, defaultRuntimeConfig, db, validatorMonitor, {})
# check that we can update head to itself
dag.updateHead(dag.head, quarantine[])
check:
dag.finalizedHead.toBlockSlotId()[] == BlockSlotId(
bid: dag.tail, slot: (dag.tail.slot.epoch+1).start_slot)
dag.getBlockRef(tailBlock.root).get().bid == dag.tail
dag.getBlockRef(blocks[^2].root).isNone()
dag.getBlockId(tailBlock.root).get() == dag.tail
dag.getBlockId(blocks[^2].root).isNone()
dag.getBlockIdAtSlot(Slot(0)).isNone() # no genesis stored in db
dag.getBlockIdAtSlot(Slot(1)).isNone()
# Should get EpochRef for the tail however
# dag.getEpochRef(dag.tail, dag.tail.slot.epoch, true).isOk()
dag.getEpochRef(dag.tail, dag.tail.slot.epoch + 1, true).isOk()
# Should not get EpochRef for random block
dag.getEpochRef(
BlockId(root: blocks[^2].root, slot: dag.tail.slot), # root/slot mismatch
dag.tail.slot.epoch, true).isErr()
dag.getEpochRef(dag.tail, dag.tail.slot.epoch + 1, true).isOk()
dag.getFinalizedEpochRef() != nil
dag.backfill == tailBlock.phase0Data.message.toBeaconBlockSummary()
# Check that we can propose right from the checkpoint state
dag.getProposalState(dag.head, dag.head.slot + 1, cache).isOk()
var
badBlock = blocks[^2].phase0Data
badBlock.signature = blocks[^3].phase0Data.signature
check:
dag.addBackfillBlock(badBlock) == AddBackRes.err BlockError.Invalid
check:
dag.addBackfillBlock(blocks[^3].phase0Data) ==
AddBackRes.err BlockError.MissingParent
dag.addBackfillBlock(genBlock.phase0Data.asSigned()) ==
AddBackRes.err BlockError.MissingParent
dag.addBackfillBlock(tailBlock.phase0Data) == AddBackRes.ok()
check:
dag.addBackfillBlock(blocks[^2].phase0Data).isOk()
dag.getBlockRef(tailBlock.root).get().bid == dag.tail
dag.getBlockRef(blocks[^2].root).isNone()
dag.getBlockId(tailBlock.root).get() == dag.tail
dag.getBlockId(blocks[^2].root).get().root == blocks[^2].root
dag.getBlockIdAtSlot(dag.tail.slot).get().bid == dag.tail
dag.backfill == blocks[^2].phase0Data.message.toBeaconBlockSummary()
check:
dag.addBackfillBlock(blocks[^3].phase0Data).isOk()
dag.getBlockIdAtSlot(dag.tail.slot - 2).get() ==
blocks[^3].toBlockId().atSlot()
dag.getBlockIdAtSlot(dag.tail.slot - 3).isNone
for i in 3..<blocks.len:
check: dag.addBackfillBlock(blocks[blocks.len - i - 1].phase0Data).isOk()
check:
dag.addBackfillBlock(genBlock.phase0Data.asSigned).isOk()
dag.backfill.slot == GENESIS_SLOT
check:
dag.getFinalizedEpochRef() != nil
suite "Latest valid hash" & preset():
setup:
var runtimeConfig = defaultRuntimeConfig

View File

@ -178,9 +178,8 @@ suite "Light client" & preset():
# Initialize new DAG from checkpoint
let cpDb = BeaconChainDB.new("", inMemory = true)
ChainDAGRef.preInit(
cpDb, genesisState[],
dag.headState, dag.getForkedBlock(dag.head.bid).get)
ChainDAGRef.preInit(cpDb, genesisState[])
ChainDAGRef.preInit(cpDb, dag.headState) # dag.getForkedBlock(dag.head.bid).get)
let cpDag = ChainDAGRef.init(
cfg, cpDb, validatorMonitor, {},
lcDataConfig = LightClientDataConfig(

View File

@ -16,12 +16,6 @@ import
export beacon_chain_db, testblockutil, kvstore, kvstore_sqlite3
proc makeTestDB*(
tailState: ForkedHashedBeaconState,
tailBlock: ForkedTrustedSignedBeaconBlock): BeaconChainDB =
result = BeaconChainDB.new("", inMemory = true)
ChainDAGRef.preInit(result, tailState, tailState, tailBlock)
proc makeTestDB*(validators: Natural): BeaconChainDB =
let
genState = (ref ForkedHashedBeaconState)(
@ -32,8 +26,9 @@ proc makeTestDB*(validators: Natural): BeaconChainDB =
0,
makeInitialDeposits(validators.uint64, flags = {skipBlsValidation}),
{skipBlsValidation}))
genBlock = get_initial_beacon_block(genState[])
makeTestDB(genState[], genBlock)
result = BeaconChainDB.new("", inMemory = true)
ChainDAGRef.preInit(result, genState[])
proc getEarliestInvalidBlockRoot*(
dag: ChainDAGRef, initialSearchRoot: Eth2Digest,