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:
parent
55607f107f
commit
c5fecd472f
|
@ -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,
|
||||||
|
|
|
@ -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`.
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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)
|
||||||
|
|
|
@ -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"
|
||||||
|
|
|
@ -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
|
||||||
# ----------------------------------------------------------------------
|
# ----------------------------------------------------------------------
|
||||||
|
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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)
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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])
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue