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:
parent
fda03548e3
commit
727920a571
|
@ -190,10 +190,11 @@ proc new*(T: type Eth2Processor,
|
|||
|
||||
proc processSignedBeaconBlock*(
|
||||
self: var Eth2Processor, src: MsgSource,
|
||||
signedBlock: ForkySignedBeaconBlock): ValidationRes =
|
||||
signedBlockAndBlobs: ForkySignedBeaconBlockMaybeBlobs): ValidationRes =
|
||||
let
|
||||
wallTime = self.getCurrentBeaconTime()
|
||||
(afterGenesis, wallSlot) = wallTime.toSlot()
|
||||
signedBlock = toSignedBeaconBlock(signedBlockAndBlobs)
|
||||
|
||||
logScope:
|
||||
blockRoot = shortLog(signedBlock.root)
|
||||
|
@ -213,7 +214,7 @@ proc processSignedBeaconBlock*(
|
|||
debug "Block received", delay
|
||||
|
||||
let v =
|
||||
self.dag.validateBeaconBlock(self.quarantine, signedBlock, wallTime, {})
|
||||
self.dag.validateBeaconBlock(self.quarantine, signedBlockAndBlobs, wallTime, {})
|
||||
|
||||
if v.isOk():
|
||||
# Block passed validation - enqueue it for processing. The block processing
|
||||
|
@ -241,67 +242,6 @@ proc processSignedBeaconBlock*(
|
|||
|
||||
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) =
|
||||
# When another client's already running, this is very likely to detect
|
||||
# potential duplicate validators, which can trigger slashing.
|
||||
|
|
|
@ -223,22 +223,72 @@ template validateBeaconBlockBellatrix(
|
|||
# cannot occur here, because Nimbus's optimistic sync waits for either
|
||||
# `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.3.0-alpha.0/specs/bellatrix/p2p-interface.md#beacon_block
|
||||
proc validateBeaconBlock*(
|
||||
dag: ChainDAGRef, quarantine: ref Quarantine,
|
||||
signed_beacon_block: phase0.SignedBeaconBlock | altair.SignedBeaconBlock |
|
||||
bellatrix.SignedBeaconBlock | capella.SignedBeaconBlock |
|
||||
eip4844.SignedBeaconBlock,
|
||||
blobs: Opt[eip4844.BlobsSidecar],
|
||||
signed_beacon_block_and_blobs: ForkySignedBeaconBlockMaybeBlobs,
|
||||
wallTime: BeaconTime, flags: UpdateFlags): Result[void, ValidationError] =
|
||||
# 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
|
||||
# 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.
|
||||
when typeof(signed_beacon_block).toFork() < BeaconBlockFork.EIP4844:
|
||||
doAssert blobs.isNone(), "Blobs with pre-EIP4844 block"
|
||||
let signed_beacon_block = toSignedBeaconBlock(signed_beacon_block_and_blobs)
|
||||
|
||||
# [IGNORE] The block is not from a future slot (with a
|
||||
# MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance) -- i.e. validate that
|
||||
|
@ -327,6 +377,11 @@ proc validateBeaconBlock*(
|
|||
# validation.
|
||||
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
|
||||
# in the quarantine for later processing
|
||||
if not quarantine[].addOrphan(
|
||||
|
@ -393,67 +448,10 @@ proc validateBeaconBlock*(
|
|||
|
||||
return errReject("BeaconBlock: Invalid proposer signature")
|
||||
|
||||
validateBlobsSidecar(signed_beacon_block_and_blobs)
|
||||
|
||||
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
|
||||
proc validateAttestation*(
|
||||
pool: ref AttestationPool,
|
||||
|
|
|
@ -1468,7 +1468,7 @@ proc installMessageValidators(node: BeaconNode) =
|
|||
getBeaconBlockAndBlobsSidecarTopic(forkDigests.eip4844),
|
||||
proc (signedBlock: eip4844.SignedBeaconBlockAndBlobsSidecar): ValidationResult =
|
||||
# TODO: take into account node.shouldSyncOptimistically(node.currentSlot)
|
||||
toValidationResult(node.processor[].processSignedBeaconBlockAndBlobsSidecar(
|
||||
toValidationResult(node.processor[].processSignedBeaconBlock(
|
||||
MsgSource.gossip, signedBlock)))
|
||||
|
||||
template installSyncCommitteeeValidators(digest: auto) =
|
||||
|
|
|
@ -14,7 +14,7 @@ import
|
|||
../beacon_node,
|
||||
../consensus_object_pools/[blockchain_dag, exit_pool, spec_cache],
|
||||
../spec/[deposit_snapshots, eth2_merkleization, forks, network, validator],
|
||||
../spec/datatypes/[phase0, altair],
|
||||
../spec/datatypes/[phase0, altair, eip4844],
|
||||
../validators/message_router_mev
|
||||
|
||||
export rest_utils
|
||||
|
@ -802,7 +802,17 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
|
|||
|
||||
withBlck(forked):
|
||||
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():
|
||||
return RestApiResponse.jsonError(
|
||||
|
@ -875,7 +885,16 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
|
|||
|
||||
let res = withBlck(forked):
|
||||
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():
|
||||
return RestApiResponse.jsonError(
|
||||
|
|
|
@ -256,6 +256,25 @@ type
|
|||
capella*: 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](
|
||||
t: type T): BeaconStateFork =
|
||||
BeaconStateFork.Phase0
|
||||
|
@ -508,6 +527,7 @@ template toFork*[T:
|
|||
eip4844.ExecutionPayload |
|
||||
eip4844.BeaconBlock |
|
||||
eip4844.SignedBeaconBlock |
|
||||
eip4844.SignedBeaconBlockAndBlobsSidecar |
|
||||
eip4844.TrustedBeaconBlock |
|
||||
eip4844.SigVerifiedSignedBeaconBlock |
|
||||
eip4844.MsgTrustedSignedBeaconBlock |
|
||||
|
|
|
@ -83,19 +83,21 @@ template getCurrentBeaconTime(router: MessageRouter): BeaconTime =
|
|||
|
||||
type RouteBlockResult* = Result[Opt[BlockRef], cstring]
|
||||
proc routeSignedBeaconBlock*(
|
||||
router: ref MessageRouter, blck: ForkySignedBeaconBlock):
|
||||
router: ref MessageRouter, blckAndBlobs: ForkySignedBeaconBlockMaybeBlobs):
|
||||
Future[RouteBlockResult] {.async.} =
|
||||
## 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
|
||||
## block passes validation but is not added, and error otherwise
|
||||
let
|
||||
wallTime = router[].getCurrentBeaconTime()
|
||||
blck = toSignedBeaconBlock(blckAndBlobs)
|
||||
|
||||
# Start with a quick gossip validation check such that broadcasting the
|
||||
# block doesn't get the node into trouble
|
||||
block:
|
||||
let res = validateBeaconBlock(
|
||||
router[].dag, router[].quarantine, blck, wallTime, {})
|
||||
router[].dag, router[].quarantine, blckAndBlobs, wallTime, {})
|
||||
|
||||
if not res.isGoodForSending():
|
||||
warn "Block failed validation",
|
||||
blockRoot = shortLog(blck.root), blck = shortLog(blck.message),
|
||||
|
|
|
@ -896,9 +896,13 @@ proc proposeBlock(node: BeaconNode,
|
|||
elif blck is capella.BeaconBlock:
|
||||
capella.SignedBeaconBlock(
|
||||
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:
|
||||
eip4844.SignedBeaconBlock(
|
||||
message: blck, signature: signature, root: blockRoot)
|
||||
eip4844.SignedBeaconBlockAndBlobsSidecar(
|
||||
beacon_block:eip4844.SignedBeaconBlock(message: blck, signature: signature, root: blockRoot),
|
||||
blobs_sidecar: eip4844.BlobsSidecar()
|
||||
)
|
||||
else:
|
||||
static: doAssert "Unknown SignedBeaconBlock type"
|
||||
newBlockRef =
|
||||
|
|
Loading…
Reference in New Issue