electra attestation gossip plumbing (#6287)

This commit is contained in:
tersec 2024-05-14 19:01:26 +03:00 committed by GitHub
parent 1c3aaa7be2
commit c1b9e82502
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
9 changed files with 272 additions and 37 deletions

View File

@ -45,8 +45,6 @@ type
Phase0Validation = Validation[CommitteeValidatorsBits]
ElectraValidation = Validation[ElectraCommitteeValidatorsBits]
AttestationCommitteeBits = BitArray[MAX_COMMITTEES_PER_SLOT.int]
Phase0AttestationEntry = object
## Each entry holds the known signatures for a particular, distinct vote
data: AttestationData

View File

@ -103,13 +103,35 @@ iterator get_attesting_indices*(shufflingRef: ShufflingRef,
# https://github.com/ethereum/consensus-specs/blob/v1.5.0-alpha.0/specs/electra/beacon-chain.md#modified-get_attesting_indices
iterator get_attesting_indices*(shufflingRef: ShufflingRef,
slot: Slot,
committee_index: CommitteeIndex,
bits: ElectraCommitteeValidatorsBits):
committee_bits: AttestationCommitteeBits,
aggregation_bits: ElectraCommitteeValidatorsBits):
ValidatorIndex =
debugRaiseAssert "spec cache get_attesting_indices for electra"
debugRaiseAssert "compatible with shuffling? needs checking"
#if not aggregation_bits.compatible_with_shuffling(shufflingRef, slot, committee_index):
if false:
trace "get_attesting_indices: inconsistent aggregation and committee length"
else:
debugComment "replace this implementation with actual iterator, after checking on conditions re repeat vals, ordering, etc; this is almost direct transcription of spec link algorithm in one of the places it doesn't make sense"
## Return the set of attesting indices corresponding to ``aggregation_bits``
## and ``committee_bits``.
var output: HashSet[ValidatorIndex]
let committee_indices = toSeq(committee_bits.oneIndices)
var committee_offset = 0
for index in committee_indices:
let committee = get_beacon_committee(shufflingRef, slot, index.CommitteeIndex)
var committee_attesters: HashSet[ValidatorIndex]
for i, index in committee:
if aggregation_bits[committee_offset + i]:
committee_attesters.incl index
output.incl committee_attesters
committee_offset += len(committee)
for validatorIndex in output:
yield validatorIndex
iterator get_attesting_indices*(
dag: ChainDAGRef, attestation: phase0.TrustedAttestation | electra.TrustedAttestation): ValidatorIndex =
dag: ChainDAGRef, attestation: phase0.TrustedAttestation): ValidatorIndex =
block: # `return` is not allowed in an inline iterator
let
slot =
@ -159,6 +181,48 @@ iterator get_attesting_indices*(
shufflingRef, slot, committeeIndex, attestation.aggregation_bits):
yield validator
iterator get_attesting_indices*(
dag: ChainDAGRef, attestation: electra.TrustedAttestation): ValidatorIndex =
debugRaiseAssert "bad duplication, mostly to avoid the get_attesting_index call from potentially getting screwed up in deployment version"
block: # `return` is not allowed in an inline iterator
let
slot =
check_attestation_slot_target(attestation.data).valueOr:
warn "Invalid attestation slot in trusted attestation",
attestation = shortLog(attestation)
doAssert strictVerification notin dag.updateFlags
break
blck =
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",
attestation = shortLog(attestation)
break
target =
blck.atCheckpoint(attestation.data.target).valueOr:
# This may happen when there's no block at the epoch boundary slot
# 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",
blck = shortLog(blck),
attestation = shortLog(attestation)
doAssert strictVerification notin dag.updateFlags
break
shufflingRef =
dag.getShufflingRef(target.blck, target.slot.epoch, false).valueOr:
warn "Attestation shuffling not found",
blck = shortLog(blck),
attestation = shortLog(attestation)
doAssert strictVerification notin dag.updateFlags
break
for validator in get_attesting_indices(
shufflingRef, slot, attestation.committee_bits, attestation.aggregation_bits):
yield validator
func get_attesting_indices_one*(shufflingRef: ShufflingRef,
slot: Slot,
committee_index: CommitteeIndex,
@ -173,6 +237,20 @@ func get_attesting_indices_one*(shufflingRef: ShufflingRef,
res = some(validator_index)
res
func get_attesting_indices_one*(shufflingRef: ShufflingRef,
slot: Slot,
committee_indices: AttestationCommitteeBits,
aggregation_bits: ElectraCommitteeValidatorsBits):
Option[ValidatorIndex] =
# A variation on get_attesting_indices that returns the validator index only
# if only one validator index is set
var res = none(ValidatorIndex)
for validator_index in get_attesting_indices(
shufflingRef, slot, committee_indices, aggregation_bits):
if res.isSome(): return none(ValidatorIndex)
res = some(validator_index)
res
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#get_attesting_indices
func get_attesting_indices*(shufflingRef: ShufflingRef,
slot: Slot,
@ -182,6 +260,14 @@ func get_attesting_indices*(shufflingRef: ShufflingRef,
for idx in get_attesting_indices(shufflingRef, slot, committee_index, bits):
result.add(idx)
func get_attesting_indices*(shufflingRef: ShufflingRef,
slot: Slot,
committee_index: CommitteeIndex,
bits: ElectraCommitteeValidatorsBits):
seq[ValidatorIndex] =
for idx in get_attesting_indices(shufflingRef, slot, committee_index, bits):
result.add(idx)
func makeAttestationData*(
epochRef: EpochRef, bs: BlockSlot,
committee_index: CommitteeIndex): AttestationData =

View File

@ -342,7 +342,8 @@ proc clearDoppelgangerProtection*(self: var Eth2Processor) =
self.doppelgangerDetection.broadcastStartEpoch = FAR_FUTURE_EPOCH
proc checkForPotentialDoppelganger(
self: var Eth2Processor, attestation: phase0.Attestation,
self: var Eth2Processor,
attestation: phase0.Attestation | electra.Attestation,
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
@ -364,7 +365,7 @@ proc checkForPotentialDoppelganger(
proc processAttestation*(
self: ref Eth2Processor, src: MsgSource,
attestation: phase0.Attestation, subnet_id: SubnetId,
attestation: phase0.Attestation | electra.Attestation, subnet_id: SubnetId,
checkSignature: bool = true): Future[ValidationRes] {.async: (raises: [CancelledError]).} =
var wallTime = self.getCurrentBeaconTime()
let (afterGenesis, wallSlot) = wallTime.toSlot()
@ -412,8 +413,10 @@ proc processAttestation*(
proc processSignedAggregateAndProof*(
self: ref Eth2Processor, src: MsgSource,
signedAggregateAndProof: phase0.SignedAggregateAndProof,
checkSignature = true, checkCover = true): Future[ValidationRes] {.async: (raises: [CancelledError]).} =
signedAggregateAndProof:
phase0.SignedAggregateAndProof | electra.SignedAggregateAndProof,
checkSignature = true, checkCover = true): Future[ValidationRes]
{.async: (raises: [CancelledError]).} =
var wallTime = self.getCurrentBeaconTime()
let (afterGenesis, wallSlot) = wallTime.toSlot()

View File

@ -168,6 +168,12 @@ func check_aggregation_count(
ok()
func check_aggregation_count(
attestation: electra.Attestation, singular: bool):
Result[void, ValidationError] =
debugComment "it's sometimes not"
ok()
func check_attestation_subnet(
shufflingRef: ShufflingRef, slot: Slot, committee_index: CommitteeIndex,
subnet_id: SubnetId): Result[void, ValidationError] =
@ -817,6 +823,71 @@ proc validateAttestation*(
return ok((validator_index, sig))
proc validateAttestation*(
pool: ref AttestationPool,
batchCrypto: ref BatchCrypto,
attestation: electra.Attestation,
wallTime: BeaconTime,
subnet_id: SubnetId, checkSignature: bool):
Future[Result[
tuple[attesting_index: ValidatorIndex, sig: CookedSig],
ValidationError]] {.async: (raises: [CancelledError]).} =
debugComment "should reject a bunch"
# [REJECT] The attestation's epoch matches its target -- i.e.
# attestation.data.target.epoch ==
# compute_epoch_at_slot(attestation.data.slot)
let slot = block:
let v = check_attestation_slot_target(attestation.data)
if v.isErr():
return pool.checkedReject(v.error())
v.get()
# 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).
# [REJECT] The block being voted for (attestation.data.beacon_block_root)
# passes validation.
# [IGNORE] if block is unseen so far and enqueue it in missing blocks
let target = block:
let v = check_beacon_and_target_block(pool[], attestation.data)
if v.isErr(): # [IGNORE/REJECT]
return pool.checkedResult(v.error)
v.get()
# The following rule follows implicitly from that we clear out any
# unviable blocks from the chain dag:
#
# [IGNORE] The current finalized_checkpoint is an ancestor of the block
# 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
fork = pool.dag.forkAtEpoch(attestation.data.slot.epoch)
attesting_index = get_attesting_indices_one(
shufflingRef, slot, attestation.committee_bits, attestation.aggregation_bits)
# 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()
# 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")
return ok((validator_index, 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
proc validateAggregate*(
@ -1031,6 +1102,64 @@ proc validateAggregate*(
return ok((attesting_indices, sig))
proc validateAggregate*(
pool: ref AttestationPool,
batchCrypto: ref BatchCrypto,
signedAggregateAndProof: electra.SignedAggregateAndProof,
wallTime: BeaconTime,
checkSignature = true, checkCover = true):
Future[Result[
tuple[attestingIndices: seq[ValidatorIndex], sig: CookedSig],
ValidationError]] {.async: (raises: [CancelledError]).} =
debugComment "is not"
template aggregate_and_proof: untyped = signedAggregateAndProof.message
template aggregate: untyped = aggregate_and_proof.aggregate
# [REJECT] The aggregate attestation's epoch matches its target -- i.e.
# `aggregate.data.target.epoch == compute_epoch_at_slot(aggregate.data.slot)`
let slot = block:
let v = check_attestation_slot_target(aggregate.data)
if v.isErr():
return pool.checkedReject(v.error)
v.get()
# [REJECT] The block being voted for (aggregate.data.beacon_block_root)
# passes validation.
# [IGNORE] if block is unseen so far and enqueue it in missing blocks
let target = block:
let v = check_beacon_and_target_block(pool[], aggregate.data)
if v.isErr(): # [IGNORE/REJECT]
return pool.checkedResult(v.error)
v.get()
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",
aggregate = shortLog(aggregate), target = shortLog(target)
return errIgnore("Aggregate: no shuffling")
# [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(aggregate.data.index)
if idx.isErr():
return pool.checkedReject(
"Attestation: committee index not within expected range")
idx.get()
let
fork = pool.dag.forkAtEpoch(aggregate.data.slot.epoch)
attesting_indices = get_attesting_indices(
shufflingRef, slot, committee_index, aggregate.aggregation_bits)
let
sig =
aggregate.signature.load().valueOr:
return pool.checkedReject("Aggregate: unable to load signature")
ok((attesting_indices, sig))
# https://github.com/ethereum/consensus-specs/blob/v1.5.0-alpha.2/specs/capella/p2p-interface.md#bls_to_execution_change
proc validateBlsToExecutionChange*(
pool: ValidatorChangePool, batchCrypto: ref BatchCrypto,

View File

@ -839,6 +839,7 @@ template gossipMaxSize(T: untyped): uint32 =
# having max sizes significantly smaller than GOSSIP_MAX_SIZE.
elif T is phase0.Attestation or T is phase0.AttesterSlashing or
T is phase0.SignedAggregateAndProof or T is phase0.SignedBeaconBlock or
T is electra.SignedAggregateAndProof or T is electra.Attestation or
T is altair.SignedBeaconBlock or T is SomeForkyLightClientObject:
GOSSIP_MAX_SIZE
else:

View File

@ -1755,28 +1755,47 @@ proc installMessageValidators(node: BeaconNode) =
# beacon_attestation_{subnet_id}
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/p2p-interface.md#beacon_attestation_subnet_id
for it in SubnetId:
debugRaiseAssert "allow for electra.Attestation"
closureScope: # Needed for inner `proc`; don't lift it out of loop.
let subnet_id = it
node.network.addAsyncValidator(
getAttestationTopic(digest, subnet_id), proc (
attestation: phase0.Attestation
): Future[ValidationResult] {.async: (raises: [CancelledError]).} =
return toValidationResult(
await node.processor.processAttestation(
MsgSource.gossip, attestation, subnet_id)))
when consensusFork >= ConsensusFork.Electra:
for it in SubnetId:
closureScope: # Needed for inner `proc`; don't lift it out of loop.
let subnet_id = it
node.network.addAsyncValidator(
getAttestationTopic(digest, subnet_id), proc (
attestation: electra.Attestation
): Future[ValidationResult] {.async: (raises: [CancelledError]).} =
return toValidationResult(
await node.processor.processAttestation(
MsgSource.gossip, attestation, subnet_id)))
else:
for it in SubnetId:
closureScope: # Needed for inner `proc`; don't lift it out of loop.
let subnet_id = it
node.network.addAsyncValidator(
getAttestationTopic(digest, subnet_id), proc (
attestation: phase0.Attestation
): Future[ValidationResult] {.async: (raises: [CancelledError]).} =
return toValidationResult(
await node.processor.processAttestation(
MsgSource.gossip, attestation, subnet_id)))
# beacon_aggregate_and_proof
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.4/specs/phase0/p2p-interface.md#beacon_aggregate_and_proof
debugRaiseAssert "allow for electra.SignedAggregateAndProof"
node.network.addAsyncValidator(
getAggregateAndProofsTopic(digest), proc (
signedAggregateAndProof: phase0.SignedAggregateAndProof
): Future[ValidationResult] {.async: (raises: [CancelledError]).} =
return toValidationResult(
await node.processor.processSignedAggregateAndProof(
MsgSource.gossip, signedAggregateAndProof)))
when consensusFork >= ConsensusFork.Electra:
node.network.addAsyncValidator(
getAggregateAndProofsTopic(digest), proc (
signedAggregateAndProof: electra.SignedAggregateAndProof
): Future[ValidationResult] {.async: (raises: [CancelledError]).} =
return toValidationResult(
await node.processor.processSignedAggregateAndProof(
MsgSource.gossip, signedAggregateAndProof)))
else:
node.network.addAsyncValidator(
getAggregateAndProofsTopic(digest), proc (
signedAggregateAndProof: phase0.SignedAggregateAndProof
): Future[ValidationResult] {.async: (raises: [CancelledError]).} =
return toValidationResult(
await node.processor.processSignedAggregateAndProof(
MsgSource.gossip, signedAggregateAndProof)))
# attester_slashing
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/p2p-interface.md#attester_slashing
@ -1856,9 +1875,6 @@ proc installMessageValidators(node: BeaconNode) =
node.processor[].processBlobSidecar(
MsgSource.gossip, blobSidecar, subnet_id)))
when consensusFork >= ConsensusFork.Electra:
discard
node.installLightClientMessageValidators()
proc stop(node: BeaconNode) =

View File

@ -596,11 +596,13 @@ type
ElectraCommitteeValidatorsBits* =
BitList[Limit MAX_VALIDATORS_PER_COMMITTEE * MAX_COMMITTEES_PER_SLOT]
AttestationCommitteeBits* = BitArray[MAX_COMMITTEES_PER_SLOT.int]
# https://github.com/ethereum/consensus-specs/blob/v1.5.0-alpha.0/specs/electra/beacon-chain.md#attestation
Attestation* = object
aggregation_bits*: ElectraCommitteeValidatorsBits
data*: AttestationData
committee_bits*: BitArray[MAX_COMMITTEES_PER_SLOT.int] # [New in Electra:EIP7549]
committee_bits*: AttestationCommitteeBits # [New in Electra:EIP7549]
signature*: ValidatorSig
TrustedAttestation* = object
@ -609,7 +611,7 @@ type
# Currently the code MUST verify the state transition as soon as the signature is verified
aggregation_bits*: ElectraCommitteeValidatorsBits
data*: AttestationData
committee_bits*: BitArray[MAX_COMMITTEES_PER_SLOT.int] # [New in Electra:EIP7549]
committee_bits*: AttestationCommitteeBits # [New in Electra:EIP7549]
signature*: TrustedSig
SomeSignedBeaconBlock* =

View File

@ -533,7 +533,7 @@ proc makeBeaconBlockWithRewards*(
forkyState.data.latest_execution_payload_header.transactions_root =
transactions_root.get
debugRaiseAssert "makeBeaconBlock doesn't support Electra (i.e. check for missing beaconblock body fields)"
debugComment "makeBeaconBlock doesn't support Electra (i.e. check for missing beaconblock body fields)"
when executionPayload is electra.ExecutionPayloadForSigning:
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/deneb/beacon-chain.md#beaconblockbody
forkyState.data.latest_block_header.body_root = hash_tree_root(

View File

@ -660,7 +660,7 @@ proc registerAttestation*(
self: var ValidatorMonitor,
src: MsgSource,
seen_timestamp: BeaconTime,
attestation: phase0.Attestation,
attestation: phase0.Attestation | electra.Attestation,
idx: ValidatorIndex) =
let
slot = attestation.data.slot
@ -685,7 +685,7 @@ proc registerAggregate*(
self: var ValidatorMonitor,
src: MsgSource,
seen_timestamp: BeaconTime,
aggregate_and_proof: phase0.AggregateAndProof,
aggregate_and_proof: phase0.AggregateAndProof | electra.AggregateAndProof,
attesting_indices: openArray[ValidatorIndex]) =
let
slot = aggregate_and_proof.aggregate.data.slot