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:
parent
c47532f2b0
commit
fd4d319450
|
@ -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
|
||||
|
|
|
@ -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,46 +27,30 @@ proc init*(T: type AttestationPool, blockPool: BlockPool): T =
|
|||
# probably be removed as a dependency of AttestationPool (or some other
|
||||
# smart refactoring)
|
||||
|
||||
blockPool.withState(blockPool.tmpState, blockPool.finalizedHead):
|
||||
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
|
||||
blockPool.tmpState,
|
||||
).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.tmpState,
|
||||
BlockSlot(blck: blck, 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
|
||||
|
||||
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 = blockPool.tmpState.data.data.current_justified_checkpoint.epoch,
|
||||
finalized_epoch = blockPool.tmpState.data.data.finalized_checkpoint.epoch,
|
||||
justified_epoch = state.current_justified_checkpoint.epoch,
|
||||
finalized_epoch = state.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
|
||||
)
|
||||
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
|
||||
|
||||
|
@ -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
|
||||
|
|
|
@ -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) =
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
|
@ -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
|
||||
|
||||
# #############################################
|
||||
#
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
|
@ -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]
|
||||
|
|
|
@ -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]
|
||||
|
||||
dag.finalizedHead = finalizedHead
|
||||
|
||||
# up to the finalization point
|
||||
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
|
||||
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
|
||||
|
||||
info "Finalized block",
|
||||
finalizedHead = shortLog(finalizedHead),
|
||||
heads = dag.heads.len
|
||||
|
|
|
@ -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,
|
||||
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[ForkChoice, string] =
|
||||
## Initialize a fork choice context
|
||||
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,
|
||||
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] =
|
||||
## Add a block to the fork choice context
|
||||
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],
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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[])
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
Loading…
Reference in New Issue