diff --git a/beacon_chain/consensus_object_pools/envelope_quarantine.nim b/beacon_chain/consensus_object_pools/envelope_quarantine.nim index 12894266df..27da7f8970 100644 --- a/beacon_chain/consensus_object_pools/envelope_quarantine.nim +++ b/beacon_chain/consensus_object_pools/envelope_quarantine.nim @@ -25,9 +25,6 @@ type func init*(T: typedesc[EnvelopeQuarantine]): T = T() -template root(v: SignedExecutionPayloadEnvelope): Eth2Digest = - v.message.beacon_block_root - func addMissing*( self: var EnvelopeQuarantine, root: Eth2Digest) = diff --git a/beacon_chain/el/el_manager.nim b/beacon_chain/el/el_manager.nim index 22c2cd792b..9bed34e368 100644 --- a/beacon_chain/el/el_manager.nim +++ b/beacon_chain/el/el_manager.nim @@ -827,27 +827,48 @@ proc sendGetBlobsV2*( proc sendNewPayload*( m: ELManager, blck: SomeForkyBeaconBlock, + envelope: NoEnvelope | gloas.ExecutionPayloadEnvelope, deadline: DeadlineFuture, retry: bool, ): Future[Opt[PayloadExecutionStatus]] {.async: (raises: [CancelledError]).} = + const consensusFork = typeof(blck).kind + + template executionPayload(): auto = + when consensusFork >= ConsensusFork.Gloas: + envelope.payload + else: + blck.body.execution_payload + if m.elConnections.len == 0: info "No execution client configured; cannot process block payloads", - executionPayload = shortLog(blck.body.execution_payload) + executionPayload = shortLog(executionPayload) return Opt.none(PayloadExecutionStatus) - const consensusFork = typeof(blck).kind - let startTime = Moment.now() - payload = blck.body.execution_payload.asEngineExecutionPayload + payload = executionPayload.asEngineExecutionPayload() when consensusFork >= ConsensusFork.Deneb: let - versioned_hashes = blck.body.blob_kzg_commitments.asEngineVersionedHashes() + versioned_hashes = + block: + let kzgCommitments = + when consensusFork >= ConsensusFork.Gloas: + envelope.blob_kzg_commitments + elif consensusFork >= ConsensusFork.Deneb: + blck.body.blob_kzg_commitments + kzgCommitments.asEngineVersionedHashes() parent_root = blck.parent_root.to(Hash32) when consensusFork >= ConsensusFork.Electra: - let execution_requests = blck.body.execution_requests.asEngineExecutionRequests() + let execution_requests = + block: + let executionRequests = + when consensusFork >= ConsensusFork.Gloas: + envelope.execution_requests + else: + blck.body.execution_requests + executionRequests.asEngineExecutionRequests() var responseProcessor = ELConsensusViolationDetector.init() diff --git a/beacon_chain/gossip_processing/block_processor.nim b/beacon_chain/gossip_processing/block_processor.nim index ccfbec1e01..c977d08bac 100644 --- a/beacon_chain/gossip_processing/block_processor.nim +++ b/beacon_chain/gossip_processing/block_processor.nim @@ -29,6 +29,8 @@ from ../consensus_object_pools/block_quarantine import remove, startProcessing, clearProcessing, UnviableKind from ../consensus_object_pools/blob_quarantine import BlobQuarantine, ColumnQuarantine, popSidecars, put +from ../consensus_object_pools/envelope_quarantine import + EnvelopeQuarantine, addMissing, popOrphan, addOrphan from ../validators/validator_monitor import MsgSource, ValidatorMonitor, registerAttestationInBlock, registerBeaconBlock, registerSyncAggregateInBlock @@ -97,6 +99,7 @@ type blobQuarantine: ref BlobQuarantine dataColumnQuarantine*: ref ColumnQuarantine + envelopeQuarantine*: ref EnvelopeQuarantine verifier: BatchVerifier lastPayload: Slot @@ -108,6 +111,9 @@ type NoSidecars | Opt[BlobSidecars] | Opt[fulu.DataColumnSidecars] | Opt[gloas.DataColumnSidecars] + SomeOptEnvelope = + NoEnvelope | Opt[gloas.SignedExecutionPayloadEnvelope] + const noSidecars* = default(NoSidecars) # Initialization @@ -121,6 +127,7 @@ proc new*(T: type BlockProcessor, validatorMonitor: ref ValidatorMonitor, blobQuarantine: ref BlobQuarantine, dataColumnQuarantine: ref ColumnQuarantine, + envelopeQuarantine: ref EnvelopeQuarantine, getBeaconTime: GetBeaconTimeFn, invalidBlockRoots: seq[Eth2Digest] = @[]): ref BlockProcessor = if invalidBlockRoots.len > 0: @@ -137,6 +144,7 @@ proc new*(T: type BlockProcessor, validatorMonitor: validatorMonitor, blobQuarantine: blobQuarantine, dataColumnQuarantine: dataColumnQuarantine, + envelopeQuarantine: envelopeQuarantine, getBeaconTime: getBeaconTime, verifier: batchVerifier[] ) @@ -178,18 +186,22 @@ from ../consensus_object_pools/block_clearance import proc verifySidecars( signedBlock: ForkySignedBeaconBlock, + envelope: NoEnvelope | gloas.SignedExecutionPayloadEnvelope, sidecarsOpt: SomeOptSidecars, ): Result[void, VerifierError] = const consensusFork = typeof(signedBlock).kind - when consensusFork == ConsensusFork.Gloas: - # For Gloas, we still need to store the columns if they're provided - # but skip validation since we don't have kzg_commitments in the block - debugGloasComment "potentially validate against payload envelope" - elif consensusFork == ConsensusFork.Fulu: + when consensusFork >= ConsensusFork.Fulu: if sidecarsOpt.isSome: let columns = sidecarsOpt.get() - let kzgCommits = signedBlock.message.body.blob_kzg_commitments.asSeq + let kzgCommits = + block: + let kzgCommits = + when consensusFork >= ConsensusFork.Gloas: + envelope.message.blob_kzg_commitments + else: + signedBlock.message.body.blob_kzg_commitments + kzgCommits.asSeq if columns.len > 0 and kzgCommits.len > 0: for i in 0 ..< columns.len: let r = verify_data_column_sidecar_kzg_proofs(columns[i][]) @@ -241,8 +253,10 @@ proc storeSidecars( proc storeSidecars(self: BlockProcessor, sidecarsOpt: NoSidecars) = discard +proc enqueuePayload*(self: ref BlockProcessor, blck: gloas.SignedBeaconBlock) + proc storeBackfillBlock( - self: var BlockProcessor, + self: ref BlockProcessor, signedBlock: ForkySignedBeaconBlock, sidecarsOpt: SomeOptSidecars, ): Result[void, VerifierError] = @@ -250,7 +264,10 @@ proc storeBackfillBlock( # In case the block was added to any part of the quarantine.. quarantine[].remove(signedBlock) - ?verifySidecars(signedBlock, sidecarsOpt) + const consensusFork = typeof(signedBlock).kind + + when consensusFork <= ConsensusFork.Fulu: + ?verifySidecars(signedBlock, noEnvelope, sidecarsOpt) let res = self.consensusManager.dag.addBackfillBlock(signedBlock) @@ -280,8 +297,13 @@ proc storeBackfillBlock( of VerifierError.Duplicate: res else: - # Only store side cars after successfully establishing block viability. - self.storeSidecars(sidecarsOpt) + when consensusFork >= ConsensusFork.Gloas: + # Columns are in quarantine as they didn't pop from `rmanBlockVerifier`, + # we simply enqueue with the valid block. + self.enqueuePayload(signedBlock) + else: + # Only store side cars after successfully establishing block viability. + self[].storeSidecars(sidecarsOpt) res @@ -293,15 +315,21 @@ from ../consensus_object_pools/spec_cache import get_attesting_indices proc newExecutionPayload*( elManager: ELManager, blck: SomeForkyBeaconBlock, + envelope: NoEnvelope | gloas.ExecutionPayloadEnvelope, deadline: DeadlineFuture, retry: bool, ): Future[Opt[PayloadExecutionStatus]] {.async: (raises: [CancelledError]).} = - template executionPayload: untyped = blck.body.execution_payload + template executionPayload: untyped = + when typeof(blck).kind >= ConsensusFork.Gloas: + envelope.payload + else: + blck.body.execution_payload debug "newPayload: inserting block into execution engine", executionPayload = shortLog(executionPayload) - let payloadStatus = ?await elManager.sendNewPayload(blck, deadline, retry) + let payloadStatus = ?await elManager.sendNewPayload( + blck, envelope, deadline, retry) debug "newPayload: succeeded", parentHash = executionPayload.parent_hash, @@ -311,26 +339,34 @@ proc newExecutionPayload*( Opt.some payloadStatus +debugGloasComment("check calls to this, mostly LC") proc newExecutionPayload*( elManager: ELManager, blck: SomeForkyBeaconBlock ): Future[Opt[PayloadExecutionStatus]] {. async: (raises: [CancelledError], raw: true).} = newExecutionPayload( - elManager, blck, sleepAsync(FORKCHOICEUPDATED_TIMEOUT), true) + elManager, blck, noEnvelope, sleepAsync(FORKCHOICEUPDATED_TIMEOUT), true) proc getExecutionValidity( elManager: ELManager, - blck: bellatrix.SignedBeaconBlock | capella.SignedBeaconBlock | - deneb.SignedBeaconBlock | electra.SignedBeaconBlock | - fulu.SignedBeaconBlock, + blck: ForkySignedBeaconBlock, + envelope: NoEnvelope | gloas.SignedExecutionPayloadEnvelope, deadline: DeadlineFuture, retry: bool, ): Future[Opt[OptimisticStatus]] {.async: (raises: [CancelledError]).} = if not blck.message.is_execution_block: return Opt.some(OptimisticStatus.valid) # vacuously - let status = (await elManager.newExecutionPayload(blck.message, deadline, retry)).valueOr: + const consensusFork = typeof(blck).kind + template forkyEnvelope(): auto = + when consensusFork >= ConsensusFork.Gloas: + envelope.message + else: + envelope + + let status = (await elManager.newExecutionPayload( + blck.message, forkyEnvelope, deadline, retry)).valueOr: return Opt.none(OptimisticStatus) let optimisticStatus = status.to(OptimisticStatus) @@ -340,9 +376,14 @@ proc getExecutionValidity( # former case, they've passed libp2p gossip validation which implies # correct signature for correct proposer,which makes spam expensive, # while for the latter, spam is limited by the request manager. + template executionPayload(): auto = + when consensusFork >= ConsensusFork.Gloas: + envelope.message.payload + else: + blck.message.body.execution_payload info "execution payload invalid from EL client newPayload", executionPayloadStatus = status, - executionPayload = shortLog(blck.message.body.execution_payload), + executionPayload = shortLog(executionPayload), blck = shortLog(blck) Opt.some(optimisticStatus) @@ -367,7 +408,7 @@ proc enqueueBlock*( if blck.message.slot <= self.consensusManager.dag.finalizedHead.slot: # let backfill blocks skip the queue - these are always "fast" to process # because there are no state rewinds to deal with - discard self[].storeBackfillBlock(blck, sidecarsOpt) + discard self.storeBackfillBlock(blck, sidecarsOpt) return # `discard` here means that the `async` task will continue running even though @@ -391,9 +432,8 @@ proc enqueueQuarantine(self: ref BlockProcessor, parent: BlockRef) = debug "Block from quarantine", parent, quarantined = shortLog(quarantined.root) withBlck(quarantined): - when consensusFork == ConsensusFork.Gloas: - debugGloasComment "" - const sidecarsOpt = noSidecars + when consensusFork >= ConsensusFork.Gloas: + let sidecarsOpt = Opt.none(gloas.DataColumnSidecars) elif consensusFork == ConsensusFork.Fulu: let sidecarsOpt = if len(forkyBlck.message.body.blob_kzg_commitments) == 0: @@ -459,37 +499,49 @@ proc onBlockAdded*( ) proc verifyPayload( - self: ref BlockProcessor, signedBlock: ForkySignedBeaconBlock + self: ref BlockProcessor, + signedBlock: ForkySignedBeaconBlock, + signedEnvelope: NoEnvelope | gloas.SignedExecutionPayloadEnvelope, ): Result[OptimisticStatus, VerifierError] = const consensusFork = typeof(signedBlock).kind # When the execution layer is not available to verify the payload, we do the # required checks on the CL instead and proceed as if the EL was syncing # https://github.com/ethereum/consensus-specs/blob/v1.6.0-alpha.6/specs/bellatrix/beacon-chain.md#verify_and_notify_new_payload # https://github.com/ethereum/consensus-specs/blob/v1.6.0-alpha.6/specs/deneb/beacon-chain.md#modified-verify_and_notify_new_payload - when consensusFork == ConsensusFork.Gloas: - debugGloasComment "no exection payload field for gloas" - ok OptimisticStatus.valid - elif consensusFork >= ConsensusFork.Bellatrix: + when consensusFork >= ConsensusFork.Bellatrix: + # Since Gloas, is_execution_block should always be true. if signedBlock.message.is_execution_block: - template payload(): auto = - signedBlock.message.body.execution_payload + template paylaod(): auto = + when consensusFork >= ConsensusFork.Gloas: + signedEnvelope.message.payload + else: + signedBlock.message.body.execution_payload template returnWithError(msg: string, extraMsg = ""): untyped = if extraMsg != "": - debug msg, reason = extraMsg, executionPayload = shortLog(payload) + debug msg, reason = extraMsg, executionPayload = shortLog(paylaod) else: - debug msg, executionPayload = shortLog(payload) + debug msg, executionPayload = shortLog(paylaod) return err(VerifierError.Invalid) - if payload.transactions.anyIt(it.len == 0): + if paylaod.transactions.anyIt(it.len == 0): returnWithError "Execution block contains zero length transactions" - if payload.block_hash != signedBlock.message.compute_execution_block_hash(): + let computedBlockHash = + when consensusFork >= ConsensusFork.Gloas: + signedBlock.message.compute_execution_block_hash(signedEnvelope.message) + else: + signedBlock.message.compute_execution_block_hash() + if paylaod.block_hash != computedBlockHash: returnWithError "Execution block hash validation failed" # [New in Deneb:EIP4844] when consensusFork >= ConsensusFork.Deneb: - let blobsRes = signedBlock.message.is_valid_versioned_hashes + let blobsRes = + when consensusFork >= ConsensusFork.Gloas: + signedBlock.message.is_valid_versioned_hashes(signedEnvelope.message) + else: + signedBlock.message.is_valid_versioned_hashes() if blobsRes.isErr: returnWithError "Blob versioned hashes invalid", blobsRes.error else: @@ -520,8 +572,9 @@ proc enqueueFromDb(self: ref BlockProcessor, root: Eth2Digest) = var sidecarsOk = true let sidecarsOpt = - when consensusFork >= ConsensusFork.Fulu: - debugGloasComment "" + when consensusFork >= ConsensusFork.Gloas: + noSidecars + elif consensusFork == ConsensusFork.Fulu: var data_column_sidecars: fulu.DataColumnSidecars for i in self.dataColumnQuarantine[].custodyColumns: let data_column = fulu.DataColumnSidecar.new() @@ -604,25 +657,38 @@ proc storeBlock( # progress in its own sync. Opt.none(OptimisticStatus) else: - when consensusFork == ConsensusFork.Gloas: - debugGloasComment "need getExecutionValidity on gloas blocks" - Opt.some OptimisticStatus.valid + when consensusFork >= ConsensusFork.Gloas: + # It is mainly for disabling the `updateExecutionHead` call. As we are + # not sure if there is a valid envelope (execution payload), the + # execution head should be updated after we get one and validate it. + Opt.none(OptimisticStatus) elif consensusFork >= ConsensusFork.Bellatrix: func shouldRetry(): bool = not dag.is_optimistic(dag.head.bid) await self.consensusManager.elManager.getExecutionValidity( - signedBlock, deadline, shouldRetry()) + signedBlock, noEnvelope, deadline, shouldRetry()) else: Opt.some(OptimisticStatus.valid) # vacuously - let optimisticStatus = ?(optimisticStatusRes or verifyPayload(self, signedBlock)) + let optimisticStatus = + when consensusFork >= ConsensusFork.Gloas: + # The execution payload validity is not known yet at block time as an + # envelope will be processed after its valid block. So always return + # `notValidated` and skip verifying payload. + # + # TODO may need a new value of `OptimisticStatus` to distinguish between + # not validated and pending? + OptimisticStatus.notValidated + else: + ?(optimisticStatusRes or verifyPayload(self, signedBlock, noEnvelope)) if OptimisticStatus.invalidated == optimisticStatus: return err(VerifierError.Invalid) let newPayloadTick = Moment.now() - ?verifySidecars(signedBlock, sidecarsOpt) + when consensusFork <= ConsensusFork.Fulu: + ?verifySidecars(signedBlock, noEnvelope, sidecarsOpt) let blck = ?dag.addHeadBlockWithParent( @@ -640,7 +706,8 @@ proc storeBlock( self[].lastPayload = signedBlock.message.slot # write blobs now that block has been written. - self[].storeSidecars(sidecarsOpt) + when consensusFork <= ConsensusFork.Fulu: + self[].storeSidecars(sidecarsOpt) let addHeadBlockTick = Moment.now() @@ -690,6 +757,11 @@ proc storeBlock( blck = shortLog(blck), validationDur, queueDur, newPayloadDur, addHeadBlockDur, updateHeadDur + when consensusFork >= ConsensusFork.Gloas: + # Enqueue payload here instead of `addBlock` for the consistency of payload + # processing with backfilling. + self.enqueuePayload(signedBlock) + ok(blck) proc addBlock*( @@ -722,7 +794,7 @@ proc addBlock*( if blck.message.slot <= dag.finalizedHead.slot: # let backfill blocks skip the queue - these are always "fast" to process # because there are no state rewinds to deal with - return self[].storeBackfillBlock(blck, sidecarsOpt) + return self.storeBackfillBlock(blck, sidecarsOpt) let queueTick = Moment.now() @@ -807,8 +879,9 @@ proc addBlock*( if sidecarsOpt.isSome: self.dataColumnQuarantine[].put(blockRoot, sidecarsOpt.get) elif sidecarsOpt is Opt[gloas.DataColumnSidecars]: - if sidecarsOpt.isSome: - debugGloasComment "" + # In Gloas, block is enqueued with NoSidecar so this should be + # non-reachable code. + discard elif sidecarsOpt is NoSidecars: discard else: @@ -839,3 +912,104 @@ proc addBlock*( err(res.error()) of VerifierError.Duplicate: err(res.error()) + +proc storePayload( + self: ref BlockProcessor, + signedBlock: gloas.SignedBeaconBlock, + signedEnvelope: gloas.SignedExecutionPayloadEnvelope, + sidecarsOpt: Opt[gloas.DataColumnSidecars], +): Future[Result[void, VerifierError]] {.async: (raises: [CancelledError]).} = + let + dag = self.consensusManager.dag + wallTime = self.getBeaconTime() + wallSlot = wallTime.slotOrZero(dag.timeParams) + deadlineTime = + block: + let slotTime = + (wallSlot + 1).start_beacon_time(dag.timeParams) - chronos.seconds(1) + if slotTime <= wallTime: + chronos.seconds(0) + else: + chronos.nanoseconds((slotTime - wallTime).nanoseconds) + deadline = sleepAsync(deadlineTime) + + let + optimisticStatusRes = + block: + debugGloasComment("handle (maybe)finalized slot") + func shouldRetry(): bool = + not dag.is_optimistic(dag.head.bid) + await self.consensusManager.elManager.getExecutionValidity( + signedBlock, signedEnvelope, deadline, shouldRetry()) + optimisticStatus = + ?(optimisticStatusRes or verifyPayload(self, signedBlock, signedEnvelope)) + + ?verifySidecars(signedBlock, signedEnvelope, sidecarsOpt) + + debugGloasComment("process and store") + debugGloasComment("update optimistic status") + ok() + +proc enqueuePayload*( + self: ref BlockProcessor, + blck: gloas.SignedBeaconBlock, + envelope: gloas.SignedExecutionPayloadEnvelope, + sidecarsOpt: Opt[gloas.DataColumnSidecars], +) = + if blck.message.slot <= self.consensusManager.dag.finalizedHead.slot: + debugGloasComment("backfilling") + + discard self.storePayload(blck, envelope, sidecarsOpt) + +proc enqueuePayload*(self: ref BlockProcessor, blck: gloas.SignedBeaconBlock) = + ## Enqueue payload processing by block that is a valid block. + + let + envelope = self.envelopeQuarantine[].popOrphan(blck).valueOr: + # We have not received the envelope yet so mark it as missing. + self.envelopeQuarantine[].addMissing(blck.root) + return + sidecarsOpt = + block: + let sidecarsOpt = + if envelope.message.blob_kzg_commitments.len() == 0: + Opt.some(default(gloas.DataColumnSidecars)) + else: + debugGloasComment("pop from ColumnQuarantine") + Opt.none(gloas.DataColumnSidecars) + if sidecarsOpt.isNone(): + # As sidecars are missing, put envelope back to quarantine. + self.consensusManager.quarantine[].addSidecarless(blck) + self.envelopeQuarantine[].addOrphan(envelope) + return + sidecarsOpt + + self.enqueuePayload(blck, envelope, sidecarsOpt) + +proc enqueuePayload*(self: ref BlockProcessor, blockRoot: Eth2Digest) = + ## Enqueue payload processing by block root. If the block is not valid, it + ## will be discarded. + + let + blockRef = self.consensusManager.dag.getBlockRef(blockRoot).valueOr: + return + blck = + block: + let forkedBlock = self.consensusManager.dag.getForkedBlock( + blockRef.bid).valueOr: + # We have checked that the block exists in the chain. There might be + # issues in reading the database or data in the memory is broken. + # Since no result is returned, we log for investigation. + debug "Enqueue payload from envelope. Block is missing in DB", + bid = shortLog(blockRef.bid) + return + withBlck(forkedBlock): + when consensusFork >= ConsensusFork.Gloas: + forkyBlck.asSigned() + else: + # Incorrect fork which shouldn't be happening. + debug "Enqueue payload from envelope. Block is in incorrect fork", + bid = shortLog(blockRef.bid) + return + + self.enqueuePayload(blck) diff --git a/beacon_chain/gossip_processing/eth2_processor.nim b/beacon_chain/gossip_processing/eth2_processor.nim index c1864b8c25..3e0179426a 100644 --- a/beacon_chain/gossip_processing/eth2_processor.nim +++ b/beacon_chain/gossip_processing/eth2_processor.nim @@ -199,6 +199,7 @@ proc new*(T: type Eth2Processor, quarantine: ref Quarantine, blobQuarantine: ref BlobQuarantine, dataColumnQuarantine: ref ColumnQuarantine, + envelopeQuarantine: ref EnvelopeQuarantine, rng: ref HmacDrbgContext, getBeaconTime: GetBeaconTimeFn, taskpool: Taskpool @@ -219,6 +220,7 @@ proc new*(T: type Eth2Processor, quarantine: quarantine, blobQuarantine: blobQuarantine, dataColumnQuarantine: dataColumnQuarantine, + envelopeQuarantine: envelopeQuarantine, getCurrentBeaconTime: getBeaconTime, batchCrypto: BatchCrypto.new( rng, dag.cfg.timeParams, @@ -280,10 +282,9 @@ proc processSignedBeaconBlock*( if not (isNil(self.dag.onBlockGossipAdded)): self.dag.onBlockGossipAdded(ForkedSignedBeaconBlock.init(signedBlock)) - when consensusFork == ConsensusFork.Gloas: - debugGloasComment "" - # gloas needs proper data column handling - let sidecarsOpt = Opt.some(default(seq[ref gloas.DataColumnSidecar])) + when consensusFork >= ConsensusFork.Gloas: + # Disable processing sidecars at block time. + const sidecarsOpt = noSidecars elif consensusFork == ConsensusFork.Fulu: let sidecarsOpt = if len(signedBlock.message.body.blob_kzg_commitments) == 0: @@ -338,7 +339,8 @@ proc processExecutionPayloadEnvelope*( execution_payload_envelopes_dropped.inc(1, [$error[0]]) return err(error) - debugGloasComment("process execution payload") + self.envelopeQuarantine[].addOrphan(signedEnvelope) + self.blockProcessor.enqueuePayload(signedEnvelope.root) execution_payload_envelopes_received.inc() execution_payload_envelope_delay.observe(delay.toFloatSeconds()) @@ -476,10 +478,8 @@ proc processDataColumnSidecar*( data_column_sidecars_dropped.inc(1, [$v.error[0]]) return v - debugGloasComment "" - # TODO: Implement quarantine logic for Gloas - # For now, just validate and drop - debug "Data column validated (not stored - quarantine TODO)" + debugGloasComment("put into ColumnQuarantine") + self.blockProcessor.enqueuePayload(dataColumnSidecar.beacon_block_root) data_column_sidecars_received.inc() v diff --git a/beacon_chain/gossip_processing/gossip_validation.nim b/beacon_chain/gossip_processing/gossip_validation.nim index b6450201c4..506876d44e 100644 --- a/beacon_chain/gossip_processing/gossip_validation.nim +++ b/beacon_chain/gossip_processing/gossip_validation.nim @@ -1075,8 +1075,8 @@ proc validateExecutionPayload*( # [REJECT] block passes validation. let blck = block: - let forkedBlock = dag.getForkedBlock(BlockId( - root: envelope.beacon_block_root, slot: envelope.slot)).valueOr: + let forkedBlock = dag.getForkedBlock( + signed_execution_payload_envelope.toBlockId()).valueOr: return dag.checkedReject("ExecutionPayload: invalid block") withBlck(forkedBlock): when consensusFork >= ConsensusFork.Gloas: diff --git a/beacon_chain/networking/eth2_network.nim b/beacon_chain/networking/eth2_network.nim index 531ce04848..00c85f94f1 100644 --- a/beacon_chain/networking/eth2_network.nim +++ b/beacon_chain/networking/eth2_network.nim @@ -849,7 +849,8 @@ template gossipMaxSize(T: untyped): uint32 = elif T is bellatrix.SignedBeaconBlock or T is capella.SignedBeaconBlock or T is deneb.SignedBeaconBlock or T is electra.SignedBeaconBlock or T is fulu.SignedBeaconBlock or T is fulu.DataColumnSidecar or - T is gloas.SignedBeaconBlock or T is gloas.DataColumnSidecar: + T is gloas.SignedBeaconBlock or T is gloas.DataColumnSidecar or + T is gloas.SignedExecutionPayloadEnvelope: MAX_PAYLOAD_SIZE # TODO https://github.com/status-im/nim-ssz-serialization/issues/20 for # Attestation, AttesterSlashing, and SignedAggregateAndProof, which all diff --git a/beacon_chain/nimbus_beacon_node.nim b/beacon_chain/nimbus_beacon_node.nim index f716be8505..755e7a3718 100644 --- a/beacon_chain/nimbus_beacon_node.nim +++ b/beacon_chain/nimbus_beacon_node.nim @@ -17,7 +17,8 @@ import eth/enr/enr, eth/p2p/discoveryv5/random2, ./consensus_object_pools/[ - blob_quarantine, blockchain_list, execution_payload_pool], + blob_quarantine, blockchain_list, envelope_quarantine, + execution_payload_pool], ./consensus_object_pools/vanity_logs/vanity_logs, ./networking/[topic_params, network_metadata_downloads], ./rpc/[rest_api, state_ttl_cache], @@ -402,6 +403,7 @@ proc initFullNode( let quarantine = newClone( Quarantine.init(dag.cfg)) + envelopeQuarantine = newClone(EnvelopeQuarantine.init()) attestationPool = newClone(AttestationPool.init( dag, quarantine, onPhase0AttestationReceived, onSingleAttestationReceived)) @@ -452,7 +454,7 @@ proc initFullNode( blockProcessor = BlockProcessor.new( config.dumpEnabled, config.dumpDirInvalid, config.dumpDirIncoming, batchVerifier, consensusManager, node.validatorMonitor, - blobQuarantine, dataColumnQuarantine, getBeaconTime, + blobQuarantine, dataColumnQuarantine, envelopeQuarantine, getBeaconTime, config.invalidBlockRoots) blockVerifier = proc(signedBlock: ForkedSignedBeaconBlock, blobs: Opt[BlobSidecars], maybeFinalized: bool): @@ -483,8 +485,7 @@ proc initFullNode( maybeFinalized: bool): Future[Result[void, VerifierError]] {.async: (raises: [CancelledError]).} = withBlck(signedBlock): - when consensusFork == ConsensusFork.Gloas: - debugGloasComment "no blob_kzg_commitments field for gloas" + when consensusFork >= ConsensusFork.Gloas: let sidecarsOpt = Opt.none(gloas.DataColumnSidecars) elif consensusFork == ConsensusFork.Fulu: let sidecarsOpt = @@ -541,7 +542,8 @@ proc initFullNode( config.doppelgangerDetection, blockProcessor, node.validatorMonitor, dag, attestationPool, validatorChangePool, node.attachedValidators, syncCommitteeMsgPool, - lightClientPool, executionPayloadBidPool, quarantine, blobQuarantine, dataColumnQuarantine, + lightClientPool, executionPayloadBidPool, + quarantine, blobQuarantine, dataColumnQuarantine, envelopeQuarantine, rng, getBeaconTime, taskpool) syncManagerFlags = if node.config.longRangeSync != LongRangeSyncMode.Lenient: @@ -2378,6 +2380,19 @@ proc installMessageValidators(node: BeaconNode) = node.processor[].processExecutionPayloadBid( MsgSource.gossip, signedBid))) + # execution_payload + # https://github.com/ethereum/consensus-specs/blob/v1.6.1/specs/gloas/p2p-interface.md#execution_payload + when consensusFork >= ConsensusFork.Gloas: + node.network.addValidator( + getExecutionPayloadTopic(digest), proc ( + signedEnvelope: SignedExecutionPayloadEnvelope, + src: PeerId, + ): ValidationResult = + toValidationResult( + node.processor[].processExecutionPayloadEnvelope( + MsgSource.gossip, signedEnvelope)) + ) + # beacon_attestation_{subnet_id} # https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/p2p-interface.md#beacon_attestation_subnet_id # https://github.com/ethereum/consensus-specs/blob/v1.6.0-beta.0/specs/gloas/p2p-interface.md#beacon_attestation_subnet_id diff --git a/beacon_chain/spec/forks.nim b/beacon_chain/spec/forks.nim index a07ed06afe..9a0ce4ab94 100644 --- a/beacon_chain/spec/forks.nim +++ b/beacon_chain/spec/forks.nim @@ -380,6 +380,10 @@ type fuluInt: ForkDigest bpos: seq[(Epoch, ConsensusFork, ForkDigest)] + NoEnvelope* = typeof(()) + +const noEnvelope* = default(NoEnvelope) + template kind*( x: typedesc[ phase0.BeaconState | @@ -1310,6 +1314,9 @@ template root*(x: ForkedSignedBeaconBlock | ForkedTrustedSignedBeaconBlock): Eth2Digest = withBlck(x): forkyBlck.root +template root*(v: gloas.SignedExecutionPayloadEnvelope): Eth2Digest = + v.message.beacon_block_root + template slot*(x: ForkedSignedBeaconBlock | ForkedTrustedSignedBeaconBlock): Slot = withBlck(x): forkyBlck.message.slot @@ -1823,6 +1830,9 @@ func toBlockId*(blck: ForkedSignedBeaconBlock | ForkedTrustedSignedBeaconBlock): BlockId = withBlck(blck): BlockId(root: forkyBlck.root, slot: forkyBlck.message.slot) +func toBlockId*(envelope: gloas.SignedExecutionPayloadEnvelope): BlockId = + BlockId(root: envelope.message.beacon_block_root, slot: envelope.message.slot) + func historical_summaries*(state: ForkedHashedBeaconState): HashList[HistoricalSummary, Limit HISTORICAL_ROOTS_LIMIT] = withState(state): diff --git a/beacon_chain/spec/helpers.nim b/beacon_chain/spec/helpers.nim index 84f5f2e997..094650ed2f 100644 --- a/beacon_chain/spec/helpers.nim +++ b/beacon_chain/spec/helpers.nim @@ -569,6 +569,17 @@ func compute_execution_block_hash*( func compute_execution_block_hash*(blck: ForkyBeaconBlock): Eth2Digest = blck.body.compute_execution_block_hash(blck.parent_root) +func compute_execution_block_hash*( + blck: gloas.BeaconBlock, + envelope: gloas.ExecutionPayloadEnvelope): Eth2Digest = + const consensusFork = typeof(blck).kind + compute_execution_block_hash( + consensusFork, + envelope.payload, + blck.parent_root, + Opt.some envelope.execution_requests.computeRequestsHash(), + ) + # https://github.com/ethereum/consensus-specs/blob/v1.6.0-alpha.6/specs/gloas/beacon-chain.md#new-is_builder_payment_withdrawable func is_builder_payment_withdrawable*( state: gloas.BeaconState, diff --git a/beacon_chain/spec/helpers_el.nim b/beacon_chain/spec/helpers_el.nim index cf85c97c69..03897a1d05 100644 --- a/beacon_chain/spec/helpers_el.nim +++ b/beacon_chain/spec/helpers_el.nim @@ -20,10 +20,23 @@ func readExecutionTransaction( err("Invalid transaction: " & exc.msg) # https://github.com/ethereum/consensus-specs/blob/v1.5.0-alpha.4/specs/deneb/beacon-chain.md#is_valid_versioned_hashes -func is_valid_versioned_hashes*(blck: ForkyBeaconBlock): Result[void, string] = - static: doAssert typeof(blck).kind >= ConsensusFork.Deneb - template transactions: untyped = blck.body.execution_payload.transactions - template commitments: untyped = blck.body.blob_kzg_commitments +func is_valid_versioned_hashes*( + blck: ForkyBeaconBlock, + envelope: NoEnvelope | gloas.ExecutionPayloadEnvelope, +): Result[void, string] = + const consensusFork = typeof(blck).kind + static: doAssert consensusFork >= ConsensusFork.Deneb + + template transactions: untyped = + when consensusFork >= ConsensusFork.Gloas: + envelope.payload.transactions + else: + blck.body.execution_payload.transactions + template commitments: untyped = + when consensusFork >= ConsensusFork.Gloas: + envelope.blob_kzg_commitments + else: + blck.body.blob_kzg_commitments var i = 0 for txBytes in transactions: @@ -39,3 +52,6 @@ func is_valid_versioned_hashes*(blck: ForkyBeaconBlock): Result[void, string] = if i != commitments.len: return err("Extra `blob_kzg_commitments` without matching blobs") ok() + +func is_valid_versioned_hashes*(blck: ForkyBeaconBlock): Result[void, string] = + is_valid_versioned_hashes(blck, noEnvelope) diff --git a/tests/test_block_processor.nim b/tests/test_block_processor.nim index 201f56ed12..dabd75470f 100644 --- a/tests/test_block_processor.nim +++ b/tests/test_block_processor.nim @@ -18,7 +18,7 @@ import ../beacon_chain/gossip_processing/block_processor, ../beacon_chain/consensus_object_pools/[ attestation_pool, blockchain_dag, blob_quarantine, block_quarantine, - block_clearance, consensus_manager, + block_clearance, consensus_manager, envelope_quarantine, ], ../beacon_chain/el/el_manager, ./[testblockutil, testdbutil, testutil] @@ -53,6 +53,7 @@ suite "Block processor" & preset(): quarantine = newClone(Quarantine.init(cfg)) blobQuarantine = newClone(BlobQuarantine()) dataColumnQuarantine = newClone(ColumnQuarantine()) + envelopeQuarantine = newClone(EnvelopeQuarantine()) attestationPool = newClone(AttestationPool.init(dag, quarantine)) elManager = new ELManager # TODO: initialise this properly actionTracker = default(ActionTracker) @@ -83,7 +84,7 @@ suite "Block processor" & preset(): let processor = BlockProcessor.new( false, "", "", batchVerifier, consensusManager, validatorMonitor, - blobQuarantine, dataColumnQuarantine, getTimeFn, + blobQuarantine, dataColumnQuarantine, envelopeQuarantine, getTimeFn, ) b1 = addTestBlock(state[], cache, cfg = cfg).bellatrixData b2 = addTestBlock(state[], cache, cfg = cfg).bellatrixData @@ -145,7 +146,7 @@ suite "Block processor" & preset(): processor = BlockProcessor.new( false, "", "", batchVerifier, consensusManager, validatorMonitor, blobQuarantine, dataColumnQuarantine, - getTimeFn, invalidBlockRoots = @[b2.root]) + envelopeQuarantine, getTimeFn, invalidBlockRoots = @[b2.root]) block: let res = await processor.addBlock(MsgSource.gossip, b2, noSidecars) @@ -175,8 +176,8 @@ suite "Block processor" & preset(): asyncTest "Process a block from each fork (without blobs)" & preset(): let processor = BlockProcessor.new( - false, "", "", batchVerifier, consensusManager, validatorMonitor, blobQuarantine, - dataColumnQuarantine, getTimeFn, + false, "", "", batchVerifier, consensusManager, validatorMonitor, + blobQuarantine, dataColumnQuarantine, envelopeQuarantine, getTimeFn, ) debugGloasComment "TODO testing"