Block validation flow v2 + Batch (serial) sig verification (#2250)

* bump nim-blscurve

* Outline the block validation flow

* introduce the SigVerified types, pass the tests

* Split clearance/quarantine to prepare for batch crypto verif

* Add a batch signature collector

* Make clearance use SigVerified block and split verification between crypto and state transition

* Always use signedBeaconBlock for the onBlockAdded callback

* RANDAO signing_root is the epoch instead of the full block

* Support skipping BLS for testing

* Fix compilation of the validator client

* Try to fix strange errors MacOS and Jenkins (Clang, unknown type name br_hmac_drbg_context in stdlib_assertions.nim.c)

* address https://github.com/status-im/nimbus-eth2/pull/2250#discussion_r561819858

* address https://github.com/status-im/nimbus-eth2/pull/2250#discussion_r561828025

* onBlockAdded callback should use TrustedSignedBeaconBlock https://github.com/status-im/nimbus-eth2/pull/2250#discussion_r561837261

* address https://github.com/status-im/nimbus-eth2/pull/2250#discussion_r561828946

* Use the application RNG: https://github.com/status-im/nimbus-eth2/pull/2250#discussion_r561815336

* Improve codegen of conversion zero-cost)

* Quick fixes with loadWithCache after #2259 (TODO: graceful error since pubkey validations is now done first in signatures_batch)

* Graceful handle rogue pubkeys and signatures now that those are lazy-loaded
This commit is contained in:
Mamy Ratsimbazafy 2021-01-25 19:45:48 +01:00 committed by GitHub
parent 5ca10d3de2
commit 70a03658e3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
20 changed files with 760 additions and 150 deletions

View File

@ -231,7 +231,7 @@ proc addAttestation*(pool: var AttestationPool,
proc addForkChoice*(pool: var AttestationPool,
epochRef: EpochRef,
blckRef: BlockRef,
blck: BeaconBlock,
blck: TrustedBeaconBlock,
wallSlot: Slot) =
## Add a verified block to the fork choice context
let state = pool.forkChoice.process_block(

View File

@ -314,12 +314,16 @@ func toBeaconBlockSummary(v: SomeBeaconBlock): BeaconBlockSummary =
parent_root: v.parent_root,
)
# TODO: we should only store TrustedSignedBeaconBlock in the DB.
proc putBlock*(db: BeaconChainDB, value: SignedBeaconBlock) =
db.put(subkey(type value, value.root), value)
db.put(subkey(BeaconBlockSummary, value.root), value.message.toBeaconBlockSummary())
proc putBlock*(db: BeaconChainDB, value: TrustedSignedBeaconBlock) =
db.put(subkey(SignedBeaconBlock, value.root), value)
db.put(subkey(BeaconBlockSummary, value.root), value.message.toBeaconBlockSummary())
proc putBlock*(db: BeaconChainDB, value: SigVerifiedSignedBeaconBlock) =
db.put(subkey(SignedBeaconBlock, value.root), value)
db.put(subkey(BeaconBlockSummary, value.root), value.message.toBeaconBlockSummary())
proc putState*(db: BeaconChainDB, key: Eth2Digest, value: BeaconState) =
# TODO prune old states - this is less easy than it seems as we never know

View File

@ -12,8 +12,9 @@ import
std/[deques, strformat, tables, hashes],
# Status libraries
stew/[endians2, byteutils], chronicles,
eth/keys,
# Internals
../spec/[datatypes, crypto, digest],
../spec/[datatypes, crypto, digest, signatures_batch],
../beacon_chain_db, ../extras
from libp2p/protocols/pubsub/pubsub import ValidationResult
@ -65,6 +66,11 @@ type
## Roots of blocks that we would like to have (either parent_root of
## unresolved blocks or block roots of attestations)
sigVerifCache*: BatchedBLSVerifierCache ##\
## A cache for batch BLS signature verification contexts
rng*: ref BrHmacDrbgContext ##\
## A reference to the Nimbus application-wide RNG
inAdd*: bool
MissingBlock* = object
@ -200,8 +206,9 @@ type
## has advanced without blocks
OnBlockAdded* = proc(
blckRef: BlockRef, blck: SignedBeaconBlock,
blckRef: BlockRef, blck: TrustedSignedBeaconBlock,
epochRef: EpochRef, state: HashedBeaconState) {.raises: [Defect], gcsafe.}
# The `{.gcsafe.}` annotation is needed to shut up the compiler.
template validator_keys*(e: EpochRef): untyped = e.validator_key_store[1][]

View File

@ -43,7 +43,7 @@ declareGauge beacon_processed_deposits_total, "Number of total deposits included
logScope: topics = "chaindag"
proc putBlock*(
dag: var ChainDAGRef, signedBlock: SignedBeaconBlock) =
dag: var ChainDAGRef, signedBlock: TrustedSignedBeaconBlock) =
dag.db.putBlock(signedBlock)
proc updateStateData*(

View File

@ -11,8 +11,9 @@ import
std/tables,
chronicles,
stew/[assign2, results],
eth/keys,
../extras, ../time,
../spec/[crypto, datatypes, digest, helpers, signatures, state_transition],
../spec/[crypto, datatypes, digest, helpers, signatures, signatures_batch, state_transition],
./block_pools_types, ./chain_dag, ./quarantine
export results
@ -27,6 +28,30 @@ export results
logScope:
topics = "clearance"
template asSigVerified(x: SignedBeaconBlock): SigVerifiedSignedBeaconBlock =
## This converts a signed beacon block to a sig verified beacon clock.
## This assumes that their bytes representation is the same.
##
## At the GC-level, the GC is type-agnostic it's all type erased so
## casting between seq[Attestation] and seq[TrustedAttestation]
## will not disrupt GC operations.
##
## This SHOULD be used in function calls to avoid expensive temporary.
## see https://github.com/status-im/nimbus-eth2/pull/2250#discussion_r562010679
cast[ptr SigVerifiedSignedBeaconBlock](signedBlock.unsafeAddr)[]
template asTrusted(x: SignedBeaconBlock or SigVerifiedBeaconBlock): TrustedSignedBeaconBlock =
## This converts a sigverified beacon block to a trusted beacon clock.
## This assumes that their bytes representation is the same.
##
## At the GC-level, the GC is type-agnostic it's all type erased so
## casting between seq[Attestation] and seq[TrustedAttestation]
## will not disrupt GC operations.
##
## This SHOULD be used in function calls to avoid expensive temporary.
## see https://github.com/status-im/nimbus-eth2/pull/2250#discussion_r562010679
cast[ptr TrustedSignedBeaconBlock](signedBlock.unsafeAddr)[]
func getOrResolve*(dag: ChainDAGRef, quarantine: var QuarantineRef, root: Eth2Digest): BlockRef =
## Fetch a block ref, or nil if not found (will be added to list of
## blocks-to-resolve)
@ -35,6 +60,13 @@ func getOrResolve*(dag: ChainDAGRef, quarantine: var QuarantineRef, root: Eth2Di
if result.isNil:
quarantine.addMissing(root)
proc batchVerify(quarantine: var QuarantineRef, sigs: openArray[SignatureSet]): bool =
var secureRandomBytes: array[32, byte]
quarantine.rng[].brHmacDrbgGenerate(secureRandomBytes)
# TODO: For now only enable serial batch verification
return batchVerifySerial(quarantine.sigVerifCache, sigs, secureRandomBytes)
proc addRawBlock*(
dag: var ChainDAGRef, quarantine: var QuarantineRef,
signedBlock: SignedBeaconBlock, onBlockAdded: OnBlockAdded
@ -42,19 +74,19 @@ proc addRawBlock*(
proc addResolvedBlock(
dag: var ChainDAGRef, quarantine: var QuarantineRef,
state: var StateData, signedBlock: SignedBeaconBlock,
state: var StateData, trustedBlock: TrustedSignedBeaconBlock,
parent: BlockRef, cache: var StateCache,
onBlockAdded: OnBlockAdded
) =
# TODO move quarantine processing out of here
doAssert state.data.data.slot == signedBlock.message.slot,
doAssert state.data.data.slot == trustedBlock.message.slot,
"state must match block"
doAssert state.blck.root == signedBlock.message.parent_root,
doAssert state.blck.root == trustedBlock.message.parent_root,
"the StateData passed into the addResolved function not yet updated!"
let
blockRoot = signedBlock.root
blockRef = BlockRef.init(blockRoot, signedBlock.message)
blockRoot = trustedBlock.root
blockRef = BlockRef.init(blockRoot, trustedBlock.message)
blockEpoch = blockRef.slot.compute_epoch_at_slot()
link(parent, blockRef)
@ -73,7 +105,7 @@ proc addResolvedBlock(
trace "Populating block dag", key = blockRoot, val = blockRef
# Resolved blocks should be stored in database
dag.putBlock(signedBlock)
dag.putBlock(trustedBlock)
var foundHead: BlockRef
for head in dag.heads.mitems():
@ -89,7 +121,7 @@ proc addResolvedBlock(
dag.heads.add(foundHead)
debug "Block resolved",
blck = shortLog(signedBlock.message),
blck = shortLog(trustedBlock.message),
blockRoot = shortLog(blockRoot),
heads = dag.heads.len()
@ -98,7 +130,7 @@ proc addResolvedBlock(
# Notify others of the new block before processing the quarantine, such that
# notifications for parents happens before those of the children
if onBlockAdded != nil:
onBlockAdded(blockRef, signedBlock, epochRef, state.data)
onBlockAdded(blockRef, trustedBlock, epochRef, state.data)
# Now that we have the new block, we should see if any of the previously
# unresolved blocks magically become resolved
@ -120,13 +152,145 @@ proc addResolvedBlock(
for v in resolved:
discard addRawBlock(dag, quarantine, v, onBlockAdded)
proc addRawBlockCheckStateTransition(
dag: var ChainDAGRef, quarantine: var QuarantineRef,
signedBlock: SomeSignedBeaconBlock, cache: var StateCache
): (ValidationResult, BlockError) =
## addRawBlock - Ensure block can be applied on a state
let
poolPtr = unsafeAddr dag # safe because restore is short-lived
func restore(v: var HashedBeaconState) =
# TODO address this ugly workaround - there should probably be a
# `state_transition` that takes a `StateData` instead and updates
# the block as well
doAssert v.addr == addr poolPtr.clearanceState.data
assign(poolPtr.clearanceState, poolPtr.headState)
if not state_transition(dag.runtimePreset, dag.clearanceState.data, signedBlock,
cache, dag.updateFlags + {slotProcessed}, restore):
info "Invalid block"
return (ValidationResult.Reject, Invalid)
return (ValidationResult.Accept, default(BlockError))
proc addRawBlockKnownParent(
dag: var ChainDAGRef, quarantine: var QuarantineRef,
signedBlock: SignedBeaconBlock,
parent: BlockRef,
onBlockAdded: OnBlockAdded
): Result[BlockRef, (ValidationResult, BlockError)] =
## addRawBlock - Block has a parent
if parent.slot >= signedBlock.message.slot:
# A block whose parent is newer than the block itself is clearly invalid -
# discard it immediately
debug "Invalid block slot",
parentBlock = shortLog(parent)
return err((ValidationResult.Reject, Invalid))
if (parent.slot < dag.finalizedHead.slot) or
(parent.slot == dag.finalizedHead.slot and
parent != dag.finalizedHead.blck):
# We finalized a block that's newer than the parent of this block - this
# block, although recent, is thus building on a history we're no longer
# interested in pursuing. This can happen if a client produces a block
# while syncing - ie it's own head block will be old, but it'll create
# a block according to the wall clock, in its own little world - this is
# correct - from their point of view, the head block they have is the
# latest thing that happened on the chain and they're performing their
# duty correctly.
debug "Unviable block, dropping",
finalizedHead = shortLog(dag.finalizedHead),
tail = shortLog(dag.tail)
return err((ValidationResult.Ignore, Unviable))
# The block might have been in either of `orphans` or `missing` - we don't
# want any more work done on its behalf
quarantine.removeOrphan(signedBlock)
# The block is resolved, now it's time to validate it to ensure that the
# blocks we add to the database are clean for the given state
# TODO if the block is from the future, we should not be resolving it (yet),
# but maybe we should use it as a hint that our clock is wrong?
var cache = StateCache()
updateStateData(
dag, dag.clearanceState, parent.atSlot(signedBlock.message.slot), true, cache)
# First batch verify crypto
if skipBLSValidation notin dag.updateFlags:
# TODO: remove skipBLSValidation
var sigs: seq[SignatureSet]
if not sigs.collectSignatureSets(signedBlock, dag.clearanceState.data.data, cache):
# A PublicKey or Signature isn't on the BLS12-381 curve
return err((ValidationResult.Reject, Invalid))
if not quarantine.batchVerify(sigs):
return err((ValidationResult.Reject, Invalid))
static: doAssert sizeof(SignedBeaconBlock) == sizeof(SigVerifiedSignedBeaconBlock)
let (valRes, blockErr) = addRawBlockCheckStateTransition(
dag, quarantine, signedBlock.asSigVerified(), cache)
if valRes != ValidationResult.Accept:
return err((valRes, blockErr))
# Careful, clearanceState.data has been updated but not blck - we need to
# create the BlockRef first!
addResolvedBlock(
dag, quarantine, dag.clearanceState,
signedBlock.asTrusted(),
parent, cache,
onBlockAdded)
return ok dag.clearanceState.blck
proc addRawBlockUnresolved(
dag: var ChainDAGRef,
quarantine: var QuarantineRef,
signedBlock: SignedBeaconBlock
): Result[BlockRef, (ValidationResult, BlockError)] =
## addRawBlock - Block is unresolved / has no parent
# This is an unresolved block - add it to the quarantine, which will cause its
# parent to be scheduled for downloading
if not quarantine.add(dag, signedBlock):
debug "Block quarantine full"
if signedBlock.message.parent_root in quarantine.missing or
containsOrphan(quarantine, signedBlock):
debug "Unresolved block (parent missing or orphaned)",
orphans = quarantine.orphans.len,
missing = quarantine.missing.len
return err((ValidationResult.Ignore, MissingParent))
# TODO if we receive spam blocks, one heurestic to implement might be to wait
# for a couple of attestations to appear before fetching parents - this
# would help prevent using up network resources for spam - this serves
# two purposes: one is that attestations are likely to appear for the
# block only if it's valid / not spam - the other is that malicious
# validators that are not proposers can sign invalid blocks and send
# them out without penalty - but signing invalid attestations carries
# a risk of being slashed, making attestations a more valuable spam
# filter.
debug "Unresolved block (parent missing)",
orphans = quarantine.orphans.len,
missing = quarantine.missing.len
return err((ValidationResult.Ignore, MissingParent))
proc addRawBlock*(
dag: var ChainDAGRef, quarantine: var QuarantineRef,
signedBlock: SignedBeaconBlock,
onBlockAdded: OnBlockAdded
): Result[BlockRef, (ValidationResult, BlockError)] =
## Try adding a block to the chain, verifying first that it passes the state
## transition function.
## transition function and contains correct cryptographic signature.
##
## Cryptographic checks can be skipped by adding skipBLSValidation to dag.updateFlags
logScope:
blck = shortLog(signedBlock.message)
@ -162,94 +326,8 @@ proc addRawBlock*(
let parent = dag.blocks.getOrDefault(blck.parent_root)
if parent != nil:
if parent.slot >= blck.slot:
# A block whose parent is newer than the block itself is clearly invalid -
# discard it immediately
debug "Invalid block slot",
parentBlock = shortLog(parent)
return err((ValidationResult.Reject, Invalid))
if (parent.slot < dag.finalizedHead.slot) or
(parent.slot == dag.finalizedHead.slot and
parent != dag.finalizedHead.blck):
# We finalized a block that's newer than the parent of this block - this
# block, although recent, is thus building on a history we're no longer
# interested in pursuing. This can happen if a client produces a block
# while syncing - ie it's own head block will be old, but it'll create
# a block according to the wall clock, in its own little world - this is
# correct - from their point of view, the head block they have is the
# latest thing that happened on the chain and they're performing their
# duty correctly.
debug "Unviable block, dropping",
finalizedHead = shortLog(dag.finalizedHead),
tail = shortLog(dag.tail)
return err((ValidationResult.Ignore, Unviable))
# The block might have been in either of `orphans` or `missing` - we don't
# want any more work done on its behalf
quarantine.removeOrphan(signedBlock)
# The block is resolved, now it's time to validate it to ensure that the
# blocks we add to the database are clean for the given state
# TODO if the block is from the future, we should not be resolving it (yet),
# but maybe we should use it as a hint that our clock is wrong?
var cache = StateCache()
updateStateData(
dag, dag.clearanceState, parent.atSlot(blck.slot), true, cache)
let
poolPtr = unsafeAddr dag # safe because restore is short-lived
func restore(v: var HashedBeaconState) =
# TODO address this ugly workaround - there should probably be a
# `state_transition` that takes a `StateData` instead and updates
# the block as well
doAssert v.addr == addr poolPtr.clearanceState.data
assign(poolPtr.clearanceState, poolPtr.headState)
if not state_transition(dag.runtimePreset, dag.clearanceState.data, signedBlock,
cache, dag.updateFlags + {slotProcessed}, restore):
info "Invalid block"
return err((ValidationResult.Reject, Invalid))
# Careful, clearanceState.data has been updated but not blck - we need to
# create the BlockRef first!
addResolvedBlock(
dag, quarantine, dag.clearanceState, signedBlock, parent, cache,
onBlockAdded)
return ok dag.clearanceState.blck
# This is an unresolved block - add it to the quarantine, which will cause its
# parent to be scheduled for downloading
if not quarantine.add(dag, signedBlock):
debug "Block quarantine full"
if blck.parent_root in quarantine.missing or
containsOrphan(quarantine, signedBlock):
debug "Unresolved block (parent missing or orphaned)",
orphans = quarantine.orphans.len,
missing = quarantine.missing.len
return err((ValidationResult.Ignore, MissingParent))
# TODO if we receive spam blocks, one heurestic to implement might be to wait
# for a couple of attestations to appear before fetching parents - this
# would help prevent using up network resources for spam - this serves
# two purposes: one is that attestations are likely to appear for the
# block only if it's valid / not spam - the other is that malicious
# validators that are not proposers can sign invalid blocks and send
# them out without penalty - but signing invalid attestations carries
# a risk of being slashed, making attestations a more valuable spam
# filter.
debug "Unresolved block (parent missing)",
orphans = quarantine.orphans.len,
missing = quarantine.missing.len
return err((ValidationResult.Ignore, MissingParent))
return addRawBlockKnownParent(dag, quarantine, signedBlock, parent, onBlockAdded)
return addRawBlockUnresolved(dag, quarantine, signedBlock)
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/p2p-interface.md#beacon_block
proc isValidBeaconBlock*(

View File

@ -11,6 +11,7 @@ import
std/[tables, options],
chronicles,
stew/bitops2,
eth/keys,
../spec/[crypto, datatypes, digest],
./block_pools_types
@ -19,6 +20,10 @@ export options, block_pools_types
logScope:
topics = "quarant"
func init*(T: type QuarantineRef, rng: ref BrHmacDrbgContext): T =
result = T()
result.rng = rng
func checkMissing*(quarantine: var QuarantineRef): seq[FetchRecord] =
## Return a list of blocks that we should try to resolve from other client -
## to be called periodically but not too often (once per slot?)

View File

@ -141,11 +141,11 @@ proc storeBlock(
attestationPool = self.attestationPool
let blck = self.chainDag.addRawBlock(self.quarantine, signedBlock) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
blckRef: BlockRef, trustedBlock: TrustedSignedBeaconBlock,
epochRef: EpochRef, state: HashedBeaconState):
# Callback add to fork choice if valid
attestationPool[].addForkChoice(
epochRef, blckRef, signedBlock.message, wallSlot)
epochRef, blckRef, trustedBlock.message, wallSlot)
# Trigger attestation sending
if blck.isOk and not self.blockReceivedDuringSlot.finished:

View File

@ -211,7 +211,7 @@ proc init*(T: type BeaconNode,
else: {}
chainDag = ChainDAGRef.init(conf.runtimePreset, db, chainDagFlags)
beaconClock = BeaconClock.init(chainDag.headState.data.data)
quarantine = QuarantineRef()
quarantine = QuarantineRef.init(rng)
databaseGenesisValidatorsRoot =
chainDag.headState.data.data.genesis_validators_root

View File

@ -372,7 +372,9 @@ proc installBeaconApiHandlers*(rpcServer: RpcServer, node: BeaconNode) =
tuple[canonical: bool, header: SignedBeaconBlockHeader]:
let bd = node.getBlockDataFromBlockId(blockId)
let tsbb = bd.data
result.header.signature = ValidatorSig.init tsbb.signature.data
static: doAssert tsbb.signature is TrustedSig and
sizeof(ValidatorSig) == sizeof(tsbb.signature)
result.header.signature = cast[ValidatorSig](tsbb.signature)
result.header.message.slot = tsbb.message.slot
result.header.message.proposer_index = tsbb.message.proposer_index

View File

@ -81,6 +81,36 @@ const
template maxSize*(n: int) {.pragma.}
# Block validation flow
# We distinguish 4 cases depending
# if the signature and/or transition logic of a
# a block have been verified:
#
# | | Signature unchecked | Signature verified |
# |----------------------------|------------------------------- |-----------------------------|
# | State transition unchecked | - UntrustedBeaconBlock | - SigVerifiedBeaconBlock |
# | | - UntrustedIndexedAttestation | - TrustedIndexedAttestation |
# | | - UntrustedAttestation | - TrustedAttestation |
# |----------------------------|------------------------------- |-----------------------------|
# | State transition verified | - TransitionVerifiedBeaconBlock | - TrustedSignedBeaconBlock |
# | | - UntrustedIndexedAttestation | - TrustedIndexedAttestation |
# | | - UntrustedAttestation | - TrustedAttestation |
#
# At the moment we only introduce SigVerifiedBeaconBlock
# and keep the old naming where BeaconBlock == UntrustedbeaconBlock
# Also for Attestation, IndexedAttestation, AttesterSlashing, ProposerSlashing.
# We only distinguish between the base version and the Trusted version
# (i.e. Attestation and TrustedAttestation)
# The Trusted version, at the moment, implies that the cryptographic signature was checked.
# It DOES NOT imply that the state transition was verified.
# Currently the code MUST verify the state transition as soon as the signature is verified
#
# TODO We could implement the trust level as either static enums or generic tags
# and reduce duplication and improve maintenance and readability,
# however this caused problems respectively of:
# - ambiguous calls, in particular for chronicles, with static enums
# - broke the compiler in SSZ and nim-serialization
type
# Domains
# ---------------------------------------------------------------
@ -123,11 +153,25 @@ type
signed_header_1*: SignedBeaconBlockHeader
signed_header_2*: SignedBeaconBlockHeader
TrustedProposerSlashing* = object
# The Trusted version, at the moment, implies that the cryptographic signature was checked.
# It DOES NOT imply that the state transition was verified.
# Currently the code MUST verify the state transition as soon as the signature is verified
signed_header_1*: TrustedSignedBeaconBlockHeader
signed_header_2*: TrustedSignedBeaconBlockHeader
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#attesterslashing
AttesterSlashing* = object
attestation_1*: IndexedAttestation
attestation_2*: IndexedAttestation
TrustedAttesterSlashing* = object
# The Trusted version, at the moment, implies that the cryptographic signature was checked.
# It DOES NOT imply that the state transition was verified.
# Currently the code MUST verify the state transition as soon as the signature is verified
attestation_1*: TrustedIndexedAttestation
attestation_2*: TrustedIndexedAttestation
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#indexedattestation
IndexedAttestation* = object
attesting_indices*: List[uint64, Limit MAX_VALIDATORS_PER_COMMITTEE]
@ -135,6 +179,9 @@ type
signature*: ValidatorSig
TrustedIndexedAttestation* = object
# The Trusted version, at the moment, implies that the cryptographic signature was checked.
# It DOES NOT imply that the state transition was verified.
# Currently the code MUST verify the state transition as soon as the signature is verified
attesting_indices*: List[uint64, Limit MAX_VALIDATORS_PER_COMMITTEE]
data*: AttestationData
signature*: TrustedSig
@ -148,6 +195,9 @@ type
signature*: ValidatorSig
TrustedAttestation* = object
# The Trusted version, at the moment, implies that the cryptographic signature was checked.
# It DOES NOT imply that the state transition was verified.
# Currently the code MUST verify the state transition as soon as the signature is verified
aggregation_bits*: CommitteeValidatorsBits
data*: AttestationData
signature*: TrustedSig
@ -225,6 +275,20 @@ type
body*: BeaconBlockBody
SigVerifiedBeaconBlock* = object
## A BeaconBlock that contains verified signatures
## but that has not been verified for state transition
slot*: Slot
proposer_index*: uint64
parent_root*: Eth2Digest ##\
## Root hash of the previous block
state_root*: Eth2Digest ##\
## The state root, _after_ this block has been processed
body*: SigVerifiedBeaconBlockBody
TrustedBeaconBlock* = object
## When we receive blocks from outside sources, they are untrusted and go
## through several layers of validation. Blocks that have gone through
@ -276,23 +340,51 @@ type
deposits*: List[Deposit, Limit MAX_DEPOSITS]
voluntary_exits*: List[SignedVoluntaryExit, Limit MAX_VOLUNTARY_EXITS]
TrustedBeaconBlockBody* = object
SigVerifiedBeaconBlockBody* = object
## A BeaconBlock body with signatures verified
## including:
## - Randao reveal
## - Attestations
## - ProposerSlashing (SignedBeaconBlockHeader)
## - AttesterSlashing (IndexedAttestation)
## - SignedVoluntaryExits
##
## - ETH1Data (Deposits) can contain invalid BLS signatures
##
## The block state transition has NOT been verified
randao_reveal*: TrustedSig
eth1_data*: Eth1Data
graffiti*: GraffitiBytes
# Operations
proposer_slashings*: List[ProposerSlashing, Limit MAX_PROPOSER_SLASHINGS]
attester_slashings*: List[AttesterSlashing, Limit MAX_ATTESTER_SLASHINGS]
proposer_slashings*: List[TrustedProposerSlashing, Limit MAX_PROPOSER_SLASHINGS]
attester_slashings*: List[TrustedAttesterSlashing, Limit MAX_ATTESTER_SLASHINGS]
attestations*: List[TrustedAttestation, Limit MAX_ATTESTATIONS]
deposits*: List[Deposit, Limit MAX_DEPOSITS]
voluntary_exits*: List[SignedVoluntaryExit, Limit MAX_VOLUNTARY_EXITS]
voluntary_exits*: List[TrustedSignedVoluntaryExit, Limit MAX_VOLUNTARY_EXITS]
SomeSignedBeaconBlock* = SignedBeaconBlock | TrustedSignedBeaconBlock
SomeBeaconBlock* = BeaconBlock | TrustedBeaconBlock
SomeBeaconBlockBody* = BeaconBlockBody | TrustedBeaconBlockBody
TrustedBeaconBlockBody* = object
## A full verified block
randao_reveal*: TrustedSig
eth1_data*: Eth1Data
graffiti*: GraffitiBytes
# Operations
proposer_slashings*: List[TrustedProposerSlashing, Limit MAX_PROPOSER_SLASHINGS]
attester_slashings*: List[TrustedAttesterSlashing, Limit MAX_ATTESTER_SLASHINGS]
attestations*: List[TrustedAttestation, Limit MAX_ATTESTATIONS]
deposits*: List[Deposit, Limit MAX_DEPOSITS]
voluntary_exits*: List[TrustedSignedVoluntaryExit, Limit MAX_VOLUNTARY_EXITS]
SomeSignedBeaconBlock* = SignedBeaconBlock | SigVerifiedSignedBeaconBlock | TrustedSignedBeaconBlock
SomeBeaconBlock* = BeaconBlock | SigVerifiedBeaconBlock | TrustedBeaconBlock
SomeBeaconBlockBody* = BeaconBlockBody | SigVerifiedBeaconBlockBody | TrustedBeaconBlockBody
SomeAttestation* = Attestation | TrustedAttestation
SomeIndexedAttestation* = IndexedAttestation | TrustedIndexedAttestation
SomeProposerSlashing* = ProposerSlashing | TrustedProposerSlashing
SomeAttesterSlashing* = AttesterSlashing | TrustedAttesterSlashing
SomeSignedBeaconBlockHeader* = SignedBeaconBlockHeader | TrustedSignedBeaconBlockHeader
SomeSignedVoluntaryExit* = SignedVoluntaryExit | TrustedSignedVoluntaryExit
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#beaconstate
BeaconState* = object
@ -413,6 +505,10 @@ type
message*: VoluntaryExit
signature*: ValidatorSig
TrustedSignedVoluntaryExit* = object
message*: VoluntaryExit
signature*: TrustedSig
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#signedbeaconblock
SignedBeaconBlock* = object
message*: BeaconBlock
@ -420,6 +516,25 @@ type
root* {.dontSerialize.}: Eth2Digest # cached root of signed beacon block
SigVerifiedSignedBeaconBlock* = object
## A SignedBeaconBlock with signatures verified
## including:
## - Block signature
## - BeaconBlockBody
## - Randao reveal
## - Attestations
## - ProposerSlashing (SignedBeaconBlockHeader)
## - AttesterSlashing (IndexedAttestation)
## - SignedVoluntaryExits
##
## - ETH1Data (Deposits) can contain invalid BLS signatures
##
## The block state transition has NOT been verified
message*: SigVerifiedBeaconBlock
signature*: TrustedSig
root* {.dontSerialize.}: Eth2Digest # cached root of signed beacon block
TrustedSignedBeaconBlock* = object
message*: TrustedBeaconBlock
signature*: TrustedSig
@ -431,6 +546,10 @@ type
message*: BeaconBlockHeader
signature*: ValidatorSig
TrustedSignedBeaconBlockHeader* = object
message*: BeaconBlockHeader
signature*: TrustedSig
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/validator.md#aggregateandproof
AggregateAndProof* = object
aggregator_index*: uint64
@ -764,7 +883,7 @@ func shortLog*(v: BeaconBlockHeader): auto =
state_root: shortLog(v.state_root)
)
func shortLog*(v: SignedBeaconBlockHeader): auto =
func shortLog*(v: SomeSignedBeaconBlockHeader): auto =
(
message: shortLog(v.message),
signature: shortLog(v.signature)
@ -815,13 +934,13 @@ func shortLog*(v: SomeIndexedAttestation): auto =
signature: shortLog(v.signature)
)
func shortLog*(v: AttesterSlashing): auto =
func shortLog*(v: SomeAttesterSlashing): auto =
(
attestation_1: shortLog(v.attestation_1),
attestation_2: shortLog(v.attestation_2),
)
func shortLog*(v: ProposerSlashing): auto =
func shortLog*(v: SomeProposerSlashing): auto =
(
signed_header_1: shortLog(v.signed_header_1),
signed_header_2: shortLog(v.signed_header_2)
@ -833,7 +952,7 @@ func shortLog*(v: VoluntaryExit): auto =
validator_index: v.validator_index
)
func shortLog*(v: SignedVoluntaryExit): auto =
func shortLog*(v: SomeSignedVoluntaryExit): auto =
(
message: shortLog(v.message),
signature: shortLog(v.signature)

View File

@ -0,0 +1,322 @@
# beacon_chain
# Copyright (c) 2018-2020 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.push raises: [Defect].}
import
# Status lib
blscurve,
stew/byteutils,
# Internal
../ssz/merkleization,
./crypto, ./datatypes, ./helpers, ./presets,
./beaconstate, ./digest
export SignatureSet, BatchedBLSVerifierCache, batchVerify, batchVerifySerial, batchVerifyParallel
func `$`*(s: SignatureSet): string =
"(pubkey: 0x" & s.pubkey.toHex() &
", signing_root: 0x" & s.message.toHex() &
", signature: 0x" & s.signature.toHex() & ')'
# Important:
# - Due to lazy loading, when we do crypto verification
# and only then state-transition verification,
# there is no guarantee that pubkeys and signatures received are valid
# unlike when Nimbus did eager loading which ensured they were correct beforehand
template loadOrExitFalse(signature: ValidatorSig): blscurve.Signature =
## Load a BLS signature from a raw signature
## Exists the **caller** with false if the signature is invalid
let sig = signature.load()
if sig.isNone:
return false # this exists the calling scope, as templates are inlined.
sig.unsafeGet()
template loadWithCacheOrExitFalse(pubkey: ValidatorPubKey): blscurve.PublicKey =
## Load a BLS signature from a raw public key
## Exists the **caller** with false if the public key is invalid
let pk = pubkey.loadWithCache()
if pk.isNone:
return false # this exists the calling scope, as templates are inlined.
pk.unsafeGet()
func addSignatureSet[T](
sigs: var seq[SignatureSet],
pubkey: blscurve.PublicKey,
sszObj: T,
signature: ValidatorSig,
genesis_validators_root: Eth2Digest,
fork: Fork,
epoch: Epoch,
domain: DomainType): bool {.raises: [Defect].}=
## Add a new signature set triplet (pubkey, message, signature)
## 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((
pubkey,
signing_root,
signature.loadOrExitFalse()
))
return true
proc aggregateAttesters(
aggPK: var blscurve.PublicKey,
attestation: IndexedAttestation,
state: BeaconState
): bool =
doAssert attestation.attesting_indices.len > 0
var attestersAgg{.noInit.}: AggregatePublicKey
attestersAgg.init(state.validators[attestation.attesting_indices[0]]
.pubkey.loadWithCacheOrExitFalse())
for i in 1 ..< attestation.attesting_indices.len:
attestersAgg.aggregate(state.validators[attestation.attesting_indices[i]]
.pubkey.loadWithCacheOrExitFalse())
aggPK.finish(attestersAgg)
return true
proc addIndexedAttestation(
sigs: var seq[SignatureSet],
attestation: IndexedAttestation,
state: BeaconState
): bool =
if attestation.attesting_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 indice in slashing
# - https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#is_valid_indexed_attestation
return false
var aggPK {.noInit.}: blscurve.PublicKey
if not aggPK.aggregateAttesters(attestation, state):
return false
if not sigs.addSignatureSet(
aggPK,
attestation.data,
attestation.signature,
state.genesis_validators_root,
state.fork,
attestation.data.target.epoch,
DOMAIN_BEACON_ATTESTER):
return false
return true
proc addAttestation(
sigs: var seq[SignatureSet],
attestation: Attestation,
state: BeaconState,
cache: var StateCache
): bool =
result = false
var attestersAgg{.noInit.}: AggregatePublicKey
for valIndex in state.get_attesting_indices(
attestation.data,
attestation.aggregation_bits,
cache
):
if not result: # first iteration
attestersAgg.init(state.validators[valIndex]
.pubkey.loadWithCacheOrExitFalse())
result = true
else:
attestersAgg.aggregate(state.validators[valIndex]
.pubkey.loadWithCacheOrExitFalse())
if not result:
# There was no attesters
return false
var attesters{.noinit.}: blscurve.PublicKey
attesters.finish(attestersAgg)
if not sigs.addSignatureSet(
attesters,
attestation.data,
attestation.signature,
state.genesis_validators_root,
state.fork,
attestation.data.target.epoch,
DOMAIN_BEACON_ATTESTER):
return false
return true
proc collectSignatureSets*(
sigs: var seq[SignatureSet],
signed_block: SignedBeaconBlock,
state: BeaconState,
cache: var StateCache): bool =
## Collect all signatures in a single signed block.
## This includes
## - Block proposer
## - Randao Reaveal
## - Proposer slashings
## - Attester slashings
## - Attestations
## - VoluntaryExits
##
## We do not include deposits as they can be invalid per protocol
## (secp256k1 signature instead of BLS)
# Metadata
# ----------------------------------------------------
let
proposer_index = signed_block.message.proposer_index
if proposer_index >= state.validators.lenu64:
return false
let pubkey = state.validators[proposer_index]
.pubkey.loadWithCacheOrExitFalse()
let epoch = signed_block.message.slot.compute_epoch_at_slot()
# 1. Block proposer
# ----------------------------------------------------
if not sigs.addSignatureSet(
pubkey,
signed_block.message,
signed_block.signature,
state.genesis_validators_root,
state.fork,
epoch,
DOMAIN_BEACON_PROPOSER):
return false
# 2. Randao Reveal
# ----------------------------------------------------
if not sigs.addSignatureSet(
pubkey,
epoch,
signed_block.message.body.randao_reveal,
state.genesis_validators_root,
state.fork,
epoch,
DOMAIN_RANDAO):
return false
# 3. Proposer slashings
# ----------------------------------------------------
# Denial-of-service:
# SSZ deserialization guarantees that blocks received from random sources
# including peer or RPC
# have at most MAX_PROPOSER_SLASHINGS proposer slashings.
for i in 0 ..< signed_block.message.body.proposer_slashings.len:
# don't use "items" for iterating over large type
# due to https://github.com/nim-lang/Nim/issues/14421
# fixed in 1.4.2
# Alias
template slashing: untyped = signed_block.message.body.proposer_slashings[i]
# Proposed block 1
block:
let header_1 = slashing.signed_header_1
let proposer1 = state.validators[header_1.message.proposer_index]
let epoch1 = header_1.message.slot.compute_epoch_at_slot()
if not sigs.addSignatureSet(
proposer1.pubkey.loadWithCacheOrExitFalse(),
header_1.message,
header_1.signature,
state.genesis_validators_root,
state.fork,
epoch1,
DOMAIN_BEACON_PROPOSER
):
return false
# Conflicting block 2
block:
let header_2 = slashing.signed_header_2
let proposer2 = state.validators[header_2.message.proposer_index]
let epoch2 = header_2.message.slot.compute_epoch_at_slot()
if not sigs.addSignatureSet(
proposer2.pubkey.loadWithCacheOrExitFalse(),
header_2.message,
header_2.signature,
state.genesis_validators_root,
state.fork,
epoch2,
DOMAIN_BEACON_PROPOSER
):
return false
# 4. Attester slashings
# ----------------------------------------------------
# Denial-of-service:
# SSZ deserialization guarantees that blocks received from random sources
# including peer or RPC
# have at most MAX_ATTESTER_SLASHINGS attester slashings.
for i in 0 ..< signed_block.message.body.attester_slashings.len:
# don't use "items" for iterating over large type
# due to https://github.com/nim-lang/Nim/issues/14421
# fixed in 1.4.2
# Alias
template slashing: untyped = signed_block.message.body.attester_slashings[i]
# Attestation 1
if not sigs.addIndexedAttestation(
slashing.attestation_1,
state):
return false
# Conflicting attestation 2
if not sigs.addIndexedAttestation(
slashing.attestation_2,
state):
return false
# 5. Attestations
# ----------------------------------------------------
# Denial-of-service:
# SSZ deserialization guarantees that blocks received from random sources
# including peer or RPC
# have at most MAX_ATTESTATIONS attestations.
for i in 0 ..< signed_block.message.body.attestations.len:
# don't use "items" for iterating over large type
# due to https://github.com/nim-lang/Nim/issues/14421
# fixed in 1.4.2
if not sigs.addAttestation(
signed_block.message.body.attestations[i],
state, cache):
return false
# 6. VoluntaryExits
# ----------------------------------------------------
# Denial-of-service:
# SSZ deserialization guarantees that blocks received from random sources
# including peer or RPC
# have at most MAX_VOLUNTARY_EXITS voluntary exits.
for i in 0 ..< signed_block.message.body.voluntary_exits.len:
# don't use "items" for iterating over large type
# due to https://github.com/nim-lang/Nim/issues/14421
# fixed in 1.4.2
template volex: untyped = signed_block.message.body.voluntary_exits[i]
if not sigs.addSignatureSet(
state.validators[volex.message.validator_index]
.pubkey.loadWithCacheOrExitFalse(),
volex.message,
volex.signature,
state.genesis_validators_root,
state.fork,
volex.message.epoch,
DOMAIN_VOLUNTARY_EXIT):
return false
return true

View File

@ -70,7 +70,7 @@ proc verify_block_signature*(
true
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#beacon-chain-state-transition-function
proc verifyStateRoot(state: BeaconState, blck: BeaconBlock): bool =
proc verifyStateRoot(state: BeaconState, blck: BeaconBlock or SigVerifiedBeaconBlock): bool =
# This is inlined in state_transition(...) in spec.
let state_root = hash_tree_root(state)
if state_root != blck.state_root:

View File

@ -27,6 +27,10 @@ import
./signatures, ./presets,
../../nbench/bench_lab
# Generics visibility issue with toSeq(items(intersection(HashSet, HashSet)))
# https://github.com/nim-lang/Nim/issues/11225
export sets
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#block-header
func process_block_header*(
state: var BeaconState, blck: SomeBeaconBlock, flags: UpdateFlags,
@ -124,7 +128,7 @@ func is_slashable_validator(validator: Validator, epoch: Epoch): bool =
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#proposer-slashings
proc check_proposer_slashing*(
state: var BeaconState, proposer_slashing: ProposerSlashing,
state: var BeaconState, proposer_slashing: SomeProposerSlashing,
flags: UpdateFlags):
Result[void, cstring] {.nbench.} =
@ -166,7 +170,7 @@ proc check_proposer_slashing*(
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#proposer-slashings
proc process_proposer_slashing*(
state: var BeaconState, proposer_slashing: ProposerSlashing,
state: var BeaconState, proposer_slashing: SomeProposerSlashing,
flags: UpdateFlags, cache: var StateCache):
Result[void, cstring] {.nbench.} =
? check_proposer_slashing(state, proposer_slashing, flags)
@ -191,7 +195,7 @@ func is_slashable_attestation_data*(
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#attester-slashings
proc check_attester_slashing*(
state: var BeaconState,
attester_slashing: AttesterSlashing,
attester_slashing: SomeAttesterSlashing,
flags: UpdateFlags
): Result[seq[ValidatorIndex], cstring] {.nbench.} =
let
@ -224,7 +228,7 @@ proc check_attester_slashing*(
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#attester-slashings
proc process_attester_slashing*(
state: var BeaconState,
attester_slashing: AttesterSlashing,
attester_slashing: SomeAttesterSlashing,
flags: UpdateFlags,
cache: var StateCache
): Result[void, cstring] {.nbench.} =
@ -242,7 +246,7 @@ proc process_attester_slashing*(
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#voluntary-exits
proc check_voluntary_exit*(
state: BeaconState,
signed_voluntary_exit: SignedVoluntaryExit,
signed_voluntary_exit: SomeSignedVoluntaryExit,
flags: UpdateFlags): Result[void, cstring] {.nbench.} =
let voluntary_exit = signed_voluntary_exit.message
@ -294,7 +298,7 @@ proc check_voluntary_exit*(
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#voluntary-exits
proc process_voluntary_exit*(
state: var BeaconState,
signed_voluntary_exit: SignedVoluntaryExit,
signed_voluntary_exit: SomeSignedVoluntaryExit,
flags: UpdateFlags,
cache: var StateCache): Result[void, cstring] {.nbench.} =
? check_voluntary_exit(state, signed_voluntary_exit, flags)

View File

@ -282,11 +282,11 @@ proc proposeSignedBlock*(node: BeaconNode,
newBlock: SignedBeaconBlock): BlockRef =
let newBlockRef = node.chainDag.addRawBlock(node.quarantine, newBlock) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
blckRef: BlockRef, trustedBlock: TrustedSignedBeaconBlock,
epochRef: EpochRef, state: HashedBeaconState):
# Callback add to fork choice if valid
# Callback add to fork choice if signed block valid (and becomes trusted)
node.attestationPool[].addForkChoice(
epochRef, blckRef, signedBlock.message,
epochRef, blckRef, trustedBlock.message,
node.beaconClock.now().slotOrZero())
if newBlockRef.isErr:
@ -714,4 +714,3 @@ proc handleValidatorDuties*(node: BeaconNode, lastSlot, slot: Slot) {.async.} =
let finalizedEpochRef = node.chainDag.getFinalizedEpochRef()
discard node.eth1Monitor.trackFinalizedState(
finalizedEpochRef.eth1_data, finalizedEpochRef.eth1_deposit_index)

View File

@ -0,0 +1,61 @@
# Block Validation Flow
This is a WIP document to explain the block validation flow.
This should be transformed into diagram that explain
the implicit block validation state machine.
## Inputs
Blocks can be received from the following sources:
- Gossipsub
- the NBC database
- a local validator block proposal
- Devtools: test suite, ncli, fuzzing
The related base types are:
- BeaconBlockBody
- BeaconBlock
- BeaconBlockBody
- + metadata (slot, blockchain state before/after, proposer)
- BeaconBlockHeader
- metadata (slot, blockchain state before/after, proposer)
- merkle hash of the BeaconBlockBody
- SignedBeaconBlock
- BeaconBlock
- + BLS signature
The base types are defined in the Eth2 specs.
On top, Nimbus builds new types to represent the level of trust and validation we have with regards to each BeaconBlock.
Those types allow the Nim compiler to help us ensure proper usage at compile-time and zero runtime cost.
### BeaconBlocks
Those are spec-defined types.
On deserialization the SSZ code guarantees that BeaconBlock are correctly max-sized
according to:
- MAX_PROPOSER_SLASHINGS
- MAX_ATTESTER_SLASHINGS
- MAX_ATTESTATIONS
- MAX_DEPOSITS
- MAX_VOLUNTARY_EXITS
### TrustedBeaconBlocks
A block that has been fully checked to be sound
both in regards to the blockchain protocol and its cryptographic signatures is known as a `TrustedBeaconBlock` or `TrustedSignedBeaconBlock`.
This allows skipping expensive signature checks.
Blocks are considered trusted if they come from:
- the NBC database
- produced by a local validator
### SigVerifiedBeaconBlocks
A block with a valid cryptographic signature is considered SigVerified.
This is a weaker guarantee than Trusted as the block might still be invalid according to the state transition function.
Such a block are produced if incoming gossip blocks' signatures are batched together for batch verification **before** being passed to state transition.
### TransitionVerifiedBeaconBlocks
A block that passes the state transition checks and can be successfully applied to the beacon chain is considered `TransitionVerified`.
Such a block can be produced if incoming blocks' signatures are batched together for batch verification **after** successfully passing state transition.

View File

@ -18,6 +18,7 @@ import
math, stats, times, strformat,
options, random, tables, os,
confutils, chronicles, eth/db/kvstore_sqlite3,
eth/keys,
../tests/[testblockutil],
../beacon_chain/spec/[beaconstate, crypto, datatypes, digest, presets,
helpers, validator, signatures, state_transition],
@ -73,7 +74,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
chainDag = ChainDAGRef.init(runtimePreset, db)
eth1Chain = Eth1Chain.init(runtimePreset, db)
merkleizer = depositContractSnapshot.createMerkleizer
quarantine = QuarantineRef()
quarantine = QuarantineRef.init(keys.newRng())
attPool = AttestationPool.init(chainDag, quarantine)
timers: array[Timers, RunningStat]
attesters: RunningStat
@ -166,7 +167,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
blockRoot, privKey)
let added = chainDag.addRawBlock(quarantine, newBlock) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
blckRef: BlockRef, signedBlock: TrustedSignedBeaconBlock,
epochRef: EpochRef, state: HashedBeaconState):
# Callback add to fork choice if valid
attPool.addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)

View File

@ -8,16 +8,21 @@
{.used.}
import
# Standard library
std/unittest,
# Status lib
chronicles, chronos,
stew/byteutils,
./testutil, ./testblockutil,
eth/keys,
# Internal
../beacon_chain/spec/[crypto, datatypes, digest, validator, state_transition,
helpers, beaconstate, presets, network],
../beacon_chain/[
beacon_node_types, attestation_pool, attestation_aggregation, extras, time],
../beacon_chain/fork_choice/[fork_choice_types, fork_choice],
../beacon_chain/block_pools/[chain_dag, clearance]
../beacon_chain/block_pools/[quarantine, chain_dag, clearance],
# Test utilities
./testutil, ./testblockutil
func combine(tgt: var Attestation, src: Attestation) =
## Combine the signature and participation bitfield, with the assumption that
@ -55,7 +60,7 @@ suiteReport "Attestation pool processing" & preset():
# Genesis state that results in 3 members per committee
var
chainDag = init(ChainDAGRef, defaultRuntimePreset, makeTestDB(SLOTS_PER_EPOCH * 3))
quarantine = QuarantineRef()
quarantine = QuarantineRef.init(keys.newRng())
pool = newClone(AttestationPool.init(chainDag, quarantine))
state = newClone(chainDag.headState)
cache = StateCache()
@ -196,7 +201,7 @@ suiteReport "Attestation pool processing" & preset():
let
b1 = addTestBlock(state.data, chainDag.tail.root, cache)
b1Add = chainDag.addRawBlock(quarantine, b1) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
blckRef: BlockRef, signedBlock: TrustedSignedBeaconBlock,
epochRef: EpochRef, state: HashedBeaconState):
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)
@ -209,7 +214,7 @@ suiteReport "Attestation pool processing" & preset():
let
b2 = addTestBlock(state.data, b1.root, cache)
b2Add = chainDag.addRawBlock(quarantine, b2) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
blckRef: BlockRef, signedBlock: TrustedSignedBeaconBlock,
epochRef: EpochRef, state: HashedBeaconState):
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)
@ -224,7 +229,7 @@ suiteReport "Attestation pool processing" & preset():
let
b10 = makeTestBlock(state.data, chainDag.tail.root, cache)
b10Add = chainDag.addRawBlock(quarantine, b10) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
blckRef: BlockRef, signedBlock: TrustedSignedBeaconBlock,
epochRef: EpochRef, state: HashedBeaconState):
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)
@ -239,7 +244,7 @@ suiteReport "Attestation pool processing" & preset():
graffiti = GraffitiBytes [1'u8, 0, 0, 0 ,0 ,0 ,0 ,0 ,0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0]
)
b11Add = chainDag.addRawBlock(quarantine, b11) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
blckRef: BlockRef, signedBlock: TrustedSignedBeaconBlock,
epochRef: EpochRef, state: HashedBeaconState):
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)
@ -284,7 +289,7 @@ suiteReport "Attestation pool processing" & preset():
let
b10 = makeTestBlock(state.data, chainDag.tail.root, cache)
b10Add = chainDag.addRawBlock(quarantine, b10) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
blckRef: BlockRef, signedBlock: TrustedSignedBeaconBlock,
epochRef: EpochRef, state: HashedBeaconState):
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)
@ -298,7 +303,7 @@ suiteReport "Attestation pool processing" & preset():
# Add back the old block to ensure we have a duplicate error
let b10_clone = b10 # Assumes deep copy
let b10Add_clone = chainDag.addRawBlock(quarantine, b10_clone) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
blckRef: BlockRef, signedBlock: TrustedSignedBeaconBlock,
epochRef: EpochRef, state: HashedBeaconState):
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)
@ -313,7 +318,7 @@ suiteReport "Attestation pool processing" & preset():
let
b10 = addTestBlock(state.data, chainDag.tail.root, cache)
b10Add = chainDag.addRawBlock(quarantine, b10) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
blckRef: BlockRef, signedBlock: TrustedSignedBeaconBlock,
epochRef: EpochRef, state: HashedBeaconState):
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)
@ -341,7 +346,7 @@ suiteReport "Attestation pool processing" & preset():
block_root = new_block.root
let blockRef = chainDag.addRawBlock(quarantine, new_block) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
blckRef: BlockRef, signedBlock: TrustedSignedBeaconBlock,
epochRef: EpochRef, state: HashedBeaconState):
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)
@ -381,7 +386,7 @@ suiteReport "Attestation pool processing" & preset():
# Add back the old block to ensure we have a duplicate error
let b10Add_clone = chainDag.addRawBlock(quarantine, b10_clone) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
blckRef: BlockRef, signedBlock: TrustedSignedBeaconBlock,
epochRef: EpochRef, state: HashedBeaconState):
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)
@ -394,7 +399,7 @@ suiteReport "Attestation validation " & preset():
# Genesis state that results in 3 members per committee
var
chainDag = init(ChainDAGRef, defaultRuntimePreset, makeTestDB(SLOTS_PER_EPOCH * 3))
quarantine = QuarantineRef()
quarantine = QuarantineRef.init(keys.newRng())
pool = newClone(AttestationPool.init(chainDag, quarantine))
state = newClone(chainDag.headState)
cache = StateCache()
@ -403,6 +408,7 @@ suiteReport "Attestation validation " & preset():
process_slots(state.data, state.data.data.slot + 1, cache)
wrappedTimedTest "Validation sanity":
# TODO: refactor tests to avoid skipping BLS validation
chainDag.updateFlags.incl {skipBLSValidation}
var
@ -411,7 +417,7 @@ suiteReport "Attestation validation " & preset():
chainDag.headState.data, chainDag.head.root, cache,
int(SLOTS_PER_EPOCH * 5), false):
let added = chainDag.addRawBlock(quarantine, blck) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
blckRef: BlockRef, signedBlock: TrustedSignedBeaconBlock,
epochRef: EpochRef, state: HashedBeaconState):
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)

View File

@ -10,6 +10,7 @@
import
std/[options, sequtils, unittest],
stew/assign2,
eth/keys,
./testutil, ./testblockutil,
../beacon_chain/spec/[datatypes, digest, helpers, state_transition, presets],
../beacon_chain/[beacon_node_types, ssz],
@ -123,7 +124,7 @@ suiteReport "Block pool processing" & preset():
var
db = makeTestDB(SLOTS_PER_EPOCH)
dag = init(ChainDAGRef, defaultRuntimePreset, db)
quarantine = QuarantineRef()
quarantine = QuarantineRef.init(keys.newRng())
stateData = newClone(dag.headState)
cache = StateCache()
b1 = addTestBlock(stateData.data, dag.tail.root, cache)
@ -335,7 +336,7 @@ suiteReport "chain DAG finalization tests" & preset():
var
db = makeTestDB(SLOTS_PER_EPOCH)
dag = init(ChainDAGRef, defaultRuntimePreset, db)
quarantine = QuarantineRef()
quarantine = QuarantineRef.init(keys.newRng())
cache = StateCache()
wrappedTimedTest "prune heads on finalization" & preset():
@ -463,7 +464,7 @@ suiteReport "chain DAG finalization tests" & preset():
var
db = makeTestDB(SLOTS_PER_EPOCH)
dag = init(ChainDAGRef, defaultRuntimePreset, db)
quarantine = QuarantineRef()
quarantine = QuarantineRef.init(keys.newRng())
cache = StateCache()
timedTest "init with gaps" & preset():

View File

@ -9,14 +9,15 @@
import std/unittest
import chronicles, chronos, testutil
import eth/keys
import ../beacon_chain/spec/[datatypes, presets]
import ../beacon_chain/exit_pool
import ../beacon_chain/block_pools/chain_dag
import ../beacon_chain/block_pools/[quarantine, chain_dag]
proc getExitPool(): auto =
let chainDag =
init(ChainDAGRef, defaultRuntimePreset, makeTestDB(SLOTS_PER_EPOCH * 3))
newClone(ExitPool.init(chainDag, QuarantineRef()))
newClone(ExitPool.init(chainDag, QuarantineRef.init(keys.newRng())))
suiteReport "Exit pool testing suite":
setup:

2
vendor/nim-blscurve vendored

@ -1 +1 @@
Subproject commit b5a77f1d29f6b5f6d5bf16853eea63d75072d0be
Subproject commit f7d0341f8044fec7603c4acec6774b529b036b32