Remove ForkySignedBeaconBlockMaybeBlobs (#4681)
This commit removes ForkySignedBeaconBlockMaybeBlobs and all references. I tried to pull that thread only as little as was needed to get rid of it. Left a placeholder BlobSidecar array (in lieu of Opt[BlobsSidecar]) in a few places; this will be used as we rebuild the decoupled implementation.
This commit is contained in:
parent
f46ed12f04
commit
3681177cf4
|
@ -46,9 +46,10 @@ const
|
||||||
## Number of slots from wall time that we start processing every payload
|
## Number of slots from wall time that we start processing every payload
|
||||||
|
|
||||||
type
|
type
|
||||||
BlockEntry* = object
|
BlobSidecars* = List[Blob, Limit MAX_BLOBS_PER_BLOCK]
|
||||||
|
BlockEntry = object
|
||||||
blck*: ForkedSignedBeaconBlock
|
blck*: ForkedSignedBeaconBlock
|
||||||
blobs*: Opt[eip4844.BlobsSidecar]
|
blobs*: BlobSidecars
|
||||||
maybeFinalized*: bool
|
maybeFinalized*: bool
|
||||||
## The block source claims the block has been finalized already
|
## The block source claims the block has been finalized already
|
||||||
resfut*: Future[Result[void, VerifierError]]
|
resfut*: Future[Result[void, VerifierError]]
|
||||||
|
@ -110,7 +111,7 @@ type
|
||||||
|
|
||||||
proc addBlock*(
|
proc addBlock*(
|
||||||
self: var BlockProcessor, src: MsgSource, blck: ForkedSignedBeaconBlock,
|
self: var BlockProcessor, src: MsgSource, blck: ForkedSignedBeaconBlock,
|
||||||
blobs: Opt[eip4844.BlobsSidecar],
|
blobs: BlobSidecars,
|
||||||
resfut: Future[Result[void, VerifierError]] = nil,
|
resfut: Future[Result[void, VerifierError]] = nil,
|
||||||
maybeFinalized = false,
|
maybeFinalized = false,
|
||||||
validationDur = Duration())
|
validationDur = Duration())
|
||||||
|
@ -170,7 +171,7 @@ from ../beacon_chain_db import putBlobsSidecar
|
||||||
proc storeBackfillBlock(
|
proc storeBackfillBlock(
|
||||||
self: var BlockProcessor,
|
self: var BlockProcessor,
|
||||||
signedBlock: ForkySignedBeaconBlock,
|
signedBlock: ForkySignedBeaconBlock,
|
||||||
blobs: Opt[eip4844.BlobsSidecar]): Result[void, VerifierError] =
|
blobs: BlobSidecars): Result[void, VerifierError] =
|
||||||
|
|
||||||
# The block is certainly not missing any more
|
# The block is certainly not missing any more
|
||||||
self.consensusManager.quarantine[].missing.del(signedBlock.root)
|
self.consensusManager.quarantine[].missing.del(signedBlock.root)
|
||||||
|
@ -179,12 +180,8 @@ proc storeBackfillBlock(
|
||||||
# writing the block in case of blob error.
|
# writing the block in case of blob error.
|
||||||
let blobsOk =
|
let blobsOk =
|
||||||
when typeof(signedBlock).toFork() >= ConsensusFork.EIP4844:
|
when typeof(signedBlock).toFork() >= ConsensusFork.EIP4844:
|
||||||
blobs.isNone or
|
blobs.len > 0 or true
|
||||||
validate_blobs_sidecar(signedBlock.message.slot,
|
# TODO: validate blobs
|
||||||
signedBlock.root,
|
|
||||||
signedBlock.message
|
|
||||||
.body.blob_kzg_commitments.asSeq,
|
|
||||||
blobs.get()).isOk()
|
|
||||||
else:
|
else:
|
||||||
true
|
true
|
||||||
if not blobsOk:
|
if not blobsOk:
|
||||||
|
@ -208,9 +205,9 @@ proc storeBackfillBlock(
|
||||||
else: discard
|
else: discard
|
||||||
return res
|
return res
|
||||||
|
|
||||||
if blobs.isSome():
|
|
||||||
# Only store blobs after successfully establishing block viability.
|
# Only store blobs after successfully establishing block viability.
|
||||||
self.consensusManager.dag.db.putBlobsSidecar(blobs.get())
|
# TODO: store blobs in db
|
||||||
|
|
||||||
res
|
res
|
||||||
|
|
||||||
|
|
||||||
|
@ -354,7 +351,7 @@ proc getExecutionValidity(
|
||||||
proc storeBlock*(
|
proc storeBlock*(
|
||||||
self: ref BlockProcessor, src: MsgSource, wallTime: BeaconTime,
|
self: ref BlockProcessor, src: MsgSource, wallTime: BeaconTime,
|
||||||
signedBlock: ForkySignedBeaconBlock,
|
signedBlock: ForkySignedBeaconBlock,
|
||||||
blobs: Opt[eip4844.BlobsSidecar],
|
blobs: BlobSidecars,
|
||||||
maybeFinalized = false,
|
maybeFinalized = false,
|
||||||
queueTick: Moment = Moment.now(), validationDur = Duration()):
|
queueTick: Moment = Moment.now(), validationDur = Duration()):
|
||||||
Future[Result[BlockRef, (VerifierError, ProcessingStatus)]] {.async.} =
|
Future[Result[BlockRef, (VerifierError, ProcessingStatus)]] {.async.} =
|
||||||
|
@ -419,15 +416,9 @@ proc storeBlock*(
|
||||||
# Establish blob viability before calling addHeadBlock to avoid
|
# Establish blob viability before calling addHeadBlock to avoid
|
||||||
# writing the block in case of blob error.
|
# writing the block in case of blob error.
|
||||||
when typeof(signedBlock).toFork() >= ConsensusFork.EIP4844:
|
when typeof(signedBlock).toFork() >= ConsensusFork.EIP4844:
|
||||||
if blobs.isSome():
|
if blobs.len > 0:
|
||||||
let res = validate_blobs_sidecar(signedBlock.message.slot,
|
discard
|
||||||
signedBlock.root,
|
# TODO: validate blobs
|
||||||
signedBlock.message
|
|
||||||
.body.blob_kzg_commitments.asSeq,
|
|
||||||
blobs.get())
|
|
||||||
if res.isErr():
|
|
||||||
debug "blobs sidecar validation failed", err = res.error()
|
|
||||||
return err((VerifierError.Invalid, ProcessingStatus.completed))
|
|
||||||
|
|
||||||
type Trusted = typeof signedBlock.asTrusted()
|
type Trusted = typeof signedBlock.asTrusted()
|
||||||
let blck = dag.addHeadBlock(self.verifier, signedBlock, payloadValid) do (
|
let blck = dag.addHeadBlock(self.verifier, signedBlock, payloadValid) do (
|
||||||
|
@ -486,9 +477,7 @@ proc storeBlock*(
|
||||||
# If the EL responded at all, we don't need to try again for a while
|
# If the EL responded at all, we don't need to try again for a while
|
||||||
self[].lastPayload = signedBlock.message.slot
|
self[].lastPayload = signedBlock.message.slot
|
||||||
|
|
||||||
# write blobs now that block has been written.
|
# TODO: store blobs in db
|
||||||
if blobs.isSome():
|
|
||||||
self.consensusManager.dag.db.putBlobsSidecar(blobs.get())
|
|
||||||
|
|
||||||
let storeBlockTick = Moment.now()
|
let storeBlockTick = Moment.now()
|
||||||
|
|
||||||
|
@ -591,7 +580,7 @@ proc storeBlock*(
|
||||||
|
|
||||||
for quarantined in self.consensusManager.quarantine[].pop(blck.get().root):
|
for quarantined in self.consensusManager.quarantine[].pop(blck.get().root):
|
||||||
# Process the blocks that had the newly accepted block as parent
|
# Process the blocks that had the newly accepted block as parent
|
||||||
self[].addBlock(MsgSource.gossip, quarantined, Opt.none(deneb.BlobsSidecar))
|
self[].addBlock(MsgSource.gossip, quarantined, BlobSidecars @[])
|
||||||
|
|
||||||
return Result[BlockRef, (VerifierError, ProcessingStatus)].ok blck.get
|
return Result[BlockRef, (VerifierError, ProcessingStatus)].ok blck.get
|
||||||
|
|
||||||
|
@ -600,7 +589,7 @@ proc storeBlock*(
|
||||||
|
|
||||||
proc addBlock*(
|
proc addBlock*(
|
||||||
self: var BlockProcessor, src: MsgSource, blck: ForkedSignedBeaconBlock,
|
self: var BlockProcessor, src: MsgSource, blck: ForkedSignedBeaconBlock,
|
||||||
blobs: Opt[eip4844.BlobsSidecar],
|
blobs: BlobSidecars,
|
||||||
resfut: Future[Result[void, VerifierError]] = nil,
|
resfut: Future[Result[void, VerifierError]] = nil,
|
||||||
maybeFinalized = false,
|
maybeFinalized = false,
|
||||||
validationDur = Duration()) =
|
validationDur = Duration()) =
|
||||||
|
|
|
@ -185,19 +185,16 @@ proc new*(T: type Eth2Processor,
|
||||||
|
|
||||||
proc processSignedBeaconBlock*(
|
proc processSignedBeaconBlock*(
|
||||||
self: var Eth2Processor, src: MsgSource,
|
self: var Eth2Processor, src: MsgSource,
|
||||||
signedBlockAndBlobs: ForkySignedBeaconBlockMaybeBlobs,
|
signedBlock: ForkySignedBeaconBlock,
|
||||||
maybeFinalized: bool = false): ValidationRes =
|
maybeFinalized: bool = false): ValidationRes =
|
||||||
let
|
let
|
||||||
wallTime = self.getCurrentBeaconTime()
|
wallTime = self.getCurrentBeaconTime()
|
||||||
(afterGenesis, wallSlot) = wallTime.toSlot()
|
(afterGenesis, wallSlot) = wallTime.toSlot()
|
||||||
signedBlock = toSignedBeaconBlock(signedBlockAndBlobs)
|
|
||||||
blobs = optBlobs(signedBlockAndBlobs)
|
|
||||||
|
|
||||||
logScope:
|
logScope:
|
||||||
blockRoot = shortLog(signedBlock.root)
|
blockRoot = shortLog(signedBlock.root)
|
||||||
blck = shortLog(signedBlock.message)
|
blck = shortLog(signedBlock.message)
|
||||||
signature = shortLog(signedBlock.signature)
|
signature = shortLog(signedBlock.signature)
|
||||||
hasBlobs = blobs.isSome
|
|
||||||
wallSlot
|
wallSlot
|
||||||
|
|
||||||
if not afterGenesis:
|
if not afterGenesis:
|
||||||
|
@ -212,7 +209,7 @@ proc processSignedBeaconBlock*(
|
||||||
debug "Block received", delay
|
debug "Block received", delay
|
||||||
|
|
||||||
let v =
|
let v =
|
||||||
self.dag.validateBeaconBlock(self.quarantine, signedBlockAndBlobs, wallTime, {})
|
self.dag.validateBeaconBlock(self.quarantine, signedBlock, 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
|
||||||
|
@ -224,7 +221,7 @@ proc processSignedBeaconBlock*(
|
||||||
|
|
||||||
self.blockProcessor[].addBlock(
|
self.blockProcessor[].addBlock(
|
||||||
src, ForkedSignedBeaconBlock.init(signedBlock),
|
src, ForkedSignedBeaconBlock.init(signedBlock),
|
||||||
blobs,
|
BlobSidecars @[],
|
||||||
maybeFinalized = maybeFinalized,
|
maybeFinalized = maybeFinalized,
|
||||||
validationDur = nanoseconds(
|
validationDur = nanoseconds(
|
||||||
(self.getCurrentBeaconTime() - wallTime).nanoseconds))
|
(self.getCurrentBeaconTime() - wallTime).nanoseconds))
|
||||||
|
|
|
@ -220,73 +220,17 @@ 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,
|
|
||||||
signed_beacon_block.beacon_block.root,
|
|
||||||
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_and_blobs: ForkySignedBeaconBlockMaybeBlobs,
|
signed_beacon_block: ForkySignedBeaconBlock,
|
||||||
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.
|
||||||
|
|
||||||
let signed_beacon_block = toSignedBeaconBlock(signed_beacon_block_and_blobs)
|
|
||||||
|
|
||||||
# [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
|
||||||
# signed_beacon_block.message.slot <= current_slot (a client MAY queue future
|
# signed_beacon_block.message.slot <= current_slot (a client MAY queue future
|
||||||
|
@ -374,8 +318,6 @@ proc validateBeaconBlock*(
|
||||||
# validation.
|
# validation.
|
||||||
return errReject("BeaconBlock: rejected, parent from unviable fork")
|
return errReject("BeaconBlock: rejected, parent from unviable fork")
|
||||||
|
|
||||||
let blobs = optBlobs(signed_beacon_block_and_blobs)
|
|
||||||
|
|
||||||
# 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(
|
||||||
|
@ -439,11 +381,8 @@ proc validateBeaconBlock*(
|
||||||
dag.validatorKey(proposer).get(),
|
dag.validatorKey(proposer).get(),
|
||||||
signed_beacon_block.signature):
|
signed_beacon_block.signature):
|
||||||
quarantine[].addUnviable(signed_beacon_block.root)
|
quarantine[].addUnviable(signed_beacon_block.root)
|
||||||
|
|
||||||
return errReject("BeaconBlock: Invalid proposer signature")
|
return errReject("BeaconBlock: Invalid proposer signature")
|
||||||
|
|
||||||
validateBlobsSidecar(signed_beacon_block_and_blobs)
|
|
||||||
|
|
||||||
ok()
|
ok()
|
||||||
|
|
||||||
# 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
|
||||||
|
|
|
@ -2626,9 +2626,9 @@ proc broadcastBeaconBlock*(
|
||||||
let topic = getBeaconBlocksTopic(node.forkDigests.capella)
|
let topic = getBeaconBlocksTopic(node.forkDigests.capella)
|
||||||
node.broadcast(topic, blck)
|
node.broadcast(topic, blck)
|
||||||
|
|
||||||
proc broadcastBeaconBlockAndBlobsSidecar*(
|
proc broadcastBeaconBlock*(
|
||||||
node: Eth2Node, blck: eip4844.SignedBeaconBlockAndBlobsSidecar): Future[SendResult] =
|
node: Eth2Node, blck: eip4844.SignedBeaconBlock): Future[SendResult] =
|
||||||
let topic = getBeaconBlockAndBlobsSidecarTopic(node.forkDigests.eip4844)
|
let topic = getBeaconBlocksTopic(node.forkDigests.eip4844)
|
||||||
node.broadcast(topic, blck)
|
node.broadcast(topic, blck)
|
||||||
|
|
||||||
from ../spec/datatypes/eip4844 import SignedBeaconBlock
|
from ../spec/datatypes/eip4844 import SignedBeaconBlock
|
||||||
|
|
|
@ -328,7 +328,7 @@ proc initFullNode(
|
||||||
# that should probably be reimagined more holistically in the future.
|
# that should probably be reimagined more holistically in the future.
|
||||||
let resfut = newFuture[Result[void, VerifierError]]("blockVerifier")
|
let resfut = newFuture[Result[void, VerifierError]]("blockVerifier")
|
||||||
blockProcessor[].addBlock(MsgSource.gossip, signedBlock,
|
blockProcessor[].addBlock(MsgSource.gossip, signedBlock,
|
||||||
Opt.none(eip4844.BlobsSidecar),
|
BlobSidecars @[],
|
||||||
resfut,
|
resfut,
|
||||||
maybeFinalized = maybeFinalized)
|
maybeFinalized = maybeFinalized)
|
||||||
resfut
|
resfut
|
||||||
|
@ -342,7 +342,7 @@ proc initFullNode(
|
||||||
# that should probably be reimagined more holistically in the future.
|
# that should probably be reimagined more holistically in the future.
|
||||||
let resfut = newFuture[Result[void, VerifierError]]("blockVerifier")
|
let resfut = newFuture[Result[void, VerifierError]]("blockVerifier")
|
||||||
blockProcessor[].addBlock(MsgSource.gossip, signedBlock,
|
blockProcessor[].addBlock(MsgSource.gossip, signedBlock,
|
||||||
Opt.some(blobs), resfut, maybeFinalized = maybeFinalized)
|
BlobSidecars @[], resfut, maybeFinalized = maybeFinalized)
|
||||||
resfut
|
resfut
|
||||||
processor = Eth2Processor.new(
|
processor = Eth2Processor.new(
|
||||||
config.doppelgangerDetection,
|
config.doppelgangerDetection,
|
||||||
|
@ -1532,7 +1532,7 @@ proc installMessageValidators(node: BeaconNode) =
|
||||||
signedBlock.beacon_block))
|
signedBlock.beacon_block))
|
||||||
else:
|
else:
|
||||||
toValidationResult(node.processor[].processSignedBeaconBlock(
|
toValidationResult(node.processor[].processSignedBeaconBlock(
|
||||||
MsgSource.gossip, signedBlock)))
|
MsgSource.gossip, signedBlock.beacon_block)))
|
||||||
|
|
||||||
template installSyncCommitteeeValidators(digest: auto) =
|
template installSyncCommitteeeValidators(digest: auto) =
|
||||||
for subcommitteeIdx in SyncSubcommitteeIndex:
|
for subcommitteeIdx in SyncSubcommitteeIndex:
|
||||||
|
|
|
@ -803,17 +803,8 @@ 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)
|
||||||
let signedBlockAndBlobs =
|
# TODO: Fetch blobs from EE when blck is eip4844.SignedBeaconBlock
|
||||||
when blck is eip4844.SignedBeaconBlock:
|
await node.router.routeSignedBeaconBlock(blck)
|
||||||
# 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(
|
||||||
|
@ -901,16 +892,7 @@ 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)
|
||||||
let signedBlockAndBlobs =
|
await node.router.routeSignedBeaconBlock(blck)
|
||||||
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(
|
||||||
|
|
|
@ -254,33 +254,6 @@ 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 |
|
|
||||||
deneb.SignedBeaconBlockAndBlobsSidecar
|
|
||||||
# ForkySignedBeaconBlockMaybeBlobs should only contain types that are gossiped.
|
|
||||||
static: doAssert not (default(deneb.SignedBeaconBlock) is ForkySignedBeaconBlockMaybeBlobs)
|
|
||||||
|
|
||||||
template toSignedBeaconBlock*(b: ForkySignedBeaconBlockMaybeBlobs): ForkySignedBeaconBlock =
|
|
||||||
when b is eip4844.SignedBeaconBlockAndBlobsSidecar:
|
|
||||||
b.beacon_block
|
|
||||||
else:
|
|
||||||
b
|
|
||||||
|
|
||||||
func optBlobs*(b: ForkySignedBeaconBlockMaybeBlobs):
|
|
||||||
Opt[deneb.BlobsSidecar] =
|
|
||||||
when b is phase0.SignedBeaconBlock or b is altair.SignedBeaconBlock or
|
|
||||||
b is bellatrix.SignedBeaconBlock or b is capella.SignedBeaconBlock:
|
|
||||||
Opt.none(eip4844.BlobsSidecar)
|
|
||||||
elif b is deneb.SignedBeaconBlockAndBlobsSidecar:
|
|
||||||
Opt.some(b.blobs_sidecar)
|
|
||||||
|
|
||||||
macro getSymbolFromForkModule(fork: static ConsensusFork,
|
macro getSymbolFromForkModule(fork: static ConsensusFork,
|
||||||
symbolName: static string): untyped =
|
symbolName: static string): untyped =
|
||||||
let moduleName = case fork
|
let moduleName = case fork
|
||||||
|
|
|
@ -80,20 +80,18 @@ template getCurrentBeaconTime(router: MessageRouter): BeaconTime =
|
||||||
|
|
||||||
type RouteBlockResult* = Result[Opt[BlockRef], cstring]
|
type RouteBlockResult* = Result[Opt[BlockRef], cstring]
|
||||||
proc routeSignedBeaconBlock*(
|
proc routeSignedBeaconBlock*(
|
||||||
router: ref MessageRouter, blckAndBlobs: ForkySignedBeaconBlockMaybeBlobs):
|
router: ref MessageRouter, blck: ForkySignedBeaconBlock):
|
||||||
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, blckAndBlobs, wallTime, {})
|
router[].dag, router[].quarantine, blck, wallTime, {})
|
||||||
|
|
||||||
if not res.isGoodForSending():
|
if not res.isGoodForSending():
|
||||||
warn "Block failed validation",
|
warn "Block failed validation",
|
||||||
|
@ -108,11 +106,7 @@ proc routeSignedBeaconBlock*(
|
||||||
# now. In fact, per the spec, we should broadcast it even if it later fails
|
# now. In fact, per the spec, we should broadcast it even if it later fails
|
||||||
# to apply to our state.
|
# to apply to our state.
|
||||||
|
|
||||||
let res =
|
let res = await router[].network.broadcastBeaconBlock(blck)
|
||||||
when blckAndBlobs is eip4844.SignedBeaconBlockAndBlobsSidecar:
|
|
||||||
await router[].network.broadcastBeaconBlockAndBlobsSidecar(blckAndBlobs)
|
|
||||||
else:
|
|
||||||
await router[].network.broadcastBeaconBlock(blck)
|
|
||||||
|
|
||||||
if res.isOk():
|
if res.isOk():
|
||||||
beacon_blocks_sent.inc()
|
beacon_blocks_sent.inc()
|
||||||
|
@ -127,7 +121,7 @@ proc routeSignedBeaconBlock*(
|
||||||
signature = shortLog(blck.signature), error = res.error()
|
signature = shortLog(blck.signature), error = res.error()
|
||||||
|
|
||||||
let newBlockRef = await router[].blockProcessor.storeBlock(
|
let newBlockRef = await router[].blockProcessor.storeBlock(
|
||||||
MsgSource.api, sendTime, blck, optBlobs(blckAndBlobs))
|
MsgSource.api, sendTime, blck, BlobSidecars @[])
|
||||||
|
|
||||||
# The boolean we return tells the caller whether the block was integrated
|
# The boolean we return tells the caller whether the block was integrated
|
||||||
# into the chain
|
# into the chain
|
||||||
|
|
|
@ -1060,10 +1060,8 @@ proc proposeBlock(node: BeaconNode,
|
||||||
capella.SignedBeaconBlock(
|
capella.SignedBeaconBlock(
|
||||||
message: blck, signature: signature, root: blockRoot)
|
message: blck, signature: signature, root: blockRoot)
|
||||||
elif blck is eip4844.BeaconBlock:
|
elif blck is eip4844.BeaconBlock:
|
||||||
eip4844.SignedBeaconBlockAndBlobsSidecar(
|
# TODO: also route blobs
|
||||||
beacon_block:eip4844.SignedBeaconBlock(message: blck, signature: signature, root: blockRoot),
|
eip4844.SignedBeaconBlock(message: blck, signature: signature, root: blockRoot)
|
||||||
blobs_sidecar: blobs_sidecar
|
|
||||||
)
|
|
||||||
else:
|
else:
|
||||||
static: doAssert "Unknown SignedBeaconBlock type"
|
static: doAssert "Unknown SignedBeaconBlock type"
|
||||||
newBlockRef =
|
newBlockRef =
|
||||||
|
|
|
@ -60,7 +60,7 @@ suite "Block processor" & preset():
|
||||||
|
|
||||||
asyncTest "Reverse order block add & get" & preset():
|
asyncTest "Reverse order block add & get" & preset():
|
||||||
let missing = await processor.storeBlock(
|
let missing = await processor.storeBlock(
|
||||||
MsgSource.gossip, b2.message.slot.start_beacon_time(), b2, Opt.none(BlobsSidecar))
|
MsgSource.gossip, b2.message.slot.start_beacon_time(), b2, BlobSidecars @[])
|
||||||
check: missing.error[0] == VerifierError.MissingParent
|
check: missing.error[0] == VerifierError.MissingParent
|
||||||
|
|
||||||
check:
|
check:
|
||||||
|
@ -70,7 +70,7 @@ suite "Block processor" & preset():
|
||||||
|
|
||||||
let
|
let
|
||||||
status = await processor.storeBlock(
|
status = await processor.storeBlock(
|
||||||
MsgSource.gossip, b2.message.slot.start_beacon_time(), b1, Opt.none(BlobsSidecar))
|
MsgSource.gossip, b2.message.slot.start_beacon_time(), b1, BlobSidecars @[])
|
||||||
b1Get = dag.getBlockRef(b1.root)
|
b1Get = dag.getBlockRef(b1.root)
|
||||||
|
|
||||||
check:
|
check:
|
||||||
|
|
Loading…
Reference in New Issue