Skip to content

Conversation

@spalladino
Copy link
Contributor

What happens if you give Claude an overview of what to do and a few Linear issues?

@spalladino
Copy link
Contributor Author

Implementation Plan: Multiple Blocks Per Slot (MBPS) P2P and Validator Updates

Overview

This plan updates the p2p, validator-client, and sequencer-publisher to handle multiple blocks per slot. The key change is that validators now attest to checkpoints (aggregations of multiple blocks within a slot) rather than individual blocks.

Key Design Decisions:

  • Parallel execution: Validators execute blocks as they arrive (while proposer builds next block)
  • Separate message types: Distinct BlockProposal and CheckpointProposal types
  • No backwards compatibility: Breaking changes to message types are acceptable
  • Slot-level timeout: Wait until slot end for missing blocks (no per-block timeouts)

New MBPS Flow

Proposer Side (from README timing model)

T=2s:   Build Block 1, broadcast BlockProposal
T=10s:  Build Block 2, broadcast BlockProposal
T=18s:  Build Block 3, broadcast BlockProposal
T=26s:  Build Block 4, broadcast BlockProposal
T=34s:  Build Block 5 (last), assemble checkpoint
T=42s:  Broadcast CheckpointProposal (includes Block 5 + checkpoint)
T=42-54s: Collect CheckpointAttestations
T=55s:  Publish to L1

Validator Side (parallel execution)

T=12s:  Receive Block 1 (10s + 2s propagation), start executing
T=20s:  Finish Block 1, receive Block 2, start executing
T=28s:  Finish Block 2, receive Block 3, start executing
T=36s:  Finish Block 3, receive Block 4, start executing
T=44s:  Finish Block 4, receive CheckpointProposal (Block 5 + checkpoint)
T=44-52s: Execute Block 5, complete checkpoint
T=52s:  Compare local checkpoint with proposal, sign and broadcast attestation

Key insight: Validators execute block N while proposer builds block N+1. This parallelism is why we need to broadcast each block immediately after building it.


Phase 1: New P2P Message Types

Location: stdlib/src/p2p/

1.1 Create BlockProposal (updated for non-last blocks)

File: stdlib/src/p2p/block_proposal.ts (update existing)

Update the existing BlockProposal to use per-block data instead of checkpoint-level data:

export class BlockProposal extends Gossipable {
  static override p2pTopic = TopicType.block_proposal;

  constructor(
    /** Per-block header */
    public readonly blockHeader: BlockHeader,
    /** Index of this block within the checkpoint (0-indexed) */
    public readonly indexWithinCheckpoint: number,
    /** inHash from the checkpoint's L1 to L2 messages (shared across checkpoint) */
    public readonly inHash: Fr,
    /** Archive root after this block */
    public readonly archiveRoot: Fr,
    /** Transaction hashes in this block */
    public readonly txHashes: TxHash[],
    /** Proposer's signature over the block data */
    public readonly signature: Signature,
    /** Optional: full transactions for DA */
    public readonly txs?: Tx[],
  ) {}

  get slotNumber(): SlotNumber {
    return this.blockHeader.getSlot();
  }

  get blockNumber(): BlockNumber {
    return this.blockHeader.globalVariables.blockNumber;
  }
}

1.2 Create CheckpointProposal (for last block + checkpoint)

File: stdlib/src/p2p/checkpoint_proposal.ts (new file)

export class CheckpointProposal extends Gossipable {
  static override p2pTopic = TopicType.checkpoint_proposal;

  constructor(
    /** Last block's header */
    public readonly lastBlockHeader: BlockHeader,
    /** Index of last block within the checkpoint */
    public readonly indexWithinCheckpoint: number,
    /** Archive root after the last block (equals checkpoint archive) */
    public readonly archiveRoot: Fr,
    /** Transaction hashes in the last block */
    public readonly txHashes: TxHash[],
    /** Complete checkpoint header for L1 submission */
    public readonly checkpointHeader: CheckpointHeader,
    /** Proposer's signature over the checkpoint (header + archive) */
    public readonly checkpointSignature: Signature,
    /** Optional: full transactions for DA */
    public readonly txs?: Tx[],
  ) {}

  get slotNumber(): SlotNumber {
    return this.checkpointHeader.slotNumber;
  }

  /** Derives inHash from the checkpointHeader */
  get inHash(): Fr {
    return this.checkpointHeader.contentCommitment.inHash;
  }
}

1.3 Create CheckpointAttestation

File: stdlib/src/p2p/checkpoint_attestation.ts (new file)

Replaces BlockAttestation for attesting to checkpoints:

export class CheckpointAttestation extends Gossipable {
  static override p2pTopic = TopicType.checkpoint_attestation;

  constructor(
    /** The checkpoint header being attested to */
    public readonly header: CheckpointHeader,
    /** Archive root of the checkpoint */
    public readonly archive: Fr,
    /** Validator's attestation signature over (header, archive) */
    public readonly signature: Signature,
    /** Proposer's signature (from the CheckpointProposal) for L1 verification */
    public readonly proposerSignature: Signature,
  ) {}

  get slotNumber(): SlotNumber {
    return this.header.slotNumber;
  }
}

1.4 Update TopicType enum

File: stdlib/src/p2p/topic_type.ts

export enum TopicType {
  tx = 'tx',
  block_proposal = 'block_proposal',           // Updated for non-last blocks
  checkpoint_proposal = 'checkpoint_proposal', // New topic
  checkpoint_attestation = 'checkpoint_attestation', // New - replaces block_attestation
  block_attestation = 'block_attestation',    // Deprecated - keep for backward compat
}

1.5 Update Signature Utilities

File: stdlib/src/p2p/signature_utils.ts

Add new domain separators:

export enum SignatureDomainSeparator {
  blockProposal = 0,
  blockAttestation = 1,         // Deprecated
  attestationsAndSigners = 2,
  checkpointProposal = 3,       // New - for checkpoint signature
  checkpointAttestation = 4,    // New - for attestation signature
}

1.6 Update exports

File: stdlib/src/p2p/index.ts

Export new types and deprecate BlockAttestation.


Phase 2: P2P Layer Updates

2.1 Update P2P Client Interface

File: p2p/src/client/interface.ts

Add new methods:

export interface P2P<T extends P2PClientType = P2PClientType.Full> {
  // Existing methods...

  /** Broadcasts a block proposal (non-last blocks) */
  broadcastBlockProposal(proposal: BlockProposal): Promise<void>;

  /** Broadcasts a checkpoint proposal (last block + checkpoint) */
  broadcastCheckpointProposal(proposal: CheckpointProposal): Promise<void>;

  /** Broadcasts checkpoint attestations */
  broadcastCheckpointAttestations(attestations: CheckpointAttestation[]): Promise<void>;

  /** Register handler for block proposals within a slot */
  registerBlockProposalHandler(callback: P2PBlockProposalCallback): void;

  /** Register handler for checkpoint proposals */
  registerCheckpointProposalHandler(callback: P2PCheckpointProposalCallback): void;

  /** Get checkpoint attestations for a slot and proposal */
  getCheckpointAttestationsForSlot(slot: SlotNumber, proposalId: string): Promise<CheckpointAttestation[]>;

  /** Add checkpoint attestations to the pool */
  addCheckpointAttestations(attestations: CheckpointAttestation[]): Promise<void>;
}

export type P2PBlockProposalCallback = (
  proposal: BlockProposal,
  sender: PeerId,
) => Promise<void>;  // No return - validators don't attest to individual blocks

export type P2PCheckpointProposalCallback = (
  proposal: CheckpointProposal,
  sender: PeerId,
) => Promise<CheckpointAttestation[] | undefined>;

2.2 Update P2P Client Implementation

File: p2p/src/client/p2p_client.ts

  1. Add checkpointProposalCallback field
  2. Implement broadcastBlockProposal(), broadcastCheckpointProposal()
  3. Implement registerCheckpointProposalHandler()
  4. Implement getCheckpointAttestationsForSlot(), addCheckpointAttestations()
  5. Update default handler to not attest to non-last blocks

2.3 Update LibP2P Service

File: p2p/src/services/libp2p/libp2p_service.ts

  1. Add subscriptions for new topics in subscribeToTopics():

    • TopicType.checkpoint_proposal
    • TopicType.checkpoint_attestation
  2. Add callback fields:

    • blockProposalCallback: P2PBlockProposalCallback
    • checkpointProposalCallback: P2PCheckpointProposalCallback
  3. Update handleNewGossipMessage() to route messages:

    case TopicType.block_proposal:
      return this.processBlockProposalFromPeer(message, peerId);
    case TopicType.checkpoint_proposal:
      return this.processCheckpointProposalFromPeer(message, peerId);
    case TopicType.checkpoint_attestation:
      return this.processCheckpointAttestationFromPeer(message, peerId);
  4. Implement handlers:

    • processBlockProposalFromPeer() - Validate and call callback, no attestation returned
    • processCheckpointProposalFromPeer() - Validate, call callback, broadcast attestations
    • processCheckpointAttestationFromPeer() - Validate and add to pool

2.4 Create New Message Validators

File: p2p/src/msg_validators/block_proposal_validator.ts (update existing)

Update to validate BlockProposal with new structure (BlockHeader instead of CheckpointHeader).

File: p2p/src/msg_validators/checkpoint_proposal_validator.ts (new file)

export class CheckpointProposalValidator {
  async validate(proposal: CheckpointProposal): Promise<string | undefined> {
    // Validate signature recovery
    const proposer = proposal.getSender();
    if (!proposer) return 'Invalid signature';

    // Validate slot is current or next
    const currentSlot = this.epochCache.getCurrentSlot();
    if (proposal.slotNumber < currentSlot - 1 || proposal.slotNumber > currentSlot + 1) {
      return 'Invalid slot';
    }

    // Validate proposer matches expected for slot
    const expectedProposer = await this.epochCache.getProposerForSlot(proposal.slotNumber);
    if (!expectedProposer.equals(proposer)) {
      return 'Wrong proposer';
    }

    return undefined; // Valid
  }
}

File: p2p/src/msg_validators/checkpoint_attestation_validator.ts (new file)

Similar pattern to existing AttestationValidator but for CheckpointAttestation.

2.5 Update Attestation Pool

File: p2p/src/mem_pools/attestation_pool/attestation_pool.ts

Add methods for checkpoint attestations:

export interface AttestationPool {
  // Keep existing methods for backward compat during transition...

  // New methods for checkpoint attestations
  addCheckpointAttestations(attestations: CheckpointAttestation[]): Promise<void>;
  getCheckpointAttestationsForSlotAndProposal(
    slot: SlotNumber,
    proposalId: string
  ): Promise<CheckpointAttestation[]>;
  deleteCheckpointAttestationsOlderThan(slot: SlotNumber): Promise<void>;
}

File: p2p/src/mem_pools/attestation_pool/memory_attestation_pool.ts

Implement the new methods with a separate map for checkpoint attestations.


Phase 3: Validator Client Updates

3.1 Create CheckpointAttestationJob

File: validator-client/src/checkpoint_attestation_job.ts (new file)

This is the core new component managing long-lived checkpoint building with parallel execution:

export class CheckpointAttestationJob {
  private checkpointBuilder: CheckpointBuilder | undefined;
  private worldStateFork: MerkleTreeWriteOperations | undefined;

  // Track execution state
  private currentlyExecutingIndex: number | undefined;
  private executedBlockIndices: Set<number> = new Set();
  private queuedProposals: Map<number, { proposal: BlockProposal; txs: Tx[] }> = new Map();
  private executionPromise: Promise<void> | undefined;

  // Track block results for final checkpoint comparison
  private executedBlocks: Map<number, L2BlockNew> = new Map();

  constructor(
    private readonly slot: SlotNumber,
    private readonly checkpointNumber: CheckpointNumber,
    private readonly parentBlockNumber: BlockNumber,
    private readonly checkpointsBuilder: FullNodeCheckpointsBuilder,
    private readonly worldState: WorldStateSynchronizer,
    private readonly l1ToL2Messages: Fr[],
    private readonly validationService: ValidationService,
    private readonly attestorAddresses: EthAddress[],
    private readonly config: ValidatorClientFullConfig,
    private readonly log: Logger,
  ) {}

  /** Initialize the job - create world state fork and checkpoint builder */
  async initialize(
    coinbase: EthAddress,
    feeRecipient: AztecAddress,
    timestamp: bigint,
  ): Promise<void> {
    this.worldStateFork = await this.worldState.fork(this.parentBlockNumber);
    this.checkpointBuilder = await this.checkpointsBuilder.startCheckpoint(
      this.checkpointNumber,
      { coinbase, feeRecipient, timestamp, slotNumber: this.slot },
      this.l1ToL2Messages,
      this.worldStateFork,
    );
  }

  /** Called when a non-last block proposal is received - starts execution immediately */
  async handleBlockProposal(proposal: BlockProposal, txs: Tx[]): Promise<void> {
    const index = proposal.indexWithinCheckpoint;
    this.log.debug(`Received block proposal ${index} for slot ${this.slot}`);

    // Queue the proposal
    this.queuedProposals.set(index, { proposal, txs });

    // Try to start executing the next block if we're not already executing
    await this.tryExecuteNextBlock();
  }

  /** Called when the checkpoint proposal is received (last block) */
  async handleCheckpointProposal(
    proposal: CheckpointProposal,
    txs: Tx[],
  ): Promise<CheckpointAttestation[] | undefined> {
    const lastBlockIndex = proposal.indexWithinCheckpoint;
    this.log.debug(`Received checkpoint proposal with last block ${lastBlockIndex}`);

    // Queue the last block for execution
    this.queuedProposals.set(lastBlockIndex, {
      proposal: this.checkpointProposalToBlockProposal(proposal),
      txs
    });

    // Wait for all blocks to finish executing (including any in progress)
    await this.waitForAllBlocksExecuted(lastBlockIndex);

    // Check if we executed all blocks
    if (this.executedBlockIndices.size !== lastBlockIndex + 1) {
      this.log.warn(`Missing blocks - only executed ${this.executedBlockIndices.size} of ${lastBlockIndex + 1}`);
      return undefined;
    }

    // Complete the checkpoint
    const localCheckpoint = await this.checkpointBuilder!.completeCheckpoint();

    // Compare with proposal
    if (!this.checkpointsMatch(localCheckpoint, proposal)) {
      this.log.warn(`Checkpoint mismatch - not attesting`, {
        localArchive: localCheckpoint.archive.root.toString(),
        proposalArchive: proposal.archiveRoot.toString(),
      });
      return undefined;
    }

    // Create and return attestations
    return this.validationService.attestToCheckpoint(
      proposal.checkpointHeader,
      proposal.archiveRoot,
      proposal.checkpointSignature,
      this.attestorAddresses,
    );
  }

  /** Tries to execute the next block if conditions are met */
  private async tryExecuteNextBlock(): Promise<void> {
    // Don't start if already executing
    if (this.executionPromise) return;

    // Find next block to execute
    const nextIndex = this.executedBlockIndices.size;
    const queued = this.queuedProposals.get(nextIndex);
    if (!queued) return;

    // Start execution
    this.currentlyExecutingIndex = nextIndex;
    this.executionPromise = this.executeBlock(queued.proposal, queued.txs, nextIndex);

    try {
      await this.executionPromise;
    } finally {
      this.executionPromise = undefined;
      this.currentlyExecutingIndex = undefined;
    }

    // Try to execute next queued block
    await this.tryExecuteNextBlock();
  }

  private async executeBlock(proposal: BlockProposal, txs: Tx[], index: number): Promise<void> {
    this.log.debug(`Executing block ${index} for slot ${this.slot}`);

    const { block } = await this.checkpointBuilder!.buildBlock(
      txs,
      proposal.blockNumber,
      proposal.blockHeader.globalVariables.timestamp,
      { /* limits based on config */ },
    );

    // Validate block matches proposal
    if (!block.archive.root.equals(proposal.archiveRoot)) {
      this.log.warn(`Block ${index} state mismatch`, {
        expected: proposal.archiveRoot.toString(),
        got: block.archive.root.toString(),
      });
      throw new ReExStateMismatchError(proposal.archiveRoot, block.archive.root);
    }

    this.executedBlocks.set(index, block);
    this.executedBlockIndices.add(index);
    this.queuedProposals.delete(index);
  }

  private async waitForAllBlocksExecuted(lastIndex: number): Promise<void> {
    // Keep trying to execute until we've done all blocks or run out of queued blocks
    while (this.executedBlockIndices.size <= lastIndex) {
      if (this.executionPromise) {
        await this.executionPromise;
      }
      await this.tryExecuteNextBlock();

      // If no execution started and we don't have all blocks, some are missing
      if (!this.executionPromise && this.executedBlockIndices.size <= lastIndex) {
        const nextNeeded = this.executedBlockIndices.size;
        if (!this.queuedProposals.has(nextNeeded)) {
          this.log.warn(`Block ${nextNeeded} not received, cannot complete checkpoint`);
          break;
        }
      }
    }
  }

  private checkpointsMatch(local: Checkpoint, proposal: CheckpointProposal): boolean {
    return (
      local.archive.root.equals(proposal.archiveRoot) &&
      local.header.hash().equals(proposal.checkpointHeader.hash())
    );
  }

  /** Convert CheckpointProposal to BlockProposal-like for unified processing */
  private checkpointProposalToBlockProposal(proposal: CheckpointProposal): BlockProposal {
    // Extract block-level data from checkpoint proposal
    return new BlockProposal(
      proposal.lastBlockHeader,
      proposal.indexWithinCheckpoint,
      proposal.inHash,
      proposal.archiveRoot,
      proposal.txHashes,
      proposal.checkpointSignature, // Use checkpoint sig for last block
      proposal.txs,
    );
  }

  async dispose(): Promise<void> {
    await this.worldStateFork?.close();
  }
}

3.2 Update ValidatorClient

File: validator-client/src/validator.ts

Key changes:

  1. Add map to track active attestation jobs:

    private activeAttestationJobs: Map<SlotNumber, CheckpointAttestationJob> = new Map();
  2. Register handlers in start():

    this.p2pClient.registerBlockProposalHandler(
      (proposal, sender) => this.handleBlockProposal(proposal, sender)
    );
    this.p2pClient.registerCheckpointProposalHandler(
      (proposal, sender) => this.handleCheckpointProposal(proposal, sender)
    );
  3. Implement handleBlockProposal():

    private async handleBlockProposal(
      proposal: BlockProposal,
      sender: PeerId,
    ): Promise<void> {
      const slot = proposal.slotNumber;
    
      // Get or create attestation job for this slot
      let job = this.activeAttestationJobs.get(slot);
      if (!job) {
        // Create new job
        job = new CheckpointAttestationJob(...);
        await job.initialize(...);
        this.activeAttestationJobs.set(slot, job);
      }
    
      // Fetch txs and forward to job
      const txs = await this.fetchTxsForProposal(proposal, sender);
      await job.handleBlockProposal(proposal, txs);
    }
  4. Implement handleCheckpointProposal():

    private async handleCheckpointProposal(
      proposal: CheckpointProposal,
      sender: PeerId,
    ): Promise<CheckpointAttestation[] | undefined> {
      const slot = proposal.slotNumber;
      const job = this.activeAttestationJobs.get(slot);
    
      if (!job) {
        // Started mid-slot, cannot attest
        this.log.warn(`No attestation job for slot ${slot}`);
        return undefined;
      }
    
      const txs = await this.fetchTxsForProposal(proposal, sender);
      const attestations = await job.handleCheckpointProposal(proposal, txs);
    
      // Cleanup job
      await job.dispose();
      this.activeAttestationJobs.delete(slot);
    
      return attestations;
    }
  5. Update createBlockProposal() to use new BlockProposal structure

  6. Add createCheckpointProposal():

    async createCheckpointProposal(
      lastBlock: L2BlockNew,
      checkpoint: Checkpoint,
      txs: Tx[],
      proposerAddress: EthAddress | undefined,
    ): Promise<CheckpointProposal> {
      return this.validationService.createCheckpointProposal(
        lastBlock.header,
        lastBlock.indexWithinCheckpoint,
        lastBlock.archive.root,
        txs.map(tx => tx.getTxHash()),
        checkpoint.header,
        txs,
        proposerAddress,
      );
    }
  7. Add collectCheckpointAttestations():

    async collectCheckpointAttestations(
      proposal: CheckpointProposal,
      requiredCount: number,
      deadline: Date,
    ): Promise<CheckpointAttestation[]> {
      // Similar to existing collectAttestations but for checkpoint attestations
    }

3.3 Update ValidationService

File: validator-client/src/duties/validation_service.ts

Add methods:

async createBlockProposal(
  blockHeader: BlockHeader,
  indexWithinCheckpoint: number,
  inHash: Fr,
  archiveRoot: Fr,
  txHashes: TxHash[],
  txs: Tx[] | undefined,
  payloadSigner: EthAddress,
): Promise<BlockProposal>;

async createCheckpointProposal(
  lastBlockHeader: BlockHeader,
  indexWithinCheckpoint: number,
  archiveRoot: Fr,
  txHashes: TxHash[],
  checkpointHeader: CheckpointHeader,
  txs: Tx[] | undefined,
  payloadSigner: EthAddress,
): Promise<CheckpointProposal>;

async attestToCheckpoint(
  header: CheckpointHeader,
  archive: Fr,
  proposerSignature: Signature,
  attestors: EthAddress[],
): Promise<CheckpointAttestation[]>;

Phase 4: Sequencer-Publisher Updates

4.1 Update CheckpointProposalJob

File: sequencer-client/src/sequencer/checkpoint_proposal_job.ts

Update buildBlocksForCheckpoint():

private async buildBlocksForCheckpoint(...): Promise<...> {
  const blocksInCheckpoint: L2BlockNew[] = [];
  let pendingBroadcast: { block: L2BlockNew; txs: Tx[] } | undefined;

  while (true) {
    // ... existing timing logic ...

    const buildResult = await this.buildSingleBlock(checkpointBuilder, { ... });
    if (!buildResult || 'error' in buildResult) break;

    const { block, usedTxs } = buildResult;
    blocksInCheckpoint.push(block);

    if (timingInfo.isLastBlock) {
      // Last block - will be sent with checkpoint proposal
      pendingBroadcast = { block, txs: usedTxs };
      break;
    }

    // Non-last block: broadcast BlockProposal immediately
    if (!this.config.fishermanMode) {
      const proposal = await this.validatorClient.createBlockProposal(
        block.header,
        block.indexWithinCheckpoint,
        checkpointGlobalVariables.inHash,
        block.archive.root,
        usedTxs,
        this.proposer,
        blockProposalOptions,
      );
      await this.p2pClient.broadcastBlockProposal(proposal);
    }

    await this.waitUntilNextSubslot(timingInfo.deadline);
  }

  return { blocksInCheckpoint, pendingBroadcast };
}

Update checkpoint proposal creation and attestation collection:

// After block building loop completes
const checkpoint = await checkpointBuilder.completeCheckpoint();

// Create and broadcast checkpoint proposal
const checkpointProposal = await this.validatorClient.createCheckpointProposal(
  pendingBroadcast.block,
  checkpoint,
  pendingBroadcast.txs,
  this.proposer,
);
await this.p2pClient.broadcastCheckpointProposal(checkpointProposal);

// Collect checkpoint attestations
this.setStateFn(SequencerState.COLLECTING_ATTESTATIONS, this.slot);
const attestations = await this.collectCheckpointAttestations(checkpointProposal);

Update waitForAttestations() to collectCheckpointAttestations():

private async collectCheckpointAttestations(
  proposal: CheckpointProposal,
): Promise<CheckpointAttestationsAndSigners> {
  // Similar to existing but queries checkpoint attestations
  const attestations = await this.validatorClient.collectCheckpointAttestations(
    proposal,
    numberOfRequiredAttestations,
    attestationDeadline,
  );

  return new CheckpointAttestationsAndSigners(orderAttestations(attestations, committee));
}

Phase 5: Edge Cases and Validation

5.1 Block Proposal Validation

In BlockProposalValidator:

  • Validate proposer signature
  • Validate slot is current or next
  • Validate proposer matches expected for slot
  • Validate indexWithinCheckpoint is reasonable (0 to MAX_BLOCKS_PER_SLOT)
  • Validate block number is sequential from parent

5.2 Checkpoint Proposal Validation

In CheckpointProposalValidator:

  • All block proposal validations
  • Validate checkpointHeader fields are consistent with last block
  • Validate indexWithinCheckpoint matches expected block count
  • Validate checkpoint signature

5.3 Out-of-Order Block Handling

In CheckpointAttestationJob:

  • Queue blocks that arrive before predecessors (due to network reordering)
  • Process queued blocks when predecessors complete
  • Blocks must be executed sequentially (block N depends on block N-1's state)

5.4 Missing Block Handling

Decision: Wait until slot ends. No per-block timeouts.

  • If checkpoint proposal arrives but blocks are missing:
    • Log warning with missing indices
    • Return undefined (no attestation)
    • Cleanup job resources when slot ends

5.5 Validator Starting Mid-Slot

  • If first received block has indexWithinCheckpoint > 0:
    • Cannot build checkpoint from scratch (missing initial state)
    • Log info message
    • Skip slot and wait for next

5.6 Slot Timeout

In ValidatorClient:

  • Calculate slot end from slotStartTimestamp + slotDuration
  • Set overall timeout on attestation job
  • When slot ends:
    • Dispose job and cleanup resources
    • Delete from activeAttestationJobs map
    • Log summary of what was executed vs. what was missed

5.7 State Mismatch During Execution

If re-execution produces different state than proposal:

  • Log warning with expected vs. actual archive roots
  • Mark block as failed but continue to next block
  • At checkpoint completion, if any block failed, don't attest
  • Consider slashing report for proposer (future work)

Phase 6: Test Updates

6.1 New Unit Tests

New test files:

  • stdlib/src/p2p/block_proposal.test.ts - Update for new structure
  • stdlib/src/p2p/checkpoint_proposal.test.ts - Serialization, signing, verification
  • stdlib/src/p2p/checkpoint_attestation.test.ts - Serialization, signing, verification
  • validator-client/src/checkpoint_attestation_job.test.ts - Job lifecycle, edge cases

6.2 Update Existing Tests

Files to update:

  • p2p/src/client/p2p_client.integration_*.test.ts - New broadcast/handler methods
  • p2p/src/msg_validators/*.test.ts - New validators
  • validator-client/src/validator.test.ts - New handler flows
  • sequencer-client/src/sequencer/checkpoint_proposal_job.test.ts - New proposal creation

6.3 Test Cases to Cover

  1. Happy path: 5 blocks built and broadcast, checkpoint attested
  2. Out-of-order blocks: Blocks 3, 1, 2 arrive - should handle correctly
  3. Missing block: Block 2 never arrives, checkpoint proposal arrives - no attestation
  4. Validator starts mid-slot: First block received has index=2 - skip slot
  5. Timeout: Slot ends before checkpoint proposal - cleanup
  6. State mismatch: Re-execution produces different state - no attestation
  7. Invalid proposer: Block from wrong proposer - reject
  8. Duplicate block: Same block index arrives twice - ignore duplicate

Implementation Order

Stage 1: Foundation (stdlib types)

  1. Update BlockProposal class with new structure
  2. Create CheckpointProposal class
  3. Create CheckpointAttestation class
  4. Update TopicType enum
  5. Update signature utilities
  6. Add unit tests

Stage 2: P2P Layer

  1. Update AttestationPool interface and implementations
  2. Create new message validators
  3. Update LibP2P service for new topics and handlers
  4. Update P2P client interface and implementation
  5. Add integration tests

Stage 3: Validator Client

  1. Create CheckpointAttestationJob class
  2. Update ValidationService with new methods
  3. Update ValidatorClient with new handlers
  4. Add unit tests

Stage 4: Sequencer

  1. Update CheckpointProposalJob
  2. Wire to new P2P APIs
  3. Update attestation collection
  4. Add tests

Stage 5: Integration

  1. Update integration tests
  2. Update E2E tests
  3. Manual testing

Critical Files Summary

File Changes
stdlib/src/p2p/block_proposal.ts Update structure for per-block data
stdlib/src/p2p/checkpoint_proposal.ts New file
stdlib/src/p2p/checkpoint_attestation.ts New file
stdlib/src/p2p/topic_type.ts Add new topics
p2p/src/client/interface.ts New methods
p2p/src/client/p2p_client.ts Implement new methods
p2p/src/services/libp2p/libp2p_service.ts New topic handlers
p2p/src/msg_validators/checkpoint_proposal_validator.ts New file
p2p/src/msg_validators/checkpoint_attestation_validator.ts New file
p2p/src/mem_pools/attestation_pool/*.ts Add checkpoint attestation support
validator-client/src/checkpoint_attestation_job.ts New file - core logic
validator-client/src/validator.ts New handlers, job management
validator-client/src/duties/validation_service.ts New creation methods
sequencer-client/src/sequencer/checkpoint_proposal_job.ts Wire new APIs

Progress Update (2025-12-23)

Completed Work

Phase 1: New P2P Message Types ✅

  • Updated BlockProposal to use BlockHeader instead of CheckpointHeader
  • Created CheckpointProposal class with proper structure
  • Created CheckpointAttestation and CheckpointAttestationPayload classes
  • Updated TopicType enum with new topics
  • Updated signature utilities with new domain separators
  • Updated exports in stdlib/src/p2p/index.ts

Phase 2: P2P Layer Updates ✅

  • Updated P2P client interface with new callback types:
    • P2PBlockReceivedCallback returns Promise<boolean> (validation only)
    • P2PCheckpointReceivedCallback returns Promise<CheckpointAttestation[] | undefined>
  • Added broadcastCheckpointProposal and broadcastCheckpointAttestations stubs to P2P interface
  • Updated DummyP2P implementation in txe package
  • Updated registerBlockProposalHandler and added registerCheckpointProposalHandler

Phase 3: Validator Client Updates (Partial) ✅

  • Updated ValidationService:
    • createBlockProposal now takes BlockHeader, indexWithinCheckpoint, inHash, etc.
    • Added attestToCheckpointProposal method
    • Deprecated attestToProposal (for backward compat with attestation pool)
  • Updated ValidatorClient:
    • validateBlockProposal returns Promise<boolean> (no attestations)
    • Added attestToCheckpointProposal method
    • Updated createBlockProposal signature
    • Added placeholder createCheckpointProposal (currently delegates to createBlockProposal)
    • Registered handlers for both block and checkpoint proposals

Phase 4: Sequencer Updates ✅

  • Updated checkpoint_proposal_job.ts:
    • Added inHash computation from L1 to L2 messages
    • Updated createBlockProposal calls with new signature
    • Updated createCheckpointProposal calls

Tests Fixed ✅

  • validation_service.test.ts - Updated to use new API
  • validator.test.ts - Fixed inHash mismatch in test setup
  • checkpoint_proposal_job.test.ts - Updated mock implementations
  • test/utils.ts - Updated createBlockProposal helper
  • e2e_multi_validator_node_key_store.test.ts - Updated wrapper function
  • reex.test.ts - Updated to use BlockProposal.createProposalFromSigner
  • attestation_pool_test_suite.ts - Updated to use new makeBlockProposal

Remaining Work

1. Attestation Pool Migration (TODO)

Priority: High

The attestation pool still uses BlockAttestation. Needs to be updated to use CheckpointAttestation:

  • Add CheckpointAttestation methods to attestation pool interface:
    • addCheckpointAttestations(attestations: CheckpointAttestation[]): Promise<void>
    • getCheckpointAttestationsForSlotAndProposal(slot, proposalId): Promise<CheckpointAttestation[]>
  • Implement in MemoryAttestationPool and KV store versions
  • Update collectAttestations to use CheckpointAttestation
  • Remove deprecated attestToProposal method in ValidationService

Files to update:

  • p2p/src/mem_pools/attestation_pool/attestation_pool.ts
  • p2p/src/mem_pools/attestation_pool/memory_attestation_pool.ts
  • p2p/src/mem_pools/attestation_pool/kv_attestation_pool.ts
  • validator-client/src/duties/validation_service.ts (remove deprecated method)
  • stdlib/src/interfaces/validator.ts (update collectAttestations signature)

See TODOs:

2. Create Actual CheckpointProposal (TODO)

Priority: Medium

Currently createCheckpointProposal in validator.ts just delegates to createBlockProposal. It should:

  • Create an actual CheckpointProposal instance (not BlockProposal)
  • Include CheckpointHeader from the completed checkpoint
  • Use proper signing with SignatureDomainSeparator.checkpointProposal

See TODO:

3. Add Checkpoint Proposal Validation (TODO)

Priority: Medium

The attestToCheckpointProposal method currently just blindly attests without validation. Should add:

  • Validate proposer matches expected for slot
  • Validate checkpoint header fields are consistent
  • Re-execute transactions and compare state (similar to block proposal validation)

See TODO:

4. Implement CheckpointAttestationJob (Optional for now)

Priority: Low

The full parallel execution model described in Phase 3.1 (CheckpointAttestationJob) is not yet implemented. Currently validators process proposals synchronously. This is fine for initial MBPS but will need to be added for optimal performance.

5. Double Proposal Prevention (TODO)

Priority: Low

Currently commented out in createBlockProposal. Should properly prevent double proposals for the same slot.

See TODO:

6. E2E Testing

Priority: High

The e2e tests have been updated to compile, but should be run to verify:

  • e2e_multi_validator_node_key_store.test.ts
  • reex.test.ts
  • Any other MBPS-related e2e tests

Test Commands

# Unit tests
cd yarn-project/validator-client && yarn test validation_service.test.ts
cd yarn-project/validator-client && yarn test validator.test.ts
cd yarn-project/sequencer-client && yarn test checkpoint_proposal_job.test.ts
cd yarn-project/p2p && yarn test attestation_pool

# E2E tests (run one at a time)
cd yarn-project/end-to-end && yarn test:e2e e2e_multi_validator_node_key_store.test.ts
cd yarn-project/end-to-end && yarn test:e2e reex.test.ts

Base automatically changed from palla/nicer-timetable to next December 24, 2025 17:23
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants