nimbus-eth2/tests/consensus_spec/test_fixture_fork_choice.nim

412 lines
15 KiB
Nim
Raw Normal View History

# beacon_chain
# Copyright (c) 2018-2024 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: [].}
2021-12-01 12:55:42 +00:00
{.used.}
import
# Status libraries
stew/byteutils, chronicles,
taskpools,
# Internals
../../beacon_chain/spec/[helpers, forks, state_transition_block],
../../beacon_chain/fork_choice/[fork_choice, fork_choice_types],
../../beacon_chain/[beacon_chain_db, beacon_clock],
../../beacon_chain/consensus_object_pools/[
blockchain_dag, block_clearance, block_quarantine, spec_cache],
# Third-party
yaml,
# Test
../testutil, ../testdbutil,
./fixtures_utils, ./os_ops
from std/json import
JsonNode, getBool, getInt, getStr, hasKey, items, len, pairs, `$`, `[]`
from std/sequtils import mapIt, toSeq
from std/strutils import contains
from ../testbcutil import addHeadBlock
# Test format described at https://github.com/ethereum/consensus-specs/tree/v1.3.0/tests/formats/fork_choice
# Note that our implementation has been optimized with "ProtoArray"
# instead of following the spec (in particular the "store").
type
OpKind = enum
opOnTick
opOnAttestation
opOnBlock
opOnMergeBlock
opOnAttesterSlashing
opInvalidateHash
opChecks
BlobData = object
blobs: seq[KzgBlob]
proofs: seq[KzgProof]
Operation = object
valid: bool
# variant specific fields
case kind: OpKind
of opOnTick:
tick: int
of opOnAttestation:
att: phase0.Attestation
of opOnBlock:
blck: ForkedSignedBeaconBlock
blobData: Opt[BlobData]
of opOnMergeBlock:
powBlock: PowBlock
of opOnAttesterSlashing:
attesterSlashing: phase0.AttesterSlashing
of opInvalidateHash:
invalidatedHash: Eth2Digest
latestValidHash: Eth2Digest
of opChecks:
checks: JsonNode
proc initialLoad(
path: string, db: BeaconChainDB,
StateType, BlockType: typedesc
): tuple[dag: ChainDAGRef, fkChoice: ref ForkChoice] {.raises: [
IOError, UnconsumedInput].} =
let
forkedState = loadForkedState(
path/"anchor_state.ssz_snappy",
StateType.kind)
blck = parseTest(
path/"anchor_block.ssz_snappy",
SSZ, BlockType)
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)
2022-11-02 10:02:38 +00:00
ChainDAGRef.preInit(db, forkedState[])
let
validatorMonitor = newClone(ValidatorMonitor.init())
dag = ChainDAGRef.init(
forkedState[].kind.genesisTestRuntimeConfig, db, validatorMonitor, {})
fkChoice = newClone(ForkChoice.init(
dag.getFinalizedEpochRef(), dag.finalizedHead.blck,
ForkChoiceVersion.Pr3431))
(dag, fkChoice)
proc loadOps(
path: string,
fork: ConsensusFork
): seq[Operation] {.raises: [
IOError, KeyError, UnconsumedInput, ValueError,
YamlConstructionError, YamlParserError].} =
let stepsYAML = os_ops.readFile(path/"steps.yaml")
let steps = yaml.loadToJson(stepsYAML)
result = @[]
for step in steps[0]:
var numExtraFields = 0
if step.hasKey"tick":
result.add Operation(kind: opOnTick,
tick: step["tick"].getInt())
elif step.hasKey"attestation":
let filename = step["attestation"].getStr()
let att = parseTest(
path/filename & ".ssz_snappy",
SSZ, phase0.Attestation
)
result.add Operation(kind: opOnAttestation,
att: att)
elif step.hasKey"block":
let filename = step["block"].getStr()
doAssert step.hasKey"blobs" == step.hasKey"proofs"
withConsensusFork(fork):
let
blck = parseTest(
path/filename & ".ssz_snappy",
SSZ, consensusFork.SignedBeaconBlock)
blobData =
when consensusFork >= ConsensusFork.Deneb:
if step.hasKey"blobs":
numExtraFields += 2
Opt.some BlobData(
blobs: distinctBase(parseTest(
path/(step["blobs"].getStr()) & ".ssz_snappy",
SSZ, List[KzgBlob, Limit MAX_BLOBS_PER_BLOCK])),
proofs: step["proofs"].mapIt(
KzgProof(bytes: fromHex(array[48, byte], it.getStr()))))
else:
Opt.none(BlobData)
else:
doAssert not step.hasKey"blobs"
Opt.none(BlobData)
result.add Operation(kind: opOnBlock,
blck: ForkedSignedBeaconBlock.init(blck),
blobData: blobData)
elif step.hasKey"attester_slashing":
let filename = step["attester_slashing"].getStr()
let attesterSlashing = parseTest(
path/filename & ".ssz_snappy",
SSZ, phase0.AttesterSlashing
)
result.add Operation(kind: opOnAttesterSlashing,
attesterSlashing: attesterSlashing)
elif step.hasKey"payload_status":
if step["payload_status"]["status"].getStr() == "INVALID":
result.add Operation(kind: opInvalidateHash,
valid: true,
invalidatedHash: Eth2Digest.fromHex(step["block_hash"].getStr()),
latestValidHash: Eth2Digest.fromHex(
step["payload_status"]["latest_valid_hash"].getStr()))
elif step.hasKey"checks":
result.add Operation(kind: opChecks,
checks: step["checks"])
else:
raiseAssert "Unknown test step: " & $step
if step.hasKey"valid":
doAssert step.len == 2 + numExtraFields
result[^1].valid = step["valid"].getBool()
elif not step.hasKey"checks" and not step.hasKey"payload_status":
doAssert step.len == 1 + numExtraFields
result[^1].valid = true
proc stepOnBlock(
2022-04-14 10:47:14 +00:00
dag: ChainDAGRef,
fkChoice: ref ForkChoice,
verifier: var BatchVerifier,
state: var ForkedHashedBeaconState,
stateCache: var StateCache,
signedBlock: ForkySignedBeaconBlock,
blobData: Opt[BlobData],
time: BeaconTime,
invalidatedHashes: Table[Eth2Digest, Eth2Digest]):
Result[BlockRef, VerifierError] =
# 1. Validate blobs
when typeof(signedBlock).kind >= ConsensusFork.Deneb:
let kzgCommits = signedBlock.message.body.blob_kzg_commitments.asSeq
if kzgCommits.len > 0 or blobData.isSome:
if blobData.isNone or kzgCommits.validate_blobs(
blobData.get.blobs, blobData.get.proofs).isErr:
return err(VerifierError.Invalid)
else:
doAssert blobData.isNone, "Pre-Deneb test with specified blob data"
# 2. Move state to proper slot
doAssert dag.updateState(
state,
Prune `BlockRef` on finalization (#3513) Up til now, the block dag has been using `BlockRef`, a structure adapted for a full DAG, to represent all of chain history. This is a correct and simple design, but does not exploit the linearity of the chain once parts of it finalize. By pruning the in-memory `BlockRef` structure at finalization, we save, at the time of writing, a cool ~250mb (or 25%:ish) chunk of memory landing us at a steady state of ~750mb normal memory usage for a validating node. Above all though, we prevent memory usage from growing proportionally with the length of the chain, something that would not be sustainable over time - instead, the steady state memory usage is roughly determined by the validator set size which grows much more slowly. With these changes, the core should remain sustainable memory-wise post-merge all the way to withdrawals (when the validator set is expected to grow). In-memory indices are still used for the "hot" unfinalized portion of the chain - this ensure that consensus performance remains unchanged. What changes is that for historical access, we use a db-based linear slot index which is cache-and-disk-friendly, keeping the cost for accessing historical data at a similar level as before, achieving the savings at no percievable cost to functionality or performance. A nice collateral benefit is the almost-instant startup since we no longer load any large indicies at dag init. The cost of this functionality instead can be found in the complexity of having to deal with two ways of traversing the chain - by `BlockRef` and by slot. * use `BlockId` instead of `BlockRef` where finalized / historical data may be required * simplify clearance pre-advancement * remove dag.finalizedBlocks (~50:ish mb) * remove `getBlockAtSlot` - use `getBlockIdAtSlot` instead * `parent` and `atSlot` for `BlockId` now require a `ChainDAGRef` instance, unlike `BlockRef` traversal * prune `BlockRef` parents on finality (~200:ish mb) * speed up ChainDAG init by not loading finalized history index * mess up light client server error handling - this need revisiting :)
2022-03-17 17:42:56 +00:00
dag.getBlockIdAtSlot(time.slotOrZero).expect("block exists"),
save = false,
stateCache
)
# 3. Add block to DAG
const consensusFork = typeof(signedBlock).kind
# In normal Nimbus flow, for this (effectively) newPayload-based INVALID, it
# is checked even before entering the DAG, by the block processor. Currently
# the optimistic sync test(s) don't include a later-fcU-INVALID case. Whilst
# this wouldn't be part of this check, presumably, their FC test vector step
# would also have `true` validity because it'd not be known they weren't, so
# adding this mock of the block processor is realistic and sufficient.
when consensusFork >= ConsensusFork.Bellatrix:
let executionBlockHash =
signedBlock.message.body.execution_payload.block_hash
if executionBlockHash in invalidatedHashes:
# Mocks fork choice INVALID list application. These tests sequence this
# in a way the block processor does not, specifying each payload_status
# before the block itself, while Nimbus fork choice treats invalidating
# a non-existent block root as a no-op and does not remember it for the
# future.
let lvh = invalidatedHashes.getOrDefault(
executionBlockHash, static(default(Eth2Digest)))
fkChoice[].mark_root_invalid(dag.getEarliestInvalidBlockRoot(
signedBlock.message.parent_root, lvh, executionBlockHash))
return err VerifierError.Invalid
let blockAdded = dag.addHeadBlock(verifier, signedBlock) do (
blckRef: BlockRef, signedBlock: consensusFork.TrustedSignedBeaconBlock,
epochRef: EpochRef, unrealized: FinalityCheckpoints):
# 4. Update fork choice if valid
let status = fkChoice[].process_block(
dag, epochRef, blckRef, unrealized, signedBlock.message, time)
doAssert status.isOk()
# 5. Update DAG with new head
var quarantine = Quarantine.init()
let newHead = fkChoice[].get_head(dag, time).get()
2023-03-02 16:13:35 +00:00
dag.updateHead(dag.getBlockRef(newHead).get(), quarantine, [])
if dag.needStateCachesAndForkChoicePruning():
dag.pruneStateCachesDAG()
let pruneRes = fkChoice[].prune()
doAssert pruneRes.isOk()
blockAdded
proc stepChecks(
checks: JsonNode,
dag: ChainDAGRef,
fkChoice: ref ForkChoice,
time: BeaconTime) {.raises: [KeyError].} =
doAssert checks.len >= 1, "No checks found"
for check, val in checks:
if check == "time":
doAssert time.ns_since_genesis == val.getInt().seconds.nanoseconds()
doAssert fkChoice.checkpoints.time.slotOrZero == time.slotOrZero
elif check == "head":
let headRoot = fkChoice[].get_head(dag, time).get()
limit by-root requests to non-finalized blocks (#3293) * limit by-root requests to non-finalized blocks Presently, we keep a mapping from block root to `BlockRef` in memory - this has simplified reasoning about the dag, but is not sustainable with the chain growing. We can distinguish between two cases where by-root access is useful: * unfinalized blocks - this is where the beacon chain is operating generally, by validating incoming data as interesting for future fork choice decisions - bounded by the length of the unfinalized period * finalized blocks - historical access in the REST API etc - no bounds, really In this PR, we limit the by-root block index to the first use case: finalized chain data can more efficiently be addressed by slot number. Future work includes: * limiting the `BlockRef` horizon in general - each instance is 40 bytes+overhead which adds up - this needs further refactoring to deal with the tail vs state problem * persisting the finalized slot-to-hash index - this one also keeps growing unbounded (albeit slowly) Anyway, this PR easily shaves ~128mb of memory usage at the time of writing. * No longer honor `BeaconBlocksByRoot` requests outside of the non-finalized period - previously, Nimbus would generously return any block through this libp2p request - per the spec, finalized blocks should be fetched via `BeaconBlocksByRange` instead. * return `Opt[BlockRef]` instead of `nil` when blocks can't be found - this becomes a lot more common now and thus deserves more attention * `dag.blocks` -> `dag.forkBlocks` - this index only carries unfinalized blocks from now - `finalizedBlocks` covers the other `BlockRef` instances * in backfill, verify that the last backfilled block leads back to genesis, or panic * add backfill timings to log * fix missing check that `BlockRef` block can be fetched with `getForkedBlock` reliably * shortcut doppelganger check when feature is not enabled * in REST/JSON-RPC, fetch blocks without involving `BlockRef` * fix dag.blocks ref
2022-01-21 11:33:16 +00:00
let headRef = dag.getBlockRef(headRoot).get()
doAssert headRef.slot == Slot(val["slot"].getInt())
doAssert headRef.root == Eth2Digest.fromHex(val["root"].getStr())
elif check == "justified_checkpoint":
let checkpointRoot = fkChoice.checkpoints.justified.checkpoint.root
let checkpointEpoch = fkChoice.checkpoints.justified.checkpoint.epoch
doAssert checkpointEpoch == Epoch(val["epoch"].getInt())
doAssert checkpointRoot == Eth2Digest.fromHex(val["root"].getStr())
elif check == "justified_checkpoint_root": # undocumented check
let checkpointRoot = fkChoice.checkpoints.justified.checkpoint.root
doAssert checkpointRoot == Eth2Digest.fromHex(val.getStr())
elif check == "finalized_checkpoint":
let checkpointRoot = fkChoice.checkpoints.finalized.root
let checkpointEpoch = fkChoice.checkpoints.finalized.epoch
doAssert checkpointEpoch == Epoch(val["epoch"].getInt())
doAssert checkpointRoot == Eth2Digest.fromHex(val["root"].getStr())
elif check == "best_justified_checkpoint":
let checkpointRoot = fkChoice.checkpoints.best_justified.root
let checkpointEpoch = fkChoice.checkpoints.best_justified.epoch
doAssert checkpointEpoch == Epoch(val["epoch"].getInt())
doAssert checkpointRoot == Eth2Digest.fromHex(val["root"].getStr())
2021-12-01 12:55:42 +00:00
elif check == "proposer_boost_root":
doAssert fkChoice.checkpoints.proposer_boost_root ==
Eth2Digest.fromHex(val.getStr())
elif check == "genesis_time":
# We do not store genesis in fork choice..
discard
else:
raiseAssert "Unsupported check '" & $check & "'"
proc doRunTest(
path: string,
fork: ConsensusFork
) {.raises: [
IOError, KeyError, UnconsumedInput, ValueError,
YamlConstructionError, YamlParserError].} =
let db = BeaconChainDB.new("", inMemory = true)
defer:
db.close()
let
stores = withConsensusFork(fork):
2024-04-07 07:58:11 +00:00
initialLoad(
path, db, consensusFork.BeaconState, consensusFork.BeaconBlock)
rng = HmacDrbgContext.new()
taskpool =
try:
Taskpool.new()
except Exception as exc:
fatal "Failed to initialize Taskpool", exc = exc.msg
fail()
return
var verifier = BatchVerifier.init(rng, taskpool)
let steps = loadOps(path, fork)
var time = stores.fkChoice.checkpoints.time
var invalidatedHashes: Table[Eth2Digest, Eth2Digest]
let state = newClone(stores.dag.headState)
var stateCache = StateCache()
for step in steps:
case step.kind
of opOnTick:
time = BeaconTime(ns_since_genesis: step.tick.seconds.nanoseconds)
let status = stores.fkChoice[].update_time(stores.dag, time)
doAssert status.isOk == step.valid
of opOnAttestation:
let status = stores.fkChoice[].on_attestation(
stores.dag, step.att.data.slot, step.att.data.beacon_block_root,
toSeq(stores.dag.get_attesting_indices(step.att.asTrusted)), time)
doAssert status.isOk == step.valid
of opOnBlock:
withBlck(step.blck):
let status = stepOnBlock(
stores.dag, stores.fkChoice,
verifier, state[], stateCache,
forkyBlck, step.blobData, time, invalidatedHashes)
doAssert status.isOk == step.valid
of opOnAttesterSlashing:
let indices =
check_attester_slashing(state[], step.attesterSlashing, flags = {})
if indices.isOk:
for idx in indices.get:
stores.fkChoice[].process_equivocation(idx)
doAssert indices.isOk == step.valid
of opInvalidateHash:
invalidatedHashes[step.invalidatedHash] = step.latestValidHash
of opChecks:
stepChecks(step.checks, stores.dag, stores.fkChoice, time)
else:
raiseAssert "Unsupported"
proc runTest(suiteName: static[string], path: string, fork: ConsensusFork) =
const SKIP = [
# protoArray can handle blocks in the future gracefully
# spec: https://github.com/ethereum/consensus-specs/blame/v1.1.3/specs/phase0/fork-choice.md#L349
# test: tests/fork_choice/scenarios/no_votes.nim
# "Ensure the head is still 4 whilst the justified epoch is 0."
"on_block_future_block",
# TODO on_merge_block
"too_early_for_merge",
"too_late_for_merge",
"block_lookup_failed",
"all_valid",
# TODO intentional reorgs
"should_override_forkchoice_update__false",
"should_override_forkchoice_update__true",
"basic_is_parent_root",
"basic_is_head_root",
]
test suiteName & " - " & path.relativeTestPathComponent():
when defined(windows):
# Some test files have very long paths
skip()
else:
if os_ops.splitPath(path).tail in SKIP:
skip()
else:
doRunTest(path, fork)
template fcSuite(suiteName: static[string], testPathElem: static[string]) =
suite "EF - " & suiteName & preset():
const presetPath = SszTestsDir/const_preset
for kind, path in walkDir(presetPath, relative = true, checkDir = true):
let testsPath = presetPath/path/testPathElem
if kind != pcDir or not os_ops.dirExists(testsPath):
continue
if testsPath.contains("/electra/") or testsPath.contains("\\electra\\"):
continue
let fork = forkForPathComponent(path).valueOr:
raiseAssert "Unknown test fork: " & testsPath
2024-04-07 07:58:11 +00:00
for kind, path in walkDir(testsPath, relative = true, checkDir = true):
let basePath = testsPath/path/"pyspec_tests"
if kind != pcDir:
continue
for kind, path in walkDir(basePath, relative = true, checkDir = true):
runTest(suiteName, basePath/path, fork)
from ../../beacon_chain/conf import loadKzgTrustedSetup
discard loadKzgTrustedSetup() # Required for Deneb tests
fcSuite("ForkChoice", "fork_choice")
fcSuite("Sync", "sync")