Use fork v2 (#1358)

* fork choice fixes, round 3

* introduce checkpoint tracker
* split out fork choice backend that is independent of dag
* correctly update best checkpoint to use for head selection
* correctly consider wall clock when processing attestations
* preload head history only (only one history is loaded from database
anyway)
* love the DAG

* switch to fork choice v2

also remove BlockRef.children

* fix
This commit is contained in:
Jacek Sieka 2020-07-25 21:41:12 +02:00 committed by GitHub
parent c47532f2b0
commit fd4d319450
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
19 changed files with 379 additions and 373 deletions

View File

@ -175,7 +175,7 @@ proc isValidAttestation*(
attestation.data.slot, attestation.data.index.CommitteeIndex)
if requiredSubnetIndex != topicCommitteeIndex:
debug "isValidAttestation: attestation's committee index not for the correct subnet",
debug "attestation's committee index not for the correct subnet",
topicCommitteeIndex = topicCommitteeIndex,
attestation_data_index = attestation.data.index,
requiredSubnetIndex = requiredSubnetIndex
@ -199,6 +199,9 @@ proc isValidAggregatedAttestation*(
aggregate_and_proof = signedAggregateAndProof.message
aggregate = aggregate_and_proof.aggregate
logScope:
aggregate = shortLog(aggregate)
# There's some overlap between this and isValidAttestation(), but unclear if
# saving a few lines of code would balance well with losing straightforward,
# spec-based synchronization.
@ -209,7 +212,7 @@ proc isValidAggregatedAttestation*(
# ATTESTATION_PROPAGATION_SLOT_RANGE >= current_slot >= aggregate.data.slot
if not (aggregate.data.slot + ATTESTATION_PROPAGATION_SLOT_RANGE >=
current_slot and current_slot >= aggregate.data.slot):
debug "isValidAggregatedAttestation: aggregation.data.slot not within ATTESTATION_PROPAGATION_SLOT_RANGE"
debug "aggregation.data.slot not within ATTESTATION_PROPAGATION_SLOT_RANGE"
return false
# [IGNORE] The valid aggregate attestation defined by
@ -231,7 +234,7 @@ proc isValidAggregatedAttestation*(
# passes validation.
let attestationBlck = pool.blockPool.getRef(aggregate.data.beacon_block_root)
if attestationBlck.isNil:
debug "isValidAggregatedAttestation: block doesn't exist in block pool"
debug "Block not found"
pool.blockPool.addMissing(aggregate.data.beacon_block_root)
return false
@ -249,7 +252,7 @@ proc isValidAggregatedAttestation*(
# But (2) would reflect an invalid aggregation in other ways, so reject it
# either way.
if isZeros(aggregate.aggregation_bits):
debug "isValidAggregatedAttestation: attestation has no or invalid aggregation bits"
debug "Attestation has no or invalid aggregation bits"
return false
if not isValidAttestationSlot(pool, aggregate.data.slot, attestationBlck):
@ -267,7 +270,7 @@ proc isValidAggregatedAttestation*(
if not is_aggregator(
state, aggregate.data.slot, aggregate.data.index.CommitteeIndex,
aggregate_and_proof.selection_proof, cache):
debug "isValidAggregatedAttestation: incorrect aggregator"
debug "Incorrect aggregator"
return false
# [REJECT] The aggregator's validator index is within the committee -- i.e.
@ -276,7 +279,7 @@ proc isValidAggregatedAttestation*(
if aggregate_and_proof.aggregator_index.ValidatorIndex notin
get_beacon_committee(
state, aggregate.data.slot, aggregate.data.index.CommitteeIndex, cache):
debug "isValidAggregatedAttestation: aggregator's validator index not in committee"
debug "Aggregator's validator index not in committee"
return false
# [REJECT] The aggregate_and_proof.selection_proof is a valid signature of the
@ -284,14 +287,14 @@ proc isValidAggregatedAttestation*(
# aggregate_and_proof.aggregator_index.
# get_slot_signature(state, aggregate.data.slot, privkey)
if aggregate_and_proof.aggregator_index >= state.validators.len.uint64:
debug "isValidAggregatedAttestation: invalid aggregator_index"
debug "Invalid aggregator_index"
return false
if not verify_slot_signature(
state.fork, state.genesis_validators_root, aggregate.data.slot,
state.validators[aggregate_and_proof.aggregator_index].pubkey,
aggregate_and_proof.selection_proof):
debug "isValidAggregatedAttestation: selection_proof signature verification failed"
debug "Selection_proof signature verification failed"
return false
# [REJECT] The aggregator signature, signed_aggregate_and_proof.signature, is valid.
@ -299,14 +302,13 @@ proc isValidAggregatedAttestation*(
state.fork, state.genesis_validators_root, aggregate_and_proof,
state.validators[aggregate_and_proof.aggregator_index].pubkey,
signed_aggregate_and_proof.signature):
debug "isValidAggregatedAttestation: signed_aggregate_and_proof signature verification failed"
debug "Signed_aggregate_and_proof signature verification failed"
return false
# [REJECT] The signature of aggregate is valid.
if not is_valid_indexed_attestation(
state, get_indexed_attestation(state, aggregate, cache), {}):
debug "isValidAggregatedAttestation: aggregate signature verification failed"
debug "Aggregate signature verification failed"
return false
debug "isValidAggregatedAttestation: succeeded"
true

View File

@ -9,7 +9,7 @@
import
# Standard libraries
deques, sequtils, tables, options,
std/[algorithm, deques, sequtils, tables, options],
# Status libraries
chronicles, stew/[byteutils], json_serialization/std/sets,
# Internal
@ -27,48 +27,32 @@ proc init*(T: type AttestationPool, blockPool: BlockPool): T =
# probably be removed as a dependency of AttestationPool (or some other
# smart refactoring)
var forkChoice = initForkChoice(
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
# finalized_root = blockPool.headState.data.data.finalized_checkpoint.root
finalized_root = blockPool.finalizedHead.blck.root
).get()
# Load all blocks since finalized head - TODO a proper test
for blck in blockPool.dag.topoSortedSinceLastFinalization():
if blck.root == blockPool.finalizedHead.blck.root:
continue
# BlockRef
# should ideally contain the justified_epoch and finalized_epoch
# so that we can pass them directly to `process_block` without having to
# redo "updateStateData"
#
# In any case, `updateStateData` should shortcut
# to `getStateDataCached`
updateStateData(
blockPool,
blockPool.withState(blockPool.tmpState, blockPool.finalizedHead):
var forkChoice = initForkChoice(
blockPool.tmpState,
BlockSlot(blck: blck, slot: blck.slot)
)
).get()
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
# Feed fork choice with unfinalized history
var blocks: seq[BlockRef]
var cur = blockPool.head.blck
while cur != blockPool.finalizedHead.blck:
blocks.add cur
cur = cur.parent
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
)
for blck in reversed(blocks):
blockPool.withState(blockPool.tmpState, blck.atSlot(blck.slot)):
debug "Preloading fork choice with block",
block_root = shortlog(blck.root),
parent_root = shortlog(blck.parent.root),
justified_epoch = state.current_justified_checkpoint.epoch,
finalized_epoch = state.finalized_checkpoint.epoch,
slot = blck.slot
doAssert status.isOk(), "Error in preloading the fork choice: " & $status.error
let status =
forkChoice.process_block(
blockPool, state, blck, blockPool.get(blck).data.message, blck.slot)
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,
@ -79,7 +63,7 @@ proc init*(T: type AttestationPool, blockPool: BlockPool): T =
mapSlotsToAttestations: initDeque[AttestationsSeen](),
blockPool: blockPool,
unresolved: initTable[Eth2Digest, UnresolvedAttestation](),
forkChoice_v2: forkChoice
forkChoice: forkChoice
)
proc combine*(tgt: var Attestation, src: Attestation, flags: UpdateFlags) =
@ -101,7 +85,6 @@ proc combine*(tgt: var Attestation, src: Attestation, flags: UpdateFlags) =
else:
trace "Ignoring overlapping attestations"
proc slotIndex(
pool: var AttestationPool, state: BeaconState, attestationSlot: Slot): int =
## Grow and garbage collect pool, returning the deque index of the slot
@ -155,22 +138,12 @@ proc slotIndex(
int(attestationSlot - pool.startingSlot)
func updateLatestVotes(
pool: var AttestationPool, state: BeaconState, attestationSlot: Slot,
participants: seq[ValidatorIndex], blck: BlockRef, targetEpoch: Epoch) =
func processAttestation(
pool: var AttestationPool, state: BeaconState,
participants: seq[ValidatorIndex], block_root: Eth2Digest, target_epoch: 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, targetEpoch)
pool.forkChoice.process_attestation(validator, block_root, target_epoch)
func addUnresolved(pool: var AttestationPool, attestation: Attestation) =
pool.unresolved[attestation.data.beacon_block_root] =
@ -266,9 +239,8 @@ 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,
attestation.data.target.epoch)
pool.processAttestation(
state, participants, a.blck.root, attestation.data.target.epoch)
info "Attestation resolved",
attestation = shortLog(attestation),
@ -286,8 +258,8 @@ proc addResolved(pool: var AttestationPool, blck: BlockRef, attestation: Attesta
blck: blck,
validations: @[validation]
))
pool.updateLatestVotes(
state, attestationSlot, participants, blck, attestation.data.target.epoch)
pool.processAttestation(
state, participants, blck.root, attestation.data.target.epoch)
info "Attestation resolved",
attestation = shortLog(attestation),
@ -310,16 +282,15 @@ proc addAttestation*(pool: var AttestationPool, attestation: Attestation) =
pool.addResolved(blck, attestation)
proc addForkChoice_v2*(pool: var AttestationPool, blck: BlockRef,
justified_epoch, finalized_epoch: Epoch) =
proc addForkChoice*(pool: var AttestationPool,
state: BeaconState,
blckRef: BlockRef,
blck: BeaconBlock,
wallSlot: Slot) =
## 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,
)
let state = pool.forkChoice.process_block(
pool.blockPool, state, blckRef, blck, wallSlot)
if state.isErr:
# TODO If this happens, it is effectively a bug - the BlockRef structure
@ -327,7 +298,7 @@ proc addForkChoice_v2*(pool: var AttestationPool, blck: BlockRef,
# 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
blck = shortLog(blck), err = state.error
proc getAttestationsForSlot*(pool: AttestationPool, newBlockSlot: Slot):
Option[AttestationsSeen] =
@ -460,101 +431,8 @@ proc resolve*(pool: var AttestationPool) =
for a in resolved:
pool.addResolved(a.blck, a.attestation)
# Fork choice v1
# ---------------------------------------------------------------
func latestAttestation(
pool: AttestationPool, pubKey: ValidatorPubKey): BlockRef =
pool.latestAttestations.getOrDefault(pubKey)
# https://github.com/ethereum/eth2.0-specs/blob/v0.8.4/specs/core/0_fork-choice.md
# The structure of this code differs from the spec since we use a different
# strategy for storing states and justification points - it should nonetheless
# be close in terms of functionality.
func lmdGhost(
pool: AttestationPool, start_state: BeaconState,
start_block: BlockRef): BlockRef =
# TODO: a Fenwick Tree datastructure to keep track of cumulated votes
# in O(log N) complexity
# https://en.wikipedia.org/wiki/Fenwick_tree
# Nim implementation for cumulative frequencies at
# https://github.com/numforge/laser/blob/990e59fffe50779cdef33aa0b8f22da19e1eb328/benchmarks/random_sampling/fenwicktree.nim
let
active_validator_indices =
get_active_validator_indices(
start_state, compute_epoch_at_slot(start_state.slot))
var latest_messages: seq[tuple[validator: ValidatorIndex, blck: BlockRef]]
for i in active_validator_indices:
let pubKey = start_state.validators[i].pubkey
if (let vote = pool.latestAttestation(pubKey); not vote.isNil):
latest_messages.add((i, vote))
# TODO: update to 0.10.1: https://github.com/ethereum/eth2.0-specs/pull/1589/files#diff-9fc3792aa94456eb29506fa77f77b918R143
template get_latest_attesting_balance(blck: BlockRef): uint64 =
var res: uint64
for validator_index, target in latest_messages.items():
if get_ancestor(target, blck.slot) == blck:
res += start_state.validators[validator_index].effective_balance
res
var head = start_block
while true:
if head.children.len() == 0:
return head
if head.children.len() == 1:
head = head.children[0]
else:
var
winner = head.children[0]
winCount = get_latest_attesting_balance(winner)
for i in 1..<head.children.len:
let
candidate = head.children[i]
candCount = get_latest_attesting_balance(candidate)
if (candCount > winCount) or
((candCount == winCount and candidate.root.data < winner.root.data)):
winner = candidate
winCount = candCount
head = winner
proc selectHead_v1(pool: AttestationPool): BlockRef =
let
justifiedHead = pool.blockPool.latestJustifiedBlock()
let newHead =
lmdGhost(pool, pool.blockPool.justifiedState.data.data, justifiedHead.blck)
newHead
# Fork choice v2
# ---------------------------------------------------------------
func getAttesterBalances*(state: BeaconState): seq[Gwei] =
## Get the balances from a state
result.newSeq(state.validators.len) # zero-init
let epoch = state.get_current_epoch()
for i in 0 ..< result.len:
# All non-active validators have a 0 balance
template validator: Validator = state.validators[i]
if validator.is_active_validator(epoch):
result[i] = validator.effective_balance
proc selectHead_v2(pool: var AttestationPool): BlockRef =
let attesterBalances = pool.blockPool.justifiedState.data.data.getAttesterBalances()
let newHead = pool.forkChoice_v2.find_head(
justified_epoch = pool.blockPool.justifiedState.data.data.get_current_epoch(),
justified_root = pool.blockPool.head.justified.blck.root,
finalized_epoch = pool.blockPool.headState.data.data.finalized_checkpoint.epoch,
justified_state_balances = attesterBalances
)
proc selectHead*(pool: var AttestationPool, wallSlot: Slot): BlockRef =
let newHead = pool.forkChoice.find_head(wallSlot, pool.blockPool)
if newHead.isErr:
error "Couldn't select head", err = newHead.error
@ -562,20 +440,6 @@ proc selectHead_v2(pool: var AttestationPool): BlockRef =
else:
pool.blockPool.getRef(newHead.get())
proc pruneBefore*(pool: var AttestationPool, finalizedHead: BlockRef) =
if (let v = pool.forkChoice_v2.maybe_prune(finalizedHead.root); v.isErr):
proc prune*(pool: var AttestationPool) =
if (let v = pool.forkChoice.prune(); 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)
return head_v1

View File

@ -325,9 +325,9 @@ proc storeBlock(
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)
node.attestationPool.addForkChoice(
state.data, blckRef, signedBlock.message,
node.beaconClock.now().slotOrZero())
node.dumpBlock(signedBlock, blck)
@ -337,13 +337,6 @@ proc storeBlock(
if blck.isErr:
return err(blck.error)
# The block we received contains attestations, and we might not yet know about
# all of them. Let's add them to the attestation pool.
for attestation in signedBlock.message.body.attestations:
debug "Attestation from block",
attestation = shortLog(attestation)
node.attestationPool.addAttestation(attestation)
ok()
proc onBeaconBlock(node: BeaconNode, signedBlock: SignedBeaconBlock) =

View File

@ -64,7 +64,7 @@ proc updateHead*(node: BeaconNode): BlockRef =
node.attestationPool.resolve()
# Grab the new head according to our latest attestation data
let newHead = node.attestationPool.selectHead()
let newHead = node.attestationPool.selectHead(node.beaconClock.now().slotOrZero())
# Store the new head in the block pool - this may cause epochs to be
# justified and finalized
@ -75,7 +75,7 @@ proc updateHead*(node: BeaconNode): BlockRef =
# Cleanup the fork choice v2 if we have a finalized head
if oldFinalized != node.blockPool.finalizedHead.blck:
node.attestationPool.pruneBefore(node.blockPool.finalizedHead.blck)
node.attestationPool.prune()
newHead

View File

@ -72,12 +72,7 @@ type
unresolved*: Table[Eth2Digest, UnresolvedAttestation]
latestAttestations*: Table[ValidatorPubKey, BlockRef] ##\
## Map that keeps track of the most recent vote of each attester - see
## fork_choice
forkChoice_v2*: ForkChoice ##\
## The alternative fork choice "proto_array" that will ultimately
## replace the original one
forkChoice*: ForkChoice
# #############################################
#

View File

@ -168,6 +168,9 @@ template headState*(pool: BlockPool): StateData =
template tmpState*(pool: BlockPool): StateData =
pool.dag.tmpState
template balanceState*(pool: BlockPool): StateData =
pool.dag.balanceState
template justifiedState*(pool: BlockPool): StateData =
pool.dag.justifiedState

View File

@ -130,6 +130,10 @@ type
## Cached state used during block clearance - should only be used in the
## clearance module to avoid the risk of modifying it in a callback
balanceState*: StateData ##\
## Cached state for fork choice balance processing - should be replaced
## with a light-weight cache of balances only
updateFlags*: UpdateFlags
runtimePreset*: RuntimePreset
@ -149,9 +153,6 @@ type
parent*: BlockRef ##\
## Not nil, except for the tail
children*: seq[BlockRef]
# TODO do we strictly need this?
slot*: Slot # TODO could calculate this by walking to root, but..
epochsInfo*: seq[EpochRef]

View File

@ -73,7 +73,6 @@ func link*(parent, child: BlockRef) =
doAssert parent.root != child.root, "self-references not allowed"
child.parent = parent
parent.children.add(child)
func isAncestorOf*(a, b: BlockRef): bool =
var b = b
@ -302,6 +301,7 @@ proc init*(T: type CandidateChains,
justifiedState: tmpState[], # This is wrong but we'll update it below
tmpState: tmpState[],
clearanceState: tmpState[],
balanceState: tmpState[],
# The only allowed flag right now is verifyFinalization, as the others all
# allow skipping some validation.
@ -314,6 +314,7 @@ proc init*(T: type CandidateChains,
res.updateStateData(res.justifiedState, justifiedHead)
res.updateStateData(res.headState, headRef.atSlot(headRef.slot))
res.clearanceState = res.headState
res.balanceState = res.justifiedState
info "Block dag initialized",
head = head.blck, justifiedHead, finalizedHead, tail = tailRef,
@ -321,25 +322,6 @@ proc init*(T: type CandidateChains,
res
iterator topoSortedSinceLastFinalization*(dag: CandidateChains): BlockRef =
## Iterate on the dag in topological order
# TODO: this uses "children" for simplicity
# but "children" should be deleted as it introduces cycles
# that causes significant overhead at least and leaks at worst
# for the GC.
# This is not perf critical, it is only used to bootstrap the fork choice.
var visited: HashSet[BlockRef]
var stack: seq[BlockRef]
stack.add dag.finalizedHead.blck
while stack.len != 0:
let node = stack.pop()
if node notin visited:
visited.incl node
stack.add node.children
yield node
proc getState(
dag: CandidateChains, db: BeaconChainDB, stateRoot: Eth2Digest, blck: BlockRef,
output: var StateData): bool =
@ -796,41 +778,36 @@ proc updateHead*(dag: CandidateChains, newHead: BlockRef) =
# cur = cur.parent
# dag.delState(cur)
block: # Clean up block refs, walking block by block
var cur = finalizedHead.blck
while cur != dag.finalizedHead.blck:
# Finalization means that we choose a single chain as the canonical one -
# it also means we're no longer interested in any branches from that chain
# up to the finalization point.
# The new finalized head should not be cleaned! We start at its parent and
# clean everything including the old finalized head.
cur = cur.parent
# TODO what about attestations? we need to drop those too, though they
# *should* be pretty harmless
if cur.parent != nil: # This happens for the genesis / tail block
for child in cur.parent.children:
if child != cur:
# TODO also remove states associated with the unviable forks!
# TODO the easiest thing to do here would probably be to use
# dag.heads to find unviable heads, then walk those chains
# and remove everything.. currently, if there's a child with
# children of its own, those children will not be pruned
# correctly from the database
dag.blocks.del(child.root)
dag.db.delBlock(child.root)
cur.parent.children = @[cur]
block: # Clean up block refs, walking block by block
# Finalization means that we choose a single chain as the canonical one -
# it also means we're no longer interested in any branches from that chain
# up to the finalization point
let hlen = dag.heads.len
for i in 0..<hlen:
let n = hlen - i - 1
let head = dag.heads[n]
if finalizedHead.blck.isAncestorOf(head.blck):
continue
var cur = head.blck
while not cur.isAncestorOf(finalizedHead.blck):
# TODO empty states need to be removed also!
let stateRoot = dag.db.getStateRoot(cur.root, cur.slot)
if stateRoot.issome():
dag.db.delState(stateRoot.get())
dag.blocks.del(cur.root)
dag.db.delBlock(cur.root)
if cur.parent.isNil:
break
cur = cur.parent
dag.heads.del(n)
dag.finalizedHead = finalizedHead
let hlen = dag.heads.len
for i in 0..<hlen:
let n = hlen - i - 1
if not dag.finalizedHead.blck.isAncestorOf(dag.heads[n].blck):
# Any heads that are not derived from the newly finalized block are no
# longer viable candidates for future head selection
dag.heads.del(n)
info "Finalized block",
finalizedHead = shortLog(finalizedHead),
heads = dag.heads.len

View File

@ -9,15 +9,16 @@
import
# Standard library
std/tables, std/typetraits,
std/[sequtils, sets, tables, typetraits],
# Status libraries
stew/results, chronicles,
# Internal
../spec/[datatypes, digest],
../spec/[beaconstate, datatypes, digest, helpers],
# Fork choice
./fork_choice_types, ./proto_array
./fork_choice_types, ./proto_array,
../block_pool
# https://github.com/ethereum/eth2.0-specs/blob/v0.11.1/specs/phase0/fork-choice.md
# https://github.com/ethereum/eth2.0-specs/blob/v0.12.1/specs/phase0/fork-choice.md
# This is a port of https://github.com/sigp/lighthouse/pull/804
# which is a port of "Proto-Array": https://github.com/protolambda/lmd-ghost
# See also:
@ -50,29 +51,70 @@ logScope:
# - The private procs uses the ForkChoiceError error code
# - The public procs use Result
func initForkChoice*(
justified_epoch: Epoch,
finalized_epoch: Epoch,
finalized_root: Eth2Digest
): Result[ForkChoice, string] =
## Initialize a fork choice context
func get_effective_balances(state: BeaconState): seq[Gwei] =
## Get the balances from a state
result.newSeq(state.validators.len) # zero-init
let epoch = state.get_current_epoch()
for i in 0 ..< result.len:
# All non-active validators have a 0 balance
template validator: Validator = state.validators[i]
if validator.is_active_validator(epoch):
result[i] = validator.effective_balance
proc initForkChoiceBackend*(justified_epoch: Epoch,
finalized_epoch: Epoch,
finalized_root: Eth2Digest,
): Result[ForkChoiceBackend, string] =
var proto_array = ProtoArray(
prune_threshold: DefaultPruneThreshold,
justified_epoch: justified_epoch,
justified_epoch: finalized_epoch,
finalized_epoch: finalized_epoch
)
let err = proto_array.on_block(
finalized_root,
hasParentInForkChoice = false,
default(Eth2Digest),
justified_epoch,
Eth2Digest(),
finalized_epoch,
finalized_epoch
)
if err.kind != fcSuccess:
return err("Failed to add finalized block to proto_array: " & $err)
return ok(ForkChoice(proto_array: proto_array))
ok(ForkChoiceBackend(
proto_array: proto_array,
))
proc initForkChoice*(
finalizedState: StateData,
): Result[ForkChoice, string] =
## Initialize a fork choice context
debug "Initializing fork choice",
state_epoch = finalizedState.data.data.get_current_epoch(),
blck = shortLog(finalizedState.blck)
let finalized_epoch = finalizedState.data.data.get_current_epoch()
let ffgCheckpoint = FFGCheckpoints(
justified: BalanceCheckpoint(
blck: finalizedState.blck,
epoch: finalized_epoch,
balances: get_effective_balances(finalizedState.data.data)),
finalized: Checkpoint(root: finalizedState.blck.root, epoch: finalized_epoch))
let backend = ? initForkChoiceBackend(
finalized_epoch, finalized_epoch, finalizedState.blck.root)
ok(ForkChoice(
backend: backend,
checkpoints: Checkpoints(
current: ffgCheckpoint,
best: ffgCheckpoint),
finalizedBlock: finalizedState.blck,
))
func extend[T](s: var seq[T], minLen: int) =
## Extend a sequence so that it can contains at least `minLen` elements.
@ -94,15 +136,18 @@ func extend[T](s: var seq[T], minLen: int) =
zeroMem(s[curLen].addr, diff * sizeof(T))
func process_attestation*(
self: var ForkChoice,
self: var ForkChoiceBackend,
validator_index: ValidatorIndex,
block_root: Eth2Digest,
target_epoch: Epoch
) =
if block_root == Eth2Digest():
return
## Add an attestation to the fork choice context
self.votes.extend(validator_index.int + 1)
template vote: untyped {.dirty.} = self.votes[validator_index.int]
template vote: untyped = self.votes[validator_index.int]
# alias
if target_epoch > vote.next_epoch or vote == default(VoteTracker):
@ -113,41 +158,154 @@ func process_attestation*(
{.noSideEffect.}:
trace "Integrating vote in fork choice",
validator_index = validator_index,
new_vote = shortlog(vote)
new_vote = shortLog(vote)
func contains*(self: ForkChoice, block_root: Eth2Digest): bool =
func process_attestation*(
self: var ForkChoice,
validator_index: ValidatorIndex,
block_root: Eth2Digest,
target_epoch: Epoch
) =
self.backend.process_attestation(validator_index, block_root, target_epoch)
func contains*(self: ForkChoiceBackend, block_root: Eth2Digest): bool =
## Returns `true` if a block is known to the fork choice
## and `false` otherwise.
##
## In particular, before adding a block, its parent must be known to the fork choice
self.proto_array.indices.contains(block_root)
func process_block*(
self: var ForkChoice,
block_root: Eth2Digest,
parent_root: Eth2Digest,
justified_epoch: Epoch,
finalized_epoch: Epoch
): Result[void, string] =
## Add a block to the fork choice context
proc get_balances_for_block(self: var Checkpoints, blck: BlockRef, pool: BlockPool): seq[Gwei] =
pool.withState(pool.balanceState, blck.atSlot(blck.slot)):
get_effective_balances(state)
proc process_state(self: var Checkpoints,
pool: BlockPool,
state: BeaconState,
blck: BlockRef): Result[void, string] =
trace "Processing state",
state_slot = state.slot,
state_justified = state.current_justified_checkpoint.epoch,
current_justified = self.current.justified.epoch,
state_finalized = state.finalized_checkpoint.epoch,
current_finalized = self.current.finalized
if (state.current_justified_checkpoint.epoch > self.current.justified.epoch) and
(state.finalized_checkpoint.epoch >= self.current.finalized.epoch):
let justifiedBlck = blck.atSlot(
state.current_justified_checkpoint.epoch.compute_start_slot_at_epoch)
if justifiedBlck.blck.root != state.current_justified_checkpoint.root:
return err("invalid history?")
let candidate = FFGCheckpoints(
justified: BalanceCheckpoint(
blck: justifiedBlck.blck,
epoch: state.current_justified_checkpoint.epoch,
balances: self.get_balances_for_block(justifiedBlck.blck, pool),
),
finalized: state.finalized_checkpoint,
)
trace "Applying candidate",
justified_block = shortLog(candidate.justified.blck),
justified_epoch = shortLog(candidate.justified.epoch),
finalized = candidate.finalized,
state_finalized = state.finalized_checkpoint.epoch
if self.current.justified.blck.isAncestorOf(justifiedBlck.blck):
trace "Updating current",
prev = shortLog(self.current.justified.blck)
self.current = candidate
else:
trace "No current update",
prev = shortLog(self.current.justified.blck)
if candidate.justified.epoch > self.best.justified.epoch:
trace "Updating best",
prev = shortLog(self.best.justified.blck)
self.best = candidate
else:
trace "No best update",
prev = shortLog(self.best.justified.blck)
# self.balances_cache.process_state(block_root, state)?;
ok()
func compute_slots_since_epoch_start(slot: Slot): uint64 =
slot - compute_start_slot_at_epoch(compute_epoch_at_slot(slot))
proc maybe_update(self: var Checkpoints, current_slot: Slot, pool: BlockPool) =
trace "Updating checkpoint",
current_slot,
best = shortLog(self.best.justified.blck),
current = shortLog(self.current.justified.blck),
updateAt = self.updateAt
if self.best.justified.epoch > self.current.justified.epoch:
let current_epoch = current_slot.compute_epoch_at_slot()
if self.update_at.isNone():
if self.best.justified.epoch > self.current.justified.epoch:
if compute_slots_since_epoch_start(current_slot) < SAFE_SLOTS_TO_UPDATE_JUSTIFIED:
self.current = self.best
else:
self.update_at = some(current_epoch + 1)
elif self.updateAt.get() <= current_epoch:
self.current = self.best
self.update_at = none(Epoch)
proc process_block*(self: var ForkChoiceBackend,
block_root: Eth2Digest,
parent_root: Eth2Digest,
justified_epoch: Epoch,
finalized_epoch: Epoch): Result[void, string] =
let err = self.proto_array.on_block(
block_root, hasParentInForkChoice = true, parent_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)
ok()
proc process_block*(self: var ForkChoice,
pool: BlockPool,
state: BeaconState,
blckRef: BlockRef,
blck: SomeBeaconBlock,
wallSlot: Slot): Result[void, string] =
? process_state(self.checkpoints, pool, state, blckRef)
# TODO current time
maybe_update(self.checkpoints, wallSlot, pool)
var cache = StateCache() # TODO reuse shuffling
for attestation in blck.body.attestations:
if attestation.data.beacon_block_root in self.backend:
let participants = toSeq(items(get_attesting_indices(
state, attestation.data, attestation.aggregation_bits, cache)))
for validator in participants:
self.process_attestation(
validator,
attestation.data.beacon_block_root,
attestation.data.target.epoch)
? process_block(
self.backend, blckRef.root, blck.parent_root,
state.current_justified_checkpoint.epoch, state.finalized_checkpoint.epoch
)
{.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(blckRef)
return ok()
func find_head*(
self: var ForkChoice,
self: var ForkChoiceBackend,
justified_epoch: Epoch,
justified_root: Eth2Digest,
finalized_epoch: Epoch,
@ -185,15 +343,31 @@ func find_head*(
{.noSideEffect.}:
debug "Fork choice requested",
justified_epoch = justified_epoch,
justified_root = shortlog(justified_root),
justified_root = shortLog(justified_root),
finalized_epoch = finalized_epoch,
fork_choice_head = shortlog(new_head)
fork_choice_head = shortLog(new_head)
return ok(new_head)
proc find_head*(self: var ForkChoice,
wallSlot: Slot, pool: BlockPool): Result[Eth2Digest, string] =
template remove_alias(blck_root: Eth2Digest): Eth2Digest =
if blck_root == Eth2Digest():
self.finalizedBlock.root
else:
blck_root
self.checkpoints.maybe_update(wallSlot, pool)
self.backend.find_head(
self.checkpoints.current.justified.epoch,
remove_alias(self.checkpoints.current.justified.blck.root),
self.checkpoints.current.finalized.epoch,
self.checkpoints.current.justified.balances,
)
func maybe_prune*(
self: var ForkChoice, finalized_root: Eth2Digest
self: var ForkChoiceBackend, finalized_root: Eth2Digest
): Result[void, string] =
## Prune blocks preceding the finalized root as they are now unneeded.
let err = self.proto_array.maybe_prune(finalized_root)
@ -201,6 +375,10 @@ func maybe_prune*(
return err("find_head maybe_pruned failed: " & $err)
return ok()
func prune*(self: var ForkChoice): Result[void, string] =
let finalized_root = self.checkpoints.current.finalized.root
self.backend.maybe_prune(finalized_root)
func compute_deltas(
deltas: var openarray[Delta],
indices: Table[Eth2Digest, Index],

View File

@ -9,11 +9,12 @@
import
# Standard library
std/tables, std/options,
std/[tables, options],
# Status
chronicles,
# Internal
../spec/[datatypes, digest]
../spec/[datatypes, digest],
../block_pool
# https://github.com/ethereum/eth2.0-specs/blob/v0.11.1/specs/phase0/fork-choice.md
# This is a port of https://github.com/sigp/lighthouse/pull/804
@ -112,6 +113,20 @@ type
best_child*: Option[Index]
best_descendant*: Option[Index]
BalanceCheckpoint* = object
blck*: BlockRef
epoch*: Epoch
balances*: seq[Gwei]
FFGCheckpoints* = object
justified*: BalanceCheckpoint
finalized*: Checkpoint
Checkpoints* = object
current*: FFGCheckpoints
best*: FFGCheckpoints
updateAt*: Option[Epoch]
const ForkChoiceSuccess* = ForkChoiceError(kind: fcSuccess)
# Fork choice high-level types
@ -123,14 +138,16 @@ type
next_root*: Eth2Digest
next_epoch*: Epoch
ForkChoice* = object
# Note: Lighthouse is protecting all fields with Reader-Writer locks.
# However, given the nature of the fields, I suspect sharing those fields
# will lead to thread contention. For now, stay single-threaded. - Mamy
ForkChoiceBackend* = object
proto_array*: ProtoArray
votes*: seq[VoteTracker]
balances*: seq[Gwei]
ForkChoice* = object
backend*: ForkChoiceBackend
checkpoints*: Checkpoints
finalizedBlock*: BlockRef ## Any finalized block used at startup
func shortlog*(vote: VoteTracker): auto =
(
current_root: vote.current_root,

View File

@ -135,7 +135,9 @@ proc createAndSendAttestation(node: BeaconNode,
num_active_validators: uint64) {.async.} =
logScope: pcs = "send_attestation"
var attestation = await validator.produceAndSignAttestation(attestationData, committeeLen, indexInCommittee, fork, genesis_validators_root)
var attestation = await validator.produceAndSignAttestation(
attestationData, committeeLen, indexInCommittee, fork,
genesis_validators_root)
node.sendAttestation(attestation, num_active_validators)
@ -228,9 +230,9 @@ proc proposeSignedBlock*(node: BeaconNode,
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)
node.attestationPool.addForkChoice(
state.data, blckRef, signedBlock.message,
node.beaconClock.now().slotOrZero())
if newBlockRef.isErr:
warn "Unable to add proposed block to block pool",

View File

@ -139,9 +139,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
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)
attPool.addForkChoice(state.data, blckRef, signedBlock.message, blckRef.slot)
blck() = added[]
blockPool.updateHead(added[])

View File

@ -60,7 +60,7 @@ type
prune_threshold*: int
expected_len*: int
func apply(ctx: var ForkChoice, id: int, op: Operation) =
func apply(ctx: var ForkChoiceBackend, id: int, op: Operation) =
## Apply the specified operation to a ForkChoice context
## ``id`` is additional debugging info. It is the
## operation index.
@ -104,7 +104,7 @@ func apply(ctx: var ForkChoice, id: int, op: Operation) =
&"prune (op #{id}): the resulting length ({ctx.proto_array.nodes.len}) was not expected ({op.expected_len})"
debugEcho " Maybe_pruned block preceding finalized block 0x", op.finalized_root
func run*(ctx: var ForkChoice, ops: seq[Operation]) =
func run*(ctx: var ForkChoiceBackend, ops: seq[Operation]) =
## Apply a sequence of fork-choice operations on a store
for i, op in ops:
ctx.apply(i, op)

View File

@ -7,12 +7,12 @@
# import ../interpreter # included to be able to use "suiteReport"
proc setup_finality_01(): tuple[fork_choice: ForkChoice, ops: seq[Operation]] =
proc setup_finality_01(): tuple[fork_choice: ForkChoiceBackend, ops: seq[Operation]] =
var balances = @[Gwei(1), Gwei(1)]
let GenesisRoot = fakeHash(0)
# Initialize the fork choice context
result.fork_choice = initForkChoice(
result.fork_choice = initForkChoiceBackend(
justified_epoch = Epoch(1),
finalized_epoch = Epoch(1),
finalized_root = GenesisRoot

View File

@ -7,12 +7,12 @@
# import ../interpreter # included to be able to use "suiteReport"
proc setup_finality_02(): tuple[fork_choice: ForkChoice, ops: seq[Operation]] =
proc setup_finality_02(): tuple[fork_choice: ForkChoiceBackend, ops: seq[Operation]] =
var balances = @[Gwei(1), Gwei(1)]
let GenesisRoot = fakeHash(0)
# Initialize the fork choice context
result.fork_choice = initForkChoice(
result.fork_choice = initForkChoiceBackend(
justified_epoch = Epoch(1),
finalized_epoch = Epoch(1),
finalized_root = GenesisRoot

View File

@ -7,12 +7,12 @@
# import ../interpreter # included to be able to use "suiteReport"
proc setup_no_votes(): tuple[fork_choice: ForkChoice, ops: seq[Operation]] =
proc setup_no_votes(): tuple[fork_choice: ForkChoiceBackend, ops: seq[Operation]] =
let balances = newSeq[Gwei](16)
let GenesisRoot = fakeHash(0)
# Initialize the fork choice context
result.fork_choice = initForkChoice(
result.fork_choice = initForkChoiceBackend(
justified_epoch = Epoch(1),
finalized_epoch = Epoch(1),
finalized_root = GenesisRoot

View File

@ -7,12 +7,12 @@
# import ../interpreter # included to be able to use "suiteReport"
proc setup_votes(): tuple[fork_choice: ForkChoice, ops: seq[Operation]] =
proc setup_votes(): tuple[fork_choice: ForkChoiceBackend, ops: seq[Operation]] =
var balances = @[Gwei(1), Gwei(1)]
let GenesisRoot = fakeHash(0)
# Initialize the fork choice context
result.fork_choice = initForkChoice(
result.fork_choice = initForkChoiceBackend(
justified_epoch = Epoch(1),
finalized_epoch = Epoch(1),
finalized_root = GenesisRoot

View File

@ -170,12 +170,10 @@ suiteReport "Attestation pool processing" & preset():
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)
pool[].addForkChoice(state.data, blckRef, signedBlock.message, blckRef.slot)
let head = pool[].selectHead()
let head = pool[].selectHead(b1Add[].slot)
check:
head == b1Add[]
@ -186,11 +184,9 @@ suiteReport "Attestation pool processing" & preset():
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)
pool[].addForkChoice(state.data, blckRef, signedBlock.message, blckRef.slot)
let head2 = pool[].selectHead()
let head2 = pool[].selectHead(b2Add[].slot)
check:
head2 == b2Add[]
@ -203,11 +199,9 @@ suiteReport "Attestation pool processing" & preset():
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)
pool[].addForkChoice(state.data, blckRef, signedBlock.message, blckRef.slot)
let head = pool[].selectHead()
let head = pool[].selectHead(b10Add[].slot)
check:
head == b10Add[]
@ -220,9 +214,7 @@ suiteReport "Attestation pool processing" & preset():
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)
pool[].addForkChoice(state.data, blckRef, signedBlock.message, blckRef.slot)
bc1 = get_beacon_committee(
state.data.data, state.data.data.slot, 1.CommitteeIndex, cache)
@ -230,7 +222,7 @@ suiteReport "Attestation pool processing" & preset():
pool[].addAttestation(attestation0)
let head2 = pool[].selectHead()
let head2 = pool[].selectHead(b10Add[].slot)
check:
# Single vote for b10 and no votes for b11
@ -241,20 +233,16 @@ suiteReport "Attestation pool processing" & preset():
attestation2 = makeAttestation(state.data.data, b11.root, bc1[2], cache)
pool[].addAttestation(attestation1)
let head3 = pool[].selectHead()
# Warning - the tiebreak are incorrect and guaranteed consensus fork, it should be bigger
let smaller = if b10.root.data < b11.root.data: b10Add else: b11Add
let head3 = pool[].selectHead(b10Add[].slot)
let bigger = if b11.root.data < b10.root.data: b10Add else: b11Add
check:
# Ties broken lexicographically in spec -> ?
# all implementations favor the biggest root
# TODO
# currently using smaller as we have used for over a year
head3 == smaller[]
head3 == bigger[]
pool[].addAttestation(attestation2)
let head4 = pool[].selectHead()
let head4 = pool[].selectHead(b11Add[].slot)
check:
# Two votes for b11
@ -268,11 +256,9 @@ suiteReport "Attestation pool processing" & preset():
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)
pool[].addForkChoice(state.data, blckRef, signedBlock.message, blckRef.slot)
let head = pool[].selectHead()
let head = pool[].selectHead(b10Add[].slot)
check:
head == b10Add[]
@ -284,15 +270,13 @@ suiteReport "Attestation pool processing" & preset():
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)
pool[].addForkChoice(state.data, blckRef, signedBlock.message, blckRef.slot)
doAssert: b10Add_clone.error == Duplicate
wrappedTimedTest "Trying to add a duplicate block from an old pruned epoch is tagged as an error":
blockpool[].addFlags {skipBLSValidation}
pool.forkChoice_v2.proto_array.prune_threshold = 1
pool.forkChoice.backend.proto_array.prune_threshold = 1
var cache = StateCache()
let
b10 = makeTestBlock(state.data, blockPool[].tail.root, cache)
@ -300,11 +284,9 @@ suiteReport "Attestation pool processing" & preset():
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)
pool[].addForkChoice(state.data, blckRef, signedBlock.message, blckRef.slot)
let head = pool[].selectHead()
let head = pool[].selectHead(b10Add[].slot)
doAssert: head == b10Add[]
@ -322,8 +304,9 @@ suiteReport "Attestation pool processing" & preset():
for epoch in 0 ..< 5:
let start_slot = compute_start_slot_at_epoch(Epoch epoch)
let committees_per_slot =
get_committee_count_per_slot(state.data.data, start_slot, cache)
for slot in start_slot ..< start_slot + SLOTS_PER_EPOCH:
let new_block = makeTestBlock(
state.data, block_root, cache, attestations = attestations)
let block_ok = state_transition(
@ -335,16 +318,14 @@ suiteReport "Attestation pool processing" & preset():
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)
pool[].addForkChoice(state.data, blckRef, signedBlock.message, blckRef.slot)
let head = pool[].selectHead()
let head = pool[].selectHead(blockRef[].slot)
doassert: head == blockRef[]
blockPool[].updateHead(head)
attestations.setlen(0)
for index in 0'u64 ..< get_committee_count_per_slot(state.data.data, slot.Slot, cache):
for index in 0'u64 ..< committees_per_slot:
let committee = get_beacon_committee(
state.data.data, state.data.data.slot, index.CommitteeIndex, cache)
@ -370,16 +351,14 @@ suiteReport "Attestation pool processing" & preset():
doAssert: blockPool[].finalizedHead.slot != 0
pool[].pruneBefore(blockPool[].finalizedHead.blck)
doAssert: b10.root notin pool.forkChoice_v2
pool[].prune()
doAssert: b10.root notin pool.forkChoice.backend
# Add back the old block to ensure we have a duplicate error
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)
pool[].addForkChoice(state.data, blckRef, signedBlock.message, blckRef.slot)
doAssert: b10Add_clone.error == Duplicate

View File

@ -192,7 +192,6 @@ suiteReport "Block pool processing" & preset():
b1Get.isSome()
b2Get.isSome()
b1Get.get().refs.children[0] == b2Get.get().refs
b2Get.get().refs.parent == b1Get.get().refs
pool.updateHead(b2Get.get().refs)
@ -308,7 +307,6 @@ suiteReport "BlockPool finalization tests" & preset():
if i == 1:
# There are 2 heads now because of the fork at slot 1
check:
pool.tail.children.len == 2
pool.heads.len == 2
blck = makeTestBlock(
@ -323,7 +321,6 @@ suiteReport "BlockPool finalization tests" & preset():
check:
pool.heads.len() == 1
pool.head.justified.slot.compute_epoch_at_slot() == 5
pool.tail.children.len == 1
block:
# The late block is a block whose parent was finalized long ago and thus