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:
parent
c344e94adf
commit
9c9cbed987
|
@ -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)
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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():
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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 =
|
||||
|
|
|
@ -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),
|
||||
|
|
|
@ -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
|
||||
))
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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]).} =
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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"
|
||||
|
||||
|
|
|
@ -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:
|
||||
|
|
Loading…
Reference in New Issue