use BeaconTime instead of Slot in fork choice (#3138)

* use v1.1.6 test vectors; use BeaconTime instead of Slot in fork choice

* tick through every slot at least once

* use div INTERVALS_PER_SLOT and use precomputed constants of them

* use correct (even if numerically equal) constant
This commit is contained in:
tersec 2021-12-21 18:56:08 +00:00 committed by GitHub
parent 0d4e49f946
commit 1a6a56bdb1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 167 additions and 100 deletions

View File

@ -401,4 +401,4 @@ OK: 1/1 Fail: 0/1 Skip: 0/1
OK: 1/1 Fail: 0/1 Skip: 0/1
---TOTAL---
OK: 216/218 Fail: 0/218 Skip: 2/218
OK: 217/219 Fail: 0/219 Skip: 2/219

View File

@ -40,13 +40,16 @@ const
# Offsets from the start of the slot to when the corresponding message should
# be sent
# https://github.com/ethereum/consensus-specs/blob/v1.1.6/specs/phase0/validator.md#attesting
attestationSlotOffset* = seconds(SECONDS_PER_SLOT.int) div 3
attestationSlotOffset* = seconds(SECONDS_PER_SLOT.int) div INTERVALS_PER_SLOT
# https://github.com/ethereum/consensus-specs/blob/v1.1.6/specs/phase0/validator.md#broadcast-aggregate
aggregateSlotOffset* = seconds(SECONDS_PER_SLOT.int) * 2 div 3
aggregateSlotOffset* =
seconds(SECONDS_PER_SLOT.int) * 2 div INTERVALS_PER_SLOT
# https://github.com/ethereum/consensus-specs/blob/v1.1.6/specs/altair/validator.md#prepare-sync-committee-message
syncCommitteeMessageSlotOffset* = seconds(SECONDS_PER_SLOT.int) div 3
syncCommitteeMessageSlotOffset* =
seconds(SECONDS_PER_SLOT.int) div INTERVALS_PER_SLOT
# https://github.com/ethereum/consensus-specs/blob/v1.1.6/specs/altair/validator.md#broadcast-sync-committee-contribution
syncContributionSlotOffset* = seconds(SECONDS_PER_SLOT.int) * 2 div 3
syncContributionSlotOffset* =
seconds(SECONDS_PER_SLOT.int) * 2 div INTERVALS_PER_SLOT
proc init*(T: type BeaconClock, genesis_time: uint64): T =
# ~290 billion years into the future

View File

@ -137,7 +137,7 @@ proc init*(T: type AttestationPool, dag: ChainDAGRef,
epochRef = dag.getEpochRef(blckRef, blckRef.slot.epoch)
withBlck(dag.get(blckRef).data):
forkChoice.process_block(
dag, epochRef, blckRef, blck.message, blckRef.slot)
dag, epochRef, blckRef, blck.message, blckRef.slot.toBeaconTime)
doAssert status.isOk(), "Error in preloading the fork choice: " & $status.error
@ -158,10 +158,10 @@ proc init*(T: type AttestationPool, dag: ChainDAGRef,
proc addForkChoiceVotes(
pool: var AttestationPool, slot: Slot,
attesting_indices: openArray[ValidatorIndex], block_root: Eth2Digest,
wallSlot: Slot) =
wallTime: BeaconTime) =
# Add attestation votes to fork choice
if (let v = pool.forkChoice.on_attestation(
pool.dag, slot, block_root, attesting_indices, wallSlot);
pool.dag, slot, block_root, attesting_indices, wallTime);
v.isErr):
# This indicates that the fork choice and the chain dag are out of sync -
# this is most likely the result of a bug, but we'll try to keep going -
@ -325,7 +325,7 @@ proc addAttestation*(pool: var AttestationPool,
attestation: Attestation,
attesting_indices: openArray[ValidatorIndex],
signature: CookedSig,
wallSlot: Slot) =
wallTime: BeaconTime) =
## Add an attestation to the pool, assuming it's been validated already.
##
## Assuming the votes in the attestation have not already been seen, the
@ -337,7 +337,7 @@ proc addAttestation*(pool: var AttestationPool,
doAssert attestation.signature == signature.toValidatorSig(),
"Deserialized signature must match the one in the attestation"
updateCurrent(pool, wallSlot)
updateCurrent(pool, wallTime.slotOrZero)
let candidateIdx = pool.candidateIdx(attestation.data.slot)
if candidateIdx.isNone:
@ -365,7 +365,7 @@ proc addAttestation*(pool: var AttestationPool,
pool.addForkChoiceVotes(
attestation.data.slot, attesting_indices,
attestation.data.beacon_block_root, wallSlot)
attestation.data.beacon_block_root, wallTime)
# Send notification about new attestation via callback.
if not(isNil(pool.onAttestationAdded)):
@ -375,10 +375,10 @@ proc addForkChoice*(pool: var AttestationPool,
epochRef: EpochRef,
blckRef: BlockRef,
blck: ForkyTrustedBeaconBlock,
wallSlot: Slot) =
wallTime: BeaconTime) =
## Add a verified block to the fork choice context
let state = pool.forkChoice.process_block(
pool.dag, epochRef, blckRef, blck, wallSlot)
pool.dag, epochRef, blckRef, blck, wallTime)
if state.isErr:
# This indicates that the fork choice and the chain dag are out of sync -
@ -712,10 +712,10 @@ proc getAggregatedAttestation*(pool: var AttestationPool,
res
proc selectHead*(pool: var AttestationPool, wallSlot: Slot): BlockRef =
proc selectHead*(pool: var AttestationPool, wallTime: BeaconTime): BlockRef =
## Trigger fork choice and returns the new head block.
## Can return `nil`
let newHead = pool.forkChoice.get_head(pool.dag, wallSlot)
let newHead = pool.forkChoice.get_head(pool.dag, wallTime)
if newHead.isErr:
error "Couldn't select head", err = newHead.error

View File

@ -13,6 +13,7 @@ import
# Status libraries
stew/results, chronicles,
# Internal
../beacon_clock,
../spec/[beaconstate, helpers],
../spec/datatypes/[phase0, altair, merge],
# Fork choice
@ -92,13 +93,26 @@ func extend[T](s: var seq[T], minLen: int) =
func compute_slots_since_epoch_start(slot: Slot): uint64 =
slot - slot.epoch().compute_start_slot_at_epoch()
proc on_tick(self: var Checkpoints, dag: ChainDAGRef, time: Slot): FcResult[void] =
func on_tick*(self: var Checkpoints, time: BeaconTime): FcResult[void] =
if self.time > time:
return err ForkChoiceError(kind: fcInconsistentTick)
let newEpoch = self.time.epoch() != time.epoch()
# Reset store.proposer_boost_root if this is a new slot
if time.slotOrZero > self.time.slotOrZero:
self.proposer_boost_root = default(Eth2Digest)
self.time = time
ok()
proc on_tick(self: var Checkpoints, dag: ChainDAGRef, time: BeaconTime):
FcResult[void] =
let prev_time = self.time
? self.on_tick(time)
let newEpoch = prev_time.slotOrZero.epoch() != time.slotOrZero.epoch()
if newEpoch and
self.best_justified.epoch > self.justified.checkpoint.epoch:
let blck = dag.getRef(self.best_justified.root)
@ -117,10 +131,17 @@ proc on_tick(self: var Checkpoints, dag: ChainDAGRef, time: Slot): FcResult[void
func process_attestation_queue(self: var ForkChoice) {.gcsafe.}
proc update_time(self: var ForkChoice, dag: ChainDAGRef, time: Slot): FcResult[void] =
proc update_time(self: var ForkChoice, dag: ChainDAGRef, time: BeaconTime):
FcResult[void] =
const step_size = seconds(SECONDS_PER_SLOT.int)
if time > self.checkpoints.time:
while time > self.checkpoints.time:
? on_tick(self.checkpoints, dag, self.checkpoints.time + 1)
# Call on_tick at least once per slot.
while time >= self.checkpoints.time + step_size:
? on_tick(self.checkpoints, dag, self.checkpoints.time + step_size)
if self.checkpoints.time < time:
# Might create two ticks for the last slot.
? on_tick(self.checkpoints, dag, time)
self.process_attestation_queue() # Only run if time changed!
@ -153,7 +174,7 @@ func process_attestation*(
func process_attestation_queue(self: var ForkChoice) =
self.queuedAttestations.keepItIf:
if it.slot < self.checkpoints.time:
if it.slot < self.checkpoints.time.slotOrZero:
for validator_index in it.attesting_indices:
self.backend.process_attestation(
validator_index, it.block_root, it.slot.epoch())
@ -175,14 +196,14 @@ proc on_attestation*(
attestation_slot: Slot,
beacon_block_root: Eth2Digest,
attesting_indices: openArray[ValidatorIndex],
wallSlot: Slot
wallTime: BeaconTime
): FcResult[void] =
? self.update_time(dag, wallSlot)
? self.update_time(dag, wallTime)
if beacon_block_root == Eth2Digest():
return ok()
if attestation_slot < self.checkpoints.time:
if attestation_slot < self.checkpoints.time.slotOrZero:
for validator_index in attesting_indices:
# attestation_slot and target epoch must match, per attestation rules
self.backend.process_attestation(
@ -202,7 +223,8 @@ func should_update_justified_checkpoint(
self: var Checkpoints,
dag: ChainDAGRef,
epochRef: EpochRef): FcResult[bool] =
if compute_slots_since_epoch_start(self.time) < SAFE_SLOTS_TO_UPDATE_JUSTIFIED:
if compute_slots_since_epoch_start(self.time.slotOrZero) <
SAFE_SLOTS_TO_UPDATE_JUSTIFIED:
return ok(true)
let
@ -300,8 +322,8 @@ proc process_block*(self: var ForkChoice,
epochRef: EpochRef,
blckRef: BlockRef,
blck: ReallyAnyBeaconBlock,
wallSlot: Slot): FcResult[void] =
? update_time(self, dag, wallSlot)
wallTime: BeaconTime): FcResult[void] =
? update_time(self, dag, wallTime)
? process_state(self.checkpoints, dag, epochRef, blckRef)
let committees_per_slot = get_committee_count_per_slot(epochRef)
@ -321,6 +343,15 @@ proc process_block*(self: var ForkChoice,
attestation.data.beacon_block_root,
attestation.data.target.epoch)
# Add proposer score boost if the block is timely
let
time_into_slot =
self.checkpoints.time - self.checkpoints.time.slotOrZero.toBeaconTime
is_before_attesting_interval = time_into_slot < attestationSlotOffset
if self.checkpoints.time.slotOrZero == blck.slot and
is_before_attesting_interval:
self.checkpoints.proposer_boost_root = blckRef.root
? process_block(
self.backend, blckRef.root, blck.parent_root,
epochRef.current_justified_checkpoint,
@ -373,8 +404,8 @@ func find_head*(
# https://github.com/ethereum/consensus-specs/blob/v0.12.1/specs/phase0/fork-choice.md#get_head
proc get_head*(self: var ForkChoice,
dag: ChainDAGRef,
wallSlot: Slot): FcResult[Eth2Digest] =
? self.update_time(dag, wallSlot)
wallTime: BeaconTime): FcResult[Eth2Digest] =
? self.update_time(dag, wallTime)
self.backend.find_head(
self.checkpoints.justified.checkpoint,

View File

@ -15,6 +15,7 @@ import
chronicles,
# Internal
../beacon_clock,
../spec/datatypes/base,
../consensus_object_pools/block_pools_types
@ -108,10 +109,11 @@ type
balances*: seq[Gwei]
Checkpoints* = object
time*: Slot
time*: BeaconTime
justified*: BalanceCheckpoint
finalized*: Checkpoint
best_justified*: Checkpoint
proposer_boost_root*: Eth2Digest
# Fork choice high-level types
# ----------------------------------------------------------------------

View File

@ -185,7 +185,7 @@ proc storeBlock*(
blckRef: BlockRef, trustedBlock: Trusted, epochRef: EpochRef):
# Callback add to fork choice if valid
attestationPool[].addForkChoice(
epochRef, blckRef, trustedBlock.message, wallSlot)
epochRef, blckRef, trustedBlock.message, wallTime)
vm[].registerBeaconBlock(
src, wallTime, trustedBlock.message)
@ -223,7 +223,7 @@ proc storeBlock*(
let storeBlockTick = Moment.now()
# Eagerly update head: the incoming block "should" get selected
self.consensusManager[].updateHead(wallSlot)
self.consensusManager[].updateHead(wallTime.slotOrZero)
let
updateHeadTick = Moment.now()

View File

@ -10,6 +10,7 @@
import
chronicles, chronos,
../spec/datatypes/base,
../beacon_clock,
../consensus_object_pools/[blockchain_dag, block_quarantine, attestation_pool]
# TODO: Move to "consensus_object_pools" folder
@ -79,7 +80,7 @@ proc updateHead*(self: var ConsensusManager, wallSlot: Slot) =
## `pruneFinalized` must be called for pruning.
# Grab the new head according to our latest attestation data
let newHead = self.attestationPool[].selectHead(wallSlot)
let newHead = self.attestationPool[].selectHead(wallSlot.toBeaconTime)
if newHead.isNil():
warn "Head selection failed, using previous head",
head = shortLog(self.dag.head), wallSlot

View File

@ -257,8 +257,8 @@ proc attestationValidator*(
self: ref Eth2Processor, src: MsgSource,
attestation: Attestation, subnet_id: SubnetId,
checkSignature: bool = true): Future[ValidationRes] {.async.} =
let wallTime = self.getCurrentBeaconTime()
var (afterGenesis, wallSlot) = wallTime.toSlot()
var wallTime = self.getCurrentBeaconTime()
let (afterGenesis, wallSlot) = wallTime.toSlot()
logScope:
attestation = shortLog(attestation)
@ -279,7 +279,7 @@ proc attestationValidator*(
self.batchCrypto, attestation, wallTime, subnet_id, checkSignature)
return if v.isOk():
# Due to async validation the wallSlot here might have changed
wallSlot = self.getCurrentBeaconTime().slotOrZero()
wallTime = self.getCurrentBeaconTime()
let (attester_index, sig) = v.get()
@ -287,7 +287,7 @@ proc attestationValidator*(
trace "Attestation validated"
self.attestationPool[].addAttestation(
attestation, [attester_index], sig, wallSlot)
attestation, [attester_index], sig, wallTime)
self.validatorMonitor[].registerAttestation(
src, wallTime, attestation, attester_index)
@ -339,7 +339,8 @@ proc aggregateValidator*(
trace "Aggregate validated"
self.attestationPool[].addAttestation(
signedAggregateAndProof.message.aggregate, attesting_indices, sig, wallSlot)
signedAggregateAndProof.message.aggregate, attesting_indices, sig,
wallTime)
self.validatorMonitor[].registerAggregate(
src, wallTime, signedAggregateAndProof, attesting_indices)

View File

@ -335,7 +335,7 @@ proc publishAttestationsAndAggregates(service: AttestationServiceRef,
let aggregateTime =
# chronos.Duration substraction could not return negative value, in such
# case it will return `ZeroDuration`.
vc.beaconClock.durationToNextSlot() - seconds(int64(SECONDS_PER_SLOT) div 3)
vc.beaconClock.durationToNextSlot() - OneThirdDuration
if aggregateTime != ZeroDuration:
await sleepAsync(aggregateTime)

View File

@ -114,7 +114,7 @@ const
DefaultDutyAndProof* = DutyAndProof(epoch: Epoch(0xFFFF_FFFF_FFFF_FFFF'u64))
SlotDuration* = int64(SECONDS_PER_SLOT).seconds
EpochDuration* = int64(SLOTS_PER_EPOCH * SECONDS_PER_SLOT).seconds
OneThirdDuration* = int64(SECONDS_PER_SLOT div 3).seconds
OneThirdDuration* = int64(SECONDS_PER_SLOT div INTERVALS_PER_SLOT).seconds
proc `$`*(bn: BeaconNodeServerRef): string =
if bn.ident.isSome():

View File

@ -137,7 +137,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
data: data,
aggregation_bits: aggregation_bits,
signature: sig.toValidatorSig()
), [validatorIdx], sig, data.slot)
), [validatorIdx], sig, data.slot.toBeaconTime)
proc handleSyncCommitteeActions(slot: Slot) =
type
@ -150,8 +150,8 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
syncCommittee = @(dag.syncCommitteeParticipants(slot + 1))
genesis_validators_root = dag.genesisValidatorsRoot
fork = dag.forkAtEpoch(slot.epoch)
messagesTime = slot.toBeaconTime(seconds(SECONDS_PER_SLOT div 3))
contributionsTime = slot.toBeaconTime(seconds(2 * SECONDS_PER_SLOT div 3))
messagesTime = slot.toBeaconTime(attestationSlotOffset)
contributionsTime = slot.toBeaconTime(syncContributionSlotOffset)
var aggregators: seq[Aggregator]
@ -309,7 +309,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
epochRef: EpochRef):
# Callback add to fork choice if valid
attPool.addForkChoice(
epochRef, blckRef, signedBlock.message, blckRef.slot)
epochRef, blckRef, signedBlock.message, blckRef.slot.toBeaconTime)
blck() = added[]
dag.updateHead(added[], quarantine[])
@ -329,7 +329,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
epochRef: EpochRef):
# Callback add to fork choice if valid
attPool.addForkChoice(
epochRef, blckRef, signedBlock.message, blckRef.slot)
epochRef, blckRef, signedBlock.message, blckRef.slot.toBeaconTime)
blck() = added[]
dag.updateHead(added[], quarantine[])
@ -349,7 +349,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
epochRef: EpochRef):
# Callback add to fork choice if valid
attPool.addForkChoice(
epochRef, blckRef, signedBlock.message, blckRef.slot)
epochRef, blckRef, signedBlock.message, blckRef.slot.toBeaconTime)
blck() = added[]
dag.updateHead(added[], quarantine[])

View File

@ -19,7 +19,7 @@ import
base,
phase0, altair, merge],
../../beacon_chain/fork_choice/[fork_choice, fork_choice_types],
../../beacon_chain/beacon_chain_db,
../../beacon_chain/[beacon_chain_db, beacon_clock],
../../beacon_chain/consensus_object_pools/[
blockchain_dag, block_clearance, spec_cache],
# Third-party
@ -164,11 +164,11 @@ proc stepOnBlock(
state: var StateData,
stateCache: var StateCache,
signedBlock: ForkySignedBeaconBlock,
time: Slot): Result[BlockRef, BlockError] =
time: BeaconTime): Result[BlockRef, BlockError] =
# 1. Move state to proper slot.
dag.updateStateData(
state,
dag.head.atSlot(time),
dag.head.atSlot(time.slotOrZero),
save = false,
stateCache
)
@ -201,9 +201,9 @@ proc stepOnAttestation(
dag: ChainDagRef,
fkChoice: ref ForkChoice,
att: Attestation,
time: Slot): FcResult[void] =
let epochRef = dag.getEpochRef(dag.head, time.compute_epoch_at_slot())
time: BeaconTime): FcResult[void] =
let epochRef =
dag.getEpochRef(dag.head, time.slotOrZero.compute_epoch_at_slot())
let attesters = epochRef.get_attesting_indices(att.data, att.aggregation_bits)
let status = fkChoice[].on_attestation(
@ -218,13 +218,13 @@ proc stepChecks(
checks: JsonNode,
dag: ChainDagRef,
fkChoice: ref ForkChoice,
time: Slot
time: BeaconTime
) =
doAssert checks.len >= 1, "No checks found"
for check, val in checks:
if check == "time":
doAssert time == Slot(val.getInt())
doAssert fkChoice.checkpoints.time == time
doAssert time.Duration == val.getInt().seconds
doAssert fkChoice.checkpoints.time.slotOrZero == time.slotOrZero
elif check == "head":
let headRoot = fkChoice[].get_head(dag, time).get()
let headRef = dag.getRef(headRoot)
@ -249,8 +249,8 @@ proc stepChecks(
doAssert checkpointEpoch == Epoch(val["epoch"].getInt())
doAssert checkpointRoot == Eth2Digest.fromHex(val["root"].getStr())
elif check == "proposer_boost_root":
# TODO needs fork choice to know about BeaconTime
discard
doAssert fkChoice.checkpoints.proposer_boost_root ==
Eth2Digest.fromHex(val.getStr())
elif check == "genesis_time":
# The fork choice is pruned regularly
# and does not store the genesis time,
@ -300,7 +300,8 @@ proc runTest(path: string, fork: BeaconBlockFork) =
for step in steps:
case step.kind
of opOnTick:
time = Slot(step.tick)
time = step.tick.seconds.BeaconTime
doAssert stores.fkChoice.checkpoints.on_tick(time).isOk
of opOnBlock:
withBlck(step.blk):
let status = stepOnBlock(
@ -328,7 +329,7 @@ suite "Ethereum Foundation - ForkChoice" & preset():
# "Ensure the head is still 4 whilst the justified epoch is 0."
"on_block_future_block",
# TODO needs fork choice to know about BeaconTime
# TODO needs the actual proposer boost enabled
"proposer_boost_correct_head"
]

View File

@ -21,6 +21,7 @@ import
block_quarantine, blockchain_dag, block_clearance, attestation_pool],
../beacon_chain/spec/datatypes/phase0,
../beacon_chain/spec/[beaconstate, helpers, state_transition, validator],
../beacon_chain/beacon_clock,
# Test utilities
./testutil, ./testdbutil, ./testblockutil
@ -84,7 +85,7 @@ suite "Attestation pool processing" & preset():
pool[].addAttestation(
attestation, @[bc0[0]], attestation.loadSig,
attestation.data.slot)
attestation.data.slot.toBeaconTime)
check:
# Added attestation, should get it back
@ -136,7 +137,7 @@ suite "Attestation pool processing" & preset():
pool[].getAttestationsForBlock(state.data, cache) == []
pool[].addAttestation(
att1, @[bc1[0]], att1.loadSig, att1.data.slot)
att1, @[bc1[0]], att1.loadSig, att1.data.slot.toBeaconTime)
check:
# but new ones should go in
@ -145,7 +146,7 @@ suite "Attestation pool processing" & preset():
let
att2 = makeAttestation(state[].data, root1, bc1[1], cache)
pool[].addAttestation(
att2, @[bc1[1]], att2.loadSig, att2.data.slot)
att2, @[bc1[1]], att2.loadSig, att2.data.slot.toBeaconTime)
let
combined = pool[].getAttestationsForBlock(state.data, cache)
@ -156,7 +157,8 @@ suite "Attestation pool processing" & preset():
combined[0].aggregation_bits.countOnes() == 2
pool[].addAttestation(
combined[0], @[bc1[1], bc1[0]], combined[0].loadSig, combined[0].data.slot)
combined[0], @[bc1[1], bc1[0]], combined[0].loadSig,
combined[0].data.slot.toBeaconTime)
check:
# readding the combined attestation shouldn't have an effect
@ -166,7 +168,7 @@ suite "Attestation pool processing" & preset():
# Someone votes for a different root
att3 = makeAttestation(state[].data, Eth2Digest(), bc1[2], cache)
pool[].addAttestation(
att3, @[bc1[2]], att3.loadSig, att3.data.slot)
att3, @[bc1[2]], att3.loadSig, att3.data.slot.toBeaconTime)
check:
# We should now get both attestations for the block, but the aggregate
@ -181,7 +183,7 @@ suite "Attestation pool processing" & preset():
# Someone votes for a different root
att4 = makeAttestation(state[].data, Eth2Digest(), bc1[2], cache)
pool[].addAttestation(
att4, @[bc1[2]], att3.loadSig, att3.data.slot)
att4, @[bc1[2]], att3.loadSig, att3.data.slot.toBeaconTime)
test "Working with aggregates" & preset():
let
@ -200,8 +202,10 @@ suite "Attestation pool processing" & preset():
att0.combine(att2)
att1.combine(att2)
pool[].addAttestation(att0, @[bc0[0], bc0[2]], att0.loadSig, att0.data.slot)
pool[].addAttestation(att1, @[bc0[1], bc0[2]], att1.loadSig, att1.data.slot)
pool[].addAttestation(
att0, @[bc0[0], bc0[2]], att0.loadSig, att0.data.slot.toBeaconTime)
pool[].addAttestation(
att1, @[bc0[1], bc0[2]], att1.loadSig, att1.data.slot.toBeaconTime)
check:
process_slots(
@ -215,7 +219,8 @@ suite "Attestation pool processing" & preset():
pool[].getAggregatedAttestation(1.Slot, 0.CommitteeIndex).isSome()
# Add in attestation 3 - both aggregates should now have it added
pool[].addAttestation(att3, @[bc0[3]], att3.loadSig, att3.data.slot)
pool[].addAttestation(
att3, @[bc0[3]], att3.loadSig, att3.data.slot.toBeaconTime)
block:
let attestations = pool[].getAttestationsForBlock(state.data, cache)
@ -227,7 +232,8 @@ suite "Attestation pool processing" & preset():
# Add in attestation 0 as single - attestation 1 is now a superset of the
# aggregates in the pool, so everything else should be removed
pool[].addAttestation(att0x, @[bc0[0]], att0x.loadSig, att0x.data.slot)
pool[].addAttestation(
att0x, @[bc0[0]], att0x.loadSig, att0x.data.slot.toBeaconTime)
block:
let attestations = pool[].getAttestationsForBlock(state.data, cache)
@ -248,7 +254,8 @@ suite "Attestation pool processing" & preset():
for j in 0..<bc0.len():
root.data[8..<16] = toBytesBE(j.uint64)
var att = makeAttestation(state[].data, root, bc0[j], cache)
pool[].addAttestation(att, @[bc0[j]], att.loadSig, att.data.slot)
pool[].addAttestation(
att, @[bc0[j]], att.loadSig, att.data.slot.toBeaconTime)
inc attestations
check:
@ -285,9 +292,11 @@ suite "Attestation pool processing" & preset():
# test reverse order
pool[].addAttestation(
attestation1, @[bc1[0]], attestation1.loadSig, attestation1.data.slot)
attestation1, @[bc1[0]], attestation1.loadSig,
attestation1.data.slot.toBeaconTime)
pool[].addAttestation(
attestation0, @[bc0[0]], attestation0.loadSig, attestation0.data.slot)
attestation0, @[bc0[0]], attestation0.loadSig,
attestation0.data.slot.toBeaconTime)
discard process_slots(
defaultRuntimeConfig, state.data,
@ -310,9 +319,11 @@ suite "Attestation pool processing" & preset():
makeAttestation(state[].data, state.blck.root, bc0[1], cache)
pool[].addAttestation(
attestation0, @[bc0[0]], attestation0.loadSig, attestation0.data.slot)
attestation0, @[bc0[0]], attestation0.loadSig,
attestation0.data.slot.toBeaconTime)
pool[].addAttestation(
attestation1, @[bc0[1]], attestation1.loadSig, attestation1.data.slot)
attestation1, @[bc0[1]], attestation1.loadSig,
attestation1.data.slot.toBeaconTime)
check:
process_slots(
@ -339,9 +350,11 @@ suite "Attestation pool processing" & preset():
attestation0.combine(attestation1)
pool[].addAttestation(
attestation0, @[bc0[0]], attestation0.loadSig, attestation0.data.slot)
attestation0, @[bc0[0]], attestation0.loadSig,
attestation0.data.slot.toBeaconTime)
pool[].addAttestation(
attestation1, @[bc0[1]], attestation1.loadSig, attestation1.data.slot)
attestation1, @[bc0[1]], attestation1.loadSig,
attestation1.data.slot.toBeaconTime)
check:
process_slots(
@ -367,9 +380,11 @@ suite "Attestation pool processing" & preset():
attestation0.combine(attestation1)
pool[].addAttestation(
attestation1, @[bc0[1]], attestation1.loadSig, attestation1.data.slot)
attestation1, @[bc0[1]], attestation1.loadSig,
attestation1.data.slot.toBeaconTime)
pool[].addAttestation(
attestation0, @[bc0[0]], attestation0.loadSig, attestation0.data.slot)
attestation0, @[bc0[0]], attestation0.loadSig,
attestation0.data.slot.toBeaconTime)
check:
process_slots(
@ -389,9 +404,10 @@ suite "Attestation pool processing" & preset():
blckRef: BlockRef, signedBlock: phase0.TrustedSignedBeaconBlock,
epochRef: EpochRef):
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)
pool[].addForkChoice(
epochRef, blckRef, signedBlock.message, blckRef.slot.toBeaconTime)
let head = pool[].selectHead(b1Add[].slot)
let head = pool[].selectHead(b1Add[].slot.toBeaconTime)
check:
head == b1Add[]
@ -402,9 +418,10 @@ suite "Attestation pool processing" & preset():
blckRef: BlockRef, signedBlock: phase0.TrustedSignedBeaconBlock,
epochRef: EpochRef):
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)
pool[].addForkChoice(
epochRef, blckRef, signedBlock.message, blckRef.slot.toBeaconTime)
let head2 = pool[].selectHead(b2Add[].slot)
let head2 = pool[].selectHead(b2Add[].slot.toBeaconTime)
check:
head2 == b2Add[]
@ -417,9 +434,10 @@ suite "Attestation pool processing" & preset():
blckRef: BlockRef, signedBlock: phase0.TrustedSignedBeaconBlock,
epochRef: EpochRef):
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)
pool[].addForkChoice(
epochRef, blckRef, signedBlock.message, blckRef.slot.toBeaconTime)
let head = pool[].selectHead(b10Add[].slot)
let head = pool[].selectHead(b10Add[].slot.toBeaconTime)
check:
head == b10Add[]
@ -432,7 +450,8 @@ suite "Attestation pool processing" & preset():
blckRef: BlockRef, signedBlock: phase0.TrustedSignedBeaconBlock,
epochRef: EpochRef):
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)
pool[].addForkChoice(
epochRef, blckRef, signedBlock.message, blckRef.slot.toBeaconTime)
bc1 = get_beacon_committee(
state[].data, getStateField(state.data, slot) - 1, 1.CommitteeIndex,
@ -440,9 +459,10 @@ suite "Attestation pool processing" & preset():
attestation0 = makeAttestation(state[].data, b10.root, bc1[0], cache)
pool[].addAttestation(
attestation0, @[bc1[0]], attestation0.loadSig, attestation0.data.slot)
attestation0, @[bc1[0]], attestation0.loadSig,
attestation0.data.slot.toBeaconTime)
let head2 = pool[].selectHead(b10Add[].slot)
let head2 = pool[].selectHead(b10Add[].slot.toBeaconTime)
check:
# Single vote for b10 and no votes for b11
@ -452,9 +472,10 @@ suite "Attestation pool processing" & preset():
attestation1 = makeAttestation(state[].data, b11.root, bc1[1], cache)
attestation2 = makeAttestation(state[].data, b11.root, bc1[2], cache)
pool[].addAttestation(
attestation1, @[bc1[1]], attestation1.loadSig, attestation1.data.slot)
attestation1, @[bc1[1]], attestation1.loadSig,
attestation1.data.slot.toBeaconTime)
let head3 = pool[].selectHead(b10Add[].slot)
let head3 = pool[].selectHead(b10Add[].slot.toBeaconTime)
let bigger = if b11.root.data < b10.root.data: b10Add else: b11Add
check:
@ -462,9 +483,10 @@ suite "Attestation pool processing" & preset():
head3 == bigger[]
pool[].addAttestation(
attestation2, @[bc1[2]], attestation2.loadSig, attestation2.data.slot)
attestation2, @[bc1[2]], attestation2.loadSig,
attestation2.data.slot.toBeaconTime)
let head4 = pool[].selectHead(b11Add[].slot)
let head4 = pool[].selectHead(b11Add[].slot.toBeaconTime)
check:
# Two votes for b11
@ -478,9 +500,10 @@ suite "Attestation pool processing" & preset():
blckRef: BlockRef, signedBlock: phase0.TrustedSignedBeaconBlock,
epochRef: EpochRef):
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)
pool[].addForkChoice(epochRef, blckRef, signedBlock.message,
blckRef.slot.toBeaconTime)
let head = pool[].selectHead(b10Add[].slot)
let head = pool[].selectHead(b10Add[].slot.toBeaconTime)
check:
head == b10Add[]
@ -492,7 +515,8 @@ suite "Attestation pool processing" & preset():
blckRef: BlockRef, signedBlock: phase0.TrustedSignedBeaconBlock,
epochRef: EpochRef):
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)
pool[].addForkChoice(
epochRef, blckRef, signedBlock.message, blckRef.slot.toBeaconTime)
doAssert: b10Add_clone.error == BlockError.Duplicate
@ -507,9 +531,10 @@ suite "Attestation pool processing" & preset():
blckRef: BlockRef, signedBlock: phase0.TrustedSignedBeaconBlock,
epochRef: EpochRef):
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)
pool[].addForkChoice(
epochRef, blckRef, signedBlock.message, blckRef.slot.toBeaconTime)
let head = pool[].selectHead(b10Add[].slot)
let head = pool[].selectHead(b10Add[].slot.toBeaconTime)
doAssert: head == b10Add[]
@ -532,9 +557,10 @@ suite "Attestation pool processing" & preset():
blckRef: BlockRef, signedBlock: phase0.TrustedSignedBeaconBlock,
epochRef: EpochRef):
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)
pool[].addForkChoice(
epochRef, blckRef, signedBlock.message, blckRef.slot.toBeaconTime)
let head = pool[].selectHead(blockRef[].slot)
let head = pool[].selectHead(blockRef[].slot.toBeaconTime)
doAssert: head == blockRef[]
dag.updateHead(head, quarantine[])
pruneAtFinalization(dag, pool[])
@ -574,6 +600,7 @@ suite "Attestation pool processing" & preset():
blckRef: BlockRef, signedBlock: phase0.TrustedSignedBeaconBlock,
epochRef: EpochRef):
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)
pool[].addForkChoice(
epochRef, blckRef, signedBlock.message, blckRef.slot.toBeaconTime)
doAssert: b10Add_clone.error == BlockError.Duplicate

View File

@ -82,7 +82,8 @@ suite "Gossip validation " & preset():
blckRef: BlockRef, signedBlock: phase0.TrustedSignedBeaconBlock,
epochRef: EpochRef):
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)
pool[].addForkChoice(
epochRef, blckRef, signedBlock.message, blckRef.slot.toBeaconTime)
check: added.isOk()
dag.updateHead(added[], quarantine[])