mirror of
https://github.com/status-im/nimbus-eth2.git
synced 2025-02-02 09:46:26 +00:00
eip4844 gossip (#4444)
* eip4844 gossip * Check BLSFieldElement range validity in gossip validation * lint/nits cleanup * Use template to avoid an assignment with copy. * More review feedback * lint * lint * processSignedBeaconBlockAndBlobsSidecar: clean up error handling flow * Undo factoring-out of beacon blocks validator installation
This commit is contained in:
parent
aff0505807
commit
8251cc223d
@ -15,7 +15,7 @@ import
|
|||||||
stew/results,
|
stew/results,
|
||||||
chronicles, chronos, metrics, taskpools,
|
chronicles, chronos, metrics, taskpools,
|
||||||
../spec/[helpers, forks],
|
../spec/[helpers, forks],
|
||||||
../spec/datatypes/[altair, phase0],
|
../spec/datatypes/[altair, phase0, eip4844],
|
||||||
../consensus_object_pools/[
|
../consensus_object_pools/[
|
||||||
block_clearance, block_quarantine, blockchain_dag, exit_pool, attestation_pool,
|
block_clearance, block_quarantine, blockchain_dag, exit_pool, attestation_pool,
|
||||||
light_client_pool, sync_committee_msg_pool],
|
light_client_pool, sync_committee_msg_pool],
|
||||||
@ -240,6 +240,64 @@ 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
|
||||||
|
|
||||||
|
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, 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),
|
||||||
|
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.
|
||||||
|
@ -25,6 +25,8 @@ import
|
|||||||
./batch_validation
|
./batch_validation
|
||||||
|
|
||||||
from ../spec/datatypes/capella import SignedBeaconBlock
|
from ../spec/datatypes/capella import SignedBeaconBlock
|
||||||
|
from ../spec/datatypes/eip4844 import
|
||||||
|
SignedBeaconBlock, SignedBeaconBlockAndBlobsSidecar, BLS_MODULUS
|
||||||
|
|
||||||
from libp2p/protocols/pubsub/pubsub import ValidationResult
|
from libp2p/protocols/pubsub/pubsub import ValidationResult
|
||||||
|
|
||||||
@ -182,7 +184,8 @@ template validateBeaconBlockBellatrix(
|
|||||||
|
|
||||||
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-alpha.2/specs/bellatrix/p2p-interface.md#beacon_block
|
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-alpha.2/specs/bellatrix/p2p-interface.md#beacon_block
|
||||||
template validateBeaconBlockBellatrix(
|
template validateBeaconBlockBellatrix(
|
||||||
signed_beacon_block: bellatrix.SignedBeaconBlock | capella.SignedBeaconBlock,
|
signed_beacon_block: bellatrix.SignedBeaconBlock |
|
||||||
|
capella.SignedBeaconBlock | eip4844.SignedBeaconBlock,
|
||||||
parent: BlockRef): untyped =
|
parent: BlockRef): untyped =
|
||||||
# If the execution is enabled for the block -- i.e.
|
# If the execution is enabled for the block -- i.e.
|
||||||
# is_execution_enabled(state, block.body) then validate the following:
|
# is_execution_enabled(state, block.body) then validate the following:
|
||||||
@ -225,7 +228,8 @@ template validateBeaconBlockBellatrix(
|
|||||||
proc validateBeaconBlock*(
|
proc validateBeaconBlock*(
|
||||||
dag: ChainDAGRef, quarantine: ref Quarantine,
|
dag: ChainDAGRef, quarantine: ref Quarantine,
|
||||||
signed_beacon_block: phase0.SignedBeaconBlock | altair.SignedBeaconBlock |
|
signed_beacon_block: phase0.SignedBeaconBlock | altair.SignedBeaconBlock |
|
||||||
bellatrix.SignedBeaconBlock | capella.SignedBeaconBlock,
|
bellatrix.SignedBeaconBlock | capella.SignedBeaconBlock |
|
||||||
|
eip4844.SignedBeaconBlock,
|
||||||
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
|
||||||
@ -387,14 +391,55 @@ proc validateBeaconBlock*(
|
|||||||
|
|
||||||
ok()
|
ok()
|
||||||
|
|
||||||
from ../spec/datatypes/eip4844 import SignedBeaconBlock
|
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())
|
||||||
|
|
||||||
proc validateBeaconBlock*(
|
|
||||||
dag: ChainDAGRef, quarantine: ref Quarantine,
|
|
||||||
signed_beacon_block: eip4844.SignedBeaconBlock,
|
|
||||||
wallTime: BeaconTime, flags: UpdateFlags): Result[void, ValidationError] =
|
|
||||||
debugRaiseAssert $eip4844ImplementationMissing & ": gossip_validation.nim: validateBeaconBlock not how EIP4844 works anymore"
|
|
||||||
err(default(ValidationError))
|
|
||||||
|
|
||||||
# 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*(
|
||||||
|
@ -817,12 +817,14 @@ func maxGossipMaxSize(): auto {.compileTime.} =
|
|||||||
max(GOSSIP_MAX_SIZE, GOSSIP_MAX_SIZE_BELLATRIX)
|
max(GOSSIP_MAX_SIZE, GOSSIP_MAX_SIZE_BELLATRIX)
|
||||||
|
|
||||||
from ../spec/datatypes/capella import SignedBeaconBlock
|
from ../spec/datatypes/capella import SignedBeaconBlock
|
||||||
|
from ../spec/datatypes/eip4844 import SignedBeaconBlockAndBlobsSidecar
|
||||||
|
|
||||||
template gossipMaxSize(T: untyped): uint32 =
|
template gossipMaxSize(T: untyped): uint32 =
|
||||||
const maxSize = static:
|
const maxSize = static:
|
||||||
when isFixedSize(T):
|
when isFixedSize(T):
|
||||||
fixedPortionSize(T)
|
fixedPortionSize(T)
|
||||||
elif T is bellatrix.SignedBeaconBlock or T is capella.SignedBeaconBlock:
|
elif T is bellatrix.SignedBeaconBlock or T is capella.SignedBeaconBlock or
|
||||||
|
T is eip4844.SignedBeaconBlockAndBlobsSidecar:
|
||||||
GOSSIP_MAX_SIZE_BELLATRIX
|
GOSSIP_MAX_SIZE_BELLATRIX
|
||||||
# TODO https://github.com/status-im/nim-ssz-serialization/issues/20 for
|
# TODO https://github.com/status-im/nim-ssz-serialization/issues/20 for
|
||||||
# Attestation, AttesterSlashing, and SignedAggregateAndProof, which all
|
# Attestation, AttesterSlashing, and SignedAggregateAndProof, which all
|
||||||
|
@ -30,6 +30,8 @@ import
|
|||||||
when defined(posix):
|
when defined(posix):
|
||||||
import system/ansi_c
|
import system/ansi_c
|
||||||
|
|
||||||
|
from ./spec/datatypes/eip4844 import SignedBeaconBlock
|
||||||
|
|
||||||
from
|
from
|
||||||
libp2p/protocols/pubsub/gossipsub
|
libp2p/protocols/pubsub/gossipsub
|
||||||
import
|
import
|
||||||
@ -1067,25 +1069,23 @@ proc updateGossipStatus(node: BeaconNode, slot: Slot) {.async.} =
|
|||||||
|
|
||||||
let forkDigests = node.forkDigests()
|
let forkDigests = node.forkDigests()
|
||||||
|
|
||||||
discard $eip4844ImplementationMissing & "nimbus_beacon_node.nim:updateGossipStatus check EIP4844 removeMessageHandlers"
|
|
||||||
const removeMessageHandlers: array[BeaconStateFork, auto] = [
|
const removeMessageHandlers: array[BeaconStateFork, auto] = [
|
||||||
removePhase0MessageHandlers,
|
removePhase0MessageHandlers,
|
||||||
removeAltairMessageHandlers,
|
removeAltairMessageHandlers,
|
||||||
removeAltairMessageHandlers, # with different forkDigest
|
removeAltairMessageHandlers, # bellatrix (altair handlers, different forkDigest)
|
||||||
removeCapellaMessageHandlers,
|
removeCapellaMessageHandlers,
|
||||||
removeCapellaMessageHandlers
|
removeCapellaMessageHandlers # eip4844 (capella handlers, different forkDigest)
|
||||||
]
|
]
|
||||||
|
|
||||||
for gossipFork in oldGossipForks:
|
for gossipFork in oldGossipForks:
|
||||||
removeMessageHandlers[gossipFork](node, forkDigests[gossipFork])
|
removeMessageHandlers[gossipFork](node, forkDigests[gossipFork])
|
||||||
|
|
||||||
discard $eip4844ImplementationMissing & "nimbus_beacon_node.nim:updateGossipStatus check EIP4844 message addMessageHandlers"
|
|
||||||
const addMessageHandlers: array[BeaconStateFork, auto] = [
|
const addMessageHandlers: array[BeaconStateFork, auto] = [
|
||||||
addPhase0MessageHandlers,
|
addPhase0MessageHandlers,
|
||||||
addAltairMessageHandlers,
|
addAltairMessageHandlers,
|
||||||
addAltairMessageHandlers, # with different forkDigest
|
addAltairMessageHandlers, # bellatrix (altair handlers, with different forkDigest)
|
||||||
addCapellaMessageHandlers,
|
addCapellaMessageHandlers,
|
||||||
addCapellaMessageHandlers
|
addCapellaMessageHandlers # eip4844 (capella handlers, different forkDigest)
|
||||||
]
|
]
|
||||||
|
|
||||||
for gossipFork in newGossipForks:
|
for gossipFork in newGossipForks:
|
||||||
@ -1463,7 +1463,12 @@ proc installMessageValidators(node: BeaconNode) =
|
|||||||
toValidationResult(node.processor[].processSignedBeaconBlock(
|
toValidationResult(node.processor[].processSignedBeaconBlock(
|
||||||
MsgSource.gossip, signedBlock)))
|
MsgSource.gossip, signedBlock)))
|
||||||
|
|
||||||
discard $eip4844ImplementationMissing & ": add validation here, but per https://github.com/ethereum/consensus-specs/blob/v1.3.0-alpha.1/specs/eip4844/p2p-interface.md#beacon_block it's not beacon_block but beacon_block_and_blobs_sidecar"
|
node.network.addValidator(
|
||||||
|
getBeaconBlockAndBlobsSidecarTopic(forkDigests.eip4844),
|
||||||
|
proc (signedBlock: eip4844.SignedBeaconBlockAndBlobsSidecar): ValidationResult =
|
||||||
|
# TODO: take into account node.shouldSyncOptimistically(node.currentSlot)
|
||||||
|
toValidationResult(node.processor[].processSignedBeaconBlockAndBlobsSidecar(
|
||||||
|
MsgSource.gossip, signedBlock)))
|
||||||
|
|
||||||
template installSyncCommitteeeValidators(digest: auto) =
|
template installSyncCommitteeeValidators(digest: auto) =
|
||||||
for subcommitteeIdx in SyncSubcommitteeIndex:
|
for subcommitteeIdx in SyncSubcommitteeIndex:
|
||||||
|
@ -33,12 +33,17 @@ const
|
|||||||
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-alpha.1/specs/eip4844/beacon-chain.md#blob
|
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-alpha.1/specs/eip4844/beacon-chain.md#blob
|
||||||
BLOB_TX_TYPE* = 0x05'u8
|
BLOB_TX_TYPE* = 0x05'u8
|
||||||
|
|
||||||
|
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-alpha.2/specs/eip4844/polynomial-commitments.md#constants
|
||||||
|
BLS_MODULUS* = "52435875175126190479447740508185965837690552500527637822603658699938581184513".u256
|
||||||
|
|
||||||
type
|
type
|
||||||
# this block belongs elsewhere - will figure out after implementing c-kzg bindings
|
# this block belongs elsewhere - will figure out after implementing c-kzg bindings
|
||||||
KZGCommitment* = array[48, byte]
|
KZGCommitment* = array[48, byte]
|
||||||
KZGProof* = array[48, byte]
|
KZGProof* = array[48, byte]
|
||||||
BLSFieldElement* = array[32, byte]
|
BLSFieldElement* = array[32, byte]
|
||||||
|
|
||||||
|
KZGCommitmentList* = List[KZGCommitment, Limit MAX_BLOBS_PER_BLOCK]
|
||||||
|
|
||||||
# TODO this apparently is suppposed to be SSZ-equivalent to Bytes32, but
|
# TODO this apparently is suppposed to be SSZ-equivalent to Bytes32, but
|
||||||
# current spec doesn't ever SSZ-serialize it or hash_tree_root it
|
# current spec doesn't ever SSZ-serialize it or hash_tree_root it
|
||||||
VersionedHash* = array[32, byte]
|
VersionedHash* = array[32, byte]
|
||||||
@ -251,7 +256,7 @@ type
|
|||||||
# Execution
|
# Execution
|
||||||
execution_payload*: ExecutionPayload
|
execution_payload*: ExecutionPayload
|
||||||
bls_to_execution_changes*: SignedBLSToExecutionChangeList
|
bls_to_execution_changes*: SignedBLSToExecutionChangeList
|
||||||
blob_kzg_commitments*: List[KZGCommitment, Limit MAX_BLOBS_PER_BLOCK] # [New in EIP-4844]
|
blob_kzg_commitments*: KZGCommitmentList # [New in EIP-4844]
|
||||||
|
|
||||||
SigVerifiedBeaconBlockBody* = object
|
SigVerifiedBeaconBlockBody* = object
|
||||||
## A BeaconBlock body with signatures verified
|
## A BeaconBlock body with signatures verified
|
||||||
|
@ -821,10 +821,10 @@ func tx_peek_blob_versioned_hashes(opaque_tx: Transaction):
|
|||||||
res.add versionedHash
|
res.add versionedHash
|
||||||
ok res
|
ok res
|
||||||
|
|
||||||
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-alpha.1/specs/eip4844/beacon-chain.md#kzg_commitment_to_versioned_hash
|
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-alpha.2/specs/eip4844/beacon-chain.md#kzg_commitment_to_versioned_hash
|
||||||
func kzg_commitment_to_versioned_hash(
|
func kzg_commitment_to_versioned_hash(
|
||||||
kzg_commitment: KZGCommitment): VersionedHash =
|
kzg_commitment: eip4844.KZGCommitment): VersionedHash =
|
||||||
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-alpha.1/specs/eip4844/beacon-chain.md#blob
|
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-alpha.2/specs/eip4844/beacon-chain.md#blob
|
||||||
const VERSIONED_HASH_VERSION_KZG = 0x01'u8
|
const VERSIONED_HASH_VERSION_KZG = 0x01'u8
|
||||||
|
|
||||||
var res: VersionedHash
|
var res: VersionedHash
|
||||||
@ -832,10 +832,10 @@ func kzg_commitment_to_versioned_hash(
|
|||||||
res[1 .. 31] = eth2digest(kzg_commitment).data.toOpenArray(1, 31)
|
res[1 .. 31] = eth2digest(kzg_commitment).data.toOpenArray(1, 31)
|
||||||
res
|
res
|
||||||
|
|
||||||
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-alpha.1/specs/eip4844/beacon-chain.md#verify_kzg_commitments_against_transactions
|
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-alpha.2/specs/eip4844/beacon-chain.md#verify_kzg_commitments_against_transactions
|
||||||
func verify_kzg_commitments_against_transactions(
|
func verify_kzg_commitments_against_transactions*(
|
||||||
transactions: seq[Transaction],
|
transactions: seq[Transaction],
|
||||||
kzg_commitments: seq[KZGCommitment]): bool =
|
kzg_commitments: seq[eip4844.KZGCommitment]): bool =
|
||||||
var all_versioned_hashes: seq[VersionedHash]
|
var all_versioned_hashes: seq[VersionedHash]
|
||||||
for tx in transactions:
|
for tx in transactions:
|
||||||
if tx[0] == BLOB_TX_TYPE:
|
if tx[0] == BLOB_TX_TYPE:
|
||||||
@ -862,10 +862,30 @@ func process_blob_kzg_commitments(
|
|||||||
else:
|
else:
|
||||||
return err("process_blob_kzg_commitments: verify_kzg_commitments_against_transactions failed")
|
return err("process_blob_kzg_commitments: verify_kzg_commitments_against_transactions failed")
|
||||||
|
|
||||||
|
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-alpha.2/specs/eip4844/beacon-chain.md#validate_blobs_sidecar
|
||||||
|
proc validate_blobs_sidecar*(slot: Slot, root: Eth2Digest,
|
||||||
|
expected_kzg_commitments: seq[eip4844.KZGCommitment],
|
||||||
|
blobs_sidecar: eip4844.BlobsSidecar):
|
||||||
|
Result[void, cstring] =
|
||||||
|
if slot != blobs_sidecar.beacon_block_slot:
|
||||||
|
return err("validate_blobs_sidecar: different slot in block and sidecar")
|
||||||
|
|
||||||
|
if root != blobs_sidecar.beacon_block_root:
|
||||||
|
return err("validate_blobs_sidecar: different root in block and sidecar")
|
||||||
|
|
||||||
|
if expected_kzg_commitments.len != blobs_sidecar.blobs.len:
|
||||||
|
return err("validate_blobs_sidecar: different commitment lengths")
|
||||||
|
|
||||||
|
# TODO
|
||||||
|
# if not kzg_4844.verify_aggregate_kzg_proof(asSeq(blobs_sidecar.blobs), expected_kzg_commitments, blobs_sidecar.kzg_aggregated_proof):
|
||||||
|
# return err("validate_blobs_sidecar: aggregated kzg proof verification failed")
|
||||||
|
|
||||||
|
ok()
|
||||||
|
|
||||||
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-alpha.1/specs/eip4844/beacon-chain.md#is_data_available
|
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-alpha.1/specs/eip4844/beacon-chain.md#is_data_available
|
||||||
func is_data_available(
|
func is_data_available(
|
||||||
slot: Slot, beacon_block_root: Eth2Digest,
|
slot: Slot, beacon_block_root: Eth2Digest,
|
||||||
blob_kzg_commitments: seq[KZGCommitment]): bool =
|
blob_kzg_commitments: seq[eip4844.KZGCommitment]): bool =
|
||||||
discard $eip4844ImplementationMissing & ": state_transition_block.nim:is_data_available"
|
discard $eip4844ImplementationMissing & ": state_transition_block.nim:is_data_available"
|
||||||
|
|
||||||
true
|
true
|
||||||
|
@ -211,7 +211,8 @@ suite "Gossip validation - Extra": # Not based on preset config
|
|||||||
const nilCallback = OnCapellaBlockAdded(nil)
|
const nilCallback = OnCapellaBlockAdded(nil)
|
||||||
dag.addHeadBlock(verifier, blck.capellaData, nilCallback)
|
dag.addHeadBlock(verifier, blck.capellaData, nilCallback)
|
||||||
of BeaconBlockFork.EIP4844:
|
of BeaconBlockFork.EIP4844:
|
||||||
raiseAssert $eip4844ImplementationMissing
|
const nilCallback = OnEIP4844BlockAdded(nil)
|
||||||
|
dag.addHeadBlock(verifier, blck.eip4844Data, nilCallback)
|
||||||
check: added.isOk()
|
check: added.isOk()
|
||||||
dag.updateHead(added[], quarantine[])
|
dag.updateHead(added[], quarantine[])
|
||||||
dag
|
dag
|
||||||
|
Loading…
x
Reference in New Issue
Block a user