Refactor block/blobs types (#4491)

* Refactor block/blobs types

Use type system to enforce invariant that a pre-4844 block cannot have
a sidecar.

* Update beacon_chain/nimbus_beacon_node.nim

Co-authored-by: tersec <tersec@users.noreply.github.com>

* review feedback

Co-authored-by: tersec <tersec@users.noreply.github.com>
This commit is contained in:
henridf 2023-01-16 17:26:48 +01:00 committed by GitHub
parent fda03548e3
commit 727920a571
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 120 additions and 137 deletions

View File

@ -190,10 +190,11 @@ proc new*(T: type Eth2Processor,
proc processSignedBeaconBlock*( proc processSignedBeaconBlock*(
self: var Eth2Processor, src: MsgSource, self: var Eth2Processor, src: MsgSource,
signedBlock: ForkySignedBeaconBlock): ValidationRes = signedBlockAndBlobs: ForkySignedBeaconBlockMaybeBlobs): ValidationRes =
let let
wallTime = self.getCurrentBeaconTime() wallTime = self.getCurrentBeaconTime()
(afterGenesis, wallSlot) = wallTime.toSlot() (afterGenesis, wallSlot) = wallTime.toSlot()
signedBlock = toSignedBeaconBlock(signedBlockAndBlobs)
logScope: logScope:
blockRoot = shortLog(signedBlock.root) blockRoot = shortLog(signedBlock.root)
@ -213,7 +214,7 @@ proc processSignedBeaconBlock*(
debug "Block received", delay debug "Block received", delay
let v = let v =
self.dag.validateBeaconBlock(self.quarantine, signedBlock, wallTime, {}) self.dag.validateBeaconBlock(self.quarantine, signedBlockAndBlobs, wallTime, {})
if v.isOk(): if v.isOk():
# Block passed validation - enqueue it for processing. The block processing # Block passed validation - enqueue it for processing. The block processing
@ -241,67 +242,6 @@ proc processSignedBeaconBlock*(
v v
proc processSignedBeaconBlockAndBlobsSidecar*(
self: var Eth2Processor, src: MsgSource,
signedBlockAndBlobsSidecar: SignedBeaconBlockAndBlobsSidecar): ValidationRes =
let
wallTime = self.getCurrentBeaconTime()
(afterGenesis, wallSlot) = wallTime.toSlot()
template signedBlock: auto = signedBlockAndBlobsSidecar.beacon_block
template blobs: auto = signedBlockAndBlobsSidecar.blobs_sidecar
logScope:
blockRoot = shortLog(signedBlock.root)
blck = shortLog(signedBlock.message)
signature = shortLog(signedBlock.signature)
wallSlot
if not afterGenesis:
notice "Block before genesis"
return errIgnore("Block before genesis")
# Potential under/overflows are fine; would just create odd metrics and logs
let delay = wallTime - signedBlock.message.slot.start_beacon_time
# Start of block processing - in reality, we have already gone through SSZ
# decoding at this stage, which may be significant
debug "Block received", delay
let blockRes =
self.dag.validateBeaconBlock(self.quarantine, signedBlock, Opt.some(blobs),
wallTime, {})
if blockRes.isErr():
debug "Dropping block", error = blockRes.error()
self.blockProcessor[].dumpInvalidBlock(signedBlock)
beacon_blocks_dropped.inc(1, [$blockRes.error[0]])
return blockRes
let sidecarRes = validateBeaconBlockAndBlobsSidecar(signedBlockAndBlobsSidecar)
if sidecarRes.isErr():
debug "Dropping block", error = sidecarRes.error()
self.blockProcessor[].dumpInvalidBlock(signedBlock)
beacon_blocks_dropped.inc(1, [$sidecarRes.error[0]])
return sidecarRes
# Block passed validation - enqueue it for processing. The block processing
# queue is effectively unbounded as we use a freestanding task to enqueue
# the block - this is done so that when blocks arrive concurrently with
# sync, we don't lose the gossip blocks, but also don't block the gossip
# propagation of seemingly good blocks
trace "Block validated"
self.blockProcessor[].addBlock(
src, ForkedSignedBeaconBlock.init(signedBlock),
Opt.some(blobs),
validationDur = nanoseconds(
(self.getCurrentBeaconTime() - wallTime).nanoseconds))
# Validator monitor registration for blocks is done by the processor
beacon_blocks_received.inc()
beacon_block_delay.observe(delay.toFloatSeconds())
sidecarRes
proc setupDoppelgangerDetection*(self: var Eth2Processor, slot: Slot) = proc setupDoppelgangerDetection*(self: var Eth2Processor, slot: Slot) =
# When another client's already running, this is very likely to detect # When another client's already running, this is very likely to detect
# potential duplicate validators, which can trigger slashing. # potential duplicate validators, which can trigger slashing.

View File

@ -223,22 +223,72 @@ template validateBeaconBlockBellatrix(
# cannot occur here, because Nimbus's optimistic sync waits for either # cannot occur here, because Nimbus's optimistic sync waits for either
# `ACCEPTED` or `SYNCING` from the EL to get this far. # `ACCEPTED` or `SYNCING` from the EL to get this far.
template validateBlobsSidecar(
signed_beacon_block: phase0.SignedBeaconBlock | altair.SignedBeaconBlock |
bellatrix.SignedBeaconBlock | capella.SignedBeaconBlock): untyped =
discard
template validateBlobsSidecar(
signed_beacon_block: eip4844.SignedBeaconBlockAndBlobsSidecar):
untyped =
# TODO
# [REJECT] The KZG commitments of the blobs are all correctly encoded
# compressed BLS G1 points -- i.e. all(bls.KeyValidate(commitment) for
# commitment in block.body.blob_kzg_commitments)
# [REJECT] The KZG commitments correspond to the versioned hashes in
# the transactions list --
# i.e. verify_kzg_commitments_against_transactions(block.body.execution_payload.transactions,
# block.body.blob_kzg_commitments)
if not verify_kzg_commitments_against_transactions(
signed_beacon_block.beacon_block.message.body.execution_payload.transactions.asSeq,
signed_beacon_block.beacon_block.message.body.blob_kzg_commitments.asSeq):
return errReject("KZG blob commitments not correctly encoded")
let sidecar = signed_beacon_block.blobs_sidecar
# [IGNORE] the sidecar.beacon_block_slot is for the current slot
# (with a MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance) -- i.e.
# sidecar.beacon_block_slot == block.slot.
if not (sidecar.beacon_block_slot == signed_beacon_block.beacon_block.message.slot):
return errIgnore("sidecar and block slots not equal")
# [REJECT] the sidecar.blobs are all well formatted, i.e. the
# BLSFieldElement in valid range (x < BLS_MODULUS).
for blob in sidecar.blobs:
for i in 0..<blob.len div 8:
let fe = UInt256.fromBytesBE(blob[i*8..(i+1)*8])
if fe >= BLS_MODULUS:
return errIgnore("BLSFieldElement outside of valid range")
# TODO
# [REJECT] The KZG proof is a correctly encoded compressed BLS G1
# point -- i.e. bls.KeyValidate(blobs_sidecar.kzg_aggregated_proof)
# [REJECT] The KZG commitments in the block are valid against the
# provided blobs sidecar -- i.e. validate_blobs_sidecar(block.slot,
# hash_tree_root(block), block.body.blob_kzg_commitments, sidecar)
let res = validate_blobs_sidecar(signed_beacon_block.beacon_block.message.slot,
hash_tree_root(signed_beacon_block.beacon_block),
signed_beacon_block.beacon_block.message
.body.blob_kzg_commitments.asSeq,
sidecar)
if res.isErr():
return errIgnore(res.error())
# https://github.com/ethereum/consensus-specs/blob/v1.1.9/specs/phase0/p2p-interface.md#beacon_block # https://github.com/ethereum/consensus-specs/blob/v1.1.9/specs/phase0/p2p-interface.md#beacon_block
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-alpha.0/specs/bellatrix/p2p-interface.md#beacon_block # https://github.com/ethereum/consensus-specs/blob/v1.3.0-alpha.0/specs/bellatrix/p2p-interface.md#beacon_block
proc validateBeaconBlock*( proc validateBeaconBlock*(
dag: ChainDAGRef, quarantine: ref Quarantine, dag: ChainDAGRef, quarantine: ref Quarantine,
signed_beacon_block: phase0.SignedBeaconBlock | altair.SignedBeaconBlock | signed_beacon_block_and_blobs: ForkySignedBeaconBlockMaybeBlobs,
bellatrix.SignedBeaconBlock | capella.SignedBeaconBlock |
eip4844.SignedBeaconBlock,
blobs: Opt[eip4844.BlobsSidecar],
wallTime: BeaconTime, flags: UpdateFlags): Result[void, ValidationError] = wallTime: BeaconTime, flags: UpdateFlags): Result[void, ValidationError] =
# In general, checks are ordered from cheap to expensive. Especially, crypto # In general, checks are ordered from cheap to expensive. Especially, crypto
# verification could be quite a bit more expensive than the rest. This is an # verification could be quite a bit more expensive than the rest. This is an
# externally easy-to-invoke function by tossing network packets at the node. # externally easy-to-invoke function by tossing network packets at the node.
# We should enforce this statically via the type system... but for now, assert. let signed_beacon_block = toSignedBeaconBlock(signed_beacon_block_and_blobs)
when typeof(signed_beacon_block).toFork() < BeaconBlockFork.EIP4844:
doAssert blobs.isNone(), "Blobs with pre-EIP4844 block"
# [IGNORE] The block is not from a future slot (with a # [IGNORE] The block is not from a future slot (with a
# MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance) -- i.e. validate that # MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance) -- i.e. validate that
@ -327,6 +377,11 @@ proc validateBeaconBlock*(
# validation. # validation.
return errReject("BeaconBlock: rejected, parent from unviable fork") return errReject("BeaconBlock: rejected, parent from unviable fork")
let blobs =
when signed_beacon_block is eip4844.SignedBeaconBlockAndBlobsSidecar:
Opt.some(signed_beacon_block.blobs_sidecar)
else:
Opt.none(eip4844.BlobsSidecar)
# When the parent is missing, we can't validate the block - we'll queue it # When the parent is missing, we can't validate the block - we'll queue it
# in the quarantine for later processing # in the quarantine for later processing
if not quarantine[].addOrphan( if not quarantine[].addOrphan(
@ -393,67 +448,10 @@ proc validateBeaconBlock*(
return errReject("BeaconBlock: Invalid proposer signature") return errReject("BeaconBlock: Invalid proposer signature")
validateBlobsSidecar(signed_beacon_block_and_blobs)
ok() ok()
proc validateBeaconBlock*(
dag: ChainDAGRef, quarantine: ref Quarantine,
signed_beacon_block: phase0.SignedBeaconBlock | altair.SignedBeaconBlock |
bellatrix.SignedBeaconBlock | capella.SignedBeaconBlock |
eip4844.SignedBeaconBlock,
wallTime: BeaconTime, flags: UpdateFlags): Result[void, ValidationError] =
dag.validateBeaconBlock(quarantine, signed_beacon_block,
Opt.none(eip4844.BlobsSidecar), wallTime, flags)
proc validateBeaconBlockAndBlobsSidecar*(signedBlock: SignedBeaconBlockAndBlobsSidecar):
Result[void, ValidationError] =
# TODO
# [REJECT] The KZG commitments of the blobs are all correctly encoded
# compressed BLS G1 points -- i.e. all(bls.KeyValidate(commitment) for
# commitment in block.body.blob_kzg_commitments)
# [REJECT] The KZG commitments correspond to the versioned hashes in
# the transactions list --
# i.e. verify_kzg_commitments_against_transactions(block.body.execution_payload.transactions,
# block.body.blob_kzg_commitments)
if not verify_kzg_commitments_against_transactions(
signedBlock.beacon_block.message.body.execution_payload.transactions.asSeq,
signedBlock.beacon_block.message.body.blob_kzg_commitments.asSeq):
return errReject("KZG blob commitments not correctly encoded")
let sidecar = signedBlock.blobs_sidecar
# [IGNORE] the sidecar.beacon_block_slot is for the current slot
# (with a MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance) -- i.e.
# sidecar.beacon_block_slot == block.slot.
if not (sidecar.beacon_block_slot == signedBlock.beacon_block.message.slot):
return errIgnore("sidecar and block slots not equal")
# [REJECT] the sidecar.blobs are all well formatted, i.e. the
# BLSFieldElement in valid range (x < BLS_MODULUS).
for blob in sidecar.blobs:
for i in 0..<blob.len div 8:
let fe = UInt256.fromBytesBE(blob[i*8..(i+1)*8])
if fe >= BLS_MODULUS:
return errIgnore("BLSFieldElement outside of valid range")
# TODO
# [REJECT] The KZG proof is a correctly encoded compressed BLS G1
# point -- i.e. bls.KeyValidate(blobs_sidecar.kzg_aggregated_proof)
# [REJECT] The KZG commitments in the block are valid against the
# provided blobs sidecar -- i.e. validate_blobs_sidecar(block.slot,
# hash_tree_root(block), block.body.blob_kzg_commitments, sidecar)
let res = validate_blobs_sidecar(signedBlock.beacon_block.message.slot,
hash_tree_root(signedBlock.beacon_block),
signedBlock.beacon_block.message
.body.blob_kzg_commitments.asSeq,
sidecar)
if res.isOk():
ok()
else:
errIgnore(res.error())
# https://github.com/ethereum/consensus-specs/blob/v1.1.9/specs/phase0/p2p-interface.md#beacon_attestation_subnet_id # https://github.com/ethereum/consensus-specs/blob/v1.1.9/specs/phase0/p2p-interface.md#beacon_attestation_subnet_id
proc validateAttestation*( proc validateAttestation*(
pool: ref AttestationPool, pool: ref AttestationPool,

View File

@ -1468,7 +1468,7 @@ proc installMessageValidators(node: BeaconNode) =
getBeaconBlockAndBlobsSidecarTopic(forkDigests.eip4844), getBeaconBlockAndBlobsSidecarTopic(forkDigests.eip4844),
proc (signedBlock: eip4844.SignedBeaconBlockAndBlobsSidecar): ValidationResult = proc (signedBlock: eip4844.SignedBeaconBlockAndBlobsSidecar): ValidationResult =
# TODO: take into account node.shouldSyncOptimistically(node.currentSlot) # TODO: take into account node.shouldSyncOptimistically(node.currentSlot)
toValidationResult(node.processor[].processSignedBeaconBlockAndBlobsSidecar( toValidationResult(node.processor[].processSignedBeaconBlock(
MsgSource.gossip, signedBlock))) MsgSource.gossip, signedBlock)))
template installSyncCommitteeeValidators(digest: auto) = template installSyncCommitteeeValidators(digest: auto) =

View File

@ -14,7 +14,7 @@ import
../beacon_node, ../beacon_node,
../consensus_object_pools/[blockchain_dag, exit_pool, spec_cache], ../consensus_object_pools/[blockchain_dag, exit_pool, spec_cache],
../spec/[deposit_snapshots, eth2_merkleization, forks, network, validator], ../spec/[deposit_snapshots, eth2_merkleization, forks, network, validator],
../spec/datatypes/[phase0, altair], ../spec/datatypes/[phase0, altair, eip4844],
../validators/message_router_mev ../validators/message_router_mev
export rest_utils export rest_utils
@ -802,7 +802,17 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
withBlck(forked): withBlck(forked):
blck.root = hash_tree_root(blck.message) blck.root = hash_tree_root(blck.message)
await node.router.routeSignedBeaconBlock(blck) let signedBlockAndBlobs =
when blck is eip4844.SignedBeaconBlock:
# TODO: Fetch blobs from EE
eip4844.SignedBeaconBlockAndBlobsSidecar(
beacon_block: blck,
blobs_sidecar: eip4844.BlobsSidecar()
)
else:
blck
await node.router.routeSignedBeaconBlock(signedBlockAndBlobs)
if res.isErr(): if res.isErr():
return RestApiResponse.jsonError( return RestApiResponse.jsonError(
@ -875,7 +885,16 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
let res = withBlck(forked): let res = withBlck(forked):
blck.root = hash_tree_root(blck.message) blck.root = hash_tree_root(blck.message)
await node.router.routeSignedBeaconBlock(blck) let signedBlockAndBlobs =
when blck is eip4844.SignedBeaconBlock:
eip4844.SignedBeaconBlockAndBlobsSidecar(
beacon_block: blck,
blobs_sidecar: eip4844.BlobsSidecar()
)
else:
blck
await node.router.routeSignedBeaconBlock(signedBlockAndBlobs)
if res.isErr(): if res.isErr():
return RestApiResponse.jsonError( return RestApiResponse.jsonError(

View File

@ -256,6 +256,25 @@ type
capella*: ForkDigest capella*: ForkDigest
eip4844*: ForkDigest eip4844*: ForkDigest
# The purpose of this type is to unify the pre- and post-EIP4844
# block gossip structures. It is for used only for
# gossip-originating blocks, which are eventually separated into the
# constituent parts before passing along into core functions.
type ForkySignedBeaconBlockMaybeBlobs* =
phase0.SignedBeaconBlock |
altair.SignedBeaconBlock |
bellatrix.SignedBeaconBlock |
capella.SignedBeaconBlock |
eip4844.SignedBeaconBlockAndBlobsSidecar
# ForkySignedBeaconBlockMaybeBlobs should only contain types that are gossiped.
static: doAssert not (default(eip4844.SignedBeaconBlock) is ForkySignedBeaconBlockMaybeBlobs)
template toSignedBeaconBlock*(b: ForkySignedBeaconBlockMaybeBlobs): ForkySignedBeaconBlock =
when b is eip4844.SignedBeaconBlockAndBlobsSidecar:
b.beacon_block
else:
b
template toFork*[T: phase0.BeaconState | phase0.HashedBeaconState]( template toFork*[T: phase0.BeaconState | phase0.HashedBeaconState](
t: type T): BeaconStateFork = t: type T): BeaconStateFork =
BeaconStateFork.Phase0 BeaconStateFork.Phase0
@ -508,6 +527,7 @@ template toFork*[T:
eip4844.ExecutionPayload | eip4844.ExecutionPayload |
eip4844.BeaconBlock | eip4844.BeaconBlock |
eip4844.SignedBeaconBlock | eip4844.SignedBeaconBlock |
eip4844.SignedBeaconBlockAndBlobsSidecar |
eip4844.TrustedBeaconBlock | eip4844.TrustedBeaconBlock |
eip4844.SigVerifiedSignedBeaconBlock | eip4844.SigVerifiedSignedBeaconBlock |
eip4844.MsgTrustedSignedBeaconBlock | eip4844.MsgTrustedSignedBeaconBlock |

View File

@ -83,19 +83,21 @@ template getCurrentBeaconTime(router: MessageRouter): BeaconTime =
type RouteBlockResult* = Result[Opt[BlockRef], cstring] type RouteBlockResult* = Result[Opt[BlockRef], cstring]
proc routeSignedBeaconBlock*( proc routeSignedBeaconBlock*(
router: ref MessageRouter, blck: ForkySignedBeaconBlock): router: ref MessageRouter, blckAndBlobs: ForkySignedBeaconBlockMaybeBlobs):
Future[RouteBlockResult] {.async.} = Future[RouteBlockResult] {.async.} =
## Validate and broadcast beacon block, then add it to the block database ## Validate and broadcast beacon block, then add it to the block database
## Returns the new Head when block is added successfully to dag, none when ## Returns the new Head when block is added successfully to dag, none when
## block passes validation but is not added, and error otherwise ## block passes validation but is not added, and error otherwise
let let
wallTime = router[].getCurrentBeaconTime() wallTime = router[].getCurrentBeaconTime()
blck = toSignedBeaconBlock(blckAndBlobs)
# Start with a quick gossip validation check such that broadcasting the # Start with a quick gossip validation check such that broadcasting the
# block doesn't get the node into trouble # block doesn't get the node into trouble
block: block:
let res = validateBeaconBlock( let res = validateBeaconBlock(
router[].dag, router[].quarantine, blck, wallTime, {}) router[].dag, router[].quarantine, blckAndBlobs, wallTime, {})
if not res.isGoodForSending(): if not res.isGoodForSending():
warn "Block failed validation", warn "Block failed validation",
blockRoot = shortLog(blck.root), blck = shortLog(blck.message), blockRoot = shortLog(blck.root), blck = shortLog(blck.message),

View File

@ -896,9 +896,13 @@ proc proposeBlock(node: BeaconNode,
elif blck is capella.BeaconBlock: elif blck is capella.BeaconBlock:
capella.SignedBeaconBlock( capella.SignedBeaconBlock(
message: blck, signature: signature, root: blockRoot) message: blck, signature: signature, root: blockRoot)
# TODO: Fetch blobs from EE
# https://github.com/ethereum/consensus-specs/blob/dev/specs/eip4844/validator.md#blob-kzg-commitments
elif blck is eip4844.BeaconBlock: elif blck is eip4844.BeaconBlock:
eip4844.SignedBeaconBlock( eip4844.SignedBeaconBlockAndBlobsSidecar(
message: blck, signature: signature, root: blockRoot) beacon_block:eip4844.SignedBeaconBlock(message: blck, signature: signature, root: blockRoot),
blobs_sidecar: eip4844.BlobsSidecar()
)
else: else:
static: doAssert "Unknown SignedBeaconBlock type" static: doAssert "Unknown SignedBeaconBlock type"
newBlockRef = newBlockRef =