batch-verify sync messages for a small perf boost (#3151)

* batch-verify sync messages for a small perf boost

Generally reuses the same structure as attestation and aggregate
verification

* normalize `signatures` and `signature_batch` to use the same pattern
of verification
* normalize parameter names, order etc for signature stuff in general
* avoid calling `blsSign` directly - instead, go through `signatures`
consistently
This commit is contained in:
Jacek Sieka 2021-12-09 13:56:54 +01:00 committed by GitHub
parent d93a279565
commit 069bccd51b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
21 changed files with 887 additions and 820 deletions

View File

@ -1496,3 +1496,59 @@ proc getProposer*(
return none(ValidatorIndex) return none(ValidatorIndex)
proposer proposer
proc aggregateAll*(
dag: ChainDAGRef,
validator_indices: openArray[ValidatorIndex]): Result[CookedPubKey, cstring] =
if validator_indices.len == 0:
# Aggregation spec requires non-empty collection
# - https://tools.ietf.org/html/draft-irtf-cfrg-bls-signature-04
# Eth2 spec requires at least one attesting index in attestation
# - https://github.com/ethereum/consensus-specs/blob/v1.1.5/specs/phase0/beacon-chain.md#is_valid_indexed_attestation
return err("aggregate: no attesting keys")
let
firstKey = dag.validatorKey(validator_indices[0])
if not firstKey.isSome():
return err("aggregate: invalid validator index")
var aggregateKey{.noInit.}: AggregatePublicKey
aggregateKey.init(firstKey.get())
for i in 1 ..< validator_indices.len:
let key = dag.validatorKey(validator_indices[i])
if not key.isSome():
return err("aggregate: invalid validator index")
aggregateKey.aggregate(key.get())
ok(finish(aggregateKey))
proc aggregateAll*(
dag: ChainDAGRef,
validator_indices: openArray[ValidatorIndex|uint64],
bits: BitSeq | BitArray): Result[CookedPubKey, cstring] =
if validator_indices.len() != bits.len():
return err("aggregateAll: mismatch in bits length")
var
aggregateKey{.noInit.}: AggregatePublicKey
inited = false
for i in 0..<bits.len():
if bits[i]:
let key = dag.validatorKey(validator_indices[i])
if not key.isSome():
return err("aggregate: invalid validator index")
if inited:
aggregateKey.aggregate(key.get)
else:
aggregateKey = AggregatePublicKey.init(key.get)
inited = true
if not inited:
err("aggregate: no attesting keys")
else:
ok(finish(aggregateKey))

View File

@ -11,7 +11,7 @@ import
std/[sets, tables], std/[sets, tables],
stew/shims/hashes, stew/shims/hashes,
chronicles, chronicles,
../spec/digest, ../spec/[crypto, digest],
../spec/datatypes/altair ../spec/datatypes/altair
export hashes, sets, tables, altair export hashes, sets, tables, altair
@ -97,7 +97,7 @@ func isSeen*(
subcommitteeIndex: subcommitteeIndex.uint64) subcommitteeIndex: subcommitteeIndex.uint64)
seenKey in pool.seenSyncMsgByAuthor seenKey in pool.seenSyncMsgByAuthor
func addSyncCommitteeMessage*( proc addSyncCommitteeMessage*(
pool: var SyncCommitteeMsgPool, pool: var SyncCommitteeMsgPool,
slot: Slot, slot: Slot,
blockRoot: Eth2Digest, blockRoot: Eth2Digest,
@ -121,6 +121,9 @@ func addSyncCommitteeMessage*(
positionInCommittee: position, positionInCommittee: position,
signature: signature) signature: signature)
debug "Sync committee message resolved",
slot = slot, blockRoot = shortLog(blockRoot), validatorIndex
func computeAggregateSig(votes: seq[TrustedSyncCommitteeMsg], func computeAggregateSig(votes: seq[TrustedSyncCommitteeMsg],
subcommitteeIndex: SyncSubcommitteeIndex, subcommitteeIndex: SyncSubcommitteeIndex,
contribution: var SyncCommitteeContribution): bool = contribution: var SyncCommitteeContribution): bool =

View File

@ -8,6 +8,7 @@
{.push raises: [Defect].} {.push raises: [Defect].}
import import
std/sequtils,
# Status # Status
chronicles, chronos, chronicles, chronos,
../spec/signatures_batch, ../spec/signatures_batch,
@ -188,6 +189,7 @@ proc getBatch(batchCrypto: ref BatchCrypto): (ref Batch, bool) =
# len will be 0 when the batch was created but nothing added to it # len will be 0 when the batch was created but nothing added to it
# because of early failures # because of early failures
(batch, batch[].len() == 0) (batch, batch[].len() == 0)
proc scheduleBatch(batchCrypto: ref BatchCrypto, fresh: bool) = proc scheduleBatch(batchCrypto: ref BatchCrypto, fresh: bool) =
if fresh: if fresh:
# Every time we start a new round of batching, we need to launch a deferred # Every time we start a new round of batching, we need to launch a deferred
@ -201,12 +203,36 @@ proc scheduleBatch(batchCrypto: ref BatchCrypto, fresh: bool) =
# If there's a full batch, process it eagerly assuming the callback allows # If there's a full batch, process it eagerly assuming the callback allows
batchCrypto.processBatch() batchCrypto.processBatch()
template orReturnErr(v: Option, error: cstring): untyped =
## Returns with given error string if the option does not have a value
let tmp = v
if tmp.isNone:
return err(error) # this exits the calling scope, as templates are inlined.
tmp.unsafeGet()
template withBatch(
batchCrypto: ref BatchCrypto, name: cstring,
body: untyped): Future[BatchResult] =
block:
let
(batch {.inject.}, fresh) = batchCrypto.getBatch()
body
let fut = newFuture[BatchResult](name)
batch[].resultsBuffer.add(fut)
batchCrypto.scheduleBatch(fresh)
fut
# See also verify_attestation_signature
proc scheduleAttestationCheck*( proc scheduleAttestationCheck*(
batchCrypto: ref BatchCrypto, batchCrypto: ref BatchCrypto,
fork: Fork, genesis_validators_root: Eth2Digest, fork: Fork, genesis_validators_root: Eth2Digest,
epochRef: EpochRef, attestationData: AttestationData,
attestation: Attestation pubkey: CookedPubKey, signature: ValidatorSig
): Result[(Future[BatchResult], CookedSig), cstring] = ): Result[tuple[fut: Future[BatchResult], sig: CookedSig], cstring] =
## Schedule crypto verification of an attestation ## Schedule crypto verification of an attestation
## ##
## The buffer is processed: ## The buffer is processed:
@ -216,37 +242,23 @@ proc scheduleAttestationCheck*(
## This returns an error if crypto sanity checks failed ## This returns an error if crypto sanity checks failed
## and a future with the deferred attestation check otherwise. ## and a future with the deferred attestation check otherwise.
## ##
let (batch, fresh) = batchCrypto.getBatch() let
sig = signature.load().orReturnErr("attestation: cannot load signature")
fut = batchCrypto.withBatch("batch_validation.scheduleAttestationCheck"):
batch.pendingBuffer.add_attestation_signature(
fork, genesis_validators_root, attestationData, pubkey, sig)
doAssert batch.pendingBuffer.len < BatchedCryptoSize ok((fut, sig))
let sig = ? batch
.pendingBuffer
.addAttestation(
fork, genesis_validators_root, epochRef,
attestation
)
let fut = newFuture[BatchResult](
"batch_validation.scheduleAttestationCheck"
)
batch[].resultsBuffer.add(fut)
batchCrypto.scheduleBatch(fresh)
return ok((fut, sig))
proc scheduleAggregateChecks*( proc scheduleAggregateChecks*(
batchCrypto: ref BatchCrypto, batchCrypto: ref BatchCrypto,
fork: Fork, genesis_validators_root: Eth2Digest, fork: Fork, genesis_validators_root: Eth2Digest,
signedAggregateAndProof: SignedAggregateAndProof,
epochRef: EpochRef, epochRef: EpochRef,
signedAggregateAndProof: SignedAggregateAndProof attesting_indices: openArray[ValidatorIndex]
): Result[ ): Result[tuple[
( aggregatorFut, slotFut, aggregateFut: Future[BatchResult],
tuple[slotCheck, aggregatorCheck, aggregateCheck: Future[BatchResult]], sig: CookedSig], cstring] =
CookedSig
), cstring] =
## Schedule crypto verification of an aggregate ## Schedule crypto verification of an aggregate
## ##
## This involves 3 checks: ## This involves 3 checks:
@ -260,72 +272,111 @@ proc scheduleAggregateChecks*(
## ##
## This returns None if the signatures could not be loaded. ## This returns None if the signatures could not be loaded.
## and 3 futures with the deferred aggregate checks otherwise. ## and 3 futures with the deferred aggregate checks otherwise.
let (batch, fresh) = batchCrypto.getBatch()
doAssert batch[].pendingBuffer.len < BatchedCryptoSize
template aggregate_and_proof: untyped = signedAggregateAndProof.message template aggregate_and_proof: untyped = signedAggregateAndProof.message
template aggregate: untyped = aggregate_and_proof.aggregate template aggregate: untyped = aggregate_and_proof.aggregate
type R = ( # Do the eager steps first to avoid polluting batches with needlessly
tuple[slotCheck, aggregatorCheck, aggregateCheck: let
Future[BatchResult]], aggregatorKey =
CookedSig) epochRef.validatorKey(aggregate_and_proof.aggregator_index).orReturnErr(
"SignedAggregateAndProof: invalid aggregator index")
aggregatorSig = signedAggregateAndProof.signature.load().orReturnErr(
"aggregateAndProof: invalid proof signature")
slotSig = aggregate_and_proof.selection_proof.load().orReturnErr(
"aggregateAndProof: invalid selection signature")
aggregateKey = ? aggregateAll(epochRef.dag, attesting_indices)
aggregateSig = aggregate.signature.load().orReturnErr(
"aggregateAndProof: invalid aggregate signature")
# Enqueue in the buffer let
# ------------------------------------------------------ aggregatorFut = batchCrypto.withBatch("scheduleAggregateChecks.aggregator"):
let aggregator = epochRef.validatorKey(aggregate_and_proof.aggregator_index) batch.pendingBuffer.add_aggregate_and_proof_signature(
if not aggregator.isSome(): fork, genesis_validators_root, aggregate_and_proof, aggregatorKey,
return err("scheduleAggregateChecks: invalid aggregator index") aggregatorSig)
block: slotFut = batchCrypto.withBatch("scheduleAggregateChecks.selection_proof"):
if (let v = batch batch.pendingBuffer.add_slot_signature(
.pendingBuffer fork, genesis_validators_root, aggregate.data.slot, aggregatorKey,
.addSlotSignature( slotSig)
fork, genesis_validators_root, aggregateFut = batchCrypto.withBatch("scheduleAggregateChecks.aggregate"):
aggregate.data.slot, batch.pendingBuffer.add_attestation_signature(
aggregator.get(), fork, genesis_validators_root, aggregate.data, aggregateKey,
aggregate_and_proof.selection_proof aggregateSig)
); v.isErr()):
return err(v.error())
let futSlot = newFuture[BatchResult]( ok((aggregatorFut, slotFut, aggregateFut, aggregateSig))
"batch_validation.scheduleAggregateChecks.slotCheck"
)
batch.resultsBuffer.add(futSlot)
block: proc scheduleSyncCommitteeMessageCheck*(
if (let v = batch batchCrypto: ref BatchCrypto,
.pendingBuffer fork: Fork, genesis_validators_root: Eth2Digest,
.addAggregateAndProofSignature( slot: Slot, beacon_block_root: Eth2Digest,
fork, genesis_validators_root, pubkey: CookedPubKey, signature: ValidatorSig
aggregate_and_proof, ): Result[tuple[fut: Future[BatchResult], sig: CookedSig], cstring] =
aggregator.get(), ## Schedule crypto verification of an attestation
signed_aggregate_and_proof.signature ##
); v.isErr()): ## The buffer is processed:
batchCrypto.scheduleBatch(fresh) ## - when eager processing is enabled and the batch is full
return err(v.error()) ## - otherwise after 10ms (BatchAttAccumTime)
##
## This returns an error if crypto sanity checks failed
## and a future with the deferred attestation check otherwise.
##
let
sig = signature.load().orReturnErr(
"SyncCommitteMessage: cannot load signature")
fut = batchCrypto.withBatch("scheduleSyncCommitteeMessageCheck"):
batch.pendingBuffer.add_sync_committee_message_signature(
fork, genesis_validators_root, slot, beacon_block_root, pubkey, sig)
let futAggregator = newFuture[BatchResult]( ok((fut, sig))
"batch_validation.scheduleAggregateChecks.aggregatorCheck"
)
batch.resultsBuffer.add(futAggregator) proc scheduleContributionChecks*(
batchCrypto: ref BatchCrypto,
fork: Fork, genesis_validators_root: Eth2Digest,
signedContributionAndProof: SignedContributionAndProof,
subcommitteeIndex: SyncSubcommitteeIndex,
dag: ChainDAGRef): Result[tuple[
aggregatorFut, proofFut, contributionFut: Future[BatchResult],
sig: CookedSig], cstring] =
## Schedule crypto verification of all signatures in a
## SignedContributionAndProof message
##
## The buffer is processed:
## - when eager processing is enabled and the batch is full
## - otherwise after 10ms (BatchAttAccumTime)
##
## This returns an error if crypto sanity checks failed
## and a future with the deferred check otherwise.
##
template contribution_and_proof: untyped = signedContributionAndProof.message
template contribution: untyped = contribution_and_proof.contribution
let sig = batch # Do the eager steps first to avoid polluting batches with needlessly
.pendingBuffer let
.addAttestation( aggregatorKey =
fork, genesis_validators_root, epochRef, dag.validatorKey(contribution_and_proof.aggregator_index).orReturnErr(
aggregate "SignedAggregateAndProof: invalid contributor index")
) aggregatorSig = signedContributionAndProof.signature.load().orReturnErr(
if sig.isErr(): "SignedContributionAndProof: invalid proof signature")
batchCrypto.scheduleBatch(fresh) proofSig = contribution_and_proof.selection_proof.load().orReturnErr(
return err(sig.error()) "SignedContributionAndProof: invalid selection signature")
contributionSig = contribution.signature.load().orReturnErr(
"SignedContributionAndProof: invalid contribution signature")
let futAggregate = newFuture[BatchResult]( contributionKey = ? aggregateAll(
"batch_validation.scheduleAggregateChecks.aggregateCheck" dag, dag.syncCommitteeParticipants(contribution.slot, subcommitteeIndex),
) contribution.aggregation_bits)
batch.resultsBuffer.add(futAggregate) let
aggregatorFut = batchCrypto.withBatch("scheduleContributionAndProofChecks.aggregator"):
batch.pendingBuffer.add_contribution_and_proof_signature(
fork, genesis_validators_root, contribution_and_proof, aggregatorKey,
aggregatorSig)
proofFut = batchCrypto.withBatch("scheduleContributionAndProofChecks.selection_proof"):
batch.pendingBuffer.add_sync_committee_selection_proof(
fork, genesis_validators_root, contribution.slot,
contribution.subcommittee_index, aggregatorKey, proofSig)
contributionFut = batchCrypto.withBatch("scheduleContributionAndProofChecks.contribution"):
batch.pendingBuffer.add_sync_committee_message_signature(
fork, genesis_validators_root, contribution.slot,
contribution.beacon_block_root, contributionKey, contributionSig)
batchCrypto.scheduleBatch(fresh) ok((aggregatorFut, proofFut, contributionFut, contributionSig))
return ok(((futSlot, futAggregator, futAggregate), sig.get()))

View File

@ -411,7 +411,7 @@ proc syncCommitteeMessageValidator*(
self: ref Eth2Processor, self: ref Eth2Processor,
syncCommitteeMsg: SyncCommitteeMessage, syncCommitteeMsg: SyncCommitteeMessage,
subcommitteeIdx: SyncSubcommitteeIndex, subcommitteeIdx: SyncSubcommitteeIndex,
checkSignature: bool = true): Result[void, ValidationError] = checkSignature: bool = true): Future[Result[void, ValidationError]] {.async.} =
let let
wallTime = self.getCurrentBeaconTime() wallTime = self.getCurrentBeaconTime()
wallSlot = wallTime.slotOrZero() wallSlot = wallTime.slotOrZero()
@ -426,9 +426,9 @@ proc syncCommitteeMessageValidator*(
debug "Sync committee message received", delay debug "Sync committee message received", delay
# Now proceed to validation # Now proceed to validation
let v = validateSyncCommitteeMessage(self.dag, self.syncCommitteeMsgPool[], let v = await validateSyncCommitteeMessage(
syncCommitteeMsg, subcommitteeIdx, self.dag, self.batchCrypto, self.syncCommitteeMsgPool[],
wallTime, checkSignature) syncCommitteeMsg, subcommitteeIdx, wallTime, checkSignature)
return if v.isOk(): return if v.isOk():
trace "Sync committee message validated" trace "Sync committee message validated"
let (positions, cookedSig) = v.get() let (positions, cookedSig) = v.get()
@ -452,7 +452,7 @@ proc syncCommitteeMessageValidator*(
proc contributionValidator*( proc contributionValidator*(
self: ref Eth2Processor, self: ref Eth2Processor,
contributionAndProof: SignedContributionAndProof, contributionAndProof: SignedContributionAndProof,
checkSignature: bool = true): Result[void, ValidationError] = checkSignature: bool = true): Future[Result[void, ValidationError]] {.async.} =
let let
wallTime = self.getCurrentBeaconTime() wallTime = self.getCurrentBeaconTime()
wallSlot = wallTime.slotOrZero() wallSlot = wallTime.slotOrZero()
@ -469,9 +469,9 @@ proc contributionValidator*(
debug "Contribution received", delay debug "Contribution received", delay
# Now proceed to validation # Now proceed to validation
let v = validateContribution( let v = await validateContribution(
self.dag, self.syncCommitteeMsgPool[], contributionAndProof, wallTime, self.dag, self.batchCrypto, self.syncCommitteeMsgPool,
checkSignature) contributionAndProof, wallTime, checkSignature)
return if v.isOk(): return if v.isOk():
trace "Contribution validated" trace "Contribution validated"

View File

@ -34,6 +34,12 @@ declareCounter beacon_attestations_dropped_queue_full,
declareCounter beacon_aggregates_dropped_queue_full, declareCounter beacon_aggregates_dropped_queue_full,
"Number of aggregates dropped because queue is full" "Number of aggregates dropped because queue is full"
declareCounter beacon_sync_messages_dropped_queue_full,
"Number of sync committee messages dropped because queue is full"
declareCounter beacon_contributions_dropped_queue_full,
"Number of sync committee contributions dropped because queue is full"
# This result is a little messy in that it returns Result.ok for # This result is a little messy in that it returns Result.ok for
# ValidationResult.Accept and an err for the others - this helps transport # ValidationResult.Accept and an err for the others - this helps transport
# an error message to callers but could arguably be done in an cleaner way. # an error message to callers but could arguably be done in an cleaner way.
@ -462,29 +468,25 @@ proc validateAttestation*(
attestation.data.target.epoch: attestation.data.target.epoch:
return errIgnore("Attestation: Validator has already voted in epoch") return errIgnore("Attestation: Validator has already voted in epoch")
block: let pubkey = epochRef.validatorKey(validator_index)
# First pass - without cryptography if pubkey.isNone():
let v = is_valid_indexed_attestation( # can't happen, in theory, because we checked the aggregator index above
fork, genesis_validators_root, epochRef, attestation, return errIgnore("Attestation: cannot find validator pubkey")
{skipBLSValidation})
if v.isErr():
return 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 = let sig =
if checkSignature: if checkSignature:
# Attestation signatures are batch-verified # Attestation signatures are batch-verified
let deferredCrypto = batchCrypto let deferredCrypto = batchCrypto
.scheduleAttestationCheck( .scheduleAttestationCheck(
fork, genesis_validators_root, epochRef, fork, genesis_validators_root, attestation.data,
attestation pubkey.get(), attestation.signature)
)
if deferredCrypto.isErr(): if deferredCrypto.isErr():
return checkedReject(deferredCrypto.error) return checkedReject(deferredCrypto.error)
let (cryptoFut, sig) = deferredCrypto.get()
# Await the crypto check # Await the crypto check
let
(cryptoFut, sig) = deferredCrypto.get()
var x = (await cryptoFut) var x = (await cryptoFut)
case x case x
of BatchResult.Invalid: of BatchResult.Invalid:
@ -625,20 +627,35 @@ proc validateAggregate*(
genesis_validators_root = genesis_validators_root =
getStateField(pool.dag.headState.data, genesis_validators_root) getStateField(pool.dag.headState.data, genesis_validators_root)
let attesting_indices = get_attesting_indices(
epochRef, aggregate.data, aggregate.aggregation_bits)
let deferredCrypto = batchCrypto let deferredCrypto = batchCrypto
.scheduleAggregateChecks( .scheduleAggregateChecks(
fork, genesis_validators_root, epochRef, fork, genesis_validators_root,
signed_aggregate_and_proof signed_aggregate_and_proof, epochRef, attesting_indices
) )
if deferredCrypto.isErr(): if deferredCrypto.isErr():
return checkedReject(deferredCrypto.error) return checkedReject(deferredCrypto.error)
let let
(cryptoFuts, sig) = deferredCrypto.get() (aggregatorFut, slotFut, aggregateFut, sig) = deferredCrypto.get()
block:
# [REJECT] The aggregator signature, signed_aggregate_and_proof.signature, is valid.
var x = await aggregatorFut
case x
of BatchResult.Invalid:
return checkedReject("Aggregate: invalid aggregator signature")
of BatchResult.Timeout:
beacon_aggregates_dropped_queue_full.inc()
return errIgnore("Aggregate: timeout checking aggregator signature")
of BatchResult.Valid:
discard
block: block:
# [REJECT] aggregate_and_proof.selection_proof # [REJECT] aggregate_and_proof.selection_proof
var x = await cryptoFuts.slotCheck var x = await slotFut
case x case x
of BatchResult.Invalid: of BatchResult.Invalid:
return checkedReject("Aggregate: invalid slot signature") return checkedReject("Aggregate: invalid slot signature")
@ -650,19 +667,7 @@ proc validateAggregate*(
block: block:
# [REJECT] The aggregator signature, signed_aggregate_and_proof.signature, is valid. # [REJECT] The aggregator signature, signed_aggregate_and_proof.signature, is valid.
var x = await cryptoFuts.aggregatorCheck var x = await aggregateFut
case x
of BatchResult.Invalid:
return checkedReject("Aggregate: invalid aggregator signature")
of BatchResult.Timeout:
beacon_aggregates_dropped_queue_full.inc()
return errIgnore("Aggregate: timeout checking aggregator signature")
of BatchResult.Valid:
discard
block:
# [REJECT] The aggregator signature, signed_aggregate_and_proof.signature, is valid.
var x = await cryptoFuts.aggregateCheck
case x case x
of BatchResult.Invalid: of BatchResult.Invalid:
return checkedReject("Aggregate: invalid aggregate signature") return checkedReject("Aggregate: invalid aggregate signature")
@ -688,9 +693,6 @@ proc validateAggregate*(
pool.nextAttestationEpoch[aggregate_and_proof.aggregator_index].aggregate = pool.nextAttestationEpoch[aggregate_and_proof.aggregator_index].aggregate =
aggregate.data.target.epoch + 1 aggregate.data.target.epoch + 1
let attesting_indices = get_attesting_indices(
epochRef, aggregate.data, aggregate.aggregation_bits)
return ok((attesting_indices, sig)) return ok((attesting_indices, sig))
# https://github.com/ethereum/consensus-specs/blob/v1.0.1/specs/phase0/p2p-interface.md#attester_slashing # https://github.com/ethereum/consensus-specs/blob/v1.0.1/specs/phase0/p2p-interface.md#attester_slashing
@ -772,17 +774,20 @@ proc validateVoluntaryExit*(
# https://github.com/ethereum/consensus-specs/blob/v1.1.0-alpha.8/specs/altair/p2p-interface.md#sync_committee_subnet_id # https://github.com/ethereum/consensus-specs/blob/v1.1.0-alpha.8/specs/altair/p2p-interface.md#sync_committee_subnet_id
proc validateSyncCommitteeMessage*( proc validateSyncCommitteeMessage*(
dag: ChainDAGRef, dag: ChainDAGRef,
batchCrypto: ref BatchCrypto,
syncCommitteeMsgPool: SyncCommitteeMsgPool, syncCommitteeMsgPool: SyncCommitteeMsgPool,
msg: SyncCommitteeMessage, msg: SyncCommitteeMessage,
subcommitteeIdx: SyncSubcommitteeIndex, subcommitteeIdx: SyncSubcommitteeIndex,
wallTime: BeaconTime, wallTime: BeaconTime,
checkSignature: bool): checkSignature: bool):
Result[(seq[uint64], CookedSig), ValidationError] = Future[Result[(seq[uint64], CookedSig), ValidationError]] {.async.} =
block: block:
# [IGNORE] The signature's slot is for the current slot # [IGNORE] The signature's slot is for the current slot
# (with a MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance) # (with a MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance)
# i.e. sync_committee_message.slot == current_slot. # i.e. sync_committee_message.slot == current_slot.
? check_propagation_slot_range(msg.slot, wallTime) let v = check_propagation_slot_range(msg.slot, wallTime)
if v.isErr():
return err(v.error())
# [REJECT] The subnet_id is valid for the given validator # [REJECT] The subnet_id is valid for the given validator
# i.e. subnet_id in compute_subnets_for_sync_committee(state, sync_committee_message.validator_index). # i.e. subnet_id in compute_subnets_for_sync_committee(state, sync_committee_message.validator_index).
@ -819,41 +824,62 @@ proc validateSyncCommitteeMessage*(
if senderPubKey.isNone(): if senderPubKey.isNone():
return errReject("SyncCommitteeMessage: invalid validator index") return errReject("SyncCommitteeMessage: invalid validator index")
var cookedSignature = msg.signature.load let sig =
if cookedSignature.isNone: if checkSignature:
return errReject("SyncCommitteeMessage: signature fails to load") # Attestation signatures are batch-verified
let deferredCrypto = batchCrypto
.scheduleSyncCommitteeMessageCheck(
fork, genesis_validators_root,
msg.slot, msg.beacon_block_root,
senderPubKey.get(), msg.signature)
if deferredCrypto.isErr():
return errReject(deferredCrypto.error)
if checkSignature and # Await the crypto check
not verify_sync_committee_message_signature(epoch, let
msg.beacon_block_root, (cryptoFut, sig) = deferredCrypto.get()
fork, genesisValidatorsRoot,
senderPubKey.get(),
cookedSignature.get):
return errReject("SyncCommitteeMessage: signature fails to verify")
ok((positionsInSubcommittee, cookedSignature.get())) var x = (await cryptoFut)
case x
of BatchResult.Invalid:
return errReject("SyncCommitteeMessage: invalid signature")
of BatchResult.Timeout:
beacon_sync_messages_dropped_queue_full.inc()
return errIgnore("SyncCommitteeMessage: timeout checking signature")
of BatchResult.Valid:
sig # keep going only in this case
else:
let sig = msg.signature.load()
if not sig.isSome():
return errReject("SyncCommitteeMessage: unable to load signature")
sig.get()
return ok((positionsInSubcommittee, sig))
# https://github.com/ethereum/consensus-specs/blob/v1.1.5/specs/altair/p2p-interface.md#sync_committee_contribution_and_proof # https://github.com/ethereum/consensus-specs/blob/v1.1.5/specs/altair/p2p-interface.md#sync_committee_contribution_and_proof
proc validateContribution*( proc validateContribution*(
dag: ChainDAGRef, dag: ChainDAGRef,
syncCommitteeMsgPool: var SyncCommitteeMsgPool, batchCrypto: ref BatchCrypto,
syncCommitteeMsgPool: ref SyncCommitteeMsgPool,
msg: SignedContributionAndProof, msg: SignedContributionAndProof,
wallTime: BeaconTime, wallTime: BeaconTime,
checkSignature: bool): checkSignature: bool):
Result[CookedSig, ValidationError] = Future[Result[CookedSig, ValidationError]] {.async.} =
let
syncCommitteeSlot = msg.message.contribution.slot
# [IGNORE] The contribution's slot is for the current slot # [IGNORE] The contribution's slot is for the current slot
# (with a MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance) # (with a MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance)
# i.e. contribution.slot == current_slot. # i.e. contribution.slot == current_slot.
? check_propagation_slot_range(msg.message.contribution.slot, wallTime) block:
let let v = check_propagation_slot_range(syncCommitteeSlot, wallTime) # [IGNORE]
aggregatorPubKey = dag.validatorKey(msg.message.aggregator_index) if v.isErr():
if aggregatorPubKey.isNone(): return err(v.error())
return errReject("SignedContributionAndProof: invalid aggregator index")
# [REJECT] The subcommittee index is in the allowed range # [REJECT] The subcommittee index is in the allowed range
# i.e. contribution.subcommittee_index < SYNC_COMMITTEE_SUBNET_COUNT. # i.e. contribution.subcommittee_index < SYNC_COMMITTEE_SUBNET_COUNT.
let committeeIdx = msg.message.contribution.subcommittee_index.validateSyncCommitteeIndexOr: let subcommitteeIdx = msg.message.contribution.subcommittee_index.validateSyncCommitteeIndexOr:
return errReject("SignedContributionAndProof: subcommittee index too high") return errReject("SignedContributionAndProof: subcommittee index too high")
# [REJECT] contribution_and_proof.selection_proof selects the validator as an aggregator for the slot # [REJECT] contribution_and_proof.selection_proof selects the validator as an aggregator for the slot
@ -861,13 +887,12 @@ proc validateContribution*(
if not is_sync_committee_aggregator(msg.message.selection_proof): if not is_sync_committee_aggregator(msg.message.selection_proof):
return errReject("SignedContributionAndProof: invalid selection_proof") return errReject("SignedContributionAndProof: invalid selection_proof")
block:
# [IGNORE] The sync committee contribution is the first valid contribution # [IGNORE] The sync committee contribution is the first valid contribution
# received for the aggregator with index contribution_and_proof.aggregator_index # received for the aggregator with index contribution_and_proof.aggregator_index
# for the slot contribution.slot and subcommittee index contribution.subcommittee_index # for the slot contribution.slot and subcommittee index contribution.subcommittee_index
# (this requires maintaining a cache of size SYNC_COMMITTEE_SIZE for this # (this requires maintaining a cache of size SYNC_COMMITTEE_SIZE for this
# topic that can be flushed after each slot). # topic that can be flushed after each slot).
if syncCommitteeMsgPool.isSeen(msg.message): if syncCommitteeMsgPool[].isSeen(msg.message):
return errIgnore("SignedContributionAndProof: duplicate contribution") return errIgnore("SignedContributionAndProof: duplicate contribution")
# [REJECT] The aggregator's validator index is in the declared subcommittee # [REJECT] The aggregator's validator index is in the declared subcommittee
@ -877,70 +902,61 @@ proc validateContribution*(
let let
epoch = msg.message.contribution.slot.epoch epoch = msg.message.contribution.slot.epoch
fork = dag.forkAtEpoch(epoch) fork = dag.forkAtEpoch(epoch)
genesisValidatorsRoot = dag.genesisValidatorsRoot genesis_validators_root = dag.genesisValidatorsRoot
# [REJECT] The aggregator signature, signed_contribution_and_proof.signature, is valid if msg.message.contribution.aggregation_bits.countOnes() == 0:
if not verify_signed_contribution_and_proof_signature(msg, fork,
genesisValidatorsRoot,
aggregatorPubKey.get()):
return errReject(
"SignedContributionAndProof: aggregator signature fails to verify")
# [REJECT] The contribution_and_proof.selection_proof is a valid signature of the
# SyncAggregatorSelectionData derived from the contribution by the validator with
# index contribution_and_proof.aggregator_index.
if not verify_selection_proof_signature(msg.message, fork,
genesisValidatorsRoot,
aggregatorPubKey.get()):
return errReject(
"SignedContributionAndProof: selection proof signature fails to verify")
# [REJECT] The aggregate signature is valid for the message beacon_block_root
# and aggregate pubkey derived from the participation info in aggregation_bits
# for the subcommittee specified by the contribution.subcommittee_index.
var
committeeAggKey {.noInit.}: AggregatePublicKey
initialized = false
syncCommitteeSlot = msg.message.contribution.slot + 1
for validatorIndex in dag.syncCommitteeParticipants(
syncCommitteeSlot,
committeeIdx,
msg.message.contribution.aggregation_bits):
let validatorPubKey = dag.validatorKey(validatorIndex)
if not validatorPubKey.isSome():
# This should never happen (!)
warn "Invalid validator index in committee cache",
validatorIndex
return errIgnore("SignedContributionAndProof: Invalid committee cache")
if not initialized:
initialized = true
committeeAggKey.init(validatorPubKey.get())
else:
committeeAggKey.aggregate(validatorPubKey.get())
if not initialized:
# [REJECT] The contribution has participants # [REJECT] The contribution has participants
# that is, any(contribution.aggregation_bits). # that is, any(contribution.aggregation_bits).
return errReject("SignedContributionAndProof: aggregation bits empty") return errReject("SignedContributionAndProof: aggregation bits empty")
let cookedSignature = msg.message.contribution.signature.load let sig = if checkSignature:
if cookedSignature.isNone: let deferredCrypto = batchCrypto.scheduleContributionChecks(
return errReject( fork, genesis_validators_root, msg, subcommitteeIdx, dag)
"SignedContributionAndProof: aggregate signature fails to load") if deferredCrypto.isErr():
return errReject(deferredCrypto.error)
if checkSignature and let
not verify_sync_committee_message_signature( (aggregatorFut, proofFut, contributionFut, sig) = deferredCrypto.get()
epoch, msg.message.contribution.beacon_block_root, fork,
genesisValidatorsRoot, committeeAggKey.finish, cookedSignature.get):
debug "failing_sync_contribution",
blk = msg.message.contribution.beacon_block_root,
slot = syncCommitteeSlot,
subnet = committeeIdx,
participants = $(msg.message.contribution.aggregation_bits)
return errReject( block:
"SignedContributionAndProof: aggregate signature fails to verify") # [REJECT] The aggregator signature, signed_contribution_and_proof.signature, is valid
var x = await aggregatorFut
case x
of BatchResult.Invalid:
return errReject("SignedContributionAndProof: invalid aggregator signature")
of BatchResult.Timeout:
beacon_contributions_dropped_queue_full.inc()
return errIgnore("SignedContributionAndProof: timeout checking aggregator signature")
of BatchResult.Valid:
discard
ok(cookedSignature.get) block:
var x = await proofFut
case x
of BatchResult.Invalid:
return errReject("SignedContributionAndProof: invalid proof")
of BatchResult.Timeout:
beacon_contributions_dropped_queue_full.inc()
return errIgnore("SignedContributionAndProof: timeout checking proof")
of BatchResult.Valid:
discard
block:
# [REJECT] The aggregator signature, signed_aggregate_and_proof.signature, is valid.
var x = await contributionFut
case x
of BatchResult.Invalid:
return errReject("SignedContributionAndProof: invalid contribution signature")
of BatchResult.Timeout:
beacon_contributions_dropped_queue_full.inc()
return errIgnore("SignedContributionAndProof: timeout checking contribution signature")
of BatchResult.Valid:
discard
sig
else:
let sig = msg.message.contribution.signature.load()
if not sig.isSome():
return errReject("SyncCommitteeMessage: unable to load signature")
sig.get()
return ok(sig)

View File

@ -1056,18 +1056,18 @@ proc installMessageValidators(node: BeaconNode) =
for committeeIdx in allSyncSubcommittees(): for committeeIdx in allSyncSubcommittees():
closureScope: closureScope:
let idx = committeeIdx let idx = committeeIdx
node.network.addValidator( node.network.addAsyncValidator(
getSyncCommitteeTopic(digest, idx), getSyncCommitteeTopic(digest, idx),
# This proc needs to be within closureScope; don't lift out of loop. # This proc needs to be within closureScope; don't lift out of loop.
proc(msg: SyncCommitteeMessage): ValidationResult = proc(msg: SyncCommitteeMessage): Future[ValidationResult] {.async.} =
toValidationResult( return toValidationResult(
node.processor.syncCommitteeMessageValidator(msg, idx))) await node.processor.syncCommitteeMessageValidator(msg, idx)))
node.network.addValidator( node.network.addAsyncValidator(
getSyncCommitteeContributionAndProofTopic(digest), getSyncCommitteeContributionAndProofTopic(digest),
proc(msg: SignedContributionAndProof): ValidationResult = proc(msg: SignedContributionAndProof): Future[ValidationResult] {.async.} =
toValidationResult( return toValidationResult(
node.processor.contributionValidator(msg))) await node.processor.contributionValidator(msg)))
installSyncCommitteeeValidators(node.dag.forkDigests.altair) installSyncCommitteeeValidators(node.dag.forkDigests.altair)
installSyncCommitteeeValidators(node.dag.forkDigests.merge) installSyncCommitteeeValidators(node.dag.forkDigests.merge)

View File

@ -303,7 +303,7 @@ proc installApiHandlers*(node: SigningNode) =
let let
forkInfo = request.forkInfo.get() forkInfo = request.forkInfo.get()
msg = request.syncAggregatorSelectionData msg = request.syncAggregatorSelectionData
cooked = get_sync_aggregator_selection_data_signature(forkInfo.fork, cooked = get_sync_committee_selection_proof(forkInfo.fork,
forkInfo.genesisValidatorsRoot, msg.slot, msg.subcommittee_index, forkInfo.genesisValidatorsRoot, msg.slot, msg.subcommittee_index,
validator.data.privateKey) validator.data.privateKey)
signature = cooked.toValidatorSig().toHex() signature = cooked.toValidatorSig().toHex()
@ -312,7 +312,7 @@ proc installApiHandlers*(node: SigningNode) =
let let
forkInfo = request.forkInfo.get() forkInfo = request.forkInfo.get()
msg = request.syncCommitteeContributionAndProof msg = request.syncCommitteeContributionAndProof
cooked = get_sync_committee_contribution_and_proof_signature( cooked = get_contribution_and_proof_signature(
forkInfo.fork, forkInfo.genesisValidatorsRoot, msg, forkInfo.fork, forkInfo.genesisValidatorsRoot, msg,
validator.data.privateKey) validator.data.privateKey)
signature = cooked.toValidatorSig().toHex() signature = cooked.toValidatorSig().toHex()

View File

@ -7,6 +7,15 @@
{.push raises: [Defect].} {.push raises: [Defect].}
## Signature production and verification for spec types - for every type of
## signature, there are 3 functions:
## * `compute_*_signing_root` - reduce message to the data that will be signed
## * `get_*_signature` - sign the signing root with a private key
## * `verify_*_signature` - verify a signature produced by `get_*_signature`
##
## See also `signatures_batch` for batch verification versions of these
## functions.
import import
./datatypes/[phase0, altair, merge], ./helpers, ./eth2_merkleization ./datatypes/[phase0, altair, merge], ./helpers, ./eth2_merkleization
@ -24,7 +33,7 @@ func getDepositMessage(depositData: DepositData): DepositMessage =
amount: depositData.amount, amount: depositData.amount,
withdrawal_credentials: depositData.withdrawal_credentials) withdrawal_credentials: depositData.withdrawal_credentials)
func compute_slot_root*( func compute_slot_signing_root*(
fork: Fork, genesis_validators_root: Eth2Digest, slot: Slot fork: Fork, genesis_validators_root: Eth2Digest, slot: Slot
): Eth2Digest = ): Eth2Digest =
let let
@ -37,21 +46,21 @@ func compute_slot_root*(
func get_slot_signature*( func get_slot_signature*(
fork: Fork, genesis_validators_root: Eth2Digest, slot: Slot, fork: Fork, genesis_validators_root: Eth2Digest, slot: Slot,
privkey: ValidatorPrivKey): CookedSig = privkey: ValidatorPrivKey): CookedSig =
blsSign(privKey, compute_slot_root(fork, genesis_validators_root, slot).data) let signing_root = compute_slot_signing_root(
fork, genesis_validators_root, slot)
blsSign(privKey, signing_root.data)
proc verify_slot_signature*( proc verify_slot_signature*(
fork: Fork, genesis_validators_root: Eth2Digest, slot: Slot, fork: Fork, genesis_validators_root: Eth2Digest, slot: Slot,
pubkey: ValidatorPubKey | CookedPubKey, signature: SomeSig): bool = pubkey: ValidatorPubKey | CookedPubKey, signature: SomeSig): bool =
withTrust(signature): withTrust(signature):
let let signing_root = compute_slot_signing_root(
epoch = compute_epoch_at_slot(slot) fork, genesis_validators_root, slot)
domain = get_domain(
fork, DOMAIN_SELECTION_PROOF, epoch, genesis_validators_root)
signing_root = compute_signing_root(slot, domain)
blsVerify(pubkey, signing_root.data, signature) blsVerify(pubkey, signing_root.data, signature)
func compute_epoch_root*( func compute_epoch_signing_root*(
fork: Fork, genesis_validators_root: Eth2Digest, epoch: Epoch fork: Fork, genesis_validators_root: Eth2Digest, epoch: Epoch
): Eth2Digest = ): Eth2Digest =
let domain = get_domain(fork, DOMAIN_RANDAO, epoch, genesis_validators_root) let domain = get_domain(fork, DOMAIN_RANDAO, epoch, genesis_validators_root)
@ -61,32 +70,37 @@ func compute_epoch_root*(
func get_epoch_signature*( func get_epoch_signature*(
fork: Fork, genesis_validators_root: Eth2Digest, epoch: Epoch, fork: Fork, genesis_validators_root: Eth2Digest, epoch: Epoch,
privkey: ValidatorPrivKey): CookedSig = privkey: ValidatorPrivKey): CookedSig =
blsSign(privKey, compute_epoch_root(fork, genesis_validators_root, epoch).data) let signing_root = compute_epoch_signing_root(
fork, genesis_validators_root, epoch)
blsSign(privKey, signing_root.data)
proc verify_epoch_signature*( proc verify_epoch_signature*(
fork: Fork, genesis_validators_root: Eth2Digest, epoch: Epoch, fork: Fork, genesis_validators_root: Eth2Digest, epoch: Epoch,
pubkey: ValidatorPubKey | CookedPubKey, signature: SomeSig): bool = pubkey: ValidatorPubKey | CookedPubKey, signature: SomeSig): bool =
withTrust(signature): withTrust(signature):
let let signing_root = compute_epoch_signing_root(
domain = get_domain(fork, DOMAIN_RANDAO, epoch, genesis_validators_root) fork, genesis_validators_root, epoch)
signing_root = compute_signing_root(epoch, domain)
blsVerify(pubkey, signing_root.data, signature) blsVerify(pubkey, signing_root.data, signature)
func compute_block_root*( func compute_block_signing_root*(
fork: Fork, genesis_validators_root: Eth2Digest, slot: Slot, fork: Fork, genesis_validators_root: Eth2Digest, slot: Slot,
root: Eth2Digest): Eth2Digest = blck: Eth2Digest | SomeSomeBeaconBlock | BeaconBlockHeader): Eth2Digest =
let let
epoch = compute_epoch_at_slot(slot) epoch = compute_epoch_at_slot(slot)
domain = get_domain( domain = get_domain(
fork, DOMAIN_BEACON_PROPOSER, epoch, genesis_validators_root) fork, DOMAIN_BEACON_PROPOSER, epoch, genesis_validators_root)
compute_signing_root(root, domain) compute_signing_root(blck, domain)
# https://github.com/ethereum/consensus-specs/blob/v1.1.6/specs/phase0/validator.md#signature # https://github.com/ethereum/consensus-specs/blob/v1.1.6/specs/phase0/validator.md#signature
func get_block_signature*( func get_block_signature*(
fork: Fork, genesis_validators_root: Eth2Digest, slot: Slot, fork: Fork, genesis_validators_root: Eth2Digest, slot: Slot,
root: Eth2Digest, privkey: ValidatorPrivKey): CookedSig = root: Eth2Digest, privkey: ValidatorPrivKey): CookedSig =
blsSign(privKey, compute_block_root(fork, genesis_validators_root, slot, root).data) let signing_root = compute_block_signing_root(
fork, genesis_validators_root, slot, root)
blsSign(privKey, signing_root.data)
proc verify_block_signature*( proc verify_block_signature*(
fork: Fork, genesis_validators_root: Eth2Digest, slot: Slot, fork: Fork, genesis_validators_root: Eth2Digest, slot: Slot,
@ -94,16 +108,14 @@ proc verify_block_signature*(
pubkey: ValidatorPubKey | CookedPubKey, signature: SomeSig): bool = pubkey: ValidatorPubKey | CookedPubKey, signature: SomeSig): bool =
withTrust(signature): withTrust(signature):
let let
epoch = compute_epoch_at_slot(slot) signing_root = compute_block_signing_root(
domain = get_domain( fork, genesis_validators_root, slot, blck)
fork, DOMAIN_BEACON_PROPOSER, epoch, genesis_validators_root)
signing_root = compute_signing_root(blck, domain)
blsVerify(pubKey, signing_root.data, signature) blsVerify(pubkey, signing_root.data, signature)
func compute_aggregate_and_proof_root*(fork: Fork, genesis_validators_root: Eth2Digest, func compute_aggregate_and_proof_signing_root*(
aggregate_and_proof: AggregateAndProof, fork: Fork, genesis_validators_root: Eth2Digest,
): Eth2Digest = aggregate_and_proof: AggregateAndProof): Eth2Digest =
let let
epoch = compute_epoch_at_slot(aggregate_and_proof.aggregate.data.slot) epoch = compute_epoch_at_slot(aggregate_and_proof.aggregate.data.slot)
domain = get_domain( domain = get_domain(
@ -114,56 +126,149 @@ func compute_aggregate_and_proof_root*(fork: Fork, genesis_validators_root: Eth2
func get_aggregate_and_proof_signature*(fork: Fork, genesis_validators_root: Eth2Digest, func get_aggregate_and_proof_signature*(fork: Fork, genesis_validators_root: Eth2Digest,
aggregate_and_proof: AggregateAndProof, aggregate_and_proof: AggregateAndProof,
privKey: ValidatorPrivKey): CookedSig = privKey: ValidatorPrivKey): CookedSig =
blsSign(privKey, compute_aggregate_and_proof_root(fork, genesis_validators_root, let signing_root = compute_aggregate_and_proof_signing_root(
aggregate_and_proof).data) fork, genesis_validators_root, aggregate_and_proof)
blsSign(privKey, signing_root.data)
proc verify_aggregate_and_proof_signature*( proc verify_aggregate_and_proof_signature*(
fork: Fork, genesis_validators_root: Eth2Digest, fork: Fork, genesis_validators_root: Eth2Digest,
aggregate_and_proof: AggregateAndProof, aggregate_and_proof: AggregateAndProof,
pubkey: ValidatorPubKey | CookedPubKey, signature: SomeSig): bool = pubkey: ValidatorPubKey | CookedPubKey, signature: SomeSig): bool =
withTrust(signature): withTrust(signature):
let let signing_root = compute_aggregate_and_proof_signing_root(
epoch = compute_epoch_at_slot(aggregate_and_proof.aggregate.data.slot) fork, genesis_validators_root, aggregate_and_proof)
domain = get_domain(
fork, DOMAIN_AGGREGATE_AND_PROOF, epoch, genesis_validators_root)
signing_root = compute_signing_root(aggregate_and_proof, domain)
blsVerify(pubKey, signing_root.data, signature) blsVerify(pubKey, signing_root.data, signature)
func compute_attestation_root*( func compute_attestation_signing_root*(
fork: Fork, genesis_validators_root: Eth2Digest, fork: Fork, genesis_validators_root: Eth2Digest,
attestation_data: AttestationData attestation_data: AttestationData): Eth2Digest =
): Eth2Digest =
let let
epoch = attestation_data.target.epoch epoch = attestation_data.target.epoch
domain = get_domain( domain = get_domain(
fork, DOMAIN_BEACON_ATTESTER, epoch, genesis_validators_root) fork, DOMAIN_BEACON_ATTESTER, epoch, genesis_validators_root)
compute_signing_root(attestation_data, domain) compute_signing_root(attestation_data, domain)
# https://github.com/ethereum/consensus-specs/blob/v1.1.6/specs/altair/validator.md#prepare-sync-committee-message # https://github.com/ethereum/consensus-specs/blob/v1.1.6/specs/phase0/validator.md#aggregate-signature
func sync_committee_msg_signing_root*( func get_attestation_signature*(
fork: Fork, epoch: Epoch, fork: Fork, genesis_validators_root: Eth2Digest,
genesis_validators_root: Eth2Digest, attestation_data: AttestationData,
block_root: Eth2Digest): Eth2Digest = privkey: ValidatorPrivKey): CookedSig =
let domain = get_domain(fork, DOMAIN_SYNC_COMMITTEE, epoch, genesis_validators_root) let signing_root = compute_attestation_signing_root(
compute_signing_root(block_root, domain) fork, genesis_validators_root, attestation_data)
# https://github.com/ethereum/consensus-specs/blob/v1.1.6/specs/altair/validator.md#signature blsSign(privKey, signing_root.data)
func contribution_and_proof_signing_root*(
fork: Fork, proc verify_attestation_signature*(
genesis_validators_root: Eth2Digest, fork: Fork, genesis_validators_root: Eth2Digest,
msg: ContributionAndProof): Eth2Digest = attestation_data: AttestationData,
let domain = get_domain(fork, DOMAIN_CONTRIBUTION_AND_PROOF, pubkeys: auto, signature: SomeSig): bool =
msg.contribution.slot.epoch, withTrust(signature):
genesis_validators_root) let signing_root = compute_attestation_signing_root(
compute_signing_root(msg, domain) fork, genesis_validators_root, attestation_data)
blsFastAggregateVerify(pubkeys, signing_root.data, signature)
func compute_deposit_signing_root*(
version: Version,
deposit_message: DepositMessage): Eth2Digest =
let
# Fork-agnostic domain since deposits are valid across forks
domain = compute_domain(DOMAIN_DEPOSIT, version)
compute_signing_root(deposit_message, domain)
# https://github.com/ethereum/consensus-specs/blob/v1.1.6/specs/phase0/beacon-chain.md#deposits
func get_deposit_signature*(preset: RuntimeConfig,
deposit: DepositData,
privkey: ValidatorPrivKey): CookedSig =
let signing_root = compute_deposit_signing_root(
preset.GENESIS_FORK_VERSION, deposit.getDepositMessage())
blsSign(privKey, signing_root.data)
func get_deposit_signature*(message: DepositMessage, version: Version,
privkey: ValidatorPrivKey): CookedSig =
let signing_root = compute_deposit_signing_root(version, message)
blsSign(privkey, signing_root.data)
proc verify_deposit_signature*(preset: RuntimeConfig,
deposit: DepositData): bool =
let
deposit_message = deposit.getDepositMessage()
signing_root = compute_deposit_signing_root(
preset.GENESIS_FORK_VERSION, deposit_message)
blsVerify(deposit.pubkey, signing_root.data, deposit.signature)
func compute_voluntary_exit_signing_root*(
fork: Fork, genesis_validators_root: Eth2Digest,
voluntary_exit: VoluntaryExit): Eth2Digest =
let
epoch = voluntary_exit.epoch
domain = get_domain(
fork, DOMAIN_VOLUNTARY_EXIT, epoch, genesis_validators_root)
compute_signing_root(voluntary_exit, domain)
func get_voluntary_exit_signature*(
fork: Fork, genesis_validators_root: Eth2Digest,
voluntary_exit: VoluntaryExit,
privkey: ValidatorPrivKey): CookedSig =
let signing_root = compute_voluntary_exit_signing_root(
fork, genesis_validators_root, voluntary_exit)
blsSign(privKey, signing_root.data)
proc verify_voluntary_exit_signature*(
fork: Fork, genesis_validators_root: Eth2Digest,
voluntary_exit: VoluntaryExit,
pubkey: ValidatorPubKey | CookedPubKey, signature: SomeSig): bool =
withTrust(signature):
let signing_root = compute_voluntary_exit_signing_root(
fork, genesis_validators_root, voluntary_exit)
blsVerify(pubkey, signing_root.data, signature)
# https://github.com/ethereum/consensus-specs/blob/v1.1.6/specs/altair/validator.md#prepare-sync-committee-message
func compute_sync_committee_message_signing_root*(
fork: Fork, genesis_validators_root: Eth2Digest,
slot: Slot, beacon_block_root: Eth2Digest): Eth2Digest =
let domain = get_domain(
fork, DOMAIN_SYNC_COMMITTEE, slot.epoch, genesis_validators_root)
compute_signing_root(beacon_block_root, domain)
func get_sync_committee_message_signature*(
fork: Fork, genesis_validators_root: Eth2Digest,
slot: Slot, beacon_block_root: Eth2Digest,
privkey: ValidatorPrivKey): CookedSig =
let signing_root = compute_sync_committee_message_signing_root(
fork, genesis_validators_root, slot, beacon_block_root)
blsSign(privkey, signing_root.data)
proc verify_sync_committee_message_signature*(
fork: Fork, genesis_validators_root: Eth2Digest,
slot: Slot, beacon_block_root: Eth2Digest,
pubkey: ValidatorPubKey | CookedPubKey, signature: SomeSig): bool =
let signing_root = compute_sync_committee_message_signing_root(
fork, genesis_validators_root, slot, beacon_block_root)
blsVerify(pubkey, signing_root.data, signature)
proc verify_sync_committee_signature*(
fork: Fork, genesis_validators_root: Eth2Digest,
slot: Slot, beacon_block_root: Eth2Digest,
pubkeys: auto, signature: SomeSig): bool =
let signing_root = compute_sync_committee_message_signing_root(
fork, genesis_validators_root, slot, beacon_block_root)
blsFastAggregateVerify(pubkeys, signing_root.data, signature)
# https://github.com/ethereum/consensus-specs/blob/v1.1.6/specs/altair/validator.md#aggregation-selection # https://github.com/ethereum/consensus-specs/blob/v1.1.6/specs/altair/validator.md#aggregation-selection
proc sync_committee_selection_proof_signing_root*( func compute_sync_committee_selection_proof_signing_root*(
fork: Fork, fork: Fork, genesis_validators_root: Eth2Digest,
genesis_validators_root: Eth2Digest, slot: Slot, subcommittee_index: uint64): Eth2Digest =
slot: Slot,
subcommittee_index: uint64): Eth2Digest =
let let
domain = get_domain(fork, DOMAIN_SYNC_COMMITTEE_SELECTION_PROOF, domain = get_domain(fork, DOMAIN_SYNC_COMMITTEE_SELECTION_PROOF,
slot.epoch, genesis_validators_root) slot.epoch, genesis_validators_root)
@ -172,118 +277,43 @@ proc sync_committee_selection_proof_signing_root*(
subcommittee_index: subcommittee_index) subcommittee_index: subcommittee_index)
compute_signing_root(signing_data, domain) compute_signing_root(signing_data, domain)
# https://github.com/ethereum/consensus-specs/blob/v1.1.6/specs/phase0/validator.md#aggregate-signature func get_sync_committee_selection_proof*(
func get_attestation_signature*(
fork: Fork, genesis_validators_root: Eth2Digest, fork: Fork, genesis_validators_root: Eth2Digest,
attestation_data: AttestationData, slot: Slot, subcommittee_index: uint64,
privkey: ValidatorPrivKey): CookedSig = privkey: ValidatorPrivKey): CookedSig =
blsSign(privKey, compute_attestation_root(fork, genesis_validators_root, let signing_root = compute_sync_committee_selection_proof_signing_root(
attestation_data).data) fork, genesis_validators_root, slot, subcommittee_index)
proc verify_attestation_signature*(
fork: Fork, genesis_validators_root: Eth2Digest,
attestation_data: AttestationData,
pubkeys: auto,
signature: SomeSig): bool =
withTrust(signature):
let
epoch = attestation_data.target.epoch
domain = get_domain(
fork, DOMAIN_BEACON_ATTESTER, epoch, genesis_validators_root)
signing_root = compute_signing_root(attestation_data, domain)
blsFastAggregateVerify(pubkeys, signing_root.data, signature)
# https://github.com/ethereum/consensus-specs/blob/v1.1.6/specs/phase0/beacon-chain.md#deposits
func get_deposit_signature*(preset: RuntimeConfig,
deposit: DepositData,
privkey: ValidatorPrivKey): CookedSig =
let
deposit_message = deposit.getDepositMessage()
# Fork-agnostic domain since deposits are valid across forks
domain = compute_domain(DOMAIN_DEPOSIT, preset.GENESIS_FORK_VERSION)
signing_root = compute_signing_root(deposit_message, domain)
blsSign(privKey, signing_root.data)
func get_deposit_signature*(message: DepositMessage, version: Version,
privkey: ValidatorPrivKey): CookedSig =
let
domain = compute_domain(DOMAIN_DEPOSIT, version)
signing_root = compute_signing_root(message, domain)
blsSign(privkey, signing_root.data) blsSign(privkey, signing_root.data)
proc verify_deposit_signature*(preset: RuntimeConfig, proc verify_sync_committee_selection_proof*(
deposit: DepositData): bool = fork: Fork, genesis_validators_root: Eth2Digest,
let slot: Slot, subcommittee_index: uint64,
deposit_message = deposit.getDepositMessage() pubkey: ValidatorPubKey | CookedPubKey, signature: SomeSig): bool =
# Fork-agnostic domain since deposits are valid across forks
domain = compute_domain(DOMAIN_DEPOSIT, preset.GENESIS_FORK_VERSION)
signing_root = compute_signing_root(deposit_message, domain)
blsVerify(deposit.pubkey, signing_root.data, deposit.signature)
func get_voluntary_exit_signature*(
fork: Fork,
genesis_validators_root: Eth2Digest,
voluntary_exit: VoluntaryExit,
privkey: ValidatorPrivKey): CookedSig =
let
domain = get_domain(
fork, DOMAIN_VOLUNTARY_EXIT, voluntary_exit.epoch, genesis_validators_root)
signing_root = compute_signing_root(voluntary_exit, domain)
blsSign(privKey, signing_root.data)
proc verify_voluntary_exit_signature*(
fork: Fork,
genesis_validators_root: Eth2Digest,
voluntary_exit: VoluntaryExit,
pubkey: ValidatorPubKey,
signature: SomeSig): bool =
withTrust(signature): withTrust(signature):
let let signing_root = compute_sync_committee_selection_proof_signing_root(
domain = get_domain( fork, genesis_validators_root, slot, subcommittee_index)
fork, DOMAIN_VOLUNTARY_EXIT, voluntary_exit.epoch, genesis_validators_root)
signing_root = compute_signing_root(voluntary_exit, domain)
blsVerify(pubkey, signing_root.data, signature) blsVerify(pubkey, signing_root.data, signature)
func get_sync_committee_message_signature*(fork: Fork, # https://github.com/ethereum/consensus-specs/blob/v1.1.6/specs/altair/validator.md#signature
genesis_validators_root: Eth2Digest, slot: Slot, func compute_contribution_and_proof_signing_root*(
block_root: Eth2Digest, privkey: ValidatorPrivKey): CookedSig = fork: Fork, genesis_validators_root: Eth2Digest,
let signing_root = sync_committee_msg_signing_root(fork, slot.epoch, msg: ContributionAndProof): Eth2Digest =
genesis_validators_root, block_root) let domain = get_domain(fork, DOMAIN_CONTRIBUTION_AND_PROOF,
blsSign(privkey, signing_root.data) msg.contribution.slot.epoch,
genesis_validators_root)
compute_signing_root(msg, domain)
func get_sync_aggregator_selection_data_signature*(fork: Fork, proc get_contribution_and_proof_signature*(
genesis_validators_root: Eth2Digest, slot: Slot, fork: Fork, genesis_validators_root: Eth2Digest,
subcommittee_index: uint64, msg: ContributionAndProof,
privkey: ValidatorPrivKey): CookedSig = privkey: ValidatorPrivKey): CookedSig =
let signing_root = sync_committee_selection_proof_signing_root(fork, let signing_root = compute_contribution_and_proof_signing_root(
genesis_validators_root, slot, subcommittee_index) fork, genesis_validators_root, msg)
blsSign(privkey, signing_root.data) blsSign(privkey, signing_root.data)
proc get_sync_committee_contribution_and_proof_signature*(fork: Fork,
genesis_validators_root: Eth2Digest, msg: ContributionAndProof,
privkey: ValidatorPrivKey): CookedSig =
let signing_root = contribution_and_proof_signing_root(fork,
genesis_validators_root, msg)
blsSign(privkey, signing_root.data)
proc verify_sync_committee_message_signature*(
epoch: Epoch,
beacon_block_root: Eth2Digest,
fork: Fork,
genesis_validators_root: Eth2Digest,
pubkey: CookedPubKey,
signature: CookedSig): bool =
let
domain = get_domain(
fork, DOMAIN_SYNC_COMMITTEE, epoch, genesis_validators_root)
signing_root = compute_signing_root(beacon_block_root, domain)
blsVerify(pubkey, signing_root.data, signature)
# https://github.com/ethereum/consensus-specs/blob/v1.1.6/specs/altair/validator.md#aggregation-selection # https://github.com/ethereum/consensus-specs/blob/v1.1.6/specs/altair/validator.md#aggregation-selection
proc is_sync_committee_aggregator*(signature: ValidatorSig): bool = proc is_sync_committee_aggregator*(signature: ValidatorSig): bool =
@ -292,30 +322,11 @@ proc is_sync_committee_aggregator*(signature: ValidatorSig): bool =
modulo = max(1'u64, (SYNC_COMMITTEE_SIZE div SYNC_COMMITTEE_SUBNET_COUNT) div TARGET_AGGREGATORS_PER_SYNC_SUBCOMMITTEE) modulo = max(1'u64, (SYNC_COMMITTEE_SIZE div SYNC_COMMITTEE_SUBNET_COUNT) div TARGET_AGGREGATORS_PER_SYNC_SUBCOMMITTEE)
bytes_to_uint64(signatureDigest.data.toOpenArray(0, 7)) mod modulo == 0 bytes_to_uint64(signatureDigest.data.toOpenArray(0, 7)) mod modulo == 0
proc verify_signed_contribution_and_proof_signature*( proc verify_contribution_and_proof_signature*(
msg: SignedContributionAndProof, fork: Fork, genesis_validators_root: Eth2Digest,
fork: Fork,
genesis_validators_root: Eth2Digest,
pubkey: ValidatorPubKey | CookedPubKey): bool =
let
domain = get_domain(
fork, DOMAIN_CONTRIBUTION_AND_PROOF, msg.message.contribution.slot.epoch, genesis_validators_root)
signing_root = compute_signing_root(msg.message, domain)
blsVerify(pubkey, signing_root.data, msg.signature)
proc verify_selection_proof_signature*(
msg: ContributionAndProof, msg: ContributionAndProof,
fork: Fork, pubkey: ValidatorPubKey | CookedPubKey, signature: SomeSig): bool =
genesis_validators_root: Eth2Digest, let signing_root = compute_contribution_and_proof_signing_root(
pubkey: ValidatorPubKey | CookedPubKey): bool = fork, genesis_validators_root, msg)
let
slot = msg.contribution.slot
domain = get_domain(
fork, DOMAIN_SYNC_COMMITTEE_SELECTION_PROOF, slot.epoch, genesis_validators_root)
signing_data = SyncAggregatorSelectionData(
slot: slot,
subcommittee_index: msg.contribution.subcommittee_index)
signing_root = compute_signing_root(signing_data, domain)
blsVerify(pubkey, signing_root.data, msg.selection_proof) blsVerify(pubkey, signing_root.data, signature)

View File

@ -7,6 +7,11 @@
{.push raises: [Defect].} {.push raises: [Defect].}
## This module contains signature verification helpers corresponding to those
## in signatures.nim, for use with signature sets / batch signature verification
## The functions follow the same structure and use the same arguments, except
## that the flow is split into separate collection and verification steps.
import import
# Status lib # Status lib
blscurve, blscurve,
@ -14,13 +19,10 @@ import
taskpools, taskpools,
bearssl, bearssl,
# Internal # Internal
"."/[helpers, beaconstate, forks], "."/[helpers, beaconstate, forks, signatures],
"."/datatypes/[altair, merge, phase0] "."/datatypes/[altair, merge, phase0]
# Otherwise, error. export results, altair, phase0, taskpools, bearssl, signatures
import chronicles
export altair, phase0, taskpools, bearssl
type type
TaskPoolPtr* = TaskPool TaskPoolPtr* = TaskPool
@ -53,35 +55,20 @@ template loadOrExit(signature: ValidatorSig, error: cstring):
return err(error) # this exits the calling scope, as templates are inlined. return err(error) # this exits the calling scope, as templates are inlined.
sig.unsafeGet() sig.unsafeGet()
func addSignatureSet[T]( func addSignatureSet(
sigs: var seq[SignatureSet], sigs: var seq[SignatureSet], pubkey: CookedPubKey, signing_root: Eth2Digest,
pubkey: CookedPubKey, signature: CookedSig) =
sszObj: T,
signature: CookedSig,
fork: Fork,
genesis_validators_root: Eth2Digest,
epoch: Epoch,
domain: DomainType) =
## Add a new signature set triplet (pubkey, message, signature) ## Add a new signature set triplet (pubkey, message, signature)
## to a collection of signature sets for batch verification. ## to a collection of signature sets for batch verification.
## Can return false if `signature` wasn't deserialized to a valid BLS signature.
let signing_root = compute_signing_root(
sszObj,
get_domain(
fork, domain,
epoch,
genesis_validators_root
)
).data
sigs.add(( sigs.add((
blscurve.PublicKey(pubkey), blscurve.PublicKey(pubkey),
signing_root, signing_root.data,
blscurve.Signature(signature) blscurve.Signature(signature)
)) ))
proc aggregateAttesters( proc aggregateAttesters(
validatorIndices: openArray[uint64], validatorIndices: openArray[uint64|ValidatorIndex],
validatorKeys: auto, validatorKeys: auto,
): Result[CookedPubKey, cstring] = ): Result[CookedPubKey, cstring] =
if validatorIndices.len == 0: if validatorIndices.len == 0:
@ -108,158 +95,135 @@ proc aggregateAttesters(
ok(finish(attestersAgg)) ok(finish(attestersAgg))
proc addIndexedAttestation( proc aggregateAttesters(
sigs: var seq[SignatureSet], validatorIndices: openArray[uint64|ValidatorIndex],
attestation: IndexedAttestation, bits: auto,
validatorKeys: auto, validatorKeys: auto,
state: ForkedHashedBeaconState, ): Result[CookedPubKey, cstring] =
): Result[void, cstring] = if validatorIndices.len == 0:
## Add an indexed attestation for batched BLS verification # Aggregation spec requires non-empty collection
## purposes # - https://tools.ietf.org/html/draft-irtf-cfrg-bls-signature-04
## This only verifies cryptography, checking that # Eth2 spec requires at least one attesting index in attestation
## the indices are sorted and unique is not checked for example. # - https://github.com/ethereum/consensus-specs/blob/v1.1.5/specs/phase0/beacon-chain.md#is_valid_indexed_attestation
return err("aggregateAttesters: no attesting indices")
let aggPk = var attestersAgg{.noInit.}: AggregatePublicKey
? aggregateAttesters(attestation.attesting_indices.asSeq(), validatorKeys)
sigs.addSignatureSet(
aggPK,
attestation.data,
attestation.signature.loadOrExit(
"addIndexedAttestation: cannot load signature"),
getStateField(state, fork),
getStateField(state, genesis_validators_root),
attestation.data.target.epoch,
DOMAIN_BEACON_ATTESTER)
ok()
proc addAttestation(
sigs: var seq[SignatureSet],
attestation: Attestation,
validatorKeys: auto,
state: ForkedHashedBeaconState,
cache: var StateCache
): Result[void, cstring] =
var inited = false var inited = false
var attestersAgg{.noInit.}: AggregatePublicKey for i in 0..<bits.len:
for valIndex in state.get_attesting_indices( if bits[i]:
attestation.data, let key = validatorKeys.load(validatorIndices[i])
attestation.aggregation_bits, if not key.isSome():
cache return err("aggregateAttesters: invalid attesting index")
): if inited:
if not inited: # first iteration attestersAgg.aggregate(key.get())
attestersAgg.init(validatorKeys.load(valIndex).get())
inited = true
else: else:
attestersAgg.aggregate(validatorKeys.load(valIndex).get()) attestersAgg = AggregatePublicKey.init(key.get)
inited = true
if not inited: if not inited:
# There were no attesters return err("aggregateAttesters:no attesting indices")
return err("addAttestation: no attesting indices")
let attesters = finish(attestersAgg) ok(finish(attestersAgg))
sigs.addSignatureSet(
attesters,
attestation.data,
attestation.signature.loadOrExit(
"addAttestation: cannot load signature"),
getStateField(state, fork),
getStateField(state, genesis_validators_root),
attestation.data.target.epoch,
DOMAIN_BEACON_ATTESTER)
ok()
# Public API # Public API
# ------------------------------------------------------ # ------------------------------------------------------
proc addAttestation*( # See also: verify_slot_signature
proc add_slot_signature*(
sigs: var seq[SignatureSet], sigs: var seq[SignatureSet],
fork: Fork, genesis_validators_root: Eth2Digest, fork: Fork, genesis_validators_root: Eth2Digest, slot: Slot,
epochRef: auto, pubkey: CookedPubKey, signature: CookedSig) =
attestation: Attestation let signing_root = compute_slot_signing_root(
): Result[CookedSig, cstring] = fork, genesis_validators_root, slot)
## Add an attestation for batched BLS verification
## purposes
## This only verifies cryptography
##
## Returns true if the attestation was added to the batching buffer
## Returns false if sanity checks failed (non-empty, keys are valid)
## In that case the seq[SignatureSet] is unmodified
mixin get_attesting_indices, validatorKey
var inited = false sigs.addSignatureSet(pubkey, signing_root, signature)
var attestersAgg{.noInit.}: AggregatePublicKey
for valIndex in epochRef.get_attesting_indices(
attestation.data,
attestation.aggregation_bits):
if not inited: # first iteration
attestersAgg.init(epochRef.validatorKey(valIndex).get())
inited = true
else:
attestersAgg.aggregate(epochRef.validatorKey(valIndex).get())
if not inited: # See also: verify_epoch_signature
# There were no attesters proc add_epoch_signature*(
return err("addAttestation: no attesting indices")
let
attesters = finish(attestersAgg)
cookedSig = attestation.signature.loadOrExit(
"addAttestation: cannot load signature")
sigs.addSignatureSet(
attesters,
attestation.data,
cookedSig,
fork,
genesis_validators_root,
attestation.data.target.epoch,
DOMAIN_BEACON_ATTESTER)
ok(CookedSig(cookedSig))
proc addSlotSignature*(
sigs: var seq[SignatureSet], sigs: var seq[SignatureSet],
fork: Fork, genesis_validators_root: Eth2Digest, fork: Fork, genesis_validators_root: Eth2Digest, epoch: Epoch,
slot: Slot, pubkey: CookedPubKey, signature: CookedSig) =
pubkey: CookedPubKey, let signing_root = compute_epoch_signing_root(
signature: ValidatorSig): Result[void, cstring] = fork, genesis_validators_root, epoch)
let epoch = compute_epoch_at_slot(slot)
sigs.addSignatureSet(
pubkey,
sszObj = slot,
signature.loadOrExit("addSlotSignature: cannot load signature"),
fork,
genesis_validators_root,
epoch,
DOMAIN_SELECTION_PROOF
)
ok() sigs.addSignatureSet(pubkey, signing_root, signature)
proc addAggregateAndProofSignature*( # See also: verify_block_signature
proc add_block_signature*(
sigs: var seq[SignatureSet],
fork: Fork, genesis_validators_root: Eth2Digest, slot: Slot,
blck: Eth2Digest | SomeSomeBeaconBlock | BeaconBlockHeader,
pubkey: CookedPubKey, signature: CookedSig) =
let signing_root = compute_block_signing_root(
fork, genesis_validators_root, slot, blck)
sigs.addSignatureSet(pubkey, signing_root, signature)
# See also: verify_aggregate_and_proof_signature
proc add_aggregate_and_proof_signature*(
sigs: var seq[SignatureSet], sigs: var seq[SignatureSet],
fork: Fork, genesis_validators_root: Eth2Digest, fork: Fork, genesis_validators_root: Eth2Digest,
aggregate_and_proof: AggregateAndProof, aggregate_and_proof: AggregateAndProof,
pubkey: CookedPubKey, pubkey: CookedPubKey, signature: CookedSig) =
signature: ValidatorSig let signing_root = compute_aggregate_and_proof_signing_root(
): Result[void, cstring] = fork, genesis_validators_root, aggregate_and_proof)
let epoch = compute_epoch_at_slot(aggregate_and_proof.aggregate.data.slot) sigs.addSignatureSet(pubkey, signing_root, signature)
sigs.addSignatureSet(
pubkey,
sszObj = aggregate_and_proof,
signature.loadOrExit("addAggregateAndProofSignature: cannot load signature"),
fork,
genesis_validators_root,
epoch,
DOMAIN_AGGREGATE_AND_PROOF
)
ok() # See also: verify_attestation_signature
proc add_attestation_signature*(
sigs: var seq[SignatureSet],
fork: Fork, genesis_validators_root: Eth2Digest,
attestation_data: AttestationData,
pubkey: CookedPubKey, signature: CookedSig) =
let signing_root = compute_attestation_signing_root(
fork, genesis_validators_root, attestation_data)
sigs.addSignatureSet(pubkey, signing_root, signature)
# See also: verify_voluntary_exit_signature
proc add_voluntary_exit_signature*(
sigs: var seq[SignatureSet],
fork: Fork, genesis_validators_root: Eth2Digest,
voluntary_exit: VoluntaryExit,
pubkey: CookedPubKey, signature: CookedSig) =
let signing_root = compute_voluntary_exit_signing_root(
fork, genesis_validators_root, voluntary_exit)
sigs.addSignatureSet(pubkey, signing_root, signature)
# See also: verify_sync_committee_message_signature
proc add_sync_committee_message_signature*(
sigs: var seq[SignatureSet],
fork: Fork, genesis_validators_root: Eth2Digest,
slot: Slot, block_root: Eth2Digest,
pubkey: CookedPubKey, signature: CookedSig) =
let signing_root = compute_sync_committee_message_signing_root(
fork, genesis_validators_root, slot, block_root)
sigs.addSignatureSet(pubkey, signing_root, signature)
# See also: verify_sync_committee_selection_proof
proc add_sync_committee_selection_proof*(
sigs: var seq[SignatureSet],
fork: Fork, genesis_validators_root: Eth2Digest,
slot: Slot, subcommittee_index: uint64,
pubkey: CookedPubKey, signature: CookedSig) =
let signing_root = compute_sync_committee_selection_proof_signing_root(
fork, genesis_validators_root, slot, subcommittee_index)
sigs.addSignatureSet(pubkey, signing_root, signature)
proc add_contribution_and_proof_signature*(
sigs: var seq[SignatureSet],
fork: Fork, genesis_validators_root: Eth2Digest,
msg: ContributionAndProof,
pubkey: CookedPubKey, signature: CookedSig) =
let signing_root = compute_contribution_and_proof_signing_root(
fork, genesis_validators_root, msg)
sigs.addSignatureSet(pubkey, signing_root, signature)
proc collectSignatureSets*( proc collectSignatureSets*(
sigs: var seq[SignatureSet], sigs: var seq[SignatureSet],
@ -267,7 +231,9 @@ proc collectSignatureSets*(
validatorKeys: auto, validatorKeys: auto,
state: ForkedHashedBeaconState, state: ForkedHashedBeaconState,
cache: var StateCache): Result[void, cstring] = cache: var StateCache): Result[void, cstring] =
## Collect all signatures in a single signed block. ## Collect all signature verifications that process_block would normally do
## except deposits, in one go.
##
## This includes ## This includes
## - Block proposer ## - Block proposer
## - Randao Reaveal ## - Randao Reaveal
@ -275,14 +241,19 @@ proc collectSignatureSets*(
## - Attester slashings ## - Attester slashings
## - Attestations ## - Attestations
## - VoluntaryExits ## - VoluntaryExits
## - SyncCommittee (altair+)
## ##
## We do not include deposits as they can be invalid per protocol ## We do not include deposits as they can be invalid while still leaving the
## (secp256k1 signature instead of BLS) ## block valid
# Metadata # Metadata
# ---------------------------------------------------- # ----------------------------------------------------
mixin load mixin load
let
fork = getStateField(state, fork)
genesis_validators_root = getStateField(state, genesis_validators_root)
let let
proposer_index = signed_block.message.proposer_index proposer_index = signed_block.message.proposer_index
proposer_key = validatorKeys.load(proposer_index) proposer_key = validatorKeys.load(proposer_index)
@ -293,27 +264,18 @@ proc collectSignatureSets*(
# 1. Block proposer # 1. Block proposer
# ---------------------------------------------------- # ----------------------------------------------------
sigs.addSignatureSet( sigs.add_block_signature(
proposer_key.get(), fork, genesis_validators_root,
signed_block.root, signed_block.message.slot, signed_block.root,
signed_block.signature.loadOrExit( proposer_key.get(), signed_block.signature.loadOrExit(
"collectSignatureSets: cannot load signature"), "collectSignatureSets: cannot load signature"))
getStateField(state, fork),
getStateField(state, genesis_validators_root),
epoch,
DOMAIN_BEACON_PROPOSER)
# 2. Randao Reveal # 2. Randao Reveal
# ---------------------------------------------------- # ----------------------------------------------------
sigs.addSignatureSet( sigs.add_epoch_signature(
proposer_key.get(), fork, genesis_validators_root, epoch, proposer_key.get(),
epoch,
signed_block.message.body.randao_reveal.loadOrExit( signed_block.message.body.randao_reveal.loadOrExit(
"collectSignatureSets: cannot load randao"), "collectSignatureSets: cannot load randao"))
getStateField(state, fork),
getStateField(state, genesis_validators_root),
epoch,
DOMAIN_RANDAO)
# 3. Proposer slashings # 3. Proposer slashings
# ---------------------------------------------------- # ----------------------------------------------------
@ -332,41 +294,28 @@ proc collectSignatureSets*(
# Proposed block 1 # Proposed block 1
block: block:
let let
header_1 = slashing.signed_header_1 header = slashing.signed_header_1
key_1 = validatorKeys.load(header_1.message.proposer_index) key = validatorKeys.load(header.message.proposer_index)
if not key_1.isSome(): if not key.isSome():
return err("collectSignatureSets: invalid slashing proposer index 1") return err("collectSignatureSets: invalid slashing proposer index 1")
let epoch1 = header_1.message.slot.compute_epoch_at_slot() sigs.add_block_signature(
sigs.addSignatureSet( fork, genesis_validators_root, header.message.slot, header.message,
key_1.get(), key.get(), header.signature.loadOrExit(
header_1.message, "collectSignatureSets: cannot load proposer slashing 1 signature"))
header_1.signature.loadOrExit(
"collectSignatureSets: cannot load proposer slashing 1 signature"),
getStateField(state, fork),
getStateField(state, genesis_validators_root),
epoch1,
DOMAIN_BEACON_PROPOSER
)
# Conflicting block 2 # Conflicting block 2
block: block:
let let
header_2 = slashing.signed_header_2 header = slashing.signed_header_2
key_2 = validatorKeys.load(header_2.message.proposer_index) key = validatorKeys.load(header.message.proposer_index)
if not key_2.isSome(): if not key.isSome():
return err("collectSignatureSets: invalid slashing proposer index 2") return err("collectSignatureSets: invalid slashing proposer index 2")
let epoch2 = header_2.message.slot.compute_epoch_at_slot()
sigs.addSignatureSet( sigs.add_block_signature(
key_2.get(), fork, genesis_validators_root, header.message.slot, header.message,
header_2.message, key.get(), header.signature.loadOrExit(
header_2.signature.loadOrExit( "collectSignatureSets: cannot load proposer slashing 2 signature"))
"collectSignatureSets: cannot load proposer slashing 2 signature"),
getStateField(state, fork),
getStateField(state, genesis_validators_root),
epoch2,
DOMAIN_BEACON_PROPOSER
)
# 4. Attester slashings # 4. Attester slashings
# ---------------------------------------------------- # ----------------------------------------------------
@ -383,10 +332,22 @@ proc collectSignatureSets*(
template slashing: untyped = signed_block.message.body.attester_slashings[i] template slashing: untyped = signed_block.message.body.attester_slashings[i]
# Attestation 1 # Attestation 1
? sigs.addIndexedAttestation(slashing.attestation_1, validatorKeys, state) block:
let
key = ? aggregateAttesters(
slashing.attestation_1.attesting_indices.asSeq(), validatorKeys)
sig = slashing.attestation_1.signature.loadOrExit("")
sigs.add_attestation_signature(
fork, genesis_validators_root, slashing.attestation_1.data, key, sig)
# Conflicting attestation 2 # Conflicting attestation 2
? sigs.addIndexedAttestation(slashing.attestation_2, validatorKeys, state) block:
let
key = ? aggregateAttesters(
slashing.attestation_2.attesting_indices.asSeq(), validatorKeys)
sig = slashing.attestation_2.signature.loadOrExit("")
sigs.add_attestation_signature(
fork, genesis_validators_root, slashing.attestation_2.data, key, sig)
# 5. Attestations # 5. Attestations
# ---------------------------------------------------- # ----------------------------------------------------
@ -398,9 +359,17 @@ proc collectSignatureSets*(
# don't use "items" for iterating over large type # don't use "items" for iterating over large type
# due to https://github.com/nim-lang/Nim/issues/14421 # due to https://github.com/nim-lang/Nim/issues/14421
# fixed in 1.4.2 # fixed in 1.4.2
? sigs.addAttestation( template attestation: untyped = signed_block.message.body.attestations[i]
signed_block.message.body.attestations[i],
validatorKeys, state, cache) let
key = ? aggregateAttesters(
get_attesting_indices(
state, attestation.data, attestation.aggregation_bits, cache),
validatorKeys)
sig = attestation.signature.loadOrExit("")
sigs.add_attestation_signature(
fork, genesis_validators_root, attestation.data, key, sig)
# 6. VoluntaryExits # 6. VoluntaryExits
# ---------------------------------------------------- # ----------------------------------------------------
@ -417,15 +386,10 @@ proc collectSignatureSets*(
if not key.isSome(): if not key.isSome():
return err("collectSignatureSets: invalid voluntary exit") return err("collectSignatureSets: invalid voluntary exit")
sigs.addSignatureSet( sigs.add_voluntary_exit_signature(
key.get(), fork, genesis_validators_root, volex.message, key.get(),
volex.message,
volex.signature.loadOrExit( volex.signature.loadOrExit(
"collectSignatureSets: cannot load voluntary exit signature"), "collectSignatureSets: cannot load voluntary exit signature"))
getStateField(state, fork),
getStateField(state, genesis_validators_root),
volex.message.epoch,
DOMAIN_VOLUNTARY_EXIT)
block: block:
# 7. SyncAggregate # 7. SyncAggregate
@ -434,42 +398,25 @@ proc collectSignatureSets*(
when stateFork >= BeaconStateFork.Altair and when stateFork >= BeaconStateFork.Altair and
(signed_block is altair.SignedBeaconBlock or (signed_block is altair.SignedBeaconBlock or
signed_block is merge.SignedBeaconBlock): signed_block is merge.SignedBeaconBlock):
if signed_block.message.body.sync_aggregate.sync_committee_bits.countOnes() == 0:
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:
let let
current_sync_committee = current_sync_committee =
state.data.get_sync_committee_cache(cache).current_sync_committee state.data.get_sync_committee_cache(cache).current_sync_committee
var inited = false
var attestersAgg{.noInit.}: AggregatePublicKey
for i in 0 ..< current_sync_committee.len:
if signed_block.message.body.sync_aggregate.sync_committee_bits[i]:
let key = validatorKeys.load(current_sync_committee[i])
if not key.isSome():
return err("Invalid key cache")
if not inited: # first iteration
attestersAgg.init(key.get())
inited = true
else:
attestersAgg.aggregate(key.get())
if not inited:
if signed_block.message.body.sync_aggregate.sync_committee_signature !=
default(CookedSig).toValidatorSig():
return err("process_sync_aggregate: empty sync aggregates need signature of point at infinity")
else:
let
attesters = finish(attestersAgg)
previous_slot = max(state.data.slot, Slot(1)) - 1 previous_slot = max(state.data.slot, Slot(1)) - 1
beacon_block_root = get_block_root_at_slot(state.data, previous_slot)
pubkey = ? aggregateAttesters(
current_sync_committee,
signed_block.message.body.sync_aggregate.sync_committee_bits,
validatorKeys)
sigs.addSignatureSet( sigs.add_sync_committee_message_signature(
attesters, fork, genesis_validators_root, previous_slot, beacon_block_root,
get_block_root_at_slot(state.data, previous_slot), pubkey,
signed_block.message.body.sync_aggregate.sync_committee_signature.loadOrExit( signed_block.message.body.sync_aggregate.sync_committee_signature.loadOrExit(
"process_sync_aggregate: cannot load signature"), "collectSignatureSets: cannot load signature"))
state.data.fork,
state.data.genesis_validators_root,
previous_slot.epoch,
DOMAIN_SYNC_COMMITTEE)
ok() ok()

View File

@ -428,15 +428,11 @@ proc process_operations(cfg: RuntimeConfig,
# https://github.com/ethereum/consensus-specs/blob/v1.1.0-alpha.6/specs/altair/beacon-chain.md#sync-committee-processing # https://github.com/ethereum/consensus-specs/blob/v1.1.0-alpha.6/specs/altair/beacon-chain.md#sync-committee-processing
proc process_sync_aggregate*( proc process_sync_aggregate*(
state: var (altair.BeaconState | merge.BeaconState), state: var (altair.BeaconState | merge.BeaconState),
aggregate: SomeSyncAggregate, total_active_balance: Gwei, cache: var StateCache): aggregate: SomeSyncAggregate, total_active_balance: Gwei,
cache: var StateCache):
Result[void, cstring] = Result[void, cstring] =
# Verify sync committee aggregate signature signing over the previous slot # Verify sync committee aggregate signature signing over the previous slot
# block root # block root
let
previous_slot = max(state.slot, Slot(1)) - 1
domain = get_domain(state, DOMAIN_SYNC_COMMITTEE, compute_epoch_at_slot(previous_slot))
signing_root = compute_signing_root(get_block_root_at_slot(state, previous_slot), domain)
when aggregate.sync_committee_signature isnot TrustedSig: when aggregate.sync_committee_signature isnot TrustedSig:
var participant_pubkeys: seq[ValidatorPubKey] var participant_pubkeys: seq[ValidatorPubKey]
for i in 0 ..< state.current_sync_committee.pubkeys.len: for i in 0 ..< state.current_sync_committee.pubkeys.len:
@ -445,13 +441,18 @@ proc process_sync_aggregate*(
# p2p-interface message validators check for empty sync committees, so it # p2p-interface message validators check for empty sync committees, so it
# shouldn't run except as part of test suite. # shouldn't run except as part of test suite.
if participant_pubkeys.len == 0 and if participant_pubkeys.len == 0:
aggregate.sync_committee_signature != default(CookedSig).toValidatorSig(): if aggregate.sync_committee_signature != ValidatorSig.infinity():
return err("process_sync_aggregate: empty sync aggregates need signature of point at infinity") return err("process_sync_aggregate: empty sync aggregates need signature of point at infinity")
else:
# Empty participants allowed # Empty participants allowed
if participant_pubkeys.len > 0 and not blsFastAggregateVerify( let
participant_pubkeys, signing_root.data, aggregate.sync_committee_signature): previous_slot = max(state.slot, Slot(1)) - 1
beacon_block_root = get_block_root_at_slot(state, previous_slot)
if not verify_sync_committee_signature(
state.fork, state.genesis_validators_root, previous_slot,
beacon_block_root, participant_pubkeys,
aggregate.sync_committee_signature):
return err("process_sync_aggregate: invalid signature") return err("process_sync_aggregate: invalid signature")
# Compute participant and proposer rewards # Compute participant and proposer rewards

View File

@ -25,8 +25,8 @@ proc serveAttestation(service: AttestationServiceRef, adata: AttestationData,
# TODO: signing_root is recomputed in signBlockProposal just after, # TODO: signing_root is recomputed in signBlockProposal just after,
# but not for locally attached validators. # but not for locally attached validators.
let signingRoot = let signingRoot =
compute_attestation_root(fork, vc.beaconGenesis.genesis_validators_root, compute_attestation_signing_root(
adata) fork, vc.beaconGenesis.genesis_validators_root, adata)
let attestationRoot = adata.hash_tree_root() let attestationRoot = adata.hash_tree_root()
let vindex = validator.index.get() let vindex = validator.index.get()

View File

@ -45,7 +45,7 @@ proc publishBlock(vc: ValidatorClientRef, currentSlot, slot: Slot,
let blockRoot = withBlck(beaconBlock): hash_tree_root(blck) let blockRoot = withBlck(beaconBlock): hash_tree_root(blck)
# TODO: signing_root is recomputed in signBlockProposal just after # TODO: signing_root is recomputed in signBlockProposal just after
let signing_root = compute_block_root(fork, genesisRoot, slot, let signing_root = compute_block_signing_root(fork, genesisRoot, slot,
blockRoot) blockRoot)
let notSlashable = vc.attachedValidators let notSlashable = vc.attachedValidators
.slashingProtection .slashingProtection

View File

@ -240,7 +240,7 @@ proc sendSyncCommitteeMessage*(
# validation will also register the message with the sync committee # validation will also register the message with the sync committee
# message pool. Notably, although libp2p calls the data handler for # message pool. Notably, although libp2p calls the data handler for
# any subscription on the subnet topic, it does not perform validation. # any subscription on the subnet topic, it does not perform validation.
let res = node.processor.syncCommitteeMessageValidator(msg, subcommitteeIdx, let res = await node.processor.syncCommitteeMessageValidator(msg, subcommitteeIdx,
checkSignature) checkSignature)
return return
if res.isGoodForSending: if res.isGoodForSending:
@ -338,7 +338,7 @@ proc sendSyncCommitteeContribution*(
node: BeaconNode, node: BeaconNode,
msg: SignedContributionAndProof, msg: SignedContributionAndProof,
checkSignature: bool): Future[SendResult] {.async.} = checkSignature: bool): Future[SendResult] {.async.} =
let res = node.processor.contributionValidator( let res = await node.processor.contributionValidator(
msg, checkSignature) msg, checkSignature)
return return
@ -501,7 +501,7 @@ proc proposeBlock(node: BeaconNode,
withBlck(forkedBlck): withBlck(forkedBlck):
let let
blockRoot = hash_tree_root(blck) blockRoot = hash_tree_root(blck)
signing_root = compute_block_root( signing_root = compute_block_signing_root(
fork, genesis_validators_root, slot, blockRoot) fork, genesis_validators_root, slot, blockRoot)
notSlashable = node.attachedValidators notSlashable = node.attachedValidators
@ -620,7 +620,7 @@ proc handleAttestations(node: BeaconNode, head: BlockRef, slot: Slot) =
let let
data = makeAttestationData(epochRef, attestationHead, committee_index) data = makeAttestationData(epochRef, attestationHead, committee_index)
# TODO signing_root is recomputed in produceAndSignAttestation/signAttestation just after # TODO signing_root is recomputed in produceAndSignAttestation/signAttestation just after
signing_root = compute_attestation_root( signing_root = compute_attestation_signing_root(
fork, genesis_validators_root, data) fork, genesis_validators_root, data)
registered = node.attachedValidators registered = node.attachedValidators
.slashingProtection .slashingProtection
@ -652,9 +652,9 @@ proc createAndSendSyncCommitteeMessage(node: BeaconNode,
genesisValidatorsRoot = node.dag.genesisValidatorsRoot genesisValidatorsRoot = node.dag.genesisValidatorsRoot
msg = msg =
block: block:
let res = await signSyncCommitteeMessage(validator, slot, fork, let res = await signSyncCommitteeMessage(validator, fork,
genesisValidatorsRoot, genesisValidatorsRoot,
head.root) slot, head.root)
if res.isErr(): if res.isErr():
error "Unable to sign committee message using remote signer", error "Unable to sign committee message using remote signer",
validator = shortLog(validator), slot = slot, validator = shortLog(validator), slot = slot,

View File

@ -319,19 +319,20 @@ proc signAggregateAndProof*(v: AttachedValidator,
# https://github.com/ethereum/consensus-specs/blob/v1.1.6/specs/altair/validator.md#prepare-sync-committee-message # https://github.com/ethereum/consensus-specs/blob/v1.1.6/specs/altair/validator.md#prepare-sync-committee-message
proc signSyncCommitteeMessage*(v: AttachedValidator, proc signSyncCommitteeMessage*(v: AttachedValidator,
slot: Slot, fork: Fork, fork: Fork,
genesis_validators_root: Eth2Digest, genesis_validators_root: Eth2Digest,
block_root: Eth2Digest slot: Slot,
beacon_block_root: Eth2Digest
): Future[SyncCommitteeMessageResult] {.async.} = ): Future[SyncCommitteeMessageResult] {.async.} =
let signature = let signature =
case v.kind case v.kind
of ValidatorKind.Local: of ValidatorKind.Local:
let signing_root = sync_committee_msg_signing_root( get_sync_committee_message_signature(
fork, slot.epoch, genesis_validators_root, block_root) fork, genesis_validators_root, slot, beacon_block_root,
blsSign(v.data.privateKey, signing_root.data).toValidatorSig() v.data.privateKey).toValidatorSig()
of ValidatorKind.Remote: of ValidatorKind.Remote:
let res = await signWithRemoteValidator(v, fork, genesis_validators_root, let res = await signWithRemoteValidator(v, fork, genesis_validators_root,
slot, block_root) slot, beacon_block_root)
if res.isErr(): if res.isErr():
return SyncCommitteeMessageResult.err(res.error()) return SyncCommitteeMessageResult.err(res.error())
res.get().toValidatorSig() res.get().toValidatorSig()
@ -340,7 +341,7 @@ proc signSyncCommitteeMessage*(v: AttachedValidator,
SyncCommitteeMessageResult.ok( SyncCommitteeMessageResult.ok(
SyncCommitteeMessage( SyncCommitteeMessage(
slot: slot, slot: slot,
beacon_block_root: block_root, beacon_block_root: beacon_block_root,
validator_index: uint64(v.index.get()), validator_index: uint64(v.index.get()),
signature: signature signature: signature
) )
@ -356,11 +357,9 @@ proc getSyncCommitteeSelectionProof*(v: AttachedValidator,
return return
case v.kind case v.kind
of ValidatorKind.Local: of ValidatorKind.Local:
let signing_root = sync_committee_selection_proof_signing_root( SignatureResult.ok(get_sync_committee_selection_proof(
fork, genesis_validators_root, slot, subcommittee_index) fork, genesis_validators_root, slot, subcommittee_index,
SignatureResult.ok( v.data.privateKey).toValidatorSig())
blsSign(v.data.privateKey, signing_root.data).toValidatorSig()
)
of ValidatorKind.Remote: of ValidatorKind.Remote:
let res = await signWithRemoteValidator(v, fork, genesis_validators_root, let res = await signWithRemoteValidator(v, fork, genesis_validators_root,
slot, subcommittee_index) slot, subcommittee_index)
@ -376,9 +375,8 @@ proc sign*(v: AttachedValidator, msg: ref SignedContributionAndProof,
msg.signature = msg.signature =
case v.kind case v.kind
of ValidatorKind.Local: of ValidatorKind.Local:
let signing_root = contribution_and_proof_signing_root( get_contribution_and_proof_signature(
fork, genesis_validators_root, msg.message) fork, genesis_validators_root, msg.message, v.data.privateKey).toValidatorSig()
blsSign(v.data.privateKey, signing_root.data).toValidatorSig()
of ValidatorKind.Remote: of ValidatorKind.Remote:
let res = await signWithRemoteValidator(v, fork, genesis_validators_root, let res = await signWithRemoteValidator(v, fork, genesis_validators_root,
msg.message) msg.message)

View File

@ -26,7 +26,7 @@ import
../beacon_chain/[beacon_chain_db, beacon_clock], ../beacon_chain/[beacon_chain_db, beacon_clock],
../beacon_chain/eth1/eth1_monitor, ../beacon_chain/eth1/eth1_monitor,
../beacon_chain/validators/validator_pool, ../beacon_chain/validators/validator_pool,
../beacon_chain/gossip_processing/gossip_validation, ../beacon_chain/gossip_processing/[batch_validation, gossip_validation],
../beacon_chain/consensus_object_pools/[blockchain_dag, block_quarantine, ../beacon_chain/consensus_object_pools/[blockchain_dag, block_quarantine,
block_clearance, attestation_pool, block_clearance, attestation_pool,
sync_committee_msg_pool], sync_committee_msg_pool],
@ -90,6 +90,8 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
verifier = BatchVerifier(rng: keys.newRng(), taskpool: taskpool) verifier = BatchVerifier(rng: keys.newRng(), taskpool: taskpool)
quarantine = newClone(Quarantine.init()) quarantine = newClone(Quarantine.init())
attPool = AttestationPool.init(dag, quarantine) attPool = AttestationPool.init(dag, quarantine)
batchCrypto = BatchCrypto.new(
keys.newRng(), eager = proc(): bool = true, taskpool)
syncCommitteePool = newClone SyncCommitteeMsgPool.init() syncCommitteePool = newClone SyncCommitteeMsgPool.init()
timers: array[Timers, RunningStat] timers: array[Timers, RunningStat]
attesters: RunningStat attesters: RunningStat
@ -145,10 +147,8 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
let let
syncCommittee = @(dag.syncCommitteeParticipants(slot + 1)) syncCommittee = @(dag.syncCommitteeParticipants(slot + 1))
genesisValidatorsRoot = dag.genesisValidatorsRoot genesis_validators_root = dag.genesisValidatorsRoot
fork = dag.forkAtEpoch(slot.epoch) fork = dag.forkAtEpoch(slot.epoch)
signingRoot = sync_committee_msg_signing_root(
fork, slot.epoch, genesisValidatorsRoot, dag.head.root)
messagesTime = slot.toBeaconTime(seconds(SECONDS_PER_SLOT div 3)) messagesTime = slot.toBeaconTime(seconds(SECONDS_PER_SLOT div 3))
contributionsTime = slot.toBeaconTime(seconds(2 * SECONDS_PER_SLOT div 3)) contributionsTime = slot.toBeaconTime(seconds(2 * SECONDS_PER_SLOT div 3))
@ -160,15 +160,17 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
continue continue
let let
validarorPrivKey = MockPrivKeys[validatorIdx] validatorPrivKey = MockPrivKeys[validatorIdx]
signature = blsSign(validarorPrivKey, signingRoot.data) signature = get_sync_committee_message_signature(
fork, genesis_validators_root, slot, dag.head.root, validatorPrivKey)
msg = SyncCommitteeMessage( msg = SyncCommitteeMessage(
slot: slot, slot: slot,
beacon_block_root: dag.head.root, beacon_block_root: dag.head.root,
validator_index: uint64 validatorIdx, validator_index: uint64 validatorIdx,
signature: signature.toValidatorSig) signature: signature.toValidatorSig)
let res = dag.validateSyncCommitteeMessage( let res = waitFor dag.validateSyncCommitteeMessage(
batchCrypto,
syncCommitteePool[], syncCommitteePool[],
msg, msg,
subcommitteeIdx, subcommitteeIdx,
@ -188,11 +190,9 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
positions) positions)
let let
selectionProofSigningRoot = selectionProofSig = get_sync_committee_selection_proof(
sync_committee_selection_proof_signing_root( fork, genesis_validators_root, slot, uint64 subcommitteeIdx,
fork, genesisValidatorsRoot, slot, uint64 subcommitteeIdx) validatorPrivKey).toValidatorSig
selectionProofSig = blsSign(
validarorPrivKey, selectionProofSigningRoot.data).toValidatorSig
if is_sync_committee_aggregator(selectionProofSig): if is_sync_committee_aggregator(selectionProofSig):
aggregators.add Aggregator( aggregators.add Aggregator(
@ -212,18 +212,18 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
contribution: contribution, contribution: contribution,
selection_proof: aggregator.selectionProof) selection_proof: aggregator.selectionProof)
signingRoot = contribution_and_proof_signing_root( validatorPrivKey =
fork, genesisValidatorsRoot, contributionAndProof)
validarorPrivKey =
MockPrivKeys[aggregator.validatorIdx.ValidatorIndex] MockPrivKeys[aggregator.validatorIdx.ValidatorIndex]
signedContributionAndProof = SignedContributionAndProof( signedContributionAndProof = SignedContributionAndProof(
message: contributionAndProof, message: contributionAndProof,
signature: blsSign(validarorPrivKey, signingRoot.data).toValidatorSig) signature: get_contribution_and_proof_signature(
fork, genesis_validators_root, contributionAndProof,
validatorPrivKey).toValidatorSig)
res = dag.validateContribution( res = waitFor dag.validateContribution(
syncCommitteePool[], batchCrypto,
syncCommitteePool,
signedContributionAndProof, signedContributionAndProof,
contributionsTime, contributionsTime,
false) false)

View File

@ -34,8 +34,6 @@ proc compute_aggregate_sync_committee_signature(
root = root =
if block_root != ZERO_HASH: block_root if block_root != ZERO_HASH: block_root
else: mockBlockForNextSlot(forked).altairData.message.parent_root else: mockBlockForNextSlot(forked).altairData.message.parent_root
signing_root = sync_committee_msg_signing_root(
state.fork, state.slot.epoch, state.genesis_validators_root, root)
var var
aggregateSig {.noInit.}: AggregateSignature aggregateSig {.noInit.}: AggregateSignature
@ -43,7 +41,8 @@ proc compute_aggregate_sync_committee_signature(
for validator_index in participants: for validator_index in participants:
let let
privkey = MockPrivKeys[validator_index] privkey = MockPrivKeys[validator_index]
signature = blsSign(privkey, signing_root.data) signature = get_sync_committee_message_signature(
state.fork, state.genesis_validators_root, state.slot, root, privkey)
if not initialized: if not initialized:
initialized = true initialized = true
aggregateSig.init(signature) aggregateSig.init(signature)

View File

@ -9,7 +9,7 @@
import import
# Standard library # Standard library
std/[json, options, os, strutils, tables], std/[json, os, strutils, tables],
# Status libraries # Status libraries
stew/[results, endians2], chronicles, stew/[results, endians2], chronicles,
eth/keys, taskpools, eth/keys, taskpools,

View File

@ -180,11 +180,15 @@ suite "Gossip validation - Extra": # Not based on preset config
var cfg = defaultRuntimeConfig var cfg = defaultRuntimeConfig
cfg.ALTAIR_FORK_EPOCH = (GENESIS_EPOCH + 1).Epoch cfg.ALTAIR_FORK_EPOCH = (GENESIS_EPOCH + 1).Epoch
cfg cfg
dag = block: taskpool = Taskpool.new()
quarantine = newClone(Quarantine.init())
batchCrypto = BatchCrypto.new(keys.newRng(), eager = proc(): bool = false, taskpool)
var var
dag = ChainDAGRef.init(
cfg, makeTestDB(num_validators), {})
verifier = BatchVerifier(rng: keys.newRng(), taskpool: Taskpool.new()) verifier = BatchVerifier(rng: keys.newRng(), taskpool: Taskpool.new())
dag = block:
let
dag = ChainDAGRef.init(cfg, makeTestDB(num_validators), {})
quarantine = newClone(Quarantine.init()) quarantine = newClone(Quarantine.init())
var cache = StateCache() var cache = StateCache()
for blck in makeTestBlocks( for blck in makeTestBlocks(
@ -217,13 +221,13 @@ suite "Gossip validation - Extra": # Not based on preset config
validator = AttachedValidator(pubKey: pubkey, validator = AttachedValidator(pubKey: pubkey,
kind: ValidatorKind.Local, data: privateItem, index: some(index)) kind: ValidatorKind.Local, data: privateItem, index: some(index))
resMsg = waitFor signSyncCommitteeMessage( resMsg = waitFor signSyncCommitteeMessage(
validator, slot, validator, state[].data.fork, state[].data.genesis_validators_root, slot,
state[].data.fork, state[].data.genesis_validators_root, state[].root) state[].root)
msg = resMsg.get() msg = resMsg.get()
syncCommitteeMsgPool = newClone(SyncCommitteeMsgPool.init()) syncCommitteeMsgPool = newClone(SyncCommitteeMsgPool.init())
res = validateSyncCommitteeMessage( res = waitFor validateSyncCommitteeMessage(
dag, syncCommitteeMsgPool[], msg, subcommitteeIdx, dag, batchCrypto, syncCommitteeMsgPool[], msg, subcommitteeIdx,
slot.toBeaconTime(), true) slot.toBeaconTime(), true)
(positions, cookedSig) = res.get() (positions, cookedSig) = res.get()
@ -257,5 +261,5 @@ suite "Gossip validation - Extra": # Not based on preset config
# Same message twice should be ignored # Same message twice should be ignored
validateSyncCommitteeMessage( validateSyncCommitteeMessage(
dag, syncCommitteeMsgPool[], msg, subcommitteeIdx, dag, batchCrypto, syncCommitteeMsgPool[], msg, subcommitteeIdx,
state[].data.slot.toBeaconTime(), true).isErr() state[].data.slot.toBeaconTime(), true).waitFor().isErr()

View File

@ -10,7 +10,7 @@
import import
std/random, std/random,
unittest2, unittest2,
../beacon_chain/spec/[crypto, signatures], ../beacon_chain/spec/[crypto, helpers, signatures],
./testblockutil ./testblockutil
suite "Message signatures": suite "Message signatures":
@ -175,69 +175,69 @@ suite "Message signatures":
test "Sync committee message signatures": test "Sync committee message signatures":
let let
epoch = default(Epoch) slot = default(Slot)
epoch = slot.epoch
block_root = default(Eth2Digest) block_root = default(Eth2Digest)
check: check:
# Matching public/private keys and genesis validator roots # Matching public/private keys and genesis validator roots
verify_sync_committee_message_signature( verify_sync_committee_message_signature(
epoch, block_root, fork0, genesis_validators_root0, load(pubkey0).get, fork0, genesis_validators_root0, slot, block_root, load(pubkey0).get,
blsSign(privkey0, sync_committee_msg_signing_root( get_sync_committee_message_signature(
fork0, epoch, genesis_validators_root0, block_root).data)) fork0, genesis_validators_root0, slot, block_root, privkey0).toValidatorSig())
# Mismatched public/private keys # Mismatched public/private keys
not verify_sync_committee_message_signature( not verify_sync_committee_message_signature(
epoch, block_root, fork0, genesis_validators_root0, load(pubkey0).get, fork0, genesis_validators_root0, slot, block_root, load(pubkey0).get,
blsSign(privkey1, sync_committee_msg_signing_root( get_sync_committee_message_signature(
fork0, epoch, genesis_validators_root0, block_root).data)) fork0, genesis_validators_root0, slot, block_root, privkey1).toValidatorSig())
# Mismatched forks # Mismatched forks
not verify_sync_committee_message_signature( not verify_sync_committee_message_signature(
epoch, block_root, fork0, genesis_validators_root0, load(pubkey0).get, fork0, genesis_validators_root0, slot, block_root, load(pubkey0).get,
blsSign(privkey0, sync_committee_msg_signing_root( get_sync_committee_message_signature(
fork1, epoch, genesis_validators_root0, block_root).data)) fork1, genesis_validators_root0, slot, block_root, privkey0).toValidatorSig())
# Mismatched genesis validator roots # Mismatched genesis validator roots
not verify_sync_committee_message_signature( not verify_sync_committee_message_signature(
epoch, block_root, fork0, genesis_validators_root0, load(pubkey0).get, fork0, genesis_validators_root0, slot, block_root, load(pubkey0).get,
blsSign(privkey0, sync_committee_msg_signing_root( get_sync_committee_message_signature(
fork0, epoch, genesis_validators_root1, block_root).data)) fork0, genesis_validators_root1, slot, block_root, privkey0).toValidatorSig())
test "Sync committee signed contribution and proof signatures": test "Sync committee signed contribution and proof signatures":
let signed_contribution_and_proof = default(SignedContributionAndProof) let contribution_and_proof = default(ContributionAndProof)
check: check:
# Matching public/private keys and genesis validator roots # Matching public/private keys and genesis validator roots
verify_signed_contribution_and_proof_signature( verify_contribution_and_proof_signature(
SignedContributionAndProof(signature: blsSign( fork0, genesis_validators_root0, contribution_and_proof,
privkey0, contribution_and_proof_signing_root( load(pubkey0).get,
get_contribution_and_proof_signature(
fork0, genesis_validators_root0, fork0, genesis_validators_root0,
signed_contribution_and_proof.message).data).toValidatorSig), contribution_and_proof, privkey0).toValidatorSig)
fork0, genesis_validators_root0, load(pubkey0).get)
# Mismatched public/private keys # Mismatched public/private keys
not verify_signed_contribution_and_proof_signature( not verify_contribution_and_proof_signature(
SignedContributionAndProof(signature: blsSign( fork0, genesis_validators_root0, contribution_and_proof,
privkey1, contribution_and_proof_signing_root( load(pubkey0).get,
get_contribution_and_proof_signature(
fork0, genesis_validators_root0, fork0, genesis_validators_root0,
signed_contribution_and_proof.message).data).toValidatorSig), contribution_and_proof, privkey1).toValidatorSig)
fork0, genesis_validators_root0, load(pubkey0).get)
# Mismatched forks # Mismatched forks
not verify_signed_contribution_and_proof_signature( not verify_contribution_and_proof_signature(
SignedContributionAndProof(signature: blsSign( fork0, genesis_validators_root0, contribution_and_proof,
privkey0, contribution_and_proof_signing_root( load(pubkey0).get,
get_contribution_and_proof_signature(
fork1, genesis_validators_root0, fork1, genesis_validators_root0,
signed_contribution_and_proof.message).data).toValidatorSig), contribution_and_proof, privkey0).toValidatorSig)
fork0, genesis_validators_root0, load(pubkey0).get)
# Mismatched genesis validator roots # Mismatched genesis validator roots
not verify_signed_contribution_and_proof_signature( not verify_contribution_and_proof_signature(
SignedContributionAndProof(signature: blsSign( fork0, genesis_validators_root0, contribution_and_proof,
privkey0, contribution_and_proof_signing_root( load(pubkey0).get,
fork0, genesis_validators_root0, get_contribution_and_proof_signature(
signed_contribution_and_proof.message).data).toValidatorSig), fork0, genesis_validators_root1,
fork0, genesis_validators_root1, load(pubkey0).get) contribution_and_proof, privkey0).toValidatorSig)
test "Sync committee selection proof signatures": test "Sync committee selection proof signatures":
let let
@ -246,45 +246,29 @@ suite "Message signatures":
check: check:
# Matching public/private keys and genesis validator roots # Matching public/private keys and genesis validator roots
verify_selection_proof_signature( verify_sync_committee_selection_proof(
ContributionAndProof( fork0, genesis_validators_root0, slot, subcommittee_index,
contribution: SyncCommitteeContribution( load(pubkey0).get, get_sync_committee_selection_proof(
slot: slot, subcommittee_index: subcommittee_index),
selection_proof: blsSign(
privkey0, sync_committee_selection_proof_signing_root(
fork0, genesis_validators_root0, slot, fork0, genesis_validators_root0, slot,
subcommittee_index).data).toValidatorSig), subcommittee_index, privkey0).toValidatorSig)
fork0, genesis_validators_root0, load(pubkey0).get)
# Mismatched public/private keys # Mismatched public/private keys
not verify_selection_proof_signature( not verify_sync_committee_selection_proof(
ContributionAndProof( fork0, genesis_validators_root0, slot, subcommittee_index,
contribution: SyncCommitteeContribution( load(pubkey0).get, get_sync_committee_selection_proof(
slot: slot, subcommittee_index: subcommittee_index),
selection_proof: blsSign(
privkey1, sync_committee_selection_proof_signing_root(
fork0, genesis_validators_root0, slot, fork0, genesis_validators_root0, slot,
subcommittee_index).data).toValidatorSig), subcommittee_index, privkey1).toValidatorSig)
fork0, genesis_validators_root0, load(pubkey0).get)
# Mismatched forks # Mismatched forks
not verify_selection_proof_signature( not verify_sync_committee_selection_proof(
ContributionAndProof( fork0, genesis_validators_root0, slot, subcommittee_index,
contribution: SyncCommitteeContribution( load(pubkey0).get, get_sync_committee_selection_proof(
slot: slot, subcommittee_index: subcommittee_index), fork1, genesis_validators_root0, slot,
selection_proof: blsSign( subcommittee_index, privkey0).toValidatorSig)
privkey0, sync_committee_selection_proof_signing_root(
fork0, genesis_validators_root0, slot,
subcommittee_index).data).toValidatorSig),
fork1, genesis_validators_root0, load(pubkey0).get)
# Mismatched genesis validator roots # Mismatched genesis validator roots
not verify_selection_proof_signature( not verify_sync_committee_selection_proof(
ContributionAndProof( fork0, genesis_validators_root0, slot, subcommittee_index,
contribution: SyncCommitteeContribution( load(pubkey0).get, get_sync_committee_selection_proof(
slot: slot, subcommittee_index: subcommittee_index), fork0, genesis_validators_root1, slot,
selection_proof: blsSign( subcommittee_index, privkey0).toValidatorSig)
privkey1, sync_committee_selection_proof_signing_root(
fork0, genesis_validators_root0, slot,
subcommittee_index).data).toValidatorSig),
fork0, genesis_validators_root1, load(pubkey0).get)

View File

@ -62,17 +62,14 @@ suite "Sync committee pool":
subcommittee1 = SyncSubcommitteeIndex(0) subcommittee1 = SyncSubcommitteeIndex(0)
subcommittee2 = SyncSubcommitteeIndex(1) subcommittee2 = SyncSubcommitteeIndex(1)
sig1 = blsSign(privkey1, sync_committee_msg_signing_root( sig1 = get_sync_committee_message_signature(
fork, root1Slot.epoch, genesisValidatorsRoot, root1).data) fork, genesisValidatorsRoot, root1Slot, root1, privkey1)
sig2 = get_sync_committee_message_signature(
sig2 = blsSign(privkey2, sync_committee_msg_signing_root( fork, genesisValidatorsRoot, root2Slot, root2, privkey1)
fork, root2Slot.epoch, genesisValidatorsRoot, root1).data) sig3 = get_sync_committee_message_signature(
fork, genesisValidatorsRoot, root3Slot, root3, privkey1)
sig3 = blsSign(privkey3, sync_committee_msg_signing_root( sig4 = get_sync_committee_message_signature(
fork, root3Slot.epoch, genesisValidatorsRoot, root1).data) fork, genesisValidatorsRoot, root3Slot, root2, privkey1)
sig4 = blsSign(privkey4, sync_committee_msg_signing_root(
fork, root3Slot.epoch, genesisValidatorsRoot, root2).data)
# Inserting sync committee messages # Inserting sync committee messages
# #

View File

@ -346,7 +346,7 @@ proc makeSyncAggregate(
fork, genesis_validators_root, fork, genesis_validators_root,
slot, latest_block_root, slot, latest_block_root,
MockPrivKeys[validatorIdx]) MockPrivKeys[validatorIdx])
selectionProofSig = get_sync_aggregator_selection_data_signature( selectionProofSig = get_sync_committee_selection_proof(
fork, genesis_validators_root, fork, genesis_validators_root,
slot, subcommitteeIdx.uint64, slot, subcommitteeIdx.uint64,
MockPrivKeys[validatorIdx]) MockPrivKeys[validatorIdx])
@ -371,7 +371,7 @@ proc makeSyncAggregate(
aggregator_index: uint64 aggregator.validatorIdx, aggregator_index: uint64 aggregator.validatorIdx,
contribution: contribution, contribution: contribution,
selection_proof: aggregator.selectionProof) selection_proof: aggregator.selectionProof)
contributionSig = get_sync_committee_contribution_and_proof_signature( contributionSig = get_contribution_and_proof_signature(
fork, genesis_validators_root, fork, genesis_validators_root,
contributionAndProof, contributionAndProof,
MockPrivKeys[aggregator.validatorIdx]) MockPrivKeys[aggregator.validatorIdx])