time: spring cleaning (#3262)

Time in the beacon chain is expressed relative to the genesis time -
this PR creates a `beacon_time` module that collects helpers and
utilities for dealing the time units - the new module does not deal with
actual wall time (that's remains in `beacon_clock`).

Collecting the time related stuff in one place makes it easier to find,
avoids some circular imports and allows more easily identifying the code
actually needs wall time to operate.

* move genesis-time-related functionality into `spec/beacon_time`
* avoid using `chronos.Duration` for time differences - it does not
support negative values (such as when something happens earlier than it
should)
* saturate conversions between `FAR_FUTURE_XXX`, so as to avoid
overflows
* fix delay reporting in validator client so it uses the expected
deadline of the slot, not "closest wall slot"
* simplify looping over the slots of an epoch
* `compute_start_slot_at_epoch` -> `start_slot`
* `compute_epoch_at_slot` -> `epoch`

A follow-up PR will (likely) introduce saturating arithmetic for the
time units - this is merely code moves, renames and fixing of small
bugs.
This commit is contained in:
Jacek Sieka 2022-01-11 11:01:54 +01:00 committed by GitHub
parent aab302aa9c
commit 805e85e1ff
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
67 changed files with 612 additions and 555 deletions

1
.gitignore vendored
View File

@ -50,3 +50,4 @@ build/
/dist
/benchmark_results
/.update.timestamp
resttest0_data

View File

@ -77,6 +77,11 @@ OK: 16/16 Fail: 0/16 Skip: 0/16
+ latest_block_root OK
```
OK: 3/3 Fail: 0/3 Skip: 0/3
## Beacon time
```diff
+ basics OK
```
OK: 1/1 Fail: 0/1 Skip: 0/1
## Block pool altair processing [Preset: mainnet]
```diff
+ Invalid signatures [Preset: mainnet] OK
@ -436,4 +441,4 @@ OK: 1/1 Fail: 0/1 Skip: 0/1
OK: 1/1 Fail: 0/1 Skip: 0/1
---TOTAL---
OK: 236/238 Fail: 0/238 Skip: 2/238
OK: 237/239 Fail: 0/239 Skip: 2/239

View File

@ -10,7 +10,7 @@
import
std/math,
chronos, chronicles,
./spec/helpers
./spec/beacon_time
from times import Time, getTime, fromUnix, `<`, `-`, inNanoseconds
@ -32,25 +32,8 @@ type
# https://ethresear.ch/t/network-adjusted-timestamps/4187
genesis: Time
BeaconTime* = distinct Duration ## Nanoseconds from beacon genesis time
GetBeaconTimeFn* = proc(): BeaconTime {.gcsafe, raises: [Defect].}
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.8/specs/phase0/validator.md#attesting
attestationSlotOffset* = seconds(SECONDS_PER_SLOT.int) div INTERVALS_PER_SLOT
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/validator.md#broadcast-aggregate
aggregateSlotOffset* =
seconds(SECONDS_PER_SLOT.int) * 2 div INTERVALS_PER_SLOT
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/altair/validator.md#prepare-sync-committee-message
syncCommitteeMessageSlotOffset* =
seconds(SECONDS_PER_SLOT.int) div INTERVALS_PER_SLOT
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/altair/validator.md#broadcast-sync-committee-contribution
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
doAssert genesis_time <= high(int64).uint64
@ -63,47 +46,12 @@ proc init*(T: type BeaconClock, genesis_time: uint64): T =
T(genesis: unixGenesis - unixGenesisOffset)
template `<`*(a, b: BeaconTime): bool =
Duration(a) < Duration(b)
template `<=`*(a, b: BeaconTime): bool =
Duration(a) <= Duration(b)
template `+`*(t: BeaconTime, offset: Duration): BeaconTime =
BeaconTime(Duration(t) + offset)
template `-`*(t: BeaconTime, offset: Duration): BeaconTime =
BeaconTime(nanoseconds(nanoseconds(Duration(t)) - nanoseconds(offset)))
template `-`*(a, b: BeaconTime): Duration =
nanoseconds(nanoseconds(Duration(a)) - nanoseconds(Duration(b)))
func toSlot*(t: BeaconTime): tuple[afterGenesis: bool, slot: Slot] =
let ti = seconds(Duration(t))
if ti >= 0:
(true, Slot(uint64(ti) div SECONDS_PER_SLOT))
else:
(false, Slot(uint64(-ti) div SECONDS_PER_SLOT))
func slotOrZero*(time: BeaconTime): Slot =
let exSlot = time.toSlot
if exSlot.afterGenesis: exSlot.slot
else: Slot(0)
func toBeaconTime*(c: BeaconClock, t: Time): BeaconTime =
BeaconTime(nanoseconds(inNanoseconds(t - c.genesis)))
BeaconTime(ns_since_genesis: inNanoseconds(t - c.genesis))
func toSlot*(c: BeaconClock, t: Time): tuple[afterGenesis: bool, slot: Slot] =
c.toBeaconTime(t).toSlot()
func toBeaconTime*(s: Slot, offset = Duration()): BeaconTime =
# BeaconTime/Duration stores nanoseconds, internally
const maxSlot = (not 0'u64 div 2 div SECONDS_PER_SLOT div 1_000_000_000).Slot
var slot = s
if slot > maxSlot:
slot = maxSlot
BeaconTime(seconds(int64(uint64(slot) * SECONDS_PER_SLOT)) + offset)
proc now*(c: BeaconClock): BeaconTime =
## Current time, in slots - this may end up being less than GENESIS_SLOT(!)
toBeaconTime(c, getTime())
@ -111,12 +59,12 @@ proc now*(c: BeaconClock): BeaconTime =
proc fromNow*(c: BeaconClock, t: BeaconTime): tuple[inFuture: bool, offset: Duration] =
let now = c.now()
if t > now:
(true, t - now)
(true, chronos.nanoseconds((t - now).nanoseconds))
else:
(false, now - t)
(false, chronos.nanoseconds((now - t).nanoseconds))
proc fromNow*(c: BeaconClock, slot: Slot): tuple[inFuture: bool, offset: Duration] =
c.fromNow(slot.toBeaconTime())
c.fromNow(slot.start_beacon_time())
proc durationToNextSlot*(c: BeaconClock): Duration =
let (afterGenesis, slot) = c.now().toSlot()
@ -128,15 +76,16 @@ proc durationToNextSlot*(c: BeaconClock): Duration =
proc durationToNextEpoch*(c: BeaconClock): Duration =
let (afterGenesis, slot) = c.now().toSlot()
if afterGenesis:
c.fromNow(compute_start_slot_at_epoch(slot.epoch() + 1'u64)).offset
c.fromNow((slot.epoch + 1).start_slot()).offset
else:
c.fromNow(compute_start_slot_at_epoch(Epoch(0))).offset
c.fromNow(Epoch(0).start_slot()).offset
func saturate*(d: tuple[inFuture: bool, offset: Duration]): Duration =
if d.inFuture: d.offset else: seconds(0)
proc addTimer*(fromNow: Duration, cb: CallbackFunc, udata: pointer = nil) =
discard setTimer(Moment.now() + fromNow, cb, udata)
proc sleepAsync*(t: TimeDiff): Future[void] =
sleepAsync(chronos.nanoseconds(
if t.nanoseconds < 0: 0'i64 else: t.nanoseconds))
func shortLog*(d: Duration): string =
$d
@ -153,8 +102,4 @@ func fromFloatSeconds*(T: type Duration, f: float): Duration =
of fcSubnormal, fcZero, fcNegZero, fcNan, fcNegInf: ZeroDuration
of fcInf: InfiniteDuration
func `$`*(v: BeaconTime): string = $(Duration v)
func shortLog*(v: BeaconTime): string = $(Duration v)
chronicles.formatIt Duration: $it
chronicles.formatIt BeaconTime: $(Duration it)

View File

@ -139,7 +139,7 @@ proc init*(T: type AttestationPool, dag: ChainDAGRef,
withBlck(dag.get(blckRef).data):
forkChoice.process_block(
dag, epochRef, blckRef, blck.message, blckRef.slot.toBeaconTime)
dag, epochRef, blckRef, blck.message, blckRef.slot.start_beacon_time)
doAssert status.isOk(), "Error in preloading the fork choice: " & $status.error
@ -459,11 +459,8 @@ func init(
template update_attestation_pool_cache(
epoch: Epoch, participation_bitmap: untyped) =
let
start_slot = epoch.compute_start_slot_at_epoch()
for committee_index in get_committee_indices(state.data, epoch, cache):
for slot in start_slot..<start_slot + SLOTS_PER_EPOCH:
for slot in epoch.slots():
let committee = get_beacon_committee(
state.data, slot, committee_index, cache)
var

View File

@ -12,7 +12,6 @@ import
chronicles,
stew/[assign2, results],
eth/keys,
".."/[beacon_clock],
../spec/[
eth2_merkleization, forks, helpers, signatures, signatures_batch,
state_transition],
@ -99,16 +98,8 @@ proc addResolvedHeadBlock(
blockRef
# TODO workaround for https://github.com/nim-lang/Nim/issues/18095
type SomeSignedBlock =
phase0.SignedBeaconBlock | phase0.SigVerifiedSignedBeaconBlock |
phase0.TrustedSignedBeaconBlock |
altair.SignedBeaconBlock | altair.SigVerifiedSignedBeaconBlock |
altair.TrustedSignedBeaconBlock |
merge.SignedBeaconBlock | merge.SigVerifiedSignedBeaconBlock |
merge.TrustedSignedBeaconBlock
proc checkStateTransition(
dag: ChainDAGRef, signedBlock: SomeSignedBlock,
dag: ChainDAGRef, signedBlock: SomeForkySignedBeaconBlock,
cache: var StateCache): Result[void, BlockError] =
## Ensure block can be applied on a state
func restore(v: var ForkedHashedBeaconState) =

View File

@ -171,7 +171,7 @@ func atSlot*(bid: BlockId): BlockSlotId =
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())
atSlot(blck, epoch.start_slot())
func atSlotEpoch*(blck: BlockRef, epoch: Epoch): BlockSlot =
## Return the last block that was included in the chain leading
@ -181,8 +181,9 @@ func atSlotEpoch*(blck: BlockRef, epoch: Epoch): BlockSlot =
if epoch == GENESIS_EPOCH:
blck.atEpochStart(epoch)
else:
let start = epoch.compute_start_slot_at_epoch()
let tmp = blck.atSlot(start - 1)
let
start = epoch.start_slot()
tmp = blck.atSlot(start - 1)
if isNil(tmp.blck):
BlockSlot()
else:
@ -215,7 +216,7 @@ func isProposed*(bsi: BlockSlotId): bool =
func dependentBlock*(head, tail: BlockRef, epoch: Epoch): BlockRef =
## The block that determined the proposer shuffling in the given epoch
let dependentSlot =
if epoch >= Epoch(1): epoch.compute_start_slot_at_epoch() - 1
if epoch >= Epoch(1): epoch.start_slot() - 1
else: Slot(0)
let res = head.atSlot(dependentSlot)
if isNil(res.blck): tail

View File

@ -145,7 +145,7 @@ func init*(
state.data.mergeData.data.latest_execution_payload_header !=
ExecutionPayloadHeader()
)
epochStart = epoch.compute_start_slot_at_epoch()
epochStart = epoch.start_slot()
doAssert epochRef.key.blck != nil, "epochAncestor should not fail for state block"
@ -278,10 +278,9 @@ func loadStateCache(
if epochRef.isSome():
cache.shuffled_active_validator_indices[epoch] =
epochRef[].shuffled_active_validator_indices
let start_slot = epoch.start_slot()
for i, idx in epochRef[].beacon_proposers:
cache.beacon_proposer_indices[
epoch.compute_start_slot_at_epoch + i] = idx
cache.beacon_proposer_indices[start_slot + i] = idx
load(epoch)
@ -311,7 +310,7 @@ func isStateCheckpoint(bs: BlockSlot): bool =
# The tail block also counts as a state checkpoint!
(bs.slot == bs.blck.slot and bs.blck.parent == nil) or
(bs.slot.isEpoch and bs.slot.epoch == (bs.blck.slot.epoch + 1))
(bs.slot.is_epoch and bs.slot.epoch == (bs.blck.slot.epoch + 1))
proc getStateData(
db: BeaconChainDB, cfg: RuntimeConfig, state: var StateData, bs: BlockSlot,
@ -545,8 +544,7 @@ proc init*(T: type ChainDAGRef, cfg: RuntimeConfig, db: BeaconChainDB,
let
finalized_checkpoint =
getStateField(dag.headState.data, finalized_checkpoint)
finalizedSlot = max(
finalized_checkpoint.epoch.compute_start_slot_at_epoch(), tailRef.slot)
finalizedSlot = max(finalized_checkpoint.epoch.start_slot(), tailRef.slot)
dag.finalizedHead = headRef.atSlot(finalizedSlot)
@ -1319,10 +1317,7 @@ proc updateHead*(
let
finalized_checkpoint =
getStateField(dag.headState.data, finalized_checkpoint)
finalizedSlot = max(
finalized_checkpoint.epoch.compute_start_slot_at_epoch(),
dag.tail.slot)
finalizedSlot = max(finalized_checkpoint.epoch.start_slot(), dag.tail.slot)
finalizedHead = newHead.atSlot(finalizedSlot)
doAssert (not finalizedHead.blck.isNil),
@ -1578,11 +1573,11 @@ proc getProposer*(
dag: ChainDAGRef, head: BlockRef, slot: Slot): Option[ValidatorIndex] =
let
epochRef = block:
let tmp = dag.getEpochRef(head, slot.compute_epoch_at_slot(), false)
let tmp = dag.getEpochRef(head, slot.epoch(), false)
if tmp.isErr():
return none(ValidatorIndex)
tmp.get()
slotInEpoch = slot - slot.compute_epoch_at_slot().compute_start_slot_at_epoch()
slotInEpoch = slot.since_epoch_start()
let proposer = epochRef.beacon_proposers[slotInEpoch]
if proposer.isSome():

View File

@ -152,8 +152,8 @@ func makeAttestationData*(
let
slot = bs.slot
current_epoch = slot.compute_epoch_at_slot()
epoch_boundary_slot = compute_start_slot_at_epoch(current_epoch)
current_epoch = slot.epoch()
epoch_boundary_slot = current_epoch.start_slot()
epoch_boundary_block = bs.blck.atSlot(epoch_boundary_slot)
doAssert current_epoch == epochRef.epoch
@ -178,9 +178,8 @@ iterator get_committee_assignments*(
let
committees_per_slot = get_committee_count_per_slot(epochRef)
epoch = epochRef.epoch
start_slot = compute_start_slot_at_epoch(epoch)
for slot in start_slot ..< start_slot + SLOTS_PER_EPOCH:
for slot in epoch.slots():
for committee_index in get_committee_indices(committees_per_slot):
if anyIt(get_beacon_committee(epochRef, slot, committee_index), it in validator_indices):
yield (

View File

@ -13,7 +13,6 @@ import
# Status libraries
stew/results, chronicles,
# Internal
../beacon_clock,
../spec/[beaconstate, helpers],
../spec/datatypes/[phase0, altair, bellatrix],
# Fork choice
@ -90,9 +89,6 @@ func extend[T](s: var seq[T], minLen: int) =
if s.len < minLen:
s.setLen(minLen)
func compute_slots_since_epoch_start(slot: Slot): uint64 =
slot - slot.epoch().compute_start_slot_at_epoch()
func on_tick*(self: var Checkpoints, time: BeaconTime): FcResult[void] =
if self.time > time:
return err ForkChoiceError(kind: fcInconsistentTick)
@ -231,12 +227,11 @@ func should_update_justified_checkpoint(
self: var Checkpoints,
dag: ChainDAGRef,
epochRef: EpochRef): FcResult[bool] =
if compute_slots_since_epoch_start(self.time.slotOrZero) <
SAFE_SLOTS_TO_UPDATE_JUSTIFIED:
if self.time.slotOrZero.since_epoch_start() < SAFE_SLOTS_TO_UPDATE_JUSTIFIED:
return ok(true)
let
justified_slot = compute_start_slot_at_epoch(self.justified.checkpoint.epoch)
justified_slot = self.justified.checkpoint.epoch.start_slot()
new_justified_checkpoint = epochRef.current_justified_checkpoint
justified_blck = dag.getRef(new_justified_checkpoint.root)
@ -366,7 +361,7 @@ proc process_block*(self: var ForkChoice,
# Add proposer score boost if the block is timely
let
time_into_slot =
self.checkpoints.time - self.checkpoints.time.slotOrZero.toBeaconTime
self.checkpoints.time - self.checkpoints.time.slotOrZero.start_beacon_time
is_before_attesting_interval = time_into_slot < attestationSlotOffset
if self.checkpoints.time.slotOrZero == blck.slot and
is_before_attesting_interval:

View File

@ -15,10 +15,7 @@ import
chronicles,
# Internal
../beacon_clock,
../spec/datatypes/base,
../consensus_object_pools/block_pools_types
../spec/datatypes/base
# https://github.com/ethereum/consensus-specs/blob/v0.11.1/specs/phase0/fork-choice.md
# This is a port of https://github.com/sigp/lighthouse/pull/804
# which is a port of "Proto-Array": https://github.com/protolambda/lmd-ghost

View File

@ -10,7 +10,6 @@
import
chronicles, chronos,
../spec/datatypes/base,
../beacon_clock,
../consensus_object_pools/[blockchain_dag, block_quarantine, attestation_pool]
# TODO: Move to "consensus_object_pools" folder
@ -80,7 +79,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.toBeaconTime)
let newHead = self.attestationPool[].selectHead(wallSlot.start_beacon_time)
if newHead.isNil():
warn "Head selection failed, using previous head",
head = shortLog(self.dag.head), wallSlot

View File

@ -194,7 +194,7 @@ proc blockValidator*(
return errIgnore("Block before genesis")
# Potential under/overflows are fine; would just create odd metrics and logs
let delay = wallTime - signedBlock.message.slot.toBeaconTime
let delay = wallTime - signedBlock.message.slot.start_beacon_time
# Start of block processing - in reality, we have already gone through SSZ
# decoding at this stage, which may be significant
@ -213,7 +213,8 @@ proc blockValidator*(
self.blockProcessor[].addBlock(
src, ForkedSignedBeaconBlock.init(signedBlock),
validationDur = self.getCurrentBeaconTime() - wallTime)
validationDur = nanoseconds(
(self.getCurrentBeaconTime() - wallTime).nanoseconds))
# Validator monitor registration for blocks is done by the processor
beacon_blocks_received.inc()
@ -296,7 +297,7 @@ proc attestationValidator*(
return errIgnore("Attestation before genesis")
# Potential under/overflows are fine; would just create odd metrics and logs
let delay = wallTime - attestation.data.slot.toBeaconTime
let delay = wallTime - attestation.data.slot.start_beacon_time
debug "Attestation received", delay
# Now proceed to validation
@ -346,7 +347,7 @@ proc aggregateValidator*(
# Potential under/overflows are fine; would just create odd logs
let delay =
wallTime - signedAggregateAndProof.message.aggregate.data.slot.toBeaconTime
wallTime - signedAggregateAndProof.message.aggregate.data.slot.start_beacon_time
debug "Aggregate received", delay
let v =
@ -467,7 +468,7 @@ proc syncCommitteeMessageValidator*(
wallSlot
# Potential under/overflows are fine; would just create odd metrics and logs
let delay = wallTime - syncCommitteeMsg.slot.toBeaconTime
let delay = wallTime - syncCommitteeMsg.slot.start_beacon_time
debug "Sync committee message received", delay
# Now proceed to validation
@ -513,7 +514,7 @@ proc contributionValidator*(
wallSlot
# Potential under/overflows are fine; would just create odd metrics and logs
let delay = wallTime - contributionAndProof.message.contribution.slot.toBeaconTime
let delay = wallTime - contributionAndProof.message.contribution.slot.start_beacon_time
debug "Contribution received", delay
# Now proceed to validation

View File

@ -124,7 +124,7 @@ func check_beacon_and_target_block(
# attestation.data.target.root
# the sanity of target.epoch has been checked by check_attestation_slot_target
let
target = blck.atSlot(compute_start_slot_at_epoch(data.target.epoch))
target = blck.atSlot(data.target.epoch.start_slot())
if isNil(target.blck):
# Shouldn't happen - we've checked that the target epoch is within range
@ -319,8 +319,7 @@ proc validateBeaconBlock*(
let
finalized_checkpoint = getStateField(
dag.headState.data, finalized_checkpoint)
ancestor = get_ancestor(
parent_ref, compute_start_slot_at_epoch(finalized_checkpoint.epoch))
ancestor = get_ancestor(parent_ref, finalized_checkpoint.epoch.start_slot)
if ancestor.isNil:
# This shouldn't happen: we should always be able to trace the parent back

View File

@ -538,7 +538,7 @@ proc init*(T: type BeaconNode,
func verifyFinalization(node: BeaconNode, slot: Slot) =
# Epoch must be >= 4 to check finalization
const SETTLING_TIME_OFFSET = 1'u64
let epoch = slot.compute_epoch_at_slot()
let epoch = slot.epoch()
# Don't static-assert this -- if this isn't called, don't require it
doAssert SLOTS_PER_EPOCH > SETTLING_TIME_OFFSET
@ -547,7 +547,7 @@ func verifyFinalization(node: BeaconNode, slot: Slot) =
# during testing.
if epoch >= 4 and slot mod SLOTS_PER_EPOCH > SETTLING_TIME_OFFSET:
let finalizedEpoch =
node.dag.finalizedHead.slot.compute_epoch_at_slot()
node.dag.finalizedHead.slot.epoch()
# Finalization rule 234, that has the most lag slots among the cases, sets
# state.finalized_checkpoint = old_previous_justified_checkpoint.epoch + 3
# and then state.slot gets incremented, to increase the maximum offset, if
@ -824,7 +824,7 @@ proc onSlotEnd(node: BeaconNode, slot: Slot) {.async.} =
.slashingProtection
# pruning is only done if the DB is set to pruning mode.
.pruneAfterFinalization(
node.dag.finalizedHead.slot.compute_epoch_at_slot()
node.dag.finalizedHead.slot.epoch()
)
# Delay part of pruning until latency critical duties are done.
@ -899,7 +899,7 @@ proc onSlotEnd(node: BeaconNode, slot: Slot) {.async.} =
# slot end since the nextActionWaitTime can be short
let
advanceCutoff = node.beaconClock.fromNow(
slot.toBeaconTime(chronos.seconds(int(SECONDS_PER_SLOT - 1))))
slot.start_beacon_time() + chronos.seconds(int(SECONDS_PER_SLOT - 1)))
if advanceCutoff.inFuture:
# We wait until there's only a second left before the next slot begins, then
# we advance the clearance state to the next slot - this gives us a high
@ -929,9 +929,8 @@ proc onSlotStart(
wallSlot = wallTime.slotOrZero
# If everything was working perfectly, the slot that we should be processing
expectedSlot = lastSlot + 1
finalizedEpoch =
node.dag.finalizedHead.blck.slot.compute_epoch_at_slot()
delay = wallTime - expectedSlot.toBeaconTime()
finalizedEpoch = node.dag.finalizedHead.blck.slot.epoch()
delay = wallTime - expectedSlot.start_beacon_time()
info "Slot start",
slot = shortLog(wallSlot),
@ -1205,14 +1204,14 @@ proc start*(node: BeaconNode) {.raises: [Defect, CatchableError].} =
let
head = node.dag.head
finalizedHead = node.dag.finalizedHead
genesisTime = node.beaconClock.fromNow(toBeaconTime(Slot 0))
genesisTime = node.beaconClock.fromNow(start_beacon_time(Slot 0))
notice "Starting beacon node",
version = fullVersionStr,
enr = node.network.announcedENR.toURI,
peerId = $node.network.switch.peerInfo.peerId,
timeSinceFinalization =
node.beaconClock.now() - finalizedHead.slot.toBeaconTime(),
node.beaconClock.now() - finalizedHead.slot.start_beacon_time(),
head = shortLog(head),
justified = shortLog(getStateField(
node.dag.headState.data, current_justified_checkpoint)),
@ -1284,7 +1283,7 @@ proc initStatusBar(node: BeaconNode) {.raises: [Defect, ValueError].} =
of "head_epoch":
$(node.dag.head.slot.epoch)
of "head_epoch_slot":
$(node.dag.head.slot mod SLOTS_PER_EPOCH)
$(node.dag.head.slot.since_epoch_start)
of "head_slot":
$(node.dag.head.slot)
@ -1293,7 +1292,7 @@ proc initStatusBar(node: BeaconNode) {.raises: [Defect, ValueError].} =
of "justifed_epoch":
$(justified.slot.epoch)
of "justifed_epoch_slot":
$(justified.slot mod SLOTS_PER_EPOCH)
$(justified.slot.since_epoch_start)
of "justifed_slot":
$(justified.slot)
@ -1302,7 +1301,7 @@ proc initStatusBar(node: BeaconNode) {.raises: [Defect, ValueError].} =
of "finalized_epoch":
$(node.dag.finalizedHead.slot.epoch)
of "finalized_epoch_slot":
$(node.dag.finalizedHead.slot mod SLOTS_PER_EPOCH)
$(node.dag.finalizedHead.slot.since_epoch_start)
of "finalized_slot":
$(node.dag.finalizedHead.slot)
@ -1310,7 +1309,7 @@ proc initStatusBar(node: BeaconNode) {.raises: [Defect, ValueError].} =
$node.currentSlot.epoch
of "epoch_slot":
$(node.currentSlot mod SLOTS_PER_EPOCH)
$(node.currentSlot.since_epoch_start)
of "slot":
$node.currentSlot

View File

@ -188,28 +188,13 @@ template makeBannerAndConfig*(clientId: string, ConfType: type): untyped =
{.pop.}
config
# TODO not sure if this belongs here but it doesn't belong in `beacon_clock.nim` either
proc sleepToSlotOffset*(clock: BeaconClock, extra: chronos.Duration,
slot: Slot, msg: static string): Future[bool] {.async.} =
let
fromNow = clock.fromNow(slot.toBeaconTime(extra))
if fromNow.inFuture:
trace msg,
slot = shortLog(slot),
fromNow = shortLog(fromNow.offset)
await sleepAsync(fromNow.offset)
return true
return false
proc checkIfShouldStopAtEpoch*(scheduledSlot: Slot, stopAtEpoch: uint64) =
# Offset backwards slightly to allow this epoch's finalization check to occur
if scheduledSlot > 3 and stopAtEpoch > 0'u64 and
(scheduledSlot - 3).compute_epoch_at_slot() >= stopAtEpoch:
(scheduledSlot - 3).epoch() >= stopAtEpoch:
info "Stopping at pre-chosen epoch",
chosenEpoch = stopAtEpoch,
epoch = scheduledSlot.compute_epoch_at_slot(),
epoch = scheduledSlot.epoch(),
slot = scheduledSlot
# Brute-force, but ensure it's reliable enough to run in CI.
@ -229,7 +214,7 @@ proc runSlotLoop*[T](node: T, startTime: BeaconTime,
var
curSlot = startTime.slotOrZero()
nextSlot = curSlot + 1 # No earlier than GENESIS_SLOT + 1
timeToNextSlot = nextSlot.toBeaconTime() - startTime
timeToNextSlot = nextSlot.start_beacon_time() - startTime
info "Scheduling first slot action",
startTime = shortLog(startTime),
@ -268,7 +253,7 @@ proc runSlotLoop*[T](node: T, startTime: BeaconTime,
wallSlot = shortLog(wallSlot)
# cur & next slot remain the same
timeToNextSlot = nextSlot.toBeaconTime() - wallTime
timeToNextSlot = nextSlot.start_beacon_time() - wallTime
continue
if wallSlot > nextSlot + SLOTS_PER_EPOCH:
@ -285,7 +270,7 @@ proc runSlotLoop*[T](node: T, startTime: BeaconTime,
elif wallSlot > nextSlot:
notice "Missed expected slot start, catching up",
delay = shortLog(wallTime - nextSlot.toBeaconTime()),
delay = shortLog(wallTime - nextSlot.start_beacon_time()),
curSlot = shortLog(curSlot),
nextSlot = shortLog(curSlot)
@ -293,4 +278,4 @@ proc runSlotLoop*[T](node: T, startTime: BeaconTime,
curSlot = wallSlot
nextSlot = wallSlot + 1
timeToNextSlot = saturate(node.beaconClock.fromNow(nextSlot))
timeToNextSlot = nextSlot.start_beacon_time() - node.beaconClock.now()

View File

@ -95,7 +95,7 @@ proc initClock(vc: ValidatorClientRef): Future[BeaconClock] {.async.} =
info "Initializing beacon clock",
genesis_time = vc.beaconGenesis.genesis_time,
current_slot = currentSlot, current_epoch = currentEpoch
let genesisTime = res.fromNow(toBeaconTime(Slot(0)))
let genesisTime = res.fromNow(start_beacon_time(Slot(0)))
if genesisTime.inFuture:
notice "Waiting for genesis", genesisIn = genesisTime.offset
await sleepAsync(genesisTime.offset)
@ -141,7 +141,7 @@ proc onSlotStart(vc: ValidatorClientRef, wallTime: BeaconTime,
let
# If everything was working perfectly, the slot that we should be processing
expectedSlot = lastSlot + 1
delay = wallTime - expectedSlot.toBeaconTime()
delay = wallTime - expectedSlot.start_beacon_time()
checkIfShouldStopAtEpoch(wallSlot.slot, vc.config.stopAtEpoch)

View File

@ -589,13 +589,12 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
var res: seq[RestBeaconStatesCommittees]
let qepoch =
if vepoch.isNone:
compute_epoch_at_slot(getStateField(stateData.data, slot))
epoch(getStateField(stateData.data, slot))
else:
vepoch.get()
if vslot.isNone():
let start_slot = qepoch.compute_start_slot_at_epoch()
for slot in start_slot ..< start_slot + SLOTS_PER_EPOCH:
for slot in qepoch.slots():
forSlot(slot, vindex, res)
else:
forSlot(vslot.get(), vindex, res)

View File

@ -1,7 +1,10 @@
import ../spec/helpers
import
../spec/beacon_time
export beacon_time
const
MaxEpoch* = compute_epoch_at_slot(not(0'u64))
MaxEpoch* = epoch(FAR_FUTURE_SLOT)
BlockValidationError* =
"The block failed validation, but was successfully broadcast anyway. It " &

View File

@ -66,7 +66,7 @@ proc getCurrentHead*(node: BeaconNode,
epoch: Epoch): Result[BlockRef, cstring] =
if epoch > MaxEpoch:
return err("Requesting epoch for which slot would overflow")
node.getCurrentHead(compute_start_slot_at_epoch(epoch))
node.getCurrentHead(epoch.start_slot())
proc getBlockSlot*(node: BeaconNode,
stateIdent: StateIdent): Result[BlockSlot, cstring] =

View File

@ -79,9 +79,8 @@ proc installValidatorApiHandlers*(router: var RestRouter, node: BeaconNode) =
let
committees_per_slot = get_committee_count_per_slot(epochRef)
start_slot = qepoch.compute_start_slot_at_epoch()
for committee_index in get_committee_indices(committees_per_slot):
for slot in start_slot ..< start_slot + SLOTS_PER_EPOCH:
for slot in qepoch.slots():
let committee = get_beacon_committee(epochRef, slot, committee_index)
for index_in_committee, validator_index in committee:
if validator_index in indexList:
@ -143,7 +142,7 @@ proc installValidatorApiHandlers*(router: var RestRouter, node: BeaconNode) =
RestProposerDuty(
pubkey: epochRef.validatorKey(bp.get()).get().toPubKey(),
validator_index: bp.get(),
slot: compute_start_slot_at_epoch(qepoch) + i
slot: qepoch.start_slot() + i
)
)
res

View File

@ -364,13 +364,12 @@ proc installBeaconApiHandlers*(rpcServer: RpcServer, node: BeaconNode) {.
let qepoch =
if epoch.isNone:
compute_epoch_at_slot(getStateField(stateData.data, slot))
epoch(getStateField(stateData.data, slot))
else:
Epoch(epoch.get())
if slot.isNone:
let start_slot = qepoch.compute_start_slot_at_epoch()
for slot in start_slot ..< start_slot + SLOTS_PER_EPOCH:
for slot in qepoch.slots():
forSlot(slot, res)
else:
forSlot(Slot(slot.get()), res)

View File

@ -44,7 +44,7 @@ proc parseRoot*(str: string): Eth2Digest {.raises: [Defect, ValueError].} =
Eth2Digest(data: hexToByteArray[32](str))
func checkEpochToSlotOverflow*(epoch: Epoch) {.raises: [Defect, ValueError].} =
const maxEpoch = compute_epoch_at_slot(not 0'u64)
const maxEpoch = epoch(FAR_FUTURE_SLOT)
if epoch >= maxEpoch:
raise newException(
ValueError, "Requesting epoch for which slot would overflow")
@ -59,7 +59,7 @@ proc doChecksAndGetCurrentHead*(node: BeaconNode, slot: Slot): BlockRef {.raises
proc doChecksAndGetCurrentHead*(node: BeaconNode, epoch: Epoch): BlockRef {.raises: [Defect, CatchableError].} =
checkEpochToSlotOverflow(epoch)
node.doChecksAndGetCurrentHead(epoch.compute_start_slot_at_epoch)
node.doChecksAndGetCurrentHead(epoch.start_slot())
proc getBlockSlotFromString*(node: BeaconNode, slot: string): BlockSlot {.raises: [Defect, CatchableError].} =
if slot.len == 0:

View File

@ -92,8 +92,7 @@ proc installValidatorApiHandlers*(rpcServer: RpcServer, node: BeaconNode) {.
let
committees_per_slot = get_committee_count_per_slot(epochRef)
start_slot = compute_start_slot_at_epoch(epoch)
for slot in start_slot ..< start_slot + SLOTS_PER_EPOCH:
for slot in epoch.slots():
for committee_index in get_committee_indices(committees_per_slot):
let committee = get_beacon_committee(epochRef, slot, committee_index)
for index_in_committee, validator_index in committee:
@ -122,7 +121,7 @@ proc installValidatorApiHandlers*(rpcServer: RpcServer, node: BeaconNode) {.
if bp.isSome():
result.add((public_key: epochRef.validatorKey(bp.get).get().toPubKey,
validator_index: bp.get(),
slot: compute_start_slot_at_epoch(epoch) + i))
slot: epoch.start_slot() + i))
rpcServer.rpc("post_v1_validator_beacon_committee_subscriptions") do (
committee_index: CommitteeIndex, slot: Slot, aggregator: bool,

View File

@ -0,0 +1,255 @@
# beacon_chain
# Copyright (c) 2018-2021 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.push raises: [Defect].}
import
std/[hashes, typetraits],
chronicles,
chronos/timer,
json_serialization,
./presets
export hashes, timer, json_serialization, presets
# A collection of time units that permeate the spec - common to all of them is
# that they expressed relative to the genesis of the chain at varying
# granularities:
#
# * BeaconTime - nanoseconds since genesis
# * Slot - SLOTS_PER_SECOND seconds since genesis
# * Epoch - EPOCHS_PER_SLOT slots since genesis
# * SyncCommitteePeriod - EPOCHS_PER_SYNC_COMMITTEE_PERIOD epochs since genesis
type
BeaconTime* = object
## A point in time, relative to the genesis of the chain
##
## Implemented as nanoseconds since genesis - negative means before
## the chain started.
ns_since_genesis*: int64
TimeDiff* = object
nanoseconds*: int64
## Difference between two points in time with nanosecond granularity
## Can be negative (unlike timer.Duration)
const
# Earlier spec versions had these at a different slot
GENESIS_SLOT* = Slot(0)
GENESIS_EPOCH* = Epoch(0) # compute_epoch_at_slot(GENESIS_SLOT)
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/fork-choice.md#constant
INTERVALS_PER_SLOT* = 3
FAR_FUTURE_BEACON_TIME* = BeaconTime(ns_since_genesis: int64.high())
FAR_FUTURE_SLOT* = Slot(not 0'u64)
# FAR_FUTURE_EPOCH* = Epoch(not 0'u64) # in presets
FAR_FUTURE_PERIOD* = SyncCommitteePeriod(not 0'u64)
NANOSECONDS_PER_SLOT = SECONDS_PER_SLOT * 1_000_000_000'u64
# TODO when https://github.com/nim-lang/Nim/issues/14440 lands in Status's Nim,
# switch proc {.noSideEffect.} to func.
template ethTimeUnit(typ: type) {.dirty.} =
proc `+`*(x: typ, y: uint64): typ {.borrow, noSideEffect.}
proc `-`*(x: typ, y: uint64): typ {.borrow, noSideEffect.}
proc `-`*(x: uint64, y: typ): typ {.borrow, noSideEffect.}
# Not closed over type in question (Slot or Epoch)
proc `mod`*(x: typ, y: uint64): uint64 {.borrow, noSideEffect.}
proc `div`*(x: typ, y: uint64): uint64 {.borrow, noSideEffect.}
proc `div`*(x: uint64, y: typ): uint64 {.borrow, noSideEffect.}
proc `-`*(x: typ, y: typ): uint64 {.borrow, noSideEffect.}
proc `*`*(x: typ, y: uint64): uint64 {.borrow, noSideEffect.}
proc `+=`*(x: var typ, y: typ) {.borrow, noSideEffect.}
proc `+=`*(x: var typ, y: uint64) {.borrow, noSideEffect.}
proc `-=`*(x: var typ, y: typ) {.borrow, noSideEffect.}
proc `-=`*(x: var typ, y: uint64) {.borrow, noSideEffect.}
# Comparison operators
proc `<`*(x: typ, y: typ): bool {.borrow, noSideEffect.}
proc `<`*(x: typ, y: uint64): bool {.borrow, noSideEffect.}
proc `<`*(x: uint64, y: typ): bool {.borrow, noSideEffect.}
proc `<=`*(x: typ, y: typ): bool {.borrow, noSideEffect.}
proc `<=`*(x: typ, y: uint64): bool {.borrow, noSideEffect.}
proc `<=`*(x: uint64, y: typ): bool {.borrow, noSideEffect.}
proc `==`*(x: typ, y: typ): bool {.borrow, noSideEffect.}
proc `==`*(x: typ, y: uint64): bool {.borrow, noSideEffect.}
proc `==`*(x: uint64, y: typ): bool {.borrow, noSideEffect.}
# Nim integration
proc `$`*(x: typ): string {.borrow, noSideEffect.}
proc hash*(x: typ): Hash {.borrow, noSideEffect.}
template asUInt64*(v: typ): uint64 = distinctBase(v)
template shortLog*(v: typ): auto = distinctBase(v)
# Serialization
proc writeValue*(writer: var JsonWriter, value: typ)
{.raises: [IOError, Defect].}=
writeValue(writer, uint64 value)
proc readValue*(reader: var JsonReader, value: var typ)
{.raises: [IOError, SerializationError, Defect].} =
value = typ reader.readValue(uint64)
ethTimeUnit Slot
ethTimeUnit Epoch
ethTimeUnit SyncCommitteePeriod
template `<`*(a, b: BeaconTime): bool = a.ns_since_genesis < b.ns_since_genesis
template `<=`*(a, b: BeaconTime): bool = a.ns_since_genesis <= b.ns_since_genesis
template `<`*(a, b: TimeDiff): bool = a.nanoseconds < b.nanoseconds
template `<=`*(a, b: TimeDiff): bool = a.nanoseconds <= b.nanoseconds
template `<`*(a: TimeDiff, b: Duration): bool = a.nanoseconds < b.nanoseconds
func toSlot*(t: BeaconTime): tuple[afterGenesis: bool, slot: Slot] =
if t == FAR_FUTURE_BEACON_TIME:
(true, FAR_FUTURE_SLOT)
elif t.ns_since_genesis >= 0:
(true, Slot(uint64(t.ns_since_genesis) div NANOSECONDS_PER_SLOT))
else:
(false, Slot(uint64(-t.ns_since_genesis) div NANOSECONDS_PER_SLOT))
template `+`*(t: BeaconTime, offset: Duration | TimeDiff): BeaconTime =
BeaconTime(ns_since_genesis: t.ns_since_genesis + offset.nanoseconds)
template `-`*(t: BeaconTime, offset: Duration | TimeDiff): BeaconTime =
BeaconTime(ns_since_genesis: t.ns_since_genesis - offset.nanoseconds)
template `-`*(a, b: BeaconTime): TimeDiff =
TimeDiff(nanoseconds: a.ns_since_genesis - b.ns_since_genesis)
template `+`*(a: TimeDiff, b: Duration): TimeDiff =
TimeDiff(nanoseconds: a.nanoseconds + b.nanoseconds)
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.8/specs/phase0/validator.md#attesting
attestationSlotOffset* = TimeDiff(nanoseconds:
NANOSECONDS_PER_SLOT.int64 div INTERVALS_PER_SLOT)
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/validator.md#broadcast-aggregate
aggregateSlotOffset* = TimeDiff(nanoseconds:
NANOSECONDS_PER_SLOT.int64 * 2 div INTERVALS_PER_SLOT)
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/altair/validator.md#prepare-sync-committee-message
syncCommitteeMessageSlotOffset* = TimeDiff(nanoseconds:
NANOSECONDS_PER_SLOT.int64 div INTERVALS_PER_SLOT)
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/altair/validator.md#broadcast-sync-committee-contribution
syncContributionSlotOffset* = TimeDiff(nanoseconds:
NANOSECONDS_PER_SLOT.int64 * 2 div INTERVALS_PER_SLOT)
func toFloatSeconds*(t: TimeDiff): float =
float(t.nanoseconds) / 1_000_000_000.0
func start_beacon_time*(s: Slot): BeaconTime =
# The point in time that a slot begins
const maxSlot = Slot(
uint64(FAR_FUTURE_BEACON_TIME.ns_since_genesis) div NANOSECONDS_PER_SLOT)
if s > maxSlot: FAR_FUTURE_BEACON_TIME
else: BeaconTime(ns_since_genesis: int64(uint64(s) * NANOSECONDS_PER_SLOT))
func block_deadline*(s: Slot): BeaconTime =
s.start_beacon_time
func attestation_deadline*(s: Slot): BeaconTime =
s.start_beacon_time + attestationSlotOffset
func aggregate_deadline*(s: Slot): BeaconTime =
s.start_beacon_time + aggregateSlotOffset
func sync_committee_message_deadline*(s: Slot): BeaconTime =
s.start_beacon_time + syncCommitteeMessageSlotOffset
func sync_contribution_deadline*(s: Slot): BeaconTime =
s.start_beacon_time + syncContributionSlotOffset
func slotOrZero*(time: BeaconTime): Slot =
let exSlot = time.toSlot
if exSlot.afterGenesis: exSlot.slot
else: Slot(0)
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/beacon-chain.md#compute_epoch_at_slot
func epoch*(slot: Slot): Epoch = # aka compute_epoch_at_slot
## Return the epoch number at ``slot``.
if slot == FAR_FUTURE_SLOT: FAR_FUTURE_EPOCH
else: Epoch(slot div SLOTS_PER_EPOCH)
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/fork-choice.md#compute_slots_since_epoch_start
func since_epoch_start*(slot: Slot): uint64 = # aka compute_slots_since_epoch_start
## How many slots since the beginning of the epoch (`[0..SLOTS_PER_EPOCH-1]`)
(slot mod SLOTS_PER_EPOCH)
template is_epoch*(slot: Slot): bool =
slot.since_epoch_start == 0
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/beacon-chain.md#compute_start_slot_at_epoch
func start_slot*(epoch: Epoch): Slot = # aka compute_start_slot_at_epoch
## Return the start slot of ``epoch``.
const maxEpoch = Epoch(FAR_FUTURE_SLOT div SLOTS_PER_EPOCH)
if epoch >= maxEpoch: FAR_FUTURE_SLOT
else: Slot(epoch * SLOTS_PER_EPOCH)
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/beacon-chain.md#get_previous_epoch
func get_previous_epoch*(current_epoch: Epoch): Epoch =
## Return the previous epoch (unless the current epoch is ``GENESIS_EPOCH``).
if current_epoch == GENESIS_EPOCH:
current_epoch
else:
current_epoch - 1
iterator slots*(epoch: Epoch): Slot =
let start_slot = start_slot(epoch)
for slot in start_slot ..< start_slot + SLOTS_PER_EPOCH:
yield slot
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/altair/validator.md#sync-committee
template sync_committee_period*(epoch: Epoch): SyncCommitteePeriod =
if epoch == FAR_FUTURE_EPOCH: FAR_FUTURE_PERIOD
else: SyncCommitteePeriod(epoch div EPOCHS_PER_SYNC_COMMITTEE_PERIOD)
template sync_committee_period*(slot: Slot): SyncCommitteePeriod =
if slot == FAR_FUTURE_SLOT: FAR_FUTURE_PERIOD
else: SyncCommitteePeriod(slot div SLOTS_PER_SYNC_COMMITTEE_PERIOD)
func since_sync_committee_period_start*(slot: Slot): uint64 =
## How many slots since the beginning of the epoch (`[0..SLOTS_PER_SYNC_COMMITTEE_PERIOD-1]`)
(slot mod SLOTS_PER_SYNC_COMMITTEE_PERIOD)
func since_sync_committee_period_start*(epoch: Epoch): uint64 =
## How many slots since the beginning of the epoch (`[0..EPOCHS_PER_SYNC_COMMITTEE_PERIOD-1]`)
(epoch mod EPOCHS_PER_SYNC_COMMITTEE_PERIOD)
template is_sync_committee_period*(slot: Slot): bool =
slot.since_sync_committee_period_start() == 0
template is_sync_committee_period*(epoch: Epoch): bool =
epoch.since_sync_committee_period_start() == 0
template start_epoch*(period: SyncCommitteePeriod): Epoch =
const maxPeriod = SyncCommitteePeriod(
FAR_FUTURE_EPOCH div EPOCHS_PER_SYNC_COMMITTEE_PERIOD)
if period >= maxPeriod: FAR_FUTURE_EPOCH
else: Epoch(period * EPOCHS_PER_SYNC_COMMITTEE_PERIOD)
func shortLog*(t: BeaconTime): string =
if t.ns_since_genesis >= 0:
$(timer.nanoseconds(t.ns_since_genesis))
else:
"-" & $(timer.nanoseconds(-t.ns_since_genesis))
func shortLog*(t: TimeDiff): string =
if t.nanoseconds >= 0:
$(timer.nanoseconds(t.nanoseconds))
else:
"-" & $(timer.nanoseconds(-t.nanoseconds))
chronicles.formatIt BeaconTime: it.shortLog
chronicles.formatIt TimeDiff: it.shortLog
chronicles.formatIt Slot: it.shortLog
chronicles.formatIt Epoch: it.shortLog
chronicles.formatIt SyncCommitteePeriod: it.shortLog

View File

@ -352,7 +352,7 @@ func get_block_root_at_slot*(state: ForkedHashedBeaconState,
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/beacon-chain.md#get_block_root
func get_block_root*(state: ForkyBeaconState, epoch: Epoch): Eth2Digest =
## Return the block root at the start of a recent ``epoch``.
get_block_root_at_slot(state, compute_start_slot_at_epoch(epoch))
get_block_root_at_slot(state, epoch.start_slot())
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/beacon-chain.md#get_total_balance
template get_total_balance(
@ -488,7 +488,7 @@ proc is_valid_indexed_attestation*(
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/p2p-interface.md#beacon_attestation_subnet_id
func check_attestation_slot_target*(data: AttestationData): Result[Slot, cstring] =
if not (data.target.epoch == compute_epoch_at_slot(data.slot)):
if not (data.target.epoch == epoch(data.slot)):
return err("Target epoch doesn't match attestation slot")
ok(data.slot)

View File

@ -24,23 +24,19 @@
{.push raises: [Defect].}
import
json_serialization
export
json_serialization
import
std/[macros, hashes, strutils, tables, typetraits],
stew/[assign2, byteutils, results],
chronicles,
json_serialization,
chronos/timer,
ssz_serialization/types as sszTypes,
../../version,
".."/[crypto, digest, presets]
".."/[beacon_time, crypto, digest, presets]
export
timer, crypto, digest, sszTypes, presets, results
tables, results, json_serialization, timer, sszTypes, beacon_time, crypto,
digest, presets
# Presently, we're reusing the data types from the serialization (uint64) in the
# objects we pass around to the beacon chain logic, thus keeping the two
@ -61,14 +57,9 @@ const SPEC_VERSION* = "1.1.8"
## Spec version we're aiming to be compatible with, right now
const
GENESIS_SLOT* = Slot(0)
GENESIS_EPOCH* = (GENESIS_SLOT.uint64 div SLOTS_PER_EPOCH).Epoch ##\
## compute_epoch_at_slot(GENESIS_SLOT)
# Not part of spec. Still useful, pending removing usage if appropriate.
ZERO_HASH* = Eth2Digest()
MAX_GRAFFITI_SIZE* = 32
FAR_FUTURE_SLOT* = (not 0'u64).Slot
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/p2p-interface.md#configuration
MAXIMUM_GOSSIP_CLOCK_DISPARITY* = 500.millis
@ -82,9 +73,6 @@ const
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/validator.md#misc
ATTESTATION_SUBNET_COUNT* = 64
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/fork-choice.md#constant
INTERVALS_PER_SLOT* = 3
template maxSize*(n: int) {.pragma.}
# Block validation flow
@ -547,51 +535,6 @@ func getImmutableValidatorData*(validator: Validator): ImmutableValidatorData2 =
pubkey: cookedKey.get(),
withdrawal_credentials: validator.withdrawal_credentials)
# TODO when https://github.com/nim-lang/Nim/issues/14440 lands in Status's Nim,
# switch proc {.noSideEffect.} to func.
template ethTimeUnit(typ: type) {.dirty.} =
proc `+`*(x: typ, y: uint64): typ {.borrow, noSideEffect.}
proc `-`*(x: typ, y: uint64): typ {.borrow, noSideEffect.}
proc `-`*(x: uint64, y: typ): typ {.borrow, noSideEffect.}
# Not closed over type in question (Slot or Epoch)
proc `mod`*(x: typ, y: uint64): uint64 {.borrow, noSideEffect.}
proc `div`*(x: typ, y: uint64): uint64 {.borrow, noSideEffect.}
proc `div`*(x: uint64, y: typ): uint64 {.borrow, noSideEffect.}
proc `-`*(x: typ, y: typ): uint64 {.borrow, noSideEffect.}
proc `*`*(x: typ, y: uint64): uint64 {.borrow, noSideEffect.}
proc `+=`*(x: var typ, y: typ) {.borrow, noSideEffect.}
proc `+=`*(x: var typ, y: uint64) {.borrow, noSideEffect.}
proc `-=`*(x: var typ, y: typ) {.borrow, noSideEffect.}
proc `-=`*(x: var typ, y: uint64) {.borrow, noSideEffect.}
# Comparison operators
proc `<`*(x: typ, y: typ): bool {.borrow, noSideEffect.}
proc `<`*(x: typ, y: uint64): bool {.borrow, noSideEffect.}
proc `<`*(x: uint64, y: typ): bool {.borrow, noSideEffect.}
proc `<=`*(x: typ, y: typ): bool {.borrow, noSideEffect.}
proc `<=`*(x: typ, y: uint64): bool {.borrow, noSideEffect.}
proc `<=`*(x: uint64, y: typ): bool {.borrow, noSideEffect.}
proc `==`*(x: typ, y: typ): bool {.borrow, noSideEffect.}
proc `==`*(x: typ, y: uint64): bool {.borrow, noSideEffect.}
proc `==`*(x: uint64, y: typ): bool {.borrow, noSideEffect.}
# Nim integration
proc `$`*(x: typ): string {.borrow, noSideEffect.}
proc hash*(x: typ): Hash {.borrow, noSideEffect.}
# Serialization
proc writeValue*(writer: var JsonWriter, value: typ)
{.raises: [IOError, Defect].}=
writeValue(writer, uint64 value)
proc readValue*(reader: var JsonReader, value: var typ)
{.raises: [IOError, SerializationError, Defect].} =
value = typ reader.readValue(uint64)
template makeLimitedU64*(T: untyped, limit: uint64) =
# A "tigher" type is often used for T, but for the range check to be effective
# it must make sense..
@ -728,10 +671,6 @@ func `as`*(d: DepositData, T: type DepositMessage): T =
withdrawal_credentials: d.withdrawal_credentials,
amount: d.amount)
ethTimeUnit Slot
ethTimeUnit Epoch
ethTimeUnit SyncCommitteePeriod
template newClone*[T: not ref](x: T): ref T =
# TODO not nil in return type: https://github.com/nim-lang/Nim/issues/14146
# TODO use only when x is a function call that returns a new instance!
@ -780,12 +719,6 @@ func `[]`*(v: ForkDigest | Version | DomainType, idx: int): byte =
template data*(v: ForkDigest | Version | DomainType): array[4, byte] =
distinctBase(v)
func shortLog*(s: Slot): uint64 =
s - GENESIS_SLOT
func shortLog*(e: Epoch): uint64 =
e - GENESIS_EPOCH
func shortLog*(v: BeaconBlockHeader): auto =
(
slot: shortLog(v.slot),
@ -867,8 +800,6 @@ func shortLog*(v: SomeSignedVoluntaryExit): auto =
signature: shortLog(v.signature)
)
chronicles.formatIt Slot: it.shortLog
chronicles.formatIt Epoch: it.shortLog
chronicles.formatIt AttestationData: it.shortLog
chronicles.formatIt Attestation: it.shortLog
chronicles.formatIt Checkpoint: it.shortLog

View File

@ -94,6 +94,11 @@ type
of BeaconBlockFork.Altair: altairData*: altair.SignedBeaconBlock
of BeaconBlockFork.Bellatrix: mergeData*: bellatrix.SignedBeaconBlock
ForkySigVerifiedSignedBeaconBlock* =
phase0.SigVerifiedSignedBeaconBlock |
altair.SigVerifiedSignedBeaconBlock |
bellatrix.SigVerifiedSignedBeaconBlock
ForkyTrustedSignedBeaconBlock* =
phase0.TrustedSignedBeaconBlock |
altair.TrustedSignedBeaconBlock |
@ -105,6 +110,11 @@ type
of BeaconBlockFork.Altair: altairData*: altair.TrustedSignedBeaconBlock
of BeaconBlockFork.Bellatrix: mergeData*: bellatrix.TrustedSignedBeaconBlock
SomeForkySignedBeaconBlock* =
ForkySignedBeaconBlock |
ForkySigVerifiedSignedBeaconBlock |
ForkyTrustedSignedBeaconBlock
EpochInfoFork* {.pure.} = enum
Phase0
Altair
@ -418,7 +428,7 @@ type
BeaconStateHeader = object
genesis_time: uint64
genesis_validators_root: Eth2Digest
slot: uint64
slot: Slot
func readSszForkedHashedBeaconState*(cfg: RuntimeConfig, data: openArray[byte]):
ForkedHashedBeaconState {.raises: [Defect, SszError].} =
@ -432,9 +442,8 @@ func readSszForkedHashedBeaconState*(cfg: RuntimeConfig, data: openArray[byte]):
BeaconStateHeader)
# careful - `result` is used, RVO didn't seem to work without
# TODO move time helpers somewhere to avoid circular imports
result = ForkedHashedBeaconState(
kind: cfg.stateForkAtEpoch(Epoch(header.slot div SLOTS_PER_EPOCH)))
kind: cfg.stateForkAtEpoch(header.slot.epoch()))
withState(result):
readSszBytes(data, state.data)
@ -460,7 +469,7 @@ func readSszForkedSignedBeaconBlock*(
# careful - `result` is used, RVO didn't seem to work without
# TODO move time helpers somewhere to avoid circular imports
result = ForkedSignedBeaconBlock(
kind: cfg.blockForkAtEpoch(Epoch(header.slot div SLOTS_PER_EPOCH)))
kind: cfg.blockForkAtEpoch(header.slot.epoch()))
withBlck(result):
readSszBytes(data, blck)

View File

@ -37,17 +37,6 @@ func integer_squareroot*(n: SomeInteger): SomeInteger =
y = (x + n div x) div 2
x
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/beacon-chain.md#compute_epoch_at_slot
func compute_epoch_at_slot*(slot: Slot|uint64): Epoch =
## Return the epoch number at ``slot``.
(slot div SLOTS_PER_EPOCH).Epoch
template epoch*(slot: Slot): Epoch =
compute_epoch_at_slot(slot)
template isEpoch*(slot: Slot): bool =
(slot mod SLOTS_PER_EPOCH) == 0
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/ssz/merkle-proofs.md#generalized_index_sibling
template generalized_index_sibling*(
index: GeneralizedIndex): GeneralizedIndex =
@ -352,18 +341,6 @@ func build_proof*(anchor: object, leaf_index: uint64,
doAssert proof.len == log2trunc(leaf_index)
build_proof_impl(anchor, leaf_index, proof)
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/altair/validator.md#sync-committee
template sync_committee_period*(epoch: Epoch): SyncCommitteePeriod =
(epoch div EPOCHS_PER_SYNC_COMMITTEE_PERIOD).SyncCommitteePeriod
template sync_committee_period*(slot: Slot): SyncCommitteePeriod =
sync_committee_period(epoch(slot))
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/beacon-chain.md#compute_start_slot_at_epoch
func compute_start_slot_at_epoch*(epoch: Epoch): Slot =
## Return the start slot of ``epoch``.
(epoch * SLOTS_PER_EPOCH).Slot
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/beacon-chain.md#is_active_validator
func is_active_validator*(validator: Validator, epoch: Epoch): bool =
## Check if ``validator`` is active
@ -405,13 +382,12 @@ func get_active_validator_indices_len*(
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/beacon-chain.md#get_current_epoch
func get_current_epoch*(state: ForkyBeaconState): Epoch =
## Return the current epoch.
doAssert state.slot >= GENESIS_SLOT, $state.slot
compute_epoch_at_slot(state.slot)
state.slot.epoch
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/beacon-chain.md#get_current_epoch
func get_current_epoch*(state: ForkedHashedBeaconState): Epoch =
## Return the current epoch.
withState(state): state.data.slot.epoch
withState(state): get_current_epoch(state.data)
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/beacon-chain.md#get_randao_mix
func get_randao_mix*(state: ForkyBeaconState, epoch: Epoch): Eth2Digest =

View File

@ -26,12 +26,8 @@ proc validate_light_client_update*(store: LightClientStore,
# Verify update does not skip a sync committee period
let
finalized_period =
compute_epoch_at_slot(store.finalized_header.slot) div
EPOCHS_PER_SYNC_COMMITTEE_PERIOD
update_period =
compute_epoch_at_slot(active_header.slot) div
EPOCHS_PER_SYNC_COMMITTEE_PERIOD
finalized_period = sync_committee_period(store.finalized_header.slot)
update_period = sync_committee_period(active_header.slot)
if update_period notin [finalized_period, finalized_period + 1]:
return false
@ -90,12 +86,8 @@ func apply_light_client_update(
store: var LightClientStore, update: LightClientUpdate) =
let
active_header = get_active_header(update)
finalized_period =
compute_epoch_at_slot(store.finalized_header.slot) div
EPOCHS_PER_SYNC_COMMITTEE_PERIOD
update_period =
compute_epoch_at_slot(active_header.slot) div
EPOCHS_PER_SYNC_COMMITTEE_PERIOD
finalized_period = sync_committee_period(store.finalized_header.slot)
update_period = sync_committee_period(active_header.slot)
if update_period == finalized_period + 1:
store.current_sync_committee = store.next_sync_committee
store.next_sync_committee = update.next_sync_committee

View File

@ -69,12 +69,12 @@ func compute_subnet_for_attestation*(
# Note, this mimics expected Phase 1 behavior where attestations will be
# mapped to their shard subnet.
let
slots_since_epoch_start = slot mod SLOTS_PER_EPOCH
slots_since_epoch_start = slot.since_epoch_start()
committees_since_epoch_start =
committees_per_slot * slots_since_epoch_start
SubnetId(
(committees_since_epoch_start + committee_index.uint64) mod
(committees_since_epoch_start + committee_index.asUInt64) mod
ATTESTATION_SUBNET_COUNT)
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/validator.md#broadcast-attestation

View File

@ -37,7 +37,7 @@ func compute_slot_signing_root*(
fork: Fork, genesis_validators_root: Eth2Digest, slot: Slot
): Eth2Digest =
let
epoch = compute_epoch_at_slot(slot)
epoch = epoch(slot)
domain = get_domain(
fork, DOMAIN_SELECTION_PROOF, epoch, genesis_validators_root)
compute_signing_root(slot, domain)
@ -88,7 +88,7 @@ func compute_block_signing_root*(
fork: Fork, genesis_validators_root: Eth2Digest, slot: Slot,
blck: Eth2Digest | SomeSomeBeaconBlock | BeaconBlockHeader): Eth2Digest =
let
epoch = compute_epoch_at_slot(slot)
epoch = epoch(slot)
domain = get_domain(
fork, DOMAIN_BEACON_PROPOSER, epoch, genesis_validators_root)
compute_signing_root(blck, domain)
@ -117,7 +117,7 @@ func compute_aggregate_and_proof_signing_root*(
fork: Fork, genesis_validators_root: Eth2Digest,
aggregate_and_proof: AggregateAndProof): Eth2Digest =
let
epoch = compute_epoch_at_slot(aggregate_and_proof.aggregate.data.slot)
epoch = epoch(aggregate_and_proof.aggregate.data.slot)
domain = get_domain(
fork, DOMAIN_AGGREGATE_AND_PROOF, epoch, genesis_validators_root)
compute_signing_root(aggregate_and_proof, domain)

View File

@ -250,7 +250,7 @@ proc collectSignatureSets*(
if not proposer_key.isSome():
return err("collectSignatureSets: invalid proposer index")
let epoch = signed_block.message.slot.compute_epoch_at_slot()
let epoch = signed_block.message.slot.epoch()
# 1. Block proposer
# ----------------------------------------------------

View File

@ -41,7 +41,6 @@
{.push raises: [Defect].}
import
std/tables,
chronicles,
stew/results,
metrics,
@ -51,15 +50,12 @@ import
beaconstate, eth2_merkleization, forks, helpers, signatures,
state_transition_block, state_transition_epoch, validator]
export extras, phase0, altair
type Foo = phase0.SignedBeaconBlock | altair.SignedBeaconBlock | phase0.TrustedSignedBeaconBlock | altair.TrustedSignedBeaconBlock | phase0.SigVerifiedSignedBeaconBlock | altair.SigVerifiedSignedBeaconBlock | bellatrix.TrustedSignedBeaconBlock | bellatrix.SigVerifiedSignedBeaconBlock | bellatrix.SignedBeaconBlock
export extras, phase0, altair, bellatrix
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/beacon-chain.md#beacon-chain-state-transition-function
proc verify_block_signature(
#state: ForkyBeaconState, signed_block: SomeSomeSignedBeaconBlock): bool =
state: ForkyBeaconState, signed_block: Foo): bool =
#state: ForkyBeaconState, signed_block: phase0.SomeSignedBeaconBlock | altair.SomeSignedBeaconBlock): bool =
state: ForkyBeaconState, signed_block: SomeForkySignedBeaconBlock): bool =
let
proposer_index = signed_block.message.proposer_index
if proposer_index >= state.validators.lenu64:
@ -153,12 +149,9 @@ func process_slot*(
func clear_epoch_from_cache(cache: var StateCache, epoch: Epoch) =
cache.shuffled_active_validator_indices.del epoch
let
start_slot = epoch.compute_start_slot_at_epoch
end_slot = (epoch + 1).compute_start_slot_at_epoch
for i in start_slot ..< end_slot:
cache.beacon_proposer_indices.del i
for slot in epoch.slots():
cache.beacon_proposer_indices.del slot
# https://github.com/ethereum/consensus-specs/blob/v1.0.1/specs/phase0/beacon-chain.md#beacon-chain-state-transition-function
proc advance_slot(
@ -172,11 +165,11 @@ proc advance_slot(
info.clear()
let is_epoch_transition = (state.slot + 1).isEpoch
let is_epoch_transition = (state.slot + 1).is_epoch
if is_epoch_transition:
# Note: Genesis epoch = 0, no need to test if before Genesis
process_epoch(cfg, state, flags, cache, info)
clear_epoch_from_cache(cache, (state.slot + 1).compute_epoch_at_slot)
clear_epoch_from_cache(cache, (state.slot + 1).epoch)
state.slot += 1
@ -329,10 +322,7 @@ proc state_transition_block*(
proc state_transition*(
cfg: RuntimeConfig,
state: var ForkedHashedBeaconState,
signedBlock: phase0.SignedBeaconBlock | phase0.SigVerifiedSignedBeaconBlock |
phase0.TrustedSignedBeaconBlock | altair.SignedBeaconBlock |
altair.TrustedSignedBeaconBlock | bellatrix.TrustedSignedBeaconBlock |
bellatrix.SignedBeaconBlock,
signedBlock: SomeForkySignedBeaconBlock,
cache: var StateCache, info: var ForkedEpochInfo, flags: UpdateFlags,
rollback: RollbackForkedHashedProc): bool =
## Apply a block to the state, advancing the slot counter as necessary. The

View File

@ -944,7 +944,7 @@ func process_participation_flag_updates*(state: var (altair.BeaconState | bellat
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/altair/beacon-chain.md#sync-committee-updates
proc process_sync_committee_updates*(state: var (altair.BeaconState | bellatrix.BeaconState)) =
let next_epoch = get_current_epoch(state) + 1
if next_epoch mod EPOCHS_PER_SYNC_COMMITTEE_PERIOD == 0:
if next_epoch.is_sync_committee_period():
state.current_sync_committee = state.next_sync_committee
state.next_sync_committee = get_next_sync_committee(state)

View File

@ -193,14 +193,6 @@ iterator get_committee_indices*(state: ForkyBeaconState | ForkedHashedBeaconStat
for committee_index in get_committee_indices(committee_count_per_slot):
yield committee_index
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/beacon-chain.md#get_previous_epoch
func get_previous_epoch*(current_epoch: Epoch): Epoch =
## Return the previous epoch (unless the current epoch is ``GENESIS_EPOCH``).
if current_epoch == GENESIS_EPOCH:
current_epoch
else:
current_epoch - 1
func get_previous_epoch*(state: ForkyBeaconState): Epoch =
## Return the previous epoch (unless the current epoch is ``GENESIS_EPOCH``).
# Return the previous epoch (unless the current epoch is ``GENESIS_EPOCH``).
@ -263,7 +255,7 @@ iterator get_beacon_committee*(
cache: var StateCache): (int, ValidatorIndex) =
## Return the beacon committee at ``slot`` for ``index``.
let
epoch = compute_epoch_at_slot(slot)
epoch = epoch(slot)
committees_per_slot = get_committee_count_per_slot(state, epoch, cache)
for index_in_committee, idx in compute_committee(
cache.get_shuffled_active_validator_indices(state, epoch),
@ -277,7 +269,7 @@ func get_beacon_committee*(
cache: var StateCache): seq[ValidatorIndex] =
## Return the beacon committee at ``slot`` for ``index``.
let
epoch = compute_epoch_at_slot(slot)
epoch = epoch(slot)
committees_per_slot = get_committee_count_per_slot(state, epoch, cache)
compute_committee(
cache.get_shuffled_active_validator_indices(state, epoch),
@ -303,7 +295,7 @@ func get_beacon_committee_len*(
cache: var StateCache): uint64 =
# Return the number of members in the beacon committee at ``slot`` for ``index``.
let
epoch = compute_epoch_at_slot(slot)
epoch = epoch(slot)
committees_per_slot = get_committee_count_per_slot(state, epoch, cache)
compute_committee_len(
@ -408,16 +400,15 @@ func get_beacon_proposer_index*(
# active validator indices are kept in cache but sorting them takes
# quite a while
indices = get_active_validator_indices(state, epoch)
start = epoch.compute_start_slot_at_epoch()
var res: Option[ValidatorIndex]
for i in 0..<SLOTS_PER_EPOCH:
buffer[32..39] = uint_to_bytes((start + i).uint64)
for epoch_slot in epoch.slots():
buffer[32..39] = uint_to_bytes(epoch_slot.asUInt64)
let seed = eth2digest(buffer)
let pi = compute_proposer_index(state, indices, seed)
if start + i == slot:
if epoch_slot == slot:
res = pi
cache.beacon_proposer_indices[start + i] = pi
cache.beacon_proposer_indices[epoch_slot] = pi
return res

View File

@ -33,13 +33,12 @@ func is_within_weak_subjectivity_period*(cfg: RuntimeConfig, current_slot: Slot,
# Clients may choose to validate the input state against the input Weak Subjectivity Checkpoint
doAssert getStateField(ws_state, latest_block_header).state_root ==
ws_checkpoint.root
doAssert compute_epoch_at_slot(getStateField(ws_state, slot)) ==
ws_checkpoint.epoch
doAssert epoch(getStateField(ws_state, slot)) == ws_checkpoint.epoch
let
ws_period = compute_weak_subjectivity_period(cfg, ws_state)
ws_state_epoch = compute_epoch_at_slot(getStateField(ws_state, slot))
current_epoch = compute_epoch_at_slot(current_slot)
ws_state_epoch = epoch(getStateField(ws_state, slot))
current_epoch = epoch(current_slot)
current_epoch <= ws_state_epoch + ws_period

View File

@ -98,7 +98,7 @@ func getMutableValidatorStatuses(state: altair.BeaconState):
func diffStates*(state0, state1: altair.BeaconState): BeaconStateDiff =
doAssert state1.slot > state0.slot
doAssert state0.slot.isEpoch
doAssert state0.slot.is_epoch
doAssert state1.slot == state0.slot + SLOTS_PER_EPOCH
# TODO not here, but in dag, an isancestorof check
@ -134,9 +134,9 @@ func diffStates*(state0, state1: altair.BeaconState): BeaconStateDiff =
balances: state1.balances.data,
# RANDAO mixes gets updated every block, in place
randao_mix: state1.randao_mixes[state0.slot.compute_epoch_at_slot.uint64 mod
randao_mix: state1.randao_mixes[state0.slot.epoch.uint64 mod
EPOCHS_PER_HISTORICAL_VECTOR.uint64],
slashing: state1.slashings[state0.slot.compute_epoch_at_slot.uint64 mod
slashing: state1.slashings[state0.slot.epoch.uint64 mod
EPOCHS_PER_HISTORICAL_VECTOR.uint64],
previous_epoch_participation: state1.previous_epoch_participation.data,

View File

@ -378,10 +378,10 @@ proc getRewindPoint*[T](sq: SyncQueue[T], failSlot: Slot,
case sq.kind
of SyncQueueKind.Forward:
# Calculate the latest finalized epoch.
let finalizedEpoch = compute_epoch_at_slot(safeSlot)
let finalizedEpoch = epoch(safeSlot)
# Calculate failure epoch.
let failEpoch = compute_epoch_at_slot(failSlot)
let failEpoch = epoch(failSlot)
# Calculate exponential rewind point in number of epochs.
let epochCount =
@ -445,16 +445,15 @@ proc getRewindPoint*[T](sq: SyncQueue[T], failSlot: Slot,
if sq.rewind.isNone():
finalizedEpoch
else:
compute_epoch_at_slot(sq.rewind.get().failSlot) -
sq.rewind.get().epochCount
compute_start_slot_at_epoch(rewindEpoch)
epoch(sq.rewind.get().failSlot) - sq.rewind.get().epochCount
rewindEpoch.start_slot()
else:
# Calculate the rewind epoch, which should not be less than the latest
# finalized epoch.
let rewindEpoch = failEpoch - epochCount
# Update and save new rewind point in SyncQueue.
sq.rewind = some(RewindPoint(failSlot: failSlot, epochCount: epochCount))
compute_start_slot_at_epoch(rewindEpoch)
rewindEpoch.start_slot()
of SyncQueueKind.Backward:
# While we perform backward sync, the only possible slot we could rewind is
# latest stored block.

View File

@ -56,7 +56,7 @@ proc serveAttestation(service: AttestationServiceRef, adata: AttestationData,
debug "Sending attestation", attestation = shortLog(attestation),
validator = shortLog(validator), validator_index = vindex,
attestation_root = shortLog(attestationRoot),
delay = vc.getDelay(attestationSlotOffset)
delay = vc.getDelay(adata.slot.attestation_deadline())
let res =
try:
@ -75,7 +75,7 @@ proc serveAttestation(service: AttestationServiceRef, adata: AttestationData,
err_name = exc.name, err_msg = exc.msg
return false
let delay = vc.getDelay(attestationSlotOffset)
let delay = vc.getDelay(adata.slot.attestation_deadline())
if res:
notice "Attestation published", attestation = shortLog(attestation),
validator = shortLog(validator),
@ -105,21 +105,20 @@ proc serveAggregateAndProof*(service: AttestationServiceRef,
if res.isErr():
error "Unable to sign aggregate and proof using remote signer",
validator = shortLog(validator),
aggregationSlot = proof.aggregate.data.slot,
attestation = shortLog(proof.aggregate),
error_msg = res.error()
return false
res.get()
let signedProof = SignedAggregateAndProof(message: proof,
signature: signature)
let aggregationSlot = proof.aggregate.data.slot
let slot = proof.aggregate.data.slot
let vindex = validator.index.get()
debug "Sending aggregated attestation",
attestation = shortLog(signedProof.message.aggregate),
validator = shortLog(validator), validator_index = vindex,
aggregationSlot = aggregationSlot,
delay = vc.getDelay(aggregateSlotOffset)
delay = vc.getDelay(slot.aggregate_deadline())
let res =
try:
@ -128,15 +127,12 @@ proc serveAggregateAndProof*(service: AttestationServiceRef,
error "Unable to publish aggregated attestation",
attestation = shortLog(signedProof.message.aggregate),
validator = shortLog(validator),
aggregationSlot = aggregationSlot,
validator_index = vindex
return false
except CatchableError as exc:
error "Unexpected error occured while publishing aggregated attestation",
attestation = shortLog(signedProof.message.aggregate),
validator = shortLog(validator),
aggregationSlot = aggregationSlot,
validator_index = vindex,
err_name = exc.name, err_msg = exc.msg
return false
@ -144,12 +140,12 @@ proc serveAggregateAndProof*(service: AttestationServiceRef,
notice "Aggregated attestation published",
attestation = shortLog(signedProof.message.aggregate),
validator = shortLog(validator),
aggregationSlot = aggregationSlot, validator_index = vindex
validator_index = vindex
else:
warn "Aggregated attestation was not accepted by beacon node",
attestation = shortLog(signedProof.message.aggregate),
validator = shortLog(validator),
aggregationSlot = aggregationSlot, validator_index = vindex
validator_index = vindex
return res
proc produceAndPublishAttestations*(service: AttestationServiceRef,
@ -201,7 +197,7 @@ proc produceAndPublishAttestations*(service: AttestationServiceRef,
inc(errored)
(succeed, errored, failed)
let delay = vc.getDelay(attestationSlotOffset)
let delay = vc.getDelay(slot.attestation_deadline())
debug "Attestation statistics", total = len(pendingAttestations),
succeed = statistics[0], failed_to_deliver = statistics[1],
not_accepted = statistics[2], delay = delay, slot = slot,
@ -288,7 +284,7 @@ proc produceAndPublishAggregates(service: AttestationServiceRef,
inc(errored)
(succeed, errored, failed)
let delay = vc.getDelay(aggregateSlotOffset)
let delay = vc.getDelay(slot.aggregate_deadline())
debug "Aggregated attestation statistics", total = len(pendingAggregates),
succeed = statistics[0], failed_to_deliver = statistics[1],
not_accepted = statistics[2], delay = delay, slot = slot,
@ -307,7 +303,7 @@ proc publishAttestationsAndAggregates(service: AttestationServiceRef,
let startTime = Moment.now()
try:
let timeout = attestationSlotOffset # 4.seconds in mainnet
await vc.waitForBlockPublished(slot).wait(timeout)
await vc.waitForBlockPublished(slot).wait(nanoseconds(timeout.nanoseconds))
let dur = Moment.now() - startTime
debug "Block proposal awaited", slot = slot, duration = dur
except AsyncTimeoutError:
@ -315,7 +311,7 @@ proc publishAttestationsAndAggregates(service: AttestationServiceRef,
debug "Block was not produced in time", slot = slot, duration = dur
block:
let delay = vc.getDelay(attestationSlotOffset)
let delay = vc.getDelay(slot.attestation_deadline())
debug "Producing attestations", delay = delay, slot = slot,
committee_index = committee_index,
duties_count = len(duties)
@ -340,7 +336,7 @@ proc publishAttestationsAndAggregates(service: AttestationServiceRef,
await sleepAsync(aggregateTime)
block:
let delay = vc.getDelay(aggregateSlotOffset)
let delay = vc.getDelay(slot.aggregate_deadline())
debug "Producing aggregate and proofs", delay = delay
await service.produceAndPublishAggregates(ad, duties)
@ -364,8 +360,9 @@ proc mainLoop(service: AttestationServiceRef) {.async.} =
service.state = ServiceState.Running
try:
while true:
let sleepTime = vc.beaconClock.durationToNextSlot() +
attestationSlotOffset
let sleepTime =
attestationSlotOffset + vc.beaconClock.durationToNextSlot()
let sres = vc.getCurrentSlot()
if sres.isSome():
let currentSlot = sres.get()

View File

@ -16,7 +16,7 @@ proc publishBlock(vc: ValidatorClientRef, currentSlot, slot: Slot,
fork = vc.fork.get()
debug "Publishing block", validator = shortLog(validator),
delay = vc.getDelay(ZeroDuration),
delay = vc.getDelay(slot.block_deadline()),
wall_slot = currentSlot,
genesis_root = genesisRoot,
graffiti = graffiti, fork = fork, slot = slot,
@ -152,7 +152,7 @@ proc contains(data: openArray[ProposerTask], duty: RestProposerDuty): bool =
false
proc checkDuty(duty: RestProposerDuty, epoch: Epoch, slot: Slot): bool =
let lastSlot = compute_start_slot_at_epoch(epoch + 1'u64)
let lastSlot = start_slot(epoch + 1'u64)
if duty.slot >= slot:
if duty.slot < lastSlot:
true

View File

@ -113,7 +113,6 @@ type
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).seconds div INTERVALS_PER_SLOT
proc `$`*(bn: BeaconNodeServerRef): string =
@ -165,7 +164,7 @@ proc getCurrentSlot*(vc: ValidatorClientRef): Option[Slot] =
wallSlot = wallTime.toSlot()
if not(wallSlot.afterGenesis):
let checkGenesisTime = vc.beaconClock.fromNow(toBeaconTime(Slot(0)))
let checkGenesisTime = vc.beaconClock.fromNow(start_beacon_time(Slot(0)))
warn "Jump in time detected, something wrong with wallclock",
wall_time = wallTime, genesisIn = checkGenesisTime.offset
none[Slot]()
@ -186,45 +185,39 @@ proc getAttesterDutiesForSlot*(vc: ValidatorClientRef,
proc getDurationToNextAttestation*(vc: ValidatorClientRef,
slot: Slot): string =
var minimumDuration = InfiniteDuration
let currentSlotTime = Duration(slot.toBeaconTime())
var minSlot = FAR_FUTURE_SLOT
let currentEpoch = slot.epoch()
for epoch in [currentEpoch, currentEpoch + 1'u64]:
for key, item in vc.attesters.pairs():
let duty = item.duties.getOrDefault(epoch, DefaultDutyAndProof)
if not(duty.isDefault()):
let dutySlotTime = Duration(duty.data.slot.toBeaconTime())
if dutySlotTime >= currentSlotTime:
let timeLeft = dutySlotTime - currentSlotTime
if timeLeft < minimumDuration:
minimumDuration = timeLeft
if minimumDuration != InfiniteDuration:
let dutySlotTime = duty.data.slot
if duty.data.slot < minSlot:
minSlot = duty.data.slot
if minSlot != FAR_FUTURE_SLOT:
break
if minimumDuration == InfiniteDuration:
if minSlot == FAR_FUTURE_SLOT:
"<unknown>"
else:
$(minimumDuration + seconds(int64(SECONDS_PER_SLOT) div 3))
$(minSlot.attestation_deadline() - slot.start_beacon_time())
proc getDurationToNextBlock*(vc: ValidatorClientRef, slot: Slot): string =
var minimumDuration = InfiniteDuration
var currentSlotTime = Duration(slot.toBeaconTime())
var minSlot = FAR_FUTURE_SLOT
let currentEpoch = slot.epoch()
for epoch in [currentEpoch, currentEpoch + 1'u64]:
let data = vc.proposers.getOrDefault(epoch)
if not(data.isDefault()):
for item in data.duties:
if item.duty.pubkey in vc.attachedValidators:
let proposalSlotTime = Duration(item.duty.slot.toBeaconTime())
if proposalSlotTime >= currentSlotTime:
let timeLeft = proposalSlotTime - currentSlotTime
if timeLeft < minimumDuration:
minimumDuration = timeLeft
if minimumDuration != InfiniteDuration:
if item.duty.slot < minSlot:
minSlot = item.duty.slot
if minSlot != FAR_FUTURE_SLOT:
break
if minimumDuration == InfiniteDuration:
if minSlot == FAR_FUTURE_SLOT:
"<unknown>"
else:
$minimumDuration
$(minSlot.block_deadline() - slot.start_beacon_time())
iterator attesterDutiesForEpoch*(vc: ValidatorClientRef,
epoch: Epoch): DutyAndProof =
@ -233,12 +226,8 @@ iterator attesterDutiesForEpoch*(vc: ValidatorClientRef,
if not(isDefault(epochDuties)):
yield epochDuties
proc getDelay*(vc: ValidatorClientRef, instant: Duration): Duration =
let currentBeaconTime = vc.beaconClock.now()
let currentTime = Duration(currentBeaconTime)
let slotStartTime = currentBeaconTime.slotOrZero().toBeaconTime()
let idealTime = Duration(slotStartTime) + instant
currentTime - idealTime
proc getDelay*(vc: ValidatorClientRef, deadline: BeaconTime): TimeDiff =
vc.beaconClock.now() - deadline
proc getValidator*(vc: ValidatorClientRef,
key: ValidatorPubkey): Option[AttachedValidator] =

View File

@ -178,8 +178,7 @@ func getNextValidatorAction*(
return FAR_FUTURE_SLOT
for slotOffset in 0 ..< SLOTS_PER_EPOCH:
let nextActionSlot =
compute_start_slot_at_epoch(bitmapEpoch) + slotOffset
let nextActionSlot = start_slot(bitmapEpoch) + slotOffset
if ((orderedActionSlots[i] and (1'u32 shl slotOffset)) != 0) and
nextActionSlot > slot:
return nextActionSlot
@ -222,7 +221,7 @@ proc updateActions*(tracker: var ActionTracker, epochRef: EpochRef) =
for (committeeIndex, subnet_id, slot) in
get_committee_assignments(epochRef, validatorIndices):
doAssert compute_epoch_at_slot(slot) == epoch
doAssert epoch(slot) == epoch
# Each get_committee_assignments() call here is on the next epoch. At any
# given time, only care about two epochs, the current and next epoch. So,

View File

@ -1193,7 +1193,7 @@ proc pruneAfterFinalization*(
## slashing protection can fallback to the minimal / high-watermark protection mode.
block: # Prune blocks
let finalizedSlot = compute_start_slot_at_epoch(finalizedEpoch)
let finalizedSlot = start_slot(finalizedEpoch)
let status = db.sqlPruneAfterFinalizationBlocks
.exec(int64 finalizedSlot)
doAssert status.isOk(),

View File

@ -381,7 +381,7 @@ proc createAndSendAttestation(node: BeaconNode,
validator.pubkey)
let wallTime = node.beaconClock.now()
let deadline = attestationData.slot.toBeaconTime(attestationSlotOffset)
let deadline = attestationData.slot.attestation_deadline()
let (delayStr, delaySecs) =
if wallTime < deadline:
@ -699,7 +699,7 @@ proc createAndSendSyncCommitteeMessage(node: BeaconNode,
let
wallTime = node.beaconClock.now()
deadline = msg.slot.toBeaconTime(syncCommitteeMessageSlotOffset)
deadline = msg.slot.start_beacon_time() + syncCommitteeMessageSlotOffset
let (delayStr, delaySecs) =
if wallTime < deadline:
@ -1067,8 +1067,7 @@ proc handleValidatorDuties*(node: BeaconNode, lastSlot, slot: Slot) {.async.} =
let
# The latest point in time when we'll be sending out attestations
attestationCutoffTime = slot.toBeaconTime(attestationSlotOffset)
attestationCutoff = node.beaconClock.fromNow(attestationCutoffTime)
attestationCutoff = node.beaconClock.fromNow(slot.attestation_deadline())
if attestationCutoff.inFuture:
debug "Waiting to send attestations",
@ -1096,7 +1095,7 @@ proc handleValidatorDuties*(node: BeaconNode, lastSlot, slot: Slot) {.async.} =
let
afterBlockTime = node.beaconClock.now() + millis(afterBlockDelay)
afterBlockCutoff = node.beaconClock.fromNow(
min(afterBlockTime, attestationCutoffTime + millis(afterBlockDelay)))
min(afterBlockTime, slot.attestation_deadline() + millis(afterBlockDelay)))
if afterBlockCutoff.inFuture:
debug "Got block, waiting to send attestations",
@ -1123,10 +1122,9 @@ proc handleValidatorDuties*(node: BeaconNode, lastSlot, slot: Slot) {.async.} =
# of the way through the `slot`-that is,
# `SECONDS_PER_SLOT * 2 / INTERVALS_PER_SLOT` seconds after the start of `slot`.
if slot > 2:
static: doAssert aggregateSlotOffset == syncContributionSlotOffset
doAssert slot.aggregate_deadline() == slot.sync_contribution_deadline()
let
aggregateCutoffTime = slot.toBeaconTime(aggregateSlotOffset)
aggregateCutoff = node.beaconClock.fromNow(aggregateCutoffTime)
aggregateCutoff = node.beaconClock.fromNow(slot.aggregate_deadline())
if aggregateCutoff.inFuture:
debug "Waiting to send aggregate attestations",
aggregateCutoff = shortLog(aggregateCutoff.offset)
@ -1181,7 +1179,7 @@ proc sendAttestation*(node: BeaconNode,
let
wallTime = node.processor.getCurrentBeaconTime()
deadline = attestation.data.slot.toBeaconTime(attestationSlotOffset)
deadline = attestation.data.slot.attestation_deadline()
(delayStr, delaySecs) =
if wallTime < deadline:
("-" & $(deadline - wallTime), -toFloatSeconds(deadline - wallTime))

View File

@ -138,22 +138,22 @@ type
## nature of gossip processing: in particular, old messages may reappear
## on the network and therefore be double-counted.
attestations: int64
attestation_min_delay: Option[Duration]
attestation_min_delay: Option[TimeDiff]
attestation_aggregate_inclusions: int64
attestation_block_inclusions: int64
attestation_min_block_inclusion_distance: Option[uint64]
aggregates: int64
aggregate_min_delay: Option[Duration]
aggregate_min_delay: Option[TimeDiff]
sync_committee_messages: int64
sync_committee_message_min_delay: Option[Duration]
sync_committee_message_min_delay: Option[TimeDiff]
sync_signature_block_inclusions: int64
sync_signature_contribution_inclusions: int64
sync_contributions: int64
sync_contribution_min_delay: Option[Duration]
sync_contribution_min_delay: Option[TimeDiff]
exits: int64
proposer_slashings: int64
@ -510,9 +510,6 @@ template withMonitor(self: var ValidatorMonitor, idx: uint64, body: untyped): un
template withMonitor(self: var ValidatorMonitor, idx: ValidatorIndex, body: untyped): untyped =
withMonitor(self, idx.uint64, body)
proc delay(slot: Slot, time: BeaconTime, offset: Duration): Duration =
time - slot.toBeaconTime(offset)
proc registerAttestation*(
self: var ValidatorMonitor,
src: MsgSource,
@ -521,7 +518,7 @@ proc registerAttestation*(
idx: ValidatorIndex) =
let
slot = attestation.data.slot
delay = delay(slot, seen_timestamp, attestationSlotOffset)
delay = seen_timestamp - slot.attestation_deadline()
self.withMonitor(idx):
let id = monitor.id
@ -545,7 +542,7 @@ proc registerAggregate*(
attesting_indices: openArray[ValidatorIndex]) =
let
slot = signed_aggregate_and_proof.message.aggregate.data.slot
delay = delay(slot, seen_timestamp, aggregateSlotOffset)
delay = seen_timestamp - slot.aggregate_deadline()
aggregator_index = signed_aggregate_and_proof.message.aggregator_index
self.withMonitor(aggregator_index):
@ -610,7 +607,7 @@ proc registerBeaconBlock*(
let
id = monitor.id
slot = blck.slot
delay = delay(slot, seen_timestamp, seconds(0))
delay = seen_timestamp - slot.block_deadline()
validator_monitor_beacon_block_total.inc(1, [$src, metricId])
validator_monitor_beacon_block_delay_seconds.observe(
@ -628,7 +625,7 @@ proc registerSyncCommitteeMessage*(
let
id = monitor.id
slot = sync_committee_message.slot
delay = delay(slot, seen_timestamp, syncCommitteeMessageSlotOffset)
delay = seen_timestamp - slot.sync_committee_message_deadline()
validator_monitor_sync_committee_messages_total.inc(1, [$src, metricId])
validator_monitor_sync_committee_messages_delay_seconds.observe(
@ -651,7 +648,7 @@ proc registerSyncContribution*(
let
slot = sync_contribution.message.contribution.slot
beacon_block_root = sync_contribution.message.contribution.beacon_block_root
delay = delay(slot, seen_timestamp, syncContributionSlotOffset)
delay = seen_timestamp - slot.sync_contribution_deadline()
let aggregator_index = sync_contribution.message.aggregator_index
self.withMonitor(aggregator_index):

View File

@ -376,8 +376,7 @@ proc sign*(v: AttachedValidator, msg: ref SignedContributionAndProof,
func genRandaoReveal*(k: ValidatorPrivKey, fork: Fork,
genesis_validators_root: Eth2Digest,
slot: Slot): CookedSig =
get_epoch_signature(fork, genesis_validators_root,
slot.compute_epoch_at_slot, k)
get_epoch_signature(fork, genesis_validators_root, slot.epoch, k)
proc genRandaoReveal*(v: AttachedValidator, fork: Fork,
genesis_validators_root: Eth2Digest, slot: Slot):
@ -390,7 +389,7 @@ proc genRandaoReveal*(v: AttachedValidator, fork: Fork,
slot).toValidatorSig())
of ValidatorKind.Remote:
let res = await signWithRemoteValidator(v, fork, genesis_validators_root,
slot.compute_epoch_at_slot())
slot.epoch())
if res.isErr():
SignatureResult.err(res.error())
else:

View File

@ -124,7 +124,7 @@ proc doSlots(conf: NcliConf) =
cache = StateCache()
info = ForkedEpochInfo()
for i in 0'u64..<conf.slot:
let isEpoch = (getStateField(stateY[], slot) + 1).isEpoch
let isEpoch = (getStateField(stateY[], slot) + 1).is_epoch
withTimer(timers[if isEpoch: tApplyEpochSlot else: tApplySlot]):
doAssert process_slots(
defaultRuntimeConfig, stateY[], getStateField(stateY[], slot) + 1,

View File

@ -244,7 +244,7 @@ proc cmdBench(conf: DbConf, cfg: RuntimeConfig) =
template processBlocks(blocks: auto) =
for b in blocks.mitems():
while getStateField(stateData[].data, slot) < b.message.slot:
let isEpoch = (getStateField(stateData[].data, slot) + 1).isEpoch()
let isEpoch = (getStateField(stateData[].data, slot) + 1).is_epoch()
withTimer(timers[if isEpoch: tAdvanceEpoch else: tAdvanceSlot]):
let ok = process_slots(
dag.cfg, stateData[].data, getStateField(stateData[].data, slot) + 1, cache,
@ -267,7 +267,7 @@ proc cmdBench(conf: DbConf, cfg: RuntimeConfig) =
dbBenchmark.putBlock(b)
withState(stateData[].data):
if state.data.slot.isEpoch and conf.storeStates:
if state.data.slot.is_epoch and conf.storeStates:
if state.data.slot.epoch < 2:
dbBenchmark.putState(state.root, state.data)
dbBenchmark.checkpoint()
@ -606,7 +606,7 @@ proc cmdValidatorPerf(conf: DbConf, cfg: RuntimeConfig) =
proc processEpoch() =
let
prev_epoch_target_slot =
state[].data.get_previous_epoch().compute_start_slot_at_epoch()
state[].data.get_previous_epoch().start_slot()
penultimate_epoch_end_slot =
if prev_epoch_target_slot == 0: Slot(0)
else: prev_epoch_target_slot - 1
@ -669,7 +669,7 @@ proc cmdValidatorPerf(conf: DbConf, cfg: RuntimeConfig) =
dag.cfg, state[].data, nextSlot, cache, info, flags)
doAssert ok, "Slot processing can't fail with correct inputs"
if getStateField(state[].data, slot).isEpoch():
if getStateField(state[].data, slot).is_epoch():
processEpoch()
if not state_transition_block(
@ -684,7 +684,7 @@ proc cmdValidatorPerf(conf: DbConf, cfg: RuntimeConfig) =
info, {})
doAssert ok, "Slot processing can't fail with correct inputs"
if getStateField(state[].data, slot).isEpoch():
if getStateField(state[].data, slot).is_epoch():
processEpoch()
echo "validator_index,attestation_hits,attestation_misses,head_attestation_hits,head_attestation_misses,target_attestation_hits,target_attestation_misses,delay_avg,first_slot_head_attester_when_first_slot_empty,first_slot_head_attester_when_first_slot_not_empty"
@ -829,7 +829,7 @@ proc cmdValidatorDb(conf: DbConf, cfg: RuntimeConfig) =
blck: phase0.TrustedSignedBeaconBlock
let
start = minEpoch.compute_start_slot_at_epoch()
start = minEpoch.start_slot()
ends = dag.finalizedHead.slot # Avoid dealing with changes
if start > ends:
@ -914,7 +914,7 @@ proc cmdValidatorDb(conf: DbConf, cfg: RuntimeConfig) =
let ok = process_slots(cfg, state[].data, nextSlot, cache, info, flags)
doAssert ok, "Slot processing can't fail with correct inputs"
if getStateField(state[].data, slot).isEpoch():
if getStateField(state[].data, slot).is_epoch():
processEpoch()
if not state_transition_block(
@ -930,7 +930,7 @@ proc cmdValidatorDb(conf: DbConf, cfg: RuntimeConfig) =
info, {})
doAssert ok, "Slot processing can't fail with correct inputs"
if getStateField(state[].data, slot).isEpoch():
if getStateField(state[].data, slot).is_epoch():
processEpoch()
if inTxn:

View File

@ -137,7 +137,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
data: data,
aggregation_bits: aggregation_bits,
signature: sig.toValidatorSig()
), [validator_index], sig, data.slot.toBeaconTime)
), [validator_index], sig, data.slot.start_beacon_time)
do:
raiseAssert "withUpdatedState failed"
@ -152,8 +152,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(attestationSlotOffset)
contributionsTime = slot.toBeaconTime(syncContributionSlotOffset)
messagesTime = slot.attestation_deadline()
contributionsTime = slot.sync_contribution_deadline()
var aggregators: seq[Aggregator]
@ -311,7 +311,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.toBeaconTime)
epochRef, blckRef, signedBlock.message, blckRef.slot.start_beacon_time)
blck() = added[]
dag.updateHead(added[], quarantine[])
@ -333,7 +333,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.toBeaconTime)
epochRef, blckRef, signedBlock.message, blckRef.slot.start_beacon_time)
blck() = added[]
dag.updateHead(added[], quarantine[])
@ -355,7 +355,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.toBeaconTime)
epochRef, blckRef, signedBlock.message, blckRef.slot.start_beacon_time)
blck() = added[]
dag.updateHead(added[], quarantine[])
@ -373,7 +373,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
let
slot = Slot(i + 1)
t =
if slot.isEpoch: tEpoch
if slot.is_epoch: tEpoch
else: tBlock
now = genesisTime + float(slot * SECONDS_PER_SLOT)
@ -424,7 +424,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
if t == tEpoch:
echo &". slot: {shortLog(slot)} ",
&"epoch: {shortLog(slot.compute_epoch_at_slot)}"
&"epoch: {shortLog(slot.epoch)}"
else:
write(stdout, ".")
flushFile(stdout)

View File

@ -91,7 +91,7 @@ cli do(slots = SLOTS_PER_EPOCH * 5,
let t =
if (getStateField(state[], slot) > GENESIS_SLOT and
(getStateField(state[], slot) + 1).isEpoch): tEpoch
(getStateField(state[], slot) + 1).is_epoch): tEpoch
else: tBlock
withTimer(timers[t]):
@ -158,7 +158,7 @@ cli do(slots = SLOTS_PER_EPOCH * 5,
flushFile(stdout)
if getStateField(state[], slot).isEpoch:
if getStateField(state[], slot).is_epoch:
echo &" slot: {shortLog(getStateField(state[], slot))} ",
&"epoch: {shortLog(state[].get_current_epoch())}"

View File

@ -15,6 +15,7 @@ import # Unit test
./test_action_tracker,
./test_attestation_pool,
./test_beacon_chain_db,
./test_beacon_time,
./test_block_dag,
./test_block_processor,
./test_datatypes,

View File

@ -162,9 +162,7 @@ suite "EF - Altair - Unittests - Sync protocol" & preset():
doAssert process_slots(
cfg, forked[], Slot(UPDATE_TIMEOUT), cache, info, flags = {})
let
snapshot_period =
compute_epoch_at_slot(store.optimistic_header.slot) div
EPOCHS_PER_SYNC_COMMITTEE_PERIOD
snapshot_period = sync_committee_period(store.optimistic_header.slot)
update_period = sync_committee_period(state.slot)
check: snapshot_period + 1 == update_period
@ -238,11 +236,8 @@ suite "EF - Altair - Unittests - Sync protocol" & preset():
check: state.finalized_checkpoint.epoch == 3
# Ensure that it's same period
let
snapshot_period =
compute_epoch_at_slot(store.optimistic_header.slot) div
EPOCHS_PER_SYNC_COMMITTEE_PERIOD
update_period =
compute_epoch_at_slot(state.slot) div EPOCHS_PER_SYNC_COMMITTEE_PERIOD
snapshot_period = sync_committee_period(store.optimistic_header.slot)
update_period = sync_committee_period(state.slot)
check: snapshot_period == update_period
# Updated sync_committee and finality
@ -258,7 +253,7 @@ suite "EF - Altair - Unittests - Sync protocol" & preset():
body_root: finalized_block.message.body.hash_tree_root())
check:
finalized_block_header.slot ==
compute_start_slot_at_epoch(state.finalized_checkpoint.epoch)
start_slot(state.finalized_checkpoint.epoch)
finalized_block_header.hash_tree_root() ==
state.finalized_checkpoint.root
var finality_branch {.noinit.}:

View File

@ -204,8 +204,7 @@ proc stepOnAttestation(
time: BeaconTime): FcResult[void] =
let epochRef =
dag.getEpochRef(
dag.head, time.slotOrZero().compute_epoch_at_slot(),
false).expect("no pruning in test")
dag.head, time.slotOrZero().epoch(), false).expect("no pruning in test")
let attesters = epochRef.get_attesting_indices(
att.data.slot, CommitteeIndex(att.data.index), att.aggregation_bits)
let status = fkChoice[].on_attestation(
@ -225,7 +224,7 @@ proc stepChecks(
doAssert checks.len >= 1, "No checks found"
for check, val in checks:
if check == "time":
doAssert time.Duration == val.getInt().seconds
doAssert time.ns_since_genesis == val.getInt().seconds.nanoseconds()
doAssert fkChoice.checkpoints.time.slotOrZero == time.slotOrZero
elif check == "head":
let headRoot = fkChoice[].get_head(dag, time).get()
@ -302,7 +301,7 @@ proc runTest(path: string, fork: BeaconBlockFork) =
for step in steps:
case step.kind
of opOnTick:
time = step.tick.seconds.BeaconTime
time = BeaconTime(ns_since_genesis: step.tick.seconds.nanoseconds)
doAssert stores.fkChoice.checkpoints.on_tick(time).isOk
of opOnBlock:
withBlck(step.blk):

View File

@ -26,7 +26,7 @@ func apply_randao_reveal(state: ForkyBeaconState, blck: var ForkySignedBeaconBlo
blck.message.body.randao_reveal = get_epoch_signature(
state.fork,
state.genesis_validators_root,
blck.message.slot.compute_epoch_at_slot,
blck.message.slot.epoch,
privkey).toValidatorSig()
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/tests/core/pyspec/eth2spec/test/helpers/block.py#L38-L54

View File

@ -660,7 +660,7 @@ suite "Slashing Protection DB" & preset():
).error.kind == DoubleProposal
db.pruneAfterFinalization(
compute_epoch_at_slot(Slot 1000)
epoch(Slot 1000)
)
doAssert db.checkSlashableBlockProposal(
@ -702,7 +702,7 @@ suite "Slashing Protection DB" & preset():
# Pruning far in the future
db.pruneAfterFinalization(
compute_epoch_at_slot(Slot 10000)
epoch(Slot 10000)
)
# Last block is still there
@ -815,7 +815,7 @@ suite "Slashing Protection DB" & preset():
# --------------------------------
db.pruneAfterFinalization(
compute_epoch_at_slot(Slot 10000)
epoch(Slot 10000)
)
# --------------------------------

View File

@ -23,7 +23,7 @@ func addMockAttestations*(
sufficient_support = false
) =
# We must be at the end of the epoch
doAssert (state.slot + 1).isEpoch
doAssert (state.slot + 1).is_epoch
# Alias the attestations container
var attestations: ptr seq[PendingAttestation]
@ -39,10 +39,9 @@ func addMockAttestations*(
var remaining_balance = state.get_total_active_balance(cache).int64 * 2 div 3
let
start_slot = compute_start_slot_at_epoch(epoch)
committees_per_slot = get_committee_count_per_slot(state, epoch, cache)
for slot in start_slot ..< start_slot + SLOTS_PER_EPOCH:
for slot in epoch.slots():
for committee_index in get_committee_indices(committees_per_slot):
let committee = get_beacon_committee(state, slot, committee_index, cache)
@ -87,5 +86,5 @@ func putCheckpointsInBlockRoots*(
state: var phase0.BeaconState,
checkpoints: openArray[Checkpoint]) =
for c in checkpoints:
let idx = c.epoch.compute_start_slot_at_epoch() mod SLOTS_PER_HISTORICAL_ROOT
let idx = c.epoch.start_slot() mod SLOTS_PER_HISTORICAL_ROOT
state.block_roots[idx] = c.root

View File

@ -85,7 +85,7 @@ suite "Attestation pool processing" & preset():
pool[].addAttestation(
attestation, @[bc0[0]], attestation.loadSig,
attestation.data.slot.toBeaconTime)
attestation.data.slot.start_beacon_time)
check:
# Added attestation, should get it back
@ -137,7 +137,7 @@ suite "Attestation pool processing" & preset():
pool[].getAttestationsForBlock(state.data, cache) == []
pool[].addAttestation(
att1, @[bc1[0]], att1.loadSig, att1.data.slot.toBeaconTime)
att1, @[bc1[0]], att1.loadSig, att1.data.slot.start_beacon_time)
check:
# but new ones should go in
@ -146,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.toBeaconTime)
att2, @[bc1[1]], att2.loadSig, att2.data.slot.start_beacon_time)
let
combined = pool[].getAttestationsForBlock(state.data, cache)
@ -158,7 +158,7 @@ suite "Attestation pool processing" & preset():
pool[].addAttestation(
combined[0], @[bc1[1], bc1[0]], combined[0].loadSig,
combined[0].data.slot.toBeaconTime)
combined[0].data.slot.start_beacon_time)
check:
# readding the combined attestation shouldn't have an effect
@ -168,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.toBeaconTime)
att3, @[bc1[2]], att3.loadSig, att3.data.slot.start_beacon_time)
check:
# We should now get both attestations for the block, but the aggregate
@ -183,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.toBeaconTime)
att4, @[bc1[2]], att3.loadSig, att3.data.slot.start_beacon_time)
test "Working with aggregates" & preset():
let
@ -203,9 +203,9 @@ suite "Attestation pool processing" & preset():
att1.combine(att2)
pool[].addAttestation(
att0, @[bc0[0], bc0[2]], att0.loadSig, att0.data.slot.toBeaconTime)
att0, @[bc0[0], bc0[2]], att0.loadSig, att0.data.slot.start_beacon_time)
pool[].addAttestation(
att1, @[bc0[1], bc0[2]], att1.loadSig, att1.data.slot.toBeaconTime)
att1, @[bc0[1], bc0[2]], att1.loadSig, att1.data.slot.start_beacon_time)
check:
process_slots(
@ -220,7 +220,7 @@ suite "Attestation pool processing" & preset():
# Add in attestation 3 - both aggregates should now have it added
pool[].addAttestation(
att3, @[bc0[3]], att3.loadSig, att3.data.slot.toBeaconTime)
att3, @[bc0[3]], att3.loadSig, att3.data.slot.start_beacon_time)
block:
let attestations = pool[].getAttestationsForBlock(state.data, cache)
@ -233,7 +233,7 @@ 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.toBeaconTime)
att0x, @[bc0[0]], att0x.loadSig, att0x.data.slot.start_beacon_time)
block:
let attestations = pool[].getAttestationsForBlock(state.data, cache)
@ -255,7 +255,7 @@ suite "Attestation pool processing" & preset():
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.toBeaconTime)
att, @[bc0[j]], att.loadSig, att.data.slot.start_beacon_time)
inc attestations
check:
@ -293,10 +293,10 @@ suite "Attestation pool processing" & preset():
# test reverse order
pool[].addAttestation(
attestation1, @[bc1[0]], attestation1.loadSig,
attestation1.data.slot.toBeaconTime)
attestation1.data.slot.start_beacon_time)
pool[].addAttestation(
attestation0, @[bc0[0]], attestation0.loadSig,
attestation0.data.slot.toBeaconTime)
attestation0.data.slot.start_beacon_time)
discard process_slots(
defaultRuntimeConfig, state.data,
@ -320,10 +320,10 @@ suite "Attestation pool processing" & preset():
pool[].addAttestation(
attestation0, @[bc0[0]], attestation0.loadSig,
attestation0.data.slot.toBeaconTime)
attestation0.data.slot.start_beacon_time)
pool[].addAttestation(
attestation1, @[bc0[1]], attestation1.loadSig,
attestation1.data.slot.toBeaconTime)
attestation1.data.slot.start_beacon_time)
check:
process_slots(
@ -351,10 +351,10 @@ suite "Attestation pool processing" & preset():
pool[].addAttestation(
attestation0, @[bc0[0]], attestation0.loadSig,
attestation0.data.slot.toBeaconTime)
attestation0.data.slot.start_beacon_time)
pool[].addAttestation(
attestation1, @[bc0[1]], attestation1.loadSig,
attestation1.data.slot.toBeaconTime)
attestation1.data.slot.start_beacon_time)
check:
process_slots(
@ -381,10 +381,10 @@ suite "Attestation pool processing" & preset():
pool[].addAttestation(
attestation1, @[bc0[1]], attestation1.loadSig,
attestation1.data.slot.toBeaconTime)
attestation1.data.slot.start_beacon_time)
pool[].addAttestation(
attestation0, @[bc0[0]], attestation0.loadSig,
attestation0.data.slot.toBeaconTime)
attestation0.data.slot.start_beacon_time)
check:
process_slots(
@ -405,9 +405,9 @@ suite "Attestation pool processing" & preset():
epochRef: EpochRef):
# Callback add to fork choice if valid
pool[].addForkChoice(
epochRef, blckRef, signedBlock.message, blckRef.slot.toBeaconTime)
epochRef, blckRef, signedBlock.message, blckRef.slot.start_beacon_time)
let head = pool[].selectHead(b1Add[].slot.toBeaconTime)
let head = pool[].selectHead(b1Add[].slot.start_beacon_time)
check:
head == b1Add[]
@ -419,9 +419,9 @@ suite "Attestation pool processing" & preset():
epochRef: EpochRef):
# Callback add to fork choice if valid
pool[].addForkChoice(
epochRef, blckRef, signedBlock.message, blckRef.slot.toBeaconTime)
epochRef, blckRef, signedBlock.message, blckRef.slot.start_beacon_time)
let head2 = pool[].selectHead(b2Add[].slot.toBeaconTime)
let head2 = pool[].selectHead(b2Add[].slot.start_beacon_time)
check:
head2 == b2Add[]
@ -435,9 +435,9 @@ suite "Attestation pool processing" & preset():
epochRef: EpochRef):
# Callback add to fork choice if valid
pool[].addForkChoice(
epochRef, blckRef, signedBlock.message, blckRef.slot.toBeaconTime)
epochRef, blckRef, signedBlock.message, blckRef.slot.start_beacon_time)
let head = pool[].selectHead(b10Add[].slot.toBeaconTime)
let head = pool[].selectHead(b10Add[].slot.start_beacon_time)
check:
head == b10Add[]
@ -451,7 +451,7 @@ suite "Attestation pool processing" & preset():
epochRef: EpochRef):
# Callback add to fork choice if valid
pool[].addForkChoice(
epochRef, blckRef, signedBlock.message, blckRef.slot.toBeaconTime)
epochRef, blckRef, signedBlock.message, blckRef.slot.start_beacon_time)
bc1 = get_beacon_committee(
state[].data, getStateField(state.data, slot) - 1, 1.CommitteeIndex,
@ -460,9 +460,9 @@ suite "Attestation pool processing" & preset():
pool[].addAttestation(
attestation0, @[bc1[0]], attestation0.loadSig,
attestation0.data.slot.toBeaconTime)
attestation0.data.slot.start_beacon_time)
let head2 = pool[].selectHead(b10Add[].slot.toBeaconTime)
let head2 = pool[].selectHead(b10Add[].slot.start_beacon_time)
check:
# Single vote for b10 and no votes for b11
@ -473,9 +473,9 @@ suite "Attestation pool processing" & preset():
attestation2 = makeAttestation(state[].data, b11.root, bc1[2], cache)
pool[].addAttestation(
attestation1, @[bc1[1]], attestation1.loadSig,
attestation1.data.slot.toBeaconTime)
attestation1.data.slot.start_beacon_time)
let head3 = pool[].selectHead(b10Add[].slot.toBeaconTime)
let head3 = pool[].selectHead(b10Add[].slot.start_beacon_time)
let bigger = if b11.root.data < b10.root.data: b10Add else: b11Add
check:
@ -484,9 +484,9 @@ suite "Attestation pool processing" & preset():
pool[].addAttestation(
attestation2, @[bc1[2]], attestation2.loadSig,
attestation2.data.slot.toBeaconTime)
attestation2.data.slot.start_beacon_time)
let head4 = pool[].selectHead(b11Add[].slot.toBeaconTime)
let head4 = pool[].selectHead(b11Add[].slot.start_beacon_time)
check:
# Two votes for b11
@ -501,9 +501,9 @@ suite "Attestation pool processing" & preset():
epochRef: EpochRef):
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message,
blckRef.slot.toBeaconTime)
blckRef.slot.start_beacon_time)
let head = pool[].selectHead(b10Add[].slot.toBeaconTime)
let head = pool[].selectHead(b10Add[].slot.start_beacon_time)
check:
head == b10Add[]
@ -516,7 +516,7 @@ suite "Attestation pool processing" & preset():
epochRef: EpochRef):
# Callback add to fork choice if valid
pool[].addForkChoice(
epochRef, blckRef, signedBlock.message, blckRef.slot.toBeaconTime)
epochRef, blckRef, signedBlock.message, blckRef.slot.start_beacon_time)
doAssert: b10Add_clone.error == BlockError.Duplicate
@ -532,9 +532,9 @@ suite "Attestation pool processing" & preset():
epochRef: EpochRef):
# Callback add to fork choice if valid
pool[].addForkChoice(
epochRef, blckRef, signedBlock.message, blckRef.slot.toBeaconTime)
epochRef, blckRef, signedBlock.message, blckRef.slot.start_beacon_time)
let head = pool[].selectHead(b10Add[].slot.toBeaconTime)
let head = pool[].selectHead(b10Add[].slot.start_beacon_time)
doAssert: head == b10Add[]
@ -546,7 +546,7 @@ suite "Attestation pool processing" & preset():
var attestations: seq[Attestation]
for epoch in 0 ..< 5:
let start_slot = compute_start_slot_at_epoch(Epoch epoch)
let start_slot = start_slot(Epoch epoch)
let committees_per_slot =
get_committee_count_per_slot(state[].data, Epoch epoch, cache)
for slot in start_slot ..< start_slot + SLOTS_PER_EPOCH:
@ -558,9 +558,9 @@ suite "Attestation pool processing" & preset():
epochRef: EpochRef):
# Callback add to fork choice if valid
pool[].addForkChoice(
epochRef, blckRef, signedBlock.message, blckRef.slot.toBeaconTime)
epochRef, blckRef, signedBlock.message, blckRef.slot.start_beacon_time)
let head = pool[].selectHead(blockRef[].slot.toBeaconTime)
let head = pool[].selectHead(blockRef[].slot.start_beacon_time)
doAssert: head == blockRef[]
dag.updateHead(head, quarantine[])
pruneAtFinalization(dag, pool[])
@ -601,6 +601,6 @@ suite "Attestation pool processing" & preset():
epochRef: EpochRef):
# Callback add to fork choice if valid
pool[].addForkChoice(
epochRef, blckRef, signedBlock.message, blckRef.slot.toBeaconTime)
epochRef, blckRef, signedBlock.message, blckRef.slot.start_beacon_time)
doAssert: b10Add_clone.error == BlockError.Duplicate

View File

@ -0,0 +1,36 @@
import
unittest2,
../beacon_chain/spec/beacon_time
suite "Beacon time":
test "basics":
let
s0 = Slot(0)
check:
s0.epoch() == Epoch(0)
s0.start_beacon_time() == BeaconTime()
s0.sync_committee_period() == SyncCommitteePeriod(0)
# Roundtrip far times we treat these as "Infinitiy"
FAR_FUTURE_SLOT.epoch.start_slot() == FAR_FUTURE_SLOT
FAR_FUTURE_EPOCH.start_slot().epoch() == FAR_FUTURE_EPOCH
FAR_FUTURE_SLOT.start_beacon_time().slotOrZero() == FAR_FUTURE_SLOT
FAR_FUTURE_PERIOD.start_epoch().sync_committee_period() == FAR_FUTURE_PERIOD
BeaconTime(ns_since_genesis: -10000000000).slotOrZero == Slot(0)
Slot(5).since_epoch_start() == 5
(Epoch(42).start_slot() + 5).since_epoch_start() == 5
Slot(5).start_beacon_time() > Slot(4).start_beacon_time()
Slot(4).start_beacon_time() +
(Slot(5).start_beacon_time() - Slot(4).start_beacon_time()) ==
Slot(5).start_beacon_time()
Epoch(3).start_slot.is_epoch()
SyncCommitteePeriod(5).start_epoch().is_sync_committee_period()
Epoch(5).start_slot.sync_committee_period ==
Epoch(5).sync_committee_period

View File

@ -60,9 +60,9 @@ suite "BlockSlot and helpers":
s2 = BlockRef(bid: BlockId(slot: Slot(2)), parent: s1)
s4 = BlockRef(bid: BlockId(slot: Slot(4)), parent: s2)
se1 = BlockRef(bid:
BlockId(slot: Epoch(1).compute_start_slot_at_epoch()), parent: s2)
BlockId(slot: Epoch(1).start_slot()), parent: s2)
se2 = BlockRef(bid:
BlockId(slot: Epoch(2).compute_start_slot_at_epoch()), parent: se1)
BlockId(slot: Epoch(2).start_slot()), parent: se1)
check:
s0.atSlot(Slot(0)).blck == s0

View File

@ -38,14 +38,14 @@ suite "Block processor" & preset():
cache = StateCache()
b1 = addTestBlock(state[], cache).phase0Data
b2 = addTestBlock(state[], cache).phase0Data
getTimeFn = proc(): BeaconTime = b2.message.slot.toBeaconTime()
getTimeFn = proc(): BeaconTime = b2.message.slot.start_beacon_time()
processor = BlockProcessor.new(
false, "", "", keys.newRng(), taskpool, consensusManager,
validatorMonitor, getTimeFn)
test "Reverse order block add & get" & preset():
let missing = processor[].storeBlock(
MsgSource.gossip, b2.message.slot.toBeaconTime(), b2)
MsgSource.gossip, b2.message.slot.start_beacon_time(), b2)
check: missing.error == BlockError.MissingParent
check:
@ -55,7 +55,7 @@ suite "Block processor" & preset():
let
status = processor[].storeBlock(
MsgSource.gossip, b2.message.slot.toBeaconTime(), b1)
MsgSource.gossip, b2.message.slot.start_beacon_time(), b1)
b1Get = dag.get(b1.root)
check:

View File

@ -49,7 +49,7 @@ suite "ChainDAG helpers":
let
farEpoch = Epoch(42)
farTail = BlockRef(
bid: BlockId(slot: farEpoch.compute_start_slot_at_epoch() + 5))
bid: BlockId(slot: farEpoch.start_slot() + 5))
check:
not isNil(epochAncestor(farTail, farEpoch).blck)
@ -172,7 +172,7 @@ suite "Block pool processing" & preset():
# A fork forces the clearance state to a point where it cannot be advanced
let
nextEpoch = dag.head.slot.epoch + 1
nextEpochSlot = nextEpoch.compute_start_slot_at_epoch()
nextEpochSlot = nextEpoch.start_slot()
stateCheckpoint = dag.head.parent.atSlot(nextEpochSlot).stateCheckpoint
check:
@ -282,7 +282,7 @@ suite "Block pool altair processing" & preset():
# Advance to altair
check:
process_slots(
cfg, state[], cfg.ALTAIR_FORK_EPOCH.compute_start_slot_at_epoch(), cache,
cfg, state[], cfg.ALTAIR_FORK_EPOCH.start_slot(), cache,
info, {})
state[].kind == BeaconStateFork.Altair

View File

@ -50,20 +50,20 @@ suite "Forked SSZ readers":
testHashedBeaconState(merge.BeaconState, 0.Slot)
test "load altair state":
testHashedBeaconState(altair.BeaconState, cfg.ALTAIR_FORK_EPOCH.compute_start_slot_at_epoch)
testHashedBeaconState(altair.BeaconState, cfg.ALTAIR_FORK_EPOCH.start_slot)
expect(SszError):
testHashedBeaconState(phase0.BeaconState, cfg.ALTAIR_FORK_EPOCH.compute_start_slot_at_epoch)
testHashedBeaconState(phase0.BeaconState, cfg.ALTAIR_FORK_EPOCH.start_slot)
expect(SszError):
testHashedBeaconState(merge.BeaconState, cfg.ALTAIR_FORK_EPOCH.compute_start_slot_at_epoch)
testHashedBeaconState(merge.BeaconState, cfg.ALTAIR_FORK_EPOCH.start_slot)
test "load merge state":
testHashedBeaconState(merge.BeaconState, cfg.MERGE_FORK_EPOCH.compute_start_slot_at_epoch)
testHashedBeaconState(merge.BeaconState, cfg.MERGE_FORK_EPOCH.start_slot)
expect(SszError):
testHashedBeaconState(phase0.BeaconState, cfg.MERGE_FORK_EPOCH.compute_start_slot_at_epoch)
testHashedBeaconState(phase0.BeaconState, cfg.MERGE_FORK_EPOCH.start_slot)
expect(SszError):
testHashedBeaconState(altair.BeaconState, cfg.MERGE_FORK_EPOCH.compute_start_slot_at_epoch)
testHashedBeaconState(altair.BeaconState, cfg.MERGE_FORK_EPOCH.start_slot)
test "should raise on unknown data":
let
@ -79,19 +79,19 @@ suite "Forked SSZ readers":
testTrustedSignedBeaconBlock(merge.TrustedSignedBeaconBlock, 0.Slot)
test "load altair block":
testTrustedSignedBeaconBlock(altair.TrustedSignedBeaconBlock, cfg.ALTAIR_FORK_EPOCH.compute_start_slot_at_epoch)
testTrustedSignedBeaconBlock(altair.TrustedSignedBeaconBlock, cfg.ALTAIR_FORK_EPOCH.start_slot)
expect(SszError):
testTrustedSignedBeaconBlock(phase0.TrustedSignedBeaconBlock, cfg.ALTAIR_FORK_EPOCH.compute_start_slot_at_epoch)
testTrustedSignedBeaconBlock(phase0.TrustedSignedBeaconBlock, cfg.ALTAIR_FORK_EPOCH.start_slot)
expect(SszError):
testTrustedSignedBeaconBlock(merge.TrustedSignedBeaconBlock, cfg.ALTAIR_FORK_EPOCH.compute_start_slot_at_epoch)
testTrustedSignedBeaconBlock(merge.TrustedSignedBeaconBlock, cfg.ALTAIR_FORK_EPOCH.start_slot)
test "load merge block":
testTrustedSignedBeaconBlock(merge.TrustedSignedBeaconBlock, cfg.MERGE_FORK_EPOCH.compute_start_slot_at_epoch)
testTrustedSignedBeaconBlock(merge.TrustedSignedBeaconBlock, cfg.MERGE_FORK_EPOCH.start_slot)
expect(SszError):
testTrustedSignedBeaconBlock(phase0.TrustedSignedBeaconBlock, cfg.MERGE_FORK_EPOCH.compute_start_slot_at_epoch)
testTrustedSignedBeaconBlock(phase0.TrustedSignedBeaconBlock, cfg.MERGE_FORK_EPOCH.start_slot)
expect(SszError):
testTrustedSignedBeaconBlock(altair.TrustedSignedBeaconBlock, cfg.MERGE_FORK_EPOCH.compute_start_slot_at_epoch)
testTrustedSignedBeaconBlock(altair.TrustedSignedBeaconBlock, cfg.MERGE_FORK_EPOCH.start_slot)
test "should raise on unknown data":
let

View File

@ -81,7 +81,7 @@ suite "Gossip validation " & preset():
epochRef: EpochRef):
# Callback add to fork choice if valid
pool[].addForkChoice(
epochRef, blckRef, signedBlock.message, blckRef.slot.toBeaconTime)
epochRef, blckRef, signedBlock.message, blckRef.slot.start_beacon_time)
check: added.isOk()
dag.updateHead(added[], quarantine[])
@ -104,7 +104,7 @@ suite "Gossip validation " & preset():
committees_per_slot,
att_1_0.data.slot, att_1_0.data.index.CommitteeIndex)
beaconTime = att_1_0.data.slot.toBeaconTime()
beaconTime = att_1_0.data.slot.start_beacon_time()
check:
validateAttestation(pool, batchCrypto, att_1_0, beaconTime, subnet, true).waitFor().isOk
@ -230,7 +230,7 @@ suite "Gossip validation - Extra": # Not based on preset config
syncCommitteeMsgPool = newClone(SyncCommitteeMsgPool.init())
res = waitFor validateSyncCommitteeMessage(
dag, batchCrypto, syncCommitteeMsgPool, msg, subcommitteeIdx,
slot.toBeaconTime(), true)
slot.start_beacon_time(), true)
(positions, cookedSig) = res.get()
syncCommitteeMsgPool[].addSyncCommitteeMessage(
@ -264,4 +264,4 @@ suite "Gossip validation - Extra": # Not based on preset config
# Same message twice should be ignored
validateSyncCommitteeMessage(
dag, batchCrypto, syncCommitteeMsgPool, msg, subcommitteeIdx,
state[].data.slot.toBeaconTime(), true).waitFor().isErr()
state[].data.slot.start_beacon_time(), true).waitFor().isErr()

View File

@ -63,14 +63,14 @@ suite "Beacon state" & preset():
check:
get_beacon_proposer_index(state[].phase0Data.data, cache, Slot 1).isSome()
get_beacon_proposer_index(
state[].phase0Data.data, cache, Epoch(1).compute_start_slot_at_epoch()).isNone()
state[].phase0Data.data, cache, Epoch(1).start_slot()).isNone()
get_beacon_proposer_index(
state[].phase0Data.data, cache, Epoch(2).compute_start_slot_at_epoch()).isNone()
state[].phase0Data.data, cache, Epoch(2).start_slot()).isNone()
check:
process_slots(cfg, state[], Epoch(1).compute_start_slot_at_epoch(), cache, info, {})
process_slots(cfg, state[], Epoch(1).start_slot(), cache, info, {})
get_beacon_proposer_index(state[].phase0Data.data, cache, Slot 1).isNone()
get_beacon_proposer_index(
state[].phase0Data.data, cache, Epoch(1).compute_start_slot_at_epoch()).isSome()
state[].phase0Data.data, cache, Epoch(1).start_slot()).isSome()
get_beacon_proposer_index(
state[].phase0Data.data, cache, Epoch(2).compute_start_slot_at_epoch()).isNone()
state[].phase0Data.data, cache, Epoch(2).start_slot()).isNone()

View File

@ -792,9 +792,9 @@ suite "SyncManager test suite":
Slot(0), Slot(0xFFFF_FFFF_FFFF_FFFFF'u64),
1'u64, getFirstSlotAtFinalizedEpoch,
collector(aq), 2)
let finalizedSlot = compute_start_slot_at_epoch(Epoch(0'u64))
let startSlot = compute_start_slot_at_epoch(Epoch(0'u64)) + 1'u64
let finishSlot = compute_start_slot_at_epoch(Epoch(2'u64))
let finalizedSlot = start_slot(Epoch(0'u64))
let startSlot = start_slot(Epoch(0'u64)) + 1'u64
let finishSlot = start_slot(Epoch(2'u64))
for i in uint64(startSlot) ..< uint64(finishSlot):
check queue.getRewindPoint(Slot(i), finalizedSlot) == finalizedSlot
@ -804,9 +804,9 @@ suite "SyncManager test suite":
Slot(0), Slot(0xFFFF_FFFF_FFFF_FFFFF'u64),
1'u64, getFirstSlotAtFinalizedEpoch,
collector(aq), 2)
let finalizedSlot = compute_start_slot_at_epoch(Epoch(1'u64))
let startSlot = compute_start_slot_at_epoch(Epoch(1'u64)) + 1'u64
let finishSlot = compute_start_slot_at_epoch(Epoch(3'u64))
let finalizedSlot = start_slot(Epoch(1'u64))
let startSlot = start_slot(Epoch(1'u64)) + 1'u64
let finishSlot = start_slot(Epoch(3'u64))
for i in uint64(startSlot) ..< uint64(finishSlot) :
check queue.getRewindPoint(Slot(i), finalizedSlot) == finalizedSlot
@ -816,16 +816,16 @@ suite "SyncManager test suite":
Slot(0), Slot(0xFFFF_FFFF_FFFF_FFFFF'u64),
1'u64, getFirstSlotAtFinalizedEpoch,
collector(aq), 2)
let finalizedSlot = compute_start_slot_at_epoch(Epoch(0'u64))
let finalizedSlot = start_slot(Epoch(0'u64))
let failSlot = Slot(0xFFFF_FFFF_FFFF_FFFFF'u64)
let failEpoch = compute_epoch_at_slot(failSlot)
let failEpoch = epoch(failSlot)
var counter = 1'u64
for i in 0 ..< 64:
if counter >= failEpoch:
break
let rewindEpoch = failEpoch - counter
let rewindSlot = compute_start_slot_at_epoch(rewindEpoch)
let rewindSlot = start_slot(rewindEpoch)
check queue.getRewindPoint(failSlot, finalizedSlot) == rewindSlot
counter = counter shl 1
@ -834,15 +834,15 @@ suite "SyncManager test suite":
Slot(0), Slot(0xFFFF_FFFF_FFFF_FFFFF'u64),
1'u64, getFirstSlotAtFinalizedEpoch,
collector(aq), 2)
let finalizedSlot = compute_start_slot_at_epoch(Epoch(1'u64))
let finalizedSlot = start_slot(Epoch(1'u64))
let failSlot = Slot(0xFFFF_FFFF_FFFF_FFFFF'u64)
let failEpoch = compute_epoch_at_slot(failSlot)
let failEpoch = epoch(failSlot)
var counter = 1'u64
for i in 0 ..< 64:
if counter >= failEpoch:
break
let rewindEpoch = failEpoch - counter
let rewindSlot = compute_start_slot_at_epoch(rewindEpoch)
let rewindSlot = start_slot(rewindEpoch)
check queue.getRewindPoint(failSlot, finalizedSlot) == rewindSlot
counter = counter shl 1

View File

@ -157,13 +157,13 @@ func makeAttestationData*(
beacon_block_root: Eth2Digest): AttestationData =
let
current_epoch = get_current_epoch(state)
start_slot = compute_start_slot_at_epoch(current_epoch)
start_slot = start_slot(current_epoch)
epoch_boundary_block_root =
if start_slot == state.slot: beacon_block_root
else: get_block_root_at_slot(state, start_slot)
doAssert slot.compute_epoch_at_slot == current_epoch,
"Computed epoch was " & $slot.compute_epoch_at_slot &
doAssert slot.epoch == current_epoch,
"Computed epoch was " & $slot.epoch &
" while the state current_epoch was " & $current_epoch
# https://github.com/ethereum/consensus-specs/blob/v1.1.8/specs/phase0/validator.md#attestation-data
@ -225,12 +225,12 @@ func makeAttestation*(
func find_beacon_committee(
state: ForkedHashedBeaconState, validator_index: ValidatorIndex,
cache: var StateCache): auto =
let epoch = compute_epoch_at_slot(getStateField(state, slot))
let epoch = epoch(getStateField(state, slot))
for epoch_committee_index in 0'u64 ..< get_committee_count_per_slot(
state, epoch, cache) * SLOTS_PER_EPOCH:
let
slot = ((epoch_committee_index mod SLOTS_PER_EPOCH) +
epoch.compute_start_slot_at_epoch.uint64).Slot
epoch.start_slot.uint64).Slot
index = CommitteeIndex(epoch_committee_index div SLOTS_PER_EPOCH)
committee = get_beacon_committee(state, slot, index, cache)
if validator_index in committee:
@ -292,9 +292,7 @@ proc makeSyncAggregate(
syncCommittee =
withState(state):
when stateFork >= BeaconStateFork.Altair:
const SLOTS_PER_PERIOD =
EPOCHS_PER_SYNC_COMMITTEE_PERIOD * SLOTS_PER_EPOCH
if (state.data.slot + 1) mod SLOTS_PER_PERIOD == 0:
if (state.data.slot + 1).is_sync_committee_period():
state.data.next_sync_committee
else:
state.data.current_sync_committee

View File

@ -71,7 +71,7 @@ proc getTestStates*(
cfg.MERGE_FORK_EPOCH = 1.Epoch
for i, epoch in stateEpochs:
let slot = epoch.Epoch.compute_start_slot_at_epoch
let slot = epoch.Epoch.start_slot
if getStateField(tmpState[], slot) < slot:
doAssert process_slots(
cfg, tmpState[], slot, cache, info, {})