implement SingleAttestation (#6783)

* implement SingleAttestation

* improve SingleAttestation shortLog

* avoid double ShuffleRef lookups in majority/happy case

* restore full electra check_aggregation_count, separately from phase0 one
This commit is contained in:
tersec 2024-12-23 16:32:29 +00:00 committed by GitHub
parent c344e94adf
commit 9c9cbed987
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
17 changed files with 328 additions and 162 deletions

View File

@ -33,8 +33,8 @@ const
type
OnPhase0AttestationCallback =
proc(data: phase0.Attestation) {.gcsafe, raises: [].}
OnElectraAttestationCallback =
proc(data: electra.Attestation) {.gcsafe, raises: [].}
OnSingleAttestationCallback =
proc(data: SingleAttestation) {.gcsafe, raises: [].}
Validation[CVBType] = object
## Validations collect a set of signatures for a distict attestation - in
@ -96,7 +96,7 @@ type
## sequence based on validator indices
onPhase0AttestationAdded: OnPhase0AttestationCallback
onElectraAttestationAdded: OnElectraAttestationCallback
onSingleAttestationAdded: OnSingleAttestationCallback
logScope: topics = "attpool"
@ -106,7 +106,7 @@ declareGauge attestation_pool_block_attestation_packing_time,
proc init*(T: type AttestationPool, dag: ChainDAGRef,
quarantine: ref Quarantine,
onPhase0Attestation: OnPhase0AttestationCallback = nil,
onElectraAttestation: OnElectraAttestationCallback = nil): T =
onSingleAttestation: OnSingleAttestationCallback = nil): T =
## Initialize an AttestationPool from the dag `headState`
## The `finalized_root` works around the finalized_checkpoint of the genesis block
## holding a zero_root.
@ -182,7 +182,7 @@ proc init*(T: type AttestationPool, dag: ChainDAGRef,
quarantine: quarantine,
forkChoice: forkChoice,
onPhase0AttestationAdded: onPhase0Attestation,
onElectraAttestationAdded: onElectraAttestation
onSingleAttestationAdded: onSingleAttestation
)
proc addForkChoiceVotes(
@ -351,13 +351,12 @@ func covers(
proc addAttestation(
entry: var AttestationEntry,
attestation: phase0.Attestation | electra.Attestation,
attestation: phase0.Attestation | electra.Attestation, _: int,
signature: CookedSig): bool =
logScope:
attestation = shortLog(attestation)
let
singleIndex = oneIndex(attestation.aggregation_bits)
let singleIndex = oneIndex(attestation.aggregation_bits)
if singleIndex.isSome():
if singleIndex.get() in entry.singles:
@ -392,6 +391,28 @@ proc addAttestation(
true
proc addAttestation(
entry: var AttestationEntry, attestation: SingleAttestation,
index_in_committee: int,
signature: CookedSig): bool =
logScope:
attestation = shortLog(attestation)
if index_in_committee in entry.singles:
trace "SingleAttestation already seen",
singles = entry.singles.len(),
aggregates = entry.aggregates.len()
return false
debug "SingleAttestation resolved",
singles = entry.singles.len(),
aggregates = entry.aggregates.len()
entry.singles[index_in_committee] = signature
true
func getAttestationCandidateKey(
data: AttestationData,
committee_index: Opt[CommitteeIndex]): Eth2Digest =
@ -403,7 +424,8 @@ func getAttestationCandidateKey(
# i.e. no committees selected, so it can't be an actual Electra attestation
hash_tree_root(data)
else:
hash_tree_root([hash_tree_root(data), hash_tree_root(committee_index.get.uint64)])
hash_tree_root([hash_tree_root(data),
hash_tree_root(committee_index.get.uint64)])
func getAttestationCandidateKey(
attestationDataRoot: Eth2Digest, committee_index: CommitteeIndex):
@ -412,9 +434,9 @@ func getAttestationCandidateKey(
proc addAttestation*(
pool: var AttestationPool,
attestation: phase0.Attestation | electra.Attestation,
attesting_indices: openArray[ValidatorIndex],
signature: CookedSig, wallTime: BeaconTime) =
attestation: phase0.Attestation | electra.Attestation | SingleAttestation,
attesting_indices: openArray[ValidatorIndex], beacon_committee_len: int,
index_in_committee: int, signature: CookedSig, wallTime: BeaconTime) =
## Add an attestation to the pool, assuming it's been validated already.
##
## Assuming the votes in the attestation have not already been seen, the
@ -445,12 +467,12 @@ proc addAttestation*(
let attestation_data_root = getAttestationCandidateKey(entry.data, committee_index)
attCandidates[candidateIdx.get()].withValue(attestation_data_root, entry) do:
if not addAttestation(entry[], attestation, signature):
if not addAttestation(entry[], attestation, index_in_committee, signature):
return
do:
if not addAttestation(
attCandidates[candidateIdx.get()].mgetOrPut(attestation_data_root, entry),
attestation, signature):
attestation, index_in_committee, signature):
# Returns from overall function, not only template
return
@ -483,9 +505,31 @@ proc addAttestation*(
attestation.data.slot, attesting_indices,
attestation.data.beacon_block_root, wallTime)
# There does not seem to be an SSE stream event corresponding to this,
# because both attestation and single_attestation specifically specify
# the `beacon_attestation_{subnet_id}` topic and that in not possible,
# for this type, in Electra because this case is always an aggregate.
template addAttToPool(_: SingleAttestation) {.used.} =
let
data = AttestationData(
slot: attestation.data.slot,
index: uint64 attestation.committee_index,
beacon_block_root: attestation.data.beacon_block_root,
source: attestation.data.source,
target: attestation.data.target)
newAttEntry = ElectraAttestationEntry(
data: data, committee_len: beacon_committee_len)
addAttToPool(
pool.electraCandidates, newAttEntry,
Opt.some attestation.committee_index.CommitteeIndex)
pool.addForkChoiceVotes(
attestation.data.slot, attesting_indices,
attestation.data.beacon_block_root, wallTime)
# Send notification about new attestation via callback.
if not(isNil(pool.onElectraAttestationAdded)):
pool.onElectraAttestationAdded(attestation)
if not(isNil(pool.onSingleAttestationAdded)):
pool.onSingleAttestationAdded(attestation)
addAttToPool(attestation)

View File

@ -173,13 +173,14 @@ iterator get_attesting_indices*(
yield validator
iterator get_attesting_indices*(
dag: ChainDAGRef, attestation: electra.TrustedAttestation,
dag: ChainDAGRef,
attestation: electra.Attestation | electra.TrustedAttestation,
on_chain: static bool): ValidatorIndex =
block gaiBlock: # `return` is not allowed in an inline iterator
let
slot =
check_attestation_slot_target(attestation.data).valueOr:
warn "Invalid attestation slot in trusted attestation",
warn "Invalid attestation slot in attestation",
attestation = shortLog(attestation)
doAssert strictVerification notin dag.updateFlags
break gaiBlock
@ -187,7 +188,7 @@ iterator get_attesting_indices*(
dag.getBlockRef(attestation.data.beacon_block_root).valueOr:
# Attestation block unknown - this is fairly common because we
# discard alternative histories on restart
debug "Pruned block in trusted attestation",
debug "Pruned block in attestation",
attestation = shortLog(attestation)
break gaiBlock
target =
@ -196,7 +197,7 @@ iterator get_attesting_indices*(
# leading to the case where the attestation block root is the
# finalized head (exists as BlockRef) but its target vote has
# already been pruned
notice "Pruned target in trusted attestation",
notice "Pruned target in attestation",
blck = shortLog(blck),
attestation = shortLog(attestation)
doAssert strictVerification notin dag.updateFlags

View File

@ -338,7 +338,7 @@ func clearDoppelgangerProtection*(self: var Eth2Processor) =
proc checkForPotentialDoppelganger(
self: var Eth2Processor,
attestation: phase0.Attestation | electra.Attestation,
attestation: phase0.Attestation | electra.Attestation | SingleAttestation,
attesterIndices: openArray[ValidatorIndex]) =
# Only check for attestations after node launch. There might be one slot of
# overlap in quick intra-slot restarts so trade off a few true negatives in
@ -360,8 +360,8 @@ proc checkForPotentialDoppelganger(
proc processAttestation*(
self: ref Eth2Processor, src: MsgSource,
attestation: phase0.Attestation | electra.Attestation, subnet_id: SubnetId,
checkSignature, checkValidator: bool
attestation: phase0.Attestation | SingleAttestation,
subnet_id: SubnetId, checkSignature, checkValidator: bool
): Future[ValidationRes] {.async: (raises: [CancelledError]).} =
var wallTime = self.getCurrentBeaconTime()
let (afterGenesis, wallSlot) = wallTime.toSlot()
@ -380,14 +380,14 @@ proc processAttestation*(
debug "Attestation received", delay
# Now proceed to validation
let v =
await self.attestationPool.validateAttestation(
let v = await self.attestationPool.validateAttestation(
self.batchCrypto, attestation, wallTime, subnet_id, checkSignature)
return if v.isOk():
# Due to async validation the wallTime here might have changed
wallTime = self.getCurrentBeaconTime()
let (attester_index, sig) = v.get()
let (attester_index, beacon_committee_len, index_in_committee, sig) =
v.get()
if checkValidator and (attester_index in self.validatorPool[]):
warn "A validator client has attempted to send an attestation from " &
@ -400,7 +400,8 @@ proc processAttestation*(
trace "Attestation validated"
self.attestationPool[].addAttestation(
attestation, [attester_index], sig, wallTime)
attestation, [attester_index], beacon_committee_len,
index_in_committee, sig, wallTime)
self.validatorMonitor[].registerAttestation(
src, wallTime, attestation, attester_index)
@ -456,8 +457,11 @@ proc processSignedAggregateAndProof*(
trace "Aggregate validated"
# -1 here is the notional index in committee for which the attestation pool
# only requires external input regarding SingleAttestation messages.
self.attestationPool[].addAttestation(
signedAggregateAndProof.message.aggregate, attesting_indices, sig,
signedAggregateAndProof.message.aggregate, attesting_indices,
signedAggregateAndProof.message.aggregate.aggregation_bits.len, -1, sig,
wallTime)
self.validatorMonitor[].registerAggregate(

View File

@ -854,7 +854,8 @@ proc validateAttestation*(
wallTime: BeaconTime,
subnet_id: SubnetId, checkSignature: bool):
Future[Result[
tuple[attesting_index: ValidatorIndex, sig: CookedSig],
tuple[attesting_index: ValidatorIndex, beacon_committee_len: int,
index_in_committee: int, sig: CookedSig],
ValidationError]] {.async: (raises: [CancelledError]).} =
# Some of the checks below have been reordered compared to the spec, to
# perform the cheap checks first - in particular, we want to avoid loading
@ -912,8 +913,7 @@ proc validateAttestation*(
# defined by attestation.data.beacon_block_root -- i.e.
# get_checkpoint_block(store, attestation.data.beacon_block_root,
# store.finalized_checkpoint.epoch) == store.finalized_checkpoint.root
let
shufflingRef =
let shufflingRef =
pool.dag.getShufflingRef(target.blck, target.slot.epoch, false).valueOr:
# Target is verified - shouldn't happen
warn "No shuffling for attestation - report bug",
@ -979,7 +979,6 @@ proc validateAttestation*(
return errIgnore("Attestation: cannot find validator pubkey")
# [REJECT] The signature of `attestation` is valid.
# In the spec, is_valid_indexed_attestation is used to verify the signature -
# here, we do a batch verification instead
let sig =
@ -1014,17 +1013,26 @@ proc validateAttestation*(
pool.nextAttestationEpoch[validator_index].subnet =
attestation.data.target.epoch + 1
return ok((validator_index, sig))
# -1 is a placeholder; it's filled in by processAttestation(), which has
# access to the required information.
ok((validator_index, attestation.aggregation_bits.len, -1, sig))
# https://github.com/ethereum/consensus-specs/blob/v1.5.0-alpha.10/specs/electra/p2p-interface.md#beacon_attestation_subnet_id
proc validateAttestation*(
pool: ref AttestationPool,
batchCrypto: ref BatchCrypto,
attestation: electra.Attestation,
attestation: SingleAttestation,
wallTime: BeaconTime,
subnet_id: SubnetId, checkSignature: bool):
Future[Result[
tuple[attesting_index: ValidatorIndex, sig: CookedSig],
tuple[attesting_index: ValidatorIndex, beacon_committee_len: int,
index_in_committee: int, sig: CookedSig],
ValidationError]] {.async: (raises: [CancelledError]).} =
# Some of the checks below have been reordered compared to the spec, to
# perform the cheap checks first - in particular, we want to avoid loading
# an `EpochRef` and checking signatures. This reordering might lead to
# different IGNORE/REJECT results in turn affecting gossip scores.
# [REJECT] The attestation's epoch matches its target -- i.e.
# attestation.data.target.epoch ==
# compute_epoch_at_slot(attestation.data.slot)
@ -1034,6 +1042,25 @@ proc validateAttestation*(
return pool.checkedReject(v.error())
v.get()
# attestation.data.slot is within the last ATTESTATION_PROPAGATION_SLOT_RANGE
# slots (within a MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance) -- i.e.
# attestation.data.slot + ATTESTATION_PROPAGATION_SLOT_RANGE >= current_slot
# >= attestation.data.slot (a client MAY queue future attestations for
# processing at the appropriate slot).
#
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.2/specs/deneb/p2p-interface.md#beacon_attestation_subnet_id
# modifies this for Deneb and newer forks.
block:
let v = check_propagation_slot_range(
pool.dag.cfg.consensusForkAtEpoch(wallTime.slotOrZero.epoch), slot,
wallTime)
if v.isErr(): # [IGNORE]
return err(v.error())
# [REJECT] attestation.data.index == 0
if not (attestation.data.index == 0):
return pool.checkedReject("SingleAttestation: attestation.data.index != 0")
# The block being voted for (attestation.data.beacon_block_root) has been seen
# (via both gossip and non-gossip sources) (a client MAY queue attestations
# for processing once block is retrieved).
@ -1053,31 +1080,66 @@ proc validateAttestation*(
# defined by attestation.data.beacon_block_root -- i.e.
# get_checkpoint_block(store, attestation.data.beacon_block_root,
# store.finalized_checkpoint.epoch) == store.finalized_checkpoint.root
let
shufflingRef =
var sigchecked = false
var sig: CookedSig
let shufflingRef =
pool.dag.findShufflingRef(target.blck.bid, target.slot.epoch).valueOr:
# getShufflingRef might be slow here, so first try to eliminate by
# signature check
sig = attestation.signature.load().valueOr:
return pool.checkedReject("SingleAttestation: unable to load signature")
sigchecked = true
pool.dag.getShufflingRef(target.blck, target.slot.epoch, false).valueOr:
# Target is verified - shouldn't happen
warn "No shuffling for attestation - report bug",
warn "No shuffling for SingleAttestation - report bug",
attestation = shortLog(attestation), target = shortLog(target)
return errIgnore("Attestation: no shuffling")
return errIgnore("SingleAttestation: no shuffling")
let attesting_index = get_attesting_indices_one(
shufflingRef, slot, attestation.committee_bits,
attestation.aggregation_bits, false)
if attestation.attester_index > high(ValidatorIndex).uint64:
return errReject("SingleAttestation: attester index too high")
let validator_index = attestation.attester_index.ValidatorIndex
# The number of aggregation bits matches the committee size, which ensures
# this condition holds.
doAssert attesting_index.isSome(),
"We've checked bits length and one count already"
let validator_index = attesting_index.get()
# [REJECT] The attester is a member of the committee -- i.e.
# attestation.attester_index in
# get_beacon_committee(state, attestation.data.slot, index).
let
beacon_committee = get_beacon_committee(
shufflingRef, attestation.data.slot,
attestation.committee_index.CommitteeIndex)
index_in_committee = find(beacon_committee, validator_index)
if index_in_committee < 0:
return pool.checkedReject("SingleAttestation: attester index not in beacon committee")
# [REJECT] The committee index is within the expected range -- i.e.
# data.index < get_committee_count_per_slot(state, data.target.epoch).
let committee_index = block:
let idx = shufflingRef.get_committee_index(attestation.data.index)
if idx.isErr():
return pool.checkedReject(
"Attestation: committee index not within expected range")
idx.get()
# [REJECT] The attestation is for the correct subnet -- i.e.
# compute_subnet_for_attestation(committees_per_slot,
# attestation.data.slot, attestation.data.index) == subnet_id, where
# committees_per_slot = get_committee_count_per_slot(state,
# attestation.data.target.epoch), which may be pre-computed along with the
# committee information for the signature check.
block:
let v = check_attestation_subnet(
shufflingRef, attestation.data.slot, committee_index, subnet_id)
if v.isErr(): # [REJECT]
return pool.checkedReject(v.error)
# In the spec, is_valid_indexed_attestation is used to verify the signature -
# here, we do a batch verification instead
let sig =
attestation.signature.load().valueOr:
return pool.checkedReject("Attestation: unable to load signature")
if not sigchecked:
# findShufflingRef did find a cached ShufflingRef, which means the early
# signature check was skipped, so do it now.
sig = attestation.signature.load().valueOr:
return pool.checkedReject("SingleAttestation: unable to load signature")
return ok((validator_index, sig))
ok((validator_index, beacon_committee.len, index_in_committee, sig))
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/p2p-interface.md#beacon_aggregate_and_proof
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/deneb/p2p-interface.md#beacon_aggregate_and_proof

View File

@ -2677,7 +2677,7 @@ proc getWallEpoch(node: Eth2Node): Epoch =
proc broadcastAttestation*(
node: Eth2Node, subnet_id: SubnetId,
attestation: phase0.Attestation | electra.Attestation):
attestation: phase0.Attestation | SingleAttestation):
Future[SendResult] {.async: (raises: [CancelledError], raw: true).} =
# Regardless of the contents of the attestation,
# https://github.com/ethereum/consensus-specs/blob/v1.5.0-alpha.9/specs/altair/p2p-interface.md#transitioning-the-gossip

View File

@ -297,7 +297,7 @@ proc initFullNode(
proc onPhase0AttestationReceived(data: phase0.Attestation) =
node.eventBus.attestQueue.emit(data)
proc onElectraAttestationReceived(data: electra.Attestation) =
proc onSingleAttestationReceived(data: SingleAttestation) =
debugComment "electra attestation queue"
proc onSyncContribution(data: SignedContributionAndProof) =
node.eventBus.contribQueue.emit(data)
@ -405,7 +405,7 @@ proc initFullNode(
Quarantine.init())
attestationPool = newClone(AttestationPool.init(
dag, quarantine, onPhase0AttestationReceived,
onElectraAttestationReceived))
onSingleAttestationReceived))
syncCommitteeMsgPool = newClone(
SyncCommitteeMsgPool.init(rng, dag.cfg, onSyncContribution))
lightClientPool = newClone(
@ -1958,7 +1958,7 @@ proc installMessageValidators(node: BeaconNode) =
let subnet_id = it
node.network.addAsyncValidator(
getAttestationTopic(digest, subnet_id), proc (
attestation: electra.Attestation
attestation: SingleAttestation
): Future[ValidationResult] {.
async: (raises: [CancelledError]).} =
return toValidationResult(

View File

@ -1450,6 +1450,25 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
# Since our validation logic supports batch processing, we will submit all
# attestations for validation.
for attestation in dres.get():
when AttestationType is electra.Attestation:
let attester_indices = toSeq(
get_attesting_indices(node.dag, attestation, true))
if len(attester_indices) != 1:
return RestApiResponse.jsonError(Http400,
InvalidAttestationObjectError,
$dres.error)
let committee_index = get_committee_index_one(
attestation.committee_bits).valueOr:
return RestApiResponse.jsonError(Http400,
InvalidAttestationObjectError,
$dres.error)
pendingAttestations.add(node.router.routeAttestation(
SingleAttestation(
committee_index: committee_index.distinctBase,
attester_index: attester_indices[0].uint64,
data: attestation.data,
signature: attestation.signature)))
else:
pendingAttestations.add(node.router.routeAttestation(attestation))
case consensusVersion.get():

View File

@ -986,6 +986,14 @@ func shortLog*(v: electra.Attestation | electra.TrustedAttestation): auto =
signature: shortLog(v.signature)
)
func shortLog*(v: SingleAttestation): auto =
(
committee_index: v.committee_index,
attester_index: v.attester_index,
data: shortLog(v.data),
signature: shortLog(v.signature)
)
func init*(
T: type Attestation,
committee_index: CommitteeIndex,

View File

@ -631,7 +631,7 @@ template Forky*(
kind.SignedBeaconBlock
# Workaround method used for tests that involve walking through
# `nim-eth2-scnarios`fork dirs, to be removed once Fulu is
# `nim-eth2-scenarios` fork dirs, to be removed once Fulu is
# included in new release.
template withAllButFulu*(
x: typedesc[ConsensusFork], body: untyped): untyped =
@ -1854,6 +1854,10 @@ func committee_index*(v: electra.Attestation, on_chain: static bool): uint64 =
else:
uint64 v.committee_bits.get_committee_index_one().expect("network attestation")
func committee_index*(
v: SingleAttestation, on_chain: static bool = false): uint64 =
v.committee_index
template init*(T: type ForkedAttestation,
attestation: phase0.Attestation,
fork: ConsensusFork): T =

View File

@ -279,6 +279,7 @@ proc produceAndPublishAttestations*(
tmp.add(RegisteredAttestation(
validator: validator,
validator_index: validator_index,
committee_index: duty.data.committee_index,
index_in_committee: duty.data.validator_committee_index,
committee_len: int duty.data.committee_length,
@ -512,6 +513,7 @@ proc produceAndPublishAttestationsV2*(
tmp.add(RegisteredAttestation(
validator: validator,
validator_index: validator_index,
committee_index: duty.data.committee_index,
index_in_committee: duty.data.validator_committee_index,
committee_len: int(duty.data.committee_length),

View File

@ -362,7 +362,7 @@ proc createAndSendAttestation(node: BeaconNode,
res =
if consensusFork >= ConsensusFork.Electra:
await node.router.routeAttestation(
registered.toElectraAttestation(signature), subnet_id,
registered.toSingleAttestation(signature), subnet_id,
checkSignature = false, checkValidator = false)
else:
await node.router.routeAttestation(
@ -1579,7 +1579,8 @@ proc sendAttestations(node: BeaconNode, head: BlockRef, slot: Slot) =
continue
tmp.add((RegisteredAttestation(
validator: validator, committee_index: committee_index,
validator: validator, validator_index: validator_index,
committee_index: committee_index,
index_in_committee: uint64 index_in_committee,
committee_len: committee.len(), data: data), subnet_id
))

View File

@ -202,7 +202,7 @@ proc routeSignedBeaconBlock*(
proc routeAttestation*(
router: ref MessageRouter,
attestation: phase0.Attestation | electra.Attestation,
attestation: phase0.Attestation | SingleAttestation,
subnet_id: SubnetId, checkSignature, checkValidator: bool):
Future[SendResult] {.async: (raises: [CancelledError]).} =
## Process and broadcast attestation - processing will register the it with
@ -236,7 +236,7 @@ proc routeAttestation*(
proc routeAttestation*(
router: ref MessageRouter,
attestation: phase0.Attestation | electra.Attestation,
attestation: phase0.Attestation | SingleAttestation,
on_chain: static bool = false):
Future[SendResult] {.async: (raises: [CancelledError]).} =
# Compute subnet, then route attestation

View File

@ -25,25 +25,33 @@ type
# the slashing protection database and is therefore ready to be signed and
# sent
validator*: AttachedValidator
validator_index*: ValidatorIndex
committee_index*: CommitteeIndex
index_in_committee*: uint64
committee_len*: int
data*: AttestationData
proc toAttestation*(
func toAttestation*(
registered: RegisteredAttestation, signature: ValidatorSig):
phase0.Attestation =
phase0.Attestation.init(
[registered.index_in_committee], registered.committee_len,
registered.data, signature).expect("valid data")
proc toElectraAttestation*(
func toElectraAttestation*(
registered: RegisteredAttestation, signature: ValidatorSig):
electra.Attestation =
electra.Attestation.init(
registered.committee_index, [registered.index_in_committee],
registered.committee_len, registered.data, signature).expect("valid data")
func toSingleAttestation*(
registered: RegisteredAttestation, signature: ValidatorSig): SingleAttestation =
SingleAttestation(
committee_index: registered.committee_index.distinctBase,
attester_index: registered.validator_index.uint64, data: registered.data,
signature: signature)
proc waitAfterBlockCutoff*(clock: BeaconClock, slot: Slot,
head: Opt[BlockRef] = Opt.none(BlockRef))
{.async: (raises: [CancelledError]).} =

View File

@ -654,11 +654,8 @@ template withMonitor(self: var ValidatorMonitor, idx: ValidatorIndex, body: unty
withMonitor(self, idx.uint64, body)
proc registerAttestation*(
self: var ValidatorMonitor,
src: MsgSource,
seen_timestamp: BeaconTime,
attestation: phase0.Attestation | electra.Attestation,
idx: ValidatorIndex) =
self: var ValidatorMonitor, src: MsgSource, seen_timestamp: BeaconTime,
attestation: phase0.Attestation | SingleAttestation, idx: ValidatorIndex) =
let
slot = attestation.data.slot
delay = seen_timestamp - slot.attestation_deadline()

View File

@ -99,7 +99,7 @@ type: [0x01, 0x00]
data: snappyFramed(ssz(SignedBeaconBlock))
```
`CompressedSignedBeackBlock` contain `SignedBeaconBlock` objects encoded using `SSZ` then compressed using the snappy [framing format](https://github.com/google/snappy/blob/master/framing_format.txt).
`CompressedSignedBeaconBlock` contains `SignedBeaconBlock` objects encoded using `SSZ` then compressed using the snappy [framing format](https://github.com/google/snappy/blob/master/framing_format.txt).
The encoding matches that of the `BeaconBlocksByRoot` and `BeaconBlocksByRange` requests from the p2p specification.
@ -183,7 +183,7 @@ Each era is identified by when it ends. Thus, the genesis era is era `0`, follow
`.era` file names follow a simple convention: `<config-name>-<era-number>-<era-count>-<short-historical-root>.era`:
* `config-name` is the `CONFIG_NAME` field of the runtime configation (`mainnet`, `sepolia`, `holesky`, etc)
* `config-name` is the `CONFIG_NAME` field of the runtime configuration (`mainnet`, `sepolia`, `holesky`, etc)
* `era-number` is the number of the _first_ era stored in the file - for example, the genesis era file has number 0 - as a 5-digit 0-filled decimal integer
* `short-era-root` is the first 4 bytes of the last historical root in the _last_ state in the era file, lower-case hex-encoded (8 characters), except the genesis era which instead uses the `genesis_validators_root` field from the genesis state.
* The root is available as `state.historical_roots[era - 1]` except for genesis, which is `state.genesis_validators_root`
@ -217,8 +217,8 @@ The `era-state` is the state in the era transition slot. The genesis group conta
The structure of the era file gives it the following properties:
* the indices at the end are fixed-length: they can be used to discover the beginning of an era if the end of it is known
* the start slot field of the state slot index idenfifies which era the group pertains to
* the state in the era file is the end state after having applied all the blocks in the era and, if applicable, the block at the first slot - the `block_roots` entries in the state can be used to discover the digest of the blocks - either to verify the intergrity of the era file or to quickly load block roots without computing them.
* the start slot field of the state slot index identifies which era the group pertains to
* the state in the era file is the end state after having applied all the blocks in the era and, if applicable, the block at the first slot - the `block_roots` entries in the state can be used to discover the digest of the blocks - either to verify the integrity of the era file or to quickly load block roots without computing them.
* each group in the era file is full, independent era file - groups can freely be split and combined
## Reading era files

View File

@ -268,31 +268,24 @@ cli do(slots = SLOTS_PER_EPOCH * 7,
sig.toValidatorSig()).expect("valid data")
attPool.addAttestation(
attestation, [validator_index], sig, data.slot.start_beacon_time)
attestation, [validator_index], attestation.aggregation_bits.len,
-1, sig, data.slot.start_beacon_time)
else:
var
data = makeAttestationData(
var data = makeAttestationData(
updatedState, slot, committee_index, bid.root)
committee_bits: BitArray[static(MAX_COMMITTEES_PER_SLOT.int)]
aggregation_bits = ElectraCommitteeValidatorsBits.init(committee.len)
let committeeidx = data.index
aggregation_bits.setBit(index_in_committee)
committee_bits.setBit(committeeidx)
data.index = 0 # obviously, fix in makeAttestationData for Electra
data.index = 0 # fix in makeAttestationData for Electra
let
sig =
get_attestation_signature(
sig = get_attestation_signature(
fork, genesis_validators_root, data,
MockPrivKeys[validator_index])
attestation = electra.Attestation(
# https://github.com/ethereum/consensus-specs/blob/v1.5.0-alpha.8/specs/electra/validator.md#construct-attestation
aggregation_bits: aggregation_bits,
data: data,
committee_bits: committee_bits,
attestation = SingleAttestation(
committee_index: committee_index.distinctBase,
attester_index: validator_index.uint64, data: data,
signature: sig.toValidatorSig())
attPool.addAttestation(
attestation, [validator_index], sig, data.slot.start_beacon_time)
attestation, [validator_index], committee.len,
index_in_committee, sig, data.slot.start_beacon_time)
do:
raiseAssert "withUpdatedState failed"

View File

@ -57,6 +57,12 @@ proc pruneAtFinalization(dag: ChainDAGRef, attPool: AttestationPool) =
dag.pruneStateCachesDAG()
# pool[].prune() # We test logic without attestation pool / fork choice pruning
# -1 here is the notional index in committee for which the attestation pool
# only requires external input regarding SingleAttestation messages. If, or
# when, this module starts testing SingleAttestation, those can't use this.
template addAttestation(a, b, c, d, e, f: untyped): untyped =
addAttestation(a, b, c, d, -1, e, f)
suite "Attestation pool processing" & preset():
## For now just test that we can compile and execute block processing with
## mock data.
@ -186,8 +192,8 @@ suite "Attestation pool processing" & preset():
state[], state[].latest_block_root, bc0[0], cache)
pool[].addAttestation(
attestation, @[bc0[0]], attestation.loadSig,
attestation.data.slot.start_beacon_time)
attestation, @[bc0[0]], attestation.aggregation_bits.len,
attestation.loadSig, attestation.data.slot.start_beacon_time)
check:
# Added attestation, should get it back
@ -240,7 +246,8 @@ suite "Attestation pool processing" & preset():
pool[].getAttestationsForBlock(state[], cache) == []
pool[].addAttestation(
att1, @[bc1[0]], att1.loadSig, att1.data.slot.start_beacon_time)
att1, @[bc1[0]], att1.aggregation_bits.len, att1.loadSig,
att1.data.slot.start_beacon_time)
check:
# but new ones should go in
@ -249,7 +256,8 @@ suite "Attestation pool processing" & preset():
let
att2 = makeAttestation(state[], root1, bc1[1], cache)
pool[].addAttestation(
att2, @[bc1[1]], att2.loadSig, att2.data.slot.start_beacon_time)
att2, @[bc1[1]], att2.aggregation_bits.len, att2.loadSig,
att2.data.slot.start_beacon_time)
let
combined = pool[].getAttestationsForBlock(state[], cache)
@ -260,8 +268,8 @@ suite "Attestation pool processing" & preset():
combined[0].aggregation_bits.countOnes() == 2
pool[].addAttestation(
combined[0], @[bc1[1], bc1[0]], combined[0].loadSig,
combined[0].data.slot.start_beacon_time)
combined[0], @[bc1[1], bc1[0]], combined[0].aggregation_bits.len,
combined[0].loadSig, combined[0].data.slot.start_beacon_time)
check:
# readding the combined attestation shouldn't have an effect
@ -271,7 +279,8 @@ suite "Attestation pool processing" & preset():
# Someone votes for a different root
att3 = makeAttestation(state[], ZERO_HASH, bc1[2], cache)
pool[].addAttestation(
att3, @[bc1[2]], att3.loadSig, att3.data.slot.start_beacon_time)
att3, @[bc1[2]], att3.aggregation_bits.len, att3.loadSig,
att3.data.slot.start_beacon_time)
check:
# We should now get both attestations for the block, but the aggregate
@ -286,7 +295,8 @@ suite "Attestation pool processing" & preset():
# Someone votes for a different root
att4 = makeAttestation(state[], ZERO_HASH, bc1[2], cache)
pool[].addAttestation(
att4, @[bc1[2]], att3.loadSig, att3.data.slot.start_beacon_time)
att4, @[bc1[2]], att4.aggregation_bits.len, att3.loadSig,
att3.data.slot.start_beacon_time)
test "Working with aggregates" & preset():
let
@ -313,9 +323,11 @@ suite "Attestation pool processing" & preset():
not pool[].covers(att0.data, att0.aggregation_bits)
pool[].addAttestation(
att0, @[bc0[0], bc0[2]], att0.loadSig, att0.data.slot.start_beacon_time)
att0, @[bc0[0], bc0[2]], att0.aggregation_bits.len, att0.loadSig,
att0.data.slot.start_beacon_time)
pool[].addAttestation(
att1, @[bc0[1], bc0[2]], att1.loadSig, att1.data.slot.start_beacon_time)
att1, @[bc0[1], bc0[2]], att1.aggregation_bits.len, att1.loadSig,
att1.data.slot.start_beacon_time)
check:
process_slots(
@ -331,7 +343,8 @@ 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.start_beacon_time)
att3, @[bc0[3]], att3.aggregation_bits.len, att3.loadSig,
att3.data.slot.start_beacon_time)
block:
let attestations = pool[].getAttestationsForBlock(state[], cache)
@ -344,7 +357,8 @@ suite "Attestation pool processing" & preset():
# Add in attestation 0 as single - attestation 1 is now a superset of the
# aggregates in the pool, so everything else should be removed
pool[].addAttestation(
att0x, @[bc0[0]], att0x.loadSig, att0x.data.slot.start_beacon_time)
att0x, @[bc0[0]], att0x.aggregation_bits.len, att0x.loadSig,
att0x.data.slot.start_beacon_time)
block:
let attestations = pool[].getAttestationsForBlock(state[], cache)
@ -366,7 +380,8 @@ suite "Attestation pool processing" & preset():
root.data[8..<16] = toBytesBE(j.uint64)
let att = makeAttestation(state[], root, bc0[j], cache)
pool[].addAttestation(
att, @[bc0[j]], att.loadSig, att.data.slot.start_beacon_time)
att, @[bc0[j]], att.aggregation_bits.len, att.loadSig,
att.data.slot.start_beacon_time)
inc attestations
check:
@ -405,11 +420,11 @@ suite "Attestation pool processing" & preset():
# test reverse order
pool[].addAttestation(
attestation1, @[bc1[0]], attestation1.loadSig,
attestation1.data.slot.start_beacon_time)
attestation1, @[bc1[0]], attestation1.aggregation_bits.len,
attestation1.loadSig, attestation1.data.slot.start_beacon_time)
pool[].addAttestation(
attestation0, @[bc0[0]], attestation0.loadSig,
attestation0.data.slot.start_beacon_time)
attestation0, @[bc0[0]], attestation0.aggregation_bits.len,
attestation0.loadSig, attestation0.data.slot.start_beacon_time)
let attestations = pool[].getAttestationsForBlock(state[], cache)
@ -428,11 +443,11 @@ suite "Attestation pool processing" & preset():
makeAttestation(state[], state[].latest_block_root, bc0[1], cache)
pool[].addAttestation(
attestation0, @[bc0[0]], attestation0.loadSig,
attestation0.data.slot.start_beacon_time)
attestation0, @[bc0[0]], attestation0.aggregation_bits.len,
attestation0.loadSig, attestation0.data.slot.start_beacon_time)
pool[].addAttestation(
attestation1, @[bc0[1]], attestation1.loadSig,
attestation1.data.slot.start_beacon_time)
attestation1, @[bc0[1]], attestation1.aggregation_bits.len,
attestation1.loadSig, attestation1.data.slot.start_beacon_time)
check:
process_slots(
@ -459,11 +474,11 @@ suite "Attestation pool processing" & preset():
attestation0.combine(attestation1)
pool[].addAttestation(
attestation0, @[bc0[0]], attestation0.loadSig,
attestation0.data.slot.start_beacon_time)
attestation0, @[bc0[0]], attestation0.aggregation_bits.len,
attestation0.loadSig, attestation0.data.slot.start_beacon_time)
pool[].addAttestation(
attestation1, @[bc0[1]], attestation1.loadSig,
attestation1.data.slot.start_beacon_time)
attestation1, @[bc0[1]], attestation1.aggregation_bits.len,
attestation1.loadSig, attestation1.data.slot.start_beacon_time)
check:
process_slots(
@ -489,11 +504,11 @@ suite "Attestation pool processing" & preset():
attestation0.combine(attestation1)
pool[].addAttestation(
attestation1, @[bc0[1]], attestation1.loadSig,
attestation1.data.slot.start_beacon_time)
attestation1, @[bc0[1]], attestation1.aggregation_bits.len,
attestation1.loadSig, attestation1.data.slot.start_beacon_time)
pool[].addAttestation(
attestation0, @[bc0[0]], attestation0.loadSig,
attestation0.data.slot.start_beacon_time)
attestation0, @[bc0[0]], attestation0.aggregation_bits.len,
attestation0.loadSig, attestation0.data.slot.start_beacon_time)
check:
process_slots(
@ -576,8 +591,8 @@ suite "Attestation pool processing" & preset():
attestation0 = makeAttestation(state[], b10.root, bc1[0], cache)
pool[].addAttestation(
attestation0, @[bc1[0]], attestation0.loadSig,
attestation0.data.slot.start_beacon_time)
attestation0, @[bc1[0]], attestation0.aggregation_bits.len,
attestation0.loadSig, attestation0.data.slot.start_beacon_time)
let head2 =
pool[].selectOptimisticHead(b10Add[].slot.start_beacon_time).get().blck
@ -590,8 +605,8 @@ suite "Attestation pool processing" & preset():
attestation1 = makeAttestation(state[], b11.root, bc1[1], cache)
attestation2 = makeAttestation(state[], b11.root, bc1[2], cache)
pool[].addAttestation(
attestation1, @[bc1[1]], attestation1.loadSig,
attestation1.data.slot.start_beacon_time)
attestation1, @[bc1[1]], attestation1.aggregation_bits.len,
attestation1.loadSig, attestation1.data.slot.start_beacon_time)
let head3 =
pool[].selectOptimisticHead(b10Add[].slot.start_beacon_time).get().blck
@ -602,8 +617,8 @@ suite "Attestation pool processing" & preset():
head3 == bigger[]
pool[].addAttestation(
attestation2, @[bc1[2]], attestation2.loadSig,
attestation2.data.slot.start_beacon_time)
attestation2, @[bc1[2]], attestation2.aggregation_bits.len,
attestation2.loadSig, attestation2.data.slot.start_beacon_time)
let head4 =
pool[].selectOptimisticHead(b11Add[].slot.start_beacon_time).get().blck
@ -775,8 +790,8 @@ suite "Attestation pool electra processing" & preset():
state[], state[].latest_block_root, bc0[0], cache)
pool[].addAttestation(
attestation, @[bc0[0]], attestation.loadSig,
attestation.data.slot.start_beacon_time)
attestation, @[bc0[0]], attestation.aggregation_bits.len,
attestation.loadSig, attestation.data.slot.start_beacon_time)
check:
process_slots(
@ -812,7 +827,8 @@ suite "Attestation pool electra processing" & preset():
pool[].getElectraAttestationsForBlock(state[], cache) == []
pool[].addAttestation(
att1, @[bc1[0]], att1.loadSig, att1.data.slot.start_beacon_time)
att1, @[bc1[0]], att1.aggregation_bits.len, att1.loadSig,
att1.data.slot.start_beacon_time)
check:
# but new ones should go in
@ -821,7 +837,8 @@ suite "Attestation pool electra processing" & preset():
let
att2 = makeElectraAttestation(state[], root1, bc1[1], cache)
pool[].addAttestation(
att2, @[bc1[1]], att2.loadSig, att2.data.slot.start_beacon_time)
att2, @[bc1[1]], att2.aggregation_bits.len, att2.loadSig,
att2.data.slot.start_beacon_time)
let
combined = pool[].getElectraAttestationsForBlock(state[], cache)
@ -832,8 +849,8 @@ suite "Attestation pool electra processing" & preset():
combined[0].aggregation_bits.countOnes() == 2
pool[].addAttestation(
combined[0], @[bc1[1], bc1[0]], combined[0].loadSig,
combined[0].data.slot.start_beacon_time)
combined[0], @[bc1[1], bc1[0]], combined[0].aggregation_bits.len,
combined[0].loadSig, combined[0].data.slot.start_beacon_time)
check:
# readding the combined attestation shouldn't have an effect
@ -843,7 +860,8 @@ suite "Attestation pool electra processing" & preset():
# Someone votes for a different root
att3 = makeElectraAttestation(state[], ZERO_HASH, bc1[2], cache)
pool[].addAttestation(
att3, @[bc1[2]], att3.loadSig, att3.data.slot.start_beacon_time)
att3, @[bc1[2]], att3.aggregation_bits.len, att3.loadSig,
att3.data.slot.start_beacon_time)
check:
# We should now get both attestations for the block, but the aggregate
@ -876,12 +894,12 @@ suite "Attestation pool electra processing" & preset():
state[], state[].latest_block_root, bc1[1], cache)
pool[].addAttestation(
attestation_1, @[bc0[0]], attestation_1.loadSig,
attestation_1.data.slot.start_beacon_time)
attestation_1, @[bc0[0]], attestation_1.aggregation_bits.len,
attestation_1.loadSig, attestation_1.data.slot.start_beacon_time)
pool[].addAttestation(
attestation_2, @[bc0[1]], attestation_2.loadSig,
attestation_2.data.slot.start_beacon_time)
attestation_2, @[bc0[1]], attestation_2.aggregation_bits.len,
attestation_2.loadSig, attestation_2.data.slot.start_beacon_time)
check:
process_slots(
@ -940,16 +958,16 @@ suite "Attestation pool electra processing" & preset():
verifyAttestationSignature(attestation_3)
pool[].addAttestation(
attestation_1, @[bc0[0]], attestation_1.loadSig,
attestation_1.data.slot.start_beacon_time)
attestation_1, @[bc0[0]], attestation_1.aggregation_bits.len,
attestation_1.loadSig, attestation_1.data.slot.start_beacon_time)
pool[].addAttestation(
attestation_2, @[bc0[1]], attestation_2.loadSig,
attestation_2.data.slot.start_beacon_time)
attestation_2, @[bc0[1]], attestation_2.aggregation_bits.len,
attestation_2.loadSig, attestation_2.data.slot.start_beacon_time)
pool[].addAttestation(
attestation_3, @[bc1[1]], attestation_3.loadSig,
attestation_3.data.slot.start_beacon_time)
attestation_3, @[bc1[1]], attestation_3.aggregation_bits.len,
attestation_3.loadSig, attestation_3.data.slot.start_beacon_time)
check:
process_slots(
@ -1019,9 +1037,11 @@ suite "Attestation pool electra processing" & preset():
not pool[].covers(att0.data, att0.aggregation_bits)
pool[].addAttestation(
att0, @[bc0[0], bc0[2]], att0.loadSig, att0.data.slot.start_beacon_time)
att0, @[bc0[0], bc0[2]], att0.aggregation_bits.len, att0.loadSig,
att0.data.slot.start_beacon_time)
pool[].addAttestation(
att1, @[bc0[1], bc0[2]], att1.loadSig, att1.data.slot.start_beacon_time)
att1, @[bc0[1], bc0[2]], att1.aggregation_bits.len, att1.loadSig,
att1.data.slot.start_beacon_time)
for att in pool[].electraAttestations(Opt.none Slot, Opt.none CommitteeIndex):
check: verifyAttestationSignature(att)
@ -1043,7 +1063,8 @@ suite "Attestation pool electra processing" & preset():
# Add in attestation 3 - both aggregates should now have it added
pool[].addAttestation(
att3, @[bc0[3]], att3.loadSig, att3.data.slot.start_beacon_time)
att3, @[bc0[3]], att3.aggregation_bits.len, att3.loadSig,
att3.data.slot.start_beacon_time)
block:
let attestations = pool[].getElectraAttestationsForBlock(state[], cache)
@ -1060,7 +1081,8 @@ suite "Attestation pool electra 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.start_beacon_time)
att0x, @[bc0[0]], att0x.aggregation_bits.len, att0x.loadSig,
att0x.data.slot.start_beacon_time)
block:
let attestations = pool[].getElectraAttestationsForBlock(state[], cache)
@ -1077,7 +1099,8 @@ suite "Attestation pool electra processing" & preset():
let att4 = makeElectraAttestation(state[], ZERO_HASH, bc0[4], cache)
check: verifyAttestationSignature(att4)
pool[].addAttestation(
att4, @[bc0[4]], att4.loadSig, att4.data.slot.start_beacon_time)
att4, @[bc0[4]], att4.aggregation_bits.len, att4.loadSig,
att4.data.slot.start_beacon_time)
# Total aggregations size should be one for that root
check: