From 1c22307f08306039c39311dc872cdb9e8ecef6c4 Mon Sep 17 00:00:00 2001 From: colin <102356659+colinlyguo@users.noreply.github.com> Date: Thu, 21 Nov 2024 11:36:40 +0700 Subject: [PATCH] refactor(rollup-relayer): remove outdated logic (#1561) Co-authored-by: colinlyguo --- common/utils/flags.go | 7 + common/version/version.go | 2 +- rollup/cmd/rollup_relayer/app/app.go | 10 +- .../internal/controller/relayer/l2_relayer.go | 359 +----- .../controller/relayer/l2_relayer_test.go | 222 +--- .../controller/relayer/relayer_test.go | 3 - .../controller/watcher/batch_proposer.go | 11 +- .../controller/watcher/batch_proposer_test.go | 1006 ++--------------- .../controller/watcher/bundle_proposer.go | 14 +- .../watcher/bundle_proposer_test.go | 101 +- .../controller/watcher/chunk_proposer.go | 10 +- .../controller/watcher/chunk_proposer_test.go | 582 +--------- .../controller/watcher/watcher_test.go | 26 +- rollup/internal/orm/orm_test.go | 16 +- rollup/tests/bridge_test.go | 3 +- rollup/tests/process_start_test.go | 4 +- rollup/tests/rollup_test.go | 233 +--- 17 files changed, 273 insertions(+), 2336 deletions(-) diff --git a/common/utils/flags.go b/common/utils/flags.go index e6e4f64222..ef5919d54a 100644 --- a/common/utils/flags.go +++ b/common/utils/flags.go @@ -21,6 +21,7 @@ var ( // RollupRelayerFlags contains flags only used in rollup-relayer RollupRelayerFlags = []cli.Flag{ &ImportGenesisFlag, + &MinCodecVersionFlag, } // ConfigFileFlag load json type config file. ConfigFileFlag = cli.StringFlag{ @@ -90,4 +91,10 @@ var ( Usage: "Genesis file of the network", Value: "./conf/genesis.json", } + // MinCodecVersionFlag defines the minimum codec version required for the chunk/batch/bundle proposers + MinCodecVersionFlag = cli.UintFlag{ + Name: "min-codec-version", + Usage: "Minimum required codec version for the chunk/batch/bundle proposers", + Required: true, + } ) diff --git a/common/version/version.go b/common/version/version.go index 01f8a6df54..0f9a7cd89a 100644 --- a/common/version/version.go +++ b/common/version/version.go @@ -5,7 +5,7 @@ import ( "runtime/debug" ) -var tag = "v4.4.76" +var tag = "v4.4.77" var commit = func() string { if info, ok := debug.ReadBuildInfo(); ok { diff --git a/rollup/cmd/rollup_relayer/app/app.go b/rollup/cmd/rollup_relayer/app/app.go index 4e086bd4ef..080f27cdf7 100644 --- a/rollup/cmd/rollup_relayer/app/app.go +++ b/rollup/cmd/rollup_relayer/app/app.go @@ -8,6 +8,7 @@ import ( "time" "github.com/prometheus/client_golang/prometheus" + "github.com/scroll-tech/da-codec/encoding" "github.com/scroll-tech/go-ethereum/ethclient" "github.com/scroll-tech/go-ethereum/log" "github.com/urfave/cli/v2" @@ -84,9 +85,10 @@ func action(ctx *cli.Context) error { log.Crit("failed to create l2 relayer", "config file", cfgFile, "error", err) } - chunkProposer := watcher.NewChunkProposer(subCtx, cfg.L2Config.ChunkProposerConfig, genesis.Config, db, registry) - batchProposer := watcher.NewBatchProposer(subCtx, cfg.L2Config.BatchProposerConfig, genesis.Config, db, registry) - bundleProposer := watcher.NewBundleProposer(subCtx, cfg.L2Config.BundleProposerConfig, genesis.Config, db, registry) + minCodecVersion := encoding.CodecVersion(ctx.Uint(utils.MinCodecVersionFlag.Name)) + chunkProposer := watcher.NewChunkProposer(subCtx, cfg.L2Config.ChunkProposerConfig, minCodecVersion, genesis.Config, db, registry) + batchProposer := watcher.NewBatchProposer(subCtx, cfg.L2Config.BatchProposerConfig, minCodecVersion, genesis.Config, db, registry) + bundleProposer := watcher.NewBundleProposer(subCtx, cfg.L2Config.BundleProposerConfig, minCodecVersion, genesis.Config, db, registry) l2watcher := watcher.NewL2WatcherClient(subCtx, l2client, cfg.L2Config.Confirmations, cfg.L2Config.L2MessageQueueAddress, cfg.L2Config.WithdrawTrieRootSlot, genesis.Config, db, registry) @@ -108,8 +110,6 @@ func action(ctx *cli.Context) error { go utils.Loop(subCtx, 2*time.Second, l2relayer.ProcessPendingBatches) - go utils.Loop(subCtx, 15*time.Second, l2relayer.ProcessCommittedBatches) - go utils.Loop(subCtx, 15*time.Second, l2relayer.ProcessPendingBundles) // Finish start all rollup relayer functions. diff --git a/rollup/internal/controller/relayer/l2_relayer.go b/rollup/internal/controller/relayer/l2_relayer.go index 879c4da9a7..c450e22fe3 100644 --- a/rollup/internal/controller/relayer/l2_relayer.go +++ b/rollup/internal/controller/relayer/l2_relayer.go @@ -394,6 +394,14 @@ func (r *Layer2Relayer) ProcessPendingBatches() { return } + // check codec version + for _, dbChunk := range dbChunks { + if dbBatch.CodecVersion != dbChunk.CodecVersion { + log.Error("batch codec version is different from chunk codec version", "batch index", dbBatch.Index, "chunk index", dbChunk.Index, "batch codec version", dbBatch.CodecVersion, "chunk codec version", dbChunk.CodecVersion) + return + } + } + chunks := make([]*encoding.Chunk, len(dbChunks)) for i, c := range dbChunks { blocks, getErr := r.l2BlockOrm.GetL2BlocksInRange(r.ctx, c.StartBlockNumber, c.EndBlockNumber) @@ -415,20 +423,19 @@ func (r *Layer2Relayer) ProcessPendingBatches() { return } + if dbParentBatch.CodecVersion > dbBatch.CodecVersion { + log.Error("parent batch codec version is greater than current batch codec version", "index", dbBatch.Index, "hash", dbBatch.Hash, "parent codec version", dbParentBatch.CodecVersion, "current codec version", dbBatch.CodecVersion) + return + } + var calldata []byte var blob *kzg4844.Blob codecVersion := encoding.CodecVersion(dbBatch.CodecVersion) switch codecVersion { - case encoding.CodecV0, encoding.CodecV1, encoding.CodecV2: - calldata, blob, err = r.constructCommitBatchPayloadCodecV0AndV1AndV2(dbBatch, dbParentBatch, dbChunks, chunks) - if err != nil { - log.Error("failed to construct commitBatch payload for V0/V1/V2", "codecVersion", codecVersion, "index", dbBatch.Index, "err", err) - return - } - case encoding.CodecV3, encoding.CodecV4: - calldata, blob, err = r.constructCommitBatchPayloadCodecV3AndV4(dbBatch, dbParentBatch, dbChunks, chunks) + case encoding.CodecV4: + calldata, blob, err = r.constructCommitBatchPayloadCodecV4(dbBatch, dbParentBatch, dbChunks, chunks) if err != nil { - log.Error("failed to construct commitBatchWithBlobProof payload for V3/V4", "codecVersion", codecVersion, "index", dbBatch.Index, "err", err) + log.Error("failed to construct commitBatchWithBlobProof payload for V4", "codecVersion", codecVersion, "index", dbBatch.Index, "err", err) return } default: @@ -488,69 +495,6 @@ func (r *Layer2Relayer) ProcessPendingBatches() { } } -// ProcessCommittedBatches submit proof to layer 1 rollup contract -func (r *Layer2Relayer) ProcessCommittedBatches() { - // retrieves the earliest batch whose rollup status is 'committed' - fields := map[string]interface{}{ - "rollup_status": types.RollupCommitted, - } - orderByList := []string{"index ASC"} - limit := 1 - batches, err := r.batchOrm.GetBatches(r.ctx, fields, orderByList, limit) - if err != nil { - log.Error("Failed to fetch committed L2 batches", "err", err) - return - } - if len(batches) != 1 { - log.Warn("Unexpected result for GetBlockBatches", "number of batches", len(batches)) - return - } - - r.metrics.rollupL2RelayerProcessCommittedBatchesTotal.Inc() - - batch := batches[0] - status := types.ProvingStatus(batch.ProvingStatus) - switch status { - case types.ProvingTaskUnassigned, types.ProvingTaskAssigned: - if batch.CommittedAt == nil { - log.Error("batch.CommittedAt is nil", "index", batch.Index, "hash", batch.Hash) - return - } - - if r.cfg.EnableTestEnvBypassFeatures && utils.NowUTC().Sub(*batch.CommittedAt) > time.Duration(r.cfg.FinalizeBatchWithoutProofTimeoutSec)*time.Second { - if err := r.finalizeBatch(batch, false); err != nil { - log.Error("Failed to finalize timeout batch without proof", "index", batch.Index, "hash", batch.Hash, "err", err) - } - } - - case types.ProvingTaskVerified: - r.metrics.rollupL2RelayerProcessCommittedBatchesFinalizedTotal.Inc() - if err := r.finalizeBatch(batch, true); err != nil { - log.Error("Failed to finalize batch with proof", "index", batch.Index, "hash", batch.Hash, "err", err) - } - - case types.ProvingTaskFailed: - // We were unable to prove this batch. There are two possibilities: - // (a) Prover bug. In this case, we should fix and redeploy the prover. - // In the meantime, we continue to commit batches to L1 as well as - // proposing and proving chunks and batches. - // (b) Unprovable batch, e.g. proof overflow. In this case we need to - // stop the ledger, fix the limit, revert all the violating blocks, - // chunks and batches and all subsequent ones, and resume, i.e. this - // case requires manual resolution. - log.Error( - "batch proving failed", - "Index", batch.Index, - "Hash", batch.Hash, - "ProvedAt", batch.ProvedAt, - "ProofTimeSec", batch.ProofTimeSec, - ) - - default: - log.Error("encounter unreachable case in ProcessCommittedBatches", "proving status", status) - } -} - // ProcessPendingBundles submits proof to layer 1 rollup contract func (r *Layer2Relayer) ProcessPendingBundles() { r.metrics.rollupL2RelayerProcessPendingBundlesTotal.Inc() @@ -615,135 +559,42 @@ func (r *Layer2Relayer) ProcessPendingBundles() { } } -func (r *Layer2Relayer) finalizeBatch(dbBatch *orm.Batch, withProof bool) error { - // Check batch status before sending `finalizeBatch` tx. - if r.cfg.ChainMonitor.Enabled { - var batchStatus bool - batchStatus, err := r.getBatchStatusByIndex(dbBatch) +func (r *Layer2Relayer) finalizeBundle(bundle *orm.Bundle, withProof bool) error { + // Check if current bundle codec version is not less than the preceding one + if bundle.StartBatchIndex > 0 { + prevBatch, err := r.batchOrm.GetBatchByIndex(r.ctx, bundle.StartBatchIndex-1) if err != nil { - r.metrics.rollupL2ChainMonitorLatestFailedCall.Inc() - log.Warn("failed to get batch status, please check chain_monitor api server", "batch_index", dbBatch.Index, "err", err) + log.Error("failed to get previous batch", + "current bundle index", bundle.Index, + "start batch index", bundle.StartBatchIndex, + "error", err) return err } - if !batchStatus { - r.metrics.rollupL2ChainMonitorLatestFailedBatchStatus.Inc() - log.Error("the batch status is false, stop finalize batch and check the reason", "batch_index", dbBatch.Index) - return errors.New("the batch status is false") + if bundle.CodecVersion < prevBatch.CodecVersion { + log.Error("current bundle codec version is less than the preceding batch", + "current bundle index", bundle.Index, + "current codec version", bundle.CodecVersion, + "prev batch index", prevBatch.Index, + "prev codec version", prevBatch.CodecVersion) + return errors.New("current bundle codec version cannot be less than the preceding batch") } } - if dbBatch.Index == 0 { - return errors.New("invalid args: batch index is 0, should only happen in finalizing genesis batch") - } - - dbParentBatch, getErr := r.batchOrm.GetBatchByIndex(r.ctx, dbBatch.Index-1) - if getErr != nil { - return fmt.Errorf("failed to get batch, index: %d, err: %w", dbBatch.Index-1, getErr) - } - - dbChunks, err := r.chunkOrm.GetChunksInRange(r.ctx, dbBatch.StartChunkIndex, dbBatch.EndChunkIndex) - if err != nil { - return fmt.Errorf("failed to fetch chunks: %w", err) - } - - var aggProof *message.BatchProof - if withProof { - aggProof, getErr = r.batchOrm.GetVerifiedProofByHash(r.ctx, dbBatch.Hash) + // Check batch status before sending `finalizeBundle` tx. + for batchIndex := bundle.StartBatchIndex; batchIndex <= bundle.EndBatchIndex; batchIndex++ { + tmpBatch, getErr := r.batchOrm.GetBatchByIndex(r.ctx, batchIndex) if getErr != nil { - return fmt.Errorf("failed to get verified proof by hash, index: %d, err: %w", dbBatch.Index, getErr) - } - - if err = aggProof.SanityCheck(); err != nil { - return fmt.Errorf("failed to check agg_proof sanity, index: %d, err: %w", dbBatch.Index, err) - } - } - - var calldata []byte - codecVersion := encoding.GetCodecVersion(r.chainCfg, dbChunks[0].StartBlockNumber, dbChunks[0].StartBlockTime) - - switch codecVersion { - case encoding.CodecV0: - log.Info("Start to roll up zk proof", "batch hash", dbBatch.Hash) - calldata, err = r.constructFinalizeBatchPayloadCodecV0(dbBatch, dbParentBatch, aggProof) - if err != nil { - return fmt.Errorf("failed to construct finalizeBatch payload codecv0, index: %v, err: %w", dbBatch.Index, err) - } - - case encoding.CodecV1, encoding.CodecV2: - log.Info("Start to roll up zk proof", "batch hash", dbBatch.Hash) - chunks := make([]*encoding.Chunk, len(dbChunks)) - for i, c := range dbChunks { - blocks, dbErr := r.l2BlockOrm.GetL2BlocksInRange(r.ctx, c.StartBlockNumber, c.EndBlockNumber) - if dbErr != nil { - return fmt.Errorf("failed to fetch blocks: %w", dbErr) - } - chunks[i] = &encoding.Chunk{Blocks: blocks} - } - calldata, err = r.constructFinalizeBatchPayloadCodecV1AndV2(dbBatch, dbParentBatch, dbChunks, chunks, aggProof) - if err != nil { - return fmt.Errorf("failed to construct finalizeBatch payload codecv1, index: %v, err: %w", dbBatch.Index, err) + log.Error("failed to get batch by index", "batch index", batchIndex, "error", getErr) + return getErr } - case encoding.CodecV3, encoding.CodecV4: - log.Debug("using finalizeBundle instead", "index", dbBatch.Index, "codec version", codecVersion) - return nil - - default: - return fmt.Errorf("unsupported codec version: %v", codecVersion) - } - - txHash, err := r.finalizeSender.SendTransaction(dbBatch.Hash, &r.cfg.RollupContractAddress, calldata, nil, 0) - if err != nil { - log.Error( - "finalizeBatch in layer1 failed", - "with proof", withProof, - "index", dbBatch.Index, - "hash", dbBatch.Hash, - "RollupContractAddress", r.cfg.RollupContractAddress, - "err", err, - "calldata", common.Bytes2Hex(calldata), - ) - return err - } - - log.Info("finalizeBatch in layer1", "with proof", withProof, "index", dbBatch.Index, "batch hash", dbBatch.Hash, "tx hash", txHash.String()) - - // Updating rollup status in database. - if err := r.batchOrm.UpdateFinalizeTxHashAndRollupStatus(r.ctx, dbBatch.Hash, txHash.String(), types.RollupFinalizing); err != nil { - log.Error("UpdateFinalizeTxHashAndRollupStatus failed", "index", dbBatch.Index, "batch hash", dbBatch.Hash, "tx hash", txHash.String(), "err", err) - return err - } - - // Updating the proving status when finalizing without proof, thus the coordinator could omit this task. - // it isn't a necessary step, so don't put in a transaction with UpdateFinalizeTxHashAndRollupStatus - if !withProof { - txErr := r.db.Transaction(func(dbTX *gorm.DB) error { - if updateErr := r.batchOrm.UpdateProvingStatus(r.ctx, dbBatch.Hash, types.ProvingTaskVerified, dbTX); updateErr != nil { - return updateErr - } - if updateErr := r.chunkOrm.UpdateProvingStatusByBatchHash(r.ctx, dbBatch.Hash, types.ProvingTaskVerified, dbTX); updateErr != nil { - return updateErr - } - return nil - }) - if txErr != nil { - log.Error("Updating chunk and batch proving status when finalizing without proof failure", "batchHash", dbBatch.Hash, "err", txErr) + // check codec version + if tmpBatch.CodecVersion != bundle.CodecVersion { + log.Error("bundle codec version is different from batch codec version", "bundle index", bundle.Index, "batch index", tmpBatch.Index, "bundle codec version", bundle.CodecVersion, "batch codec version", tmpBatch.CodecVersion) + return errors.New("bundle codec version is different from batch codec version") } - } - - r.metrics.rollupL2RelayerProcessCommittedBatchesFinalizedSuccessTotal.Inc() - return nil -} -func (r *Layer2Relayer) finalizeBundle(bundle *orm.Bundle, withProof bool) error { - // Check batch status before sending `finalizeBundle` tx. - if r.cfg.ChainMonitor.Enabled { - for batchIndex := bundle.StartBatchIndex; batchIndex <= bundle.EndBatchIndex; batchIndex++ { - tmpBatch, getErr := r.batchOrm.GetBatchByIndex(r.ctx, batchIndex) - if getErr != nil { - log.Error("failed to get batch by index", "batch index", batchIndex, "error", getErr) - return getErr - } + if r.cfg.ChainMonitor.Enabled { batchStatus, getErr := r.getBatchStatusByIndex(tmpBatch) if getErr != nil { r.metrics.rollupL2ChainMonitorLatestFailedCall.Inc() @@ -776,7 +627,7 @@ func (r *Layer2Relayer) finalizeBundle(bundle *orm.Bundle, withProof bool) error } } - calldata, err := r.constructFinalizeBundlePayloadCodecV3AndV4(dbBatch, aggProof) + calldata, err := r.constructFinalizeBundlePayloadCodecV4(dbBatch, aggProof) if err != nil { return fmt.Errorf("failed to construct finalizeBundle payload codecv3, index: %v, err: %w", dbBatch.Index, err) } @@ -982,45 +833,7 @@ func (r *Layer2Relayer) handleL2RollupRelayerConfirmLoop(ctx context.Context) { } } -func (r *Layer2Relayer) constructCommitBatchPayloadCodecV0AndV1AndV2(dbBatch *orm.Batch, dbParentBatch *orm.Batch, dbChunks []*orm.Chunk, chunks []*encoding.Chunk) ([]byte, *kzg4844.Blob, error) { - codec, err := encoding.CodecFromVersion(encoding.CodecVersion(dbBatch.CodecVersion)) - if err != nil { - return nil, nil, fmt.Errorf("failed to get codec from version %d, err: %w", dbBatch.CodecVersion, err) - } - - batch := &encoding.Batch{ - Index: dbBatch.Index, - TotalL1MessagePoppedBefore: dbChunks[0].TotalL1MessagesPoppedBefore, - ParentBatchHash: common.HexToHash(dbParentBatch.Hash), - Chunks: chunks, - } - - daBatch, createErr := codec.NewDABatch(batch) - if createErr != nil { - return nil, nil, fmt.Errorf("failed to create DA batch: %w", createErr) - } - - encodedChunks := make([][]byte, len(dbChunks)) - for i, c := range dbChunks { - daChunk, createErr := codec.NewDAChunk(chunks[i], c.TotalL1MessagesPoppedBefore) - if createErr != nil { - return nil, nil, fmt.Errorf("failed to create DA chunk: %w", createErr) - } - daChunkBytes, encodeErr := daChunk.Encode() - if encodeErr != nil { - return nil, nil, fmt.Errorf("failed to encode DA chunk: %w", encodeErr) - } - encodedChunks[i] = daChunkBytes - } - - calldata, packErr := r.l1RollupABI.Pack("commitBatch", daBatch.Version(), dbParentBatch.BatchHeader, encodedChunks, daBatch.SkippedL1MessageBitmap()) - if packErr != nil { - return nil, nil, fmt.Errorf("failed to pack commitBatch: %w", packErr) - } - return calldata, daBatch.Blob(), nil -} - -func (r *Layer2Relayer) constructCommitBatchPayloadCodecV3AndV4(dbBatch *orm.Batch, dbParentBatch *orm.Batch, dbChunks []*orm.Chunk, chunks []*encoding.Chunk) ([]byte, *kzg4844.Blob, error) { +func (r *Layer2Relayer) constructCommitBatchPayloadCodecV4(dbBatch *orm.Batch, dbParentBatch *orm.Batch, dbChunks []*orm.Chunk, chunks []*encoding.Chunk) ([]byte, *kzg4844.Blob, error) { batch := &encoding.Batch{ Index: dbBatch.Index, TotalL1MessagePoppedBefore: dbChunks[0].TotalL1MessagesPoppedBefore, @@ -1062,91 +875,7 @@ func (r *Layer2Relayer) constructCommitBatchPayloadCodecV3AndV4(dbBatch *orm.Bat return calldata, daBatch.Blob(), nil } -func (r *Layer2Relayer) constructFinalizeBatchPayloadCodecV0(dbBatch *orm.Batch, dbParentBatch *orm.Batch, aggProof *message.BatchProof) ([]byte, error) { - if aggProof != nil { // finalizeBatch with proof. - calldata, packErr := r.l1RollupABI.Pack( - "finalizeBatchWithProof", - dbBatch.BatchHeader, - common.HexToHash(dbParentBatch.StateRoot), - common.HexToHash(dbBatch.StateRoot), - common.HexToHash(dbBatch.WithdrawRoot), - aggProof.Proof, - ) - if packErr != nil { - return nil, fmt.Errorf("failed to pack finalizeBatchWithProof: %w", packErr) - } - return calldata, nil - } - - // finalizeBatch without proof. - calldata, packErr := r.l1RollupABI.Pack( - "finalizeBatch", - dbBatch.BatchHeader, - common.HexToHash(dbParentBatch.StateRoot), - common.HexToHash(dbBatch.StateRoot), - common.HexToHash(dbBatch.WithdrawRoot), - ) - if packErr != nil { - return nil, fmt.Errorf("failed to pack finalizeBatch: %w", packErr) - } - return calldata, nil -} - -func (r *Layer2Relayer) constructFinalizeBatchPayloadCodecV1AndV2(dbBatch *orm.Batch, dbParentBatch *orm.Batch, dbChunks []*orm.Chunk, chunks []*encoding.Chunk, aggProof *message.BatchProof) ([]byte, error) { - batch := &encoding.Batch{ - Index: dbBatch.Index, - TotalL1MessagePoppedBefore: dbChunks[0].TotalL1MessagesPoppedBefore, - ParentBatchHash: common.HexToHash(dbParentBatch.Hash), - Chunks: chunks, - } - - codec, err := encoding.CodecFromVersion(encoding.CodecVersion(dbBatch.CodecVersion)) - if err != nil { - return nil, fmt.Errorf("failed to get codec from version %d, err: %w", dbBatch.CodecVersion, err) - } - - daBatch, createErr := codec.NewDABatch(batch) - if createErr != nil { - return nil, fmt.Errorf("failed to create DA batch: %w", createErr) - } - - blobDataProof, getErr := daBatch.BlobDataProofForPointEvaluation() - if getErr != nil { - return nil, fmt.Errorf("failed to get blob data proof: %w", getErr) - } - - if aggProof != nil { // finalizeBatch4844 with proof. - calldata, packErr := r.l1RollupABI.Pack( - "finalizeBatchWithProof4844", - dbBatch.BatchHeader, - common.HexToHash(dbParentBatch.StateRoot), - common.HexToHash(dbBatch.StateRoot), - common.HexToHash(dbBatch.WithdrawRoot), - blobDataProof, - aggProof.Proof, - ) - if packErr != nil { - return nil, fmt.Errorf("failed to pack finalizeBatchWithProof4844: %w", packErr) - } - return calldata, nil - } - - // finalizeBatch4844 without proof. - calldata, packErr := r.l1RollupABI.Pack( - "finalizeBatch4844", - dbBatch.BatchHeader, - common.HexToHash(dbParentBatch.StateRoot), - common.HexToHash(dbBatch.StateRoot), - common.HexToHash(dbBatch.WithdrawRoot), - blobDataProof, - ) - if packErr != nil { - return nil, fmt.Errorf("failed to pack finalizeBatch4844: %w", packErr) - } - return calldata, nil -} - -func (r *Layer2Relayer) constructFinalizeBundlePayloadCodecV3AndV4(dbBatch *orm.Batch, aggProof *message.BundleProof) ([]byte, error) { +func (r *Layer2Relayer) constructFinalizeBundlePayloadCodecV4(dbBatch *orm.Batch, aggProof *message.BundleProof) ([]byte, error) { if aggProof != nil { // finalizeBundle with proof. calldata, packErr := r.l1RollupABI.Pack( "finalizeBundleWithProof", diff --git a/rollup/internal/controller/relayer/l2_relayer_test.go b/rollup/internal/controller/relayer/l2_relayer_test.go index c84c10e66a..a40513318d 100644 --- a/rollup/internal/controller/relayer/l2_relayer_test.go +++ b/rollup/internal/controller/relayer/l2_relayer_test.go @@ -51,21 +51,17 @@ func testCreateNewRelayer(t *testing.T) { } func testL2RelayerProcessPendingBatches(t *testing.T) { - codecVersions := []encoding.CodecVersion{encoding.CodecV0, encoding.CodecV1, encoding.CodecV2, encoding.CodecV3} + codecVersions := []encoding.CodecVersion{encoding.CodecV4} for _, codecVersion := range codecVersions { db := setupL2RelayerDB(t) defer database.CloseDB(db) l2Cfg := cfg.L2Config var chainConfig *params.ChainConfig - if codecVersion == encoding.CodecV0 { - chainConfig = ¶ms.ChainConfig{} - } else if codecVersion == encoding.CodecV1 { - chainConfig = ¶ms.ChainConfig{BernoulliBlock: big.NewInt(0)} - } else if codecVersion == encoding.CodecV2 { - chainConfig = ¶ms.ChainConfig{BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0)} + if codecVersion == encoding.CodecV4 { + chainConfig = ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64), DarwinV2Time: new(uint64)} } else { - chainConfig = ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64)} + assert.Fail(t, "unsupported codec version, expected CodecV4") } relayer, err := NewLayer2Relayer(context.Background(), l2Cli, db, l2Cfg.RelayerConfig, chainConfig, true, ServiceTypeL2RollupRelayer, nil) @@ -106,85 +102,16 @@ func testL2RelayerProcessPendingBatches(t *testing.T) { } } -func testL2RelayerProcessCommittedBatches(t *testing.T) { - codecVersions := []encoding.CodecVersion{encoding.CodecV0, encoding.CodecV1, encoding.CodecV2} - for _, codecVersion := range codecVersions { - db := setupL2RelayerDB(t) - defer database.CloseDB(db) - - l2Cfg := cfg.L2Config - var chainConfig *params.ChainConfig - if codecVersion == encoding.CodecV0 { - chainConfig = ¶ms.ChainConfig{} - } else if codecVersion == encoding.CodecV1 { - chainConfig = ¶ms.ChainConfig{BernoulliBlock: big.NewInt(0)} - } else { - chainConfig = ¶ms.ChainConfig{BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0)} - } - relayer, err := NewLayer2Relayer(context.Background(), l2Cli, db, l2Cfg.RelayerConfig, chainConfig, true, ServiceTypeL2RollupRelayer, nil) - assert.NoError(t, err) - - l2BlockOrm := orm.NewL2Block(db) - err = l2BlockOrm.InsertL2Blocks(context.Background(), []*encoding.Block{block1, block2}) - assert.NoError(t, err) - chunkOrm := orm.NewChunk(db) - _, err = chunkOrm.InsertChunk(context.Background(), chunk1, codecVersion, rutils.ChunkMetrics{}) - assert.NoError(t, err) - _, err = chunkOrm.InsertChunk(context.Background(), chunk2, codecVersion, rutils.ChunkMetrics{}) - assert.NoError(t, err) - - batch := &encoding.Batch{ - Index: 1, - TotalL1MessagePoppedBefore: 0, - ParentBatchHash: common.Hash{}, - Chunks: []*encoding.Chunk{chunk1, chunk2}, - } - - batchOrm := orm.NewBatch(db) - dbBatch, err := batchOrm.InsertBatch(context.Background(), batch, codecVersion, rutils.BatchMetrics{}) - assert.NoError(t, err) - - err = batchOrm.UpdateRollupStatus(context.Background(), dbBatch.Hash, types.RollupCommitted) - assert.NoError(t, err) - - err = batchOrm.UpdateProvingStatus(context.Background(), dbBatch.Hash, types.ProvingTaskVerified) - assert.NoError(t, err) - - relayer.ProcessCommittedBatches() - - statuses, err := batchOrm.GetRollupStatusByHashList(context.Background(), []string{dbBatch.Hash}) - assert.NoError(t, err) - assert.Equal(t, 1, len(statuses)) - // no valid proof, rollup status remains the same - assert.Equal(t, types.RollupCommitted, statuses[0]) - - proof := &message.BatchProof{ - Proof: []byte{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31}, - Instances: []byte{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31}, - Vk: []byte{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31}, - } - err = batchOrm.UpdateProofByHash(context.Background(), dbBatch.Hash, proof, 100) - assert.NoError(t, err) - - relayer.ProcessCommittedBatches() - statuses, err = batchOrm.GetRollupStatusByHashList(context.Background(), []string{dbBatch.Hash}) - assert.NoError(t, err) - assert.Equal(t, 1, len(statuses)) - assert.Equal(t, types.RollupFinalizing, statuses[0]) - relayer.StopSenders() - } -} - func testL2RelayerProcessPendingBundles(t *testing.T) { - codecVersions := []encoding.CodecVersion{encoding.CodecV3} + codecVersions := []encoding.CodecVersion{encoding.CodecV4} for _, codecVersion := range codecVersions { db := setupL2RelayerDB(t) defer database.CloseDB(db) l2Cfg := cfg.L2Config var chainConfig *params.ChainConfig - if codecVersion == encoding.CodecV3 { - chainConfig = ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64)} + if codecVersion == encoding.CodecV4 { + chainConfig = ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64), DarwinV2Time: new(uint64)} } relayer, err := NewLayer2Relayer(context.Background(), l2Cli, db, l2Cfg.RelayerConfig, chainConfig, true, ServiceTypeL2RollupRelayer, nil) assert.NoError(t, err) @@ -235,79 +162,8 @@ func testL2RelayerProcessPendingBundles(t *testing.T) { } } -func testL2RelayerFinalizeTimeoutBatches(t *testing.T) { - codecVersions := []encoding.CodecVersion{encoding.CodecV0, encoding.CodecV1, encoding.CodecV2} - for _, codecVersion := range codecVersions { - db := setupL2RelayerDB(t) - defer database.CloseDB(db) - - l2Cfg := cfg.L2Config - l2Cfg.RelayerConfig.EnableTestEnvBypassFeatures = true - l2Cfg.RelayerConfig.FinalizeBatchWithoutProofTimeoutSec = 0 - var chainConfig *params.ChainConfig - if codecVersion == encoding.CodecV0 { - chainConfig = ¶ms.ChainConfig{} - } else if codecVersion == encoding.CodecV1 { - chainConfig = ¶ms.ChainConfig{BernoulliBlock: big.NewInt(0)} - } else { - chainConfig = ¶ms.ChainConfig{BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0)} - } - relayer, err := NewLayer2Relayer(context.Background(), l2Cli, db, l2Cfg.RelayerConfig, chainConfig, true, ServiceTypeL2RollupRelayer, nil) - assert.NoError(t, err) - - l2BlockOrm := orm.NewL2Block(db) - err = l2BlockOrm.InsertL2Blocks(context.Background(), []*encoding.Block{block1, block2}) - assert.NoError(t, err) - chunkOrm := orm.NewChunk(db) - chunkDB1, err := chunkOrm.InsertChunk(context.Background(), chunk1, codecVersion, rutils.ChunkMetrics{}) - assert.NoError(t, err) - chunkDB2, err := chunkOrm.InsertChunk(context.Background(), chunk2, codecVersion, rutils.ChunkMetrics{}) - assert.NoError(t, err) - - batch := &encoding.Batch{ - Index: 1, - TotalL1MessagePoppedBefore: 0, - ParentBatchHash: common.Hash{}, - Chunks: []*encoding.Chunk{chunk1, chunk2}, - } - - batchOrm := orm.NewBatch(db) - dbBatch, err := batchOrm.InsertBatch(context.Background(), batch, codecVersion, rutils.BatchMetrics{}) - assert.NoError(t, err) - - err = batchOrm.UpdateRollupStatus(context.Background(), dbBatch.Hash, types.RollupCommitted) - assert.NoError(t, err) - - err = chunkOrm.UpdateBatchHashInRange(context.Background(), chunkDB1.Index, chunkDB2.Index, dbBatch.Hash, nil) - assert.NoError(t, err) - - assert.Eventually(t, func() bool { - relayer.ProcessCommittedBatches() - - batchInDB, batchErr := batchOrm.GetBatches(context.Background(), map[string]interface{}{"hash": dbBatch.Hash}, nil, 0) - if batchErr != nil { - return false - } - - batchStatus := len(batchInDB) == 1 && types.RollupStatus(batchInDB[0].RollupStatus) == types.RollupFinalizing && - types.ProvingStatus(batchInDB[0].ProvingStatus) == types.ProvingTaskVerified - - chunks, chunkErr := chunkOrm.GetChunksByBatchHash(context.Background(), dbBatch.Hash) - if chunkErr != nil { - return false - } - - chunkStatus := len(chunks) == 2 && types.ProvingStatus(chunks[0].ProvingStatus) == types.ProvingTaskVerified && - types.ProvingStatus(chunks[1].ProvingStatus) == types.ProvingTaskVerified - - return batchStatus && chunkStatus - }, 5*time.Second, 100*time.Millisecond, "Batch or Chunk status did not update as expected") - relayer.StopSenders() - } -} - func testL2RelayerFinalizeTimeoutBundles(t *testing.T) { - codecVersions := []encoding.CodecVersion{encoding.CodecV3} + codecVersions := []encoding.CodecVersion{encoding.CodecV4} for _, codecVersion := range codecVersions { db := setupL2RelayerDB(t) defer database.CloseDB(db) @@ -316,8 +172,8 @@ func testL2RelayerFinalizeTimeoutBundles(t *testing.T) { l2Cfg.RelayerConfig.EnableTestEnvBypassFeatures = true l2Cfg.RelayerConfig.FinalizeBundleWithoutProofTimeoutSec = 0 var chainConfig *params.ChainConfig - if codecVersion == encoding.CodecV3 { - chainConfig = ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64)} + if codecVersion == encoding.CodecV4 { + chainConfig = ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64), DarwinV2Time: new(uint64)} } relayer, err := NewLayer2Relayer(context.Background(), l2Cli, db, l2Cfg.RelayerConfig, chainConfig, true, ServiceTypeL2RollupRelayer, nil) assert.NoError(t, err) @@ -443,62 +299,6 @@ func testL2RelayerCommitConfirm(t *testing.T) { assert.True(t, ok) } -func testL2RelayerFinalizeBatchConfirm(t *testing.T) { - db := setupL2RelayerDB(t) - defer database.CloseDB(db) - - // Create and set up the Layer2 Relayer. - l2Cfg := cfg.L2Config - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - l2Relayer, err := NewLayer2Relayer(ctx, l2Cli, db, l2Cfg.RelayerConfig, ¶ms.ChainConfig{}, true, ServiceTypeL2RollupRelayer, nil) - assert.NoError(t, err) - defer l2Relayer.StopSenders() - - // Simulate message confirmations. - isSuccessful := []bool{true, false} - batchOrm := orm.NewBatch(db) - batchHashes := make([]string, len(isSuccessful)) - for i := range batchHashes { - batch := &encoding.Batch{ - Index: uint64(i + 1), - TotalL1MessagePoppedBefore: 0, - ParentBatchHash: common.Hash{}, - Chunks: []*encoding.Chunk{chunk1, chunk2}, - } - - dbBatch, err := batchOrm.InsertBatch(context.Background(), batch, encoding.CodecV0, rutils.BatchMetrics{}) - assert.NoError(t, err) - batchHashes[i] = dbBatch.Hash - } - - for i, batchHash := range batchHashes { - l2Relayer.finalizeSender.SendConfirmation(&sender.Confirmation{ - ContextID: batchHash, - IsSuccessful: isSuccessful[i], - TxHash: common.HexToHash("0x123456789abcdef"), - SenderType: types.SenderTypeFinalizeBatch, - }) - } - - // Check the database for the updated status using TryTimes. - ok := utils.TryTimes(5, func() bool { - expectedStatuses := []types.RollupStatus{ - types.RollupFinalized, - types.RollupFinalizeFailed, - } - - for i, batchHash := range batchHashes { - batchInDB, err := batchOrm.GetBatches(context.Background(), map[string]interface{}{"hash": batchHash}, nil, 0) - if err != nil || len(batchInDB) != 1 || types.RollupStatus(batchInDB[0].RollupStatus) != expectedStatuses[i] { - return false - } - } - return true - }) - assert.True(t, ok) -} - func testL2RelayerFinalizeBundleConfirm(t *testing.T) { db := setupL2RelayerDB(t) defer database.CloseDB(db) @@ -529,7 +329,7 @@ func testL2RelayerFinalizeBundleConfirm(t *testing.T) { assert.NoError(t, err) batchHashes[i] = dbBatch.Hash - bundle, err := bundleOrm.InsertBundle(context.Background(), []*orm.Batch{dbBatch}, encoding.CodecV3) + bundle, err := bundleOrm.InsertBundle(context.Background(), []*orm.Batch{dbBatch}, encoding.CodecV4) assert.NoError(t, err) bundleHashes[i] = bundle.Hash diff --git a/rollup/internal/controller/relayer/relayer_test.go b/rollup/internal/controller/relayer/relayer_test.go index c9d33fef16..bf78c9518d 100644 --- a/rollup/internal/controller/relayer/relayer_test.go +++ b/rollup/internal/controller/relayer/relayer_test.go @@ -124,12 +124,9 @@ func TestFunctions(t *testing.T) { // Run l2 relayer test cases. t.Run("TestCreateNewRelayer", testCreateNewRelayer) t.Run("TestL2RelayerProcessPendingBatches", testL2RelayerProcessPendingBatches) - t.Run("TestL2RelayerProcessCommittedBatches", testL2RelayerProcessCommittedBatches) t.Run("TestL2RelayerProcessPendingBundles", testL2RelayerProcessPendingBundles) - t.Run("TestL2RelayerFinalizeTimeoutBatches", testL2RelayerFinalizeTimeoutBatches) t.Run("TestL2RelayerFinalizeTimeoutBundles", testL2RelayerFinalizeTimeoutBundles) t.Run("TestL2RelayerCommitConfirm", testL2RelayerCommitConfirm) - t.Run("TestL2RelayerFinalizeBatchConfirm", testL2RelayerFinalizeBatchConfirm) t.Run("TestL2RelayerFinalizeBundleConfirm", testL2RelayerFinalizeBundleConfirm) t.Run("TestL2RelayerGasOracleConfirm", testL2RelayerGasOracleConfirm) t.Run("TestLayer2RelayerProcessGasPriceOracle", testLayer2RelayerProcessGasPriceOracle) diff --git a/rollup/internal/controller/watcher/batch_proposer.go b/rollup/internal/controller/watcher/batch_proposer.go index ec25c8f247..8b69cd9c25 100644 --- a/rollup/internal/controller/watcher/batch_proposer.go +++ b/rollup/internal/controller/watcher/batch_proposer.go @@ -34,7 +34,8 @@ type BatchProposer struct { gasCostIncreaseMultiplier float64 maxUncompressedBatchBytesSize uint64 - chainCfg *params.ChainConfig + minCodecVersion encoding.CodecVersion + chainCfg *params.ChainConfig batchProposerCircleTotal prometheus.Counter proposeBatchFailureTotal prometheus.Counter @@ -58,7 +59,7 @@ type BatchProposer struct { } // NewBatchProposer creates a new BatchProposer instance. -func NewBatchProposer(ctx context.Context, cfg *config.BatchProposerConfig, chainCfg *params.ChainConfig, db *gorm.DB, reg prometheus.Registerer) *BatchProposer { +func NewBatchProposer(ctx context.Context, cfg *config.BatchProposerConfig, minCodecVersion encoding.CodecVersion, chainCfg *params.ChainConfig, db *gorm.DB, reg prometheus.Registerer) *BatchProposer { log.Info("new batch proposer", "maxL1CommitGasPerBatch", cfg.MaxL1CommitGasPerBatch, "maxL1CommitCalldataSizePerBatch", cfg.MaxL1CommitCalldataSizePerBatch, @@ -78,6 +79,7 @@ func NewBatchProposer(ctx context.Context, cfg *config.BatchProposerConfig, chai batchTimeoutSec: cfg.BatchTimeoutSec, gasCostIncreaseMultiplier: cfg.GasCostIncreaseMultiplier, maxUncompressedBatchBytesSize: cfg.MaxUncompressedBatchBytesSize, + minCodecVersion: minCodecVersion, chainCfg: chainCfg, batchProposerCircleTotal: promauto.With(reg).NewCounter(prometheus.CounterOpts{ @@ -247,6 +249,11 @@ func (p *BatchProposer) proposeBatch() error { if codec == nil { return fmt.Errorf("failed to retrieve codec for block number %v and time %v", firstUnbatchedChunk.StartBlockNumber, firstUnbatchedChunk.StartBlockTime) } + + if codec.Version() < p.minCodecVersion { + return fmt.Errorf("unsupported codec version: %v, expected at least %v", codec.Version(), p.minCodecVersion) + } + maxChunksThisBatch := codec.MaxNumChunksPerBatch() // select at most maxChunkNumPerBatch chunks diff --git a/rollup/internal/controller/watcher/batch_proposer_test.go b/rollup/internal/controller/watcher/batch_proposer_test.go index 428f7226f7..7c426d5b10 100644 --- a/rollup/internal/controller/watcher/batch_proposer_test.go +++ b/rollup/internal/controller/watcher/batch_proposer_test.go @@ -20,7 +20,7 @@ import ( "scroll-tech/rollup/internal/utils" ) -func testBatchProposerCodecv0Limits(t *testing.T) { +func testBatchProposerLimitsCodecV4(t *testing.T) { tests := []struct { name string maxL1CommitGas uint64 @@ -60,785 +60,118 @@ func testBatchProposerCodecv0Limits(t *testing.T) { }, { name: "MaxL1CommitGasPerBatchIsFirstChunk", - maxL1CommitGas: 200000, - maxL1CommitCalldataSize: 1000000, - batchTimeoutSec: 1000000000000, - expectedBatchesLen: 1, - expectedChunksInFirstBatch: 1, - }, - { - name: "MaxL1CommitCalldataSizePerBatchIsFirstChunk", - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 298, - batchTimeoutSec: 1000000000000, - expectedBatchesLen: 1, - expectedChunksInFirstBatch: 1, - }, - } - - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - db := setupDB(t) - defer database.CloseDB(db) - - // Add genesis batch. - block := &encoding.Block{ - Header: &gethTypes.Header{ - Number: big.NewInt(0), - }, - RowConsumption: &gethTypes.RowConsumption{}, - } - chunk := &encoding.Chunk{ - Blocks: []*encoding.Block{block}, - } - chunkOrm := orm.NewChunk(db) - _, err := chunkOrm.InsertChunk(context.Background(), chunk, encoding.CodecV0, utils.ChunkMetrics{}) - assert.NoError(t, err) - batch := &encoding.Batch{ - Index: 0, - TotalL1MessagePoppedBefore: 0, - ParentBatchHash: common.Hash{}, - Chunks: []*encoding.Chunk{chunk}, - } - batchOrm := orm.NewBatch(db) - _, err = batchOrm.InsertBatch(context.Background(), batch, encoding.CodecV0, utils.BatchMetrics{}) - assert.NoError(t, err) - - l2BlockOrm := orm.NewL2Block(db) - err = l2BlockOrm.InsertL2Blocks(context.Background(), []*encoding.Block{block1, block2}) - assert.NoError(t, err) - - cp := NewChunkProposer(context.Background(), &config.ChunkProposerConfig{ - MaxBlockNumPerChunk: 1, - MaxTxNumPerChunk: 10000, - MaxL1CommitGasPerChunk: 50000000000, - MaxL1CommitCalldataSizePerChunk: 1000000, - MaxRowConsumptionPerChunk: 1000000, - ChunkTimeoutSec: 300, - GasCostIncreaseMultiplier: 1.2, - }, ¶ms.ChainConfig{}, db, nil) - cp.TryProposeChunk() // chunk1 contains block1 - cp.TryProposeChunk() // chunk2 contains block2 - - chunks, err := chunkOrm.GetChunksInRange(context.Background(), 1, 2) - assert.NoError(t, err) - assert.Equal(t, uint64(5082), chunks[0].TotalL1CommitGas) - assert.Equal(t, uint64(298), chunks[0].TotalL1CommitCalldataSize) - assert.Equal(t, uint64(93658), chunks[1].TotalL1CommitGas) - assert.Equal(t, uint64(5737), chunks[1].TotalL1CommitCalldataSize) - - bp := NewBatchProposer(context.Background(), &config.BatchProposerConfig{ - MaxL1CommitGasPerBatch: tt.maxL1CommitGas, - MaxL1CommitCalldataSizePerBatch: tt.maxL1CommitCalldataSize, - BatchTimeoutSec: tt.batchTimeoutSec, - GasCostIncreaseMultiplier: 1.2, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, ¶ms.ChainConfig{}, db, nil) - bp.TryProposeBatch() - - batches, err := batchOrm.GetBatches(context.Background(), map[string]interface{}{}, []string{}, 0) - assert.NoError(t, err) - assert.Len(t, batches, tt.expectedBatchesLen+1) - batches = batches[1:] - if tt.expectedBatchesLen > 0 { - assert.Equal(t, uint64(1), batches[0].StartChunkIndex) - assert.Equal(t, tt.expectedChunksInFirstBatch, batches[0].EndChunkIndex) - assert.Equal(t, types.RollupPending, types.RollupStatus(batches[0].RollupStatus)) - assert.Equal(t, types.ProvingTaskUnassigned, types.ProvingStatus(batches[0].ProvingStatus)) - - dbChunks, err := chunkOrm.GetChunksInRange(context.Background(), 1, tt.expectedChunksInFirstBatch) - assert.NoError(t, err) - assert.Len(t, dbChunks, int(tt.expectedChunksInFirstBatch)) - for _, chunk := range dbChunks { - assert.Equal(t, batches[0].Hash, chunk.BatchHash) - assert.Equal(t, types.ProvingTaskUnassigned, types.ProvingStatus(chunk.ProvingStatus)) - } - } - }) - } -} - -func testBatchProposerCodecv1Limits(t *testing.T) { - tests := []struct { - name string - maxL1CommitGas uint64 - maxL1CommitCalldataSize uint64 - batchTimeoutSec uint64 - expectedBatchesLen int - expectedChunksInFirstBatch uint64 // only be checked when expectedBatchesLen > 0 - }{ - { - name: "NoLimitReached", - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - batchTimeoutSec: 1000000000000, - expectedBatchesLen: 0, - }, - { - name: "Timeout", - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - batchTimeoutSec: 0, - expectedBatchesLen: 1, - expectedChunksInFirstBatch: 2, - }, - { - name: "MaxL1CommitGasPerBatchIs0", - maxL1CommitGas: 0, - maxL1CommitCalldataSize: 1000000, - batchTimeoutSec: 1000000000000, - expectedBatchesLen: 0, - }, - { - name: "MaxL1CommitCalldataSizePerBatchIs0", - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 0, - batchTimeoutSec: 1000000000000, - expectedBatchesLen: 0, - }, - { - name: "MaxL1CommitGasPerBatchIsFirstChunk", - maxL1CommitGas: 190330, - maxL1CommitCalldataSize: 1000000, - batchTimeoutSec: 1000000000000, - expectedBatchesLen: 1, - expectedChunksInFirstBatch: 1, - }, - { - name: "MaxL1CommitCalldataSizePerBatchIsFirstChunk", - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 60, - batchTimeoutSec: 1000000000000, - expectedBatchesLen: 1, - expectedChunksInFirstBatch: 1, - }, - } - - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - db := setupDB(t) - defer database.CloseDB(db) - - // Add genesis batch. - block := &encoding.Block{ - Header: &gethTypes.Header{ - Number: big.NewInt(0), - }, - RowConsumption: &gethTypes.RowConsumption{}, - } - chunk := &encoding.Chunk{ - Blocks: []*encoding.Block{block}, - } - chunkOrm := orm.NewChunk(db) - _, err := chunkOrm.InsertChunk(context.Background(), chunk, encoding.CodecV0, utils.ChunkMetrics{}) - assert.NoError(t, err) - batch := &encoding.Batch{ - Index: 0, - TotalL1MessagePoppedBefore: 0, - ParentBatchHash: common.Hash{}, - Chunks: []*encoding.Chunk{chunk}, - } - batchOrm := orm.NewBatch(db) - _, err = batchOrm.InsertBatch(context.Background(), batch, encoding.CodecV0, utils.BatchMetrics{}) - assert.NoError(t, err) - - l2BlockOrm := orm.NewL2Block(db) - err = l2BlockOrm.InsertL2Blocks(context.Background(), []*encoding.Block{block1, block2}) - assert.NoError(t, err) - - cp := NewChunkProposer(context.Background(), &config.ChunkProposerConfig{ - MaxBlockNumPerChunk: 1, - MaxTxNumPerChunk: 10000, - MaxL1CommitGasPerChunk: 50000000000, - MaxL1CommitCalldataSizePerChunk: 1000000, - MaxRowConsumptionPerChunk: 1000000, - ChunkTimeoutSec: 300, - GasCostIncreaseMultiplier: 1.2, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, ¶ms.ChainConfig{ - BernoulliBlock: big.NewInt(0), - }, db, nil) - cp.TryProposeChunk() // chunk1 contains block1 - cp.TryProposeChunk() // chunk2 contains block2 - - chunks, err := chunkOrm.GetChunksInRange(context.Background(), 1, 2) - assert.NoError(t, err) - assert.Equal(t, uint64(1124), chunks[0].TotalL1CommitGas) - assert.Equal(t, uint64(60), chunks[0].TotalL1CommitCalldataSize) - assert.Equal(t, uint64(1124), chunks[1].TotalL1CommitGas) - assert.Equal(t, uint64(60), chunks[1].TotalL1CommitCalldataSize) - - bp := NewBatchProposer(context.Background(), &config.BatchProposerConfig{ - MaxL1CommitGasPerBatch: tt.maxL1CommitGas, - MaxL1CommitCalldataSizePerBatch: tt.maxL1CommitCalldataSize, - BatchTimeoutSec: tt.batchTimeoutSec, - GasCostIncreaseMultiplier: 1.2, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, ¶ms.ChainConfig{ - BernoulliBlock: big.NewInt(0), - }, db, nil) - bp.TryProposeBatch() - - batches, err := batchOrm.GetBatches(context.Background(), map[string]interface{}{}, []string{}, 0) - assert.NoError(t, err) - assert.Len(t, batches, tt.expectedBatchesLen+1) - batches = batches[1:] - if tt.expectedBatchesLen > 0 { - assert.Equal(t, uint64(1), batches[0].StartChunkIndex) - assert.Equal(t, tt.expectedChunksInFirstBatch, batches[0].EndChunkIndex) - assert.Equal(t, types.RollupPending, types.RollupStatus(batches[0].RollupStatus)) - assert.Equal(t, types.ProvingTaskUnassigned, types.ProvingStatus(batches[0].ProvingStatus)) - - dbChunks, err := chunkOrm.GetChunksInRange(context.Background(), 1, tt.expectedChunksInFirstBatch) - assert.NoError(t, err) - assert.Len(t, dbChunks, int(tt.expectedChunksInFirstBatch)) - for _, chunk := range dbChunks { - assert.Equal(t, batches[0].Hash, chunk.BatchHash) - assert.Equal(t, types.ProvingTaskUnassigned, types.ProvingStatus(chunk.ProvingStatus)) - } - } - }) - } -} - -func testBatchProposerCodecv2Limits(t *testing.T) { - tests := []struct { - name string - maxL1CommitGas uint64 - maxL1CommitCalldataSize uint64 - batchTimeoutSec uint64 - expectedBatchesLen int - expectedChunksInFirstBatch uint64 // only be checked when expectedBatchesLen > 0 - }{ - { - name: "NoLimitReached", - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - batchTimeoutSec: 1000000000000, - expectedBatchesLen: 0, - }, - { - name: "Timeout", - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - batchTimeoutSec: 0, - expectedBatchesLen: 1, - expectedChunksInFirstBatch: 2, - }, - { - name: "MaxL1CommitGasPerBatchIs0", - maxL1CommitGas: 0, - maxL1CommitCalldataSize: 1000000, - batchTimeoutSec: 1000000000000, - expectedBatchesLen: 0, - }, - { - name: "MaxL1CommitCalldataSizePerBatchIs0", - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 0, - batchTimeoutSec: 1000000000000, - expectedBatchesLen: 0, - }, - { - name: "MaxL1CommitGasPerBatchIsFirstChunk", - maxL1CommitGas: 189179, - maxL1CommitCalldataSize: 1000000, - batchTimeoutSec: 1000000000000, - expectedBatchesLen: 1, - expectedChunksInFirstBatch: 1, - }, - { - name: "MaxL1CommitCalldataSizePerBatchIsFirstChunk", - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 60, - batchTimeoutSec: 1000000000000, - expectedBatchesLen: 1, - expectedChunksInFirstBatch: 1, - }, - } - - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - db := setupDB(t) - defer database.CloseDB(db) - - // Add genesis batch. - block := &encoding.Block{ - Header: &gethTypes.Header{ - Number: big.NewInt(0), - }, - RowConsumption: &gethTypes.RowConsumption{}, - } - chunk := &encoding.Chunk{ - Blocks: []*encoding.Block{block}, - } - chunkOrm := orm.NewChunk(db) - _, err := chunkOrm.InsertChunk(context.Background(), chunk, encoding.CodecV0, utils.ChunkMetrics{}) - assert.NoError(t, err) - batch := &encoding.Batch{ - Index: 0, - TotalL1MessagePoppedBefore: 0, - ParentBatchHash: common.Hash{}, - Chunks: []*encoding.Chunk{chunk}, - } - batchOrm := orm.NewBatch(db) - _, err = batchOrm.InsertBatch(context.Background(), batch, encoding.CodecV0, utils.BatchMetrics{}) - assert.NoError(t, err) - - l2BlockOrm := orm.NewL2Block(db) - err = l2BlockOrm.InsertL2Blocks(context.Background(), []*encoding.Block{block1, block2}) - assert.NoError(t, err) - - cp := NewChunkProposer(context.Background(), &config.ChunkProposerConfig{ - MaxBlockNumPerChunk: 1, - MaxTxNumPerChunk: 10000, - MaxL1CommitGasPerChunk: 50000000000, - MaxL1CommitCalldataSizePerChunk: 1000000, - MaxRowConsumptionPerChunk: 1000000, - ChunkTimeoutSec: 300, - GasCostIncreaseMultiplier: 1.2, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, ¶ms.ChainConfig{ - BernoulliBlock: big.NewInt(0), - CurieBlock: big.NewInt(0), - }, db, nil) - cp.TryProposeChunk() // chunk1 contains block1 - cp.TryProposeChunk() // chunk2 contains block2 - - chunks, err := chunkOrm.GetChunksInRange(context.Background(), 1, 2) - assert.NoError(t, err) - assert.Equal(t, uint64(1124), chunks[0].TotalL1CommitGas) - assert.Equal(t, uint64(60), chunks[0].TotalL1CommitCalldataSize) - assert.Equal(t, uint64(1124), chunks[1].TotalL1CommitGas) - assert.Equal(t, uint64(60), chunks[1].TotalL1CommitCalldataSize) - - bp := NewBatchProposer(context.Background(), &config.BatchProposerConfig{ - MaxL1CommitGasPerBatch: tt.maxL1CommitGas, - MaxL1CommitCalldataSizePerBatch: tt.maxL1CommitCalldataSize, - BatchTimeoutSec: tt.batchTimeoutSec, - GasCostIncreaseMultiplier: 1.2, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, ¶ms.ChainConfig{ - BernoulliBlock: big.NewInt(0), - CurieBlock: big.NewInt(0), - }, db, nil) - bp.TryProposeBatch() - - batches, err := batchOrm.GetBatches(context.Background(), map[string]interface{}{}, []string{}, 0) - assert.NoError(t, err) - assert.Len(t, batches, tt.expectedBatchesLen+1) - batches = batches[1:] - if tt.expectedBatchesLen > 0 { - assert.Equal(t, uint64(1), batches[0].StartChunkIndex) - assert.Equal(t, tt.expectedChunksInFirstBatch, batches[0].EndChunkIndex) - assert.Equal(t, types.RollupPending, types.RollupStatus(batches[0].RollupStatus)) - assert.Equal(t, types.ProvingTaskUnassigned, types.ProvingStatus(batches[0].ProvingStatus)) - - dbChunks, err := chunkOrm.GetChunksInRange(context.Background(), 1, tt.expectedChunksInFirstBatch) - assert.NoError(t, err) - assert.Len(t, dbChunks, int(tt.expectedChunksInFirstBatch)) - for _, chunk := range dbChunks { - assert.Equal(t, batches[0].Hash, chunk.BatchHash) - assert.Equal(t, types.ProvingTaskUnassigned, types.ProvingStatus(chunk.ProvingStatus)) - } - } - }) - } -} - -func testBatchProposerCodecv3Limits(t *testing.T) { - tests := []struct { - name string - maxL1CommitGas uint64 - maxL1CommitCalldataSize uint64 - batchTimeoutSec uint64 - expectedBatchesLen int - expectedChunksInFirstBatch uint64 // only be checked when expectedBatchesLen > 0 - }{ - { - name: "NoLimitReached", - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - batchTimeoutSec: 1000000000000, - expectedBatchesLen: 0, - }, - { - name: "Timeout", - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - batchTimeoutSec: 0, - expectedBatchesLen: 1, - expectedChunksInFirstBatch: 2, - }, - { - name: "MaxL1CommitGasPerBatchIs0", - maxL1CommitGas: 0, - maxL1CommitCalldataSize: 1000000, - batchTimeoutSec: 1000000000000, - expectedBatchesLen: 0, - }, - { - name: "MaxL1CommitCalldataSizePerBatchIs0", - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 0, - batchTimeoutSec: 1000000000000, - expectedBatchesLen: 0, - }, - { - name: "MaxL1CommitGasPerBatchIsFirstChunk", - maxL1CommitGas: 249179, - maxL1CommitCalldataSize: 1000000, - batchTimeoutSec: 1000000000000, - expectedBatchesLen: 1, - expectedChunksInFirstBatch: 1, - }, - { - name: "MaxL1CommitCalldataSizePerBatchIsFirstChunk", - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 60, - batchTimeoutSec: 1000000000000, - expectedBatchesLen: 1, - expectedChunksInFirstBatch: 1, - }, - } - - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - db := setupDB(t) - defer database.CloseDB(db) - - // Add genesis batch. - block := &encoding.Block{ - Header: &gethTypes.Header{ - Number: big.NewInt(0), - }, - RowConsumption: &gethTypes.RowConsumption{}, - } - chunk := &encoding.Chunk{ - Blocks: []*encoding.Block{block}, - } - chunkOrm := orm.NewChunk(db) - _, err := chunkOrm.InsertChunk(context.Background(), chunk, encoding.CodecV0, utils.ChunkMetrics{}) - assert.NoError(t, err) - batch := &encoding.Batch{ - Index: 0, - TotalL1MessagePoppedBefore: 0, - ParentBatchHash: common.Hash{}, - Chunks: []*encoding.Chunk{chunk}, - } - batchOrm := orm.NewBatch(db) - _, err = batchOrm.InsertBatch(context.Background(), batch, encoding.CodecV0, utils.BatchMetrics{}) - assert.NoError(t, err) - - l2BlockOrm := orm.NewL2Block(db) - err = l2BlockOrm.InsertL2Blocks(context.Background(), []*encoding.Block{block1, block2}) - assert.NoError(t, err) - - cp := NewChunkProposer(context.Background(), &config.ChunkProposerConfig{ - MaxBlockNumPerChunk: 1, - MaxTxNumPerChunk: 10000, - MaxL1CommitGasPerChunk: 50000000000, - MaxL1CommitCalldataSizePerChunk: 1000000, - MaxRowConsumptionPerChunk: 1000000, - ChunkTimeoutSec: 300, - GasCostIncreaseMultiplier: 1.2, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, ¶ms.ChainConfig{ - LondonBlock: big.NewInt(0), - BernoulliBlock: big.NewInt(0), - CurieBlock: big.NewInt(0), - DarwinTime: new(uint64), - }, db, nil) - cp.TryProposeChunk() // chunk1 contains block1 - cp.TryProposeChunk() // chunk2 contains block2 - - chunks, err := chunkOrm.GetChunksInRange(context.Background(), 1, 2) - assert.NoError(t, err) - assert.Equal(t, uint64(51124), chunks[0].TotalL1CommitGas) - assert.Equal(t, uint64(60), chunks[0].TotalL1CommitCalldataSize) - assert.Equal(t, uint64(51124), chunks[1].TotalL1CommitGas) - assert.Equal(t, uint64(60), chunks[1].TotalL1CommitCalldataSize) - - bp := NewBatchProposer(context.Background(), &config.BatchProposerConfig{ - MaxL1CommitGasPerBatch: tt.maxL1CommitGas, - MaxL1CommitCalldataSizePerBatch: tt.maxL1CommitCalldataSize, - BatchTimeoutSec: tt.batchTimeoutSec, - GasCostIncreaseMultiplier: 1.2, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, ¶ms.ChainConfig{ - LondonBlock: big.NewInt(0), - BernoulliBlock: big.NewInt(0), - CurieBlock: big.NewInt(0), - DarwinTime: new(uint64), - }, db, nil) - bp.TryProposeBatch() - - batches, err := batchOrm.GetBatches(context.Background(), map[string]interface{}{}, []string{}, 0) - assert.NoError(t, err) - assert.Len(t, batches, tt.expectedBatchesLen+1) - batches = batches[1:] - if tt.expectedBatchesLen > 0 { - assert.Equal(t, uint64(1), batches[0].StartChunkIndex) - assert.Equal(t, tt.expectedChunksInFirstBatch, batches[0].EndChunkIndex) - assert.Equal(t, types.RollupPending, types.RollupStatus(batches[0].RollupStatus)) - assert.Equal(t, types.ProvingTaskUnassigned, types.ProvingStatus(batches[0].ProvingStatus)) - - dbChunks, err := chunkOrm.GetChunksInRange(context.Background(), 1, tt.expectedChunksInFirstBatch) - assert.NoError(t, err) - assert.Len(t, dbChunks, int(tt.expectedChunksInFirstBatch)) - for _, chunk := range dbChunks { - assert.Equal(t, batches[0].Hash, chunk.BatchHash) - assert.Equal(t, types.ProvingTaskUnassigned, types.ProvingStatus(chunk.ProvingStatus)) - } - } - }) - } -} - -func testBatchCommitGasAndCalldataSizeCodecv0Estimation(t *testing.T) { - db := setupDB(t) - defer database.CloseDB(db) - - // Add genesis batch. - block := &encoding.Block{ - Header: &gethTypes.Header{ - Number: big.NewInt(0), - }, - RowConsumption: &gethTypes.RowConsumption{}, - } - chunk := &encoding.Chunk{ - Blocks: []*encoding.Block{block}, - } - chunkOrm := orm.NewChunk(db) - _, err := chunkOrm.InsertChunk(context.Background(), chunk, encoding.CodecV0, utils.ChunkMetrics{}) - assert.NoError(t, err) - batch := &encoding.Batch{ - Index: 0, - TotalL1MessagePoppedBefore: 0, - ParentBatchHash: common.Hash{}, - Chunks: []*encoding.Chunk{chunk}, - } - batchOrm := orm.NewBatch(db) - _, err = batchOrm.InsertBatch(context.Background(), batch, encoding.CodecV0, utils.BatchMetrics{}) - assert.NoError(t, err) - - l2BlockOrm := orm.NewL2Block(db) - err = l2BlockOrm.InsertL2Blocks(context.Background(), []*encoding.Block{block1, block2}) - assert.NoError(t, err) - - cp := NewChunkProposer(context.Background(), &config.ChunkProposerConfig{ - MaxBlockNumPerChunk: 1, - MaxTxNumPerChunk: 10000, - MaxL1CommitGasPerChunk: 50000000000, - MaxL1CommitCalldataSizePerChunk: 1000000, - MaxRowConsumptionPerChunk: 1000000, - ChunkTimeoutSec: 300, - GasCostIncreaseMultiplier: 1.2, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, ¶ms.ChainConfig{}, db, nil) - cp.TryProposeChunk() // chunk1 contains block1 - cp.TryProposeChunk() // chunk2 contains block2 - - chunks, err := chunkOrm.GetChunksInRange(context.Background(), 1, 2) - assert.NoError(t, err) - assert.Equal(t, uint64(5082), chunks[0].TotalL1CommitGas) - assert.Equal(t, uint64(298), chunks[0].TotalL1CommitCalldataSize) - assert.Equal(t, uint64(93658), chunks[1].TotalL1CommitGas) - assert.Equal(t, uint64(5737), chunks[1].TotalL1CommitCalldataSize) - - bp := NewBatchProposer(context.Background(), &config.BatchProposerConfig{ - MaxL1CommitGasPerBatch: 50000000000, - MaxL1CommitCalldataSizePerBatch: 1000000, - BatchTimeoutSec: 0, - GasCostIncreaseMultiplier: 1.2, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, ¶ms.ChainConfig{}, db, nil) - bp.TryProposeBatch() - - batches, err := batchOrm.GetBatches(context.Background(), map[string]interface{}{}, []string{}, 0) - assert.NoError(t, err) - assert.Len(t, batches, 2) - batches = batches[1:] - assert.Equal(t, uint64(1), batches[0].StartChunkIndex) - assert.Equal(t, uint64(2), batches[0].EndChunkIndex) - assert.Equal(t, types.RollupPending, types.RollupStatus(batches[0].RollupStatus)) - assert.Equal(t, types.ProvingTaskUnassigned, types.ProvingStatus(batches[0].ProvingStatus)) - - dbChunks, err := chunkOrm.GetChunksInRange(context.Background(), 1, 2) - assert.NoError(t, err) - assert.Len(t, dbChunks, 2) - for _, chunk := range dbChunks { - assert.Equal(t, batches[0].Hash, chunk.BatchHash) - assert.Equal(t, types.ProvingTaskUnassigned, types.ProvingStatus(chunk.ProvingStatus)) - } - - assert.Equal(t, uint64(256463), batches[0].TotalL1CommitGas) - assert.Equal(t, uint64(6035), batches[0].TotalL1CommitCalldataSize) -} - -func testBatchCommitGasAndCalldataSizeCodecv1Estimation(t *testing.T) { - db := setupDB(t) - defer database.CloseDB(db) - - // Add genesis batch. - block := &encoding.Block{ - Header: &gethTypes.Header{ - Number: big.NewInt(0), - }, - RowConsumption: &gethTypes.RowConsumption{}, - } - chunk := &encoding.Chunk{ - Blocks: []*encoding.Block{block}, - } - chunkOrm := orm.NewChunk(db) - _, err := chunkOrm.InsertChunk(context.Background(), chunk, encoding.CodecV0, utils.ChunkMetrics{}) - assert.NoError(t, err) - batch := &encoding.Batch{ - Index: 0, - TotalL1MessagePoppedBefore: 0, - ParentBatchHash: common.Hash{}, - Chunks: []*encoding.Chunk{chunk}, - } - batchOrm := orm.NewBatch(db) - _, err = batchOrm.InsertBatch(context.Background(), batch, encoding.CodecV0, utils.BatchMetrics{}) - assert.NoError(t, err) - - l2BlockOrm := orm.NewL2Block(db) - err = l2BlockOrm.InsertL2Blocks(context.Background(), []*encoding.Block{block1, block2}) - assert.NoError(t, err) - - cp := NewChunkProposer(context.Background(), &config.ChunkProposerConfig{ - MaxBlockNumPerChunk: 1, - MaxTxNumPerChunk: 10000, - MaxL1CommitGasPerChunk: 50000000000, - MaxL1CommitCalldataSizePerChunk: 1000000, - MaxRowConsumptionPerChunk: 1000000, - ChunkTimeoutSec: 300, - GasCostIncreaseMultiplier: 1.2, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, ¶ms.ChainConfig{BernoulliBlock: big.NewInt(0)}, db, nil) - cp.TryProposeChunk() // chunk1 contains block1 - cp.TryProposeChunk() // chunk2 contains block2 - - chunks, err := chunkOrm.GetChunksInRange(context.Background(), 1, 2) - assert.NoError(t, err) - assert.Equal(t, uint64(1124), chunks[0].TotalL1CommitGas) - assert.Equal(t, uint64(60), chunks[0].TotalL1CommitCalldataSize) - assert.Equal(t, uint64(1124), chunks[1].TotalL1CommitGas) - assert.Equal(t, uint64(60), chunks[1].TotalL1CommitCalldataSize) - - bp := NewBatchProposer(context.Background(), &config.BatchProposerConfig{ - MaxL1CommitGasPerBatch: 50000000000, - MaxL1CommitCalldataSizePerBatch: 1000000, - BatchTimeoutSec: 0, - GasCostIncreaseMultiplier: 1.2, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, ¶ms.ChainConfig{BernoulliBlock: big.NewInt(0)}, db, nil) - bp.TryProposeBatch() - - batches, err := batchOrm.GetBatches(context.Background(), map[string]interface{}{}, []string{}, 0) - assert.NoError(t, err) - assert.Len(t, batches, 2) - batches = batches[1:] - assert.Equal(t, uint64(1), batches[0].StartChunkIndex) - assert.Equal(t, uint64(2), batches[0].EndChunkIndex) - assert.Equal(t, types.RollupPending, types.RollupStatus(batches[0].RollupStatus)) - assert.Equal(t, types.ProvingTaskUnassigned, types.ProvingStatus(batches[0].ProvingStatus)) - - dbChunks, err := chunkOrm.GetChunksInRange(context.Background(), 1, 2) - assert.NoError(t, err) - assert.Len(t, dbChunks, 2) - for _, chunk := range dbChunks { - assert.Equal(t, batches[0].Hash, chunk.BatchHash) - assert.Equal(t, types.ProvingTaskUnassigned, types.ProvingStatus(chunk.ProvingStatus)) + maxL1CommitGas: 249179, + maxL1CommitCalldataSize: 1000000, + batchTimeoutSec: 1000000000000, + expectedBatchesLen: 1, + expectedChunksInFirstBatch: 1, + }, + { + name: "MaxL1CommitCalldataSizePerBatchIsFirstChunk", + maxL1CommitGas: 50000000000, + maxL1CommitCalldataSize: 60, + batchTimeoutSec: 1000000000000, + expectedBatchesLen: 1, + expectedChunksInFirstBatch: 1, + }, } - assert.Equal(t, uint64(159350), batches[0].TotalL1CommitGas) - assert.Equal(t, uint64(120), batches[0].TotalL1CommitCalldataSize) -} - -func testBatchCommitGasAndCalldataSizeCodecv2Estimation(t *testing.T) { - db := setupDB(t) - defer database.CloseDB(db) + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + db := setupDB(t) + defer database.CloseDB(db) - // Add genesis batch. - block := &encoding.Block{ - Header: &gethTypes.Header{ - Number: big.NewInt(0), - }, - RowConsumption: &gethTypes.RowConsumption{}, - } - chunk := &encoding.Chunk{ - Blocks: []*encoding.Block{block}, - } - chunkOrm := orm.NewChunk(db) - _, err := chunkOrm.InsertChunk(context.Background(), chunk, encoding.CodecV0, utils.ChunkMetrics{}) - assert.NoError(t, err) - batch := &encoding.Batch{ - Index: 0, - TotalL1MessagePoppedBefore: 0, - ParentBatchHash: common.Hash{}, - Chunks: []*encoding.Chunk{chunk}, - } - batchOrm := orm.NewBatch(db) - _, err = batchOrm.InsertBatch(context.Background(), batch, encoding.CodecV0, utils.BatchMetrics{}) - assert.NoError(t, err) + // Add genesis batch. + block := &encoding.Block{ + Header: &gethTypes.Header{ + Number: big.NewInt(0), + }, + RowConsumption: &gethTypes.RowConsumption{}, + } + chunk := &encoding.Chunk{ + Blocks: []*encoding.Block{block}, + } + chunkOrm := orm.NewChunk(db) + _, err := chunkOrm.InsertChunk(context.Background(), chunk, encoding.CodecV0, utils.ChunkMetrics{}) + assert.NoError(t, err) + batch := &encoding.Batch{ + Index: 0, + TotalL1MessagePoppedBefore: 0, + ParentBatchHash: common.Hash{}, + Chunks: []*encoding.Chunk{chunk}, + } + batchOrm := orm.NewBatch(db) + _, err = batchOrm.InsertBatch(context.Background(), batch, encoding.CodecV0, utils.BatchMetrics{}) + assert.NoError(t, err) - l2BlockOrm := orm.NewL2Block(db) - err = l2BlockOrm.InsertL2Blocks(context.Background(), []*encoding.Block{block1, block2}) - assert.NoError(t, err) + l2BlockOrm := orm.NewL2Block(db) + err = l2BlockOrm.InsertL2Blocks(context.Background(), []*encoding.Block{block1, block2}) + assert.NoError(t, err) - cp := NewChunkProposer(context.Background(), &config.ChunkProposerConfig{ - MaxBlockNumPerChunk: 1, - MaxTxNumPerChunk: 10000, - MaxL1CommitGasPerChunk: 50000000000, - MaxL1CommitCalldataSizePerChunk: 1000000, - MaxRowConsumptionPerChunk: 1000000, - ChunkTimeoutSec: 300, - GasCostIncreaseMultiplier: 1.2, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, ¶ms.ChainConfig{BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0)}, db, nil) - cp.TryProposeChunk() // chunk1 contains block1 - cp.TryProposeChunk() // chunk2 contains block2 + cp := NewChunkProposer(context.Background(), &config.ChunkProposerConfig{ + MaxBlockNumPerChunk: 1, + MaxTxNumPerChunk: 10000, + MaxL1CommitGasPerChunk: 50000000000, + MaxL1CommitCalldataSizePerChunk: 1000000, + MaxRowConsumptionPerChunk: 1000000, + ChunkTimeoutSec: 300, + GasCostIncreaseMultiplier: 1.2, + MaxUncompressedBatchBytesSize: math.MaxUint64, + }, encoding.CodecV4, ¶ms.ChainConfig{ + LondonBlock: big.NewInt(0), + BernoulliBlock: big.NewInt(0), + CurieBlock: big.NewInt(0), + DarwinTime: new(uint64), + DarwinV2Time: new(uint64), + }, db, nil) + cp.TryProposeChunk() // chunk1 contains block1 + cp.TryProposeChunk() // chunk2 contains block2 - chunks, err := chunkOrm.GetChunksInRange(context.Background(), 1, 2) - assert.NoError(t, err) - assert.Equal(t, uint64(1124), chunks[0].TotalL1CommitGas) - assert.Equal(t, uint64(60), chunks[0].TotalL1CommitCalldataSize) - assert.Equal(t, uint64(1124), chunks[1].TotalL1CommitGas) - assert.Equal(t, uint64(60), chunks[1].TotalL1CommitCalldataSize) + chunks, err := chunkOrm.GetChunksInRange(context.Background(), 1, 2) + assert.NoError(t, err) + assert.Equal(t, uint64(51124), chunks[0].TotalL1CommitGas) + assert.Equal(t, uint64(60), chunks[0].TotalL1CommitCalldataSize) + assert.Equal(t, uint64(51124), chunks[1].TotalL1CommitGas) + assert.Equal(t, uint64(60), chunks[1].TotalL1CommitCalldataSize) - bp := NewBatchProposer(context.Background(), &config.BatchProposerConfig{ - MaxL1CommitGasPerBatch: 50000000000, - MaxL1CommitCalldataSizePerBatch: 1000000, - BatchTimeoutSec: 0, - GasCostIncreaseMultiplier: 1.2, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, ¶ms.ChainConfig{BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0)}, db, nil) - bp.TryProposeBatch() + bp := NewBatchProposer(context.Background(), &config.BatchProposerConfig{ + MaxL1CommitGasPerBatch: tt.maxL1CommitGas, + MaxL1CommitCalldataSizePerBatch: tt.maxL1CommitCalldataSize, + BatchTimeoutSec: tt.batchTimeoutSec, + GasCostIncreaseMultiplier: 1.2, + MaxUncompressedBatchBytesSize: math.MaxUint64, + }, encoding.CodecV4, ¶ms.ChainConfig{ + LondonBlock: big.NewInt(0), + BernoulliBlock: big.NewInt(0), + CurieBlock: big.NewInt(0), + DarwinTime: new(uint64), + DarwinV2Time: new(uint64), + }, db, nil) + bp.TryProposeBatch() - batches, err := batchOrm.GetBatches(context.Background(), map[string]interface{}{}, []string{}, 0) - assert.NoError(t, err) - assert.Len(t, batches, 2) - batches = batches[1:] - assert.Equal(t, uint64(1), batches[0].StartChunkIndex) - assert.Equal(t, uint64(2), batches[0].EndChunkIndex) - assert.Equal(t, types.RollupPending, types.RollupStatus(batches[0].RollupStatus)) - assert.Equal(t, types.ProvingTaskUnassigned, types.ProvingStatus(batches[0].ProvingStatus)) + batches, err := batchOrm.GetBatches(context.Background(), map[string]interface{}{}, []string{}, 0) + assert.NoError(t, err) + assert.Len(t, batches, tt.expectedBatchesLen+1) + batches = batches[1:] + if tt.expectedBatchesLen > 0 { + assert.Equal(t, uint64(1), batches[0].StartChunkIndex) + assert.Equal(t, tt.expectedChunksInFirstBatch, batches[0].EndChunkIndex) + assert.Equal(t, types.RollupPending, types.RollupStatus(batches[0].RollupStatus)) + assert.Equal(t, types.ProvingTaskUnassigned, types.ProvingStatus(batches[0].ProvingStatus)) - dbChunks, err := chunkOrm.GetChunksInRange(context.Background(), 1, 2) - assert.NoError(t, err) - assert.Len(t, dbChunks, 2) - for _, chunk := range dbChunks { - assert.Equal(t, batches[0].Hash, chunk.BatchHash) - assert.Equal(t, types.ProvingTaskUnassigned, types.ProvingStatus(chunk.ProvingStatus)) + dbChunks, err := chunkOrm.GetChunksInRange(context.Background(), 1, tt.expectedChunksInFirstBatch) + assert.NoError(t, err) + assert.Len(t, dbChunks, int(tt.expectedChunksInFirstBatch)) + for _, chunk := range dbChunks { + assert.Equal(t, batches[0].Hash, chunk.BatchHash) + assert.Equal(t, types.ProvingTaskUnassigned, types.ProvingStatus(chunk.ProvingStatus)) + } + } + }) } - - assert.Equal(t, uint64(159350), batches[0].TotalL1CommitGas) - assert.Equal(t, uint64(120), batches[0].TotalL1CommitCalldataSize) } -func testBatchCommitGasAndCalldataSizeCodecv3Estimation(t *testing.T) { +func testBatchCommitGasAndCalldataSizeEstimationCodecV4(t *testing.T) { db := setupDB(t) defer database.CloseDB(db) @@ -878,7 +211,7 @@ func testBatchCommitGasAndCalldataSizeCodecv3Estimation(t *testing.T) { ChunkTimeoutSec: 300, GasCostIncreaseMultiplier: 1.2, MaxUncompressedBatchBytesSize: math.MaxUint64, - }, ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64)}, db, nil) + }, encoding.CodecV4, ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64), DarwinV2Time: new(uint64)}, db, nil) cp.TryProposeChunk() // chunk1 contains block1 cp.TryProposeChunk() // chunk2 contains block2 @@ -895,7 +228,7 @@ func testBatchCommitGasAndCalldataSizeCodecv3Estimation(t *testing.T) { BatchTimeoutSec: 0, GasCostIncreaseMultiplier: 1.2, MaxUncompressedBatchBytesSize: math.MaxUint64, - }, ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64)}, db, nil) + }, encoding.CodecV4, ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64), DarwinV2Time: new(uint64)}, db, nil) bp.TryProposeBatch() batches, err := batchOrm.GetBatches(context.Background(), map[string]interface{}{}, []string{}, 0) @@ -919,8 +252,8 @@ func testBatchCommitGasAndCalldataSizeCodecv3Estimation(t *testing.T) { assert.Equal(t, uint64(120), batches[0].TotalL1CommitCalldataSize) } -func testBatchProposerBlobSizeLimit(t *testing.T) { - codecVersions := []encoding.CodecVersion{encoding.CodecV0, encoding.CodecV1, encoding.CodecV2, encoding.CodecV3} +func testBatchProposerBlobSizeLimitCodecV4(t *testing.T) { + codecVersions := []encoding.CodecVersion{encoding.CodecV4} for _, codecVersion := range codecVersions { db := setupDB(t) @@ -948,14 +281,10 @@ func testBatchProposerBlobSizeLimit(t *testing.T) { assert.NoError(t, err) var chainConfig *params.ChainConfig - if codecVersion == encoding.CodecV0 { // will never hit blob size limit - chainConfig = ¶ms.ChainConfig{} - } else if codecVersion == encoding.CodecV1 { - chainConfig = ¶ms.ChainConfig{BernoulliBlock: big.NewInt(0)} - } else if codecVersion == encoding.CodecV2 { - chainConfig = ¶ms.ChainConfig{BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0)} + if codecVersion == encoding.CodecV4 { + chainConfig = ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64), DarwinV2Time: new(uint64)} } else { - chainConfig = ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64)} + assert.Fail(t, "unsupported codec version, expected CodecV4") } cp := NewChunkProposer(context.Background(), &config.ChunkProposerConfig{ @@ -967,7 +296,7 @@ func testBatchProposerBlobSizeLimit(t *testing.T) { ChunkTimeoutSec: 0, GasCostIncreaseMultiplier: 1, MaxUncompressedBatchBytesSize: math.MaxUint64, - }, chainConfig, db, nil) + }, encoding.CodecV4, chainConfig, db, nil) blockHeight := int64(0) block = readBlockFromJSON(t, "../../../testdata/blockTrace_03.json") @@ -988,7 +317,7 @@ func testBatchProposerBlobSizeLimit(t *testing.T) { BatchTimeoutSec: math.MaxUint32, GasCostIncreaseMultiplier: 1, MaxUncompressedBatchBytesSize: math.MaxUint64, - }, chainConfig, db, nil) + }, encoding.CodecV4, chainConfig, db, nil) for i := 0; i < 2; i++ { bp.TryProposeBatch() @@ -1000,18 +329,11 @@ func testBatchProposerBlobSizeLimit(t *testing.T) { var expectedNumBatches int var numChunksMultiplier uint64 - if codecVersion == encoding.CodecV0 { - expectedNumBatches = 2 - numChunksMultiplier = 15 - } else if codecVersion == encoding.CodecV1 { - expectedNumBatches = 2 - numChunksMultiplier = 1 - } else if codecVersion == encoding.CodecV2 { + if codecVersion == encoding.CodecV4 { expectedNumBatches = 2 numChunksMultiplier = 45 } else { - expectedNumBatches = 2 - numChunksMultiplier = 45 + assert.Fail(t, "unsupported codec version, expected CodecV4") } assert.Len(t, batches, expectedNumBatches) @@ -1022,8 +344,8 @@ func testBatchProposerBlobSizeLimit(t *testing.T) { } } -func testBatchProposerMaxChunkNumPerBatchLimit(t *testing.T) { - codecVersions := []encoding.CodecVersion{encoding.CodecV0, encoding.CodecV1, encoding.CodecV2, encoding.CodecV3} +func testBatchProposerMaxChunkNumPerBatchLimitCodecV4(t *testing.T) { + codecVersions := []encoding.CodecVersion{encoding.CodecV4} for _, codecVersion := range codecVersions { db := setupDB(t) @@ -1052,18 +374,11 @@ func testBatchProposerMaxChunkNumPerBatchLimit(t *testing.T) { var expectedChunkNum uint64 var chainConfig *params.ChainConfig - if codecVersion == encoding.CodecV0 { - chainConfig = ¶ms.ChainConfig{} - expectedChunkNum = 15 - } else if codecVersion == encoding.CodecV1 { - chainConfig = ¶ms.ChainConfig{BernoulliBlock: big.NewInt(0)} - expectedChunkNum = 15 - } else if codecVersion == encoding.CodecV2 { - chainConfig = ¶ms.ChainConfig{BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0)} + if codecVersion == encoding.CodecV4 { + chainConfig = ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64), DarwinV2Time: new(uint64)} expectedChunkNum = 45 } else { - chainConfig = ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64)} - expectedChunkNum = 45 + assert.Fail(t, "unsupported codec version, expected CodecV4") } cp := NewChunkProposer(context.Background(), &config.ChunkProposerConfig{ @@ -1075,7 +390,7 @@ func testBatchProposerMaxChunkNumPerBatchLimit(t *testing.T) { ChunkTimeoutSec: 0, GasCostIncreaseMultiplier: 1, MaxUncompressedBatchBytesSize: math.MaxUint64, - }, chainConfig, db, nil) + }, encoding.CodecV4, chainConfig, db, nil) block = readBlockFromJSON(t, "../../../testdata/blockTrace_03.json") for blockHeight := int64(1); blockHeight <= 60; blockHeight++ { @@ -1091,7 +406,7 @@ func testBatchProposerMaxChunkNumPerBatchLimit(t *testing.T) { BatchTimeoutSec: math.MaxUint32, GasCostIncreaseMultiplier: 1, MaxUncompressedBatchBytesSize: math.MaxUint64, - }, chainConfig, db, nil) + }, encoding.CodecV4, chainConfig, db, nil) bp.TryProposeBatch() batches, err := batchOrm.GetBatches(context.Background(), map[string]interface{}{}, []string{}, 0) @@ -1104,86 +419,3 @@ func testBatchProposerMaxChunkNumPerBatchLimit(t *testing.T) { database.CloseDB(db) } } - -func testBatchProposerRespectHardforks(t *testing.T) { - db := setupDB(t) - defer database.CloseDB(db) - - chainConfig := ¶ms.ChainConfig{ - LondonBlock: big.NewInt(0), - BernoulliBlock: big.NewInt(1), - CurieBlock: big.NewInt(2), - DarwinTime: func() *uint64 { t := uint64(4); return &t }(), - } - - // Add genesis batch. - block := &encoding.Block{ - Header: &gethTypes.Header{ - Number: big.NewInt(0), - }, - RowConsumption: &gethTypes.RowConsumption{}, - } - chunk := &encoding.Chunk{ - Blocks: []*encoding.Block{block}, - } - chunkOrm := orm.NewChunk(db) - _, err := chunkOrm.InsertChunk(context.Background(), chunk, encoding.CodecV0, utils.ChunkMetrics{}) - assert.NoError(t, err) - batch := &encoding.Batch{ - Index: 0, - TotalL1MessagePoppedBefore: 0, - ParentBatchHash: common.Hash{}, - Chunks: []*encoding.Chunk{chunk}, - } - batchOrm := orm.NewBatch(db) - _, err = batchOrm.InsertBatch(context.Background(), batch, encoding.CodecV0, utils.BatchMetrics{}) - assert.NoError(t, err) - - cp := NewChunkProposer(context.Background(), &config.ChunkProposerConfig{ - MaxBlockNumPerChunk: math.MaxUint64, - MaxTxNumPerChunk: math.MaxUint64, - MaxL1CommitGasPerChunk: math.MaxUint64, - MaxL1CommitCalldataSizePerChunk: math.MaxUint64, - MaxRowConsumptionPerChunk: math.MaxUint64, - ChunkTimeoutSec: 0, - GasCostIncreaseMultiplier: 1, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, chainConfig, db, nil) - - block = readBlockFromJSON(t, "../../../testdata/blockTrace_02.json") - for i := int64(1); i <= 60; i++ { - block.Header.Number = big.NewInt(i) - block.Header.Time = uint64(i) - err = orm.NewL2Block(db).InsertL2Blocks(context.Background(), []*encoding.Block{block}) - assert.NoError(t, err) - } - - for i := 0; i < 5; i++ { - cp.TryProposeChunk() - } - - bp := NewBatchProposer(context.Background(), &config.BatchProposerConfig{ - MaxL1CommitGasPerBatch: math.MaxUint64, - MaxL1CommitCalldataSizePerBatch: math.MaxUint64, - BatchTimeoutSec: 0, - GasCostIncreaseMultiplier: 1, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, chainConfig, db, nil) - - for i := 0; i < 5; i++ { - bp.TryProposeBatch() - } - - batches, err := batchOrm.GetBatches(context.Background(), map[string]interface{}{}, []string{}, 0) - assert.NoError(t, err) - assert.Len(t, batches, 4) - - expectedEndChunkIndices := []uint64{0, 1, 3, 4} - expectedEndBlockNumbers := []uint64{0, 1, 3, 60} - for i, batch := range batches { - assert.Equal(t, expectedEndChunkIndices[i], batch.EndChunkIndex) - chunk, err := chunkOrm.GetChunkByIndex(context.Background(), batch.EndChunkIndex) - assert.NoError(t, err) - assert.Equal(t, expectedEndBlockNumbers[i], chunk.EndBlockNumber) - } -} diff --git a/rollup/internal/controller/watcher/bundle_proposer.go b/rollup/internal/controller/watcher/bundle_proposer.go index 686ad580c0..1a37a6265a 100644 --- a/rollup/internal/controller/watcher/bundle_proposer.go +++ b/rollup/internal/controller/watcher/bundle_proposer.go @@ -3,6 +3,7 @@ package watcher import ( "context" "errors" + "fmt" "time" "github.com/prometheus/client_golang/prometheus" @@ -28,7 +29,8 @@ type BundleProposer struct { maxBatchNumPerBundle uint64 bundleTimeoutSec uint64 - chainCfg *params.ChainConfig + minCodecVersion encoding.CodecVersion + chainCfg *params.ChainConfig bundleProposerCircleTotal prometheus.Counter proposeBundleFailureTotal prometheus.Counter @@ -40,7 +42,7 @@ type BundleProposer struct { } // NewBundleProposer creates a new BundleProposer instance. -func NewBundleProposer(ctx context.Context, cfg *config.BundleProposerConfig, chainCfg *params.ChainConfig, db *gorm.DB, reg prometheus.Registerer) *BundleProposer { +func NewBundleProposer(ctx context.Context, cfg *config.BundleProposerConfig, minCodecVersion encoding.CodecVersion, chainCfg *params.ChainConfig, db *gorm.DB, reg prometheus.Registerer) *BundleProposer { log.Info("new bundle proposer", "bundleBatchesNum", cfg.MaxBatchNumPerBundle, "bundleTimeoutSec", cfg.BundleTimeoutSec) p := &BundleProposer{ @@ -51,6 +53,7 @@ func NewBundleProposer(ctx context.Context, cfg *config.BundleProposerConfig, ch bundleOrm: orm.NewBundle(db), maxBatchNumPerBundle: cfg.MaxBatchNumPerBundle, bundleTimeoutSec: cfg.BundleTimeoutSec, + minCodecVersion: minCodecVersion, chainCfg: chainCfg, bundleProposerCircleTotal: promauto.With(reg).NewCounter(prometheus.CounterOpts{ @@ -130,7 +133,7 @@ func (p *BundleProposer) proposeBundle() error { // select at most maxBlocksThisChunk blocks maxBatchesThisBundle := p.maxBatchNumPerBundle - batches, err := p.batchOrm.GetBatchesGEIndexGECodecVersion(p.ctx, firstUnbundledBatchIndex, encoding.CodecV3, int(maxBatchesThisBundle)) + batches, err := p.batchOrm.GetBatchesGEIndexGECodecVersion(p.ctx, firstUnbundledBatchIndex, p.minCodecVersion, int(maxBatchesThisBundle)) if err != nil { return err } @@ -153,6 +156,11 @@ func (p *BundleProposer) proposeBundle() error { hardforkName := encoding.GetHardforkName(p.chainCfg, firstChunk.StartBlockNumber, firstChunk.StartBlockTime) codecVersion := encoding.CodecVersion(batches[0].CodecVersion) + + if codecVersion < p.minCodecVersion { + return fmt.Errorf("unsupported codec version: %v, expected at least %v", codecVersion, p.minCodecVersion) + } + for i := 1; i < len(batches); i++ { chunk, err := p.chunkOrm.GetChunkByIndex(p.ctx, batches[i].StartChunkIndex) if err != nil { diff --git a/rollup/internal/controller/watcher/bundle_proposer_test.go b/rollup/internal/controller/watcher/bundle_proposer_test.go index c866f5f321..4a0a4219b0 100644 --- a/rollup/internal/controller/watcher/bundle_proposer_test.go +++ b/rollup/internal/controller/watcher/bundle_proposer_test.go @@ -20,7 +20,7 @@ import ( "scroll-tech/rollup/internal/utils" ) -func testBundleProposerLimits(t *testing.T) { +func testBundleProposerLimitsCodecV4(t *testing.T) { tests := []struct { name string maxBatchNumPerBundle uint64 @@ -88,7 +88,7 @@ func testBundleProposerLimits(t *testing.T) { err = l2BlockOrm.InsertL2Blocks(context.Background(), []*encoding.Block{block1, block2}) assert.NoError(t, err) - chainConfig := ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64)} + chainConfig := ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64), DarwinV2Time: new(uint64)} cp := NewChunkProposer(context.Background(), &config.ChunkProposerConfig{ MaxBlockNumPerChunk: 1, @@ -99,7 +99,7 @@ func testBundleProposerLimits(t *testing.T) { ChunkTimeoutSec: math.MaxUint32, GasCostIncreaseMultiplier: 1, MaxUncompressedBatchBytesSize: math.MaxUint64, - }, chainConfig, db, nil) + }, encoding.CodecV4, chainConfig, db, nil) bap := NewBatchProposer(context.Background(), &config.BatchProposerConfig{ MaxL1CommitGasPerBatch: math.MaxUint64, @@ -107,7 +107,7 @@ func testBundleProposerLimits(t *testing.T) { BatchTimeoutSec: 0, GasCostIncreaseMultiplier: 1, MaxUncompressedBatchBytesSize: math.MaxUint64, - }, chainConfig, db, nil) + }, encoding.CodecV4, chainConfig, db, nil) cp.TryProposeChunk() // chunk1 contains block1 bap.TryProposeBatch() // batch1 contains chunk1 @@ -117,7 +117,7 @@ func testBundleProposerLimits(t *testing.T) { bup := NewBundleProposer(context.Background(), &config.BundleProposerConfig{ MaxBatchNumPerBundle: tt.maxBatchNumPerBundle, BundleTimeoutSec: tt.bundleTimeoutSec, - }, chainConfig, db, nil) + }, encoding.CodecV4, chainConfig, db, nil) bup.TryProposeBundle() @@ -134,94 +134,3 @@ func testBundleProposerLimits(t *testing.T) { }) } } - -func testBundleProposerRespectHardforks(t *testing.T) { - db := setupDB(t) - defer database.CloseDB(db) - - chainConfig := ¶ms.ChainConfig{ - LondonBlock: big.NewInt(0), - BernoulliBlock: big.NewInt(1), - CurieBlock: big.NewInt(2), - DarwinTime: func() *uint64 { t := uint64(4); return &t }(), - } - - // Add genesis batch. - block := &encoding.Block{ - Header: &gethTypes.Header{ - Number: big.NewInt(0), - }, - RowConsumption: &gethTypes.RowConsumption{}, - } - chunk := &encoding.Chunk{ - Blocks: []*encoding.Block{block}, - } - chunkOrm := orm.NewChunk(db) - _, err := chunkOrm.InsertChunk(context.Background(), chunk, encoding.CodecV0, utils.ChunkMetrics{}) - assert.NoError(t, err) - batch := &encoding.Batch{ - Index: 0, - TotalL1MessagePoppedBefore: 0, - ParentBatchHash: common.Hash{}, - Chunks: []*encoding.Chunk{chunk}, - } - batchOrm := orm.NewBatch(db) - _, err = batchOrm.InsertBatch(context.Background(), batch, encoding.CodecV0, utils.BatchMetrics{}) - assert.NoError(t, err) - - cp := NewChunkProposer(context.Background(), &config.ChunkProposerConfig{ - MaxBlockNumPerChunk: math.MaxUint64, - MaxTxNumPerChunk: math.MaxUint64, - MaxL1CommitGasPerChunk: math.MaxUint64, - MaxL1CommitCalldataSizePerChunk: math.MaxUint64, - MaxRowConsumptionPerChunk: math.MaxUint64, - ChunkTimeoutSec: 0, - GasCostIncreaseMultiplier: 1, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, chainConfig, db, nil) - - block = readBlockFromJSON(t, "../../../testdata/blockTrace_02.json") - for i := int64(1); i <= 60; i++ { - block.Header.Number = big.NewInt(i) - block.Header.Time = uint64(i) - err = orm.NewL2Block(db).InsertL2Blocks(context.Background(), []*encoding.Block{block}) - assert.NoError(t, err) - } - - for i := 0; i < 5; i++ { - cp.TryProposeChunk() - } - - bap := NewBatchProposer(context.Background(), &config.BatchProposerConfig{ - MaxL1CommitGasPerBatch: math.MaxUint64, - MaxL1CommitCalldataSizePerBatch: math.MaxUint64, - BatchTimeoutSec: 0, - GasCostIncreaseMultiplier: 1, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, chainConfig, db, nil) - - for i := 0; i < 5; i++ { - bap.TryProposeBatch() - } - - bup := NewBundleProposer(context.Background(), &config.BundleProposerConfig{ - MaxBatchNumPerBundle: math.MaxUint64, - BundleTimeoutSec: 0, - }, chainConfig, db, nil) - - for i := 0; i < 5; i++ { - bup.TryProposeBundle() - } - - bundleOrm := orm.NewBundle(db) - bundles, err := bundleOrm.GetBundles(context.Background(), map[string]interface{}{}, []string{}, 0) - assert.NoError(t, err) - assert.Len(t, bundles, 1) - - expectedStartBatchIndices := []uint64{3} - expectedEndChunkIndices := []uint64{3} - for i, bundle := range bundles { - assert.Equal(t, expectedStartBatchIndices[i], bundle.StartBatchIndex) - assert.Equal(t, expectedEndChunkIndices[i], bundle.EndBatchIndex) - } -} diff --git a/rollup/internal/controller/watcher/chunk_proposer.go b/rollup/internal/controller/watcher/chunk_proposer.go index 99eaca9d44..87056e6464 100644 --- a/rollup/internal/controller/watcher/chunk_proposer.go +++ b/rollup/internal/controller/watcher/chunk_proposer.go @@ -34,7 +34,8 @@ type ChunkProposer struct { gasCostIncreaseMultiplier float64 maxUncompressedBatchBytesSize uint64 - chainCfg *params.ChainConfig + minCodecVersion encoding.CodecVersion + chainCfg *params.ChainConfig chunkProposerCircleTotal prometheus.Counter proposeChunkFailureTotal prometheus.Counter @@ -60,7 +61,7 @@ type ChunkProposer struct { } // NewChunkProposer creates a new ChunkProposer instance. -func NewChunkProposer(ctx context.Context, cfg *config.ChunkProposerConfig, chainCfg *params.ChainConfig, db *gorm.DB, reg prometheus.Registerer) *ChunkProposer { +func NewChunkProposer(ctx context.Context, cfg *config.ChunkProposerConfig, minCodecVersion encoding.CodecVersion, chainCfg *params.ChainConfig, db *gorm.DB, reg prometheus.Registerer) *ChunkProposer { log.Info("new chunk proposer", "maxBlockNumPerChunk", cfg.MaxBlockNumPerChunk, "maxTxNumPerChunk", cfg.MaxTxNumPerChunk, @@ -85,6 +86,7 @@ func NewChunkProposer(ctx context.Context, cfg *config.ChunkProposerConfig, chai chunkTimeoutSec: cfg.ChunkTimeoutSec, gasCostIncreaseMultiplier: cfg.GasCostIncreaseMultiplier, maxUncompressedBatchBytesSize: cfg.MaxUncompressedBatchBytesSize, + minCodecVersion: minCodecVersion, chainCfg: chainCfg, chunkProposerCircleTotal: promauto.With(reg).NewCounter(prometheus.CounterOpts{ @@ -277,6 +279,10 @@ func (p *ChunkProposer) proposeChunk() error { codecVersion := encoding.GetCodecVersion(p.chainCfg, blocks[0].Header.Number.Uint64(), blocks[0].Header.Time) + if codecVersion < p.minCodecVersion { + return fmt.Errorf("unsupported codec version: %v, expected at least %v", codecVersion, p.minCodecVersion) + } + // Including Curie block in a sole chunk. if p.chainCfg.CurieBlock != nil && blocks[0].Header.Number.Cmp(p.chainCfg.CurieBlock) == 0 { chunk := encoding.Chunk{Blocks: blocks[:1]} diff --git a/rollup/internal/controller/watcher/chunk_proposer_test.go b/rollup/internal/controller/watcher/chunk_proposer_test.go index 31d4c1ddf4..4886fbebe0 100644 --- a/rollup/internal/controller/watcher/chunk_proposer_test.go +++ b/rollup/internal/controller/watcher/chunk_proposer_test.go @@ -16,517 +16,7 @@ import ( "scroll-tech/rollup/internal/orm" ) -func testChunkProposerCodecv0Limits(t *testing.T) { - tests := []struct { - name string - maxBlockNum uint64 - maxTxNum uint64 - maxL1CommitGas uint64 - maxL1CommitCalldataSize uint64 - maxRowConsumption uint64 - chunkTimeoutSec uint64 - expectedChunksLen int - expectedBlocksInFirstChunk int // only be checked when expectedChunksLen > 0 - }{ - { - name: "NoLimitReached", - maxBlockNum: 100, - maxTxNum: 10000, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 0, - }, - { - name: "Timeout", - maxBlockNum: 100, - maxTxNum: 10000, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1000000, - chunkTimeoutSec: 0, - expectedChunksLen: 1, - expectedBlocksInFirstChunk: 2, - }, - { - name: "MaxTxNumPerChunkIs0", - maxBlockNum: 10, - maxTxNum: 0, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 0, - }, - { - name: "MaxL1CommitGasPerChunkIs0", - maxBlockNum: 10, - maxTxNum: 10000, - maxL1CommitGas: 0, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 0, - }, - { - name: "MaxL1CommitCalldataSizePerChunkIs0", - maxBlockNum: 10, - maxTxNum: 10000, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 0, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 0, - }, - { - name: "MaxRowConsumptionPerChunkIs0", - maxBlockNum: 100, - maxTxNum: 10000, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 0, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 0, - }, - { - name: "MaxBlockNumPerChunkIs1", - maxBlockNum: 1, - maxTxNum: 10000, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 1, - expectedBlocksInFirstChunk: 1, - }, - { - name: "MaxTxNumPerChunkIsFirstBlock", - maxBlockNum: 10, - maxTxNum: 2, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 1, - expectedBlocksInFirstChunk: 1, - }, - { - name: "MaxL1CommitGasPerChunkIsFirstBlock", - maxBlockNum: 10, - maxTxNum: 10000, - maxL1CommitGas: 7250, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 1, - expectedBlocksInFirstChunk: 1, - }, - { - name: "MaxL1CommitCalldataSizePerChunkIsFirstBlock", - maxBlockNum: 10, - maxTxNum: 10000, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 298, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 1, - expectedBlocksInFirstChunk: 1, - }, - { - name: "MaxRowConsumptionPerChunkIs1", - maxBlockNum: 10, - maxTxNum: 10000, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 1, - expectedBlocksInFirstChunk: 1, - }, - } - - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - db := setupDB(t) - defer database.CloseDB(db) - - l2BlockOrm := orm.NewL2Block(db) - err := l2BlockOrm.InsertL2Blocks(context.Background(), []*encoding.Block{block1, block2}) - assert.NoError(t, err) - - cp := NewChunkProposer(context.Background(), &config.ChunkProposerConfig{ - MaxBlockNumPerChunk: tt.maxBlockNum, - MaxTxNumPerChunk: tt.maxTxNum, - MaxL1CommitGasPerChunk: tt.maxL1CommitGas, - MaxL1CommitCalldataSizePerChunk: tt.maxL1CommitCalldataSize, - MaxRowConsumptionPerChunk: tt.maxRowConsumption, - ChunkTimeoutSec: tt.chunkTimeoutSec, - GasCostIncreaseMultiplier: 1.2, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, ¶ms.ChainConfig{}, db, nil) - cp.TryProposeChunk() - - chunkOrm := orm.NewChunk(db) - chunks, err := chunkOrm.GetChunksGEIndex(context.Background(), 0, 0) - assert.NoError(t, err) - assert.Len(t, chunks, tt.expectedChunksLen) - - if len(chunks) > 0 { - blockOrm := orm.NewL2Block(db) - chunkHashes, err := blockOrm.GetChunkHashes(context.Background(), tt.expectedBlocksInFirstChunk) - assert.NoError(t, err) - assert.Len(t, chunkHashes, tt.expectedBlocksInFirstChunk) - firstChunkHash := chunks[0].Hash - for _, chunkHash := range chunkHashes { - assert.Equal(t, firstChunkHash, chunkHash) - } - } - }) - } -} - -func testChunkProposerCodecv1Limits(t *testing.T) { - tests := []struct { - name string - maxBlockNum uint64 - maxTxNum uint64 - maxL1CommitGas uint64 - maxL1CommitCalldataSize uint64 - maxRowConsumption uint64 - chunkTimeoutSec uint64 - expectedChunksLen int - expectedBlocksInFirstChunk int // only be checked when expectedChunksLen > 0 - }{ - { - name: "NoLimitReached", - maxBlockNum: 100, - maxTxNum: 10000, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 0, - }, - { - name: "Timeout", - maxBlockNum: 100, - maxTxNum: 10000, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1000000, - chunkTimeoutSec: 0, - expectedChunksLen: 1, - expectedBlocksInFirstChunk: 2, - }, - { - name: "MaxTxNumPerChunkIs0", - maxBlockNum: 10, - maxTxNum: 0, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 0, - }, - { - name: "MaxL1CommitGasPerChunkIs0", - maxBlockNum: 10, - maxTxNum: 10000, - maxL1CommitGas: 0, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 0, - }, - { - name: "MaxL1CommitCalldataSizePerChunkIs0", - maxBlockNum: 10, - maxTxNum: 10000, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 0, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 0, - }, - { - name: "MaxRowConsumptionPerChunkIs0", - maxBlockNum: 100, - maxTxNum: 10000, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 0, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 0, - }, - { - name: "MaxBlockNumPerChunkIs1", - maxBlockNum: 1, - maxTxNum: 10000, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 1, - expectedBlocksInFirstChunk: 1, - }, - { - name: "MaxTxNumPerChunkIsFirstBlock", - maxBlockNum: 10, - maxTxNum: 2, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 1, - expectedBlocksInFirstChunk: 1, - }, - { - name: "MaxL1CommitGasPerChunkIsFirstBlock", - maxBlockNum: 10, - maxTxNum: 10000, - maxL1CommitGas: 2500, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 1, - expectedBlocksInFirstChunk: 1, - }, - { - name: "MaxL1CommitCalldataSizePerChunkIsFirstBlock", - maxBlockNum: 10, - maxTxNum: 10000, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 60, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 1, - expectedBlocksInFirstChunk: 1, - }, - { - name: "MaxRowConsumptionPerChunkIs1", - maxBlockNum: 10, - maxTxNum: 10000, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 1, - expectedBlocksInFirstChunk: 1, - }, - } - - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - db := setupDB(t) - defer database.CloseDB(db) - - l2BlockOrm := orm.NewL2Block(db) - err := l2BlockOrm.InsertL2Blocks(context.Background(), []*encoding.Block{block1, block2}) - assert.NoError(t, err) - - cp := NewChunkProposer(context.Background(), &config.ChunkProposerConfig{ - MaxBlockNumPerChunk: tt.maxBlockNum, - MaxTxNumPerChunk: tt.maxTxNum, - MaxL1CommitGasPerChunk: tt.maxL1CommitGas, - MaxL1CommitCalldataSizePerChunk: tt.maxL1CommitCalldataSize, - MaxRowConsumptionPerChunk: tt.maxRowConsumption, - ChunkTimeoutSec: tt.chunkTimeoutSec, - GasCostIncreaseMultiplier: 1.2, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, ¶ms.ChainConfig{BernoulliBlock: big.NewInt(0)}, db, nil) - cp.TryProposeChunk() - - chunkOrm := orm.NewChunk(db) - chunks, err := chunkOrm.GetChunksGEIndex(context.Background(), 0, 0) - assert.NoError(t, err) - assert.Len(t, chunks, tt.expectedChunksLen) - - if len(chunks) > 0 { - blockOrm := orm.NewL2Block(db) - chunkHashes, err := blockOrm.GetChunkHashes(context.Background(), tt.expectedBlocksInFirstChunk) - assert.NoError(t, err) - assert.Len(t, chunkHashes, tt.expectedBlocksInFirstChunk) - firstChunkHash := chunks[0].Hash - for _, chunkHash := range chunkHashes { - assert.Equal(t, firstChunkHash, chunkHash) - } - } - }) - } -} - -func testChunkProposerCodecv2Limits(t *testing.T) { - tests := []struct { - name string - maxBlockNum uint64 - maxTxNum uint64 - maxL1CommitGas uint64 - maxL1CommitCalldataSize uint64 - maxRowConsumption uint64 - chunkTimeoutSec uint64 - expectedChunksLen int - expectedBlocksInFirstChunk int // only be checked when expectedChunksLen > 0 - }{ - { - name: "NoLimitReached", - maxBlockNum: 100, - maxTxNum: 10000, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 0, - }, - { - name: "Timeout", - maxBlockNum: 100, - maxTxNum: 10000, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1000000, - chunkTimeoutSec: 0, - expectedChunksLen: 1, - expectedBlocksInFirstChunk: 2, - }, - { - name: "MaxTxNumPerChunkIs0", - maxBlockNum: 10, - maxTxNum: 0, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 0, - }, - { - name: "MaxL1CommitGasPerChunkIs0", - maxBlockNum: 10, - maxTxNum: 10000, - maxL1CommitGas: 0, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 0, - }, - { - name: "MaxL1CommitCalldataSizePerChunkIs0", - maxBlockNum: 10, - maxTxNum: 10000, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 0, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 0, - }, - { - name: "MaxRowConsumptionPerChunkIs0", - maxBlockNum: 100, - maxTxNum: 10000, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 0, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 0, - }, - { - name: "MaxBlockNumPerChunkIs1", - maxBlockNum: 1, - maxTxNum: 10000, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 1, - expectedBlocksInFirstChunk: 1, - }, - { - name: "MaxTxNumPerChunkIsFirstBlock", - maxBlockNum: 10, - maxTxNum: 2, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 1, - expectedBlocksInFirstChunk: 1, - }, - { - name: "MaxL1CommitGasPerChunkIsFirstBlock", - maxBlockNum: 10, - maxTxNum: 10000, - maxL1CommitGas: 2500, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 1, - expectedBlocksInFirstChunk: 1, - }, - { - name: "MaxL1CommitCalldataSizePerChunkIsFirstBlock", - maxBlockNum: 10, - maxTxNum: 10000, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 60, - maxRowConsumption: 1000000, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 1, - expectedBlocksInFirstChunk: 1, - }, - { - name: "MaxRowConsumptionPerChunkIs1", - maxBlockNum: 10, - maxTxNum: 10000, - maxL1CommitGas: 50000000000, - maxL1CommitCalldataSize: 1000000, - maxRowConsumption: 1, - chunkTimeoutSec: 1000000000000, - expectedChunksLen: 1, - expectedBlocksInFirstChunk: 1, - }, - } - - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - db := setupDB(t) - defer database.CloseDB(db) - - l2BlockOrm := orm.NewL2Block(db) - err := l2BlockOrm.InsertL2Blocks(context.Background(), []*encoding.Block{block1, block2}) - assert.NoError(t, err) - - cp := NewChunkProposer(context.Background(), &config.ChunkProposerConfig{ - MaxBlockNumPerChunk: tt.maxBlockNum, - MaxTxNumPerChunk: tt.maxTxNum, - MaxL1CommitGasPerChunk: tt.maxL1CommitGas, - MaxL1CommitCalldataSizePerChunk: tt.maxL1CommitCalldataSize, - MaxRowConsumptionPerChunk: tt.maxRowConsumption, - ChunkTimeoutSec: tt.chunkTimeoutSec, - GasCostIncreaseMultiplier: 1.2, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, ¶ms.ChainConfig{BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0)}, db, nil) - cp.TryProposeChunk() - - chunkOrm := orm.NewChunk(db) - chunks, err := chunkOrm.GetChunksGEIndex(context.Background(), 0, 0) - assert.NoError(t, err) - assert.Len(t, chunks, tt.expectedChunksLen) - - if len(chunks) > 0 { - blockOrm := orm.NewL2Block(db) - chunkHashes, err := blockOrm.GetChunkHashes(context.Background(), tt.expectedBlocksInFirstChunk) - assert.NoError(t, err) - assert.Len(t, chunkHashes, tt.expectedBlocksInFirstChunk) - firstChunkHash := chunks[0].Hash - for _, chunkHash := range chunkHashes { - assert.Equal(t, firstChunkHash, chunkHash) - } - } - }) - } -} - -func testChunkProposerCodecv3Limits(t *testing.T) { +func testChunkProposerLimitsCodecV4(t *testing.T) { tests := []struct { name string maxBlockNum uint64 @@ -674,7 +164,7 @@ func testChunkProposerCodecv3Limits(t *testing.T) { ChunkTimeoutSec: tt.chunkTimeoutSec, GasCostIncreaseMultiplier: 1.2, MaxUncompressedBatchBytesSize: math.MaxUint64, - }, ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64)}, db, nil) + }, encoding.CodecV4, ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64), DarwinV2Time: new(uint64)}, db, nil) cp.TryProposeChunk() chunkOrm := orm.NewChunk(db) @@ -696,8 +186,8 @@ func testChunkProposerCodecv3Limits(t *testing.T) { } } -func testChunkProposerBlobSizeLimit(t *testing.T) { - codecVersions := []encoding.CodecVersion{encoding.CodecV0, encoding.CodecV1, encoding.CodecV2, encoding.CodecV3} +func testChunkProposerBlobSizeLimitCodecV4(t *testing.T) { + codecVersions := []encoding.CodecVersion{encoding.CodecV4} for _, codecVersion := range codecVersions { db := setupDB(t) block := readBlockFromJSON(t, "../../../testdata/blockTrace_03.json") @@ -709,14 +199,10 @@ func testChunkProposerBlobSizeLimit(t *testing.T) { } var chainConfig *params.ChainConfig - if codecVersion == encoding.CodecV0 { // will never hit blob size limit - chainConfig = ¶ms.ChainConfig{} - } else if codecVersion == encoding.CodecV1 { - chainConfig = ¶ms.ChainConfig{BernoulliBlock: big.NewInt(0)} - } else if codecVersion == encoding.CodecV2 { - chainConfig = ¶ms.ChainConfig{BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0)} + if codecVersion == encoding.CodecV4 { + chainConfig = ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64), DarwinV2Time: new(uint64)} } else { - chainConfig = ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64)} + assert.Fail(t, "unsupported codec version, expected CodecV4") } cp := NewChunkProposer(context.Background(), &config.ChunkProposerConfig{ @@ -728,7 +214,7 @@ func testChunkProposerBlobSizeLimit(t *testing.T) { ChunkTimeoutSec: math.MaxUint32, GasCostIncreaseMultiplier: 1, MaxUncompressedBatchBytesSize: math.MaxUint64, - }, chainConfig, db, nil) + }, encoding.CodecV4, chainConfig, db, nil) for i := 0; i < 2; i++ { cp.TryProposeChunk() @@ -740,14 +226,10 @@ func testChunkProposerBlobSizeLimit(t *testing.T) { var expectedNumChunks int = 2 var numBlocksMultiplier uint64 - if codecVersion == encoding.CodecV0 { - numBlocksMultiplier = 255 - } else if codecVersion == encoding.CodecV1 { - numBlocksMultiplier = 22 - } else if codecVersion == encoding.CodecV2 { + if codecVersion == encoding.CodecV4 { numBlocksMultiplier = 255 } else { - numBlocksMultiplier = 255 + assert.Fail(t, "unsupported codec version, expected CodecV4") } assert.Len(t, chunks, expectedNumChunks) @@ -761,47 +243,3 @@ func testChunkProposerBlobSizeLimit(t *testing.T) { database.CloseDB(db) } } - -func testChunkProposerRespectHardforks(t *testing.T) { - db := setupDB(t) - defer database.CloseDB(db) - - block := readBlockFromJSON(t, "../../../testdata/blockTrace_02.json") - for i := int64(1); i <= 20; i++ { - l2BlockOrm := orm.NewL2Block(db) - block.Header.Number = big.NewInt(i) - block.Header.Time = uint64(i) - err := l2BlockOrm.InsertL2Blocks(context.Background(), []*encoding.Block{block}) - assert.NoError(t, err) - } - - cp := NewChunkProposer(context.Background(), &config.ChunkProposerConfig{ - MaxBlockNumPerChunk: math.MaxUint64, - MaxTxNumPerChunk: math.MaxUint64, - MaxL1CommitGasPerChunk: math.MaxUint64, - MaxL1CommitCalldataSizePerChunk: math.MaxUint64, - MaxRowConsumptionPerChunk: math.MaxUint64, - ChunkTimeoutSec: 0, - GasCostIncreaseMultiplier: 1, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, ¶ms.ChainConfig{ - LondonBlock: big.NewInt(0), - BernoulliBlock: big.NewInt(1), - CurieBlock: big.NewInt(2), - DarwinTime: func() *uint64 { t := uint64(4); return &t }(), - }, db, nil) - - for i := 0; i < 5; i++ { - cp.TryProposeChunk() - } - - chunkOrm := orm.NewChunk(db) - chunks, err := chunkOrm.GetChunksGEIndex(context.Background(), 0, 0) - assert.NoError(t, err) - - assert.Len(t, chunks, 4) - expectedEndBlockNumbers := []uint64{1, 2, 3, 20} - for i, chunk := range chunks { - assert.Equal(t, expectedEndBlockNumbers[i], chunk.EndBlockNumber) - } -} diff --git a/rollup/internal/controller/watcher/watcher_test.go b/rollup/internal/controller/watcher/watcher_test.go index 03498b3c30..465b873e33 100644 --- a/rollup/internal/controller/watcher/watcher_test.go +++ b/rollup/internal/controller/watcher/watcher_test.go @@ -100,29 +100,17 @@ func TestFunction(t *testing.T) { t.Run("TestFetchRunningMissingBlocks", testFetchRunningMissingBlocks) // Run chunk proposer test cases. - t.Run("TestChunkProposerCodecv0Limits", testChunkProposerCodecv0Limits) - t.Run("TestChunkProposerCodecv1Limits", testChunkProposerCodecv1Limits) - t.Run("TestChunkProposerCodecv2Limits", testChunkProposerCodecv2Limits) - t.Run("TestChunkProposerCodecv3Limits", testChunkProposerCodecv3Limits) - t.Run("TestChunkProposerBlobSizeLimit", testChunkProposerBlobSizeLimit) - t.Run("TestChunkProposerRespectHardforks", testChunkProposerRespectHardforks) + t.Run("TestChunkProposerLimitsCodecV4", testChunkProposerLimitsCodecV4) + t.Run("TestChunkProposerBlobSizeLimitCodecV4", testChunkProposerBlobSizeLimitCodecV4) // Run batch proposer test cases. - t.Run("TestBatchProposerCodecv0Limits", testBatchProposerCodecv0Limits) - t.Run("TestBatchProposerCodecv1Limits", testBatchProposerCodecv1Limits) - t.Run("TestBatchProposerCodecv2Limits", testBatchProposerCodecv2Limits) - t.Run("TestBatchProposerCodecv3Limits", testBatchProposerCodecv3Limits) - t.Run("TestBatchCommitGasAndCalldataSizeCodecv0Estimation", testBatchCommitGasAndCalldataSizeCodecv0Estimation) - t.Run("TestBatchCommitGasAndCalldataSizeCodecv1Estimation", testBatchCommitGasAndCalldataSizeCodecv1Estimation) - t.Run("TestBatchCommitGasAndCalldataSizeCodecv2Estimation", testBatchCommitGasAndCalldataSizeCodecv2Estimation) - t.Run("TestBatchCommitGasAndCalldataSizeCodecv3Estimation", testBatchCommitGasAndCalldataSizeCodecv3Estimation) - t.Run("TestBatchProposerBlobSizeLimit", testBatchProposerBlobSizeLimit) - t.Run("TestBatchProposerMaxChunkNumPerBatchLimit", testBatchProposerMaxChunkNumPerBatchLimit) - t.Run("TestBatchProposerRespectHardforks", testBatchProposerRespectHardforks) + t.Run("TestBatchProposerLimitsCodecV4", testBatchProposerLimitsCodecV4) + t.Run("TestBatchCommitGasAndCalldataSizeEstimationCodecV4", testBatchCommitGasAndCalldataSizeEstimationCodecV4) + t.Run("TestBatchProposerBlobSizeLimitCodecV4", testBatchProposerBlobSizeLimitCodecV4) + t.Run("TestBatchProposerMaxChunkNumPerBatchLimitCodecV4", testBatchProposerMaxChunkNumPerBatchLimitCodecV4) // Run bundle proposer test cases. - t.Run("TestBundleProposerLimits", testBundleProposerLimits) - t.Run("TestBundleProposerRespectHardforks", testBundleProposerRespectHardforks) + t.Run("TestBundleProposerLimitsCodecV4", testBundleProposerLimitsCodecV4) } func readBlockFromJSON(t *testing.T, filename string) *encoding.Block { diff --git a/rollup/internal/orm/orm_test.go b/rollup/internal/orm/orm_test.go index 390c860925..f16274370f 100644 --- a/rollup/internal/orm/orm_test.go +++ b/rollup/internal/orm/orm_test.go @@ -166,7 +166,7 @@ func TestL2BlockOrm(t *testing.T) { } func TestChunkOrm(t *testing.T) { - codecVersions := []encoding.CodecVersion{encoding.CodecV0, encoding.CodecV1, encoding.CodecV2, encoding.CodecV3} + codecVersions := []encoding.CodecVersion{encoding.CodecV0, encoding.CodecV1, encoding.CodecV2, encoding.CodecV3, encoding.CodecV4} chunk1 := &encoding.Chunk{Blocks: []*encoding.Block{block1}} chunk2 := &encoding.Chunk{Blocks: []*encoding.Block{block2}} for _, codecVersion := range codecVersions { @@ -229,7 +229,7 @@ func TestChunkOrm(t *testing.T) { } func TestBatchOrm(t *testing.T) { - codecVersions := []encoding.CodecVersion{encoding.CodecV0, encoding.CodecV1, encoding.CodecV2, encoding.CodecV3} + codecVersions := []encoding.CodecVersion{encoding.CodecV0, encoding.CodecV1, encoding.CodecV2, encoding.CodecV3, encoding.CodecV4} chunk1 := &encoding.Chunk{Blocks: []*encoding.Block{block1}} chunk2 := &encoding.Chunk{Blocks: []*encoding.Block{block2}} for _, codecVersion := range codecVersions { @@ -378,7 +378,7 @@ func TestBundleOrm(t *testing.T) { Index: 0, Chunks: []*encoding.Chunk{chunk1}, } - dbBatch1, err := batchOrm.InsertBatch(context.Background(), batch1, encoding.CodecV3, utils.BatchMetrics{}) + dbBatch1, err := batchOrm.InsertBatch(context.Background(), batch1, encoding.CodecV4, utils.BatchMetrics{}) assert.NoError(t, err) chunk2 := &encoding.Chunk{Blocks: []*encoding.Block{block2}} @@ -386,30 +386,30 @@ func TestBundleOrm(t *testing.T) { Index: 1, Chunks: []*encoding.Chunk{chunk2}, } - dbBatch2, err := batchOrm.InsertBatch(context.Background(), batch2, encoding.CodecV3, utils.BatchMetrics{}) + dbBatch2, err := batchOrm.InsertBatch(context.Background(), batch2, encoding.CodecV4, utils.BatchMetrics{}) assert.NoError(t, err) var bundle1 *Bundle var bundle2 *Bundle t.Run("InsertBundle", func(t *testing.T) { - bundle1, err = bundleOrm.InsertBundle(context.Background(), []*Batch{dbBatch1}, encoding.CodecV3) + bundle1, err = bundleOrm.InsertBundle(context.Background(), []*Batch{dbBatch1}, encoding.CodecV4) assert.NoError(t, err) assert.NotNil(t, bundle1) assert.Equal(t, uint64(0), bundle1.StartBatchIndex) assert.Equal(t, uint64(0), bundle1.EndBatchIndex) assert.Equal(t, dbBatch1.Hash, bundle1.StartBatchHash) assert.Equal(t, dbBatch1.Hash, bundle1.EndBatchHash) - assert.Equal(t, encoding.CodecV3, encoding.CodecVersion(bundle1.CodecVersion)) + assert.Equal(t, encoding.CodecV4, encoding.CodecVersion(bundle1.CodecVersion)) - bundle2, err = bundleOrm.InsertBundle(context.Background(), []*Batch{dbBatch2}, encoding.CodecV3) + bundle2, err = bundleOrm.InsertBundle(context.Background(), []*Batch{dbBatch2}, encoding.CodecV4) assert.NoError(t, err) assert.NotNil(t, bundle2) assert.Equal(t, uint64(1), bundle2.StartBatchIndex) assert.Equal(t, uint64(1), bundle2.EndBatchIndex) assert.Equal(t, dbBatch2.Hash, bundle2.StartBatchHash) assert.Equal(t, dbBatch2.Hash, bundle2.EndBatchHash) - assert.Equal(t, encoding.CodecV3, encoding.CodecVersion(bundle2.CodecVersion)) + assert.Equal(t, encoding.CodecV4, encoding.CodecVersion(bundle2.CodecVersion)) }) t.Run("GetFirstUnbundledBatchIndex", func(t *testing.T) { diff --git a/rollup/tests/bridge_test.go b/rollup/tests/bridge_test.go index afc137f8c4..41cba06469 100644 --- a/rollup/tests/bridge_test.go +++ b/rollup/tests/bridge_test.go @@ -208,8 +208,7 @@ func TestFunction(t *testing.T) { // l1 rollup and watch rollup events t.Run("TestCommitAndFinalizeGenesisBatch", testCommitAndFinalizeGenesisBatch) - t.Run("testCommitBatchAndFinalizeBatchOrBundleWithAllCodecVersions", testCommitBatchAndFinalizeBatchOrBundleWithAllCodecVersions) - t.Run("TestCommitBatchAndFinalizeBatchOrBundleCrossingAllTransitions", testCommitBatchAndFinalizeBatchOrBundleCrossingAllTransitions) + t.Run("TestCommitBatchAndFinalizeBundleCodecV4", testCommitBatchAndFinalizeBundleCodecV4) // l1/l2 gas oracle t.Run("TestImportL1GasPrice", testImportL1GasPrice) diff --git a/rollup/tests/process_start_test.go b/rollup/tests/process_start_test.go index 0b77656824..08b9659990 100644 --- a/rollup/tests/process_start_test.go +++ b/rollup/tests/process_start_test.go @@ -19,7 +19,7 @@ func testProcessStart(t *testing.T) { db := setupDB(t) defer database.CloseDB(db) rollupApp.RunApp(t, cutils.GasOracleApp, "--genesis", "../conf/genesis.json") - rollupApp.RunApp(t, cutils.RollupRelayerApp, "--genesis", "../conf/genesis.json") + rollupApp.RunApp(t, cutils.RollupRelayerApp, "--genesis", "../conf/genesis.json", "--min-codec-version", "4") rollupApp.WaitExit() } @@ -36,7 +36,7 @@ func testProcessStartEnableMetrics(t *testing.T) { port, err = rand.Int(rand.Reader, big.NewInt(10000)) assert.NoError(t, err) svrPort = strconv.FormatInt(port.Int64()+30000, 10) - rollupApp.RunApp(t, cutils.RollupRelayerApp, "--metrics", "--metrics.addr", "localhost", "--metrics.port", svrPort, "--genesis", "../conf/genesis.json") + rollupApp.RunApp(t, cutils.RollupRelayerApp, "--metrics", "--metrics.addr", "localhost", "--metrics.port", svrPort, "--genesis", "../conf/genesis.json", "--min-codec-version", "4") rollupApp.WaitExit() } diff --git a/rollup/tests/rollup_test.go b/rollup/tests/rollup_test.go index ea99b508e3..4251aa273b 100644 --- a/rollup/tests/rollup_test.go +++ b/rollup/tests/rollup_test.go @@ -52,24 +52,18 @@ func testCommitAndFinalizeGenesisBatch(t *testing.T) { assert.Equal(t, types.RollupFinalized, types.RollupStatus(batch.RollupStatus)) } -func testCommitBatchAndFinalizeBatchOrBundleWithAllCodecVersions(t *testing.T) { - codecVersions := []encoding.CodecVersion{encoding.CodecV0, encoding.CodecV1, encoding.CodecV2, encoding.CodecV3, encoding.CodecV4} +func testCommitBatchAndFinalizeBundleCodecV4(t *testing.T) { + codecVersions := []encoding.CodecVersion{encoding.CodecV4} for _, codecVersion := range codecVersions { db := setupDB(t) prepareContracts(t) var chainConfig *params.ChainConfig - if codecVersion == encoding.CodecV0 { - chainConfig = ¶ms.ChainConfig{} - } else if codecVersion == encoding.CodecV1 { - chainConfig = ¶ms.ChainConfig{BernoulliBlock: big.NewInt(0)} - } else if codecVersion == encoding.CodecV2 { - chainConfig = ¶ms.ChainConfig{BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0)} - } else if codecVersion == encoding.CodecV3 { - chainConfig = ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64)} + if codecVersion == encoding.CodecV4 { + chainConfig = ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64), DarwinV2Time: new(uint64)} } else { - chainConfig = ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(0), CurieBlock: big.NewInt(0), DarwinTime: new(uint64)} + assert.Fail(t, "unsupported codec version, expected CodecV4") } // Create L2Relayer @@ -103,19 +97,19 @@ func testCommitBatchAndFinalizeBatchOrBundleWithAllCodecVersions(t *testing.T) { MaxRowConsumptionPerChunk: 1048319, ChunkTimeoutSec: 300, MaxUncompressedBatchBytesSize: math.MaxUint64, - }, chainConfig, db, nil) + }, encoding.CodecV4, chainConfig, db, nil) bap := watcher.NewBatchProposer(context.Background(), &config.BatchProposerConfig{ MaxL1CommitGasPerBatch: 50000000000, MaxL1CommitCalldataSizePerBatch: 1000000, BatchTimeoutSec: 300, MaxUncompressedBatchBytesSize: math.MaxUint64, - }, chainConfig, db, nil) + }, encoding.CodecV4, chainConfig, db, nil) bup := watcher.NewBundleProposer(context.Background(), &config.BundleProposerConfig{ MaxBatchNumPerBundle: 1000000, BundleTimeoutSec: 300, - }, chainConfig, db, nil) + }, encoding.CodecV4, chainConfig, db, nil) l2BlockOrm := orm.NewL2Block(db) err = l2BlockOrm.InsertL2Blocks(context.Background(), blocks[:5]) @@ -178,7 +172,6 @@ func testCommitBatchAndFinalizeBatchOrBundleWithAllCodecVersions(t *testing.T) { } assert.Eventually(t, func() bool { - l2Relayer.ProcessCommittedBatches() l2Relayer.ProcessPendingBundles() batches, err := batchOrm.GetBatches(context.Background(), map[string]interface{}{}, nil, 0) @@ -198,26 +191,24 @@ func testCommitBatchAndFinalizeBatchOrBundleWithAllCodecVersions(t *testing.T) { bundles, err := bundleOrm.GetBundles(context.Background(), map[string]interface{}{}, nil, 0) assert.NoError(t, err) - if codecVersion == encoding.CodecV0 || codecVersion == encoding.CodecV1 || codecVersion == encoding.CodecV2 { - assert.Len(t, bundles, 0) - } else { - assert.Len(t, bundles, 1) - bundle := bundles[0] - if types.RollupStatus(bundle.RollupStatus) != types.RollupFinalized { - return false - } - assert.NotEmpty(t, bundle.FinalizeTxHash) - receipt, err := l1Client.TransactionReceipt(context.Background(), common.HexToHash(bundle.FinalizeTxHash)) - assert.NoError(t, err) - assert.Equal(t, gethTypes.ReceiptStatusSuccessful, receipt.Status) - batches, err := batchOrm.GetBatches(context.Background(), map[string]interface{}{"bundle_hash": bundle.Hash}, nil, 0) - assert.NoError(t, err) - assert.Len(t, batches, 2) - for _, batch := range batches { - assert.Equal(t, batch.RollupStatus, bundle.RollupStatus) - assert.Equal(t, bundle.FinalizeTxHash, batch.FinalizeTxHash) - } + assert.Len(t, bundles, 1) + + bundle := bundles[0] + if types.RollupStatus(bundle.RollupStatus) != types.RollupFinalized { + return false } + assert.NotEmpty(t, bundle.FinalizeTxHash) + receipt, err := l1Client.TransactionReceipt(context.Background(), common.HexToHash(bundle.FinalizeTxHash)) + assert.NoError(t, err) + assert.Equal(t, gethTypes.ReceiptStatusSuccessful, receipt.Status) + batches, err = batchOrm.GetBatches(context.Background(), map[string]interface{}{"bundle_hash": bundle.Hash}, nil, 0) + assert.NoError(t, err) + assert.Len(t, batches, 2) + for _, batch := range batches { + assert.Equal(t, batch.RollupStatus, bundle.RollupStatus) + assert.Equal(t, bundle.FinalizeTxHash, batch.FinalizeTxHash) + } + return true }, 30*time.Second, time.Second) @@ -225,177 +216,3 @@ func testCommitBatchAndFinalizeBatchOrBundleWithAllCodecVersions(t *testing.T) { database.CloseDB(db) } } - -func testCommitBatchAndFinalizeBatchOrBundleCrossingAllTransitions(t *testing.T) { - db := setupDB(t) - defer database.CloseDB(db) - - prepareContracts(t) - - // Create L2Relayer - l2Cfg := rollupApp.Config.L2Config - chainConfig := ¶ms.ChainConfig{LondonBlock: big.NewInt(0), BernoulliBlock: big.NewInt(1), CurieBlock: big.NewInt(2), DarwinTime: func() *uint64 { t := uint64(4); return &t }()} - l2Relayer, err := relayer.NewLayer2Relayer(context.Background(), l2Client, db, l2Cfg.RelayerConfig, chainConfig, true, relayer.ServiceTypeL2RollupRelayer, nil) - assert.NoError(t, err) - defer l2Relayer.StopSenders() - - // add some blocks to db - var blocks []*encoding.Block - for i := int64(0); i < 10; i++ { - header := gethTypes.Header{ - Number: big.NewInt(i + 1), - ParentHash: common.Hash{}, - Difficulty: big.NewInt(0), - BaseFee: big.NewInt(0), - Root: common.HexToHash("0x1"), - Time: uint64(i + 1), - } - blocks = append(blocks, &encoding.Block{ - Header: &header, - Transactions: nil, - WithdrawRoot: common.HexToHash("0x2"), - RowConsumption: &gethTypes.RowConsumption{}, - }) - } - - l2BlockOrm := orm.NewL2Block(db) - err = l2BlockOrm.InsertL2Blocks(context.Background(), blocks) - assert.NoError(t, err) - - cp := watcher.NewChunkProposer(context.Background(), &config.ChunkProposerConfig{ - MaxBlockNumPerChunk: 100, - MaxTxNumPerChunk: 10000, - MaxL1CommitGasPerChunk: 50000000000, - MaxL1CommitCalldataSizePerChunk: 1000000, - MaxRowConsumptionPerChunk: 1048319, - ChunkTimeoutSec: 300, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, chainConfig, db, nil) - - bap := watcher.NewBatchProposer(context.Background(), &config.BatchProposerConfig{ - MaxL1CommitGasPerBatch: 50000000000, - MaxL1CommitCalldataSizePerBatch: 1000000, - BatchTimeoutSec: 300, - MaxUncompressedBatchBytesSize: math.MaxUint64, - }, chainConfig, db, nil) - - bup := watcher.NewBundleProposer(context.Background(), &config.BundleProposerConfig{ - MaxBatchNumPerBundle: 1000000, - BundleTimeoutSec: 300, - }, chainConfig, db, nil) - - cp.TryProposeChunk() - cp.TryProposeChunk() - cp.TryProposeChunk() - cp.TryProposeChunk() - cp.TryProposeChunk() - - bap.TryProposeBatch() - bap.TryProposeBatch() - bap.TryProposeBatch() - bap.TryProposeBatch() - - bup.TryProposeBundle() - - l2Relayer.ProcessPendingBatches() - - batchOrm := orm.NewBatch(db) - bundleOrm := orm.NewBundle(db) - - assert.Eventually(t, func() bool { - batches, getErr := batchOrm.GetBatches(context.Background(), map[string]interface{}{}, nil, 0) - assert.NoError(t, getErr) - assert.Len(t, batches, 4) - batches = batches[1:] - for _, batch := range batches { - if types.RollupCommitted != types.RollupStatus(batch.RollupStatus) { - return false - } - } - return true - }, 30*time.Second, time.Second) - - batchProof := &message.BatchProof{ - Proof: []byte{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31}, - Instances: []byte{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31}, - Vk: []byte{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31}, - } - batches, err := batchOrm.GetBatches(context.Background(), map[string]interface{}{}, nil, 0) - assert.NoError(t, err) - batches = batches[1:] - for _, batch := range batches { - err = batchOrm.UpdateProofByHash(context.Background(), batch.Hash, batchProof, 600) - assert.NoError(t, err) - err = batchOrm.UpdateProvingStatus(context.Background(), batch.Hash, types.ProvingTaskVerified) - assert.NoError(t, err) - } - - bundleProof := &message.BundleProof{ - Proof: []byte{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31}, - Instances: []byte{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31}, - Vk: []byte{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31}, - } - bundles, err := bundleOrm.GetBundles(context.Background(), map[string]interface{}{}, nil, 0) - assert.NoError(t, err) - for _, bundle := range bundles { - err = bundleOrm.UpdateProofAndProvingStatusByHash(context.Background(), bundle.Hash, bundleProof, types.ProvingTaskVerified, 100) - assert.NoError(t, err) - } - - assert.Eventually(t, func() bool { - l2Relayer.ProcessCommittedBatches() - - batches, err := batchOrm.GetBatches(context.Background(), map[string]interface{}{}, nil, 0) - assert.NoError(t, err) - assert.Len(t, batches, 4) - batches = batches[1:2] - for _, batch := range batches { - if types.RollupStatus(batch.RollupStatus) != types.RollupFinalized { - return false - } - assert.NotEmpty(t, batch.FinalizeTxHash) - receipt, getErr := l1Client.TransactionReceipt(context.Background(), common.HexToHash(batch.FinalizeTxHash)) - assert.NoError(t, getErr) - assert.Equal(t, gethTypes.ReceiptStatusSuccessful, receipt.Status) - } - return true - }, 30*time.Second, time.Second) - - assert.Eventually(t, func() bool { - l2Relayer.ProcessPendingBundles() - - batches, err := batchOrm.GetBatches(context.Background(), map[string]interface{}{}, nil, 0) - assert.NoError(t, err) - assert.Len(t, batches, 4) - batches = batches[3:] - for _, batch := range batches { - if types.RollupStatus(batch.RollupStatus) != types.RollupFinalized { - return false - } - assert.NotEmpty(t, batch.FinalizeTxHash) - receipt, getErr := l1Client.TransactionReceipt(context.Background(), common.HexToHash(batch.FinalizeTxHash)) - assert.NoError(t, getErr) - assert.Equal(t, gethTypes.ReceiptStatusSuccessful, receipt.Status) - } - - bundles, err := bundleOrm.GetBundles(context.Background(), map[string]interface{}{}, nil, 0) - assert.NoError(t, err) - assert.Len(t, bundles, 1) - bundle := bundles[0] - if types.RollupStatus(bundle.RollupStatus) != types.RollupFinalized { - return false - } - assert.NotEmpty(t, bundle.FinalizeTxHash) - receipt, err := l1Client.TransactionReceipt(context.Background(), common.HexToHash(bundle.FinalizeTxHash)) - assert.NoError(t, err) - assert.Equal(t, gethTypes.ReceiptStatusSuccessful, receipt.Status) - batches, err = batchOrm.GetBatches(context.Background(), map[string]interface{}{"bundle_hash": bundle.Hash}, nil, 0) - assert.NoError(t, err) - assert.Len(t, batches, 1) - for _, batch := range batches { - assert.Equal(t, batch.RollupStatus, bundle.RollupStatus) - assert.Equal(t, bundle.FinalizeTxHash, batch.FinalizeTxHash) - } - return true - }, 30*time.Second, time.Second) -}