segregate sync committee messages by period / fork (#4953)

`SyncCommitteeMsgPool` grouped messages by their `beacon_block_root`.
This is problematic around sync committee period boundaries and forks.
Around sync committee period boundaries, members from both the current
and next sync committee may sign the same `beacon_block_root`; mixing
the signatures from both committees together is a mistake. Likewise,
around fork transitions, the `signing_root` changes, so those messages
also need to be segregated.
This commit is contained in:
Etan Kissling 2023-05-17 06:55:55 +02:00 committed by GitHub
parent b3c3b9a7d8
commit 40e89937c5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 450 additions and 229 deletions

View File

@ -532,8 +532,11 @@ OK: 1/1 Fail: 0/1 Skip: 0/1
+ An empty pool is safe to prune OK
+ An empty pool is safe to prune 2 OK
+ An empty pool is safe to use OK
+ Missed slots across fork transition OK
+ Missed slots across sync committee period boundary OK
+ isSeen OK
```
OK: 4/4 Fail: 0/4 Skip: 0/4
OK: 7/7 Fail: 0/7 Skip: 0/7
## SyncManager test suite
```diff
+ Process all unviable blocks OK
@ -677,4 +680,4 @@ OK: 2/2 Fail: 0/2 Skip: 0/2
OK: 9/9 Fail: 0/9 Skip: 0/9
---TOTAL---
OK: 386/391 Fail: 0/391 Skip: 5/391
OK: 389/394 Fail: 0/394 Skip: 5/394

View File

@ -8,11 +8,11 @@
{.push raises: [].}
import
std/[sets, tables],
std/[algorithm, sequtils, sets, tables],
stew/shims/hashes,
eth/p2p/discoveryv5/random2,
chronicles,
../spec/[crypto, digest],
../spec/[crypto, digest, forks],
../spec/datatypes/altair
export hashes, sets, tables, altair
@ -24,12 +24,11 @@ const
type
SyncCommitteeMsgKey = object
originator: uint64 # ValidatorIndex avoiding mess with invalid values
originator: uint64 # ValidatorIndex to avoid invalid values
slot: Slot
subcommitteeIdx: uint64 # SyncSubcommitteeIndex avoiding mess with invalid values
subcommitteeIdx: uint64 # SyncSubcommitteeIndex to avoid invalid values
TrustedSyncCommitteeMsg* = object
slot*: Slot
subcommitteeIdx*: SyncSubcommitteeIndex
positionInCommittee*: uint64
signature*: CookedSig
@ -40,32 +39,61 @@ type
signature*: CookedSig
BestSyncSubcommitteeContributions* = object
slot*: Slot
subnets*: array[SYNC_COMMITTEE_SUBNET_COUNT,
BestSyncSubcommitteeContribution]
OnSyncContributionCallback* =
proc(data: SignedContributionAndProof) {.gcsafe, raises: [Defect].}
# Messages from different slots / forks may sign the same beacon block root.
# Messages across slots are compatible, but not across forks (signing root).
# Messages from different periods have different signers, so are incompatible.
# Note that the sync committee is determined by `message.slot + 1`, the fork
# is determined by `message.slot`, and both can be different from `bid.slot`.
SyncMsgTarget = object
bid: BlockId # Based on message `beacon_block_root`
period: SyncCommitteePeriod # Based on message `slot + 1`
fork: ConsensusFork # Based on message `slot`
SyncCommitteeMsgPool* = object
seenSyncMsgByAuthor*: HashSet[SyncCommitteeMsgKey]
seenSyncMsgByAuthor*: Table[SyncCommitteeMsgKey, Eth2Digest]
seenContributionByAuthor*: HashSet[SyncCommitteeMsgKey]
syncMessages*: Table[Eth2Digest, seq[TrustedSyncCommitteeMsg]]
bestContributions*: Table[Eth2Digest, BestSyncSubcommitteeContributions]
syncMessages*: Table[SyncMsgTarget, seq[TrustedSyncCommitteeMsg]]
bestContributions*: Table[SyncMsgTarget, BestSyncSubcommitteeContributions]
onContributionReceived*: OnSyncContributionCallback
rng: ref HmacDrbgContext
cfg: RuntimeConfig
func hash*(x: SyncCommitteeMsgKey): Hash =
hashAllFields(x)
func toSyncMsgTarget(
cfg: RuntimeConfig, bid: BlockId, slot: Slot): SyncMsgTarget =
SyncMsgTarget(
bid: bid,
period: (slot + 1).sync_committee_period,
fork: cfg.consensusForkAtEpoch(slot.epoch))
func hash(x: SyncMsgTarget): Hash =
hashAllFields(x)
func `<`(x, y: SyncMsgTarget): bool =
if x.bid.slot != y.bid.slot:
x.bid.slot < y.bid.slot
elif x.period != y.period:
x.period < y.period
else:
x.fork < y.fork
func init*(T: type SyncCommitteeMsgPool,
rng: ref HmacDrbgContext,
cfg: RuntimeConfig,
onSyncContribution: OnSyncContributionCallback = nil
): SyncCommitteeMsgPool =
T(rng: rng, onContributionReceived: onSyncContribution)
T(rng: rng, cfg: cfg, onContributionReceived: onSyncContribution)
func pruneData*(pool: var SyncCommitteeMsgPool, slot: Slot) =
func pruneData*(pool: var SyncCommitteeMsgPool, slot: Slot, force = false) =
## This should be called at the end of slot.
clear pool.seenContributionByAuthor
clear pool.seenSyncMsgByAuthor
@ -73,60 +101,70 @@ func pruneData*(pool: var SyncCommitteeMsgPool, slot: Slot) =
if slot < syncCommitteeMsgsRetentionSlots:
return
let minSlotToRetain = slot - syncCommitteeMsgsRetentionSlots
var syncMsgsToDelete: seq[Eth2Digest]
var contributionsToDelete: seq[Eth2Digest]
# Messages signing a `beacon_block_root` may remain valid over multiple slots.
# Therefore, we filter by the targeted `BlockId` instead of message `slot`.
let
minSlotToRetain = slot - syncCommitteeMsgsRetentionSlots
minEntriesToKeep = if force: 0 else: syncCommitteeMsgsRetentionSlots
for blockRoot, msgs in pool.syncMessages:
if msgs[0].slot < minSlotToRetain:
syncMsgsToDelete.add blockRoot
template pruneTable(table: untyped) =
if table.len > minEntriesToKeep:
var targets = table.keys().toSeq()
targets.sort(order = SortOrder.Descending)
for i in minEntriesToKeep ..< targets.len:
if targets[i].bid.slot < minSlotToRetain:
table.del targets[i]
for blockRoot in syncMsgsToDelete:
pool.syncMessages.del blockRoot
for blockRoot, bestContributions in pool.bestContributions:
if bestContributions.slot < minSlotToRetain:
contributionsToDelete.add blockRoot
for blockRoot in contributionsToDelete:
pool.bestContributions.del blockRoot
pruneTable pool.syncMessages
pruneTable pool.bestContributions
func isSeen*(
pool: SyncCommitteeMsgPool,
msg: SyncCommitteeMessage,
subcommitteeIdx: SyncSubcommitteeIndex): bool =
subcommitteeIdx: SyncSubcommitteeIndex,
headBid: BlockId): bool =
let seenKey = SyncCommitteeMsgKey(
originator: msg.validator_index, # Might be unvalidated at this point
originator: msg.validator_index, # Might be unvalidated at this point
slot: msg.slot,
subcommitteeIdx: subcommitteeIdx.uint64)
seenKey in pool.seenSyncMsgByAuthor
return
if seenKey notin pool.seenSyncMsgByAuthor:
false
elif msg.beacon_block_root == headBid.root:
pool.seenSyncMsgByAuthor.getOrDefault(seenKey) == headBid.root
else:
true
proc addSyncCommitteeMessage*(
pool: var SyncCommitteeMsgPool,
slot: Slot,
blockRoot: Eth2Digest,
bid: BlockId,
validatorIndex: uint64,
signature: CookedSig,
subcommitteeIdx: SyncSubcommitteeIndex,
positionsInCommittee: openArray[uint64]) =
positionsInCommittee: seq[uint64]) =
let seenKey = SyncCommitteeMsgKey(
originator: validatorIndex,
slot: slot,
subcommitteeIdx: subcommitteeIdx.uint64)
pool.seenSyncMsgByAuthor[seenKey] = bid.root
let
seenKey = SyncCommitteeMsgKey(
originator: validatorIndex,
slot: slot,
subcommitteeIdx: subcommitteeIdx.uint64)
pool.seenSyncMsgByAuthor.incl seenKey
for position in positionsInCommittee:
pool.syncMessages.mgetOrPut(blockRoot, @[]).add TrustedSyncCommitteeMsg(
slot: slot,
subcommitteeIdx: subcommitteeIdx,
positionInCommittee: position,
signature: signature)
func registerVotes(votes: var seq[TrustedSyncCommitteeMsg]) =
for position in positionsInCommittee:
block addVote:
for vote in votes:
if vote.subcommitteeIdx == subcommitteeIdx and
vote.positionInCommittee == position:
break addVote
votes.add TrustedSyncCommitteeMsg(
subcommitteeIdx: subcommitteeIdx,
positionInCommittee: position,
signature: signature)
let target = pool.cfg.toSyncMsgTarget(bid, slot)
pool.syncMessages.mgetOrPut(target, @[]).registerVotes()
debug "Sync committee message resolved",
slot = slot, blockRoot = shortLog(blockRoot), validatorIndex
slot = slot, blockRoot = shortLog(target.bid.root), validatorIndex
func computeAggregateSig(votes: seq[TrustedSyncCommitteeMsg],
subcommitteeIdx: SyncSubcommitteeIndex,
@ -135,6 +173,7 @@ func computeAggregateSig(votes: seq[TrustedSyncCommitteeMsg],
aggregateSig {.noinit.}: AggregateSignature
initialized = false
contribution.aggregation_bits.reset()
for vote in votes:
if vote.subcommitteeIdx != subcommitteeIdx:
continue
@ -150,21 +189,24 @@ func computeAggregateSig(votes: seq[TrustedSyncCommitteeMsg],
if initialized:
contribution.signature = aggregateSig.finish.toValidatorSig
else:
contribution.signature = ValidatorSig.infinity
initialized
func produceContribution*(
pool: SyncCommitteeMsgPool,
slot: Slot,
headRoot: Eth2Digest,
headBid: BlockId,
subcommitteeIdx: SyncSubcommitteeIndex,
outContribution: var SyncCommitteeContribution): bool =
if headRoot in pool.syncMessages:
let target = pool.cfg.toSyncMsgTarget(headBid, slot)
if target in pool.syncMessages:
outContribution.slot = slot
outContribution.beacon_block_root = headRoot
outContribution.beacon_block_root = headBid.root
outContribution.subcommittee_index = subcommitteeIdx.asUInt64
try:
computeAggregateSig(pool.syncMessages[headRoot],
computeAggregateSig(pool.syncMessages[target],
subcommitteeIdx,
outContribution)
except KeyError:
@ -203,11 +245,13 @@ func covers(
func covers*(
pool: var SyncCommitteeMsgPool,
contribution: SyncCommitteeContribution): bool =
contribution: SyncCommitteeContribution,
bid: BlockId): bool =
## Return true iff the given contribution brings no new information compared
## to the contributions already seen in the pool, ie if the contriubution is a
## subset of the best contribution so far
pool.bestContributions.withValue(contribution.beacon_block_root, best):
let target = pool.cfg.toSyncMsgTarget(bid, contribution.slot)
pool.bestContributions.withValue(target, best):
return best[].covers(contribution)
return false
@ -215,6 +259,7 @@ func covers*(
proc addContribution(pool: var SyncCommitteeMsgPool,
aggregator_index: uint64,
contribution: SyncCommitteeContribution,
bid: BlockId,
signature: CookedSig) =
let seenKey = SyncCommitteeMsgKey(
originator: aggregator_index,
@ -222,12 +267,10 @@ proc addContribution(pool: var SyncCommitteeMsgPool,
subcommitteeIdx: contribution.subcommittee_index)
pool.seenContributionByAuthor.incl seenKey
template blockRoot: auto = contribution.beacon_block_root
if blockRoot notin pool.bestContributions:
let target = pool.cfg.toSyncMsgTarget(bid, contribution.slot)
if target notin pool.bestContributions:
let totalParticipants = countOnes(contribution.aggregation_bits)
var initialBestContributions = BestSyncSubcommitteeContributions(
slot: contribution.slot)
var initialBestContributions = BestSyncSubcommitteeContributions()
initialBestContributions.subnets[contribution.subcommittee_index] =
BestSyncSubcommitteeContribution(
@ -235,59 +278,66 @@ proc addContribution(pool: var SyncCommitteeMsgPool,
participationBits: contribution.aggregation_bits,
signature: signature)
pool.bestContributions[blockRoot] = initialBestContributions
pool.bestContributions[target] = initialBestContributions
else:
try:
addAggregateAux(pool.bestContributions[blockRoot], contribution)
addAggregateAux(pool.bestContributions[target], contribution)
except KeyError:
raiseAssert "We have checked for the key upfront"
proc addContribution*(pool: var SyncCommitteeMsgPool,
scproof: SignedContributionAndProof,
bid: BlockId,
signature: CookedSig) =
pool.addContribution(
scproof.message.aggregator_index, scproof.message.contribution, signature)
scproof.message.aggregator_index,
scproof.message.contribution,
bid, signature)
if not(isNil(pool.onContributionReceived)):
pool.onContributionReceived(scproof)
proc produceSyncAggregateAux(
bestContributions: BestSyncSubcommitteeContributions): SyncAggregate =
contributions: BestSyncSubcommitteeContributions): SyncAggregate =
var
aggregateSig {.noinit.}: AggregateSignature
initialized = false
startTime = Moment.now
aggregate: SyncAggregate
for subcommitteeIdx in SyncSubcommitteeIndex:
if bestContributions.subnets[subcommitteeIdx].totalParticipants == 0:
if contributions.subnets[subcommitteeIdx].totalParticipants == 0:
continue
for pos, value in bestContributions.subnets[subcommitteeIdx].participationBits:
for pos, value in contributions.subnets[subcommitteeIdx].participationBits:
if value:
let globalPos = subcommitteeIdx.asInt * SYNC_SUBCOMMITTEE_SIZE + pos
result.sync_committee_bits.setBit globalPos
aggregate.sync_committee_bits.setBit globalPos
if not initialized:
initialized = true
aggregateSig.init(bestContributions.subnets[subcommitteeIdx].signature)
aggregateSig.init(contributions.subnets[subcommitteeIdx].signature)
else:
aggregateSig.aggregate(bestContributions.subnets[subcommitteeIdx].signature)
aggregateSig.aggregate(contributions.subnets[subcommitteeIdx].signature)
if initialized:
result.sync_committee_signature = aggregateSig.finish.toValidatorSig
aggregate.sync_committee_signature = aggregateSig.finish.toValidatorSig
else:
result.sync_committee_signature = ValidatorSig.infinity
aggregate.sync_committee_signature = ValidatorSig.infinity
let duration = Moment.now - startTime
debug "SyncAggregate produced", duration,
bits = result.sync_committee_bits
bits = aggregate.sync_committee_bits
aggregate
proc produceSyncAggregate*(
pool: SyncCommitteeMsgPool,
targetRoot: Eth2Digest): SyncAggregate =
if targetRoot in pool.bestContributions:
bid: BlockId,
slot: Slot): SyncAggregate =
let target = pool.cfg.toSyncMsgTarget(bid, slot)
if target in pool.bestContributions:
try:
produceSyncAggregateAux(pool.bestContributions[targetRoot])
produceSyncAggregateAux(pool.bestContributions[target])
except KeyError:
raiseAssert "We have checked for the key upfront"
else:

View File

@ -587,15 +587,15 @@ proc processSyncCommitteeMessage*(
# Now proceed to validation
let v = await validateSyncCommitteeMessage(
self.dag, self.batchCrypto, self.syncCommitteeMsgPool,
self.dag, self.quarantine, self.batchCrypto, self.syncCommitteeMsgPool,
syncCommitteeMsg, subcommitteeIdx, wallTime, checkSignature)
return if v.isOk():
trace "Sync committee message validated"
let (positions, cookedSig) = v.get()
let (bid, cookedSig, positions) = v.get()
self.syncCommitteeMsgPool[].addSyncCommitteeMessage(
syncCommitteeMsg.slot,
syncCommitteeMsg.beacon_block_root,
bid,
syncCommitteeMsg.validator_index,
cookedSig,
subcommitteeIdx,
@ -633,16 +633,19 @@ proc processSignedContributionAndProof*(
# Now proceed to validation
let v = await validateContribution(
self.dag, self.batchCrypto, self.syncCommitteeMsgPool,
self.dag, self.quarantine, self.batchCrypto, self.syncCommitteeMsgPool,
contributionAndProof, wallTime, checkSignature)
return if v.isOk():
trace "Contribution validated"
let (bid, sig, participants) = v.get
self.syncCommitteeMsgPool[].addContribution(
contributionAndProof, v.get()[0])
contributionAndProof, bid, sig)
self.validatorMonitor[].registerSyncContribution(
src, wallTime, contributionAndProof.message, v.get()[1])
src, wallTime, contributionAndProof.message, participants)
beacon_sync_committee_contributions_received.inc()

View File

@ -1024,13 +1024,15 @@ proc validateVoluntaryExit*(
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/altair/p2p-interface.md#sync_committee_subnet_id
proc validateSyncCommitteeMessage*(
dag: ChainDAGRef,
quarantine: ref Quarantine,
batchCrypto: ref BatchCrypto,
syncCommitteeMsgPool: ref SyncCommitteeMsgPool,
msg: SyncCommitteeMessage,
subcommitteeIdx: SyncSubcommitteeIndex,
wallTime: BeaconTime,
checkSignature: bool):
Future[Result[(seq[uint64], CookedSig), ValidationError]] {.async.} =
Future[Result[
(BlockId, CookedSig, seq[uint64]), ValidationError]] {.async.} =
block:
# [IGNORE] The message's slot is for the current slot (with a
# `MAXIMUM_GOSSIP_CLOCK_DISPARITY` allowance), i.e.
@ -1051,6 +1053,19 @@ proc validateSyncCommitteeMessage*(
return dag.checkedReject(
"SyncCommitteeMessage: originator not part of sync committee")
# [IGNORE] The block being signed (`sync_committee_message.beacon_block_root`)
# has been seen (via both gossip and non-gossip sources) (a client MAY queue
# sync committee messages for processing once block is received)
# [REJECT] The block being signed (`sync_committee_message.beacon_block_root`)
# passes validation.
let
blockRoot = msg.beacon_block_root
blck = dag.getBlockRef(blockRoot).valueOr:
if blockRoot in quarantine[].unviable:
return dag.checkedReject("SyncCommitteeMessage: target invalid")
quarantine[].addMissing(blockRoot)
return errIgnore("SyncCommitteeMessage: target not found")
block:
# [IGNORE] There has been no other valid sync committee message for the
# declared `slot` for the validator referenced by
@ -1059,14 +1074,12 @@ proc validateSyncCommitteeMessage*(
# Note this validation is per topic so that for a given slot, multiple
# messages could be forwarded with the same validator_index as long as
# the subnet_ids are distinct.
if syncCommitteeMsgPool[].isSeen(msg, subcommitteeIdx):
if syncCommitteeMsgPool[].isSeen(msg, subcommitteeIdx, dag.head.bid):
return errIgnore("SyncCommitteeMessage: duplicate message")
# [REJECT] The signature is valid for the message beacon_block_root for the
# validator referenced by validator_index.
let
epoch = msg.slot.epoch
fork = dag.forkAtEpoch(epoch)
senderPubKey = dag.validatorKey(msg.validator_index).valueOr:
return dag.checkedReject("SyncCommitteeMessage: invalid validator index")
@ -1075,7 +1088,8 @@ proc validateSyncCommitteeMessage*(
# Attestation signatures are batch-verified
let deferredCrypto = batchCrypto
.scheduleSyncCommitteeMessageCheck(
fork, msg.slot, msg.beacon_block_root,
dag.forkAtEpoch(msg.slot.epoch),
msg.slot, msg.beacon_block_root,
senderPubKey, msg.signature)
if deferredCrypto.isErr():
return dag.checkedReject(deferredCrypto.error)
@ -1098,17 +1112,19 @@ proc validateSyncCommitteeMessage*(
return dag.checkedReject(
"SyncCommitteeMessage: unable to load signature")
return ok((positionsInSubcommittee, sig))
return ok((blck.bid, sig, positionsInSubcommittee))
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/altair/p2p-interface.md#sync_committee_contribution_and_proof
proc validateContribution*(
dag: ChainDAGRef,
quarantine: ref Quarantine,
batchCrypto: ref BatchCrypto,
syncCommitteeMsgPool: ref SyncCommitteeMsgPool,
msg: SignedContributionAndProof,
wallTime: BeaconTime,
checkSignature: bool
): Future[Result[(CookedSig, seq[ValidatorIndex]), ValidationError]] {.async.} =
): Future[Result[
(BlockId, CookedSig, seq[ValidatorIndex]), ValidationError]] {.async.} =
let
syncCommitteeSlot = msg.message.contribution.slot
@ -1124,16 +1140,19 @@ proc validateContribution*(
# i.e. contribution.subcommittee_index < SYNC_COMMITTEE_SUBNET_COUNT.
let subcommitteeIdx = SyncSubcommitteeIndex.init(
msg.message.contribution.subcommittee_index).valueOr:
return dag.checkedReject(
"SignedContributionAndProof: subcommittee index too high")
return dag.checkedReject("Contribution: subcommittee index too high")
# [REJECT] The contribution has participants
# that is, any(contribution.aggregation_bits).
if msg.message.contribution.aggregation_bits.isZeros:
return dag.checkedReject("Contribution: aggregation bits empty")
# [REJECT] contribution_and_proof.selection_proof selects the validator
# as an aggregator for the slot
# i.e. is_sync_committee_aggregator(contribution_and_proof.selection_proof)
# returns True.
if not is_sync_committee_aggregator(msg.message.selection_proof):
return dag.checkedReject(
"SignedContributionAndProof: invalid selection_proof")
return dag.checkedReject("Contribution: invalid selection_proof")
# [IGNORE] The sync committee contribution is the first valid
# contribution received for the aggregator with index
@ -1142,37 +1161,48 @@ proc validateContribution*(
# (this requires maintaining a cache of size SYNC_COMMITTEE_SIZE for this
# topic that can be flushed after each slot).
if syncCommitteeMsgPool[].isSeen(msg.message):
return errIgnore("SignedContributionAndProof: duplicate contribution")
return errIgnore("Contribution: duplicate contribution")
# [REJECT] The aggregator's validator index is in the declared subcommittee
# of the current sync committee.
# i.e. state.validators[contribution_and_proof.aggregator_index].pubkey in
# get_sync_subcommittee_pubkeys(state, contribution.subcommittee_index).
let
epoch = msg.message.contribution.slot.epoch
fork = dag.forkAtEpoch(epoch)
aggregator_index =
ValidatorIndex.init(msg.message.aggregator_index).valueOr:
return dag.checkedReject("Contribution: invalid aggregator index")
# TODO we take a copy of the participants to avoid the data going stale
# between validation and use - nonetheless, a design that avoids it and
# stays safe would be nice
participants = dag.syncCommitteeParticipants(
msg.message.contribution.slot, subcommitteeIdx)
if aggregator_index notin participants:
return dag.checkedReject("Contribution: aggregator not in subcommittee")
if msg.message.contribution.aggregation_bits.countOnes() == 0:
# [REJECT] The contribution has participants
# that is, any(contribution.aggregation_bits).
return dag.checkedReject(
"SignedContributionAndProof: aggregation bits empty")
# [IGNORE] The block being signed
# (`contribution_and_proof.contribution.beacon_block_root`) has been seen
# (via both gossip and non-gossip sources) (a client MAY queue sync committee
# contributions for processing once block is received)
# [REJECT] The block being signed
# (`contribution_and_proof.contribution.beacon_block_root`) passes validation.
let
blockRoot = msg.message.contribution.beacon_block_root
blck = dag.getBlockRef(blockRoot).valueOr:
if blockRoot in quarantine[].unviable:
return dag.checkedReject("Contribution: target invalid")
quarantine[].addMissing(blockRoot)
return errIgnore("Contribution: target not found")
# [IGNORE] A valid sync committee contribution with equal `slot`,
# `beacon_block_root` and `subcommittee_index` whose `aggregation_bits`
# is non-strict superset has _not_ already been seen.
if syncCommitteeMsgPool[].covers(msg.message.contribution):
return errIgnore("SignedContributionAndProof: duplicate contribution")
# TODO we take a copy of the participants to avoid the data going stale
# between validation and use - nonetheless, a design that avoids it and
# stays safe would be nice
let participants = dag.syncCommitteeParticipants(
msg.message.contribution.slot, subcommitteeIdx)
if syncCommitteeMsgPool[].covers(msg.message.contribution, blck.bid):
return errIgnore("Contribution: duplicate contribution")
let sig = if checkSignature:
let deferredCrypto = batchCrypto.scheduleContributionChecks(
fork, msg, subcommitteeIdx, dag)
dag.forkAtEpoch(msg.message.contribution.slot.epoch),
msg, subcommitteeIdx, dag)
if deferredCrypto.isErr():
return dag.checkedReject(deferredCrypto.error)
@ -1186,11 +1216,11 @@ proc validateContribution*(
case x
of BatchResult.Invalid:
return dag.checkedReject(
"SignedContributionAndProof: invalid aggregator signature")
"Contribution: invalid aggregator signature")
of BatchResult.Timeout:
beacon_contributions_dropped_queue_full.inc()
return errIgnore(
"SignedContributionAndProof: timeout checking aggregator signature")
"Contribution: timeout checking aggregator signature")
of BatchResult.Valid:
discard
@ -1202,10 +1232,10 @@ proc validateContribution*(
let x = await proofFut
case x
of BatchResult.Invalid:
return dag.checkedReject("SignedContributionAndProof: invalid proof")
return dag.checkedReject("Contribution: invalid proof")
of BatchResult.Timeout:
beacon_contributions_dropped_queue_full.inc()
return errIgnore("SignedContributionAndProof: timeout checking proof")
return errIgnore("Contribution: timeout checking proof")
of BatchResult.Valid:
discard
@ -1217,12 +1247,12 @@ proc validateContribution*(
let x = await contributionFut
case x
of BatchResult.Invalid:
return errReject( # TODO Triggers in local tests around fork transition
"SignedContributionAndProof: invalid contribution signature")
return dag.checkedReject(
"Contribution: invalid contribution signature")
of BatchResult.Timeout:
beacon_contributions_dropped_queue_full.inc()
return errIgnore(
"SignedContributionAndProof: timeout checking contribution signature")
"Contribution: timeout checking contribution signature")
of BatchResult.Valid:
discard
sig
@ -1230,7 +1260,7 @@ proc validateContribution*(
msg.message.contribution.signature.load().valueOr:
return dag.checkedReject("SyncCommitteeMessage: unable to load signature")
return ok((sig, participants))
return ok((blck.bid, sig, participants))
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/altair/light-client/p2p-interface.md#light_client_finality_update
proc validateLightClientFinalityUpdate*(

View File

@ -312,7 +312,7 @@ proc initFullNode(
attestationPool = newClone(
AttestationPool.init(dag, quarantine, onAttestationReceived))
syncCommitteeMsgPool = newClone(
SyncCommitteeMsgPool.init(rng, onSyncContribution))
SyncCommitteeMsgPool.init(rng, dag.cfg, onSyncContribution))
lightClientPool = newClone(
LightClientPool())
validatorChangePool = newClone(

View File

@ -837,8 +837,11 @@ proc installValidatorApiHandlers*(router: var RestRouter, node: BeaconNode) =
return RestApiResponse.jsonError(Http503, BeaconNodeInSyncError)
var contribution = SyncCommitteeContribution()
let res = node.syncCommitteeMsgPool[].produceContribution(
qslot, qroot, qindex, contribution)
let
blck = node.dag.getBlockRef(qroot).valueOr:
return RestApiResponse.jsonError(Http404, BlockNotFoundError)
res = node.syncCommitteeMsgPool[].produceContribution(
qslot, blck.bid, qindex, contribution)
if not(res):
return RestApiResponse.jsonError(Http400, ProduceContributionError)
return RestApiResponse.jsonResponse(contribution)

View File

@ -388,7 +388,7 @@ proc collectSignatureSets*(
# ----------------------------------------------------
withState(state):
when consensusFork >= ConsensusFork.Altair:
if signed_block.message.body.sync_aggregate.sync_committee_bits.countOnes() == 0:
if signed_block.message.body.sync_aggregate.sync_committee_bits.isZeros:
if signed_block.message.body.sync_aggregate.sync_committee_signature != ValidatorSig.infinity():
return err("collectSignatureSets: empty sync aggregates need signature of point at infinity")
else:

View File

@ -408,10 +408,10 @@ proc makeBeaconBlockForHeadAndSlot*(
node.validatorChangePool[].getBeaconBlockValidatorChanges(
node.dag.cfg, forkyState.data)
syncAggregate =
if slot.epoch < node.dag.cfg.ALTAIR_FORK_EPOCH:
SyncAggregate.init()
if slot.epoch >= node.dag.cfg.ALTAIR_FORK_EPOCH:
node.syncCommitteeMsgPool[].produceSyncAggregate(head.bid, slot)
else:
node.syncCommitteeMsgPool[].produceSyncAggregate(head.root)
SyncAggregate.init()
payload = (await payloadFut).valueOr:
beacon_block_production_errors.inc()
warn "Unable to get execution payload. Skipping block proposal",
@ -1190,7 +1190,7 @@ proc signAndSendContribution(node: BeaconNode,
if not node.syncCommitteeMsgPool[].produceContribution(
slot,
head.root,
head.bid,
subcommitteeIdx,
msg.message.contribution):
return

View File

@ -325,7 +325,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
batchCrypto = BatchCrypto.new(
keys.newRng(), eager = func(): bool = true,
genesis_validators_root = dag.genesis_validators_root, taskpool)
syncCommitteePool = newClone SyncCommitteeMsgPool.init(keys.newRng())
syncCommitteePool = newClone SyncCommitteeMsgPool.init(keys.newRng(), cfg)
timers: array[Timers, RunningStat]
attesters: RunningStat
r = initRand(1)
@ -394,7 +394,8 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
let
validatorPrivKey = MockPrivKeys[validatorIdx]
signature = get_sync_committee_message_signature(
fork, genesis_validators_root, slot, dag.head.root, validatorPrivKey)
fork, genesis_validators_root,
slot, dag.head.root, validatorPrivKey)
msg = SyncCommitteeMessage(
slot: slot,
beacon_block_root: dag.head.root,
@ -402,6 +403,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
signature: signature.toValidatorSig)
let res = waitFor dag.validateSyncCommitteeMessage(
quarantine,
batchCrypto,
syncCommitteePool,
msg,
@ -411,11 +413,11 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
doAssert res.isOk
let (positions, cookedSig) = res.get()
let (bid, cookedSig, positions) = res.get()
syncCommitteePool[].addSyncCommitteeMessage(
msg.slot,
msg.beacon_block_root,
bid,
msg.validator_index,
cookedSig,
subcommitteeIdx,
@ -435,7 +437,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
for aggregator in aggregators:
var contribution: SyncCommitteeContribution
let contributionWasProduced = syncCommitteePool[].produceContribution(
slot, dag.head.root, aggregator.subcommitteeIdx, contribution)
slot, dag.head.bid, aggregator.subcommitteeIdx, contribution)
if contributionWasProduced:
let
@ -454,19 +456,20 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
validatorPrivKey).toValidatorSig)
res = waitFor dag.validateContribution(
quarantine,
batchCrypto,
syncCommitteePool,
signedContributionAndProof,
contributionsTime,
false)
if res.isOk():
let (bid, sig, _) = res.get
syncCommitteePool[].addContribution(
signedContributionAndProof, res.get()[0])
signedContributionAndProof, bid, sig)
else:
# We ignore duplicates / already-covered contributions
doAssert res.error()[0] == ValidationResult.Ignore
proc getNewBlock[T](
state: var ForkedHashedBeaconState, slot: Slot, cache: var StateCache): T =
let
@ -479,13 +482,10 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
finalizedEpochRef.eth1_data,
finalizedEpochRef.eth1_deposit_index)
sync_aggregate =
when T is phase0.SignedBeaconBlock:
SyncAggregate.init()
elif T is altair.SignedBeaconBlock or T is bellatrix.SignedBeaconBlock or
T is capella.SignedBeaconBlock or T is deneb.SignedBeaconBlock:
syncCommitteePool[].produceSyncAggregate(dag.head.root)
when T.toFork >= ConsensusFork.Altair:
syncCommitteePool[].produceSyncAggregate(dag.head.bid, slot)
else:
static: doAssert false
SyncAggregate.init()
hashedState =
when T is phase0.SignedBeaconBlock:
addr state.phase0Data

View File

@ -1,5 +1,5 @@
# beacon_chain
# Copyright (c) 2019-2022 Status Research & Development GmbH
# Copyright (c) 2019-2023 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
@ -129,7 +129,7 @@ cli do(slots = SLOTS_PER_EPOCH * 5,
if (rand(r, high(int)).float * attesterRatio).int <= high(int):
attestation.aggregation_bits.setBit index_in_committee
if attestation.aggregation_bits.countOnes() > 0:
if not attestation.aggregation_bits.isZeros:
if validate:
attestation.signature = makeAttestationSig(
forkyState.data.fork, genesis_validators_root,

View File

@ -240,13 +240,13 @@ cli do(validatorsDir: string, secretsDir: string,
signature = get_attestation_signature(
fork, genesis_validators_root, attestation.data,
validators[validator_index])
if attestation.aggregation_bits.countOnes() == 0:
if attestation.aggregation_bits.isZeros:
agg = AggregateSignature.init(signature)
else:
agg.aggregate(signature)
attestation.aggregation_bits.setBit(index_in_committee)
if attestation.aggregation_bits.countOnes() > 0:
if not attestation.aggregation_bits.isZeros:
attestation.signature = agg.finish().toValidatorSig()
if aggregates.len == 128:

View File

@ -226,7 +226,6 @@ suite "Gossip validation - Extra": # Not based on preset config
var
state = assignClone(dag.headState.altairData)
slot = state[].data.slot
beaconBlockRoot = state[].latest_block_root
subcommitteeIdx = 0.SyncSubcommitteeIndex
syncCommittee = @(dag.syncCommitteeParticipants(slot))
@ -241,47 +240,50 @@ suite "Gossip validation - Extra": # Not based on preset config
kind: ValidatorKind.Local, data: keystoreData, index: Opt.some index)
resMsg = waitFor getSyncCommitteeMessage(
validator, state[].data.fork, state[].data.genesis_validators_root,
slot, beaconBlockRoot)
slot, state[].latest_block_root)
msg = resMsg.get()
syncCommitteeMsgPool = newClone(SyncCommitteeMsgPool.init(keys.newRng()))
syncCommitteePool = newClone(
SyncCommitteeMsgPool.init(keys.newRng(), cfg))
res = waitFor validateSyncCommitteeMessage(
dag, batchCrypto, syncCommitteeMsgPool, msg, subcommitteeIdx,
slot.start_beacon_time(), true)
(positions, cookedSig) = res.get()
dag, quarantine, batchCrypto, syncCommitteePool,
msg, subcommitteeIdx, slot.start_beacon_time(), true)
(bid, cookedSig, positions) = res.get()
syncCommitteeMsgPool[].addSyncCommitteeMessage(
syncCommitteePool[].addSyncCommitteeMessage(
msg.slot,
msg.beacon_block_root,
bid,
msg.validator_index,
cookedSig,
subcommitteeIdx,
positions)
let
contribution = block:
let contribution = (ref SignedContributionAndProof)()
contrib = block:
let contrib = (ref SignedContributionAndProof)()
check:
syncCommitteeMsgPool[].produceContribution(
slot, beaconBlockRoot, subcommitteeIdx,
contribution.message.contribution)
syncCommitteeMsgPool[].addContribution(
contribution[], contribution.message.contribution.signature.load.get)
syncCommitteePool[].produceContribution(
slot, bid, subcommitteeIdx,
contrib.message.contribution)
syncCommitteePool[].addContribution(
contrib[], bid,
contrib.message.contribution.signature.load.get)
let signRes = waitFor validator.getContributionAndProofSignature(
state[].data.fork, state[].data.genesis_validators_root,
contribution[].message)
contrib[].message)
doAssert(signRes.isOk())
contribution[].signature = signRes.get()
contribution
aggregate = syncCommitteeMsgPool[].produceSyncAggregate(beaconBlockRoot)
contrib[].signature = signRes.get()
contrib
aggregate = syncCommitteePool[].produceSyncAggregate(bid, slot)
check:
expectedCount > 1 # Cover edge case
res.isOk
contribution.message.contribution.aggregation_bits.countOnes == expectedCount
contrib.message.contribution.aggregation_bits.countOnes == expectedCount
aggregate.sync_committee_bits.countOnes == expectedCount
# Same message twice should be ignored
validateSyncCommitteeMessage(
dag, batchCrypto, syncCommitteeMsgPool, msg, subcommitteeIdx,
state[].data.slot.start_beacon_time(), true).waitFor().isErr()
dag, quarantine, batchCrypto, syncCommitteePool,
msg, subcommitteeIdx, state[].data.slot.start_beacon_time(), true
).waitFor().isErr()

View File

@ -1,3 +1,10 @@
# beacon_chain
# Copyright (c) 2021-2023 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.used.}
import
@ -16,21 +23,27 @@ func aggregate(sigs: openArray[CookedSig]): CookedSig =
suite "Sync committee pool":
setup:
var pool = SyncCommitteeMsgPool.init(keys.newRng())
let cfg = block:
var res = defaultRuntimeConfig
res.ALTAIR_FORK_EPOCH = 0.Epoch
res.BELLATRIX_FORK_EPOCH = 20.Epoch
res
var pool = SyncCommitteeMsgPool.init(keys.newRng(), cfg)
test "An empty pool is safe to use":
let headRoot = eth2digest(@[1.byte, 2, 3])
let headBid =
BlockId(slot: Slot(1), root: eth2digest(@[1.byte, 2, 3]))
var outContribution: SyncCommitteeContribution
let success = pool.produceContribution(
Slot(1),
headRoot,
headBid,
SyncSubcommitteeIndex(0),
outContribution)
check(success == false)
let aggregate = pool.produceSyncAggregate(headRoot)
let aggregate = pool.produceSyncAggregate(headBid, headBid.slot)
check:
aggregate.sync_committee_bits.isZeros
@ -42,9 +55,124 @@ suite "Sync committee pool":
test "An empty pool is safe to prune 2":
pool.pruneData(Slot(10000))
test "Missed slots across sync committee period boundary":
let
genesis_validators_root = eth2digest(@[5.byte, 6, 7])
privkey1 = MockPrivKeys[1.ValidatorIndex]
privkey2 = MockPrivKeys[2.ValidatorIndex]
nextPeriod = cfg.BELLATRIX_FORK_EPOCH.sync_committee_period + 1
bid1 = BlockId(
slot: Slot(nextPeriod.start_slot - 2), # Committee based on `slot + 1`
root: eth2digest(@[1.byte]))
sig1 = get_sync_committee_message_signature(
bellatrixFork(cfg), genesis_validators_root,
bid1.slot, bid1.root, privkey1)
sig2 = get_sync_committee_message_signature(
bellatrixFork(cfg), genesis_validators_root,
bid1.slot + 1, bid1.root, privkey2)
pool.addSyncCommitteeMessage(
bid1.slot, bid1, 1, sig1, SyncSubcommitteeIndex(0), @[1'u64])
pool.addSyncCommitteeMessage(
# Same participant index in next period represents different validator
bid1.slot + 1, bid1, 2, sig2, SyncSubcommitteeIndex(0), @[1'u64])
var contribution: SyncCommitteeContribution
let success = pool.produceContribution(
bid1.slot + 1, bid1, SyncSubcommitteeIndex(0), contribution)
check:
success
contribution.slot == bid1.slot + 1
contribution.beacon_block_root == bid1.root
contribution.subcommittee_index == SyncSubcommitteeIndex(0).uint64
contribution.aggregation_bits.countOnes == 1
contribution.aggregation_bits[1] == true
contribution.signature == sig2.toValidatorSig
test "Missed slots across fork transition":
let
genesis_validators_root = eth2digest(@[5.byte, 6, 7])
privkey1 = MockPrivKeys[1.ValidatorIndex]
privkey2 = MockPrivKeys[1.ValidatorIndex]
bid1 = BlockId(
slot: Slot(cfg.BELLATRIX_FORK_EPOCH.start_slot - 1),
root: eth2digest(@[1.byte]))
sig1 = get_sync_committee_message_signature(
altairFork(cfg), genesis_validators_root,
bid1.slot, bid1.root, privkey1)
sig2 = get_sync_committee_message_signature(
bellatrixFork(cfg), genesis_validators_root,
bid1.slot + 1, bid1.root, privkey2)
pool.addSyncCommitteeMessage(
bid1.slot, bid1, 1, sig1, SyncSubcommitteeIndex(0), @[1'u64])
pool.addSyncCommitteeMessage(
bid1.slot + 1, bid1, 2, sig2, SyncSubcommitteeIndex(0), @[2'u64])
var contribution: SyncCommitteeContribution
let success = pool.produceContribution(
bid1.slot + 1, bid1, SyncSubcommitteeIndex(0), contribution)
check:
success
contribution.slot == bid1.slot + 1
contribution.beacon_block_root == bid1.root
contribution.subcommittee_index == SyncSubcommitteeIndex(0).uint64
contribution.aggregation_bits.countOnes == 1
contribution.aggregation_bits[2] == true
contribution.signature == sig2.toValidatorSig
test "isSeen":
let
fork = altairFork(cfg)
genesis_validators_root = eth2digest(@[5.byte, 6, 7])
privkey1 = MockPrivKeys[1.ValidatorIndex]
bid1 = BlockId(slot: Slot(100), root: eth2digest(@[1.byte]))
bid2 = BlockId(slot: Slot(101), root: eth2digest(@[1.byte, 2]))
sig1 = get_sync_committee_message_signature(
fork, genesis_validators_root, bid2.slot, bid1.root, privkey1)
sig2 = get_sync_committee_message_signature(
fork, genesis_validators_root, bid2.slot, bid2.root, privkey1)
msg1 = SyncCommitteeMessage(
slot: bid2.slot,
beacon_block_root: bid1.root,
validator_index: 1,
signature: sig1.toValidatorSig)
msg2 = SyncCommitteeMessage(
slot: bid2.slot,
beacon_block_root: bid2.root,
validator_index: 1,
signature: sig2.toValidatorSig)
check:
not pool.isSeen(msg1, SyncSubcommitteeIndex(0), bid2)
not pool.isSeen(msg2, SyncSubcommitteeIndex(0), bid2)
pool.addSyncCommitteeMessage(
bid2.slot, bid1, 1, sig1, SyncSubcommitteeIndex(0), @[1'u64])
check:
pool.isSeen(msg1, SyncSubcommitteeIndex(0), bid2)
not pool.isSeen(msg2, SyncSubcommitteeIndex(0), bid2)
pool.addSyncCommitteeMessage(
bid2.slot, bid2, 1, sig1, SyncSubcommitteeIndex(0), @[1'u64])
check:
pool.isSeen(msg1, SyncSubcommitteeIndex(0), bid2)
pool.isSeen(msg2, SyncSubcommitteeIndex(0), bid2)
test "Aggregating votes":
let
fork = altairFork(defaultRuntimeConfig)
fork = altairFork(cfg)
genesis_validators_root = eth2digest(@[5.byte, 6, 7])
privkey1 = MockPrivKeys[1.ValidatorIndex]
@ -52,35 +180,36 @@ suite "Sync committee pool":
privkey3 = MockPrivKeys[3.ValidatorIndex]
privkey4 = MockPrivKeys[4.ValidatorIndex]
root1 = eth2digest(@[1.byte])
root2 = eth2digest(@[1.byte, 2])
root3 = eth2digest(@[1.byte, 2, 3])
root1Slot = Slot(100)
root2Slot = Slot(101)
root3Slot = Slot(101)
bid1 = BlockId(slot: Slot(100), root: eth2digest(@[1.byte]))
bid2 = BlockId(slot: Slot(101), root: eth2digest(@[1.byte, 2]))
bid3 = BlockId(slot: Slot(101), root: eth2digest(@[1.byte, 2, 3]))
subcommittee1 = SyncSubcommitteeIndex(0)
subcommittee2 = SyncSubcommitteeIndex(1)
sig1 = get_sync_committee_message_signature(
fork, genesis_validators_root, root1Slot, root1, privkey1)
fork, genesis_validators_root, bid1.slot, bid1.root, privkey1)
sig2 = get_sync_committee_message_signature(
fork, genesis_validators_root, root2Slot, root2, privkey1)
fork, genesis_validators_root, bid2.slot, bid2.root, privkey1)
sig3 = get_sync_committee_message_signature(
fork, genesis_validators_root, root3Slot, root3, privkey1)
fork, genesis_validators_root, bid3.slot, bid3.root, privkey1)
sig4 = get_sync_committee_message_signature(
fork, genesis_validators_root, root3Slot, root2, privkey1)
fork, genesis_validators_root, bid3.slot, bid2.root, privkey1)
# Inserting sync committee messages
#
pool.addSyncCommitteeMessage(root1Slot, root1, 1, sig1, subcommittee1, [1'u64])
pool.addSyncCommitteeMessage(root1Slot, root1, 2, sig2, subcommittee1, [10'u64])
pool.addSyncCommitteeMessage(root2Slot, root1, 3, sig3, subcommittee2, [7'u64])
pool.addSyncCommitteeMessage(root2Slot, root2, 4, sig4, subcommittee2, [3'u64])
pool.addSyncCommitteeMessage(
bid1.slot, bid1, 1, sig1, subcommittee1, @[1'u64])
pool.addSyncCommitteeMessage(
bid1.slot, bid1, 2, sig2, subcommittee1, @[10'u64])
pool.addSyncCommitteeMessage(
bid2.slot, bid1, 3, sig3, subcommittee2, @[7'u64])
pool.addSyncCommitteeMessage(
bid2.slot, bid2, 4, sig4, subcommittee2, @[3'u64])
# Insert a duplicate message (this should be handled gracefully)
pool.addSyncCommitteeMessage(root1Slot, root1, 1, sig1, subcommittee1, [1'u64])
pool.addSyncCommitteeMessage(
bid1.slot, bid1, 1, sig1, subcommittee1, @[1'u64])
# Producing contributions
#
@ -88,8 +217,8 @@ suite "Sync committee pool":
# Checking a committee where there was no activity:
var outContribution: SyncCommitteeContribution
let success = pool.produceContribution(
root2Slot,
root2,
bid2.slot,
bid2,
subcommittee1,
outContribution)
@ -101,56 +230,56 @@ suite "Sync committee pool":
var outContribution: SignedContributionAndProof
template contribution: untyped = outContribution.message.contribution
let success = pool.produceContribution(
root1Slot,
root1,
bid1.slot,
bid1,
subcommittee1,
contribution)
let expectedSig = aggregate [sig1, sig2]
let sig = aggregate [sig1, sig2]
check:
success
contribution.slot == root1Slot
contribution.beacon_block_root == root1
contribution.slot == bid1.slot
contribution.beacon_block_root == bid1.root
contribution.subcommittee_index == subcommittee1.uint64
contribution.aggregation_bits.countOnes == 2
contribution.aggregation_bits[1] == true
contribution.aggregation_bits[8] == false
contribution.aggregation_bits[10] == true
contribution.signature == expectedSig.toValidatorSig
contribution.signature == sig.toValidatorSig
check:
not pool.covers(contribution)
not pool.covers(contribution, bid1)
pool.addContribution(outContribution, expectedSig)
pool.addContribution(outContribution, bid1, sig)
check:
pool.isSeen(outContribution.message)
pool.covers(contribution)
pool.covers(contribution, bid1)
block:
# Checking a committee with a signle participant:
var outContribution: SignedContributionAndProof
template contribution: untyped = outContribution.message.contribution
let success = pool.produceContribution(
root1Slot,
root1,
bid1.slot,
bid1,
subcommittee2,
contribution)
check:
success
contribution.slot == root1Slot
contribution.beacon_block_root == root1
contribution.slot == bid1.slot
contribution.beacon_block_root == bid1.root
contribution.subcommittee_index == subcommittee2.uint64
contribution.aggregation_bits.countOnes == 1
contribution.aggregation_bits[7] == true
contribution.signature == sig3.toValidatorSig
check:
not pool.covers(contribution)
pool.addContribution(outContribution, sig3)
not pool.covers(contribution, bid1)
pool.addContribution(outContribution, bid1, sig3)
check:
pool.isSeen(outContribution.message)
pool.covers(contribution)
pool.covers(contribution, bid1)
block:
# Checking another committee with a signle participant
@ -158,35 +287,35 @@ suite "Sync committee pool":
var outContribution: SignedContributionAndProof
template contribution: untyped = outContribution.message.contribution
let success = pool.produceContribution(
root2Slot,
root2,
bid2.slot,
bid2,
subcommittee2,
contribution)
check:
success
contribution.slot == root2Slot
contribution.beacon_block_root == root2
contribution.slot == bid2.slot
contribution.beacon_block_root == bid2.root
contribution.subcommittee_index == subcommittee2.uint64
contribution.aggregation_bits.countOnes == 1
contribution.aggregation_bits[3] == true
contribution.signature == sig4.toValidatorSig
check:
not pool.covers(contribution)
pool.addContribution(outContribution, sig4)
not pool.covers(contribution, bid2)
pool.addContribution(outContribution, bid2, sig4)
check:
pool.isSeen(outContribution.message)
pool.covers(contribution)
pool.covers(contribution, bid2)
block:
# Checking a block root nobody voted for
var outContribution: SignedContributionAndProof
template contribution: untyped = outContribution.message.contribution
let success = pool.produceContribution(
root3Slot,
root3,
bid3.slot,
bid3,
subcommittee2,
contribution)
@ -197,43 +326,43 @@ suite "Sync committee pool":
#
block:
# Checking for a block that got no votes
let aggregate = pool.produceSyncAggregate(root3)
let aggregate = pool.produceSyncAggregate(bid3, bid3.slot)
check:
aggregate.sync_committee_bits.isZeros
aggregate.sync_committee_signature == ValidatorSig.infinity
block:
# Checking for a block that got votes from 1 committee
let aggregate = pool.produceSyncAggregate(root2)
let aggregate = pool.produceSyncAggregate(bid2, bid2.slot)
check:
aggregate.sync_committee_bits.countOnes == 1
aggregate.sync_committee_signature == sig4.toValidatorSig
block:
# Checking for a block that got votes from 2 committees
let aggregate = pool.produceSyncAggregate(root1)
let expectedSig = aggregate [sig1, sig2, sig3]
let aggregate = pool.produceSyncAggregate(bid1, bid1.slot)
let sig = aggregate [sig1, sig2, sig3]
check:
aggregate.sync_committee_bits.countOnes == 3
aggregate.sync_committee_signature == expectedSig.toValidatorSig
aggregate.sync_committee_signature == sig.toValidatorSig
# Pruning the data
#
pool.pruneData(Slot(200))
pool.pruneData(Slot(200), force = true)
block:
# After pruning, all votes are gone
var outContribution: SyncCommitteeContribution
let success = pool.produceContribution(
root1Slot,
root1,
bid1.slot,
bid1,
subcommittee1,
outContribution)
check:
not success
let aggregate = pool.produceSyncAggregate(root2)
let aggregate = pool.produceSyncAggregate(bid2, bid2.slot)
check:
aggregate.sync_committee_bits.isZeros
aggregate.sync_committee_signature == ValidatorSig.infinity

View File

@ -421,9 +421,9 @@ proc makeSyncAggregate(
getStateField(state, genesis_validators_root)
slot =
getStateField(state, slot)
latest_block_root =
withState(state): forkyState.latest_block_root
syncCommitteePool = newClone(SyncCommitteeMsgPool.init(keys.newRng()))
latest_block_id =
withState(state): forkyState.latest_block_id
syncCommitteePool = newClone(SyncCommitteeMsgPool.init(keys.newRng(), cfg))
type
Aggregator = object
@ -484,11 +484,11 @@ proc makeSyncAggregate(
let signature = get_sync_committee_message_signature(
fork, genesis_validators_root,
slot, latest_block_root,
slot, latest_block_id.root,
MockPrivKeys[validatorIdx])
syncCommitteePool[].addSyncCommitteeMessage(
slot,
latest_block_root,
latest_block_id,
uint64 validatorIdx,
signature,
subcommitteeIdx,
@ -497,7 +497,7 @@ proc makeSyncAggregate(
for aggregator in aggregators:
var contribution: SyncCommitteeContribution
if syncCommitteePool[].produceContribution(
slot, latest_block_root, aggregator.subcommitteeIdx, contribution):
slot, latest_block_id, aggregator.subcommitteeIdx, contribution):
let
contributionAndProof = ContributionAndProof(
aggregator_index: uint64 aggregator.validatorIdx,
@ -511,9 +511,10 @@ proc makeSyncAggregate(
message: contributionAndProof,
signature: contributionSig.toValidatorSig)
syncCommitteePool[].addContribution(
signedContributionAndProof, contribution.signature.load.get)
signedContributionAndProof,
latest_block_id, contribution.signature.load.get)
syncCommitteePool[].produceSyncAggregate(latest_block_root)
syncCommitteePool[].produceSyncAggregate(latest_block_id, slot)
iterator makeTestBlocks*(
state: ForkedHashedBeaconState,