Fork choice fixes (#1350)

* remove cruft

* reenable fork choice and fix several issues

* in addForkChoice_v2, the `.error` field would be accessed even when
Result is ok
* remove workaround for invalid block structure in fork choice
* fix `tmpState` being used recursively in callback, causing state
corruption while processing attestation
* fix block callback being called twice per block
* pass state to callback to avoid unnecessary rewinding

* enable head select, fix another bug

* never use `get` without `isOk`
* log nil blockref in case blockref is nil

* add missing error checking

* use correct epoch when updating attestation message
This commit is contained in:
Jacek Sieka 2020-07-22 11:42:55 +02:00 committed by GitHub
parent 6b77f3dda5
commit f0720faf17
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 185 additions and 212 deletions

View File

@ -27,13 +27,7 @@ proc init*(T: type AttestationPool, blockPool: BlockPool): T =
# probably be removed as a dependency of AttestationPool (or some other
# smart refactoring)
# TODO: Return Value Optimization
# TODO: In tests, on blockpool.init the finalized root
# from the `headState` and `justifiedState` is zero
var forkChoice = initForkChoice(
finalized_block_slot = default(Slot), # This is unnecessary for fork choice but may help external components for example logging/debugging
finalized_block_state_root = default(Eth2Digest), # This is unnecessary for fork choice but may help external components for example logging/debugging
justified_epoch = blockPool.headState.data.data.current_justified_checkpoint.epoch,
finalized_epoch = blockPool.headState.data.data.finalized_checkpoint.epoch,
# We should use the checkpoint, but at genesis the headState finalized checkpoint is 0x0000...0000
@ -63,25 +57,22 @@ proc init*(T: type AttestationPool, blockPool: BlockPool): T =
debug "Preloading fork choice with block",
block_root = shortlog(blck.root),
parent_root = shortlog(blck.parent.root),
justified_epoch = $blockPool.tmpState.data.data.current_justified_checkpoint.epoch,
finalized_epoch = $blockPool.tmpState.data.data.finalized_checkpoint.epoch,
slot = $blck.slot
justified_epoch = blockPool.tmpState.data.data.current_justified_checkpoint.epoch,
finalized_epoch = blockPool.tmpState.data.data.finalized_checkpoint.epoch,
slot = blck.slot
let status = forkChoice.process_block(
block_root = blck.root,
parent_root = blck.parent.root,
justified_epoch = blockPool.tmpState.data.data.current_justified_checkpoint.epoch,
finalized_epoch = blockPool.tmpState.data.data.finalized_checkpoint.epoch,
# Unused in fork choice - i.e. for logging or caching extra metadata
slot = blck.slot,
state_root = default(Eth2Digest)
finalized_epoch = blockPool.tmpState.data.data.finalized_checkpoint.epoch
)
doAssert status.isOk(), "Error in preloading the fork choice: " & $status.error
info "Fork choice initialized",
justified_epoch = $blockPool.headState.data.data.current_justified_checkpoint.epoch,
finalized_epoch = $blockPool.headState.data.data.finalized_checkpoint.epoch,
justified_epoch = blockPool.headState.data.data.current_justified_checkpoint.epoch,
finalized_epoch = blockPool.headState.data.data.finalized_checkpoint.epoch,
finalized_root = shortlog(blockPool.finalizedHead.blck.root)
T(
@ -166,21 +157,20 @@ proc slotIndex(
func updateLatestVotes(
pool: var AttestationPool, state: BeaconState, attestationSlot: Slot,
participants: seq[ValidatorIndex], blck: BlockRef) =
# ForkChoice v2
let target_epoch = compute_epoch_at_slot(attestationSlot)
participants: seq[ValidatorIndex], blck: BlockRef, targetEpoch: Epoch) =
for validator in participants:
# ForkChoice v1
let
pubKey = state.validators[validator].pubkey
current = pool.latestAttestations.getOrDefault(pubKey)
# TODO using attestationSlot here is wrong, it should be target epoch -
# clean this up
if current.isNil or current.slot < attestationSlot:
pool.latestAttestations[pubKey] = blck
# # ForkChoice v2
# pool.forkChoice_v2.process_attestation(validator, blck.root, target_epoch)
# ForkChoice v2
pool.forkChoice_v2.process_attestation(validator, blck.root, targetEpoch)
func addUnresolved(pool: var AttestationPool, attestation: Attestation) =
pool.unresolved[attestation.data.beacon_block_root] =
@ -279,7 +269,9 @@ proc addResolved(pool: var AttestationPool, blck: BlockRef, attestation: Attesta
not it.aggregation_bits.isSubsetOf(validation.aggregation_bits))
a.validations.add(validation)
pool.updateLatestVotes(state, attestationSlot, participants, a.blck)
pool.updateLatestVotes(
state, attestationSlot, participants, a.blck,
attestation.data.target.epoch)
info "Attestation resolved",
attestation = shortLog(attestation),
@ -297,7 +289,8 @@ proc addResolved(pool: var AttestationPool, blck: BlockRef, attestation: Attesta
blck: blck,
validations: @[validation]
))
pool.updateLatestVotes(state, attestationSlot, participants, blck)
pool.updateLatestVotes(
state, attestationSlot, participants, blck, attestation.data.target.epoch)
info "Attestation resolved",
attestation = shortLog(attestation),
@ -320,67 +313,24 @@ proc addAttestation*(pool: var AttestationPool, attestation: Attestation) =
pool.addResolved(blck, attestation)
proc addForkChoice_v2*(pool: var AttestationPool, blck: BlockRef) =
proc addForkChoice_v2*(pool: var AttestationPool, blck: BlockRef,
justified_epoch, finalized_epoch: Epoch) =
## Add a verified block to the fork choice context
## The current justifiedState of the block pool is used as reference
let state = pool.forkChoice_v2.process_block(
block_root = blck.root,
parent_root = if not blck.parent.isNil: blck.parent.root else: default(Eth2Digest),
justified_epoch = justified_epoch,
finalized_epoch = finalized_epoch,
)
# TODO: add(BlockPool, blockRoot: Eth2Digest, SignedBeaconBlock): BlockRef
# should ideally return the justified_epoch and finalized_epoch
# so that we can pass them directly to this proc without having to
# redo "updateStateData"
#
# In any case, `updateStateData` should shortcut
# to `getStateDataCached`
var state: Result[void, string]
# A stack of block to add in case recovery is needed
var blockStack: seq[BlockSlot]
var current = BlockSlot(blck: blck, slot: blck.slot)
while true: # The while loop should not be needed but it seems a block addition
# scenario is unaccounted for
updateStateData(
pool.blockPool,
pool.blockPool.tmpState,
current
)
let blockData = pool.blockPool.get(current.blck)
state = pool.forkChoice_v2.process_block(
slot = current.blck.slot,
block_root = current.blck.root,
parent_root = if not current.blck.parent.isNil: current.blck.parent.root else: default(Eth2Digest),
state_root = default(Eth2Digest), # This is unnecessary for fork choice but may help external components
justified_epoch = pool.blockPool.tmpState.data.data.current_justified_checkpoint.epoch,
finalized_epoch = pool.blockPool.tmpState.data.data.finalized_checkpoint.epoch,
)
# This should not happen and might lead to unresponsive networking while processing occurs
if state.isErr:
# TODO investigate, potential sources:
# - Pruning
# - Quarantine adding multiple blocks at once
# - Own block proposal
error "Desync between fork_choice and blockpool services, trying to recover.",
msg = state.error,
blck = shortlog(current.blck),
parent = shortlog(current.blck.parent),
finalizedHead = shortLog(pool.blockPool.finalizedHead),
justifiedHead = shortLog(pool.blockPool.head.justified),
head = shortLog(pool.blockPool.head.blck)
blockStack.add(current)
current = BlockSlot(blck: blck.parent, slot: blck.parent.slot)
elif blockStack.len == 0:
break
else:
info "Re-added missing or pruned block to fork choice",
msg = state.error,
blck = shortlog(current.blck),
parent = shortlog(current.blck.parent),
finalizedHead = shortLog(pool.blockPool.finalizedHead),
justifiedHead = shortLog(pool.blockPool.head.justified),
head = shortLog(pool.blockPool.head.blck)
current = blockStack.pop()
if state.isErr:
# TODO If this happens, it is effectively a bug - the BlockRef structure
# guarantees that the DAG is valid and the state transition should
# guarantee that the justified and finalized epochs are ok! However,
# we'll log it for now to avoid crashes
error "Unexpected error when applying block",
blck = shortLog(blck), justified_epoch, finalized_epoch, err = state.error
proc getAttestationsForSlot*(pool: AttestationPool, newBlockSlot: Slot):
Option[AttestationsSeen] =
@ -607,23 +557,28 @@ proc selectHead_v2(pool: var AttestationPool): BlockRef =
justified_root = pool.blockPool.head.justified.blck.root,
finalized_epoch = pool.blockPool.headState.data.data.finalized_checkpoint.epoch,
justified_state_balances = attesterBalances
).get()
)
pool.blockPool.getRef(newHead)
if newHead.isErr:
error "Couldn't select head", err = newHead.error
nil
else:
pool.blockPool.getRef(newHead.get())
proc pruneBefore*(pool: var AttestationPool, finalizedhead: BlockSlot) =
pool.forkChoice_v2.maybe_prune(finalizedHead.blck.root).get()
if (let v = pool.forkChoice_v2.maybe_prune(finalizedHead.blck.root); v.isErr):
error "Pruning failed", err = v.error() # TODO should never happen
# Dual-Headed Fork choice
# ---------------------------------------------------------------
proc selectHead*(pool: var AttestationPool): BlockRef =
let head_v1 = pool.selectHead_v1()
# let head_v2 = pool.selectHead_v2()
#
# if head_v1 != head_v2:
# error "Fork choice engines in disagreement, using block from v1.",
# v1_block = shortlog(head_v1),
# v2_block = shortlog(head_v2)
let head_v2 = pool.selectHead_v2()
if head_v1 != head_v2:
error "Fork choice engines in disagreement, using block from v1.",
v1_block = shortLog(head_v1),
v2_block = shortLog(head_v2)
return head_v1

View File

@ -321,10 +321,13 @@ proc storeBlock(
beacon_blocks_received.inc()
{.gcsafe.}: # TODO: fork choice and blockpool should sync via messages instead of callbacks
let blck = node.blockPool.addRawBlock(signedBlock) do (validBlock: BlockRef):
let blck = node.blockPool.addRawBlock(signedBlock) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
state: HashedBeaconState):
# Callback add to fork choice if valid
# node.attestationPool.addForkChoice_v2(validBlock)
discard "TODO: Deactivated"
node.attestationPool.addForkChoice_v2(
blckRef, state.data.current_justified_checkpoint.epoch,
state.data.finalized_checkpoint.epoch)
node.dumpBlock(signedBlock, blck)

View File

@ -54,7 +54,7 @@ template finalizedHead*(pool: BlockPool): BlockSlot =
pool.dag.finalizedHead
proc addRawBlock*(pool: var BlockPool, signedBlock: SignedBeaconBlock,
callback: proc(blck: BlockRef)
onBlockAdded: OnBlockAdded
): Result[BlockRef, BlockError] =
## Add a raw block to the blockpool
## Trigger "callback" on success
@ -64,7 +64,7 @@ proc addRawBlock*(pool: var BlockPool, signedBlock: SignedBeaconBlock,
# - the ugly `inAdd` field
# - the callback
# - callback may be problematic as it's called in async validator duties
result = addRawBlock(pool.dag, pool.quarantine, signedBlock, callback)
result = addRawBlock(pool.dag, pool.quarantine, signedBlock, onBlockAdded)
export parent # func parent*(bs: BlockSlot): BlockSlot
export isAncestorOf # func isAncestorOf*(a, b: BlockRef): bool

View File

@ -188,6 +188,10 @@ type
blck*: BlockRef
justified*: BlockSlot
OnBlockAdded* = proc(
blckRef: BlockRef, blck: SignedBeaconBlock,
state: HashedBeaconState) {.raises: [Defect], gcsafe.}
proc shortLog*(v: BlockSlot): string =
if v.blck.slot == v.slot:
v.blck.root.data[0..3].toHex() & ":" & $v.blck.slot
@ -195,7 +199,10 @@ proc shortLog*(v: BlockSlot): string =
v.blck.root.data[0..3].toHex() & ":" & $v.blck.slot & "@" & $v.slot
proc shortLog*(v: BlockRef): string =
v.root.data[0..3].toHex() & ":" & $v.slot
if v == nil:
"BlockRef(nil)"
else:
v.root.data[0..3].toHex() & ":" & $v.slot
chronicles.formatIt BlockSlot: shortLog(it)
chronicles.formatIt BlockRef: shortLog(it)

View File

@ -36,13 +36,13 @@ func getOrResolve*(dag: CandidateChains, quarantine: var Quarantine, root: Eth2D
proc addRawBlock*(
dag: var CandidateChains, quarantine: var Quarantine,
signedBlock: SignedBeaconBlock, callback: proc(blck: BlockRef)
signedBlock: SignedBeaconBlock, onBlockAdded: OnBlockAdded
): Result[BlockRef, BlockError]
proc addResolvedBlock(
dag: var CandidateChains, quarantine: var Quarantine,
state: BeaconState, signedBlock: SignedBeaconBlock, parent: BlockRef,
callback: proc(blck: BlockRef)
state: HashedBeaconState, signedBlock: SignedBeaconBlock, parent: BlockRef,
onBlockAdded: OnBlockAdded
): BlockRef =
# TODO: `addResolvedBlock` is accumulating significant cruft
# and is in dire need of refactoring
@ -50,13 +50,13 @@ proc addResolvedBlock(
# - the callback
# - callback may be problematic as it's called in async validator duties
logScope: pcs = "block_resolution"
doAssert state.slot == signedBlock.message.slot, "state must match block"
doAssert state.data.slot == signedBlock.message.slot, "state must match block"
let
blockRoot = signedBlock.root
blockRef = BlockRef.init(blockRoot, signedBlock.message)
blockRef.epochsInfo = filterIt(parent.epochsInfo,
it.epoch + 1 >= state.slot.compute_epoch_at_slot)
it.epoch + 1 >= state.data.slot.compute_epoch_at_slot)
link(parent, blockRef)
dag.blocks[blockRoot] = blockRef
@ -68,7 +68,7 @@ proc addResolvedBlock(
# This block *might* have caused a justification - make sure we stow away
# that information:
let justifiedSlot =
state.current_justified_checkpoint.epoch.compute_start_slot_at_epoch()
state.data.current_justified_checkpoint.epoch.compute_start_slot_at_epoch()
var foundHead: Option[Head]
for head in dag.heads.mitems():
@ -94,7 +94,8 @@ proc addResolvedBlock(
heads = dag.heads.len()
# This MUST be added before the quarantine
callback(blockRef)
if onBlockAdded != nil:
onBlockAdded(blockRef, signedBlock, state)
# Now that we have the new block, we should see if any of the previously
# unresolved blocks magically become resolved
@ -110,7 +111,7 @@ proc addResolvedBlock(
while keepGoing:
let retries = quarantine.orphans
for _, v in retries:
discard addRawBlock(dag, quarantine, v, callback)
discard addRawBlock(dag, quarantine, v, onBlockAdded)
# Keep going for as long as the pending dag is shrinking
# TODO inefficient! so what?
keepGoing = quarantine.orphans.len < retries.len
@ -120,7 +121,7 @@ proc addResolvedBlock(
proc addRawBlock*(
dag: var CandidateChains, quarantine: var Quarantine,
signedBlock: SignedBeaconBlock,
callback: proc(blck: BlockRef)
onBlockAdded: OnBlockAdded
): Result[BlockRef, BlockError] =
## return the block, if resolved...
@ -213,16 +214,16 @@ proc addRawBlock*(
notice "Invalid block"
return err Invalid
# Careful, clearanceState.data has been updated but not blck - we need to create
# the BlockRef first!
# Careful, clearanceState.data has been updated but not blck - we need to
# create the BlockRef first!
dag.clearanceState.blck = addResolvedBlock(
dag, quarantine,
dag.clearanceState.data.data, signedBlock, parent,
callback
dag, quarantine, dag.clearanceState.data, signedBlock, parent,
onBlockAdded
)
dag.putState(dag.clearanceState.data, dag.clearanceState.blck)
callback(dag.clearanceState.blck)
return ok dag.clearanceState.blck
# TODO already checked hash though? main reason to keep this is because

View File

@ -51,8 +51,6 @@ logScope:
# - The public procs use Result
func initForkChoice*(
finalized_block_slot: Slot, # This is unnecessary for fork choice but helps external components
finalized_block_state_root: Eth2Digest, # This is unnecessary for fork choice but helps external components
justified_epoch: Epoch,
finalized_epoch: Epoch,
finalized_root: Eth2Digest
@ -65,11 +63,9 @@ func initForkChoice*(
)
let err = proto_array.on_block(
finalized_block_slot,
finalized_root,
hasParentInForkChoice = false,
default(Eth2Digest),
finalized_block_state_root,
justified_epoch,
finalized_epoch
)
@ -97,7 +93,6 @@ func extend[T](s: var seq[T], minLen: int) =
s.setLen(minLen)
zeroMem(s[curLen].addr, diff * sizeof(T))
func process_attestation*(
self: var ForkChoice,
validator_index: ValidatorIndex,
@ -117,22 +112,22 @@ func process_attestation*(
{.noSideEffect.}:
trace "Integrating vote in fork choice",
validator_index = $validator_index,
validator_index = validator_index,
new_vote = shortlog(vote)
else:
{.noSideEffect.}:
if vote.next_epoch != target_epoch or vote.next_root != block_root:
warn "Change of vote ignored for fork choice. Potential for slashing.",
validator_index = $validator_index,
validator_index = validator_index,
current_vote = shortlog(vote),
ignored_block_root = shortlog(block_root),
ignored_target_epoch = $target_epoch
ignored_target_epoch = target_epoch
else:
trace "Ignoring double-vote for fork choice",
validator_index = $validator_index,
validator_index = validator_index,
current_vote = shortlog(vote),
ignored_block_root = shortlog(block_root),
ignored_target_epoch = $target_epoch
ignored_target_epoch = target_epoch
func contains*(self: ForkChoice, block_root: Eth2Digest): bool =
## Returns `true` if a block is known to the fork choice
@ -143,26 +138,24 @@ func contains*(self: ForkChoice, block_root: Eth2Digest): bool =
func process_block*(
self: var ForkChoice,
slot: Slot,
block_root: Eth2Digest,
parent_root: Eth2Digest,
state_root: Eth2Digest,
justified_epoch: Epoch,
finalized_epoch: Epoch
): Result[void, string] =
## Add a block to the fork choice context
let err = self.proto_array.on_block(
slot, block_root, hasParentInForkChoice = true, parent_root, state_root, justified_epoch, finalized_epoch
block_root, hasParentInForkChoice = true, parent_root, justified_epoch, finalized_epoch
)
if err.kind != fcSuccess:
return err("process_block_error: " & $err)
{.noSideEffect.}:
trace "Integrating block in fork choice",
block_root = $shortlog(block_root),
parent_root = $shortlog(parent_root),
justified_epoch = $justified_epoch,
finalized_epoch = $finalized_epoch
block_root = shortlog(block_root),
parent_root = shortlog(parent_root),
justified_epoch = justified_epoch,
finalized_epoch = finalized_epoch
return ok()
@ -205,9 +198,9 @@ func find_head*(
{.noSideEffect.}:
debug "Fork choice requested",
justified_epoch = $justified_epoch,
justified_epoch = justified_epoch,
justified_root = shortlog(justified_root),
finalized_epoch = $finalized_epoch,
finalized_epoch = finalized_epoch,
fork_choice_head = shortlog(new_head)
return ok(new_head)

View File

@ -104,10 +104,6 @@ type
indices*: Table[Eth2Digest, Index]
ProtoNode* = object
# TODO: generic "Metadata" field for slot/state_root
slot*: Slot # This is unnecessary for fork choice but helps external components
state_root*: Eth2Digest # This is unnecessary for fork choice but helps external components
# Fields used in fork choice
root*: Eth2Digest
parent*: Option[Index]
justified_epoch*: Epoch

View File

@ -156,11 +156,9 @@ func apply_score_changes*(
func on_block*(
self: var ProtoArray,
slot: Slot,
root: Eth2Digest,
hasParentInForkChoice: bool,
parent: Eth2Digest,
state_root: Eth2Digest,
justified_epoch: Epoch,
finalized_epoch: Epoch
): ForkChoiceError =
@ -185,9 +183,8 @@ func on_block*(
error "Trying to add block with unknown parent",
child_root = shortLog(root),
parent_root = shortLog(parent),
justified_epoch = $justified_epoch,
finalized_epoch = $finalized_epoch,
slot_optional = $slot
justified_epoch = justified_epoch,
finalized_epoch = finalized_epoch
return ForkChoiceError(
kind: fcErrUnknownParent,
@ -200,8 +197,6 @@ func on_block*(
let node_index = self.nodes.len
let node = ProtoNode(
slot: slot,
state_root: state_root,
root: root,
parent: parent_index,
justified_epoch: justified_epoch,

View File

@ -224,10 +224,13 @@ proc proposeSignedBlock*(node: BeaconNode,
newBlock: SignedBeaconBlock): Future[BlockRef] {.async.} =
{.gcsafe.}: # TODO: fork choice and blockpool should sync via messages instead of callbacks
let newBlockRef = node.blockPool.addRawBlock(newBlock) do (validBlock: BlockRef):
# Callback Add to fork choice
# node.attestationPool.addForkChoice_v2(validBlock)
discard "TODO: Deactivated"
let newBlockRef = node.blockPool.addRawBlock(newBlock) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
state: HashedBeaconState):
# Callback add to fork choice if valid
node.attestationPool.addForkChoice_v2(
blckRef, state.data.current_justified_checkpoint.epoch,
state.data.finalized_checkpoint.epoch)
if newBlockRef.isErr:
warn "Unable to add proposed block to block pool",

View File

@ -134,9 +134,14 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
state.fork, state.genesis_validators_root, newBlock.message.slot,
blockRoot, privKey)
let added = blockPool.addRawBlock(newBlock) do (validBlock: BlockRef):
# Callback Add to fork choice
attPool.addForkChoice_v2(validBlock)
let added = blockPool.addRawBlock(newBlock) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
state: HashedBeaconState):
# Callback add to fork choice if valid
attPool.addForkChoice_v2(
blckRef, state.data.current_justified_checkpoint.epoch,
state.data.finalized_checkpoint.epoch)
blck() = added[]
blockPool.updateHead(added[])

View File

@ -82,10 +82,8 @@ func apply(ctx: var ForkChoice, id: int, op: Operation) =
debugEcho " Detected an expected invalid head"
of ProcessBlock:
let r = ctx.process_block(
slot = default(Slot), # unused in fork choice, only helpful for external components
block_root = op.root,
parent_root = op.parent_root,
state_root = default(Eth2Digest), # unused in fork choice, only helpful for external components
justified_epoch = op.blk_justified_epoch,
finalized_epoch = op.blk_finalized_epoch
)

View File

@ -13,8 +13,6 @@ proc setup_finality_01(): tuple[fork_choice: ForkChoice, ops: seq[Operation]] =
# Initialize the fork choice context
result.fork_choice = initForkChoice(
finalized_block_slot = Slot(0), # Metadata unused in fork choice
finalized_block_state_root = default(Eth2Digest), # Metadata unused in fork choice
justified_epoch = Epoch(1),
finalized_epoch = Epoch(1),
finalized_root = GenesisRoot

View File

@ -13,8 +13,6 @@ proc setup_finality_02(): tuple[fork_choice: ForkChoice, ops: seq[Operation]] =
# Initialize the fork choice context
result.fork_choice = initForkChoice(
finalized_block_slot = Slot(0), # Metadata unused in fork choice
finalized_block_state_root = default(Eth2Digest), # Metadata unused in fork choice
justified_epoch = Epoch(1),
finalized_epoch = Epoch(1),
finalized_root = GenesisRoot

View File

@ -13,8 +13,6 @@ proc setup_no_votes(): tuple[fork_choice: ForkChoice, ops: seq[Operation]] =
# Initialize the fork choice context
result.fork_choice = initForkChoice(
finalized_block_slot = Slot(0), # Metadata unused in fork choice
finalized_block_state_root = default(Eth2Digest), # Metadata unused in fork choice
justified_epoch = Epoch(1),
finalized_epoch = Epoch(1),
finalized_root = GenesisRoot

View File

@ -13,8 +13,6 @@ proc setup_votes(): tuple[fork_choice: ForkChoice, ops: seq[Operation]] =
# Initialize the fork choice context
result.fork_choice = initForkChoice(
finalized_block_slot = Slot(0), # Metadata unused in fork choice
finalized_block_state_root = default(Eth2Digest), # Metadata unused in fork choice
justified_epoch = Epoch(1),
finalized_epoch = Epoch(1),
finalized_root = GenesisRoot

View File

@ -166,9 +166,14 @@ suiteReport "Attestation pool processing" & preset():
var cache = StateCache()
let
b1 = addTestBlock(state.data, blockPool[].tail.root, cache)
b1Add = blockpool[].addRawBlock(b1) do (validBlock: BlockRef):
# Callback Add to fork choice
pool[].addForkChoice_v2(validBlock)
b1Add = blockpool[].addRawBlock(b1) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
state: HashedBeaconState):
# Callback add to fork choice if valid
pool[].addForkChoice_v2(
blckRef, state.data.current_justified_checkpoint.epoch,
state.data.finalized_checkpoint.epoch)
let head = pool[].selectHead()
@ -177,9 +182,13 @@ suiteReport "Attestation pool processing" & preset():
let
b2 = addTestBlock(state.data, b1.root, cache)
b2Add = blockpool[].addRawBlock(b2) do (validBlock: BlockRef):
# Callback Add to fork choice
pool[].addForkChoice_v2(validBlock)
b2Add = blockpool[].addRawBlock(b2) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
state: HashedBeaconState):
# Callback add to fork choice if valid
pool[].addForkChoice_v2(
blckRef, state.data.current_justified_checkpoint.epoch,
state.data.finalized_checkpoint.epoch)
let head2 = pool[].selectHead()
@ -190,9 +199,13 @@ suiteReport "Attestation pool processing" & preset():
var cache = StateCache()
let
b10 = makeTestBlock(state.data, blockPool[].tail.root, cache)
b10Add = blockpool[].addRawBlock(b10) do (validBlock: BlockRef):
# Callback Add to fork choice
pool[].addForkChoice_v2(validBlock)
b10Add = blockpool[].addRawBlock(b10) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
state: HashedBeaconState):
# Callback add to fork choice if valid
pool[].addForkChoice_v2(
blckRef, state.data.current_justified_checkpoint.epoch,
state.data.finalized_checkpoint.epoch)
let head = pool[].selectHead()
@ -203,9 +216,13 @@ suiteReport "Attestation pool processing" & preset():
b11 = makeTestBlock(state.data, blockPool[].tail.root, cache,
graffiti = GraffitiBytes [1'u8, 0, 0, 0 ,0 ,0 ,0 ,0 ,0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0]
)
b11Add = blockpool[].addRawBlock(b11) do (validBlock: BlockRef):
# Callback Add to fork choice
pool[].addForkChoice_v2(validBlock)
b11Add = blockpool[].addRawBlock(b11) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
state: HashedBeaconState):
# Callback add to fork choice if valid
pool[].addForkChoice_v2(
blckRef, state.data.current_justified_checkpoint.epoch,
state.data.finalized_checkpoint.epoch)
bc1 = get_beacon_committee(
state.data.data, state.data.data.slot, 1.CommitteeIndex, cache)
@ -247,9 +264,13 @@ suiteReport "Attestation pool processing" & preset():
var cache = StateCache()
let
b10 = makeTestBlock(state.data, blockPool[].tail.root, cache)
b10Add = blockpool[].addRawBlock(b10) do (validBlock: BlockRef):
# Callback Add to fork choice
pool[].addForkChoice_v2(validBlock)
b10Add = blockpool[].addRawBlock(b10) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
state: HashedBeaconState):
# Callback add to fork choice if valid
pool[].addForkChoice_v2(
blckRef, state.data.current_justified_checkpoint.epoch,
state.data.finalized_checkpoint.epoch)
let head = pool[].selectHead()
@ -259,9 +280,14 @@ suiteReport "Attestation pool processing" & preset():
# -------------------------------------------------------------
# Add back the old block to ensure we have a duplicate error
let b10_clone = b10 # Assumes deep copy
let b10Add_clone = blockpool[].addRawBlock(b10_clone) do (validBlock: BlockRef):
# Callback Add to fork choice
pool[].addForkChoice_v2(validBlock)
let b10Add_clone = blockpool[].addRawBlock(b10_clone) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
state: HashedBeaconState):
# Callback add to fork choice if valid
pool[].addForkChoice_v2(
blckRef, state.data.current_justified_checkpoint.epoch,
state.data.finalized_checkpoint.epoch)
doAssert: b10Add_clone.error == Duplicate
wrappedTimedTest "Trying to add a duplicate block from an old pruned epoch is tagged as an error":
@ -272,9 +298,13 @@ suiteReport "Attestation pool processing" & preset():
let
b10 = makeTestBlock(state.data, blockPool[].tail.root, cache)
b10Add = blockpool[].addRawBlock(b10) do (validBlock: BlockRef):
# Callback Add to fork choice
pool[].addForkChoice_v2(validBlock)
b10Add = blockpool[].addRawBlock(b10) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
state: HashedBeaconState):
# Callback add to fork choice if valid
pool[].addForkChoice_v2(
blckRef, state.data.current_justified_checkpoint.epoch,
state.data.finalized_checkpoint.epoch)
let head = pool[].selectHead()
@ -301,9 +331,13 @@ suiteReport "Attestation pool processing" & preset():
doAssert: block_ok
block_root = new_block.root
let blockRef = blockpool[].addRawBlock(new_block) do (validBlock: BlockRef):
# Callback Add to fork choice
pool[].addForkChoice_v2(validBlock)
let blockRef = blockpool[].addRawBlock(new_block) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
state: HashedBeaconState):
# Callback add to fork choice if valid
pool[].addForkChoice_v2(
blckRef, state.data.current_justified_checkpoint.epoch,
state.data.finalized_checkpoint.epoch)
let head = pool[].selectHead()
doassert: head == blockRef[]
@ -334,14 +368,18 @@ suiteReport "Attestation pool processing" & preset():
# -------------------------------------------------------------
# Prune
echo "\nPruning all blocks before: ", shortlog(blockPool[].finalizedHead), '\n'
doAssert: blockPool[].finalizedHead.slot != 0
pool[].pruneBefore(blockPool[].finalizedHead)
doAssert: b10.root notin pool.forkChoice_v2
# Add back the old block to ensure we have a duplicate error
let b10Add_clone = blockpool[].addRawBlock(b10_clone) do (validBlock: BlockRef):
# Callback Add to fork choice
pool[].addForkChoice_v2(validBlock)
let b10Add_clone = blockpool[].addRawBlock(b10_clone) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
state: HashedBeaconState):
# Callback add to fork choice if valid
pool[].addForkChoice_v2(
blckRef, state.data.current_justified_checkpoint.epoch,
state.data.finalized_checkpoint.epoch)
doAssert: b10Add_clone.error == Duplicate

View File

@ -110,8 +110,7 @@ suiteReport "Block pool processing" & preset():
timedTest "Simple block add&get" & preset():
let
b1Add = pool.addRawBlock(b1) do (validBlock: BlockRef):
discard
b1Add = pool.addRawBlock(b1, nil)
b1Get = pool.get(b1.root)
check:
@ -122,8 +121,7 @@ suiteReport "Block pool processing" & preset():
pool.heads[0].blck == b1Add[]
let
b2Add = pool.addRawBlock(b2) do (validBlock: BlockRef):
discard
b2Add = pool.addRawBlock(b2, nil)
b2Get = pool.get(b2.root)
check:
@ -139,8 +137,7 @@ suiteReport "Block pool processing" & preset():
let
b4 = addTestBlock(stateData.data, b2.root, cache)
b4Add = pool.addRawBlock(b4) do (validBlock: BlockRef):
discard
b4Add = pool.addRawBlock(b4, nil)
check:
b4Add[].parent == b2Add[]
@ -176,16 +173,14 @@ suiteReport "Block pool processing" & preset():
blocks[0..<2] == [BlockRef nil, nil] # block 3 is missing!
timedTest "Reverse order block add & get" & preset():
let missing = pool.addRawBlock(b2) do (validBlock: BLockRef):
discard
let missing = pool.addRawBlock(b2, nil)
check: missing.error == MissingParent
check:
pool.get(b2.root).isNone() # Unresolved, shouldn't show up
FetchRecord(root: b1.root) in pool.checkMissing()
let status = pool.addRawBlock(b1) do (validBlock: BlockRef):
discard
let status = pool.addRawBlock(b1, nil)
check: status.isOk
@ -222,10 +217,8 @@ suiteReport "Block pool processing" & preset():
timedTest "Adding the same block twice returns a Duplicate error" & preset():
let
b10 = pool.addRawBlock(b1) do (validBlock: BlockRef):
discard
b11 = pool.addRawBlock(b1) do (validBlock: BlockRef):
discard
b10 = pool.addRawBlock(b1, nil)
b11 = pool.addRawBlock(b1, nil)
check:
b11.error == Duplicate
@ -233,8 +226,7 @@ suiteReport "Block pool processing" & preset():
timedTest "updateHead updates head and headState" & preset():
let
b1Add = pool.addRawBlock(b1) do (validBlock: BlockRef):
discard
b1Add = pool.addRawBlock(b1, nil)
pool.updateHead(b1Add[])
@ -244,10 +236,8 @@ suiteReport "Block pool processing" & preset():
timedTest "updateStateData sanity" & preset():
let
b1Add = pool.addRawBlock(b1) do (validBlock: BlockRef):
discard
b2Add = pool.addRawBlock(b2) do (validBlock: BlockRef):
discard
b1Add = pool.addRawBlock(b1, nil)
b2Add = pool.addRawBlock(b2, nil)
bs1 = BlockSlot(blck: b1Add[], slot: b1.message.slot)
bs1_3 = b1Add[].atSlot(3.Slot)
bs2_3 = b2Add[].atSlot(3.Slot)
@ -310,8 +300,7 @@ suiteReport "BlockPool finalization tests" & preset():
let lateBlock = makeTestBlock(tmpState[], pool.head.blck.root, cache)
block:
let status = pool.addRawBlock(blck) do (validBlock: BlockRef):
discard
let status = pool.addRawBlock(blck, nil)
check: status.isOk()
@ -327,8 +316,7 @@ suiteReport "BlockPool finalization tests" & preset():
attestations = makeFullAttestations(
pool.headState.data.data, pool.head.blck.root,
pool.headState.data.data.slot, cache, {}))
let added = pool.addRawBlock(blck) do (validBlock: BlockRef):
discard
let added = pool.addRawBlock(blck, nil)
check: added.isOk()
pool.updateHead(added[])
@ -340,8 +328,7 @@ suiteReport "BlockPool finalization tests" & preset():
block:
# The late block is a block whose parent was finalized long ago and thus
# is no longer a viable head candidate
let status = pool.addRawBlock(lateBlock) do (validBlock: BlockRef):
discard
let status = pool.addRawBlock(lateBlock, nil)
check: status.error == Unviable
let