diff --git a/beacon_chain/consensus_object_pools/attestation_pool.nim b/beacon_chain/consensus_object_pools/attestation_pool.nim index ff70a1f43..30a41142d 100644 --- a/beacon_chain/consensus_object_pools/attestation_pool.nim +++ b/beacon_chain/consensus_object_pools/attestation_pool.nim @@ -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 @@ -469,7 +491,7 @@ proc addAttestation*( template addAttToPool(_: electra.Attestation) {.used.} = let committee_index = get_committee_index_one(attestation.committee_bits).expect("TODO") - data = AttestationData( + data = AttestationData( slot: attestation.data.slot, index: uint64 committee_index, beacon_block_root: attestation.data.beacon_block_root, @@ -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) diff --git a/beacon_chain/consensus_object_pools/spec_cache.nim b/beacon_chain/consensus_object_pools/spec_cache.nim index b64ca9de1..171d5ead8 100644 --- a/beacon_chain/consensus_object_pools/spec_cache.nim +++ b/beacon_chain/consensus_object_pools/spec_cache.nim @@ -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 diff --git a/beacon_chain/gossip_processing/eth2_processor.nim b/beacon_chain/gossip_processing/eth2_processor.nim index 89d155ca4..276d7e7c3 100644 --- a/beacon_chain/gossip_processing/eth2_processor.nim +++ b/beacon_chain/gossip_processing/eth2_processor.nim @@ -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( - self.batchCrypto, attestation, wallTime, subnet_id, checkSignature) + 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( diff --git a/beacon_chain/gossip_processing/gossip_validation.nim b/beacon_chain/gossip_processing/gossip_validation.nim index f4309bce9..5b4161feb 100644 --- a/beacon_chain/gossip_processing/gossip_validation.nim +++ b/beacon_chain/gossip_processing/gossip_validation.nim @@ -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,13 +913,12 @@ 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 = - pool.dag.getShufflingRef(target.blck, target.slot.epoch, false).valueOr: - # Target is verified - shouldn't happen - warn "No shuffling for attestation - report bug", - attestation = shortLog(attestation), target = shortLog(target) - return errIgnore("Attestation: no shuffling") + 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", + attestation = shortLog(attestation), target = shortLog(target) + return errIgnore("Attestation: no shuffling") # [REJECT] The committee index is within the expected range -- i.e. # data.index < get_committee_count_per_slot(state, data.target.epoch). @@ -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 diff --git a/beacon_chain/networking/eth2_network.nim b/beacon_chain/networking/eth2_network.nim index dd8f1f01a..8e831e387 100644 --- a/beacon_chain/networking/eth2_network.nim +++ b/beacon_chain/networking/eth2_network.nim @@ -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 diff --git a/beacon_chain/nimbus_beacon_node.nim b/beacon_chain/nimbus_beacon_node.nim index e99b93358..6246ec310 100644 --- a/beacon_chain/nimbus_beacon_node.nim +++ b/beacon_chain/nimbus_beacon_node.nim @@ -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( diff --git a/beacon_chain/rpc/rest_beacon_api.nim b/beacon_chain/rpc/rest_beacon_api.nim index 1457cf981..1800649ce 100644 --- a/beacon_chain/rpc/rest_beacon_api.nim +++ b/beacon_chain/rpc/rest_beacon_api.nim @@ -1445,12 +1445,31 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) = let dres = decodeBody(seq[AttestationType], contentBody.get()) if dres.isErr(): return RestApiResponse.jsonError(Http400, - InvalidAttestationObjectError, - $dres.error) + InvalidAttestationObjectError, + $dres.error) # Since our validation logic supports batch processing, we will submit all # attestations for validation. for attestation in dres.get(): - pendingAttestations.add(node.router.routeAttestation(attestation)) + 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(): of ConsensusFork.Phase0 .. ConsensusFork.Deneb: @@ -1515,7 +1534,7 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) = let contextFork = node.dag.cfg.consensusForkAtEpoch(node.currentSlot.epoch) - + withConsensusFork(contextFork): when consensusFork < ConsensusFork.Electra: RestApiResponse.jsonResponseWVersion( diff --git a/beacon_chain/spec/datatypes/electra.nim b/beacon_chain/spec/datatypes/electra.nim index 278d49ad1..2ea6db44b 100644 --- a/beacon_chain/spec/datatypes/electra.nim +++ b/beacon_chain/spec/datatypes/electra.nim @@ -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, diff --git a/beacon_chain/spec/forks.nim b/beacon_chain/spec/forks.nim index 1b5a97b0f..da13a3f9b 100644 --- a/beacon_chain/spec/forks.nim +++ b/beacon_chain/spec/forks.nim @@ -630,9 +630,9 @@ template Forky*( kind: static ConsensusFork): auto = kind.SignedBeaconBlock -# Workaround method used for tests that involve walking through -# `nim-eth2-scnarios`fork dirs, to be removed once Fulu is -# included in new release. +# Workaround method used for tests that involve walking through +# `nim-eth2-scenarios` fork dirs, to be removed once Fulu is +# included in new release. template withAllButFulu*( x: typedesc[ConsensusFork], body: untyped): untyped = static: doAssert ConsensusFork.high == ConsensusFork.Fulu @@ -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 = diff --git a/beacon_chain/validator_client/attestation_service.nim b/beacon_chain/validator_client/attestation_service.nim index 9e215feeb..74c5af50f 100644 --- a/beacon_chain/validator_client/attestation_service.nim +++ b/beacon_chain/validator_client/attestation_service.nim @@ -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), diff --git a/beacon_chain/validators/beacon_validators.nim b/beacon_chain/validators/beacon_validators.nim index 4a68ca8a7..a15dddb44 100644 --- a/beacon_chain/validators/beacon_validators.nim +++ b/beacon_chain/validators/beacon_validators.nim @@ -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 )) diff --git a/beacon_chain/validators/message_router.nim b/beacon_chain/validators/message_router.nim index 5cde690eb..0965453b9 100644 --- a/beacon_chain/validators/message_router.nim +++ b/beacon_chain/validators/message_router.nim @@ -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 diff --git a/beacon_chain/validators/validator_duties.nim b/beacon_chain/validators/validator_duties.nim index c7c9accc9..bc42e4b8a 100644 --- a/beacon_chain/validators/validator_duties.nim +++ b/beacon_chain/validators/validator_duties.nim @@ -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]).} = diff --git a/beacon_chain/validators/validator_monitor.nim b/beacon_chain/validators/validator_monitor.nim index 1259cfeec..a981042d6 100644 --- a/beacon_chain/validators/validator_monitor.nim +++ b/beacon_chain/validators/validator_monitor.nim @@ -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() diff --git a/docs/e2store.md b/docs/e2store.md index 492a672c0..7fffaea0c 100644 --- a/docs/e2store.md +++ b/docs/e2store.md @@ -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: `---.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 diff --git a/research/block_sim.nim b/research/block_sim.nim index af9ae5b7e..94529e526 100644 --- a/research/block_sim.nim +++ b/research/block_sim.nim @@ -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( - 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 + var data = makeAttestationData( + updatedState, slot, committee_index, bid.root) + data.index = 0 # fix in makeAttestationData for Electra let - 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, + sig = get_attestation_signature( + fork, genesis_validators_root, data, + MockPrivKeys[validator_index]) + 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" @@ -609,4 +602,4 @@ cli do(slots = SLOTS_PER_EPOCH * 7, echo "Done!" - printTimers(dag.headState, attesters, true, timers) + printTimers(dag.headState, attesters, true, timers) \ No newline at end of file diff --git a/tests/test_attestation_pool.nim b/tests/test_attestation_pool.nim index 4fccbf087..b19d9d426 100644 --- a/tests/test_attestation_pool.nim +++ b/tests/test_attestation_pool.nim @@ -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: