more fork-choice fixes (#1388)

* more fork-choice fixes

* use target block/epoch to validate attestations
* make addLocalValidators sync
* add current and previous epoch to cache before doing state transition
* update head state using clearance state as a shortcut, when possible
* use blockslot for fork choice balances
* send attestations using epochref cache

* fix invalid finalized parent being used

also simplify epoch block traversal

* single error handling style in fork choice

* import fix, remove unused async
This commit is contained in:
Jacek Sieka 2020-07-30 17:48:25 +02:00 committed by GitHub
parent 55607f107f
commit c5fecd472f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 261 additions and 281 deletions

View File

@ -169,9 +169,16 @@ proc isValidAttestation*(
# Not in spec - check that rewinding to the state is sane # Not in spec - check that rewinding to the state is sane
return false return false
let tgtBlck = pool.blockPool.getRef(attestation.data.target.root)
if tgtBlck.isNil:
debug "Target block not found"
pool.blockPool.addMissing(attestation.data.beacon_block_root)
return
# TODO this could be any state in the target epoch
pool.blockPool.withState( pool.blockPool.withState(
pool.blockPool.tmpState, pool.blockPool.tmpState,
BlockSlot(blck: attestationBlck, slot: attestation.data.slot)): tgtBlck.atSlot(attestation.data.target.epoch.compute_start_slot_at_epoch)):
# https://github.com/ethereum/eth2.0-specs/blob/v0.12.1/specs/phase0/p2p-interface.md#attestation-subnets # https://github.com/ethereum/eth2.0-specs/blob/v0.12.1/specs/phase0/p2p-interface.md#attestation-subnets
# [REJECT] The attestation is for the correct subnet (i.e. # [REJECT] The attestation is for the correct subnet (i.e.
# compute_subnet_for_attestation(state, attestation) == subnet_id). # compute_subnet_for_attestation(state, attestation) == subnet_id).
@ -274,10 +281,17 @@ proc isValidAggregatedAttestation*(
# [REJECT] aggregate_and_proof.selection_proof selects the validator as an # [REJECT] aggregate_and_proof.selection_proof selects the validator as an
# aggregator for the slot -- i.e. is_aggregator(state, aggregate.data.slot, # aggregator for the slot -- i.e. is_aggregator(state, aggregate.data.slot,
# aggregate.data.index, aggregate_and_proof.selection_proof) returns True. # aggregate.data.index, aggregate_and_proof.selection_proof) returns True.
# TODO use withEpochState when it works more reliably
let tgtBlck = pool.blockPool.getRef(aggregate.data.target.root)
if tgtBlck.isNil:
debug "Target block not found"
pool.blockPool.addMissing(aggregate.data.beacon_block_root)
return
# TODO this could be any state in the target epoch
pool.blockPool.withState( pool.blockPool.withState(
pool.blockPool.tmpState, pool.blockPool.tmpState,
BlockSlot(blck: attestationBlck, slot: aggregate.data.slot)): tgtBlck.atSlot(aggregate.data.target.epoch.compute_start_slot_at_epoch)):
var cache = getEpochCache(blck, state) var cache = getEpochCache(blck, state)
if not is_aggregator( if not is_aggregator(
state, aggregate.data.slot, aggregate.data.index.CommitteeIndex, state, aggregate.data.slot, aggregate.data.index.CommitteeIndex,

View File

@ -25,7 +25,8 @@ import
spec/state_transition, spec/state_transition,
conf, time, beacon_chain_db, validator_pool, extras, conf, time, beacon_chain_db, validator_pool, extras,
attestation_pool, block_pool, eth2_network, eth2_discovery, attestation_pool, block_pool, eth2_network, eth2_discovery,
beacon_node_common, beacon_node_types, block_pools/block_pools_types, beacon_node_common, beacon_node_types,
block_pools/[block_pools_types, candidate_chains],
nimbus_binary_common, network_metadata, nimbus_binary_common, network_metadata,
mainchain_monitor, version, ssz/[merkleization], sszdump, merkle_minimal, mainchain_monitor, version, ssz/[merkleization], sszdump, merkle_minimal,
sync_protocol, request_manager, keystore_management, interop, statusbar, sync_protocol, request_manager, keystore_management, interop, statusbar,
@ -251,7 +252,7 @@ proc init*(T: type BeaconNode,
enrForkId = enrForkIdFromState(blockPool.headState.data.data) enrForkId = enrForkIdFromState(blockPool.headState.data.data)
topicBeaconBlocks = getBeaconBlocksTopic(enrForkId.forkDigest) topicBeaconBlocks = getBeaconBlocksTopic(enrForkId.forkDigest)
topicAggregateAndProofs = getAggregateAndProofsTopic(enrForkId.forkDigest) topicAggregateAndProofs = getAggregateAndProofsTopic(enrForkId.forkDigest)
network = await createEth2Node(rng, conf, enrForkId) network = createEth2Node(rng, conf, enrForkId)
var res = BeaconNode( var res = BeaconNode(
nickname: nickname, nickname: nickname,
@ -277,7 +278,7 @@ proc init*(T: type BeaconNode,
onBeaconBlock(res, signedBlock) onBeaconBlock(res, signedBlock)
) )
await res.addLocalValidators() res.addLocalValidators()
# This merely configures the BeaconSync # This merely configures the BeaconSync
# The traffic will be started when we join the network. # The traffic will be started when we join the network.
@ -935,9 +936,8 @@ when hasPrompt:
# p.useHistoryFile() # p.useHistoryFile()
proc dataResolver(expr: string): string = proc dataResolver(expr: string): string =
template justified: untyped = node.blockPool.head.atSlot( template justified: untyped = node.blockPool.head.atEpochStart(
node.blockPool.headState.data.data.current_justified_checkpoint.epoch. node.blockPool.headState.data.data.current_justified_checkpoint.epoch)
compute_start_slot_at_epoch)
# TODO: # TODO:
# We should introduce a general API for resolving dot expressions # We should introduce a general API for resolving dot expressions
# such as `db.latest_block.slot` or `metrics.connected_peers`. # such as `db.latest_block.slot` or `metrics.connected_peers`.

View File

@ -153,7 +153,8 @@ type
slot*: Slot # TODO could calculate this by walking to root, but.. slot*: Slot # TODO could calculate this by walking to root, but..
epochsInfo*: seq[EpochRef] epochsInfo*: seq[EpochRef] ##\
## Cached information about the epochs starting at this block.
## Could be multiple, since blocks could skip slots, but usually, not many ## Could be multiple, since blocks could skip slots, but usually, not many
## Even if competing forks happen later during this epoch, potential empty ## Even if competing forks happen later during this epoch, potential empty
## slots beforehand must all be from this fork. getEpochInfo() is the only ## slots beforehand must all be from this fork. getEpochInfo() is the only

View File

@ -162,6 +162,14 @@ func atSlot*(blck: BlockRef, slot: Slot): BlockSlot =
## block proposal) ## block proposal)
BlockSlot(blck: blck.getAncestorAt(slot), slot: slot) BlockSlot(blck: blck.getAncestorAt(slot), slot: slot)
func atEpochStart*(blck: BlockRef, epoch: Epoch): BlockSlot =
## Return the BlockSlot corresponding to the first slot in the given epoch
atSlot(blck, epoch.compute_start_slot_at_epoch)
func atEpochEnd*(blck: BlockRef, epoch: Epoch): BlockSlot =
## Return the BlockSlot corresponding to the last slot in the given epoch
atSlot(blck, (epoch + 1).compute_start_slot_at_epoch - 1)
func getEpochInfo*(blck: BlockRef, state: BeaconState): EpochRef = func getEpochInfo*(blck: BlockRef, state: BeaconState): EpochRef =
# This is the only intended mechanism by which to get an EpochRef # This is the only intended mechanism by which to get an EpochRef
let let
@ -186,7 +194,20 @@ func getEpochInfo*(blck: BlockRef, state: BeaconState): EpochRef =
func getEpochCache*(blck: BlockRef, state: BeaconState): StateCache = func getEpochCache*(blck: BlockRef, state: BeaconState): StateCache =
let epochInfo = getEpochInfo(blck, state) let epochInfo = getEpochInfo(blck, state)
result = StateCache() if epochInfo.epoch > 0:
# When doing state transitioning, both the current and previous epochs are
# useful from a cache perspective since attestations may come from either -
# we'll use the last slot from the epoch because it is more likely to
# be filled in already, compared to the first slot where the block might
# be from the epoch before.
let
prevEpochBlck = blck.atEpochEnd(epochInfo.epoch - 1).blck
for ei in prevEpochBlck.epochsInfo:
if ei.epoch == epochInfo.epoch - 1:
result.shuffled_active_validator_indices[ei.epoch] =
ei.shuffled_active_validator_indices
result.shuffled_active_validator_indices[state.get_current_epoch()] = result.shuffled_active_validator_indices[state.get_current_epoch()] =
epochInfo.shuffled_active_validator_indices epochInfo.shuffled_active_validator_indices
@ -279,9 +300,8 @@ proc init*(T: type CandidateChains,
# from the same epoch as the head, thus the finalized and justified slots are # from the same epoch as the head, thus the finalized and justified slots are
# the same - these only change on epoch boundaries. # the same - these only change on epoch boundaries.
let let
finalizedSlot = finalizedHead = headRef.atEpochStart(
tmpState.data.data.finalized_checkpoint.epoch.compute_start_slot_at_epoch() tmpState.data.data.finalized_checkpoint.epoch)
finalizedHead = headRef.atSlot(finalizedSlot)
let res = CandidateChains( let res = CandidateChains(
blocks: blocks, blocks: blocks,
@ -316,12 +336,18 @@ proc init*(T: type CandidateChains,
res res
proc getEpochRef*(pool: CandidateChains, blck: BlockRef, epoch: Epoch): EpochRef = proc getEpochRef*(pool: CandidateChains, blck: BlockRef, epoch: Epoch): EpochRef =
let bs = blck.atSlot(epoch.compute_start_slot_at_epoch) var bs = blck.atEpochEnd(epoch)
for e in bs.blck.epochsInfo:
if e.epoch == epoch: while true:
return e # Any block from within the same epoch will carry the same epochinfo, so
# we start at the most recent one
for e in bs.blck.epochsInfo:
if e.epoch == epoch:
return e
if bs.slot == epoch.compute_start_slot_at_epoch:
break
bs = bs.parent
# TODO use any state from epoch
pool.withState(pool.tmpState, bs): pool.withState(pool.tmpState, bs):
getEpochInfo(blck, state) getEpochInfo(blck, state)
@ -723,13 +749,19 @@ proc updateHead*(dag: CandidateChains, newHead: BlockRef) =
lastHead = dag.head lastHead = dag.head
dag.db.putHeadBlock(newHead.root) dag.db.putHeadBlock(newHead.root)
# Start off by making sure we have the right state # Start off by making sure we have the right state - as a special case, we'll
updateStateData( # check the last block that was cleared by clearance - it might be just the
dag, dag.headState, BlockSlot(blck: newHead, slot: newHead.slot)) # thing we're looking for
if dag.clearanceState.blck == newHead and
dag.clearanceState.data.data.slot == newHead.slot:
assign(dag.headState, dag.clearanceState)
else:
updateStateData(
dag, dag.headState, newHead.atSlot(newHead.slot))
dag.head = newHead dag.head = newHead
# TODO isAncestorOf may be expensive - too expensive?
if not lastHead.isAncestorOf(newHead): if not lastHead.isAncestorOf(newHead):
info "Updated head block with reorg", info "Updated head block with reorg",
lastHead = shortLog(lastHead), lastHead = shortLog(lastHead),
@ -750,10 +782,8 @@ proc updateHead*(dag: CandidateChains, newHead: BlockRef) =
justified = shortLog(dag.headState.data.data.current_justified_checkpoint), justified = shortLog(dag.headState.data.data.current_justified_checkpoint),
finalized = shortLog(dag.headState.data.data.finalized_checkpoint) finalized = shortLog(dag.headState.data.data.finalized_checkpoint)
let let
finalizedEpochStartSlot = finalizedHead = newHead.atEpochStart(
dag.headState.data.data.finalized_checkpoint.epoch. dag.headState.data.data.finalized_checkpoint.epoch)
compute_start_slot_at_epoch()
finalizedHead = newHead.atSlot(finalizedEpochStartSlot)
doAssert (not finalizedHead.blck.isNil), doAssert (not finalizedHead.blck.isNil),
"Block graph should always lead to a finalized block" "Block graph should always lead to a finalized block"
@ -773,7 +803,6 @@ proc updateHead*(dag: CandidateChains, newHead: BlockRef) =
# cur = cur.parent # cur = cur.parent
# dag.delState(cur) # dag.delState(cur)
block: # Clean up block refs, walking block by block block: # Clean up block refs, walking block by block
# Finalization means that we choose a single chain as the canonical one - # 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 # it also means we're no longer interested in any branches from that chain

View File

@ -8,7 +8,7 @@
{.push raises: [Defect].} {.push raises: [Defect].}
import import
std/tables, std/[sequtils, tables],
chronicles, chronicles,
metrics, stew/results, metrics, stew/results,
../extras, ../extras,
@ -46,20 +46,21 @@ proc addResolvedBlock(
parent: BlockRef, cache: StateCache, parent: BlockRef, cache: StateCache,
onBlockAdded: OnBlockAdded onBlockAdded: OnBlockAdded
): BlockRef = ): BlockRef =
# TODO: `addResolvedBlock` is accumulating significant cruft # TODO move quarantine processing out of here
# and is in dire need of refactoring
# - the ugly `quarantine.inAdd` field
# - the callback
# - callback may be problematic as it's called in async validator duties
logScope: pcs = "block_resolution" logScope: pcs = "block_resolution"
doAssert state.data.slot == signedBlock.message.slot, "state must match block" doAssert state.data.slot == signedBlock.message.slot, "state must match block"
let let
blockRoot = signedBlock.root blockRoot = signedBlock.root
blockRef = BlockRef.init(blockRoot, signedBlock.message) blockRef = BlockRef.init(blockRoot, signedBlock.message)
if parent.slot.compute_epoch_at_slot() == blockRef.slot.compute_epoch_at_slot: blockEpoch = blockRef.slot.compute_epoch_at_slot()
blockRef.epochsInfo = @[parent.epochsInfo[0]] if parent.slot.compute_epoch_at_slot() == blockEpoch:
# If the parent and child blocks are from the same epoch, we can reuse
# the epoch cache - but we'll only use the current epoch because the new
# block might have affected what the next epoch looks like
blockRef.epochsInfo = filterIt(parent.epochsInfo, it.epoch == blockEpoch)
else: else:
# Ensure we collect the epoch info if it's missing
discard getEpochInfo(blockRef, state.data) discard getEpochInfo(blockRef, state.data)
link(parent, blockRef) link(parent, blockRef)
@ -88,7 +89,8 @@ proc addResolvedBlock(
blockRoot = shortLog(blockRoot), blockRoot = shortLog(blockRoot),
heads = dag.heads.len() heads = dag.heads.len()
# This MUST be added before the quarantine # Notify others of the new block before processing the quarantine, such that
# notifications for parents happens before those of the children
if onBlockAdded != nil: if onBlockAdded != nil:
onBlockAdded(blockRef, signedBlock, state) onBlockAdded(blockRef, signedBlock, state)
@ -119,13 +121,8 @@ proc addRawBlock*(
signedBlock: SignedBeaconBlock, signedBlock: SignedBeaconBlock,
onBlockAdded: OnBlockAdded onBlockAdded: OnBlockAdded
): Result[BlockRef, BlockError] = ): Result[BlockRef, BlockError] =
## return the block, if resolved... ## Try adding a block to the chain, verifying first that it passes the state
## transition function.
# TODO: `addRawBlock` is accumulating significant cruft
# and is in dire need of refactoring
# - the ugly `quarantine.inAdd` field
# - the callback
# - callback may be problematic as it's called in async validator duties
logScope: logScope:
blck = shortLog(signedBlock.message) blck = shortLog(signedBlock.message)
@ -134,14 +131,12 @@ proc addRawBlock*(
template blck(): untyped = signedBlock.message # shortcuts without copy template blck(): untyped = signedBlock.message # shortcuts without copy
template blockRoot(): untyped = signedBlock.root template blockRoot(): untyped = signedBlock.root
# Already seen this block??
if blockRoot in dag.blocks: if blockRoot in dag.blocks:
debug "Block already exists" debug "Block already exists"
# There can be a scenario where we receive a block we already received. # We should not call the block added callback for blocks that already
# However this block was before the last finalized epoch and so its parent # existed in the pool, as that may confuse consumers such as the fork
# was pruned from the ForkChoice. Trying to add it again, even if the fork choice # choice.
# supports duplicate will lead to a crash.
return err Duplicate return err Duplicate
quarantine.missing.del(blockRoot) quarantine.missing.del(blockRoot)
@ -168,7 +163,9 @@ proc addRawBlock*(
return err Invalid return err Invalid
if parent.slot < dag.finalizedHead.slot: if (parent.slot < dag.finalizedHead.slot) or
(parent.slot == dag.finalizedHead.slot and
parent != dag.finalizedHead.blck):
# We finalized a block that's newer than the parent of this block - this # We finalized a block that's newer than the parent of this block - this
# block, although recent, is thus building on a history we're no longer # block, although recent, is thus building on a history we're no longer
# interested in pursuing. This can happen if a client produces a block # interested in pursuing. This can happen if a client produces a block

View File

@ -1142,7 +1142,7 @@ func gossipId(data: openArray[byte]): string =
func msgIdProvider(m: messages.Message): string = func msgIdProvider(m: messages.Message): string =
gossipId(m.data) gossipId(m.data)
proc createEth2Node*(rng: ref BrHmacDrbgContext, conf: BeaconNodeConf, enrForkId: ENRForkID): Future[Eth2Node] {.async, gcsafe.} = proc createEth2Node*(rng: ref BrHmacDrbgContext, conf: BeaconNodeConf, enrForkId: ENRForkID): Eth2Node {.gcsafe.} =
var var
(extIp, extTcpPort, extUdpPort) = setupNat(conf) (extIp, extTcpPort, extUdpPort) = setupNat(conf)
hostAddress = tcpEndPoint(conf.libp2pAddress, conf.tcpPort) hostAddress = tcpEndPoint(conf.libp2pAddress, conf.tcpPort)

View File

@ -18,7 +18,7 @@ import
./fork_choice_types, ./proto_array, ./fork_choice_types, ./proto_array,
../block_pool, ../block_pools/candidate_chains ../block_pool, ../block_pools/candidate_chains
export sets export sets, results, fork_choice_types
# https://github.com/ethereum/eth2.0-specs/blob/v0.12.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 # This is a port of https://github.com/sigp/lighthouse/pull/804
@ -39,7 +39,7 @@ func compute_deltas(
votes: var openArray[VoteTracker], votes: var openArray[VoteTracker],
old_balances: openarray[Gwei], old_balances: openarray[Gwei],
new_balances: openarray[Gwei] new_balances: openarray[Gwei]
): ForkChoiceError ): FcResult[void]
# TODO: raises [Defect] - once https://github.com/nim-lang/Nim/issues/12862 is fixed # TODO: raises [Defect] - once https://github.com/nim-lang/Nim/issues/12862 is fixed
# https://github.com/status-im/nim-beacon-chain/pull/865#pullrequestreview-389117232 # https://github.com/status-im/nim-beacon-chain/pull/865#pullrequestreview-389117232
@ -68,14 +68,14 @@ func get_effective_balances(state: BeaconState): seq[Gwei] =
proc initForkChoiceBackend*(justified_epoch: Epoch, proc initForkChoiceBackend*(justified_epoch: Epoch,
finalized_epoch: Epoch, finalized_epoch: Epoch,
finalized_root: Eth2Digest, finalized_root: Eth2Digest,
): Result[ForkChoiceBackend, string] = ): FcResult[ForkChoiceBackend] =
var proto_array = ProtoArray( var proto_array = ProtoArray(
prune_threshold: DefaultPruneThreshold, prune_threshold: DefaultPruneThreshold,
justified_epoch: finalized_epoch, justified_epoch: finalized_epoch,
finalized_epoch: finalized_epoch finalized_epoch: finalized_epoch
) )
let err = proto_array.on_block( ? proto_array.on_block(
finalized_root, finalized_root,
hasParentInForkChoice = false, hasParentInForkChoice = false,
Eth2Digest(), Eth2Digest(),
@ -83,16 +83,11 @@ proc initForkChoiceBackend*(justified_epoch: Epoch,
finalized_epoch finalized_epoch
) )
if err.kind != fcSuccess:
return err("Failed to add finalized block to proto_array: " & $err)
ok(ForkChoiceBackend( ok(ForkChoiceBackend(
proto_array: proto_array, proto_array: proto_array,
)) ))
proc initForkChoice*( proc initForkChoice*(finalizedState: StateData, ): FcResult[ForkChoice] =
finalizedState: StateData,
): Result[ForkChoice, string] =
## Initialize a fork choice context ## Initialize a fork choice context
debug "Initializing fork choice", debug "Initializing fork choice",
state_epoch = finalizedState.data.data.get_current_epoch(), state_epoch = finalizedState.data.data.get_current_epoch(),
@ -122,20 +117,8 @@ func extend[T](s: var seq[T], minLen: int) =
## Extend a sequence so that it can contains at least `minLen` elements. ## Extend a sequence so that it can contains at least `minLen` elements.
## If it's already bigger, the sequence is unmodified. ## If it's already bigger, the sequence is unmodified.
## The extension is zero-initialized ## The extension is zero-initialized
let curLen = s.len if s.len < minLen:
let diff = minLen - curLen
if diff > 0:
# Note: seq has a length and a capacity.
# If the new length is less than the original capacity
# => setLen will not zeroMem
# If the capacity was too small
# => reallocation occurs
# => the fresh buffer is zeroMem-ed
# In the second case our own zeroMem is redundant
# but this should happen rarely as we reuse the buffer
# most of the time
s.setLen(minLen) s.setLen(minLen)
zeroMem(s[curLen].addr, diff * sizeof(T))
func process_attestation*( func process_attestation*(
self: var ForkChoiceBackend, self: var ForkChoiceBackend,
@ -177,14 +160,14 @@ func contains*(self: ForkChoiceBackend, block_root: Eth2Digest): bool =
## In particular, before adding a block, its parent must be known to the fork choice ## In particular, before adding a block, its parent must be known to the fork choice
self.proto_array.indices.contains(block_root) self.proto_array.indices.contains(block_root)
proc get_balances_for_block(self: var Checkpoints, blck: BlockRef, pool: BlockPool): seq[Gwei] = proc get_balances_for_block(self: var Checkpoints, blck: BlockSlot, pool: BlockPool): seq[Gwei] =
pool.withState(pool.balanceState, blck.atSlot(blck.slot)): pool.withState(pool.balanceState, blck):
get_effective_balances(state) get_effective_balances(state)
proc process_state(self: var Checkpoints, proc process_state(self: var Checkpoints,
pool: BlockPool, pool: BlockPool,
state: BeaconState, state: BeaconState,
blck: BlockRef): Result[void, string] = blck: BlockRef) =
trace "Processing state", trace "Processing state",
state_slot = state.slot, state_slot = state.slot,
state_justified = state.current_justified_checkpoint.epoch, state_justified = state.current_justified_checkpoint.epoch,
@ -194,17 +177,16 @@ proc process_state(self: var Checkpoints,
if (state.current_justified_checkpoint.epoch > self.current.justified.epoch) and if (state.current_justified_checkpoint.epoch > self.current.justified.epoch) and
(state.finalized_checkpoint.epoch >= self.current.finalized.epoch): (state.finalized_checkpoint.epoch >= self.current.finalized.epoch):
let justifiedBlck = blck.atSlot( let justifiedBlck = blck.atEpochStart(
state.current_justified_checkpoint.epoch.compute_start_slot_at_epoch) state.current_justified_checkpoint.epoch)
if justifiedBlck.blck.root != state.current_justified_checkpoint.root: doAssert justifiedBlck.blck.root == state.current_justified_checkpoint.root
return err("invalid history?")
let candidate = FFGCheckpoints( let candidate = FFGCheckpoints(
justified: BalanceCheckpoint( justified: BalanceCheckpoint(
blck: justifiedBlck.blck, blck: justifiedBlck.blck,
epoch: state.current_justified_checkpoint.epoch, epoch: state.current_justified_checkpoint.epoch,
balances: self.get_balances_for_block(justifiedBlck.blck, pool), balances: self.get_balances_for_block(justifiedBlck, pool),
), ),
finalized: state.finalized_checkpoint, finalized: state.finalized_checkpoint,
) )
@ -233,8 +215,6 @@ proc process_state(self: var Checkpoints,
# self.balances_cache.process_state(block_root, state)?; # self.balances_cache.process_state(block_root, state)?;
ok()
func compute_slots_since_epoch_start(slot: Slot): uint64 = func compute_slots_since_epoch_start(slot: Slot): uint64 =
slot - compute_start_slot_at_epoch(compute_epoch_at_slot(slot)) slot - compute_start_slot_at_epoch(compute_epoch_at_slot(slot))
@ -262,35 +242,32 @@ proc process_block*(self: var ForkChoiceBackend,
block_root: Eth2Digest, block_root: Eth2Digest,
parent_root: Eth2Digest, parent_root: Eth2Digest,
justified_epoch: Epoch, justified_epoch: Epoch,
finalized_epoch: Epoch): Result[void, string] = finalized_epoch: Epoch): FcResult[void] =
let err = self.proto_array.on_block( self.proto_array.on_block(
block_root, hasParentInForkChoice = true, parent_root, block_root, hasParentInForkChoice = true, parent_root,
justified_epoch, finalized_epoch) justified_epoch, finalized_epoch)
if err.kind != fcSuccess:
return err("process_block_error: " & $err)
ok()
proc process_block*(self: var ForkChoice, proc process_block*(self: var ForkChoice,
pool: BlockPool, pool: BlockPool,
state: BeaconState, state: BeaconState,
blckRef: BlockRef, blckRef: BlockRef,
blck: SomeBeaconBlock, blck: SomeBeaconBlock,
wallSlot: Slot): Result[void, string] = wallSlot: Slot): FcResult[void] =
? process_state(self.checkpoints, pool, state, blckRef) process_state(self.checkpoints, pool, state, blckRef)
# TODO current time
maybe_update(self.checkpoints, wallSlot, pool) maybe_update(self.checkpoints, wallSlot, pool)
for attestation in blck.body.attestations: for attestation in blck.body.attestations:
let targetBlck = pool.dag.getRef(attestation.data.target.root) let targetBlck = pool.dag.getRef(attestation.data.target.root)
if targetBlck.isNil: if targetBlck.isNil:
continue continue
let epochRef =
pool.dag.getEpochRef(targetBlck, attestation.data.target.epoch)
if attestation.data.beacon_block_root in self.backend: if attestation.data.beacon_block_root in self.backend:
let participants = get_attesting_indices( let
epochRef, attestation.data, attestation.aggregation_bits) epochRef =
pool.dag.getEpochRef(targetBlck, attestation.data.target.epoch)
participants = get_attesting_indices(
epochRef, attestation.data, attestation.aggregation_bits)
for validator in participants: for validator in participants:
self.process_attestation( self.process_attestation(
@ -303,47 +280,40 @@ proc process_block*(self: var ForkChoice,
state.current_justified_checkpoint.epoch, state.finalized_checkpoint.epoch state.current_justified_checkpoint.epoch, state.finalized_checkpoint.epoch
) )
{.noSideEffect.}: trace "Integrating block in fork choice",
trace "Integrating block in fork choice", block_root = shortLog(blckRef)
block_root = shortLog(blckRef)
return ok() ok()
func find_head*( proc find_head*(
self: var ForkChoiceBackend, self: var ForkChoiceBackend,
justified_epoch: Epoch, justified_epoch: Epoch,
justified_root: Eth2Digest, justified_root: Eth2Digest,
finalized_epoch: Epoch, finalized_epoch: Epoch,
justified_state_balances: seq[Gwei] justified_state_balances: seq[Gwei]
): Result[Eth2Digest, string] = ): FcResult[Eth2Digest] =
## Returns the new blockchain head ## Returns the new blockchain head
# Compute deltas with previous call # Compute deltas with previous call
# we might want to reuse the `deltas` buffer across calls # we might want to reuse the `deltas` buffer across calls
var deltas = newSeq[Delta](self.proto_array.indices.len) var deltas = newSeq[Delta](self.proto_array.indices.len)
let delta_err = deltas.compute_deltas( ? deltas.compute_deltas(
indices = self.proto_array.indices, indices = self.proto_array.indices,
votes = self.votes, votes = self.votes,
old_balances = self.balances, old_balances = self.balances,
new_balances = justified_state_balances new_balances = justified_state_balances
) )
if delta_err.kind != fcSuccess:
return err("find_head compute_deltas failed: " & $delta_err)
# Apply score changes # Apply score changes
let score_err = self.proto_array.apply_score_changes( ? self.proto_array.apply_score_changes(
deltas, justified_epoch, finalized_epoch deltas, justified_epoch, finalized_epoch
) )
if score_err.kind != fcSuccess:
return err("find_head apply_score_changes failed: " & $score_err)
self.balances = justified_state_balances self.balances = justified_state_balances
# Find the best block # Find the best block
var new_head{.noInit.}: Eth2Digest var new_head{.noInit.}: Eth2Digest
let ghost_err = self.proto_array.find_head(new_head, justified_root) ? self.proto_array.find_head(new_head, justified_root)
if ghost_err.kind != fcSuccess:
return err("find_head failed: " & $ghost_err)
{.noSideEffect.}: {.noSideEffect.}:
debug "Fork choice requested", debug "Fork choice requested",
@ -355,7 +325,7 @@ func find_head*(
return ok(new_head) return ok(new_head)
proc find_head*(self: var ForkChoice, proc find_head*(self: var ForkChoice,
wallSlot: Slot, pool: BlockPool): Result[Eth2Digest, string] = wallSlot: Slot, pool: BlockPool): FcResult[Eth2Digest] =
template remove_alias(blck_root: Eth2Digest): Eth2Digest = template remove_alias(blck_root: Eth2Digest): Eth2Digest =
if blck_root == Eth2Digest(): if blck_root == Eth2Digest():
self.finalizedBlock.root self.finalizedBlock.root
@ -373,14 +343,11 @@ proc find_head*(self: var ForkChoice,
func maybe_prune*( func maybe_prune*(
self: var ForkChoiceBackend, finalized_root: Eth2Digest self: var ForkChoiceBackend, finalized_root: Eth2Digest
): Result[void, string] = ): FcResult[void] =
## Prune blocks preceding the finalized root as they are now unneeded. ## Prune blocks preceding the finalized root as they are now unneeded.
let err = self.proto_array.maybe_prune(finalized_root) self.proto_array.maybe_prune(finalized_root)
if err.kind != fcSuccess:
return err("find_head maybe_pruned failed: " & $err)
return ok()
func prune*(self: var ForkChoice): Result[void, string] = func prune*(self: var ForkChoice): FcResult[void] =
let finalized_root = self.checkpoints.current.finalized.root let finalized_root = self.checkpoints.current.finalized.root
self.backend.maybe_prune(finalized_root) self.backend.maybe_prune(finalized_root)
@ -390,7 +357,7 @@ func compute_deltas(
votes: var openArray[VoteTracker], votes: var openArray[VoteTracker],
old_balances: openarray[Gwei], old_balances: openarray[Gwei],
new_balances: openarray[Gwei] new_balances: openarray[Gwei]
): ForkChoiceError = ): FcResult[void] =
## Update `deltas` ## Update `deltas`
## between old and new balances ## between old and new balances
## between votes ## between votes
@ -429,8 +396,8 @@ func compute_deltas(
if vote.current_root in indices: if vote.current_root in indices:
let index = indices.unsafeGet(vote.current_root) let index = indices.unsafeGet(vote.current_root)
if index >= deltas.len: if index >= deltas.len:
return ForkChoiceError( return err ForkChoiceError(
kind: fcErrInvalidNodeDelta, kind: fcInvalidNodeDelta,
index: index index: index
) )
deltas[index] -= Delta old_balance deltas[index] -= Delta old_balance
@ -440,8 +407,8 @@ func compute_deltas(
if vote.next_root in indices: if vote.next_root in indices:
let index = indices.unsafeGet(vote.next_root) let index = indices.unsafeGet(vote.next_root)
if index >= deltas.len: if index >= deltas.len:
return ForkChoiceError( return err ForkChoiceError(
kind: fcErrInvalidNodeDelta, kind: fcInvalidNodeDelta,
index: index index: index
) )
deltas[index] += Delta new_balance deltas[index] += Delta new_balance
@ -449,7 +416,7 @@ func compute_deltas(
# TODO: is int64 big enough? # TODO: is int64 big enough?
vote.current_root = vote.next_root vote.current_root = vote.next_root
return ForkChoiceSuccess return ok()
# Sanity checks # Sanity checks
# ---------------------------------------------------------------------- # ----------------------------------------------------------------------
@ -487,7 +454,7 @@ when isMainModule:
indices, votes, old_balances, new_balances indices, votes, old_balances, new_balances
) )
doAssert err.kind == fcSuccess, "compute_deltas finished with error: " & $err doAssert err.isOk, "compute_deltas finished with error: " & $err
doAssert deltas == newSeq[Delta](validator_count), "deltas should be zeros" doAssert deltas == newSeq[Delta](validator_count), "deltas should be zeros"
@ -522,7 +489,7 @@ when isMainModule:
indices, votes, old_balances, new_balances indices, votes, old_balances, new_balances
) )
doAssert err.kind == fcSuccess, "compute_deltas finished with error: " & $err doAssert err.isOk, "compute_deltas finished with error: " & $err
for i, delta in deltas.pairs: for i, delta in deltas.pairs:
if i == 0: if i == 0:
@ -561,7 +528,7 @@ when isMainModule:
indices, votes, old_balances, new_balances indices, votes, old_balances, new_balances
) )
doAssert err.kind == fcSuccess, "compute_deltas finished with error: " & $err doAssert err.isOk, "compute_deltas finished with error: " & $err
for i, delta in deltas.pairs: for i, delta in deltas.pairs:
doAssert delta == Delta(Balance), "Each root should have a delta" doAssert delta == Delta(Balance), "Each root should have a delta"
@ -599,7 +566,7 @@ when isMainModule:
indices, votes, old_balances, new_balances indices, votes, old_balances, new_balances
) )
doAssert err.kind == fcSuccess, "compute_deltas finished with error: " & $err doAssert err.isOk, "compute_deltas finished with error: " & $err
for i, delta in deltas.pairs: for i, delta in deltas.pairs:
if i == 0: if i == 0:
@ -647,7 +614,7 @@ when isMainModule:
indices, votes, old_balances, new_balances indices, votes, old_balances, new_balances
) )
doAssert err.kind == fcSuccess, "compute_deltas finished with error: " & $err doAssert err.isOk, "compute_deltas finished with error: " & $err
doAssert deltas[0] == -Delta(Balance)*2, "The 0th block should have lost both balances." doAssert deltas[0] == -Delta(Balance)*2, "The 0th block should have lost both balances."
@ -686,7 +653,7 @@ when isMainModule:
indices, votes, old_balances, new_balances indices, votes, old_balances, new_balances
) )
doAssert err.kind == fcSuccess, "compute_deltas finished with error: " & $err doAssert err.isOk, "compute_deltas finished with error: " & $err
for i, delta in deltas.pairs: for i, delta in deltas.pairs:
if i == 0: if i == 0:
@ -730,7 +697,7 @@ when isMainModule:
indices, votes, old_balances, new_balances indices, votes, old_balances, new_balances
) )
doAssert err.kind == fcSuccess, "compute_deltas finished with error: " & $err doAssert err.isOk, "compute_deltas finished with error: " & $err
doAssert deltas[0] == -Delta(Balance), "Block 1 should have lost only 1 balance" doAssert deltas[0] == -Delta(Balance), "Block 1 should have lost only 1 balance"
doAssert deltas[1] == Delta(Balance)*2, "Block 2 should have gained 2 balances" doAssert deltas[1] == Delta(Balance)*2, "Block 2 should have gained 2 balances"
@ -769,7 +736,7 @@ when isMainModule:
indices, votes, old_balances, new_balances indices, votes, old_balances, new_balances
) )
doAssert err.kind == fcSuccess, "compute_deltas finished with error: " & $err doAssert err.isOk, "compute_deltas finished with error: " & $err
doAssert deltas[0] == -Delta(Balance)*2, "Block 1 should have lost 2 balances" doAssert deltas[0] == -Delta(Balance)*2, "Block 1 should have lost 2 balances"
doAssert deltas[1] == Delta(Balance), "Block 2 should have gained 1 balance" doAssert deltas[1] == Delta(Balance), "Block 2 should have gained 1 balance"

View File

@ -11,6 +11,8 @@ import
# Standard library # Standard library
std/[tables, options], std/[tables, options],
# Status # Status
stew/results,
chronicles, chronicles,
# Internal # Internal
../spec/[datatypes, digest], ../spec/[datatypes, digest],
@ -28,27 +30,26 @@ import
# ---------------------------------------------------------------------- # ----------------------------------------------------------------------
type type
FcErrKind* = enum fcKind* = enum
## Fork Choice Error Kinds ## Fork Choice Error Kinds
fcSuccess fcFinalizedNodeUnknown
fcErrFinalizedNodeUnknown fcJustifiedNodeUnknown
fcErrJustifiedNodeUnknown fcInvalidFinalizedRootCHange
fcErrInvalidFinalizedRootCHange fcInvalidNodeIndex
fcErrInvalidNodeIndex fcInvalidParentIndex
fcErrInvalidParentIndex fcInvalidBestChildIndex
fcErrInvalidBestChildIndex fcInvalidJustifiedIndex
fcErrInvalidJustifiedIndex fcInvalidBestDescendant
fcErrInvalidBestDescendant fcInvalidParentDelta
fcErrInvalidParentDelta fcInvalidNodeDelta
fcErrInvalidNodeDelta fcDeltaUnderflow
fcErrDeltaUnderflow fcIndexUnderflow
fcErrIndexUnderflow fcInvalidDeltaLen
fcErrInvalidDeltaLen fcRevertedFinalizedEpoch
fcErrRevertedFinalizedEpoch fcInvalidBestNode
fcErrInvalidBestNode
# ------------------------- # -------------------------
# TODO: Extra error modes beyond Proto/Lighthouse to be reviewed # TODO: Extra error modes beyond Proto/Lighthouse to be reviewed
fcErrUnknownParent fcUnknownParent
FcUnderflowKind* = enum FcUnderflowKind* = enum
## Fork Choice Overflow Kinds ## Fork Choice Overflow Kinds
@ -61,42 +62,42 @@ type
## Delta indices ## Delta indices
ForkChoiceError* = object ForkChoiceError* = object
case kind*: FcErrKind case kind*: fcKind
of fcSuccess: of fcFinalizedNodeUnknown,
discard fcJustifiedNodeUnknown:
of fcErrFinalizedNodeUnknown,
fcErrJustifiedNodeUnknown:
block_root*: Eth2Digest block_root*: Eth2Digest
of fcErrInvalidFinalizedRootChange: of fcInvalidFinalizedRootChange:
discard discard
of fcErrInvalidNodeIndex, of fcInvalidNodeIndex,
fcErrInvalidParentIndex, fcInvalidParentIndex,
fcErrInvalidBestChildIndex, fcInvalidBestChildIndex,
fcErrInvalidJustifiedIndex, fcInvalidJustifiedIndex,
fcErrInvalidBestDescendant, fcInvalidBestDescendant,
fcErrInvalidParentDelta, fcInvalidParentDelta,
fcErrInvalidNodeDelta, fcInvalidNodeDelta,
fcErrDeltaUnderflow: fcDeltaUnderflow:
index*: Index index*: Index
of fcErrIndexUnderflow: of fcIndexUnderflow:
underflowKind*: FcUnderflowKind underflowKind*: FcUnderflowKind
of fcErrInvalidDeltaLen: of fcInvalidDeltaLen:
deltasLen*: int deltasLen*: int
indicesLen*: int indicesLen*: int
of fcErrRevertedFinalizedEpoch: of fcRevertedFinalizedEpoch:
current_finalized_epoch*: Epoch current_finalized_epoch*: Epoch
new_finalized_epoch*: Epoch new_finalized_epoch*: Epoch
of fcErrInvalidBestNode: of fcInvalidBestNode:
start_root*: Eth2Digest start_root*: Eth2Digest
justified_epoch*: Epoch justified_epoch*: Epoch
finalized_epoch*: Epoch finalized_epoch*: Epoch
head_root*: Eth2Digest head_root*: Eth2Digest
head_justified_epoch*: Epoch head_justified_epoch*: Epoch
head_finalized_epoch*: Epoch head_finalized_epoch*: Epoch
of fcErrUnknownParent: of fcUnknownParent:
child_root*: Eth2Digest child_root*: Eth2Digest
parent_root*: Eth2Digest parent_root*: Eth2Digest
FcResult*[T] = Result[T, ForkChoiceError]
ProtoArray* = object ProtoArray* = object
prune_threshold*: int prune_threshold*: int
justified_epoch*: Epoch justified_epoch*: Epoch
@ -127,8 +128,6 @@ type
best*: FFGCheckpoints best*: FFGCheckpoints
updateAt*: Option[Epoch] updateAt*: Option[Epoch]
const ForkChoiceSuccess* = ForkChoiceError(kind: fcSuccess)
# Fork choice high-level types # Fork choice high-level types
# ---------------------------------------------------------------------- # ----------------------------------------------------------------------

View File

@ -12,6 +12,7 @@ import
std/tables, std/options, std/typetraits, std/tables, std/options, std/typetraits,
# Status libraries # Status libraries
chronicles, chronicles,
stew/results,
# Internal # Internal
../spec/[datatypes, digest], ../spec/[datatypes, digest],
# Fork choice # Fork choice
@ -20,6 +21,8 @@ import
logScope: logScope:
topics = "fork_choice" topics = "fork_choice"
export results
# 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.11.1/specs/phase0/fork-choice.md
# This is a port of https://github.com/sigp/lighthouse/pull/804 # 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 # which is a port of "Proto-Array": https://github.com/protolambda/lmd-ghost
@ -65,9 +68,10 @@ template unsafeGet*[K, V](table: Table[K, V], key: K): V =
# Forward declarations # Forward declarations
# ---------------------------------------------------------------------- # ----------------------------------------------------------------------
func maybe_update_best_child_and_descendant(self: var ProtoArray, parent_index: Index, child_index: Index): ForkChoiceError func maybe_update_best_child_and_descendant(
self: var ProtoArray, parent_index: Index, child_index: Index): FcResult[void]
func node_is_viable_for_head(self: ProtoArray, node: ProtoNode): bool func node_is_viable_for_head(self: ProtoArray, node: ProtoNode): bool
func node_leads_to_viable_head(self: ProtoArray, node: ProtoNode): tuple[viable: bool, err: ForkChoiceError] func node_leads_to_viable_head(self: ProtoArray, node: ProtoNode): FcResult[bool]
# ProtoArray routines # ProtoArray routines
# ---------------------------------------------------------------------- # ----------------------------------------------------------------------
@ -77,7 +81,7 @@ func apply_score_changes*(
deltas: var openarray[Delta], deltas: var openarray[Delta],
justified_epoch: Epoch, justified_epoch: Epoch,
finalized_epoch: Epoch finalized_epoch: Epoch
): ForkChoiceError = ): FcResult[void] =
## Iterate backwards through the array, touching all nodes and their parents ## Iterate backwards through the array, touching all nodes and their parents
## and potentially the best-child of each parent. ## and potentially the best-child of each parent.
## ##
@ -92,8 +96,8 @@ func apply_score_changes*(
## updating if the current node should become the best-child ## updating if the current node should become the best-child
## 4. If required, update the parent's best-descendant with the current node or its best-descendant ## 4. If required, update the parent's best-descendant with the current node or its best-descendant
if deltas.len != self.indices.len: if deltas.len != self.indices.len:
return ForkChoiceError( return err ForkChoiceError(
kind: fcErrInvalidDeltaLen, kind: fcInvalidDeltaLen,
deltasLen: deltas.len, deltasLen: deltas.len,
indicesLen: self.indices.len indicesLen: self.indices.len
) )
@ -115,8 +119,8 @@ func apply_score_changes*(
# and we can probably assume that # and we can probably assume that
# `self.indices.len == self.nodes.len` by construction # `self.indices.len == self.nodes.len` by construction
# and avoid this check in a loop or altogether # and avoid this check in a loop or altogether
return ForkChoiceError( return err ForkChoiceError(
kind: fcErrInvalidNodeDelta, kind: fcInvalidNodeDelta,
index: node_index index: node_index
) )
let node_delta = deltas[node_index] let node_delta = deltas[node_index]
@ -126,8 +130,8 @@ func apply_score_changes*(
# Note that delta can be negative but weight cannot # Note that delta can be negative but weight cannot
let weight = node.weight + node_delta let weight = node.weight + node_delta
if weight < 0: if weight < 0:
return ForkChoiceError( return err ForkChoiceError(
kind: fcErrDeltaUnderflow, kind: fcDeltaUnderflow,
index: node_index index: node_index
) )
node.weight = weight node.weight = weight
@ -139,20 +143,17 @@ func apply_score_changes*(
# and a "no exceptions" (only panics) implementation. # and a "no exceptions" (only panics) implementation.
let parent_index = node.parent.unsafeGet() let parent_index = node.parent.unsafeGet()
if parent_index notin {0..deltas.len-1}: if parent_index notin {0..deltas.len-1}:
return ForkChoiceError( return err ForkChoiceError(
kind: fcErrInvalidParentDelta, kind: fcInvalidParentDelta,
index: parent_index index: parent_index
) )
# Back-propagate the nodes delta to its parent. # Back-propagate the nodes delta to its parent.
deltas[parent_index] += node_delta deltas[parent_index] += node_delta
let err = self.maybe_update_best_child_and_descendant(parent_index, node_index) ? self.maybe_update_best_child_and_descendant(parent_index, node_index)
if err.kind != fcSuccess:
return err
return ForkChoiceSuccess
return ok()
func on_block*( func on_block*(
self: var ProtoArray, self: var ProtoArray,
@ -161,7 +162,7 @@ func on_block*(
parent: Eth2Digest, parent: Eth2Digest,
justified_epoch: Epoch, justified_epoch: Epoch,
finalized_epoch: Epoch finalized_epoch: Epoch
): ForkChoiceError = ): FcResult[void] =
## Register a block with the fork choice ## Register a block with the fork choice
## A block `hasParentInForkChoice` may be false ## A block `hasParentInForkChoice` may be false
## on fork choice initialization: ## on fork choice initialization:
@ -172,22 +173,15 @@ func on_block*(
# If the block is already known, ignore it # If the block is already known, ignore it
if root in self.indices: if root in self.indices:
return ForkChoiceSuccess return ok()
var parent_index: Option[int] var parent_index: Option[int]
if not hasParentInForkChoice: if not hasParentInForkChoice:
# Genesis (but Genesis might not be default(Eth2Digest)) # Genesis (but Genesis might not be default(Eth2Digest))
parent_index = none(int) parent_index = none(int)
elif parent notin self.indices: elif parent notin self.indices:
{.noSideEffect.}: return err ForkChoiceError(
error "Trying to add block with unknown parent", kind: fcUnknownParent,
child_root = shortLog(root),
parent_root = shortLog(parent),
justified_epoch = justified_epoch,
finalized_epoch = finalized_epoch
return ForkChoiceError(
kind: fcErrUnknownParent,
child_root: root, child_root: root,
parent_root: parent parent_root: parent
) )
@ -210,17 +204,15 @@ func on_block*(
self.nodes.add node # TODO: if this is costly, we can setLen + construct the node in-place self.nodes.add node # TODO: if this is costly, we can setLen + construct the node in-place
if parent_index.isSome(): # parent_index is always valid except for Genesis if parent_index.isSome(): # parent_index is always valid except for Genesis
let err = self.maybe_update_best_child_and_descendant(parent_index.unsafeGet(), node_index) ? self.maybe_update_best_child_and_descendant(parent_index.unsafeGet(), node_index)
if err.kind != fcSuccess:
return err
return ForkChoiceSuccess return ok()
func find_head*( func find_head*(
self: var ProtoArray, self: var ProtoArray,
head: var Eth2Digest, head: var Eth2Digest,
justified_root: Eth2Digest justified_root: Eth2Digest
): ForkChoiceError = ): FcResult[void] =
## Follows the best-descendant links to find the best-block (i.e. head-block) ## Follows the best-descendant links to find the best-block (i.e. head-block)
## ##
## ⚠️ Warning ## ⚠️ Warning
@ -229,18 +221,18 @@ func find_head*(
## update the whole tree. ## update the whole tree.
let justified_index = self.indices.getOrFailcase(justified_root): let justified_index = self.indices.getOrFailcase(justified_root):
return ForkChoiceError( return err ForkChoiceError(
kind: fcErrJustifiedNodeUnknown, kind: fcJustifiedNodeUnknown,
block_root: justified_root block_root: justified_root
) )
if justified_index notin {0..self.nodes.len-1}: if justified_index notin {0..self.nodes.len-1}:
return ForkChoiceError( return err ForkChoiceError(
kind: fcErrInvalidJustifiedIndex, kind: fcInvalidJustifiedIndex,
index: justified_index index: justified_index
) )
template justified_node: untyped {.dirty.} = self.nodes[justified_index] template justified_node: untyped = self.nodes[justified_index]
# Alias, IndexError are defects # Alias, IndexError are defects
let best_descendant_index = block: let best_descendant_index = block:
@ -250,17 +242,17 @@ func find_head*(
justified_index justified_index
if best_descendant_index notin {0..self.nodes.len-1}: if best_descendant_index notin {0..self.nodes.len-1}:
return ForkChoiceError( return err ForkChoiceError(
kind: fcErrInvalidBestDescendant, kind: fcInvalidBestDescendant,
index: best_descendant_index index: best_descendant_index
) )
template best_node: untyped {.dirty.} = self.nodes[best_descendant_index] template best_node: untyped = self.nodes[best_descendant_index]
# Alias, IndexError are defects # Alias, IndexError are defects
# Perform a sanity check to ensure the node can be head # Perform a sanity check to ensure the node can be head
if not self.node_is_viable_for_head(best_node): if not self.node_is_viable_for_head(best_node):
return ForkChoiceError( return err ForkChoiceError(
kind: fcErrInvalidBestNode, kind: fcInvalidBestNode,
start_root: justified_root, start_root: justified_root,
justified_epoch: self.justified_epoch, justified_epoch: self.justified_epoch,
finalized_epoch: self.finalized_epoch, finalized_epoch: self.finalized_epoch,
@ -270,7 +262,7 @@ func find_head*(
) )
head = best_node.root head = best_node.root
return ForkChoiceSuccess return ok()
# TODO: pruning can be made cheaper by keeping the new offset as a field # TODO: pruning can be made cheaper by keeping the new offset as a field
# in proto_array instead of scanning the table to substract the offset. # in proto_array instead of scanning the table to substract the offset.
@ -279,7 +271,7 @@ func find_head*(
func maybe_prune*( func maybe_prune*(
self: var ProtoArray, self: var ProtoArray,
finalized_root: Eth2Digest finalized_root: Eth2Digest
): ForkChoiceError = ): FcResult[void] =
## Update the tree with new finalization information. ## Update the tree with new finalization information.
## The tree is pruned if and only if: ## The tree is pruned if and only if:
## - The `finalized_root` and finalized epoch are different from current ## - The `finalized_root` and finalized epoch are different from current
@ -290,26 +282,25 @@ func maybe_prune*(
## - The finalized epoch matches the current one but the finalized root is different ## - The finalized epoch matches the current one but the finalized root is different
## - Internal error due to invalid indices in `self` ## - Internal error due to invalid indices in `self`
let finalized_index = self.indices.getOrFailcase(finalized_root): let finalized_index = self.indices.getOrFailcase(finalized_root):
return ForkChoiceError( return err ForkChoiceError(
kind: fcErrFinalizedNodeUnknown, kind: fcFinalizedNodeUnknown,
block_root: finalized_root block_root: finalized_root
) )
if finalized_index < self.prune_threshold: if finalized_index < self.prune_threshold:
# Pruning small numbers of nodes incurs more overhead than leaving them as is # Pruning small numbers of nodes incurs more overhead than leaving them as is
return ForkChoiceSuccess return ok()
# Remove the `self.indices` key/values for the nodes slated for deletion # Remove the `self.indices` key/values for the nodes slated for deletion
if finalized_index notin {0..self.nodes.len-1}: if finalized_index notin {0..self.nodes.len-1}:
return ForkChoiceError( return err ForkChoiceError(
kind: fcErrInvalidNodeIndex, kind: fcInvalidNodeIndex,
index: finalized_index index: finalized_index
) )
{.noSideEffect.}: trace "Pruning blocks from fork choice",
debug "Pruning blocks from fork choice", finalizedRoot = shortlog(finalized_root),
finalizedRoot = shortlog(finalized_root), pcs = "prune"
pcs = "prune"
for node_index in 0 ..< finalized_index: for node_index in 0 ..< finalized_index:
self.indices.del(self.nodes[node_index].root) self.indices.del(self.nodes[node_index].root)
@ -326,8 +317,8 @@ func maybe_prune*(
for index in self.indices.mvalues(): for index in self.indices.mvalues():
index -= finalized_index index -= finalized_index
if index < 0: if index < 0:
return ForkChoiceError( return err ForkChoiceError(
kind: fcErrIndexUnderflow, kind: fcIndexUnderflow,
underflowKind: fcUnderflowIndices underflowKind: fcUnderflowIndices
) )
@ -345,8 +336,8 @@ func maybe_prune*(
if node.best_child.isSome(): if node.best_child.isSome():
let new_best_child = node.best_child.unsafeGet() - finalized_index let new_best_child = node.best_child.unsafeGet() - finalized_index
if new_best_child < 0: if new_best_child < 0:
return ForkChoiceError( return err ForkChoiceError(
kind: fcErrIndexUnderflow, kind: fcIndexUnderflow,
underflowKind: fcUnderflowBestChild underflowKind: fcUnderflowBestChild
) )
node.best_child = some(new_best_child) node.best_child = some(new_best_child)
@ -354,19 +345,19 @@ func maybe_prune*(
if node.best_descendant.isSome(): if node.best_descendant.isSome():
let new_best_descendant = node.best_descendant.unsafeGet() - finalized_index let new_best_descendant = node.best_descendant.unsafeGet() - finalized_index
if new_best_descendant < 0: if new_best_descendant < 0:
return ForkChoiceError( return err ForkChoiceError(
kind: fcErrIndexUnderflow, kind: fcIndexUnderflow,
underflowKind: fcUnderflowBestDescendant underflowKind: fcUnderflowBestDescendant
) )
node.best_descendant = some(new_best_descendant) node.best_descendant = some(new_best_descendant)
return ForkChoiceSuccess return ok()
func maybe_update_best_child_and_descendant( func maybe_update_best_child_and_descendant(
self: var ProtoArray, self: var ProtoArray,
parent_index: Index, parent_index: Index,
child_index: Index): ForkChoiceError = child_index: Index): Result[void, ForkChoiceError] =
## Observe the parent at `parent_index` with respect to the child at `child_index` and ## Observe the parent at `parent_index` with respect to the child at `child_index` and
## potentiatlly modify the `parent.best_child` and `parent.best_descendant` values ## potentiatlly modify the `parent.best_child` and `parent.best_descendant` values
## ##
@ -380,13 +371,13 @@ func maybe_update_best_child_and_descendant(
## 4. The child is not the best child and does not become the best child ## 4. The child is not the best child and does not become the best child
if child_index notin {0..self.nodes.len-1}: if child_index notin {0..self.nodes.len-1}:
return ForkChoiceError( return err ForkChoiceError(
kind: fcErrInvalidNodeIndex, kind: fcInvalidNodeIndex,
index: child_index index: child_index
) )
if parent_index notin {0..self.nodes.len-1}: if parent_index notin {0..self.nodes.len-1}:
return ForkChoiceError( return err ForkChoiceError(
kind: fcErrInvalidNodeIndex, kind: fcInvalidNodeIndex,
index: parent_index index: parent_index
) )
@ -394,9 +385,7 @@ func maybe_update_best_child_and_descendant(
template child: untyped {.dirty.} = self.nodes[child_index] template child: untyped {.dirty.} = self.nodes[child_index]
template parent: untyped {.dirty.} = self.nodes[parent_index] template parent: untyped {.dirty.} = self.nodes[parent_index]
let (child_leads_to_viable_head, err) = self.node_leads_to_viable_head(child) let child_leads_to_viable_head = ? self.node_leads_to_viable_head(child)
if err.kind != fcSuccess:
return err
let # Aliases to the 3 possible (best_child, best_descendant) tuples let # Aliases to the 3 possible (best_child, best_descendant) tuples
change_to_none = (none(Index), none(Index)) change_to_none = (none(Index), none(Index))
@ -422,15 +411,14 @@ func maybe_update_best_child_and_descendant(
change_to_child change_to_child
else: else:
if best_child_index notin {0..self.nodes.len-1}: if best_child_index notin {0..self.nodes.len-1}:
return ForkChoiceError( return err ForkChoiceError(
kind: fcErrInvalidBestDescendant, kind: fcInvalidBestDescendant,
index: best_child_index index: best_child_index
) )
let best_child = self.nodes[best_child_index] let best_child = self.nodes[best_child_index]
let (best_child_leads_to_viable_head, err) = self.node_leads_to_viable_head(best_child) let best_child_leads_to_viable_head =
if err.kind != fcSuccess: ? self.node_leads_to_viable_head(best_child)
return err
if child_leads_to_viable_head and not best_child_leads_to_viable_head: if child_leads_to_viable_head and not best_child_leads_to_viable_head:
# The child leads to a viable head, but the current best-child doesn't # The child leads to a viable head, but the current best-child doesn't
@ -460,34 +448,28 @@ func maybe_update_best_child_and_descendant(
self.nodes[parent_index].best_child = new_best_child self.nodes[parent_index].best_child = new_best_child
self.nodes[parent_index].best_descendant = new_best_descendant self.nodes[parent_index].best_descendant = new_best_descendant
return ForkChoiceSuccess return ok()
func node_leads_to_viable_head( func node_leads_to_viable_head(
self: ProtoArray, node: ProtoNode self: ProtoArray, node: ProtoNode
): tuple[viable: bool, err: ForkChoiceError] = ): FcResult[bool] =
## Indicates if the node itself or its best-descendant are viable ## Indicates if the node itself or its best-descendant are viable
## for blockchain head ## for blockchain head
let best_descendant_is_viable_for_head = block: let best_descendant_is_viable_for_head = block:
if node.best_descendant.isSome(): if node.best_descendant.isSome():
let best_descendant_index = node.best_descendant.unsafeGet() let best_descendant_index = node.best_descendant.unsafeGet()
if best_descendant_index notin {0..self.nodes.len-1}: if best_descendant_index notin {0..self.nodes.len-1}:
return ( return err ForkChoiceError(
false, kind: fcInvalidBestDescendant,
ForkChoiceError(
kind: fcErrInvalidBestDescendant,
index: best_descendant_index index: best_descendant_index
) )
)
let best_descendant = self.nodes[best_descendant_index] let best_descendant = self.nodes[best_descendant_index]
self.node_is_viable_for_head(best_descendant) self.node_is_viable_for_head(best_descendant)
else: else:
false false
return ( return ok(best_descendant_is_viable_for_head or
best_descendant_is_viable_for_head or self.node_is_viable_for_head(node))
self.node_is_viable_for_head(node),
ForkChoiceSuccess
)
func node_is_viable_for_head(self: ProtoArray, node: ProtoNode): bool = func node_is_viable_for_head(self: ProtoArray, node: ProtoNode): bool =
## This is the equivalent of `filter_block_tree` function in eth2 spec ## This is the equivalent of `filter_block_tree` function in eth2 spec

View File

@ -16,7 +16,7 @@ import
# Local modules # Local modules
spec/[datatypes, digest, crypto, validator, beaconstate, helpers], spec/[datatypes, digest, crypto, validator, beaconstate, helpers],
block_pool, ssz/merkleization, block_pool, block_pools/candidate_chains, ssz/merkleization,
beacon_node_common, beacon_node_types, beacon_node_common, beacon_node_types,
validator_duties, eth2_network, validator_duties, eth2_network,
spec/eth2_apis/callsigs_types, spec/eth2_apis/callsigs_types,
@ -160,9 +160,8 @@ proc stateIdToBlockSlot(node: BeaconNode, stateId: string): BlockSlot =
of "finalized": of "finalized":
node.blockPool.finalizedHead node.blockPool.finalizedHead
of "justified": of "justified":
node.blockPool.head.atSlot( node.blockPool.head.atEpochStart(
node.blockPool.headState.data.data.current_justified_checkpoint. node.blockPool.headState.data.data.current_justified_checkpoint.epoch)
epoch.compute_start_slot_at_epoch)
else: else:
if stateId.startsWith("0x"): if stateId.startsWith("0x"):
let blckRoot = parseRoot(stateId) let blckRoot = parseRoot(stateId)
@ -353,7 +352,7 @@ proc installValidatorApiHandlers*(rpcServer: RpcServer, node: BeaconNode) =
debug "post_v1_validator_duties_attester", epoch = epoch debug "post_v1_validator_duties_attester", epoch = epoch
let head = node.doChecksAndGetCurrentHead(epoch) let head = node.doChecksAndGetCurrentHead(epoch)
let attestationHead = head.atSlot(compute_start_slot_at_epoch(epoch)) let attestationHead = head.atEpochStart(epoch)
node.blockPool.withState(node.blockPool.tmpState, attestationHead): node.blockPool.withState(node.blockPool.tmpState, attestationHead):
for pubkey in public_keys: for pubkey in public_keys:
let idx = state.validators.asSeq.findIt(it.pubKey == pubkey) let idx = state.validators.asSeq.findIt(it.pubKey == pubkey)

View File

@ -54,17 +54,11 @@ proc addLocalValidator*(node: BeaconNode,
node.attachedValidators.addLocalValidator(pubKey, privKey) node.attachedValidators.addLocalValidator(pubKey, privKey)
proc addLocalValidators*(node: BeaconNode) {.async.} = proc addLocalValidators*(node: BeaconNode) =
let for validatorKey in node.config.validatorKeys:
head = node.blockPool.head node.addLocalValidator node.blockPool.headState.data.data, validatorKey
node.blockPool.withState(node.blockPool.tmpState, head.atSlot(head.slot)): info "Local validators attached ", count = node.attachedValidators.count
for validatorKey in node.config.validatorKeys:
node.addLocalValidator state, validatorKey
# Allow some network events to be processed:
await sleepAsync(0.seconds)
info "Local validators attached ", count = node.attachedValidators.count
func getAttachedValidator*(node: BeaconNode, func getAttachedValidator*(node: BeaconNode,
state: BeaconState, state: BeaconState,
@ -116,12 +110,10 @@ proc sendAttestation*(node: BeaconNode, attestation: Attestation) =
debug "Attempt to send attestation without corresponding block" debug "Attempt to send attestation without corresponding block"
return return
node.blockPool.withEpochState( node.sendAttestation(
node.blockPool.tmpState, attestation,
BlockSlot(blck: attestationBlck, slot: attestation.data.slot)): count_active_validators(
node.sendAttestation( node.blockPool.dag.getEpochRef(attestationBlck, attestation.data.target.epoch)))
attestation,
count_active_validators(blck.getEpochInfo(state)))
proc createAndSendAttestation(node: BeaconNode, proc createAndSendAttestation(node: BeaconNode,
fork: Fork, fork: Fork,

View File

@ -20,7 +20,7 @@ asyncTest "connect two nodes":
var n1PersistentAddress = c1.getPersistenBootstrapAddr( var n1PersistentAddress = c1.getPersistenBootstrapAddr(
ValidIpAddress.init("127.0.0.1"), Port c1.tcpPort) ValidIpAddress.init("127.0.0.1"), Port c1.tcpPort)
var n1 = await createEth2Node(c1, ENRForkID()) var n1 = createEth2Node(c1, ENRForkID())
echo "Node 1 persistent address: ", n1PersistentAddress echo "Node 1 persistent address: ", n1PersistentAddress
@ -34,7 +34,7 @@ asyncTest "connect two nodes":
c2.dataDir = OutDir(tempDir / "node-2") c2.dataDir = OutDir(tempDir / "node-2")
c2.tcpPort = 50001 c2.tcpPort = 50001
c2.nat = "none" c2.nat = "none"
var n2 = await createEth2Node(c2, ENRForkID()) var n2 = createEth2Node(c2, ENRForkID())
await n2.startLookingForPeers(@[n1PersistentAddress]) await n2.startLookingForPeers(@[n1PersistentAddress])