From 18b6b6587ce8930d01a152a16f82040c62e28fb2 Mon Sep 17 00:00:00 2001 From: colinlyguo Date: Tue, 29 Jul 2025 22:18:52 +0800 Subject: [PATCH 01/14] feat(rollup-relayer): add sanity checks --- common/version/version.go | 2 +- .../internal/controller/relayer/l2_relayer.go | 409 +++++++++++++++++- rollup/internal/orm/batch.go | 13 + 3 files changed, 422 insertions(+), 2 deletions(-) diff --git a/common/version/version.go b/common/version/version.go index b50743fd1d..f31fe8e48a 100644 --- a/common/version/version.go +++ b/common/version/version.go @@ -5,7 +5,7 @@ import ( "runtime/debug" ) -var tag = "v4.5.35" +var tag = "v4.5.36" var commit = func() string { if info, ok := debug.ReadBuildInfo(); ok { diff --git a/rollup/internal/controller/relayer/l2_relayer.go b/rollup/internal/controller/relayer/l2_relayer.go index 18110d7b17..5d52c73bef 100644 --- a/rollup/internal/controller/relayer/l2_relayer.go +++ b/rollup/internal/controller/relayer/l2_relayer.go @@ -473,6 +473,12 @@ func (r *Layer2Relayer) ProcessPendingBatches() { log.Info("Forcing submission of batches due to timeout", "batch index", batchesToSubmit[0].Batch.Index, "first block created at", oldestBlockTimestamp) } + // Sanity checks before constructing the transaction + if err := r.sanityChecksBeforeConstructingTransaction(batchesToSubmit); err != nil { + log.Error("Sanity checks failed before constructing transaction", "batches count", len(batchesToSubmit), "first batch index", batchesToSubmit[0].Batch.Index, "last batch index", batchesToSubmit[len(batchesToSubmit)-1].Batch.Index, "err", err) + return + } + // We have at least 1 batch to commit firstBatch := batchesToSubmit[0].Batch lastBatch := batchesToSubmit[len(batchesToSubmit)-1].Batch @@ -941,6 +947,25 @@ func (r *Layer2Relayer) handleL2RollupRelayerConfirmLoop(ctx context.Context) { } func (r *Layer2Relayer) constructCommitBatchPayloadCodecV7(batchesToSubmit []*dbBatchWithChunks, firstBatch, lastBatch *orm.Batch) ([]byte, []*kzg4844.Blob, uint64, uint64, error) { + // Basic sanity checks + if len(batchesToSubmit) == 0 { + return nil, nil, 0, 0, fmt.Errorf("no batches to submit") + } + if firstBatch == nil { + return nil, nil, 0, 0, fmt.Errorf("first batch is nil") + } + if lastBatch == nil { + return nil, nil, 0, 0, fmt.Errorf("last batch is nil") + } + + // Check firstBatch and lastBatch match batchesToSubmit + if firstBatch.Index != batchesToSubmit[0].Batch.Index { + return nil, nil, 0, 0, fmt.Errorf("first batch index mismatch: expected %d, got %d", batchesToSubmit[0].Batch.Index, firstBatch.Index) + } + if lastBatch.Index != batchesToSubmit[len(batchesToSubmit)-1].Batch.Index { + return nil, nil, 0, 0, fmt.Errorf("last batch index mismatch: expected %d, got %d", batchesToSubmit[len(batchesToSubmit)-1].Batch.Index, lastBatch.Index) + } + var maxBlockHeight uint64 var totalGasUsed uint64 blobs := make([]*kzg4844.Blob, 0, len(batchesToSubmit)) @@ -961,6 +986,16 @@ func (r *Layer2Relayer) constructCommitBatchPayloadCodecV7(batchesToSubmit []*db return nil, nil, 0, 0, fmt.Errorf("failed to get blocks in range for batch %d: %w", b.Batch.Index, err) } + if len(blocks) == 0 { + return nil, nil, 0, 0, fmt.Errorf("batch %d chunk %d has no blocks in range [%d, %d]", b.Batch.Index, c.Index, c.StartBlockNumber, c.EndBlockNumber) + } + + // Check that we got the expected number of blocks + expectedBlockCount := c.EndBlockNumber - c.StartBlockNumber + 1 + if uint64(len(blocks)) != expectedBlockCount { + return nil, nil, 0, 0, fmt.Errorf("batch %d chunk %d expected %d blocks but got %d", b.Batch.Index, c.Index, expectedBlockCount, len(blocks)) + } + batchBlocks = append(batchBlocks, blocks...) if c.EndBlockNumber > maxBlockHeight { @@ -977,6 +1012,37 @@ func (r *Layer2Relayer) constructCommitBatchPayloadCodecV7(batchesToSubmit []*db Blocks: batchBlocks, } + // Check encoding batch fields are not zero hashes + if encodingBatch.ParentBatchHash == (common.Hash{}) { + return nil, nil, 0, 0, fmt.Errorf("batch %d parent batch hash is zero", b.Batch.Index) + } + + // Check L1 message queue hash consistency + var totalL1MessagesInBatch uint64 + for _, c := range b.Chunks { + totalL1MessagesInBatch += c.TotalL1MessagesPoppedInChunk + } + + // Check L1 message queue hash consistency + firstChunk := b.Chunks[0] + lastChunk := b.Chunks[len(b.Chunks)-1] + + // If there were L1 messages processed before this batch, prev hash should not be zero + if firstChunk.TotalL1MessagesPoppedBefore > 0 && encodingBatch.PrevL1MessageQueueHash == (common.Hash{}) { + return nil, nil, 0, 0, fmt.Errorf("batch %d prev L1 message queue hash is zero but %d L1 messages were processed before", b.Batch.Index, firstChunk.TotalL1MessagesPoppedBefore) + } + + // If there are any L1 messages processed up to this batch, post hash should not be zero + totalL1MessagesProcessed := lastChunk.TotalL1MessagesPoppedBefore + lastChunk.TotalL1MessagesPoppedInChunk + if totalL1MessagesProcessed > 0 && encodingBatch.PostL1MessageQueueHash == (common.Hash{}) { + return nil, nil, 0, 0, fmt.Errorf("batch %d post L1 message queue hash is zero but %d L1 messages were processed in total", b.Batch.Index, totalL1MessagesProcessed) + } + + // If L1 messages were processed in this batch, prev and post hashes should be different + if totalL1MessagesInBatch > 0 && encodingBatch.PrevL1MessageQueueHash == encodingBatch.PostL1MessageQueueHash { + return nil, nil, 0, 0, fmt.Errorf("batch %d has same prev and post L1 message queue hashes but processed %d L1 messages in this batch", b.Batch.Index, totalL1MessagesInBatch) + } + codec, err := encoding.CodecFromVersion(version) if err != nil { return nil, nil, 0, 0, fmt.Errorf("failed to get codec from version %d, err: %w", b.Batch.CodecVersion, err) @@ -987,17 +1053,36 @@ func (r *Layer2Relayer) constructCommitBatchPayloadCodecV7(batchesToSubmit []*db return nil, nil, 0, 0, fmt.Errorf("failed to create DA batch: %w", err) } - blobs = append(blobs, daBatch.Blob()) + blob := daBatch.Blob() + if blob == nil { + return nil, nil, 0, 0, fmt.Errorf("batch %d generated nil blob", b.Batch.Index) + } + + blobs = append(blobs, blob) } calldata, err := r.l1RollupABI.Pack("commitBatches", version, common.HexToHash(firstBatch.ParentBatchHash), common.HexToHash(lastBatch.Hash)) if err != nil { return nil, nil, 0, 0, fmt.Errorf("failed to pack commitBatches: %w", err) } + + if len(calldata) == 0 { + return nil, nil, 0, 0, fmt.Errorf("generated calldata is empty") + } + return calldata, blobs, maxBlockHeight, totalGasUsed, nil } func (r *Layer2Relayer) constructCommitBatchPayloadValidium(batch *dbBatchWithChunks) ([]byte, uint64, uint64, error) { + // Basic sanity checks + if batch == nil || batch.Batch == nil { + return nil, 0, 0, fmt.Errorf("batch is nil") + } + + if len(batch.Chunks) == 0 { + return nil, 0, 0, fmt.Errorf("batch %d has no chunks", batch.Batch.Index) + } + // Calculate metrics var maxBlockHeight uint64 var totalGasUsed uint64 @@ -1017,16 +1102,50 @@ func (r *Layer2Relayer) constructCommitBatchPayloadValidium(batch *dbBatchWithCh lastChunk := batch.Chunks[len(batch.Chunks)-1] commitment := common.HexToHash(lastChunk.EndBlockHash) + + if commitment == (common.Hash{}) { + return nil, 0, 0, fmt.Errorf("batch %d last chunk end block hash is zero, cannot create commitment", batch.Batch.Index) + } + + // Check parent batch hash is not zero + parentBatchHash := common.HexToHash(batch.Batch.ParentBatchHash) + if parentBatchHash == (common.Hash{}) { + return nil, 0, 0, fmt.Errorf("batch %d parent batch hash is zero", batch.Batch.Index) + } + version := encoding.CodecVersion(batch.Batch.CodecVersion) calldata, err := r.validiumABI.Pack("commitBatch", version, common.HexToHash(batch.Batch.ParentBatchHash), common.HexToHash(batch.Batch.StateRoot), common.HexToHash(batch.Batch.WithdrawRoot), commitment[:]) if err != nil { return nil, 0, 0, fmt.Errorf("failed to pack commitBatch: %w", err) } + + if len(calldata) == 0 { + return nil, 0, 0, fmt.Errorf("generated calldata is empty for batch %d", batch.Batch.Index) + } + log.Info("Validium commitBatch", "maxBlockHeight", maxBlockHeight, "commitment", commitment.Hex()) return calldata, maxBlockHeight, totalGasUsed, nil } func (r *Layer2Relayer) constructFinalizeBundlePayloadCodecV7(dbBatch *orm.Batch, endChunk *orm.Chunk, aggProof *message.OpenVMBundleProof) ([]byte, error) { + // Basic sanity checks + if dbBatch == nil { + return nil, fmt.Errorf("batch is nil") + } + if endChunk == nil { + return nil, fmt.Errorf("end chunk is nil") + } + + // Check batch header + if len(dbBatch.BatchHeader) == 0 { + return nil, fmt.Errorf("batch %d header is empty", dbBatch.Index) + } + + // Check proof if present + if aggProof != nil && len(aggProof.Proof()) == 0 { + return nil, fmt.Errorf("aggregate proof is empty") + } + if aggProof != nil { // finalizeBundle with proof. calldata, packErr := r.l1RollupABI.Pack( "finalizeBundlePostEuclidV2", @@ -1039,6 +1158,11 @@ func (r *Layer2Relayer) constructFinalizeBundlePayloadCodecV7(dbBatch *orm.Batch if packErr != nil { return nil, fmt.Errorf("failed to pack finalizeBundlePostEuclidV2 with proof: %w", packErr) } + + if len(calldata) == 0 { + return nil, fmt.Errorf("generated calldata with proof is empty") + } + return calldata, nil } @@ -1055,10 +1179,33 @@ func (r *Layer2Relayer) constructFinalizeBundlePayloadCodecV7(dbBatch *orm.Batch if packErr != nil { return nil, fmt.Errorf("failed to pack finalizeBundlePostEuclidV2NoProof: %w", packErr) } + + if len(calldata) == 0 { + return nil, fmt.Errorf("generated calldata without proof is empty") + } + return calldata, nil } func (r *Layer2Relayer) constructFinalizeBundlePayloadValidium(dbBatch *orm.Batch, endChunk *orm.Chunk, aggProof *message.OpenVMBundleProof) ([]byte, error) { + // Basic sanity checks + if dbBatch == nil { + return nil, fmt.Errorf("batch is nil") + } + if endChunk == nil { + return nil, fmt.Errorf("end chunk is nil") + } + + // Check batch header is not empty + if len(dbBatch.BatchHeader) == 0 { + return nil, fmt.Errorf("batch %d header is empty", dbBatch.Index) + } + + // Check proof if present + if aggProof != nil && len(aggProof.Proof()) == 0 { + return nil, fmt.Errorf("aggregate proof is empty") + } + log.Info("Packing validium finalizeBundle", "batchHeaderLength", len(dbBatch.BatchHeader), "codecVersion", dbBatch.CodecVersion, "totalL1Messages", endChunk.TotalL1MessagesPoppedBefore+endChunk.TotalL1MessagesPoppedInChunk, "stateRoot", dbBatch.StateRoot, "withdrawRoot", dbBatch.WithdrawRoot, "withProof", aggProof != nil) var proof []byte @@ -1075,6 +1222,11 @@ func (r *Layer2Relayer) constructFinalizeBundlePayloadValidium(dbBatch *orm.Batc if packErr != nil { return nil, fmt.Errorf("failed to pack validium finalizeBundle: %w", packErr) } + + if len(calldata) == 0 { + return nil, fmt.Errorf("generated calldata is empty for batch %d", dbBatch.Index) + } + return calldata, nil } @@ -1242,3 +1394,258 @@ func addrFromSignerConfig(config *config.SignerConfig) (common.Address, error) { return common.Address{}, fmt.Errorf("failed to determine signer address, unknown signer type: %v", config.SignerType) } } + +// sanityChecksBeforeConstructingTransaction performs sanity checks before constructing a transaction. +func (r *Layer2Relayer) sanityChecksBeforeConstructingTransaction(batchesToSubmit []*dbBatchWithChunks) error { + if len(batchesToSubmit) == 0 { + return fmt.Errorf("no batches to submit") + } + + // Basic validation + if err := r.validateBatchesBasic(batchesToSubmit); err != nil { + return err + } + + // Codec version validation + if err := r.validateCodecVersions(batchesToSubmit); err != nil { + return err + } + + // Get previous chunk for continuity check + prevChunk, err := r.getPreviousChunkForContinuity(batchesToSubmit[0]) + if err != nil { + return err + } + + // Validate each batch in detail + if err := r.validateBatchesDetailed(batchesToSubmit, prevChunk); err != nil { + return err + } + + log.Info("Sanity check passed before constructing transaction", "batches count", len(batchesToSubmit)) + return nil +} + +// validateBatchesBasic performs basic validation on all batches +func (r *Layer2Relayer) validateBatchesBasic(batchesToSubmit []*dbBatchWithChunks) error { + for i, batch := range batchesToSubmit { + if batch == nil || batch.Batch == nil { + return fmt.Errorf("batch %d is nil", i) + } + + if len(batch.Chunks) == 0 { + return fmt.Errorf("batch %d has no chunks", batch.Batch.Index) + } + } + return nil +} + +// validateCodecVersions checks all batches have the same codec version +func (r *Layer2Relayer) validateCodecVersions(batchesToSubmit []*dbBatchWithChunks) error { + firstBatchCodecVersion := batchesToSubmit[0].Batch.CodecVersion + for _, batch := range batchesToSubmit { + if batch.Batch.CodecVersion != firstBatchCodecVersion { + return fmt.Errorf("batch %d has different codec version %d, expected %d", batch.Batch.Index, batch.Batch.CodecVersion, firstBatchCodecVersion) + } + } + return nil +} + +// getPreviousChunkForContinuity gets the previous chunk for block continuity check +func (r *Layer2Relayer) getPreviousChunkForContinuity(firstBatch *dbBatchWithChunks) (*orm.Chunk, error) { + firstChunk := firstBatch.Chunks[0] + if firstChunk.Index == 0 { + return nil, fmt.Errorf("genesis chunk should not be in normal batch submission flow, chunk index: %d", firstChunk.Index) + } + + prevChunk, err := r.chunkOrm.GetChunkByIndex(r.ctx, firstChunk.Index-1) + if err != nil { + return nil, fmt.Errorf("failed to get previous chunk %d for continuity check: %w", firstChunk.Index-1, err) + } + + return prevChunk, nil +} + +// validateBatchesDetailed performs detailed validation on each batch +func (r *Layer2Relayer) validateBatchesDetailed(batchesToSubmit []*dbBatchWithChunks, prevChunkFromPrevBatch *orm.Chunk) error { + for i, batch := range batchesToSubmit { + if err := r.validateSingleBatch(batch, i, batchesToSubmit, prevChunkFromPrevBatch); err != nil { + return err + } + } + return nil +} + +// validateSingleBatch validates a single batch and its chunks +func (r *Layer2Relayer) validateSingleBatch(batch *dbBatchWithChunks, i int, allBatches []*dbBatchWithChunks, prevChunkFromPrevBatch *orm.Chunk) error { + // Validate batch fields + if err := r.validateBatchFields(batch, i, allBatches); err != nil { + return err + } + + // Validate message queue consistency + if err := r.validateMessageQueueConsistency(batch); err != nil { + return err + } + + // Validate chunks + if err := r.validateBatchChunks(batch, i, allBatches, prevChunkFromPrevBatch); err != nil { + return err + } + + return nil +} + +// validateBatchFields validates essential batch fields +func (r *Layer2Relayer) validateBatchFields(batch *dbBatchWithChunks, i int, allBatches []*dbBatchWithChunks) error { + // Check essential batch fields are not zero values + batchHash := common.HexToHash(batch.Batch.Hash) + if batchHash == (common.Hash{}) { + return fmt.Errorf("batch %d hash is zero", batch.Batch.Index) + } + + if batch.Batch.Index == 0 { + return fmt.Errorf("batch %d has zero index (only genesis batch should have index 0)", i) + } + + // Check batch index is sequential + if i > 0 { + prevBatch := allBatches[i-1] + if batch.Batch.Index != prevBatch.Batch.Index+1 { + return fmt.Errorf("batch index is not sequential: prev batch index %d, current batch index %d", prevBatch.Batch.Index, batch.Batch.Index) + } + } else { + // For the first batch, check continuity with the parent batch from database + parentBatch, err := r.batchOrm.GetBatchByHash(r.ctx, batch.Batch.ParentBatchHash) + if err != nil { + return fmt.Errorf("failed to get parent batch %s for batch %d: %w", batch.Batch.ParentBatchHash, batch.Batch.Index, err) + } + if batch.Batch.Index != parentBatch.Index+1 { + return fmt.Errorf("first batch index is not sequential with parent: parent batch index %d, current batch index %d", parentBatch.Index, batch.Batch.Index) + } + } + + parentBatchHash := common.HexToHash(batch.Batch.ParentBatchHash) + if parentBatchHash == (common.Hash{}) { + return fmt.Errorf("batch %d parent batch hash is zero", batch.Batch.Index) + } + + return nil +} + +// validateMessageQueueConsistency validates L1 message queue hash consistency +func (r *Layer2Relayer) validateMessageQueueConsistency(batch *dbBatchWithChunks) error { + if batch.Batch.Index == 0 { + return nil + } + + firstChunk := batch.Chunks[0] + lastChunk := batch.Chunks[len(batch.Chunks)-1] + + prevL1MsgQueueHash := common.HexToHash(batch.Batch.PrevL1MessageQueueHash) + postL1MsgQueueHash := common.HexToHash(batch.Batch.PostL1MessageQueueHash) + + // Calculate total L1 messages in this batch + var batchTotalL1MessagesInBatch uint64 + for _, chunk := range batch.Chunks { + batchTotalL1MessagesInBatch += chunk.TotalL1MessagesPoppedInChunk + } + + // If there were L1 messages processed before this batch, prev hash should not be zero + if firstChunk.TotalL1MessagesPoppedBefore > 0 && prevL1MsgQueueHash == (common.Hash{}) { + return fmt.Errorf("batch %d prev L1 message queue hash is zero but %d L1 messages were processed before", batch.Batch.Index, firstChunk.TotalL1MessagesPoppedBefore) + } + + // If there are any L1 messages processed up to this batch, post hash should not be zero + totalL1MessagesProcessed := lastChunk.TotalL1MessagesPoppedBefore + lastChunk.TotalL1MessagesPoppedInChunk + if totalL1MessagesProcessed > 0 && postL1MsgQueueHash == (common.Hash{}) { + return fmt.Errorf("batch %d post L1 message queue hash is zero but %d L1 messages were processed in total", batch.Batch.Index, totalL1MessagesProcessed) + } + + // Prev and post queue hashes should be different if L1 messages were processed in this batch + if batchTotalL1MessagesInBatch > 0 && prevL1MsgQueueHash == postL1MsgQueueHash { + return fmt.Errorf("batch %d has same prev and post L1 message queue hashes but processed %d L1 messages in this batch", batch.Batch.Index, batchTotalL1MessagesInBatch) + } + + return nil +} + +// validateBatchChunks validates all chunks in a batch +func (r *Layer2Relayer) validateBatchChunks(batch *dbBatchWithChunks, i int, allBatches []*dbBatchWithChunks, prevChunkFromPrevBatch *orm.Chunk) error { + // Check all chunks in this batch have the same codec version as the batch + for _, chunk := range batch.Chunks { + if chunk.CodecVersion != batch.Batch.CodecVersion { + return fmt.Errorf("batch %d chunk %d has different codec version %d, expected %d", batch.Batch.Index, chunk.Index, chunk.CodecVersion, batch.Batch.CodecVersion) + } + } + + for j, chunk := range batch.Chunks { + if err := r.validateSingleChunk(chunk, j, batch, i, allBatches, prevChunkFromPrevBatch); err != nil { + return err + } + } + + return nil +} + +// validateSingleChunk validates a single chunk +func (r *Layer2Relayer) validateSingleChunk(chunk *orm.Chunk, chunkIndex int, batch *dbBatchWithChunks, i int, allBatches []*dbBatchWithChunks, prevChunkFromPrevBatch *orm.Chunk) error { + if chunk == nil { + return fmt.Errorf("batch %d chunk %d is nil", batch.Batch.Index, chunkIndex) + } + + chunkHash := common.HexToHash(chunk.Hash) + if chunkHash == (common.Hash{}) { + return fmt.Errorf("batch %d chunk %d hash is zero", batch.Batch.Index, chunk.Index) + } + + // Get previous chunk for continuity check + var prevChunk *orm.Chunk + if chunkIndex > 0 { + prevChunk = batch.Chunks[chunkIndex-1] + } else if i == 0 { + prevChunk = prevChunkFromPrevBatch + } else if i > 0 { + // Use the last chunk from the previous batch + prevBatch := allBatches[i-1] + prevChunk = prevBatch.Chunks[len(prevBatch.Chunks)-1] + } + + // Check chunk index is sequential + if chunk.Index != prevChunk.Index+1 { + return fmt.Errorf("batch %d chunk %d index is not sequential: prev chunk index %d, current chunk index %d", batch.Batch.Index, chunkIndex, prevChunk.Index, chunk.Index) + } + + // Check L1 messages popped continuity + expectedPoppedBefore := prevChunk.TotalL1MessagesPoppedBefore + prevChunk.TotalL1MessagesPoppedInChunk + if chunk.TotalL1MessagesPoppedBefore != expectedPoppedBefore { + return fmt.Errorf("batch %d chunk %d L1 messages popped before is incorrect: expected %d, got %d", + batch.Batch.Index, chunk.Index, expectedPoppedBefore, chunk.TotalL1MessagesPoppedBefore) + } + + if chunk.StartBlockNumber == 0 && chunk.EndBlockNumber == 0 { + return fmt.Errorf("batch %d chunk %d has zero block range", batch.Batch.Index, chunk.Index) + } + + if chunk.StartBlockNumber > chunk.EndBlockNumber { + return fmt.Errorf("batch %d chunk %d has invalid block range: start %d > end %d", batch.Batch.Index, chunk.Index, chunk.StartBlockNumber, chunk.EndBlockNumber) + } + + // Check chunk hash fields + startBlockHash := common.HexToHash(chunk.StartBlockHash) + if startBlockHash == (common.Hash{}) { + return fmt.Errorf("batch %d chunk %d start block hash is zero", batch.Batch.Index, chunk.Index) + } + + endBlockHash := common.HexToHash(chunk.EndBlockHash) + if endBlockHash == (common.Hash{}) { + return fmt.Errorf("batch %d chunk %d end block hash is zero", batch.Batch.Index, chunk.Index) + } + + // Check chunk continuity: previous chunk's end block number + 1 should equal current chunk's start block number + if prevChunk.EndBlockNumber+1 != chunk.StartBlockNumber { + return fmt.Errorf("batch %d chunk %d is not continuous with previous chunk: prev chunk %d end %d, current chunk start %d", batch.Batch.Index, chunk.Index, prevChunk.Index, prevChunk.EndBlockNumber, chunk.StartBlockNumber) + } + + return nil +} diff --git a/rollup/internal/orm/batch.go b/rollup/internal/orm/batch.go index 9bb699937f..14812b1dab 100644 --- a/rollup/internal/orm/batch.go +++ b/rollup/internal/orm/batch.go @@ -263,6 +263,19 @@ func (o *Batch) GetBatchByIndex(ctx context.Context, index uint64) (*Batch, erro return &batch, nil } +// GetBatchByHash retrieves the batch by the given hash. +func (o *Batch) GetBatchByHash(ctx context.Context, hash string) (*Batch, error) { + db := o.db.WithContext(ctx) + db = db.Model(&Batch{}) + db = db.Where("hash = ?", hash) + + var batch Batch + if err := db.First(&batch).Error; err != nil { + return nil, fmt.Errorf("Batch.GetBatchByHash error: %w, batch hash: %v", err, hash) + } + return &batch, nil +} + // InsertBatch inserts a new batch into the database. func (o *Batch) InsertBatch(ctx context.Context, batch *encoding.Batch, codecVersion encoding.CodecVersion, metrics rutils.BatchMetrics, dbTX ...*gorm.DB) (*Batch, error) { if batch == nil { From f225d251c60745e5843e4debe4cbc4d7ec7728c3 Mon Sep 17 00:00:00 2001 From: colinlyguo Date: Wed, 30 Jul 2025 00:33:04 +0800 Subject: [PATCH 02/14] add state root checks --- .../internal/controller/relayer/l2_relayer.go | 36 ++++++++++++++++++- .../controller/relayer/l2_relayer_test.go | 7 ++-- 2 files changed, 40 insertions(+), 3 deletions(-) diff --git a/rollup/internal/controller/relayer/l2_relayer.go b/rollup/internal/controller/relayer/l2_relayer.go index 5d52c73bef..050bec324c 100644 --- a/rollup/internal/controller/relayer/l2_relayer.go +++ b/rollup/internal/controller/relayer/l2_relayer.go @@ -277,6 +277,19 @@ func (r *Layer2Relayer) initializeGenesis() error { } func (r *Layer2Relayer) commitGenesisBatch(batchHash string, batchHeader []byte, stateRoot common.Hash) error { + // Basic sanity checks + if batchHash == "" { + return fmt.Errorf("batch hash is empty") + } + + if len(batchHeader) == 0 { + return fmt.Errorf("batch header is empty") + } + + if stateRoot == (common.Hash{}) { + return fmt.Errorf("state root is zero") + } + var calldata []byte var packErr error @@ -296,6 +309,11 @@ func (r *Layer2Relayer) commitGenesisBatch(batchHash string, batchHeader []byte, log.Info("Rollup importGenesis", "calldata", common.Bytes2Hex(calldata), "stateRoot", stateRoot) } + // Check generated calldata is not empty + if len(calldata) == 0 { + return fmt.Errorf("generated calldata is empty") + } + // submit genesis batch to L1 rollup contract txHash, _, err := r.commitSender.SendTransaction(batchHash, &r.cfg.RollupContractAddress, calldata, nil) if err != nil { @@ -1083,6 +1101,12 @@ func (r *Layer2Relayer) constructCommitBatchPayloadValidium(batch *dbBatchWithCh return nil, 0, 0, fmt.Errorf("batch %d has no chunks", batch.Batch.Index) } + // Check state root is not zero + stateRoot := common.HexToHash(batch.Batch.StateRoot) + if stateRoot == (common.Hash{}) { + return nil, 0, 0, fmt.Errorf("batch %d state root is zero", batch.Batch.Index) + } + // Calculate metrics var maxBlockHeight uint64 var totalGasUsed uint64 @@ -1102,7 +1126,6 @@ func (r *Layer2Relayer) constructCommitBatchPayloadValidium(batch *dbBatchWithCh lastChunk := batch.Chunks[len(batch.Chunks)-1] commitment := common.HexToHash(lastChunk.EndBlockHash) - if commitment == (common.Hash{}) { return nil, 0, 0, fmt.Errorf("batch %d last chunk end block hash is zero, cannot create commitment", batch.Batch.Index) } @@ -1141,6 +1164,12 @@ func (r *Layer2Relayer) constructFinalizeBundlePayloadCodecV7(dbBatch *orm.Batch return nil, fmt.Errorf("batch %d header is empty", dbBatch.Index) } + // Check state root is not zero + stateRoot := common.HexToHash(dbBatch.StateRoot) + if stateRoot == (common.Hash{}) { + return nil, fmt.Errorf("batch %d state root is zero", dbBatch.Index) + } + // Check proof if present if aggProof != nil && len(aggProof.Proof()) == 0 { return nil, fmt.Errorf("aggregate proof is empty") @@ -1530,6 +1559,11 @@ func (r *Layer2Relayer) validateBatchFields(batch *dbBatchWithChunks, i int, all return fmt.Errorf("batch %d parent batch hash is zero", batch.Batch.Index) } + stateRoot := common.HexToHash(batch.Batch.StateRoot) + if stateRoot == (common.Hash{}) { + return fmt.Errorf("batch %d state root is zero", batch.Batch.Index) + } + return nil } diff --git a/rollup/internal/controller/relayer/l2_relayer_test.go b/rollup/internal/controller/relayer/l2_relayer_test.go index 216507d506..4074dbc7e7 100644 --- a/rollup/internal/controller/relayer/l2_relayer_test.go +++ b/rollup/internal/controller/relayer/l2_relayer_test.go @@ -70,15 +70,18 @@ func testL2RelayerProcessPendingBatches(t *testing.T) { _, err = chunkOrm.InsertChunk(context.Background(), chunk2, encoding.CodecV7, rutils.ChunkMetrics{}) assert.NoError(t, err) + batchOrm := orm.NewBatch(db) + genesisBatch, err := batchOrm.GetBatchByIndex(context.Background(), 0) + assert.NoError(t, err) + batch := &encoding.Batch{ Index: 1, TotalL1MessagePoppedBefore: 0, - ParentBatchHash: common.Hash{}, + ParentBatchHash: common.HexToHash(genesisBatch.Hash), Chunks: []*encoding.Chunk{chunk1, chunk2}, Blocks: []*encoding.Block{block1, block2}, } - batchOrm := orm.NewBatch(db) dbBatch, err := batchOrm.InsertBatch(context.Background(), batch, encoding.CodecV7, rutils.BatchMetrics{}) assert.NoError(t, err) From ae28f99d1b0c86af7fbac7b3fc009e3442f949e0 Mon Sep 17 00:00:00 2001 From: colinlyguo Date: Wed, 30 Jul 2025 01:11:07 +0800 Subject: [PATCH 03/14] fix unit tests --- rollup/internal/controller/relayer/l2_relayer.go | 2 +- rollup/internal/controller/relayer/relayer_test.go | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/rollup/internal/controller/relayer/l2_relayer.go b/rollup/internal/controller/relayer/l2_relayer.go index 050bec324c..fe978f742b 100644 --- a/rollup/internal/controller/relayer/l2_relayer.go +++ b/rollup/internal/controller/relayer/l2_relayer.go @@ -1678,7 +1678,7 @@ func (r *Layer2Relayer) validateSingleChunk(chunk *orm.Chunk, chunkIndex int, ba // Check chunk continuity: previous chunk's end block number + 1 should equal current chunk's start block number if prevChunk.EndBlockNumber+1 != chunk.StartBlockNumber { - return fmt.Errorf("batch %d chunk %d is not continuous with previous chunk: prev chunk %d end %d, current chunk start %d", batch.Batch.Index, chunk.Index, prevChunk.Index, prevChunk.EndBlockNumber, chunk.StartBlockNumber) + return fmt.Errorf("batch %d chunk %d is not continuous with previous chunk: prev chunk %d end block %d, current chunk start block %d", batch.Batch.Index, chunk.Index, prevChunk.Index, prevChunk.EndBlockNumber, chunk.StartBlockNumber) } return nil diff --git a/rollup/internal/controller/relayer/relayer_test.go b/rollup/internal/controller/relayer/relayer_test.go index 72ed612f40..48a65578cd 100644 --- a/rollup/internal/controller/relayer/relayer_test.go +++ b/rollup/internal/controller/relayer/relayer_test.go @@ -81,6 +81,7 @@ func setupEnv(t *testing.T) { block1 = &encoding.Block{} err = json.Unmarshal(templateBlockTrace1, block1) assert.NoError(t, err) + block1.Header.Number = big.NewInt(1) chunk1 = &encoding.Chunk{Blocks: []*encoding.Block{block1}} codec, err := encoding.CodecFromVersion(encoding.CodecV0) assert.NoError(t, err) @@ -94,6 +95,7 @@ func setupEnv(t *testing.T) { block2 = &encoding.Block{} err = json.Unmarshal(templateBlockTrace2, block2) assert.NoError(t, err) + block2.Header.Number = big.NewInt(2) chunk2 = &encoding.Chunk{Blocks: []*encoding.Block{block2}} daChunk2, err := codec.NewDAChunk(chunk2, chunk1.NumL1Messages(0)) assert.NoError(t, err) From 604e33ae7b2da98eef7356d98ae7f8f0add2720c Mon Sep 17 00:00:00 2001 From: colinlyguo Date: Wed, 30 Jul 2025 15:34:29 +0800 Subject: [PATCH 04/14] move sanity check functions to l2_relayer_sanity.go --- .../internal/controller/relayer/l2_relayer.go | 260 ----------------- .../controller/relayer/l2_relayer_sanity.go | 270 ++++++++++++++++++ 2 files changed, 270 insertions(+), 260 deletions(-) create mode 100644 rollup/internal/controller/relayer/l2_relayer_sanity.go diff --git a/rollup/internal/controller/relayer/l2_relayer.go b/rollup/internal/controller/relayer/l2_relayer.go index fe978f742b..03b8a12e72 100644 --- a/rollup/internal/controller/relayer/l2_relayer.go +++ b/rollup/internal/controller/relayer/l2_relayer.go @@ -1423,263 +1423,3 @@ func addrFromSignerConfig(config *config.SignerConfig) (common.Address, error) { return common.Address{}, fmt.Errorf("failed to determine signer address, unknown signer type: %v", config.SignerType) } } - -// sanityChecksBeforeConstructingTransaction performs sanity checks before constructing a transaction. -func (r *Layer2Relayer) sanityChecksBeforeConstructingTransaction(batchesToSubmit []*dbBatchWithChunks) error { - if len(batchesToSubmit) == 0 { - return fmt.Errorf("no batches to submit") - } - - // Basic validation - if err := r.validateBatchesBasic(batchesToSubmit); err != nil { - return err - } - - // Codec version validation - if err := r.validateCodecVersions(batchesToSubmit); err != nil { - return err - } - - // Get previous chunk for continuity check - prevChunk, err := r.getPreviousChunkForContinuity(batchesToSubmit[0]) - if err != nil { - return err - } - - // Validate each batch in detail - if err := r.validateBatchesDetailed(batchesToSubmit, prevChunk); err != nil { - return err - } - - log.Info("Sanity check passed before constructing transaction", "batches count", len(batchesToSubmit)) - return nil -} - -// validateBatchesBasic performs basic validation on all batches -func (r *Layer2Relayer) validateBatchesBasic(batchesToSubmit []*dbBatchWithChunks) error { - for i, batch := range batchesToSubmit { - if batch == nil || batch.Batch == nil { - return fmt.Errorf("batch %d is nil", i) - } - - if len(batch.Chunks) == 0 { - return fmt.Errorf("batch %d has no chunks", batch.Batch.Index) - } - } - return nil -} - -// validateCodecVersions checks all batches have the same codec version -func (r *Layer2Relayer) validateCodecVersions(batchesToSubmit []*dbBatchWithChunks) error { - firstBatchCodecVersion := batchesToSubmit[0].Batch.CodecVersion - for _, batch := range batchesToSubmit { - if batch.Batch.CodecVersion != firstBatchCodecVersion { - return fmt.Errorf("batch %d has different codec version %d, expected %d", batch.Batch.Index, batch.Batch.CodecVersion, firstBatchCodecVersion) - } - } - return nil -} - -// getPreviousChunkForContinuity gets the previous chunk for block continuity check -func (r *Layer2Relayer) getPreviousChunkForContinuity(firstBatch *dbBatchWithChunks) (*orm.Chunk, error) { - firstChunk := firstBatch.Chunks[0] - if firstChunk.Index == 0 { - return nil, fmt.Errorf("genesis chunk should not be in normal batch submission flow, chunk index: %d", firstChunk.Index) - } - - prevChunk, err := r.chunkOrm.GetChunkByIndex(r.ctx, firstChunk.Index-1) - if err != nil { - return nil, fmt.Errorf("failed to get previous chunk %d for continuity check: %w", firstChunk.Index-1, err) - } - - return prevChunk, nil -} - -// validateBatchesDetailed performs detailed validation on each batch -func (r *Layer2Relayer) validateBatchesDetailed(batchesToSubmit []*dbBatchWithChunks, prevChunkFromPrevBatch *orm.Chunk) error { - for i, batch := range batchesToSubmit { - if err := r.validateSingleBatch(batch, i, batchesToSubmit, prevChunkFromPrevBatch); err != nil { - return err - } - } - return nil -} - -// validateSingleBatch validates a single batch and its chunks -func (r *Layer2Relayer) validateSingleBatch(batch *dbBatchWithChunks, i int, allBatches []*dbBatchWithChunks, prevChunkFromPrevBatch *orm.Chunk) error { - // Validate batch fields - if err := r.validateBatchFields(batch, i, allBatches); err != nil { - return err - } - - // Validate message queue consistency - if err := r.validateMessageQueueConsistency(batch); err != nil { - return err - } - - // Validate chunks - if err := r.validateBatchChunks(batch, i, allBatches, prevChunkFromPrevBatch); err != nil { - return err - } - - return nil -} - -// validateBatchFields validates essential batch fields -func (r *Layer2Relayer) validateBatchFields(batch *dbBatchWithChunks, i int, allBatches []*dbBatchWithChunks) error { - // Check essential batch fields are not zero values - batchHash := common.HexToHash(batch.Batch.Hash) - if batchHash == (common.Hash{}) { - return fmt.Errorf("batch %d hash is zero", batch.Batch.Index) - } - - if batch.Batch.Index == 0 { - return fmt.Errorf("batch %d has zero index (only genesis batch should have index 0)", i) - } - - // Check batch index is sequential - if i > 0 { - prevBatch := allBatches[i-1] - if batch.Batch.Index != prevBatch.Batch.Index+1 { - return fmt.Errorf("batch index is not sequential: prev batch index %d, current batch index %d", prevBatch.Batch.Index, batch.Batch.Index) - } - } else { - // For the first batch, check continuity with the parent batch from database - parentBatch, err := r.batchOrm.GetBatchByHash(r.ctx, batch.Batch.ParentBatchHash) - if err != nil { - return fmt.Errorf("failed to get parent batch %s for batch %d: %w", batch.Batch.ParentBatchHash, batch.Batch.Index, err) - } - if batch.Batch.Index != parentBatch.Index+1 { - return fmt.Errorf("first batch index is not sequential with parent: parent batch index %d, current batch index %d", parentBatch.Index, batch.Batch.Index) - } - } - - parentBatchHash := common.HexToHash(batch.Batch.ParentBatchHash) - if parentBatchHash == (common.Hash{}) { - return fmt.Errorf("batch %d parent batch hash is zero", batch.Batch.Index) - } - - stateRoot := common.HexToHash(batch.Batch.StateRoot) - if stateRoot == (common.Hash{}) { - return fmt.Errorf("batch %d state root is zero", batch.Batch.Index) - } - - return nil -} - -// validateMessageQueueConsistency validates L1 message queue hash consistency -func (r *Layer2Relayer) validateMessageQueueConsistency(batch *dbBatchWithChunks) error { - if batch.Batch.Index == 0 { - return nil - } - - firstChunk := batch.Chunks[0] - lastChunk := batch.Chunks[len(batch.Chunks)-1] - - prevL1MsgQueueHash := common.HexToHash(batch.Batch.PrevL1MessageQueueHash) - postL1MsgQueueHash := common.HexToHash(batch.Batch.PostL1MessageQueueHash) - - // Calculate total L1 messages in this batch - var batchTotalL1MessagesInBatch uint64 - for _, chunk := range batch.Chunks { - batchTotalL1MessagesInBatch += chunk.TotalL1MessagesPoppedInChunk - } - - // If there were L1 messages processed before this batch, prev hash should not be zero - if firstChunk.TotalL1MessagesPoppedBefore > 0 && prevL1MsgQueueHash == (common.Hash{}) { - return fmt.Errorf("batch %d prev L1 message queue hash is zero but %d L1 messages were processed before", batch.Batch.Index, firstChunk.TotalL1MessagesPoppedBefore) - } - - // If there are any L1 messages processed up to this batch, post hash should not be zero - totalL1MessagesProcessed := lastChunk.TotalL1MessagesPoppedBefore + lastChunk.TotalL1MessagesPoppedInChunk - if totalL1MessagesProcessed > 0 && postL1MsgQueueHash == (common.Hash{}) { - return fmt.Errorf("batch %d post L1 message queue hash is zero but %d L1 messages were processed in total", batch.Batch.Index, totalL1MessagesProcessed) - } - - // Prev and post queue hashes should be different if L1 messages were processed in this batch - if batchTotalL1MessagesInBatch > 0 && prevL1MsgQueueHash == postL1MsgQueueHash { - return fmt.Errorf("batch %d has same prev and post L1 message queue hashes but processed %d L1 messages in this batch", batch.Batch.Index, batchTotalL1MessagesInBatch) - } - - return nil -} - -// validateBatchChunks validates all chunks in a batch -func (r *Layer2Relayer) validateBatchChunks(batch *dbBatchWithChunks, i int, allBatches []*dbBatchWithChunks, prevChunkFromPrevBatch *orm.Chunk) error { - // Check all chunks in this batch have the same codec version as the batch - for _, chunk := range batch.Chunks { - if chunk.CodecVersion != batch.Batch.CodecVersion { - return fmt.Errorf("batch %d chunk %d has different codec version %d, expected %d", batch.Batch.Index, chunk.Index, chunk.CodecVersion, batch.Batch.CodecVersion) - } - } - - for j, chunk := range batch.Chunks { - if err := r.validateSingleChunk(chunk, j, batch, i, allBatches, prevChunkFromPrevBatch); err != nil { - return err - } - } - - return nil -} - -// validateSingleChunk validates a single chunk -func (r *Layer2Relayer) validateSingleChunk(chunk *orm.Chunk, chunkIndex int, batch *dbBatchWithChunks, i int, allBatches []*dbBatchWithChunks, prevChunkFromPrevBatch *orm.Chunk) error { - if chunk == nil { - return fmt.Errorf("batch %d chunk %d is nil", batch.Batch.Index, chunkIndex) - } - - chunkHash := common.HexToHash(chunk.Hash) - if chunkHash == (common.Hash{}) { - return fmt.Errorf("batch %d chunk %d hash is zero", batch.Batch.Index, chunk.Index) - } - - // Get previous chunk for continuity check - var prevChunk *orm.Chunk - if chunkIndex > 0 { - prevChunk = batch.Chunks[chunkIndex-1] - } else if i == 0 { - prevChunk = prevChunkFromPrevBatch - } else if i > 0 { - // Use the last chunk from the previous batch - prevBatch := allBatches[i-1] - prevChunk = prevBatch.Chunks[len(prevBatch.Chunks)-1] - } - - // Check chunk index is sequential - if chunk.Index != prevChunk.Index+1 { - return fmt.Errorf("batch %d chunk %d index is not sequential: prev chunk index %d, current chunk index %d", batch.Batch.Index, chunkIndex, prevChunk.Index, chunk.Index) - } - - // Check L1 messages popped continuity - expectedPoppedBefore := prevChunk.TotalL1MessagesPoppedBefore + prevChunk.TotalL1MessagesPoppedInChunk - if chunk.TotalL1MessagesPoppedBefore != expectedPoppedBefore { - return fmt.Errorf("batch %d chunk %d L1 messages popped before is incorrect: expected %d, got %d", - batch.Batch.Index, chunk.Index, expectedPoppedBefore, chunk.TotalL1MessagesPoppedBefore) - } - - if chunk.StartBlockNumber == 0 && chunk.EndBlockNumber == 0 { - return fmt.Errorf("batch %d chunk %d has zero block range", batch.Batch.Index, chunk.Index) - } - - if chunk.StartBlockNumber > chunk.EndBlockNumber { - return fmt.Errorf("batch %d chunk %d has invalid block range: start %d > end %d", batch.Batch.Index, chunk.Index, chunk.StartBlockNumber, chunk.EndBlockNumber) - } - - // Check chunk hash fields - startBlockHash := common.HexToHash(chunk.StartBlockHash) - if startBlockHash == (common.Hash{}) { - return fmt.Errorf("batch %d chunk %d start block hash is zero", batch.Batch.Index, chunk.Index) - } - - endBlockHash := common.HexToHash(chunk.EndBlockHash) - if endBlockHash == (common.Hash{}) { - return fmt.Errorf("batch %d chunk %d end block hash is zero", batch.Batch.Index, chunk.Index) - } - - // Check chunk continuity: previous chunk's end block number + 1 should equal current chunk's start block number - if prevChunk.EndBlockNumber+1 != chunk.StartBlockNumber { - return fmt.Errorf("batch %d chunk %d is not continuous with previous chunk: prev chunk %d end block %d, current chunk start block %d", batch.Batch.Index, chunk.Index, prevChunk.Index, prevChunk.EndBlockNumber, chunk.StartBlockNumber) - } - - return nil -} diff --git a/rollup/internal/controller/relayer/l2_relayer_sanity.go b/rollup/internal/controller/relayer/l2_relayer_sanity.go new file mode 100644 index 0000000000..110b583865 --- /dev/null +++ b/rollup/internal/controller/relayer/l2_relayer_sanity.go @@ -0,0 +1,270 @@ +package relayer + +import ( + "fmt" + + "github.com/scroll-tech/go-ethereum/common" + "github.com/scroll-tech/go-ethereum/log" + + "scroll-tech/rollup/internal/orm" +) + +// validateMessageQueueConsistency validates L1 message queue hash consistency +func (r *Layer2Relayer) validateMessageQueueConsistency(batch *dbBatchWithChunks) error { + if batch.Batch.Index == 0 { + return nil + } + + firstChunk := batch.Chunks[0] + lastChunk := batch.Chunks[len(batch.Chunks)-1] + + prevL1MsgQueueHash := common.HexToHash(batch.Batch.PrevL1MessageQueueHash) + postL1MsgQueueHash := common.HexToHash(batch.Batch.PostL1MessageQueueHash) + + // Calculate total L1 messages in this batch + var batchTotalL1MessagesInBatch uint64 + for _, chunk := range batch.Chunks { + batchTotalL1MessagesInBatch += chunk.TotalL1MessagesPoppedInChunk + } + + // If there were L1 messages processed before this batch, prev hash should not be zero + if firstChunk.TotalL1MessagesPoppedBefore > 0 && prevL1MsgQueueHash == (common.Hash{}) { + return fmt.Errorf("batch %d prev L1 message queue hash is zero but %d L1 messages were processed before", batch.Batch.Index, firstChunk.TotalL1MessagesPoppedBefore) + } + + // If there are any L1 messages processed up to this batch, post hash should not be zero + totalL1MessagesProcessed := lastChunk.TotalL1MessagesPoppedBefore + lastChunk.TotalL1MessagesPoppedInChunk + if totalL1MessagesProcessed > 0 && postL1MsgQueueHash == (common.Hash{}) { + return fmt.Errorf("batch %d post L1 message queue hash is zero but %d L1 messages were processed in total", batch.Batch.Index, totalL1MessagesProcessed) + } + + // Prev and post queue hashes should be different if L1 messages were processed in this batch + if batchTotalL1MessagesInBatch > 0 && prevL1MsgQueueHash == postL1MsgQueueHash { + return fmt.Errorf("batch %d has same prev and post L1 message queue hashes but processed %d L1 messages in this batch", batch.Batch.Index, batchTotalL1MessagesInBatch) + } + + return nil +} + +// sanityChecksBeforeConstructingTransaction performs sanity checks before constructing a transaction. +func (r *Layer2Relayer) sanityChecksBeforeConstructingTransaction(batchesToSubmit []*dbBatchWithChunks) error { + if len(batchesToSubmit) == 0 { + return fmt.Errorf("no batches to submit") + } + + // Basic validation + if err := r.validateBatchesBasic(batchesToSubmit); err != nil { + return err + } + + // Codec version validation + if err := r.validateCodecVersions(batchesToSubmit); err != nil { + return err + } + + // Get previous chunk for continuity check + prevChunk, err := r.getPreviousChunkForContinuity(batchesToSubmit[0]) + if err != nil { + return err + } + + // Validate each batch in detail + if err := r.validateBatchesDetailed(batchesToSubmit, prevChunk); err != nil { + return err + } + + log.Info("Sanity check passed before constructing transaction", "batches count", len(batchesToSubmit)) + return nil +} + +// validateBatchesBasic performs basic validation on all batches +func (r *Layer2Relayer) validateBatchesBasic(batchesToSubmit []*dbBatchWithChunks) error { + for i, batch := range batchesToSubmit { + if batch == nil || batch.Batch == nil { + return fmt.Errorf("batch %d is nil", i) + } + + if len(batch.Chunks) == 0 { + return fmt.Errorf("batch %d has no chunks", batch.Batch.Index) + } + } + return nil +} + +// validateCodecVersions checks all batches have the same codec version +func (r *Layer2Relayer) validateCodecVersions(batchesToSubmit []*dbBatchWithChunks) error { + firstBatchCodecVersion := batchesToSubmit[0].Batch.CodecVersion + for _, batch := range batchesToSubmit { + if batch.Batch.CodecVersion != firstBatchCodecVersion { + return fmt.Errorf("batch %d has different codec version %d, expected %d", batch.Batch.Index, batch.Batch.CodecVersion, firstBatchCodecVersion) + } + } + return nil +} + +// getPreviousChunkForContinuity gets the previous chunk for block continuity check +func (r *Layer2Relayer) getPreviousChunkForContinuity(firstBatch *dbBatchWithChunks) (*orm.Chunk, error) { + firstChunk := firstBatch.Chunks[0] + if firstChunk.Index == 0 { + return nil, fmt.Errorf("genesis chunk should not be in normal batch submission flow, chunk index: %d", firstChunk.Index) + } + + prevChunk, err := r.chunkOrm.GetChunkByIndex(r.ctx, firstChunk.Index-1) + if err != nil { + return nil, fmt.Errorf("failed to get previous chunk %d for continuity check: %w", firstChunk.Index-1, err) + } + + return prevChunk, nil +} + +// validateBatchesDetailed performs detailed validation on each batch +func (r *Layer2Relayer) validateBatchesDetailed(batchesToSubmit []*dbBatchWithChunks, prevChunkFromPrevBatch *orm.Chunk) error { + for i, batch := range batchesToSubmit { + if err := r.validateSingleBatch(batch, i, batchesToSubmit, prevChunkFromPrevBatch); err != nil { + return err + } + } + return nil +} + +// validateSingleBatch validates a single batch and its chunks +func (r *Layer2Relayer) validateSingleBatch(batch *dbBatchWithChunks, i int, allBatches []*dbBatchWithChunks, prevChunkFromPrevBatch *orm.Chunk) error { + // Validate batch fields + if err := r.validateBatchFields(batch, i, allBatches); err != nil { + return err + } + + // Validate message queue consistency + if err := r.validateMessageQueueConsistency(batch); err != nil { + return err + } + + // Validate chunks + if err := r.validateBatchChunks(batch, i, allBatches, prevChunkFromPrevBatch); err != nil { + return err + } + + return nil +} + +// validateBatchFields validates essential batch fields +func (r *Layer2Relayer) validateBatchFields(batch *dbBatchWithChunks, i int, allBatches []*dbBatchWithChunks) error { + // Check essential batch fields are not zero values + batchHash := common.HexToHash(batch.Batch.Hash) + if batchHash == (common.Hash{}) { + return fmt.Errorf("batch %d hash is zero", batch.Batch.Index) + } + + if batch.Batch.Index == 0 { + return fmt.Errorf("batch %d has zero index (only genesis batch should have index 0)", i) + } + + // Check batch index is sequential + if i > 0 { + prevBatch := allBatches[i-1] + if batch.Batch.Index != prevBatch.Batch.Index+1 { + return fmt.Errorf("batch index is not sequential: prev batch index %d, current batch index %d", prevBatch.Batch.Index, batch.Batch.Index) + } + } else { + // For the first batch, check continuity with the parent batch from database + parentBatch, err := r.batchOrm.GetBatchByHash(r.ctx, batch.Batch.ParentBatchHash) + if err != nil { + return fmt.Errorf("failed to get parent batch %s for batch %d: %w", batch.Batch.ParentBatchHash, batch.Batch.Index, err) + } + if batch.Batch.Index != parentBatch.Index+1 { + return fmt.Errorf("first batch index is not sequential with parent: parent batch index %d, current batch index %d", parentBatch.Index, batch.Batch.Index) + } + } + + parentBatchHash := common.HexToHash(batch.Batch.ParentBatchHash) + if parentBatchHash == (common.Hash{}) { + return fmt.Errorf("batch %d parent batch hash is zero", batch.Batch.Index) + } + + stateRoot := common.HexToHash(batch.Batch.StateRoot) + if stateRoot == (common.Hash{}) { + return fmt.Errorf("batch %d state root is zero", batch.Batch.Index) + } + + return nil +} + +// validateBatchChunks validates all chunks in a batch +func (r *Layer2Relayer) validateBatchChunks(batch *dbBatchWithChunks, i int, allBatches []*dbBatchWithChunks, prevChunkFromPrevBatch *orm.Chunk) error { + // Check all chunks in this batch have the same codec version as the batch + for _, chunk := range batch.Chunks { + if chunk.CodecVersion != batch.Batch.CodecVersion { + return fmt.Errorf("batch %d chunk %d has different codec version %d, expected %d", batch.Batch.Index, chunk.Index, chunk.CodecVersion, batch.Batch.CodecVersion) + } + } + + for j, chunk := range batch.Chunks { + if err := r.validateSingleChunk(chunk, j, batch, i, allBatches, prevChunkFromPrevBatch); err != nil { + return err + } + } + + return nil +} + +// validateSingleChunk validates a single chunk +func (r *Layer2Relayer) validateSingleChunk(chunk *orm.Chunk, chunkIndex int, batch *dbBatchWithChunks, i int, allBatches []*dbBatchWithChunks, prevChunkFromPrevBatch *orm.Chunk) error { + if chunk == nil { + return fmt.Errorf("batch %d chunk %d is nil", batch.Batch.Index, chunkIndex) + } + + chunkHash := common.HexToHash(chunk.Hash) + if chunkHash == (common.Hash{}) { + return fmt.Errorf("batch %d chunk %d hash is zero", batch.Batch.Index, chunk.Index) + } + + // Get previous chunk for continuity check + var prevChunk *orm.Chunk + if chunkIndex > 0 { + prevChunk = batch.Chunks[chunkIndex-1] + } else if i == 0 { + prevChunk = prevChunkFromPrevBatch + } else if i > 0 { + // Use the last chunk from the previous batch + prevBatch := allBatches[i-1] + prevChunk = prevBatch.Chunks[len(prevBatch.Chunks)-1] + } + + // Check chunk index is sequential + if chunk.Index != prevChunk.Index+1 { + return fmt.Errorf("batch %d chunk %d index is not sequential: prev chunk index %d, current chunk index %d", batch.Batch.Index, chunkIndex, prevChunk.Index, chunk.Index) + } + + // Check L1 messages popped continuity + expectedPoppedBefore := prevChunk.TotalL1MessagesPoppedBefore + prevChunk.TotalL1MessagesPoppedInChunk + if chunk.TotalL1MessagesPoppedBefore != expectedPoppedBefore { + return fmt.Errorf("batch %d chunk %d L1 messages popped before is incorrect: expected %d, got %d", + batch.Batch.Index, chunk.Index, expectedPoppedBefore, chunk.TotalL1MessagesPoppedBefore) + } + + if chunk.StartBlockNumber == 0 && chunk.EndBlockNumber == 0 { + return fmt.Errorf("batch %d chunk %d has zero block range", batch.Batch.Index, chunk.Index) + } + + if chunk.StartBlockNumber > chunk.EndBlockNumber { + return fmt.Errorf("batch %d chunk %d has invalid block range: start %d > end %d", batch.Batch.Index, chunk.Index, chunk.StartBlockNumber, chunk.EndBlockNumber) + } + + // Check chunk hash fields + startBlockHash := common.HexToHash(chunk.StartBlockHash) + if startBlockHash == (common.Hash{}) { + return fmt.Errorf("batch %d chunk %d start block hash is zero", batch.Batch.Index, chunk.Index) + } + + endBlockHash := common.HexToHash(chunk.EndBlockHash) + if endBlockHash == (common.Hash{}) { + return fmt.Errorf("batch %d chunk %d end block hash is zero", batch.Batch.Index, chunk.Index) + } + + // Check chunk continuity: previous chunk's end block number + 1 should equal current chunk's start block number + if prevChunk.EndBlockNumber+1 != chunk.StartBlockNumber { + return fmt.Errorf("batch %d chunk %d is not continuous with previous chunk: prev chunk %d end block %d, current chunk start block %d", batch.Batch.Index, chunk.Index, prevChunk.Index, prevChunk.EndBlockNumber, chunk.StartBlockNumber) + } + + return nil +} From c67888c0bb860286d60a09d619481ae4849cf638 Mon Sep 17 00:00:00 2001 From: colinlyguo Date: Wed, 30 Jul 2025 16:18:02 +0800 Subject: [PATCH 05/14] address comments --- .../internal/controller/relayer/l2_relayer.go | 25 ++-------------- .../controller/relayer/l2_relayer_sanity.go | 29 ++++++++++--------- 2 files changed, 17 insertions(+), 37 deletions(-) diff --git a/rollup/internal/controller/relayer/l2_relayer.go b/rollup/internal/controller/relayer/l2_relayer.go index 03b8a12e72..d5806a409e 100644 --- a/rollup/internal/controller/relayer/l2_relayer.go +++ b/rollup/internal/controller/relayer/l2_relayer.go @@ -1036,29 +1036,8 @@ func (r *Layer2Relayer) constructCommitBatchPayloadCodecV7(batchesToSubmit []*db } // Check L1 message queue hash consistency - var totalL1MessagesInBatch uint64 - for _, c := range b.Chunks { - totalL1MessagesInBatch += c.TotalL1MessagesPoppedInChunk - } - - // Check L1 message queue hash consistency - firstChunk := b.Chunks[0] - lastChunk := b.Chunks[len(b.Chunks)-1] - - // If there were L1 messages processed before this batch, prev hash should not be zero - if firstChunk.TotalL1MessagesPoppedBefore > 0 && encodingBatch.PrevL1MessageQueueHash == (common.Hash{}) { - return nil, nil, 0, 0, fmt.Errorf("batch %d prev L1 message queue hash is zero but %d L1 messages were processed before", b.Batch.Index, firstChunk.TotalL1MessagesPoppedBefore) - } - - // If there are any L1 messages processed up to this batch, post hash should not be zero - totalL1MessagesProcessed := lastChunk.TotalL1MessagesPoppedBefore + lastChunk.TotalL1MessagesPoppedInChunk - if totalL1MessagesProcessed > 0 && encodingBatch.PostL1MessageQueueHash == (common.Hash{}) { - return nil, nil, 0, 0, fmt.Errorf("batch %d post L1 message queue hash is zero but %d L1 messages were processed in total", b.Batch.Index, totalL1MessagesProcessed) - } - - // If L1 messages were processed in this batch, prev and post hashes should be different - if totalL1MessagesInBatch > 0 && encodingBatch.PrevL1MessageQueueHash == encodingBatch.PostL1MessageQueueHash { - return nil, nil, 0, 0, fmt.Errorf("batch %d has same prev and post L1 message queue hashes but processed %d L1 messages in this batch", b.Batch.Index, totalL1MessagesInBatch) + if err := r.validateMessageQueueConsistency(encodingBatch.Index, b.Chunks, encodingBatch.PrevL1MessageQueueHash, encodingBatch.PostL1MessageQueueHash); err != nil { + return nil, nil, 0, 0, err } codec, err := encoding.CodecFromVersion(version) diff --git a/rollup/internal/controller/relayer/l2_relayer_sanity.go b/rollup/internal/controller/relayer/l2_relayer_sanity.go index 110b583865..1fdf76eb8b 100644 --- a/rollup/internal/controller/relayer/l2_relayer_sanity.go +++ b/rollup/internal/controller/relayer/l2_relayer_sanity.go @@ -10,37 +10,38 @@ import ( ) // validateMessageQueueConsistency validates L1 message queue hash consistency -func (r *Layer2Relayer) validateMessageQueueConsistency(batch *dbBatchWithChunks) error { - if batch.Batch.Index == 0 { +func (r *Layer2Relayer) validateMessageQueueConsistency(batchIndex uint64, chunks []*orm.Chunk, prevL1MsgQueueHash common.Hash, postL1MsgQueueHash common.Hash) error { + if batchIndex == 0 { return nil } - firstChunk := batch.Chunks[0] - lastChunk := batch.Chunks[len(batch.Chunks)-1] + if len(chunks) == 0 { + return fmt.Errorf("batch %d has no chunks for message queue validation", batchIndex) + } - prevL1MsgQueueHash := common.HexToHash(batch.Batch.PrevL1MessageQueueHash) - postL1MsgQueueHash := common.HexToHash(batch.Batch.PostL1MessageQueueHash) + firstChunk := chunks[0] + lastChunk := chunks[len(chunks)-1] // Calculate total L1 messages in this batch - var batchTotalL1MessagesInBatch uint64 - for _, chunk := range batch.Chunks { - batchTotalL1MessagesInBatch += chunk.TotalL1MessagesPoppedInChunk + var totalL1MessagesInBatch uint64 + for _, chunk := range chunks { + totalL1MessagesInBatch += chunk.TotalL1MessagesPoppedInChunk } // If there were L1 messages processed before this batch, prev hash should not be zero if firstChunk.TotalL1MessagesPoppedBefore > 0 && prevL1MsgQueueHash == (common.Hash{}) { - return fmt.Errorf("batch %d prev L1 message queue hash is zero but %d L1 messages were processed before", batch.Batch.Index, firstChunk.TotalL1MessagesPoppedBefore) + return fmt.Errorf("batch %d prev L1 message queue hash is zero but %d L1 messages were processed before", batchIndex, firstChunk.TotalL1MessagesPoppedBefore) } // If there are any L1 messages processed up to this batch, post hash should not be zero totalL1MessagesProcessed := lastChunk.TotalL1MessagesPoppedBefore + lastChunk.TotalL1MessagesPoppedInChunk if totalL1MessagesProcessed > 0 && postL1MsgQueueHash == (common.Hash{}) { - return fmt.Errorf("batch %d post L1 message queue hash is zero but %d L1 messages were processed in total", batch.Batch.Index, totalL1MessagesProcessed) + return fmt.Errorf("batch %d post L1 message queue hash is zero but %d L1 messages were processed in total", batchIndex, totalL1MessagesProcessed) } // Prev and post queue hashes should be different if L1 messages were processed in this batch - if batchTotalL1MessagesInBatch > 0 && prevL1MsgQueueHash == postL1MsgQueueHash { - return fmt.Errorf("batch %d has same prev and post L1 message queue hashes but processed %d L1 messages in this batch", batch.Batch.Index, batchTotalL1MessagesInBatch) + if totalL1MessagesInBatch > 0 && prevL1MsgQueueHash == postL1MsgQueueHash { + return fmt.Errorf("batch %d has same prev and post L1 message queue hashes but processed %d L1 messages in this batch", batchIndex, totalL1MessagesInBatch) } return nil @@ -135,7 +136,7 @@ func (r *Layer2Relayer) validateSingleBatch(batch *dbBatchWithChunks, i int, all } // Validate message queue consistency - if err := r.validateMessageQueueConsistency(batch); err != nil { + if err := r.validateMessageQueueConsistency(batch.Batch.Index, batch.Chunks, common.HexToHash(batch.Batch.PrevL1MessageQueueHash), common.HexToHash(batch.Batch.PostL1MessageQueueHash)); err != nil { return err } From 1b28ff5b5ae43357cd24d27f0eb23a7d4eaefa9a Mon Sep 17 00:00:00 2001 From: colinlyguo Date: Mon, 4 Aug 2025 14:02:09 +0800 Subject: [PATCH 06/14] address comments --- .../controller/relayer/l2_relayer_sanity.go | 112 +++++++----------- 1 file changed, 44 insertions(+), 68 deletions(-) diff --git a/rollup/internal/controller/relayer/l2_relayer_sanity.go b/rollup/internal/controller/relayer/l2_relayer_sanity.go index 1fdf76eb8b..70d8f39692 100644 --- a/rollup/internal/controller/relayer/l2_relayer_sanity.go +++ b/rollup/internal/controller/relayer/l2_relayer_sanity.go @@ -53,24 +53,14 @@ func (r *Layer2Relayer) sanityChecksBeforeConstructingTransaction(batchesToSubmi return fmt.Errorf("no batches to submit") } - // Basic validation - if err := r.validateBatchesBasic(batchesToSubmit); err != nil { - return err - } - - // Codec version validation - if err := r.validateCodecVersions(batchesToSubmit); err != nil { - return err - } - // Get previous chunk for continuity check prevChunk, err := r.getPreviousChunkForContinuity(batchesToSubmit[0]) if err != nil { return err } - // Validate each batch in detail - if err := r.validateBatchesDetailed(batchesToSubmit, prevChunk); err != nil { + // Validate batches (including basic, codec versions, and detailed checks) + if err := r.validateBatches(batchesToSubmit, prevChunk); err != nil { return err } @@ -78,58 +68,55 @@ func (r *Layer2Relayer) sanityChecksBeforeConstructingTransaction(batchesToSubmi return nil } -// validateBatchesBasic performs basic validation on all batches -func (r *Layer2Relayer) validateBatchesBasic(batchesToSubmit []*dbBatchWithChunks) error { +// getPreviousChunkForContinuity gets the previous chunk for block continuity check +func (r *Layer2Relayer) getPreviousChunkForContinuity(firstBatch *dbBatchWithChunks) (*orm.Chunk, error) { + firstChunk := firstBatch.Chunks[0] + if firstChunk.Index == 0 { + return nil, fmt.Errorf("genesis chunk should not be in normal batch submission flow, chunk index: %d", firstChunk.Index) + } + + prevChunk, err := r.chunkOrm.GetChunkByIndex(r.ctx, firstChunk.Index-1) + if err != nil { + return nil, fmt.Errorf("failed to get previous chunk %d for continuity check: %w", firstChunk.Index-1, err) + } + + return prevChunk, nil +} + +// validateBatches performs validation on all batches including basic checks, codec version consistency, and detailed checks. +func (r *Layer2Relayer) validateBatches(batchesToSubmit []*dbBatchWithChunks, initialPrevChunk *orm.Chunk) error { + // Basic validation: ensure each batch and its chunks are non-empty. for i, batch := range batchesToSubmit { if batch == nil || batch.Batch == nil { return fmt.Errorf("batch %d is nil", i) } - if len(batch.Chunks) == 0 { return fmt.Errorf("batch %d has no chunks", batch.Batch.Index) } } - return nil -} -// validateCodecVersions checks all batches have the same codec version -func (r *Layer2Relayer) validateCodecVersions(batchesToSubmit []*dbBatchWithChunks) error { + // Check that all batches have the same codec version. firstBatchCodecVersion := batchesToSubmit[0].Batch.CodecVersion for _, batch := range batchesToSubmit { if batch.Batch.CodecVersion != firstBatchCodecVersion { return fmt.Errorf("batch %d has different codec version %d, expected %d", batch.Batch.Index, batch.Batch.CodecVersion, firstBatchCodecVersion) } } - return nil -} - -// getPreviousChunkForContinuity gets the previous chunk for block continuity check -func (r *Layer2Relayer) getPreviousChunkForContinuity(firstBatch *dbBatchWithChunks) (*orm.Chunk, error) { - firstChunk := firstBatch.Chunks[0] - if firstChunk.Index == 0 { - return nil, fmt.Errorf("genesis chunk should not be in normal batch submission flow, chunk index: %d", firstChunk.Index) - } - prevChunk, err := r.chunkOrm.GetChunkByIndex(r.ctx, firstChunk.Index-1) - if err != nil { - return nil, fmt.Errorf("failed to get previous chunk %d for continuity check: %w", firstChunk.Index-1, err) - } - - return prevChunk, nil -} - -// validateBatchesDetailed performs detailed validation on each batch -func (r *Layer2Relayer) validateBatchesDetailed(batchesToSubmit []*dbBatchWithChunks, prevChunkFromPrevBatch *orm.Chunk) error { + // Validate each batch in detail, updating the previous chunk as we go. + currentPrevChunk := initialPrevChunk for i, batch := range batchesToSubmit { - if err := r.validateSingleBatch(batch, i, batchesToSubmit, prevChunkFromPrevBatch); err != nil { + if err := r.validateSingleBatch(batch, i, batchesToSubmit, currentPrevChunk); err != nil { return err } + // Update the previous chunk to the last chunk of this batch for the next batch. + currentPrevChunk = batch.Chunks[len(batch.Chunks)-1] } return nil } // validateSingleBatch validates a single batch and its chunks -func (r *Layer2Relayer) validateSingleBatch(batch *dbBatchWithChunks, i int, allBatches []*dbBatchWithChunks, prevChunkFromPrevBatch *orm.Chunk) error { +func (r *Layer2Relayer) validateSingleBatch(batch *dbBatchWithChunks, i int, allBatches []*dbBatchWithChunks, prevChunk *orm.Chunk) error { // Validate batch fields if err := r.validateBatchFields(batch, i, allBatches); err != nil { return err @@ -141,7 +128,7 @@ func (r *Layer2Relayer) validateSingleBatch(batch *dbBatchWithChunks, i int, all } // Validate chunks - if err := r.validateBatchChunks(batch, i, allBatches, prevChunkFromPrevBatch); err != nil { + if err := r.validateBatchChunks(batch, prevChunk); err != nil { return err } @@ -191,8 +178,8 @@ func (r *Layer2Relayer) validateBatchFields(batch *dbBatchWithChunks, i int, all } // validateBatchChunks validates all chunks in a batch -func (r *Layer2Relayer) validateBatchChunks(batch *dbBatchWithChunks, i int, allBatches []*dbBatchWithChunks, prevChunkFromPrevBatch *orm.Chunk) error { - // Check all chunks in this batch have the same codec version as the batch +func (r *Layer2Relayer) validateBatchChunks(batch *dbBatchWithChunks, prevChunk *orm.Chunk) error { + // Check codec version consistency. for _, chunk := range batch.Chunks { if chunk.CodecVersion != batch.Batch.CodecVersion { return fmt.Errorf("batch %d chunk %d has different codec version %d, expected %d", batch.Batch.Index, chunk.Index, chunk.CodecVersion, batch.Batch.CodecVersion) @@ -200,71 +187,60 @@ func (r *Layer2Relayer) validateBatchChunks(batch *dbBatchWithChunks, i int, all } for j, chunk := range batch.Chunks { - if err := r.validateSingleChunk(chunk, j, batch, i, allBatches, prevChunkFromPrevBatch); err != nil { - return err + if err := r.validateSingleChunk(chunk, prevChunk); err != nil { + return fmt.Errorf("batch %d chunk %d: %w", batch.Batch.Index, j, err) } + // Update the previous chunk to the current one for the next chunk. + prevChunk = chunk } return nil } // validateSingleChunk validates a single chunk -func (r *Layer2Relayer) validateSingleChunk(chunk *orm.Chunk, chunkIndex int, batch *dbBatchWithChunks, i int, allBatches []*dbBatchWithChunks, prevChunkFromPrevBatch *orm.Chunk) error { +func (r *Layer2Relayer) validateSingleChunk(chunk *orm.Chunk, prevChunk *orm.Chunk) error { if chunk == nil { - return fmt.Errorf("batch %d chunk %d is nil", batch.Batch.Index, chunkIndex) + return fmt.Errorf("chunk is nil") } chunkHash := common.HexToHash(chunk.Hash) if chunkHash == (common.Hash{}) { - return fmt.Errorf("batch %d chunk %d hash is zero", batch.Batch.Index, chunk.Index) - } - - // Get previous chunk for continuity check - var prevChunk *orm.Chunk - if chunkIndex > 0 { - prevChunk = batch.Chunks[chunkIndex-1] - } else if i == 0 { - prevChunk = prevChunkFromPrevBatch - } else if i > 0 { - // Use the last chunk from the previous batch - prevBatch := allBatches[i-1] - prevChunk = prevBatch.Chunks[len(prevBatch.Chunks)-1] + return fmt.Errorf("chunk %d hash is zero", chunk.Index) } // Check chunk index is sequential if chunk.Index != prevChunk.Index+1 { - return fmt.Errorf("batch %d chunk %d index is not sequential: prev chunk index %d, current chunk index %d", batch.Batch.Index, chunkIndex, prevChunk.Index, chunk.Index) + return fmt.Errorf("chunk index is not sequential: prev chunk index %d, current chunk index %d", prevChunk.Index, chunk.Index) } // Check L1 messages popped continuity expectedPoppedBefore := prevChunk.TotalL1MessagesPoppedBefore + prevChunk.TotalL1MessagesPoppedInChunk if chunk.TotalL1MessagesPoppedBefore != expectedPoppedBefore { - return fmt.Errorf("batch %d chunk %d L1 messages popped before is incorrect: expected %d, got %d", - batch.Batch.Index, chunk.Index, expectedPoppedBefore, chunk.TotalL1MessagesPoppedBefore) + return fmt.Errorf("L1 messages popped before is incorrect: expected %d, got %d", expectedPoppedBefore, chunk.TotalL1MessagesPoppedBefore) } if chunk.StartBlockNumber == 0 && chunk.EndBlockNumber == 0 { - return fmt.Errorf("batch %d chunk %d has zero block range", batch.Batch.Index, chunk.Index) + return fmt.Errorf("chunk %d has zero block range", chunk.Index) } if chunk.StartBlockNumber > chunk.EndBlockNumber { - return fmt.Errorf("batch %d chunk %d has invalid block range: start %d > end %d", batch.Batch.Index, chunk.Index, chunk.StartBlockNumber, chunk.EndBlockNumber) + return fmt.Errorf("chunk %d has invalid block range: start %d > end %d", chunk.Index, chunk.StartBlockNumber, chunk.EndBlockNumber) } // Check chunk hash fields startBlockHash := common.HexToHash(chunk.StartBlockHash) if startBlockHash == (common.Hash{}) { - return fmt.Errorf("batch %d chunk %d start block hash is zero", batch.Batch.Index, chunk.Index) + return fmt.Errorf("chunk %d start block hash is zero", chunk.Index) } endBlockHash := common.HexToHash(chunk.EndBlockHash) if endBlockHash == (common.Hash{}) { - return fmt.Errorf("batch %d chunk %d end block hash is zero", batch.Batch.Index, chunk.Index) + return fmt.Errorf("chunk %d end block hash is zero", chunk.Index) } // Check chunk continuity: previous chunk's end block number + 1 should equal current chunk's start block number if prevChunk.EndBlockNumber+1 != chunk.StartBlockNumber { - return fmt.Errorf("batch %d chunk %d is not continuous with previous chunk: prev chunk %d end block %d, current chunk start block %d", batch.Batch.Index, chunk.Index, prevChunk.Index, prevChunk.EndBlockNumber, chunk.StartBlockNumber) + return fmt.Errorf("chunk is not continuous with previous chunk %d: prev end block %d, current start block %d", prevChunk.Index, prevChunk.EndBlockNumber, chunk.StartBlockNumber) } return nil From 925ef74867c4a2f3beca276aa3b0c6c834ff4784 Mon Sep 17 00:00:00 2001 From: colinlyguo Date: Mon, 4 Aug 2025 14:56:15 +0800 Subject: [PATCH 07/14] add sanity checks in calldata and blob --- .../internal/controller/relayer/l2_relayer.go | 6 + .../controller/relayer/l2_relayer_sanity.go | 103 ++++++++++++++++++ 2 files changed, 109 insertions(+) diff --git a/rollup/internal/controller/relayer/l2_relayer.go b/rollup/internal/controller/relayer/l2_relayer.go index 1cf8f1f1a0..d0b5f8e02d 100644 --- a/rollup/internal/controller/relayer/l2_relayer.go +++ b/rollup/internal/controller/relayer/l2_relayer.go @@ -526,6 +526,12 @@ func (r *Layer2Relayer) ProcessPendingBatches() { log.Error("failed to construct normal payload", "codecVersion", codecVersion, "start index", firstBatch.Index, "end index", lastBatch.Index, "err", err) return } + + err = r.sanityChecksCommitBatchCodecV7CalldataAndBlobs(calldata, blobs, batchesToSubmit, firstBatch, lastBatch) + if err != nil { + log.Error("Sanity check failed for calldata and blobs", "err", err) + return + } } default: log.Error("unsupported codec version in ProcessPendingBatches", "codecVersion", codecVersion, "start index", firstBatch, "end index", lastBatch.Index) diff --git a/rollup/internal/controller/relayer/l2_relayer_sanity.go b/rollup/internal/controller/relayer/l2_relayer_sanity.go index 70d8f39692..e2d9d982d4 100644 --- a/rollup/internal/controller/relayer/l2_relayer_sanity.go +++ b/rollup/internal/controller/relayer/l2_relayer_sanity.go @@ -3,7 +3,9 @@ package relayer import ( "fmt" + "github.com/scroll-tech/da-codec/encoding" "github.com/scroll-tech/go-ethereum/common" + "github.com/scroll-tech/go-ethereum/crypto/kzg4844" "github.com/scroll-tech/go-ethereum/log" "scroll-tech/rollup/internal/orm" @@ -245,3 +247,104 @@ func (r *Layer2Relayer) validateSingleChunk(chunk *orm.Chunk, prevChunk *orm.Chu return nil } + +func (r *Layer2Relayer) sanityChecksCommitBatchCodecV7CalldataAndBlobs(calldata []byte, blobs []*kzg4844.Blob, batchesToSubmit []*dbBatchWithChunks, firstBatch, lastBatch *orm.Batch, +) error { + // Check blob count matches batch count + if len(blobs) != len(batchesToSubmit) { + return fmt.Errorf("blob count mismatch: got %d, want %d", len(blobs), len(batchesToSubmit)) + } + + // Parse calldata (after first 4 bytes: method selector) + method := r.l1RollupABI.Methods["commitBatches"] + if len(calldata) < 4 { + return fmt.Errorf("calldata too short to contain method selector") + } + decoded, err := method.Inputs.Unpack(calldata[4:]) + if err != nil { + return fmt.Errorf("failed to unpack commitBatches calldata: %w", err) + } + + if len(decoded) != 3 { + return fmt.Errorf("unexpected number of decoded parameters: got %d, want 3", len(decoded)) + } + + version, ok := decoded[0].(uint8) + if !ok { + return fmt.Errorf("failed to type assert version to uint8") + } + parentBatchHashB, ok := decoded[1].([32]uint8) + if !ok { + return fmt.Errorf("failed to type assert parentBatchHash to [32]uint8") + } + parentBatchHash := common.BytesToHash(parentBatchHashB[:]) + lastBatchHashB, ok := decoded[2].([32]uint8) + if !ok { + return fmt.Errorf("failed to type assert lastBatchHash to [32]uint8") + } + lastBatchHash := common.BytesToHash(lastBatchHashB[:]) + + // Check version and batch hashes + if version != uint8(firstBatch.CodecVersion) { + return fmt.Errorf("sanity check failed: version mismatch: calldata=%d, db=%d", version, firstBatch.CodecVersion) + } + if parentBatchHash != common.HexToHash(firstBatch.ParentBatchHash) { + return fmt.Errorf("sanity check failed: parentBatchHash mismatch: calldata=%s, db=%s", parentBatchHash.Hex(), firstBatch.ParentBatchHash) + } + if lastBatchHash != common.HexToHash(lastBatch.Hash) { + return fmt.Errorf("sanity check failed: lastBatchHash mismatch: calldata=%s, db=%s", lastBatchHash.Hex(), lastBatch.Hash) + } + + // Get codec for blob decoding + codec, err := encoding.CodecFromVersion(encoding.CodecVersion(firstBatch.CodecVersion)) + if err != nil { + return fmt.Errorf("failed to get codec: %w", err) + } + + // Loop through each batch and blob, decode and compare + for i, blob := range blobs { + dbBatch := batchesToSubmit[i].Batch + dbChunks := batchesToSubmit[i].Chunks + + // Collect all blocks for the batch + var batchBlocks []*encoding.Block + for _, c := range dbChunks { + blocks, err := r.l2BlockOrm.GetL2BlocksInRange(r.ctx, c.StartBlockNumber, c.EndBlockNumber) + if err != nil { + return fmt.Errorf("failed to get blocks for batch %d chunk %d: %w", dbBatch.Index, c.Index, err) + } + batchBlocks = append(batchBlocks, blocks...) + } + + // Decode blob payload + payload, err := codec.DecodeBlob(blob) + if err != nil { + return fmt.Errorf("failed to decode blob for batch %d: %w", dbBatch.Index, err) + } + + // Check L1 message queue hashes + if payload.PrevL1MessageQueueHash() != common.HexToHash(dbBatch.PrevL1MessageQueueHash) { + return fmt.Errorf("sanity check failed: prevL1MessageQueueHash mismatch for batch %d: decoded=%s, db=%s", + dbBatch.Index, payload.PrevL1MessageQueueHash().Hex(), dbBatch.PrevL1MessageQueueHash) + } + if payload.PostL1MessageQueueHash() != common.HexToHash(dbBatch.PostL1MessageQueueHash) { + return fmt.Errorf("sanity check failed: postL1MessageQueueHash mismatch for batch %d: decoded=%s, db=%s", + dbBatch.Index, payload.PostL1MessageQueueHash().Hex(), dbBatch.PostL1MessageQueueHash) + } + + // Compare block count and block numbers + decodedBlocks := payload.Blocks() + if len(decodedBlocks) != len(batchBlocks) { + return fmt.Errorf("sanity check failed: block count mismatch in batch %d: decoded=%d, db=%d", dbBatch.Index, len(decodedBlocks), len(batchBlocks)) + } + for j, b := range batchBlocks { + if decodedBlocks[j].Number() != b.Header.Number.Uint64() { + return fmt.Errorf("sanity check failed: block number mismatch in batch %d block %d: decoded=%d, db=%d", + dbBatch.Index, j, decodedBlocks[j].Number(), b.Header.Number.Uint64()) + } + } + } + + // All checks passed + return nil +} From cf852259f36191d55e7d92a77b8fd8a6523d667d Mon Sep 17 00:00:00 2001 From: colinlyguo Date: Mon, 4 Aug 2025 15:52:29 +0800 Subject: [PATCH 08/14] refactor --- .../internal/controller/relayer/l2_relayer.go | 156 +----- .../controller/relayer/l2_relayer_sanity.go | 460 +++++++++++------- 2 files changed, 289 insertions(+), 327 deletions(-) diff --git a/rollup/internal/controller/relayer/l2_relayer.go b/rollup/internal/controller/relayer/l2_relayer.go index d0b5f8e02d..e966a9a701 100644 --- a/rollup/internal/controller/relayer/l2_relayer.go +++ b/rollup/internal/controller/relayer/l2_relayer.go @@ -278,19 +278,6 @@ func (r *Layer2Relayer) initializeGenesis() error { } func (r *Layer2Relayer) commitGenesisBatch(batchHash string, batchHeader []byte, stateRoot common.Hash) error { - // Basic sanity checks - if batchHash == "" { - return fmt.Errorf("batch hash is empty") - } - - if len(batchHeader) == 0 { - return fmt.Errorf("batch header is empty") - } - - if stateRoot == (common.Hash{}) { - return fmt.Errorf("state root is zero") - } - var calldata []byte var packErr error @@ -303,6 +290,12 @@ func (r *Layer2Relayer) commitGenesisBatch(batchHash string, batchHeader []byte, log.Info("Validium importGenesis", "calldata", common.Bytes2Hex(calldata)) } else { // rollup mode: pass batchHeader and stateRoot + + // Check state root is not zero + if stateRoot == (common.Hash{}) { + return fmt.Errorf("state root is zero") + } + calldata, packErr = r.l1RollupABI.Pack("importGenesisBatch", batchHeader, stateRoot) if packErr != nil { return fmt.Errorf("failed to pack rollup importGenesisBatch with batch header: %v and state root: %v. error: %v", common.Bytes2Hex(batchHeader), stateRoot, packErr) @@ -310,11 +303,6 @@ func (r *Layer2Relayer) commitGenesisBatch(batchHash string, batchHeader []byte, log.Info("Rollup importGenesis", "calldata", common.Bytes2Hex(calldata), "stateRoot", stateRoot) } - // Check generated calldata is not empty - if len(calldata) == 0 { - return fmt.Errorf("generated calldata is empty") - } - // submit genesis batch to L1 rollup contract txHash, _, err := r.commitSender.SendTransaction(batchHash, &r.cfg.RollupContractAddress, calldata, nil) if err != nil { @@ -492,12 +480,6 @@ func (r *Layer2Relayer) ProcessPendingBatches() { log.Info("Forcing submission of batches due to timeout", "batch index", batchesToSubmit[0].Batch.Index, "first block created at", oldestBlockTimestamp) } - // Sanity checks before constructing the transaction - if err := r.sanityChecksBeforeConstructingTransaction(batchesToSubmit); err != nil { - log.Error("Sanity checks failed before constructing transaction", "batches count", len(batchesToSubmit), "first batch index", batchesToSubmit[0].Batch.Index, "last batch index", batchesToSubmit[len(batchesToSubmit)-1].Batch.Index, "err", err) - return - } - // We have at least 1 batch to commit firstBatch := batchesToSubmit[0].Batch lastBatch := batchesToSubmit[len(batchesToSubmit)-1].Batch @@ -527,9 +509,8 @@ func (r *Layer2Relayer) ProcessPendingBatches() { return } - err = r.sanityChecksCommitBatchCodecV7CalldataAndBlobs(calldata, blobs, batchesToSubmit, firstBatch, lastBatch) - if err != nil { - log.Error("Sanity check failed for calldata and blobs", "err", err) + if err = r.sanityChecksCommitBatchCodecV7CalldataAndBlobs(calldata, blobs); err != nil { + log.Error("Sanity check failed for calldata and blobs", "codecVersion", codecVersion, "start index", firstBatch.Index, "end index", lastBatch.Index, "err", err) return } } @@ -972,25 +953,6 @@ func (r *Layer2Relayer) handleL2RollupRelayerConfirmLoop(ctx context.Context) { } func (r *Layer2Relayer) constructCommitBatchPayloadCodecV7(batchesToSubmit []*dbBatchWithChunks, firstBatch, lastBatch *orm.Batch) ([]byte, []*kzg4844.Blob, uint64, uint64, error) { - // Basic sanity checks - if len(batchesToSubmit) == 0 { - return nil, nil, 0, 0, fmt.Errorf("no batches to submit") - } - if firstBatch == nil { - return nil, nil, 0, 0, fmt.Errorf("first batch is nil") - } - if lastBatch == nil { - return nil, nil, 0, 0, fmt.Errorf("last batch is nil") - } - - // Check firstBatch and lastBatch match batchesToSubmit - if firstBatch.Index != batchesToSubmit[0].Batch.Index { - return nil, nil, 0, 0, fmt.Errorf("first batch index mismatch: expected %d, got %d", batchesToSubmit[0].Batch.Index, firstBatch.Index) - } - if lastBatch.Index != batchesToSubmit[len(batchesToSubmit)-1].Batch.Index { - return nil, nil, 0, 0, fmt.Errorf("last batch index mismatch: expected %d, got %d", batchesToSubmit[len(batchesToSubmit)-1].Batch.Index, lastBatch.Index) - } - var maxBlockHeight uint64 var totalGasUsed uint64 blobs := make([]*kzg4844.Blob, 0, len(batchesToSubmit)) @@ -1011,16 +973,6 @@ func (r *Layer2Relayer) constructCommitBatchPayloadCodecV7(batchesToSubmit []*db return nil, nil, 0, 0, fmt.Errorf("failed to get blocks in range for batch %d: %w", b.Batch.Index, err) } - if len(blocks) == 0 { - return nil, nil, 0, 0, fmt.Errorf("batch %d chunk %d has no blocks in range [%d, %d]", b.Batch.Index, c.Index, c.StartBlockNumber, c.EndBlockNumber) - } - - // Check that we got the expected number of blocks - expectedBlockCount := c.EndBlockNumber - c.StartBlockNumber + 1 - if uint64(len(blocks)) != expectedBlockCount { - return nil, nil, 0, 0, fmt.Errorf("batch %d chunk %d expected %d blocks but got %d", b.Batch.Index, c.Index, expectedBlockCount, len(blocks)) - } - batchBlocks = append(batchBlocks, blocks...) if c.EndBlockNumber > maxBlockHeight { @@ -1037,16 +989,6 @@ func (r *Layer2Relayer) constructCommitBatchPayloadCodecV7(batchesToSubmit []*db Blocks: batchBlocks, } - // Check encoding batch fields are not zero hashes - if encodingBatch.ParentBatchHash == (common.Hash{}) { - return nil, nil, 0, 0, fmt.Errorf("batch %d parent batch hash is zero", b.Batch.Index) - } - - // Check L1 message queue hash consistency - if err := r.validateMessageQueueConsistency(encodingBatch.Index, b.Chunks, encodingBatch.PrevL1MessageQueueHash, encodingBatch.PostL1MessageQueueHash); err != nil { - return nil, nil, 0, 0, err - } - codec, err := encoding.CodecFromVersion(version) if err != nil { return nil, nil, 0, 0, fmt.Errorf("failed to get codec from version %d, err: %w", b.Batch.CodecVersion, err) @@ -1057,42 +999,29 @@ func (r *Layer2Relayer) constructCommitBatchPayloadCodecV7(batchesToSubmit []*db return nil, nil, 0, 0, fmt.Errorf("failed to create DA batch: %w", err) } - blob := daBatch.Blob() - if blob == nil { - return nil, nil, 0, 0, fmt.Errorf("batch %d generated nil blob", b.Batch.Index) - } - - blobs = append(blobs, blob) + blobs = append(blobs, daBatch.Blob()) } calldata, err := r.l1RollupABI.Pack("commitBatches", version, common.HexToHash(firstBatch.ParentBatchHash), common.HexToHash(lastBatch.Hash)) if err != nil { return nil, nil, 0, 0, fmt.Errorf("failed to pack commitBatches: %w", err) } - - if len(calldata) == 0 { - return nil, nil, 0, 0, fmt.Errorf("generated calldata is empty") - } - return calldata, blobs, maxBlockHeight, totalGasUsed, nil } func (r *Layer2Relayer) constructCommitBatchPayloadValidium(batch *dbBatchWithChunks) ([]byte, uint64, uint64, error) { - // Basic sanity checks - if batch == nil || batch.Batch == nil { - return nil, 0, 0, fmt.Errorf("batch is nil") - } - - if len(batch.Chunks) == 0 { - return nil, 0, 0, fmt.Errorf("batch %d has no chunks", batch.Batch.Index) - } - // Check state root is not zero stateRoot := common.HexToHash(batch.Batch.StateRoot) if stateRoot == (common.Hash{}) { return nil, 0, 0, fmt.Errorf("batch %d state root is zero", batch.Batch.Index) } + // Check parent batch hash is not zero + parentBatchHash := common.HexToHash(batch.Batch.ParentBatchHash) + if parentBatchHash == (common.Hash{}) { + return nil, 0, 0, fmt.Errorf("batch %d parent batch hash is zero", batch.Batch.Index) + } + // Calculate metrics var maxBlockHeight uint64 var totalGasUsed uint64 @@ -1112,44 +1041,17 @@ func (r *Layer2Relayer) constructCommitBatchPayloadValidium(batch *dbBatchWithCh lastChunk := batch.Chunks[len(batch.Chunks)-1] commitment := common.HexToHash(lastChunk.EndBlockHash) - if commitment == (common.Hash{}) { - return nil, 0, 0, fmt.Errorf("batch %d last chunk end block hash is zero, cannot create commitment", batch.Batch.Index) - } - - // Check parent batch hash is not zero - parentBatchHash := common.HexToHash(batch.Batch.ParentBatchHash) - if parentBatchHash == (common.Hash{}) { - return nil, 0, 0, fmt.Errorf("batch %d parent batch hash is zero", batch.Batch.Index) - } version := encoding.CodecVersion(batch.Batch.CodecVersion) calldata, err := r.validiumABI.Pack("commitBatch", version, common.HexToHash(batch.Batch.ParentBatchHash), common.HexToHash(batch.Batch.StateRoot), common.HexToHash(batch.Batch.WithdrawRoot), commitment[:]) if err != nil { return nil, 0, 0, fmt.Errorf("failed to pack commitBatch: %w", err) } - - if len(calldata) == 0 { - return nil, 0, 0, fmt.Errorf("generated calldata is empty for batch %d", batch.Batch.Index) - } - log.Info("Validium commitBatch", "maxBlockHeight", maxBlockHeight, "commitment", commitment.Hex()) return calldata, maxBlockHeight, totalGasUsed, nil } func (r *Layer2Relayer) constructFinalizeBundlePayloadCodecV7(dbBatch *orm.Batch, endChunk *orm.Chunk, aggProof *message.OpenVMBundleProof) ([]byte, error) { - // Basic sanity checks - if dbBatch == nil { - return nil, fmt.Errorf("batch is nil") - } - if endChunk == nil { - return nil, fmt.Errorf("end chunk is nil") - } - - // Check batch header - if len(dbBatch.BatchHeader) == 0 { - return nil, fmt.Errorf("batch %d header is empty", dbBatch.Index) - } - // Check state root is not zero stateRoot := common.HexToHash(dbBatch.StateRoot) if stateRoot == (common.Hash{}) { @@ -1173,11 +1075,6 @@ func (r *Layer2Relayer) constructFinalizeBundlePayloadCodecV7(dbBatch *orm.Batch if packErr != nil { return nil, fmt.Errorf("failed to pack finalizeBundlePostEuclidV2 with proof: %w", packErr) } - - if len(calldata) == 0 { - return nil, fmt.Errorf("generated calldata with proof is empty") - } - return calldata, nil } @@ -1194,28 +1091,10 @@ func (r *Layer2Relayer) constructFinalizeBundlePayloadCodecV7(dbBatch *orm.Batch if packErr != nil { return nil, fmt.Errorf("failed to pack finalizeBundlePostEuclidV2NoProof: %w", packErr) } - - if len(calldata) == 0 { - return nil, fmt.Errorf("generated calldata without proof is empty") - } - return calldata, nil } func (r *Layer2Relayer) constructFinalizeBundlePayloadValidium(dbBatch *orm.Batch, endChunk *orm.Chunk, aggProof *message.OpenVMBundleProof) ([]byte, error) { - // Basic sanity checks - if dbBatch == nil { - return nil, fmt.Errorf("batch is nil") - } - if endChunk == nil { - return nil, fmt.Errorf("end chunk is nil") - } - - // Check batch header is not empty - if len(dbBatch.BatchHeader) == 0 { - return nil, fmt.Errorf("batch %d header is empty", dbBatch.Index) - } - // Check proof if present if aggProof != nil && len(aggProof.Proof()) == 0 { return nil, fmt.Errorf("aggregate proof is empty") @@ -1237,11 +1116,6 @@ func (r *Layer2Relayer) constructFinalizeBundlePayloadValidium(dbBatch *orm.Batc if packErr != nil { return nil, fmt.Errorf("failed to pack validium finalizeBundle: %w", packErr) } - - if len(calldata) == 0 { - return nil, fmt.Errorf("generated calldata is empty for batch %d", dbBatch.Index) - } - return calldata, nil } diff --git a/rollup/internal/controller/relayer/l2_relayer_sanity.go b/rollup/internal/controller/relayer/l2_relayer_sanity.go index e2d9d982d4..d81b161f67 100644 --- a/rollup/internal/controller/relayer/l2_relayer_sanity.go +++ b/rollup/internal/controller/relayer/l2_relayer_sanity.go @@ -6,140 +6,213 @@ import ( "github.com/scroll-tech/da-codec/encoding" "github.com/scroll-tech/go-ethereum/common" "github.com/scroll-tech/go-ethereum/crypto/kzg4844" - "github.com/scroll-tech/go-ethereum/log" "scroll-tech/rollup/internal/orm" ) -// validateMessageQueueConsistency validates L1 message queue hash consistency -func (r *Layer2Relayer) validateMessageQueueConsistency(batchIndex uint64, chunks []*orm.Chunk, prevL1MsgQueueHash common.Hash, postL1MsgQueueHash common.Hash) error { - if batchIndex == 0 { - return nil +// sanityChecksCommitBatchCodecV7CalldataAndBlobs performs comprehensive validation of the constructed +// transaction data (calldata and blobs) by parsing them and comparing against database records. +// This ensures the constructed transaction data is correct and consistent with the database state. +func (r *Layer2Relayer) sanityChecksCommitBatchCodecV7CalldataAndBlobs(calldata []byte, blobs []*kzg4844.Blob) error { + if len(blobs) == 0 { + return fmt.Errorf("no blobs provided") } - if len(chunks) == 0 { - return fmt.Errorf("batch %d has no chunks for message queue validation", batchIndex) + calldataInfo, err := r.parseCommitBatchesCalldata(calldata) + if err != nil { + return fmt.Errorf("failed to parse calldata: %w", err) } - firstChunk := chunks[0] - lastChunk := chunks[len(chunks)-1] + batchesToValidate, firstBatch, lastBatch, err := r.getBatchesFromCalldata(calldataInfo) + if err != nil { + return fmt.Errorf("failed to get batches from database: %w", err) + } - // Calculate total L1 messages in this batch - var totalL1MessagesInBatch uint64 - for _, chunk := range chunks { - totalL1MessagesInBatch += chunk.TotalL1MessagesPoppedInChunk + if len(blobs) != len(batchesToValidate) { + return fmt.Errorf("blob count mismatch: got %d blobs, expected %d batches", len(blobs), len(batchesToValidate)) } - // If there were L1 messages processed before this batch, prev hash should not be zero - if firstChunk.TotalL1MessagesPoppedBefore > 0 && prevL1MsgQueueHash == (common.Hash{}) { - return fmt.Errorf("batch %d prev L1 message queue hash is zero but %d L1 messages were processed before", batchIndex, firstChunk.TotalL1MessagesPoppedBefore) + if err := r.validateCalldataAgainstDatabase(calldataInfo, firstBatch, lastBatch); err != nil { + return fmt.Errorf("calldata validation failed: %w", err) } - // If there are any L1 messages processed up to this batch, post hash should not be zero - totalL1MessagesProcessed := lastChunk.TotalL1MessagesPoppedBefore + lastChunk.TotalL1MessagesPoppedInChunk - if totalL1MessagesProcessed > 0 && postL1MsgQueueHash == (common.Hash{}) { - return fmt.Errorf("batch %d post L1 message queue hash is zero but %d L1 messages were processed in total", batchIndex, totalL1MessagesProcessed) + if err := r.validateDatabaseConsistency(batchesToValidate); err != nil { + return fmt.Errorf("database consistency validation failed: %w", err) } - // Prev and post queue hashes should be different if L1 messages were processed in this batch - if totalL1MessagesInBatch > 0 && prevL1MsgQueueHash == postL1MsgQueueHash { - return fmt.Errorf("batch %d has same prev and post L1 message queue hashes but processed %d L1 messages in this batch", batchIndex, totalL1MessagesInBatch) + if err := r.validateBlobsAgainstDatabase(blobs, batchesToValidate); err != nil { + return fmt.Errorf("blob validation failed: %w", err) } return nil } -// sanityChecksBeforeConstructingTransaction performs sanity checks before constructing a transaction. -func (r *Layer2Relayer) sanityChecksBeforeConstructingTransaction(batchesToSubmit []*dbBatchWithChunks) error { - if len(batchesToSubmit) == 0 { - return fmt.Errorf("no batches to submit") - } +// CalldataInfo holds parsed information from commitBatches calldata +type CalldataInfo struct { + Version uint8 + ParentBatchHash common.Hash + LastBatchHash common.Hash +} - // Get previous chunk for continuity check - prevChunk, err := r.getPreviousChunkForContinuity(batchesToSubmit[0]) +// parseCommitBatchesCalldata parses the commitBatches calldata and extracts key information +func (r *Layer2Relayer) parseCommitBatchesCalldata(calldata []byte) (*CalldataInfo, error) { + method := r.l1RollupABI.Methods["commitBatches"] + decoded, err := method.Inputs.Unpack(calldata[4:]) if err != nil { - return err + return nil, fmt.Errorf("failed to unpack commitBatches calldata: %w", err) } - // Validate batches (including basic, codec versions, and detailed checks) - if err := r.validateBatches(batchesToSubmit, prevChunk); err != nil { - return err + if len(decoded) != 3 { + return nil, fmt.Errorf("unexpected number of decoded parameters: got %d, want 3", len(decoded)) } - log.Info("Sanity check passed before constructing transaction", "batches count", len(batchesToSubmit)) - return nil + version, ok := decoded[0].(uint8) + if !ok { + return nil, fmt.Errorf("failed to type assert version to uint8") + } + + parentBatchHashB, ok := decoded[1].([32]uint8) + if !ok { + return nil, fmt.Errorf("failed to type assert parentBatchHash to [32]uint8") + } + parentBatchHash := common.BytesToHash(parentBatchHashB[:]) + + lastBatchHashB, ok := decoded[2].([32]uint8) + if !ok { + return nil, fmt.Errorf("failed to type assert lastBatchHash to [32]uint8") + } + lastBatchHash := common.BytesToHash(lastBatchHashB[:]) + + return &CalldataInfo{ + Version: version, + ParentBatchHash: parentBatchHash, + LastBatchHash: lastBatchHash, + }, nil } -// getPreviousChunkForContinuity gets the previous chunk for block continuity check -func (r *Layer2Relayer) getPreviousChunkForContinuity(firstBatch *dbBatchWithChunks) (*orm.Chunk, error) { - firstChunk := firstBatch.Chunks[0] - if firstChunk.Index == 0 { - return nil, fmt.Errorf("genesis chunk should not be in normal batch submission flow, chunk index: %d", firstChunk.Index) +// getBatchesFromCalldata retrieves the relevant batches from database based on calldata information +func (r *Layer2Relayer) getBatchesFromCalldata(info *CalldataInfo) ([]*dbBatchWithChunks, *orm.Batch, *orm.Batch, error) { + // Get the parent batch to determine the starting point + parentBatch, err := r.batchOrm.GetBatchByHash(r.ctx, info.ParentBatchHash.Hex()) + if err != nil { + return nil, nil, nil, fmt.Errorf("failed to get parent batch by hash %s: %w", info.ParentBatchHash.Hex(), err) } - prevChunk, err := r.chunkOrm.GetChunkByIndex(r.ctx, firstChunk.Index-1) + // Get the last batch to determine the ending point + lastBatch, err := r.batchOrm.GetBatchByHash(r.ctx, info.LastBatchHash.Hex()) if err != nil { - return nil, fmt.Errorf("failed to get previous chunk %d for continuity check: %w", firstChunk.Index-1, err) + return nil, nil, nil, fmt.Errorf("failed to get last batch by hash %s: %w", info.LastBatchHash.Hex(), err) } - return prevChunk, nil -} + // Get all batches in the range (parent+1 to last) + firstBatchIndex := parentBatch.Index + 1 + lastBatchIndex := lastBatch.Index -// validateBatches performs validation on all batches including basic checks, codec version consistency, and detailed checks. -func (r *Layer2Relayer) validateBatches(batchesToSubmit []*dbBatchWithChunks, initialPrevChunk *orm.Chunk) error { - // Basic validation: ensure each batch and its chunks are non-empty. - for i, batch := range batchesToSubmit { - if batch == nil || batch.Batch == nil { - return fmt.Errorf("batch %d is nil", i) + // Check if the range is valid + if firstBatchIndex > lastBatchIndex { + return nil, nil, nil, fmt.Errorf("no batches found in range: first index %d, last index %d", firstBatchIndex, lastBatchIndex) + } + + var batchesToValidate []*dbBatchWithChunks + for batchIndex := firstBatchIndex; batchIndex <= lastBatchIndex; batchIndex++ { + dbBatch, err := r.batchOrm.GetBatchByIndex(r.ctx, batchIndex) + if err != nil { + return nil, nil, nil, fmt.Errorf("failed to get batch by index %d: %w", batchIndex, err) } - if len(batch.Chunks) == 0 { - return fmt.Errorf("batch %d has no chunks", batch.Batch.Index) + + // Get chunks for this batch + dbChunks, err := r.chunkOrm.GetChunksInRange(r.ctx, dbBatch.StartChunkIndex, dbBatch.EndChunkIndex) + if err != nil { + return nil, nil, nil, fmt.Errorf("failed to get chunks for batch %d: %w", batchIndex, err) } + + batchesToValidate = append(batchesToValidate, &dbBatchWithChunks{ + Batch: dbBatch, + Chunks: dbChunks, + }) } - // Check that all batches have the same codec version. - firstBatchCodecVersion := batchesToSubmit[0].Batch.CodecVersion - for _, batch := range batchesToSubmit { - if batch.Batch.CodecVersion != firstBatchCodecVersion { - return fmt.Errorf("batch %d has different codec version %d, expected %d", batch.Batch.Index, batch.Batch.CodecVersion, firstBatchCodecVersion) - } + // Get first batch for return + firstBatch := batchesToValidate[0].Batch + + return batchesToValidate, firstBatch, lastBatch, nil +} + +// validateCalldataAgainstDatabase validates calldata parameters against database records +func (r *Layer2Relayer) validateCalldataAgainstDatabase(info *CalldataInfo, firstBatch, lastBatch *orm.Batch) error { + // Validate codec version + if info.Version != uint8(firstBatch.CodecVersion) { + return fmt.Errorf("version mismatch: calldata=%d, db=%d", info.Version, firstBatch.CodecVersion) } - // Validate each batch in detail, updating the previous chunk as we go. - currentPrevChunk := initialPrevChunk - for i, batch := range batchesToSubmit { - if err := r.validateSingleBatch(batch, i, batchesToSubmit, currentPrevChunk); err != nil { - return err - } - // Update the previous chunk to the last chunk of this batch for the next batch. - currentPrevChunk = batch.Chunks[len(batch.Chunks)-1] + // Validate parent batch hash + if info.ParentBatchHash != common.HexToHash(firstBatch.ParentBatchHash) { + return fmt.Errorf("parentBatchHash mismatch: calldata=%s, db=%s", info.ParentBatchHash.Hex(), firstBatch.ParentBatchHash) + } + + // Validate last batch hash + if info.LastBatchHash != common.HexToHash(lastBatch.Hash) { + return fmt.Errorf("lastBatchHash mismatch: calldata=%s, db=%s", info.LastBatchHash.Hex(), lastBatch.Hash) } + return nil } -// validateSingleBatch validates a single batch and its chunks -func (r *Layer2Relayer) validateSingleBatch(batch *dbBatchWithChunks, i int, allBatches []*dbBatchWithChunks, prevChunk *orm.Chunk) error { - // Validate batch fields - if err := r.validateBatchFields(batch, i, allBatches); err != nil { - return err +// validateDatabaseConsistency performs comprehensive validation of database records +func (r *Layer2Relayer) validateDatabaseConsistency(batchesToValidate []*dbBatchWithChunks) error { + if len(batchesToValidate) == 0 { + return fmt.Errorf("no batches to validate") } - // Validate message queue consistency - if err := r.validateMessageQueueConsistency(batch.Batch.Index, batch.Chunks, common.HexToHash(batch.Batch.PrevL1MessageQueueHash), common.HexToHash(batch.Batch.PostL1MessageQueueHash)); err != nil { - return err + // Get previous chunk for continuity check + firstChunk := batchesToValidate[0].Chunks[0] + if firstChunk.Index == 0 { + return fmt.Errorf("genesis chunk should not be in normal batch submission flow, chunk index: %d", firstChunk.Index) } - // Validate chunks - if err := r.validateBatchChunks(batch, prevChunk); err != nil { - return err + prevChunk, err := r.chunkOrm.GetChunkByIndex(r.ctx, firstChunk.Index-1) + if err != nil { + return fmt.Errorf("failed to get previous chunk %d for continuity check: %w", firstChunk.Index-1, err) + } + + // Validate codec version consistency across all batches + firstBatchCodecVersion := batchesToValidate[0].Batch.CodecVersion + for i, batch := range batchesToValidate { + if batch.Batch.CodecVersion != firstBatchCodecVersion { + return fmt.Errorf("batch %d has different codec version %d, expected %d", batch.Batch.Index, batch.Batch.CodecVersion, firstBatchCodecVersion) + } + + // Validate individual batch + if err := r.validateSingleBatchConsistency(batch, i, batchesToValidate); err != nil { + return err + } + + // Validate chunks in this batch + if err := r.validateBatchChunksConsistency(batch, prevChunk); err != nil { + return err + } + + // Update prevChunk to the last chunk of this batch for next iteration + if len(batch.Chunks) == 0 { + return fmt.Errorf("batch %d has no chunks", batch.Batch.Index) + } + prevChunk = batch.Chunks[len(batch.Chunks)-1] } return nil } -// validateBatchFields validates essential batch fields -func (r *Layer2Relayer) validateBatchFields(batch *dbBatchWithChunks, i int, allBatches []*dbBatchWithChunks) error { - // Check essential batch fields are not zero values +// validateSingleBatchConsistency validates a single batch's consistency +func (r *Layer2Relayer) validateSingleBatchConsistency(batch *dbBatchWithChunks, i int, allBatches []*dbBatchWithChunks) error { + if batch == nil || batch.Batch == nil { + return fmt.Errorf("batch %d is nil", i) + } + + if len(batch.Chunks) == 0 { + return fmt.Errorf("batch %d has no chunks", batch.Batch.Index) + } + + // Validate essential batch fields batchHash := common.HexToHash(batch.Batch.Hash) if batchHash == (common.Hash{}) { return fmt.Errorf("batch %d hash is zero", batch.Batch.Index) @@ -149,14 +222,24 @@ func (r *Layer2Relayer) validateBatchFields(batch *dbBatchWithChunks, i int, all return fmt.Errorf("batch %d has zero index (only genesis batch should have index 0)", i) } - // Check batch index is sequential + parentBatchHash := common.HexToHash(batch.Batch.ParentBatchHash) + if parentBatchHash == (common.Hash{}) { + return fmt.Errorf("batch %d parent batch hash is zero", batch.Batch.Index) + } + + stateRoot := common.HexToHash(batch.Batch.StateRoot) + if stateRoot == (common.Hash{}) { + return fmt.Errorf("batch %d state root is zero", batch.Batch.Index) + } + + // Check batch index continuity if i > 0 { prevBatch := allBatches[i-1] if batch.Batch.Index != prevBatch.Batch.Index+1 { return fmt.Errorf("batch index is not sequential: prev batch index %d, current batch index %d", prevBatch.Batch.Index, batch.Batch.Index) } } else { - // For the first batch, check continuity with the parent batch from database + // For the first batch, verify continuity with parent batch from database parentBatch, err := r.batchOrm.GetBatchByHash(r.ctx, batch.Batch.ParentBatchHash) if err != nil { return fmt.Errorf("failed to get parent batch %s for batch %d: %w", batch.Batch.ParentBatchHash, batch.Batch.Index, err) @@ -166,41 +249,37 @@ func (r *Layer2Relayer) validateBatchFields(batch *dbBatchWithChunks, i int, all } } - parentBatchHash := common.HexToHash(batch.Batch.ParentBatchHash) - if parentBatchHash == (common.Hash{}) { - return fmt.Errorf("batch %d parent batch hash is zero", batch.Batch.Index) - } - - stateRoot := common.HexToHash(batch.Batch.StateRoot) - if stateRoot == (common.Hash{}) { - return fmt.Errorf("batch %d state root is zero", batch.Batch.Index) + // Validate L1 message queue consistency + if err := r.validateMessageQueueConsistency(batch.Batch.Index, batch.Chunks, common.HexToHash(batch.Batch.PrevL1MessageQueueHash), common.HexToHash(batch.Batch.PostL1MessageQueueHash)); err != nil { + return err } return nil } -// validateBatchChunks validates all chunks in a batch -func (r *Layer2Relayer) validateBatchChunks(batch *dbBatchWithChunks, prevChunk *orm.Chunk) error { - // Check codec version consistency. +// validateBatchChunksConsistency validates chunks within a batch +func (r *Layer2Relayer) validateBatchChunksConsistency(batch *dbBatchWithChunks, prevChunk *orm.Chunk) error { + // Check codec version consistency between chunks and batch for _, chunk := range batch.Chunks { if chunk.CodecVersion != batch.Batch.CodecVersion { return fmt.Errorf("batch %d chunk %d has different codec version %d, expected %d", batch.Batch.Index, chunk.Index, chunk.CodecVersion, batch.Batch.CodecVersion) } } + // Validate each chunk individually + currentPrevChunk := prevChunk for j, chunk := range batch.Chunks { - if err := r.validateSingleChunk(chunk, prevChunk); err != nil { + if err := r.validateSingleChunkConsistency(chunk, currentPrevChunk); err != nil { return fmt.Errorf("batch %d chunk %d: %w", batch.Batch.Index, j, err) } - // Update the previous chunk to the current one for the next chunk. - prevChunk = chunk + currentPrevChunk = chunk } return nil } -// validateSingleChunk validates a single chunk -func (r *Layer2Relayer) validateSingleChunk(chunk *orm.Chunk, prevChunk *orm.Chunk) error { +// validateSingleChunkConsistency validates a single chunk +func (r *Layer2Relayer) validateSingleChunkConsistency(chunk *orm.Chunk, prevChunk *orm.Chunk) error { if chunk == nil { return fmt.Errorf("chunk is nil") } @@ -210,17 +289,12 @@ func (r *Layer2Relayer) validateSingleChunk(chunk *orm.Chunk, prevChunk *orm.Chu return fmt.Errorf("chunk %d hash is zero", chunk.Index) } - // Check chunk index is sequential + // Check chunk index continuity if chunk.Index != prevChunk.Index+1 { return fmt.Errorf("chunk index is not sequential: prev chunk index %d, current chunk index %d", prevChunk.Index, chunk.Index) } - // Check L1 messages popped continuity - expectedPoppedBefore := prevChunk.TotalL1MessagesPoppedBefore + prevChunk.TotalL1MessagesPoppedInChunk - if chunk.TotalL1MessagesPoppedBefore != expectedPoppedBefore { - return fmt.Errorf("L1 messages popped before is incorrect: expected %d, got %d", expectedPoppedBefore, chunk.TotalL1MessagesPoppedBefore) - } - + // Validate block range if chunk.StartBlockNumber == 0 && chunk.EndBlockNumber == 0 { return fmt.Errorf("chunk %d has zero block range", chunk.Index) } @@ -229,7 +303,7 @@ func (r *Layer2Relayer) validateSingleChunk(chunk *orm.Chunk, prevChunk *orm.Chu return fmt.Errorf("chunk %d has invalid block range: start %d > end %d", chunk.Index, chunk.StartBlockNumber, chunk.EndBlockNumber) } - // Check chunk hash fields + // Check hash fields startBlockHash := common.HexToHash(chunk.StartBlockHash) if startBlockHash == (common.Hash{}) { return fmt.Errorf("chunk %d start block hash is zero", chunk.Index) @@ -240,111 +314,125 @@ func (r *Layer2Relayer) validateSingleChunk(chunk *orm.Chunk, prevChunk *orm.Chu return fmt.Errorf("chunk %d end block hash is zero", chunk.Index) } - // Check chunk continuity: previous chunk's end block number + 1 should equal current chunk's start block number + // Check block continuity with previous chunk if prevChunk.EndBlockNumber+1 != chunk.StartBlockNumber { return fmt.Errorf("chunk is not continuous with previous chunk %d: prev end block %d, current start block %d", prevChunk.Index, prevChunk.EndBlockNumber, chunk.StartBlockNumber) } + // Check L1 messages continuity + expectedPoppedBefore := prevChunk.TotalL1MessagesPoppedBefore + prevChunk.TotalL1MessagesPoppedInChunk + if chunk.TotalL1MessagesPoppedBefore != expectedPoppedBefore { + return fmt.Errorf("L1 messages popped before is incorrect: expected %d, got %d", expectedPoppedBefore, chunk.TotalL1MessagesPoppedBefore) + } + return nil } -func (r *Layer2Relayer) sanityChecksCommitBatchCodecV7CalldataAndBlobs(calldata []byte, blobs []*kzg4844.Blob, batchesToSubmit []*dbBatchWithChunks, firstBatch, lastBatch *orm.Batch, -) error { - // Check blob count matches batch count - if len(blobs) != len(batchesToSubmit) { - return fmt.Errorf("blob count mismatch: got %d, want %d", len(blobs), len(batchesToSubmit)) - } - - // Parse calldata (after first 4 bytes: method selector) - method := r.l1RollupABI.Methods["commitBatches"] - if len(calldata) < 4 { - return fmt.Errorf("calldata too short to contain method selector") - } - decoded, err := method.Inputs.Unpack(calldata[4:]) +// validateBlobsAgainstDatabase validates blobs against database records +func (r *Layer2Relayer) validateBlobsAgainstDatabase(blobs []*kzg4844.Blob, batchesToValidate []*dbBatchWithChunks) error { + // Get codec for blob decoding + firstBatch := batchesToValidate[0].Batch + codec, err := encoding.CodecFromVersion(encoding.CodecVersion(firstBatch.CodecVersion)) if err != nil { - return fmt.Errorf("failed to unpack commitBatches calldata: %w", err) + return fmt.Errorf("failed to get codec: %w", err) } - if len(decoded) != 3 { - return fmt.Errorf("unexpected number of decoded parameters: got %d, want 3", len(decoded)) + // Validate each blob against its corresponding batch + for i, blob := range blobs { + dbBatch := batchesToValidate[i].Batch + dbChunks := batchesToValidate[i].Chunks + + if err := r.validateSingleBlobAgainstBatch(blob, dbBatch, dbChunks, codec); err != nil { + return fmt.Errorf("blob validation failed for batch %d: %w", dbBatch.Index, err) + } } - version, ok := decoded[0].(uint8) - if !ok { - return fmt.Errorf("failed to type assert version to uint8") + return nil +} + +// validateSingleBlobAgainstBatch validates a single blob against its batch data +func (r *Layer2Relayer) validateSingleBlobAgainstBatch(blob *kzg4844.Blob, dbBatch *orm.Batch, dbChunks []*orm.Chunk, codec encoding.Codec) error { + // Collect all blocks for the batch + var batchBlocks []*encoding.Block + for _, c := range dbChunks { + blocks, err := r.l2BlockOrm.GetL2BlocksInRange(r.ctx, c.StartBlockNumber, c.EndBlockNumber) + if err != nil { + return fmt.Errorf("failed to get blocks for chunk %d: %w", c.Index, err) + } + + if len(blocks) == 0 { + return fmt.Errorf("chunk %d has no blocks in range [%d, %d]", c.Index, c.StartBlockNumber, c.EndBlockNumber) + } + + // Verify block count matches expected range + expectedBlockCount := c.EndBlockNumber - c.StartBlockNumber + 1 + if len(blocks) != int(expectedBlockCount) { + return fmt.Errorf("chunk %d expected %d blocks but got %d", c.Index, expectedBlockCount, len(blocks)) + } + + batchBlocks = append(batchBlocks, blocks...) } - parentBatchHashB, ok := decoded[1].([32]uint8) - if !ok { - return fmt.Errorf("failed to type assert parentBatchHash to [32]uint8") + + // Decode blob payload + payload, err := codec.DecodeBlob(blob) + if err != nil { + return fmt.Errorf("failed to decode blob: %w", err) } - parentBatchHash := common.BytesToHash(parentBatchHashB[:]) - lastBatchHashB, ok := decoded[2].([32]uint8) - if !ok { - return fmt.Errorf("failed to type assert lastBatchHash to [32]uint8") + + // Validate L1 message queue hashes + if payload.PrevL1MessageQueueHash() != common.HexToHash(dbBatch.PrevL1MessageQueueHash) { + return fmt.Errorf("prevL1MessageQueueHash mismatch: decoded=%s, db=%s", payload.PrevL1MessageQueueHash().Hex(), dbBatch.PrevL1MessageQueueHash) } - lastBatchHash := common.BytesToHash(lastBatchHashB[:]) - // Check version and batch hashes - if version != uint8(firstBatch.CodecVersion) { - return fmt.Errorf("sanity check failed: version mismatch: calldata=%d, db=%d", version, firstBatch.CodecVersion) + if payload.PostL1MessageQueueHash() != common.HexToHash(dbBatch.PostL1MessageQueueHash) { + return fmt.Errorf("postL1MessageQueueHash mismatch: decoded=%s, db=%s", payload.PostL1MessageQueueHash().Hex(), dbBatch.PostL1MessageQueueHash) } - if parentBatchHash != common.HexToHash(firstBatch.ParentBatchHash) { - return fmt.Errorf("sanity check failed: parentBatchHash mismatch: calldata=%s, db=%s", parentBatchHash.Hex(), firstBatch.ParentBatchHash) + + // Validate block data + decodedBlocks := payload.Blocks() + if len(decodedBlocks) != len(batchBlocks) { + return fmt.Errorf("block count mismatch: decoded=%d, db=%d", len(decodedBlocks), len(batchBlocks)) } - if lastBatchHash != common.HexToHash(lastBatch.Hash) { - return fmt.Errorf("sanity check failed: lastBatchHash mismatch: calldata=%s, db=%s", lastBatchHash.Hex(), lastBatch.Hash) + + for j, dbBlock := range batchBlocks { + if decodedBlocks[j].Number() != dbBlock.Header.Number.Uint64() { + return fmt.Errorf("block number mismatch at index %d: decoded=%d, db=%d", j, decodedBlocks[j].Number(), dbBlock.Header.Number.Uint64()) + } } - // Get codec for blob decoding - codec, err := encoding.CodecFromVersion(encoding.CodecVersion(firstBatch.CodecVersion)) - if err != nil { - return fmt.Errorf("failed to get codec: %w", err) + return nil +} + +// validateMessageQueueConsistency validates L1 message queue hash consistency +func (r *Layer2Relayer) validateMessageQueueConsistency(batchIndex uint64, chunks []*orm.Chunk, prevL1MsgQueueHash common.Hash, postL1MsgQueueHash common.Hash) error { + if len(chunks) == 0 { + return fmt.Errorf("batch %d has no chunks for message queue validation", batchIndex) } - // Loop through each batch and blob, decode and compare - for i, blob := range blobs { - dbBatch := batchesToSubmit[i].Batch - dbChunks := batchesToSubmit[i].Chunks - - // Collect all blocks for the batch - var batchBlocks []*encoding.Block - for _, c := range dbChunks { - blocks, err := r.l2BlockOrm.GetL2BlocksInRange(r.ctx, c.StartBlockNumber, c.EndBlockNumber) - if err != nil { - return fmt.Errorf("failed to get blocks for batch %d chunk %d: %w", dbBatch.Index, c.Index, err) - } - batchBlocks = append(batchBlocks, blocks...) - } + firstChunk := chunks[0] + lastChunk := chunks[len(chunks)-1] - // Decode blob payload - payload, err := codec.DecodeBlob(blob) - if err != nil { - return fmt.Errorf("failed to decode blob for batch %d: %w", dbBatch.Index, err) - } + // Calculate total L1 messages in this batch + var totalL1MessagesInBatch uint64 + for _, chunk := range chunks { + totalL1MessagesInBatch += chunk.TotalL1MessagesPoppedInChunk + } - // Check L1 message queue hashes - if payload.PrevL1MessageQueueHash() != common.HexToHash(dbBatch.PrevL1MessageQueueHash) { - return fmt.Errorf("sanity check failed: prevL1MessageQueueHash mismatch for batch %d: decoded=%s, db=%s", - dbBatch.Index, payload.PrevL1MessageQueueHash().Hex(), dbBatch.PrevL1MessageQueueHash) - } - if payload.PostL1MessageQueueHash() != common.HexToHash(dbBatch.PostL1MessageQueueHash) { - return fmt.Errorf("sanity check failed: postL1MessageQueueHash mismatch for batch %d: decoded=%s, db=%s", - dbBatch.Index, payload.PostL1MessageQueueHash().Hex(), dbBatch.PostL1MessageQueueHash) - } + // If there were L1 messages processed before this batch, prev hash should not be zero + if firstChunk.TotalL1MessagesPoppedBefore > 0 && prevL1MsgQueueHash == (common.Hash{}) { + return fmt.Errorf("batch %d prev L1 message queue hash is zero but %d L1 messages were processed before", batchIndex, firstChunk.TotalL1MessagesPoppedBefore) + } - // Compare block count and block numbers - decodedBlocks := payload.Blocks() - if len(decodedBlocks) != len(batchBlocks) { - return fmt.Errorf("sanity check failed: block count mismatch in batch %d: decoded=%d, db=%d", dbBatch.Index, len(decodedBlocks), len(batchBlocks)) - } - for j, b := range batchBlocks { - if decodedBlocks[j].Number() != b.Header.Number.Uint64() { - return fmt.Errorf("sanity check failed: block number mismatch in batch %d block %d: decoded=%d, db=%d", - dbBatch.Index, j, decodedBlocks[j].Number(), b.Header.Number.Uint64()) - } - } + // If there are any L1 messages processed up to this batch, post hash should not be zero + totalL1MessagesProcessed := lastChunk.TotalL1MessagesPoppedBefore + lastChunk.TotalL1MessagesPoppedInChunk + if totalL1MessagesProcessed > 0 && postL1MsgQueueHash == (common.Hash{}) { + return fmt.Errorf("batch %d post L1 message queue hash is zero but %d L1 messages were processed in total", batchIndex, totalL1MessagesProcessed) + } + + // Prev and post queue hashes should be different if L1 messages were processed in this batch + if totalL1MessagesInBatch > 0 && prevL1MsgQueueHash == postL1MsgQueueHash { + return fmt.Errorf("batch %d has same prev and post L1 message queue hashes but processed %d L1 messages in this batch", batchIndex, totalL1MessagesInBatch) } - // All checks passed return nil } From be22866f4290b185574142c766d10036e45a7df3 Mon Sep 17 00:00:00 2001 From: colinlyguo Date: Wed, 6 Aug 2025 00:27:33 +0800 Subject: [PATCH 09/14] fix a comment --- rollup/internal/controller/relayer/l2_relayer_sanity.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rollup/internal/controller/relayer/l2_relayer_sanity.go b/rollup/internal/controller/relayer/l2_relayer_sanity.go index d81b161f67..8901e4145d 100644 --- a/rollup/internal/controller/relayer/l2_relayer_sanity.go +++ b/rollup/internal/controller/relayer/l2_relayer_sanity.go @@ -175,9 +175,9 @@ func (r *Layer2Relayer) validateDatabaseConsistency(batchesToValidate []*dbBatch return fmt.Errorf("failed to get previous chunk %d for continuity check: %w", firstChunk.Index-1, err) } - // Validate codec version consistency across all batches firstBatchCodecVersion := batchesToValidate[0].Batch.CodecVersion for i, batch := range batchesToValidate { + // Validate codec version consistency if batch.Batch.CodecVersion != firstBatchCodecVersion { return fmt.Errorf("batch %d has different codec version %d, expected %d", batch.Batch.Index, batch.Batch.CodecVersion, firstBatchCodecVersion) } From 3f9d9cc1b57103344fa1a6e8fb80e1b9bc49f0e6 Mon Sep 17 00:00:00 2001 From: colinlyguo Date: Wed, 6 Aug 2025 02:05:24 +0800 Subject: [PATCH 10/14] compare the batch hashes of the calldata + blob hash vs database --- .../controller/relayer/l2_relayer_sanity.go | 189 +++++++++--------- 1 file changed, 99 insertions(+), 90 deletions(-) diff --git a/rollup/internal/controller/relayer/l2_relayer_sanity.go b/rollup/internal/controller/relayer/l2_relayer_sanity.go index 8901e4145d..29805ab54d 100644 --- a/rollup/internal/controller/relayer/l2_relayer_sanity.go +++ b/rollup/internal/controller/relayer/l2_relayer_sanity.go @@ -2,9 +2,11 @@ package relayer import ( "fmt" + "math/big" "github.com/scroll-tech/da-codec/encoding" "github.com/scroll-tech/go-ethereum/common" + "github.com/scroll-tech/go-ethereum/core/types" "github.com/scroll-tech/go-ethereum/crypto/kzg4844" "scroll-tech/rollup/internal/orm" @@ -14,36 +16,24 @@ import ( // transaction data (calldata and blobs) by parsing them and comparing against database records. // This ensures the constructed transaction data is correct and consistent with the database state. func (r *Layer2Relayer) sanityChecksCommitBatchCodecV7CalldataAndBlobs(calldata []byte, blobs []*kzg4844.Blob) error { - if len(blobs) == 0 { - return fmt.Errorf("no blobs provided") - } - calldataInfo, err := r.parseCommitBatchesCalldata(calldata) if err != nil { return fmt.Errorf("failed to parse calldata: %w", err) } - batchesToValidate, firstBatch, lastBatch, err := r.getBatchesFromCalldata(calldataInfo) + batchesToValidate, err := r.getBatchesFromCalldata(calldataInfo) if err != nil { return fmt.Errorf("failed to get batches from database: %w", err) } - if len(blobs) != len(batchesToValidate) { - return fmt.Errorf("blob count mismatch: got %d blobs, expected %d batches", len(blobs), len(batchesToValidate)) - } - - if err := r.validateCalldataAgainstDatabase(calldataInfo, firstBatch, lastBatch); err != nil { - return fmt.Errorf("calldata validation failed: %w", err) + if err := r.validateCalldataAndBlobsAgainstDatabase(calldataInfo, blobs, batchesToValidate); err != nil { + return fmt.Errorf("calldata and blobs validation failed: %w", err) } if err := r.validateDatabaseConsistency(batchesToValidate); err != nil { return fmt.Errorf("database consistency validation failed: %w", err) } - if err := r.validateBlobsAgainstDatabase(blobs, batchesToValidate); err != nil { - return fmt.Errorf("blob validation failed: %w", err) - } - return nil } @@ -91,17 +81,17 @@ func (r *Layer2Relayer) parseCommitBatchesCalldata(calldata []byte) (*CalldataIn } // getBatchesFromCalldata retrieves the relevant batches from database based on calldata information -func (r *Layer2Relayer) getBatchesFromCalldata(info *CalldataInfo) ([]*dbBatchWithChunks, *orm.Batch, *orm.Batch, error) { +func (r *Layer2Relayer) getBatchesFromCalldata(info *CalldataInfo) ([]*dbBatchWithChunks, error) { // Get the parent batch to determine the starting point parentBatch, err := r.batchOrm.GetBatchByHash(r.ctx, info.ParentBatchHash.Hex()) if err != nil { - return nil, nil, nil, fmt.Errorf("failed to get parent batch by hash %s: %w", info.ParentBatchHash.Hex(), err) + return nil, fmt.Errorf("failed to get parent batch by hash %s: %w", info.ParentBatchHash.Hex(), err) } // Get the last batch to determine the ending point lastBatch, err := r.batchOrm.GetBatchByHash(r.ctx, info.LastBatchHash.Hex()) if err != nil { - return nil, nil, nil, fmt.Errorf("failed to get last batch by hash %s: %w", info.LastBatchHash.Hex(), err) + return nil, fmt.Errorf("failed to get last batch by hash %s: %w", info.LastBatchHash.Hex(), err) } // Get all batches in the range (parent+1 to last) @@ -110,20 +100,20 @@ func (r *Layer2Relayer) getBatchesFromCalldata(info *CalldataInfo) ([]*dbBatchWi // Check if the range is valid if firstBatchIndex > lastBatchIndex { - return nil, nil, nil, fmt.Errorf("no batches found in range: first index %d, last index %d", firstBatchIndex, lastBatchIndex) + return nil, fmt.Errorf("no batches found in range: first index %d, last index %d", firstBatchIndex, lastBatchIndex) } var batchesToValidate []*dbBatchWithChunks for batchIndex := firstBatchIndex; batchIndex <= lastBatchIndex; batchIndex++ { dbBatch, err := r.batchOrm.GetBatchByIndex(r.ctx, batchIndex) if err != nil { - return nil, nil, nil, fmt.Errorf("failed to get batch by index %d: %w", batchIndex, err) + return nil, fmt.Errorf("failed to get batch by index %d: %w", batchIndex, err) } // Get chunks for this batch dbChunks, err := r.chunkOrm.GetChunksInRange(r.ctx, dbBatch.StartChunkIndex, dbBatch.EndChunkIndex) if err != nil { - return nil, nil, nil, fmt.Errorf("failed to get chunks for batch %d: %w", batchIndex, err) + return nil, fmt.Errorf("failed to get chunks for batch %d: %w", batchIndex, err) } batchesToValidate = append(batchesToValidate, &dbBatchWithChunks{ @@ -132,30 +122,7 @@ func (r *Layer2Relayer) getBatchesFromCalldata(info *CalldataInfo) ([]*dbBatchWi }) } - // Get first batch for return - firstBatch := batchesToValidate[0].Batch - - return batchesToValidate, firstBatch, lastBatch, nil -} - -// validateCalldataAgainstDatabase validates calldata parameters against database records -func (r *Layer2Relayer) validateCalldataAgainstDatabase(info *CalldataInfo, firstBatch, lastBatch *orm.Batch) error { - // Validate codec version - if info.Version != uint8(firstBatch.CodecVersion) { - return fmt.Errorf("version mismatch: calldata=%d, db=%d", info.Version, firstBatch.CodecVersion) - } - - // Validate parent batch hash - if info.ParentBatchHash != common.HexToHash(firstBatch.ParentBatchHash) { - return fmt.Errorf("parentBatchHash mismatch: calldata=%s, db=%s", info.ParentBatchHash.Hex(), firstBatch.ParentBatchHash) - } - - // Validate last batch hash - if info.LastBatchHash != common.HexToHash(lastBatch.Hash) { - return fmt.Errorf("lastBatchHash mismatch: calldata=%s, db=%s", info.LastBatchHash.Hex(), lastBatch.Hash) - } - - return nil + return batchesToValidate, nil } // validateDatabaseConsistency performs comprehensive validation of database records @@ -328,10 +295,38 @@ func (r *Layer2Relayer) validateSingleChunkConsistency(chunk *orm.Chunk, prevChu return nil } -// validateBlobsAgainstDatabase validates blobs against database records -func (r *Layer2Relayer) validateBlobsAgainstDatabase(blobs []*kzg4844.Blob, batchesToValidate []*dbBatchWithChunks) error { - // Get codec for blob decoding +// validateCalldataAndBlobsAgainstDatabase validates calldata and blobs against database records +func (r *Layer2Relayer) validateCalldataAndBlobsAgainstDatabase(calldataInfo *CalldataInfo, blobs []*kzg4844.Blob, batchesToValidate []*dbBatchWithChunks) error { + // Validate blobs + if len(blobs) == 0 { + return fmt.Errorf("no blobs provided") + } + + // Validate blob count + if len(blobs) != len(batchesToValidate) { + return fmt.Errorf("blob count mismatch: got %d blobs, expected %d batches", len(blobs), len(batchesToValidate)) + } + + // Get first and last batches for validation, length check is already done above firstBatch := batchesToValidate[0].Batch + lastBatch := batchesToValidate[len(batchesToValidate)-1].Batch + + // Validate codec version + if calldataInfo.Version != uint8(firstBatch.CodecVersion) { + return fmt.Errorf("version mismatch: calldata=%d, db=%d", calldataInfo.Version, firstBatch.CodecVersion) + } + + // Validate parent batch hash + if calldataInfo.ParentBatchHash != common.HexToHash(firstBatch.ParentBatchHash) { + return fmt.Errorf("parentBatchHash mismatch: calldata=%s, db=%s", calldataInfo.ParentBatchHash.Hex(), firstBatch.ParentBatchHash) + } + + // Validate last batch hash + if calldataInfo.LastBatchHash != common.HexToHash(lastBatch.Hash) { + return fmt.Errorf("lastBatchHash mismatch: calldata=%s, db=%s", calldataInfo.LastBatchHash.Hex(), lastBatch.Hash) + } + + // Get codec for blob decoding codec, err := encoding.CodecFromVersion(encoding.CodecVersion(firstBatch.CodecVersion)) if err != nil { return fmt.Errorf("failed to get codec: %w", err) @@ -340,9 +335,7 @@ func (r *Layer2Relayer) validateBlobsAgainstDatabase(blobs []*kzg4844.Blob, batc // Validate each blob against its corresponding batch for i, blob := range blobs { dbBatch := batchesToValidate[i].Batch - dbChunks := batchesToValidate[i].Chunks - - if err := r.validateSingleBlobAgainstBatch(blob, dbBatch, dbChunks, codec); err != nil { + if err := r.validateSingleBlobAgainstBatch(calldataInfo, blob, dbBatch, codec); err != nil { return fmt.Errorf("blob validation failed for batch %d: %w", dbBatch.Index, err) } } @@ -351,53 +344,21 @@ func (r *Layer2Relayer) validateBlobsAgainstDatabase(blobs []*kzg4844.Blob, batc } // validateSingleBlobAgainstBatch validates a single blob against its batch data -func (r *Layer2Relayer) validateSingleBlobAgainstBatch(blob *kzg4844.Blob, dbBatch *orm.Batch, dbChunks []*orm.Chunk, codec encoding.Codec) error { - // Collect all blocks for the batch - var batchBlocks []*encoding.Block - for _, c := range dbChunks { - blocks, err := r.l2BlockOrm.GetL2BlocksInRange(r.ctx, c.StartBlockNumber, c.EndBlockNumber) - if err != nil { - return fmt.Errorf("failed to get blocks for chunk %d: %w", c.Index, err) - } - - if len(blocks) == 0 { - return fmt.Errorf("chunk %d has no blocks in range [%d, %d]", c.Index, c.StartBlockNumber, c.EndBlockNumber) - } - - // Verify block count matches expected range - expectedBlockCount := c.EndBlockNumber - c.StartBlockNumber + 1 - if len(blocks) != int(expectedBlockCount) { - return fmt.Errorf("chunk %d expected %d blocks but got %d", c.Index, expectedBlockCount, len(blocks)) - } - - batchBlocks = append(batchBlocks, blocks...) - } - +func (r *Layer2Relayer) validateSingleBlobAgainstBatch(calldataInfo *CalldataInfo, blob *kzg4844.Blob, dbBatch *orm.Batch, codec encoding.Codec) error { // Decode blob payload payload, err := codec.DecodeBlob(blob) if err != nil { return fmt.Errorf("failed to decode blob: %w", err) } - // Validate L1 message queue hashes - if payload.PrevL1MessageQueueHash() != common.HexToHash(dbBatch.PrevL1MessageQueueHash) { - return fmt.Errorf("prevL1MessageQueueHash mismatch: decoded=%s, db=%s", payload.PrevL1MessageQueueHash().Hex(), dbBatch.PrevL1MessageQueueHash) - } - - if payload.PostL1MessageQueueHash() != common.HexToHash(dbBatch.PostL1MessageQueueHash) { - return fmt.Errorf("postL1MessageQueueHash mismatch: decoded=%s, db=%s", payload.PostL1MessageQueueHash().Hex(), dbBatch.PostL1MessageQueueHash) - } - - // Validate block data - decodedBlocks := payload.Blocks() - if len(decodedBlocks) != len(batchBlocks) { - return fmt.Errorf("block count mismatch: decoded=%d, db=%d", len(decodedBlocks), len(batchBlocks)) + // Validate batch hash + daBatch, err := assembleDABatchFromPayload(calldataInfo, payload, dbBatch, codec) + if err != nil { + return fmt.Errorf("failed to assemble batch from payload: %w", err) } - for j, dbBlock := range batchBlocks { - if decodedBlocks[j].Number() != dbBlock.Header.Number.Uint64() { - return fmt.Errorf("block number mismatch at index %d: decoded=%d, db=%d", j, decodedBlocks[j].Number(), dbBlock.Header.Number.Uint64()) - } + if daBatch.Hash() != common.HexToHash(dbBatch.Hash) { + return fmt.Errorf("batch hash mismatch: decoded from blob=%s, db=%s", daBatch.Hash().Hex(), dbBatch.Hash) } return nil @@ -436,3 +397,51 @@ func (r *Layer2Relayer) validateMessageQueueConsistency(batchIndex uint64, chunk return nil } + +func assembleDABatchFromPayload(calldataInfo *CalldataInfo, payload encoding.DABlobPayload, dbBatch *orm.Batch, codec encoding.Codec) (encoding.DABatch, error) { + blocks, err := assembleBlocksFromPayload(payload) + if err != nil { + return nil, fmt.Errorf("failed to assemble blocks from payload batch_index=%d codec_version=%d parent_batch_hash=%s: %w", dbBatch.Index, dbBatch.CodecVersion, calldataInfo.ParentBatchHash.Hex(), err) + } + parentBatchHash := calldataInfo.ParentBatchHash + batch := &encoding.Batch{ + Index: dbBatch.Index, // The database provides only batch index, other fields are derived from blob payload + ParentBatchHash: parentBatchHash, + PrevL1MessageQueueHash: payload.PrevL1MessageQueueHash(), + PostL1MessageQueueHash: payload.PostL1MessageQueueHash(), + Blocks: blocks, + Chunks: []*encoding.Chunk{ // One chunk for this batch to pass sanity checks when building DABatch + { + Blocks: blocks, + PrevL1MessageQueueHash: payload.PrevL1MessageQueueHash(), + PostL1MessageQueueHash: payload.PostL1MessageQueueHash(), + }, + }, + } + daBatch, err := codec.NewDABatch(batch) + if err != nil { + return nil, fmt.Errorf("failed to build DABatch batch_index=%d codec_version=%d parent_batch_hash=%s: %w", dbBatch.Index, dbBatch.CodecVersion, calldataInfo.ParentBatchHash.Hex(), err) + } + return daBatch, nil +} + +func assembleBlocksFromPayload(payload encoding.DABlobPayload) ([]*encoding.Block, error) { + daBlocks := payload.Blocks() + txss := payload.Transactions() + if len(daBlocks) != len(txss) { + return nil, fmt.Errorf("mismatched number of blocks and transactions: %d blocks, %d transactions", len(daBlocks), len(txss)) + } + blocks := make([]*encoding.Block, len(daBlocks)) + for i := range daBlocks { + blocks[i] = &encoding.Block{ + Header: &types.Header{ + Number: new(big.Int).SetUint64(daBlocks[i].Number()), + Time: daBlocks[i].Timestamp(), + BaseFee: daBlocks[i].BaseFee(), + GasLimit: daBlocks[i].GasLimit(), + }, + Transactions: encoding.TxsToTxsData(txss[i]), + } + } + return blocks, nil +} From 91721497c64aafef32b68623c8040ac939d1feca Mon Sep 17 00:00:00 2001 From: colinlyguo Date: Wed, 6 Aug 2025 02:54:26 +0800 Subject: [PATCH 11/14] tweak a naming --- rollup/internal/controller/relayer/l2_relayer_sanity.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/rollup/internal/controller/relayer/l2_relayer_sanity.go b/rollup/internal/controller/relayer/l2_relayer_sanity.go index 29805ab54d..68a37cd7cb 100644 --- a/rollup/internal/controller/relayer/l2_relayer_sanity.go +++ b/rollup/internal/controller/relayer/l2_relayer_sanity.go @@ -427,9 +427,9 @@ func assembleDABatchFromPayload(calldataInfo *CalldataInfo, payload encoding.DAB func assembleBlocksFromPayload(payload encoding.DABlobPayload) ([]*encoding.Block, error) { daBlocks := payload.Blocks() - txss := payload.Transactions() - if len(daBlocks) != len(txss) { - return nil, fmt.Errorf("mismatched number of blocks and transactions: %d blocks, %d transactions", len(daBlocks), len(txss)) + txns := payload.Transactions() + if len(daBlocks) != len(txns) { + return nil, fmt.Errorf("mismatched number of blocks and transactions: %d blocks, %d transactions", len(daBlocks), len(txns)) } blocks := make([]*encoding.Block, len(daBlocks)) for i := range daBlocks { @@ -440,7 +440,7 @@ func assembleBlocksFromPayload(payload encoding.DABlobPayload) ([]*encoding.Bloc BaseFee: daBlocks[i].BaseFee(), GasLimit: daBlocks[i].GasLimit(), }, - Transactions: encoding.TxsToTxsData(txss[i]), + Transactions: encoding.TxsToTxsData(txns[i]), } } return blocks, nil From 2d4f792a6640113900ce6b6660cb9379b6c3c98f Mon Sep 17 00:00:00 2001 From: colinlyguo Date: Wed, 6 Aug 2025 14:09:15 +0800 Subject: [PATCH 12/14] fix a bug --- .../controller/relayer/l2_relayer_sanity.go | 20 ++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/rollup/internal/controller/relayer/l2_relayer_sanity.go b/rollup/internal/controller/relayer/l2_relayer_sanity.go index 68a37cd7cb..d87052f242 100644 --- a/rollup/internal/controller/relayer/l2_relayer_sanity.go +++ b/rollup/internal/controller/relayer/l2_relayer_sanity.go @@ -205,6 +205,9 @@ func (r *Layer2Relayer) validateSingleBatchConsistency(batch *dbBatchWithChunks, if batch.Batch.Index != prevBatch.Batch.Index+1 { return fmt.Errorf("batch index is not sequential: prev batch index %d, current batch index %d", prevBatch.Batch.Index, batch.Batch.Index) } + if parentBatchHash != common.HexToHash(prevBatch.Batch.Hash) { + return fmt.Errorf("parent batch hash does not match previous batch hash: expected %s, got %s", prevBatch.Batch.Hash, batch.Batch.ParentBatchHash) + } } else { // For the first batch, verify continuity with parent batch from database parentBatch, err := r.batchOrm.GetBatchByHash(r.ctx, batch.Batch.ParentBatchHash) @@ -335,7 +338,7 @@ func (r *Layer2Relayer) validateCalldataAndBlobsAgainstDatabase(calldataInfo *Ca // Validate each blob against its corresponding batch for i, blob := range blobs { dbBatch := batchesToValidate[i].Batch - if err := r.validateSingleBlobAgainstBatch(calldataInfo, blob, dbBatch, codec); err != nil { + if err := r.validateSingleBlobAgainstBatch(blob, dbBatch, codec); err != nil { return fmt.Errorf("blob validation failed for batch %d: %w", dbBatch.Index, err) } } @@ -344,7 +347,7 @@ func (r *Layer2Relayer) validateCalldataAndBlobsAgainstDatabase(calldataInfo *Ca } // validateSingleBlobAgainstBatch validates a single blob against its batch data -func (r *Layer2Relayer) validateSingleBlobAgainstBatch(calldataInfo *CalldataInfo, blob *kzg4844.Blob, dbBatch *orm.Batch, codec encoding.Codec) error { +func (r *Layer2Relayer) validateSingleBlobAgainstBatch(blob *kzg4844.Blob, dbBatch *orm.Batch, codec encoding.Codec) error { // Decode blob payload payload, err := codec.DecodeBlob(blob) if err != nil { @@ -352,7 +355,7 @@ func (r *Layer2Relayer) validateSingleBlobAgainstBatch(calldataInfo *CalldataInf } // Validate batch hash - daBatch, err := assembleDABatchFromPayload(calldataInfo, payload, dbBatch, codec) + daBatch, err := assembleDABatchFromPayload(payload, dbBatch, codec) if err != nil { return fmt.Errorf("failed to assemble batch from payload: %w", err) } @@ -398,15 +401,14 @@ func (r *Layer2Relayer) validateMessageQueueConsistency(batchIndex uint64, chunk return nil } -func assembleDABatchFromPayload(calldataInfo *CalldataInfo, payload encoding.DABlobPayload, dbBatch *orm.Batch, codec encoding.Codec) (encoding.DABatch, error) { +func assembleDABatchFromPayload(payload encoding.DABlobPayload, dbBatch *orm.Batch, codec encoding.Codec) (encoding.DABatch, error) { blocks, err := assembleBlocksFromPayload(payload) if err != nil { - return nil, fmt.Errorf("failed to assemble blocks from payload batch_index=%d codec_version=%d parent_batch_hash=%s: %w", dbBatch.Index, dbBatch.CodecVersion, calldataInfo.ParentBatchHash.Hex(), err) + return nil, fmt.Errorf("failed to assemble blocks from payload batch_index=%d codec_version=%d parent_batch_hash=%s: %w", dbBatch.Index, dbBatch.CodecVersion, dbBatch.ParentBatchHash, err) } - parentBatchHash := calldataInfo.ParentBatchHash batch := &encoding.Batch{ - Index: dbBatch.Index, // The database provides only batch index, other fields are derived from blob payload - ParentBatchHash: parentBatchHash, + Index: dbBatch.Index, // The database provides only batch index, other fields are derived from blob payload + ParentBatchHash: common.HexToHash(dbBatch.ParentBatchHash), // The first batch's parent hash is verified with calldata, subsequent batches are linked via dbBatch.ParentBatchHash and verified in database consistency checks PrevL1MessageQueueHash: payload.PrevL1MessageQueueHash(), PostL1MessageQueueHash: payload.PostL1MessageQueueHash(), Blocks: blocks, @@ -420,7 +422,7 @@ func assembleDABatchFromPayload(calldataInfo *CalldataInfo, payload encoding.DAB } daBatch, err := codec.NewDABatch(batch) if err != nil { - return nil, fmt.Errorf("failed to build DABatch batch_index=%d codec_version=%d parent_batch_hash=%s: %w", dbBatch.Index, dbBatch.CodecVersion, calldataInfo.ParentBatchHash.Hex(), err) + return nil, fmt.Errorf("failed to build DABatch batch_index=%d codec_version=%d parent_batch_hash=%s: %w", dbBatch.Index, dbBatch.CodecVersion, dbBatch.ParentBatchHash, err) } return daBatch, nil } From 871728190c18ca112a9ac3c2d9d8e8b97a2957e4 Mon Sep 17 00:00:00 2001 From: colinlyguo Date: Wed, 6 Aug 2025 18:16:24 +0800 Subject: [PATCH 13/14] remove proof sanity check inside transaction construction function --- rollup/internal/controller/relayer/l2_relayer.go | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/rollup/internal/controller/relayer/l2_relayer.go b/rollup/internal/controller/relayer/l2_relayer.go index e966a9a701..fcfffdab24 100644 --- a/rollup/internal/controller/relayer/l2_relayer.go +++ b/rollup/internal/controller/relayer/l2_relayer.go @@ -1058,11 +1058,6 @@ func (r *Layer2Relayer) constructFinalizeBundlePayloadCodecV7(dbBatch *orm.Batch return nil, fmt.Errorf("batch %d state root is zero", dbBatch.Index) } - // Check proof if present - if aggProof != nil && len(aggProof.Proof()) == 0 { - return nil, fmt.Errorf("aggregate proof is empty") - } - if aggProof != nil { // finalizeBundle with proof. calldata, packErr := r.l1RollupABI.Pack( "finalizeBundlePostEuclidV2", @@ -1095,11 +1090,6 @@ func (r *Layer2Relayer) constructFinalizeBundlePayloadCodecV7(dbBatch *orm.Batch } func (r *Layer2Relayer) constructFinalizeBundlePayloadValidium(dbBatch *orm.Batch, endChunk *orm.Chunk, aggProof *message.OpenVMBundleProof) ([]byte, error) { - // Check proof if present - if aggProof != nil && len(aggProof.Proof()) == 0 { - return nil, fmt.Errorf("aggregate proof is empty") - } - log.Info("Packing validium finalizeBundle", "batchHeaderLength", len(dbBatch.BatchHeader), "codecVersion", dbBatch.CodecVersion, "totalL1Messages", endChunk.TotalL1MessagesPoppedBefore+endChunk.TotalL1MessagesPoppedInChunk, "stateRoot", dbBatch.StateRoot, "withdrawRoot", dbBatch.WithdrawRoot, "withProof", aggProof != nil) var proof []byte From dd0ddbc6ea49695d451b66e64509b696c784cfae Mon Sep 17 00:00:00 2001 From: colinlyguo Date: Thu, 7 Aug 2025 08:17:26 +0000 Subject: [PATCH 14/14] =?UTF-8?q?chore:=20auto=20version=20bump=E2=80=89[b?= =?UTF-8?q?ot]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- common/version/version.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/version/version.go b/common/version/version.go index cf4f030ac1..00c3f74b0e 100644 --- a/common/version/version.go +++ b/common/version/version.go @@ -5,7 +5,7 @@ import ( "runtime/debug" ) -var tag = "v4.5.38" +var tag = "v4.5.39" var commit = func() string { if info, ok := debug.ReadBuildInfo(); ok {