From 26a0794bb4bdf0470dd08ae667260687adcde100 Mon Sep 17 00:00:00 2001 From: dapplion <35266934+dapplion@users.noreply.github.com> Date: Tue, 15 Nov 2022 15:33:38 +0100 Subject: [PATCH] De-couple block production --- .../api/src/beacon/routes/beacon/block.ts | 12 --- packages/api/src/beacon/routes/validator.ts | 12 --- .../src/api/impl/beacon/blocks/index.ts | 22 ----- .../src/api/impl/validator/index.ts | 31 ------- packages/beacon-node/src/chain/chain.ts | 52 ++++++++--- packages/beacon-node/src/chain/interface.ts | 1 - packages/beacon-node/src/chain/options.ts | 1 + .../chain/produceBlock/produceBlockBody.ts | 65 +++++++++++-- .../beacon-node/src/execution/engine/http.ts | 18 +++- .../src/execution/engine/interface.ts | 10 +- packages/validator/src/services/block.ts | 92 +++++-------------- 11 files changed, 138 insertions(+), 178 deletions(-) diff --git a/packages/api/src/beacon/routes/beacon/block.ts b/packages/api/src/beacon/routes/beacon/block.ts index aad21e900dea..4638d72a8636 100644 --- a/packages/api/src/beacon/routes/beacon/block.ts +++ b/packages/api/src/beacon/routes/beacon/block.ts @@ -2,7 +2,6 @@ import {ContainerType} from "@chainsafe/ssz"; import {ForkName} from "@lodestar/params"; import {IChainForkConfig} from "@lodestar/config"; import {phase0, allForks, Slot, Root, ssz, RootHex} from "@lodestar/types"; -import {SignedBeaconBlockAndBlobsSidecar} from "@lodestar/types/eip4844"; import { RoutesData, @@ -111,8 +110,6 @@ export type Api = { * transactions beacon node gets in response. */ publishBlindedBlock(block: allForks.SignedBlindedBeaconBlock): Promise; - - publishBlockWithBlobs(blockWithBlobs: SignedBeaconBlockAndBlobsSidecar): Promise; }; /** @@ -126,7 +123,6 @@ export const routesData: RoutesData = { getBlockHeaders: {url: "/eth/v1/beacon/headers", method: "GET"}, getBlockRoot: {url: "/eth/v1/beacon/blocks/{block_id}/root", method: "GET"}, publishBlock: {url: "/eth/v1/beacon/blocks", method: "POST"}, - publishBlockWithBlobs: {url: "/eth/v1/beacon/blocksWithBlobs", method: "POST"}, publishBlindedBlock: {url: "/eth/v1/beacon/blinded_blocks", method: "POST"}, }; @@ -142,7 +138,6 @@ export type ReqTypes = { getBlockHeaders: {query: {slot?: number; parent_root?: string}}; getBlockRoot: BlockIdOnlyReq; publishBlock: {body: unknown}; - publishBlockWithBlobs: {body: unknown}; publishBlindedBlock: {body: unknown}; }; @@ -162,12 +157,6 @@ export function getReqSerializers(config: IChainForkConfig): ReqSerializers getSignedBeaconBlockType((data as unknown) as allForks.SignedBeaconBlock).fromJson(data), }; - const BlockWithBlobsSsz = ssz.eip4844.SignedBeaconBlockAndBlobsSidecar; - const SignedBeaconBlockWithBlobs: TypeJson = { - toJson: (data) => BlockWithBlobsSsz.toJson(data), - fromJson: (data) => BlockWithBlobsSsz.fromJson(data), - }; - const getSignedBlindedBeaconBlockType = ( data: allForks.SignedBlindedBeaconBlock ): allForks.AllForksBlindedSSZTypes["SignedBeaconBlock"] => @@ -191,7 +180,6 @@ export function getReqSerializers(config: IChainForkConfig): ReqSerializers; - produceBlockWithBlobs( - slot: Slot, - randaoReveal: BLSSignature, - graffiti: string - ): Promise<{data: allForks.BeaconBlock; version: ForkName; blobs: eip4844.Blobs}>; - produceBlindedBlock( slot: Slot, randaoReveal: BLSSignature, @@ -253,7 +245,6 @@ export const routesData: RoutesData = { getSyncCommitteeDuties: {url: "/eth/v1/validator/duties/sync/{epoch}", method: "POST"}, produceBlock: {url: "/eth/v1/validator/blocks/{slot}", method: "GET"}, produceBlockV2: {url: "/eth/v2/validator/blocks/{slot}", method: "GET"}, - produceBlockWithBlobs: {url: "/eth/v3/validator/blocks/{slot}", method: "GET"}, produceBlindedBlock: {url: "/eth/v1/validator/blinded_blocks/{slot}", method: "GET"}, produceAttestationData: {url: "/eth/v1/validator/attestation_data", method: "GET"}, produceSyncCommitteeContribution: {url: "/eth/v1/validator/sync_committee_contribution", method: "GET"}, @@ -274,7 +265,6 @@ export type ReqTypes = { getSyncCommitteeDuties: {params: {epoch: Epoch}; body: U64Str[]}; produceBlock: {params: {slot: number}; query: {randao_reveal: string; graffiti: string}}; produceBlockV2: {params: {slot: number}; query: {randao_reveal: string; graffiti: string}}; - produceBlockWithBlobs: {params: {slot: number}; query: {randao_reveal: string; graffiti: string}}; produceBlindedBlock: {params: {slot: number}; query: {randao_reveal: string; graffiti: string}}; produceAttestationData: {query: {slot: number; committee_index: number}}; produceSyncCommitteeContribution: {query: {slot: number; subcommittee_index: number; beacon_block_root: string}}; @@ -350,7 +340,6 @@ export function getReqSerializers(): ReqSerializers { produceBlock: produceBlock, produceBlockV2: produceBlock, - produceBlockWithBlobs: produceBlock, produceBlindedBlock: produceBlock, produceAttestationData: { @@ -454,7 +443,6 @@ export function getReturnTypes(): ReturnTypes { getSyncCommitteeDuties: ContainerDataExecutionOptimistic(ArrayOf(SyncDuty)), produceBlock: ContainerData(ssz.phase0.BeaconBlock), produceBlockV2: WithVersion((fork: ForkName) => ssz[fork].BeaconBlock), - produceBlockWithBlobs: WithBlobs(WithVersion((fork: ForkName) => ssz[fork].BeaconBlock)), produceBlindedBlock: WithVersion((fork: ForkName) => { if (fork === ForkName.phase0 || fork === ForkName.altair) { throw Error(`No BlindedBlock for fork ${fork} previous to bellatrix`); diff --git a/packages/beacon-node/src/api/impl/beacon/blocks/index.ts b/packages/beacon-node/src/api/impl/beacon/blocks/index.ts index 72dee95d82c1..a1733d5be0cd 100644 --- a/packages/beacon-node/src/api/impl/beacon/blocks/index.ts +++ b/packages/beacon-node/src/api/impl/beacon/blocks/index.ts @@ -202,27 +202,5 @@ export function getBeaconBlockApi({ }), ]); }, - - async publishBlockWithBlobs(signedBeaconBlockAndBlobsSidecar) { - const {beaconBlock} = signedBeaconBlockAndBlobsSidecar; - const {message} = beaconBlock; - - const seenTimestampSec = Date.now() / 1000; - await waitForSlot(message.slot); - - metrics?.registerBeaconBlock(OpSource.api, seenTimestampSec, message); - - await Promise.all([ - network.gossip.publishSignedBeaconBlockAndBlobsSidecar(signedBeaconBlockAndBlobsSidecar), - // TODO EIP-4844 processBlock for signedBeaconBlockAndBlobsSidecar - // We need to save the blob? - chain.processBlock(beaconBlock).catch((e) => { - if (e instanceof BlockError && e.type.code === BlockErrorCode.PARENT_UNKNOWN) { - network.events.emit(NetworkEvent.unknownBlockParent, beaconBlock, network.peerId.toString()); - } - throw e; - }), - ]); - }, }; } diff --git a/packages/beacon-node/src/api/impl/validator/index.ts b/packages/beacon-node/src/api/impl/validator/index.ts index e301c432a78e..22b38d4d6045 100644 --- a/packages/beacon-node/src/api/impl/validator/index.ts +++ b/packages/beacon-node/src/api/impl/validator/index.ts @@ -237,40 +237,9 @@ export function getValidatorApi({chain, config, logger, metrics, network, sync}: } }; - const produceBlockWithBlobs: routes.validator.Api["produceBlockWithBlobs"] = async function produceBlock( - slot, - randaoReveal, - graffiti - ) { - let timer; - metrics?.blockProductionRequests.inc(); - try { - notWhileSyncing(); - await waitForSlot(slot); // Must never request for a future slot > currentSlot - - // Process the queued attestations in the forkchoice for correct head estimation - // forkChoice.updateTime() might have already been called by the onSlot clock - // handler, in which case this should just return. - chain.forkChoice.updateTime(slot); - chain.recomputeForkChoiceHead(); - - timer = metrics?.blockProductionTime.startTimer(); - const {block, blobs} = await chain.produceBlockWithBlobs({ - slot, - randaoReveal, - graffiti: toGraffitiBuffer(graffiti || ""), - }); - metrics?.blockProductionSuccess.inc(); - return {data: block, version: config.getForkName(block.slot), blobs}; - } finally { - if (timer) timer(); - } - }; - return { produceBlock: produceBlock, produceBlockV2: produceBlock, - produceBlockWithBlobs, produceBlindedBlock, async produceAttestationData(committeeIndex, slot) { diff --git a/packages/beacon-node/src/chain/chain.ts b/packages/beacon-node/src/chain/chain.ts index a4fc34e52d66..4cfda7dff008 100644 --- a/packages/beacon-node/src/chain/chain.ts +++ b/packages/beacon-node/src/chain/chain.ts @@ -1,4 +1,5 @@ import path from "node:path"; +import {computeAggregateKzgProof} from "c-kzg"; import { BeaconStateAllForks, CachedBeaconStateAllForks, @@ -60,7 +61,7 @@ import {SeenBlockAttesters} from "./seenCache/seenBlockAttesters.js"; import {BeaconProposerCache} from "./beaconProposerCache.js"; import {CheckpointBalancesCache} from "./balancesCache.js"; import {AssembledBlockType, BlockType} from "./produceBlock/index.js"; -import {BlockAttributes, produceBlockBody} from "./produceBlock/produceBlockBody.js"; +import {BlobsResultType, BlockAttributes, produceBlockBody} from "./produceBlock/produceBlockBody.js"; import {computeNewStateRoot} from "./produceBlock/computeNewStateRoot.js"; export class BeaconChain implements IBeaconChain { @@ -118,6 +119,8 @@ export class BeaconChain implements IBeaconChain { private successfulExchangeTransition = false; private readonly exchangeTransitionConfigurationEverySlots: number; + private readonly producedBlobsCache = new Map(); + private readonly faultInspectionWindow: number; private readonly allowedFaults: number; private processShutdownCallback: ProcessShutdownCallback; @@ -340,26 +343,18 @@ export class BeaconChain implements IBeaconChain { return await this.db.block.get(fromHexString(block.blockRoot)); } - async produceBlock(blockAttributes: BlockAttributes): Promise { - const {block} = await this.produceBlockWrapper(BlockType.Full, blockAttributes); - return block; - } - - async produceBlockWithBlobs( - blockAttributes: BlockAttributes - ): Promise<{block: allForks.BeaconBlock; blobs: eip4844.Blobs}> { + produceBlock(blockAttributes: BlockAttributes): Promise { return this.produceBlockWrapper(BlockType.Full, blockAttributes); } - async produceBlindedBlock(blockAttributes: BlockAttributes): Promise { - const {block} = await this.produceBlockWrapper(BlockType.Blinded, blockAttributes); - return block; + produceBlindedBlock(blockAttributes: BlockAttributes): Promise { + return this.produceBlockWrapper(BlockType.Blinded, blockAttributes); } async produceBlockWrapper( blockType: T, {randaoReveal, graffiti, slot}: BlockAttributes - ): Promise<{block: AssembledBlockType; blobs: eip4844.Blobs}> { + ): Promise> { const head = this.forkChoice.getHead(); const state = await this.regen.getBlockSlotState(head.blockRoot, slot, RegenCaller.produceBlock); const parentBlockRoot = fromHexString(head.blockRoot); @@ -386,7 +381,36 @@ export class BeaconChain implements IBeaconChain { block.stateRoot = computeNewStateRoot(this.metrics, state, block); - return {block, blobs: blobs ?? []}; + // Cache for latter broadcasting + if (blobs.type === BlobsResultType.produced) { + this.producedBlobsCache.set(blobs.blockRoot, blobs.blobs); + } + + return block; + } + + /** + * https://github.com/ethereum/consensus-specs/blob/dev/specs/eip4844/validator.md#sidecar + * def get_blobs_sidecar(block: BeaconBlock, blobs: Sequence[Blob]) -> BlobsSidecar: + * return BlobsSidecar( + * beacon_block_root=hash_tree_root(block), + * beacon_block_slot=block.slot, + * blobs=blobs, + * kzg_aggregated_proof=compute_proof_from_blobs(blobs), + * ) + */ + getBlobsSidecar(beaconBlockRoot: RootHex): eip4844.BlobsSidecar { + const blobs = this.producedBlobsCache.get(beaconBlockRoot); + if (!blobs) { + throw Error(`No blobs for beaconBlockRoot ${beaconBlockRoot}`); + } + + return { + beaconBlockRoot: beaconBlockRoot, + beaconBlockSlot: blobs.slot, + blobs: blobs, + kzgAggregatedProof: computeAggregateKzgProof(blobs), + }; } async processBlock(block: allForks.SignedBeaconBlock, opts?: ImportBlockOpts): Promise { diff --git a/packages/beacon-node/src/chain/interface.ts b/packages/beacon-node/src/chain/interface.ts index e2580ec07fb1..021626b5b58f 100644 --- a/packages/beacon-node/src/chain/interface.ts +++ b/packages/beacon-node/src/chain/interface.ts @@ -109,7 +109,6 @@ export interface IBeaconChain { getCanonicalBlockAtSlot(slot: Slot): Promise; produceBlock(blockAttributes: BlockAttributes): Promise; - produceBlockWithBlobs(blockAttributes: BlockAttributes): Promise<{block: allForks.BeaconBlock; blobs: eip4844.Blobs}>; produceBlindedBlock(blockAttributes: BlockAttributes): Promise; /** Process a block until complete */ diff --git a/packages/beacon-node/src/chain/options.ts b/packages/beacon-node/src/chain/options.ts index e827a3a0b286..995099b2df4b 100644 --- a/packages/beacon-node/src/chain/options.ts +++ b/packages/beacon-node/src/chain/options.ts @@ -20,6 +20,7 @@ export type IChainOptions = BlockProcessOpts & faultInspectionWindow?: number; /** Number of missed slots allowed in the faultInspectionWindow for builder circuit*/ allowedFaults?: number; + sanityCheckExecutionEngineBlocks?: boolean; }; export type BlockProcessOpts = { diff --git a/packages/beacon-node/src/chain/produceBlock/produceBlockBody.ts b/packages/beacon-node/src/chain/produceBlock/produceBlockBody.ts index 56ebb9d82070..86ccdebde6aa 100644 --- a/packages/beacon-node/src/chain/produceBlock/produceBlockBody.ts +++ b/packages/beacon-node/src/chain/produceBlock/produceBlockBody.ts @@ -3,7 +3,6 @@ import { phase0, allForks, altair, - eip4844, Root, RootHex, Slot, @@ -12,6 +11,7 @@ import { BLSPubkey, BLSSignature, capella, + eip4844, } from "@lodestar/types"; import { CachedBeaconStateAllForks, @@ -24,7 +24,7 @@ import { isMergeTransitionComplete, } from "@lodestar/state-transition"; import {IChainForkConfig} from "@lodestar/config"; -import {ForkName} from "@lodestar/params"; +import {ForkName, ForkSeq} from "@lodestar/params"; import {toHex, sleep} from "@lodestar/utils"; import type {BeaconChain} from "../chain.js"; @@ -32,6 +32,7 @@ import {PayloadId, IExecutionEngine, IExecutionBuilder} from "../../execution/in import {ZERO_HASH, ZERO_HASH_HEX} from "../../constants/index.js"; import {IEth1ForBlockProduction} from "../../eth1/index.js"; import {numToQuantity} from "../../eth1/provider/utils.js"; +import {byteArrayEquals} from "../../util/bytes.js"; // Time to provide the EL to generate a payload from new payload id const PAYLOAD_GENERATION_TIME_MS = 500; @@ -58,6 +59,13 @@ export type AssembledBlockType = T extends BlockType.Full ? allForks.BeaconBlock : allForks.BlindedBeaconBlock; +export enum BlobsResultType { + preEIP4844, + produced, +} + +export type BlobsResult = {type: BlobsResultType.preEIP4844} | {type: BlobsResultType.produced; blobs: eip4844.Blobs}; + export async function produceBlockBody( this: BeaconChain, blockType: T, @@ -76,7 +84,7 @@ export async function produceBlockBody( proposerIndex: ValidatorIndex; proposerPubKey: BLSPubkey; } -): Promise<{body: AssembledBodyType; blobs: eip4844.Blobs | null}> { +): Promise<{body: AssembledBodyType; blobs: BlobsResult}> { // TODO: // Iterate through the naive aggregation pool and ensure all the attestations from there // are included in the operation pool. @@ -162,7 +170,16 @@ export async function produceBlockBody( Buffer.alloc(32, 0) ); } - } else { + + if (forkName === ForkName.eip4844) { + // Empty blobs for now + (blockBody as eip4844.BeaconBlockBody).blobKzgCommitments = []; + blobs = []; + } + } + + // blockType === BlockType.Full + else { // try catch payload fetch here, because there is still a recovery path possible if we // are pre-merge. We don't care the same for builder segment as the execution block // will takeover if the builder flow was activated and errors @@ -202,10 +219,31 @@ export async function produceBlockBody( } if (forkName === ForkName.eip4844) { + // SPEC: https://github.com/ethereum/consensus-specs/blob/dev/specs/eip4844/validator.md#blob-kzg-commitments + // After retrieving the execution payload from the execution engine as specified in Bellatrix, use the + // payload_id to retrieve blobs and blob_kzg_commitments via get_blobs_and_kzg_commitments(payload_id) const blobsBundle = await this.executionEngine.getBlobsBundle(payloadId); - // TODO EIP-4844: Optional (do it in a follow-up): sanity-check that the KZG commitments match blob contents as described by the spec - (blockBody as eip4844.BeaconBlockBody).blobKzgCommitments = blobsBundle.kzgs ?? []; + if (this.opts.sanityCheckExecutionEngineBlocks) { + // Optionally sanity-check that the KZG commitments match the versioned hashes in the transactions + verify_kzg_commitments_against_transactions(payload.transactions, blobsBundle.kzgs); + + // Optionally sanity-check that the KZG commitments match the blobs (as produced by the execution engine) + if (blobsBundle.blobs.length !== blobsBundle.kzgs.length) { + throw Error( + `Blobs bundle blobs len ${blobsBundle.blobs.length} != kzgs len ${blobsBundle.kzgs.length}` + ); + } + + for (let i = 0; i < blobsBundle.blobs.length; i++) { + const kzg = blob_to_kzg_commitment(blobsBundle.blobs[i]) as eip4844.KZGCommitment; + if (!byteArrayEquals(kzg, blobsBundle.kzgs[i])) { + throw Error(`Wrong KZG[${i}] ${toHex(blobsBundle.kzgs[i])} expected ${toHex(kzg)}`); + } + } + } + + (blockBody as eip4844.BeaconBlockBody).blobKzgCommitments = blobsBundle.kzgs; blobs = blobsBundle.blobs; } @@ -238,7 +276,20 @@ export async function produceBlockBody( } } - return {body: blockBody as AssembledBodyType, blobs}; + // Type-safe for blobs variable. Translate 'null' value into 'preEIP4844' enum + // TODO: Not ideal, but better than just using null. + // TODO: Does not guarantee that preEIP4844 enum goes with a preEIP4844 block + let blobsResult: BlobsResult; + if (currentState.config.getForkSeq(blockSlot) >= ForkSeq.eip4844) { + if (!blobs) { + throw Error("Blobs are null post eip4844"); + } + blobsResult = {type: BlobsResultType.produced, blobs}; + } else { + blobsResult = {type: BlobsResultType.preEIP4844}; + } + + return {body: blockBody as AssembledBodyType, blobs: blobsResult}; } /** diff --git a/packages/beacon-node/src/execution/engine/http.ts b/packages/beacon-node/src/execution/engine/http.ts index 8f19b9fc4e36..cc1fa752ad86 100644 --- a/packages/beacon-node/src/execution/engine/http.ts +++ b/packages/beacon-node/src/execution/engine/http.ts @@ -322,7 +322,7 @@ export class ExecutionEngineHttp implements IExecutionEngine { getPayloadOpts ); - return blobsBundle; + return parseBlobsBundle(blobsBundle); } /** @@ -405,7 +405,7 @@ type EngineApiRpcReturnTypes = { */ engine_exchangeTransitionConfigurationV1: TransitionConfigurationV1; - engine_getBlobsBundleV1: BlobsBundle; + engine_getBlobsBundleV1: BlobsBundleRpc; }; type ExecutionPayloadRpc = { @@ -427,6 +427,12 @@ type ExecutionPayloadRpc = { excessDataGas?: QUANTITY; // EIP-4844 }; +interface BlobsBundleRpc { + blockHash: DATA; // 32 Bytes + kzgs: DATA[]; // each 48 bytes + blobs: DATA[]; // each 4096 * 32 = 131072 bytes +} + export function serializeExecutionPayload(data: allForks.ExecutionPayload): ExecutionPayloadRpc { const payload: ExecutionPayloadRpc = { parentHash: bytesToData(data.parentHash), @@ -491,6 +497,14 @@ export function parseExecutionPayload(data: ExecutionPayloadRpc): allForks.Execu return payload; } +export function parseBlobsBundle(data: BlobsBundleRpc): BlobsBundle { + return { + blockHash: dataToBytes(data.blockHash), + kzgs: data.kzgs.map((kzg) => dataToBytes(kzg)), + blobs: data.blobs.map((blob) => dataToBytes(blob)), + }; +} + type EngineRequestKey = keyof EngineApiRpcParamTypes; type EngineRequestByKey = { [K in EngineRequestKey]: {method: K; params: EngineApiRpcParamTypes[K]; methodOpts: ReqOpts}; diff --git a/packages/beacon-node/src/execution/engine/interface.ts b/packages/beacon-node/src/execution/engine/interface.ts index aa16f5db83b2..67e2091149a2 100644 --- a/packages/beacon-node/src/execution/engine/interface.ts +++ b/packages/beacon-node/src/execution/engine/interface.ts @@ -56,10 +56,9 @@ export type TransitionConfigurationV1 = { }; export type BlobsBundle = { - blockHash: DATA; - kzgs: KZGCommitment[] | null; - blobs: Blob[] | null; - aggregatedProof: DATA; + blockHash: Uint8Array; + kzgs: KZGCommitment[]; + blobs: Blob[]; }; /** @@ -119,9 +118,6 @@ export interface IExecutionEngine { * describes as `get_blobs_and_kzg_commitments(payload_id)`. * * The Engine API spec is in PR: https://github.com/ethereum/execution-apis/pull/197 - * - * @param payloadId - * @returns BlobsBundle */ getBlobsBundle(payloadId: PayloadId): Promise; diff --git a/packages/validator/src/services/block.ts b/packages/validator/src/services/block.ts index 43881cba4ed8..a68bd932433b 100644 --- a/packages/validator/src/services/block.ts +++ b/packages/validator/src/services/block.ts @@ -1,12 +1,9 @@ -import {computeAggregateKzgProof} from "c-kzg"; -import {BLSPubkey, Slot, BLSSignature, allForks, bellatrix, isBlindedBeaconBlock, eip4844, ssz} from "@lodestar/types"; +import {BLSPubkey, Slot, BLSSignature, allForks, bellatrix, isBlindedBeaconBlock} from "@lodestar/types"; import {IChainForkConfig} from "@lodestar/config"; -import {ForkName, ForkSeq} from "@lodestar/params"; +import {ForkName} from "@lodestar/params"; import {extendError, prettyBytes} from "@lodestar/utils"; import {toHexString} from "@chainsafe/ssz"; import {Api} from "@lodestar/api"; -import {Blobs, BlobsSidecar} from "@lodestar/types/eip4844"; -import {blindedOrFullBlockHashTreeRoot} from "@lodestar/state-transition"; import {IClock, ILoggerVc} from "../util/index.js"; import {PubkeyHex} from "../types.js"; import {Metrics} from "../metrics.js"; @@ -79,7 +76,7 @@ export class BlockProposingService { const isBuilderEnabled = this.validatorStore.isBuilderEnabled(pubkeyHex); const expectedFeeRecipient = this.validatorStore.getFeeRecipient(pubkeyHex); - const {block, blobs, blockDebugLogCtx} = await this.produceBlockWrapper(slot, randaoReveal, graffiti, { + const block = await this.produceBlockWrapper(slot, randaoReveal, graffiti, { expectedFeeRecipient, strictFeeRecipientCheck, isBuilderEnabled, @@ -88,61 +85,24 @@ export class BlockProposingService { throw extendError(e, "Failed to produce block"); }); - this.logger.debug("Produced block", {...debugLogCtx, ...blockDebugLogCtx}); + this.logger.debug("Produced block", {...debugLogCtx, ...block.debugLogCtx}); this.metrics?.blocksProduced.inc(); - const signedBlock = await this.validatorStore.signBlock(pubkey, block, slot); + const signedBlock = await this.validatorStore.signBlock(pubkey, block.data, slot); this.metrics?.proposerStepCallPublishBlock.observe(this.clock.secFromSlot(slot)); - const onPublishError = (e: Error): void => { + await this.publishBlockWrapper(signedBlock).catch((e: Error) => { this.metrics?.blockProposingErrors.inc({error: "publish"}); throw extendError(e, "Failed to publish block"); - }; - - if (this.config.getForkSeq(block.slot) >= ForkSeq.eip4844) { - if (!blobs) { - return onPublishError(new Error("Produced an EIP-4844 block but it was missing blobs!")); - } - - const signedBlockWithBlobs = ssz.eip4844.SignedBeaconBlockAndBlobsSidecar.defaultValue(); - signedBlockWithBlobs.beaconBlock = signedBlock as eip4844.SignedBeaconBlock; - signedBlockWithBlobs.blobsSidecar = this.getBlobsSidecar(block, blobs); - - // TODO EIP-4844: Blinded blocks??? No clue! - await this.api.beacon.publishBlockWithBlobs(signedBlockWithBlobs).catch(onPublishError); - } else { - await this.publishBlockWrapper(signedBlock).catch(onPublishError); - } - - this.logger.info("Published block", {...logCtx, graffiti, ...blockDebugLogCtx}); + }); + this.logger.info("Published block", {...logCtx, graffiti, ...block.debugLogCtx}); this.metrics?.blocksPublished.inc(); } catch (e) { this.logger.error("Error proposing block", logCtx, e as Error); } } - /** - * https://github.com/ethereum/consensus-specs/blob/dev/specs/eip4844/validator.md#sidecar - * def get_blobs_sidecar(block: BeaconBlock, blobs: Sequence[Blob]) -> BlobsSidecar: - * return BlobsSidecar( - * beacon_block_root=hash_tree_root(block), - * beacon_block_slot=block.slot, - * blobs=blobs, - * kzg_aggregated_proof=compute_proof_from_blobs(blobs), - * ) - */ - private getBlobsSidecar(block: allForks.FullOrBlindedBeaconBlock, blobs: Blobs): BlobsSidecar { - const blobsSidecar = ssz.eip4844.BlobsSidecar.defaultValue(); - - blobsSidecar.beaconBlockRoot = blindedOrFullBlockHashTreeRoot(this.config, block); - blobsSidecar.beaconBlockSlot = block.slot; - blobsSidecar.blobs = blobs; - blobsSidecar.kzgAggregatedProof = computeAggregateKzgProof(blobs); - - return blobsSidecar; - } - private publishBlockWrapper = async (signedBlock: allForks.FullOrBlindedSignedBeaconBlock): Promise => { return isBlindedBeaconBlock(signedBlock.message) ? this.api.beacon.publishBlindedBlock(signedBlock as bellatrix.SignedBlindedBeaconBlock) @@ -158,12 +118,7 @@ export class BlockProposingService { strictFeeRecipientCheck, isBuilderEnabled, }: {expectedFeeRecipient: string; strictFeeRecipientCheck: boolean; isBuilderEnabled: boolean} - ): Promise<{ - block: allForks.FullOrBlindedBeaconBlock; - blockDebugLogCtx: Record; - blobs: eip4844.Blobs | undefined; - }> => { - // TODO EIP-4844: How does 4844 interact with the Builder API? + ): Promise<{data: allForks.FullOrBlindedBeaconBlock} & {debugLogCtx: Record}> => { const blindedBlockPromise = isBuilderEnabled ? this.api.validator.produceBlindedBlock(slot, randaoReveal, graffiti).catch((e: Error) => { this.logger.error("Failed to produce builder block", {}, e as Error); @@ -183,11 +138,10 @@ export class BlockProposingService { // A metric on the choice between blindedBlock and normal block can be applied if (blindedBlock) { - const blockDebugLogCtx = {source: "builder"}; - // TODO EIP-4844: What are we doing with blobs for blinded blocks? - return {block: blindedBlock.data, blockDebugLogCtx, blobs: []}; + const debugLogCtx = {source: "builder"}; + return {...blindedBlock, debugLogCtx}; } else { - const blockDebugLogCtx = {source: "engine"}; + const debugLogCtx = {source: "engine"}; if (!fullBlock) { throw Error("Failed to produce engine or builder block"); } @@ -209,28 +163,26 @@ export class BlockProposingService { if (feeRecipient !== expectedFeeRecipient && strictFeeRecipientCheck) { throw Error(`Invalid feeRecipient=${feeRecipient}, expected=${expectedFeeRecipient}`); } - Object.assign(blockDebugLogCtx, {feeRecipient}); + Object.assign(debugLogCtx, {feeRecipient}); } - return {block: fullBlock.data, blockDebugLogCtx, blobs: fullBlock.blobs}; + return {...fullBlock, debugLogCtx}; + // throw Error("random") } }; /** Wrapper around the API's different methods for producing blocks across forks */ - private produceBlock = async ( - slot: Slot, - randaoReveal: BLSSignature, - graffiti: string - ): Promise<{data: allForks.BeaconBlock; blobs?: Blobs}> => { + private produceBlock: Api["validator"]["produceBlock"] = async ( + slot, + randaoReveal, + graffiti + ): Promise<{data: allForks.BeaconBlock}> => { switch (this.config.getForkName(slot)) { case ForkName.phase0: return this.api.validator.produceBlock(slot, randaoReveal, graffiti); + // All subsequent forks are expected to use v2 too case ForkName.altair: - case ForkName.bellatrix: - case ForkName.capella: - return this.api.validator.produceBlockV2(slot, randaoReveal, graffiti); default: - // EIP-4844 and later - return this.api.validator.produceBlockWithBlobs(slot, randaoReveal, graffiti); + return this.api.validator.produceBlockV2(slot, randaoReveal, graffiti); } }; }