electra attestation gossip plumbing (#6287)
This commit is contained in:
parent
1c3aaa7be2
commit
c1b9e82502
|
@ -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
|
||||
|
|
|
@ -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 =
|
||||
|
|
|
@ -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()
|
||||
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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:
|
||||
|
|
|
@ -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) =
|
||||
|
|
|
@ -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* =
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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
|
||||
|
|
Loading…
Reference in New Issue