feat(rollup-relayer): add a tool to analyze chunk/batch/bundle proposing (#1645)

Co-authored-by: colinlyguo <colinlyguo@users.noreply.github.com>
This commit is contained in:
colin
2025-04-23 13:41:03 +08:00
committed by GitHub
parent e80f030246
commit 3ac69bec51
15 changed files with 574 additions and 5 deletions

View File

@@ -22,6 +22,10 @@ var (
RollupRelayerFlags = []cli.Flag{
&MinCodecVersionFlag,
}
// ProposerToolFlags contains flags only used in proposer tool
ProposerToolFlags = []cli.Flag{
&StartL2BlockFlag,
}
// ConfigFileFlag load json type config file.
ConfigFileFlag = cli.StringFlag{
Name: "config",
@@ -90,4 +94,10 @@ var (
Usage: "Minimum required codec version for the chunk/batch/bundle proposers",
Required: true,
}
// StartL2BlockFlag indicates the start L2 block number for proposer tool
StartL2BlockFlag = cli.Uint64Flag{
Name: "start-l2-block",
Usage: "Start L2 block number for proposer tool",
Value: 0,
}
)

View File

@@ -5,7 +5,7 @@ import (
"runtime/debug"
)
var tag = "v4.5.5"
var tag = "v4.5.6"
var commit = func() string {
if info, ok := debug.ReadBuildInfo(); ok {

View File

@@ -33,3 +33,45 @@ make rollup_bins
./build/bin/gas_oracle --config ./conf/config.json
./build/bin/rollup_relayer --config ./conf/config.json
```
## Proposer Tool
The Proposer Tool replays historical blocks with custom configurations (e.g., future hardfork configs, custom chunk/batch/bundle proposer configs) to generate chunks/batches/bundles, helping test parameter changes before protocol upgrade.
You can:
1. Enable different hardforks in the genesis configuration.
2. Set custom chunk-proposer, batch-proposer, and bundle-proposer parameters.
3. Analyze resulting metrics (blob size, block count, transaction count, gas usage).
## How to run the proposer tool?
### Set the configs
1. Set genesis config to enable desired hardforks in [`proposer-tool-genesis.json`](./proposer-tool-genesis.json).
2. Set proposer config in [`proposer-tool-config.json`](./proposer-tool-config.json) for data analysis.
3. Set `start-l2-block` in the launch command of proposer-tool in [`docker-compose-proposer-tool.yml`](./docker-compose-proposer-tool.yml) to the block number you want to start from. The default is `0`, which means starting from the genesis block.
### Start the proposer tool using docker-compose
Prerequisite: an RPC URL to an archive L2 node. The default url in [`proposer-tool-config.json`](./proposer-tool-config.json) is `https://rpc.scroll.io`.
```
cd rollup
DOCKER_BUILDKIT=1 docker-compose -f docker-compose-proposer-tool.yml up -d
```
> Note: The port 5432 of database is mapped to the host machine. You can use `psql` or any db clients to connect to the database.
> The DSN for the database is `postgres://postgres:postgres@db:5432/scroll?sslmode=disable`.
### Reset env
```
docker-compose -f docker-compose-proposer-tool.yml down -v
```
If you need to rebuild the images, removing the old images is necessary. You can do this by running the following command:
```
docker images | grep rollup | awk '{print $3}' | xargs docker rmi -f
```

View File

@@ -0,0 +1,93 @@
package app
import (
"context"
"fmt"
"os"
"os/signal"
"github.com/scroll-tech/da-codec/encoding"
"github.com/scroll-tech/go-ethereum/log"
"github.com/urfave/cli/v2"
"scroll-tech/common/utils"
"scroll-tech/common/version"
"scroll-tech/rollup/internal/config"
"scroll-tech/rollup/internal/controller/watcher"
)
var app *cli.App
func init() {
// Set up proposer-tool app info.
app = cli.NewApp()
app.Action = action
app.Name = "proposer-tool"
app.Usage = "The Scroll Proposer Tool"
app.Version = version.Version
app.Flags = append(app.Flags, utils.CommonFlags...)
app.Flags = append(app.Flags, utils.RollupRelayerFlags...)
app.Flags = append(app.Flags, utils.ProposerToolFlags...)
app.Commands = []*cli.Command{}
app.Before = func(ctx *cli.Context) error {
return utils.LogSetup(ctx)
}
}
func action(ctx *cli.Context) error {
// Load config file.
cfgFile := ctx.String(utils.ConfigFileFlag.Name)
cfg, err := config.NewConfigForReplay(cfgFile)
if err != nil {
log.Crit("failed to load config file", "config file", cfgFile, "error", err)
}
subCtx, cancel := context.WithCancel(ctx.Context)
startL2BlockHeight := ctx.Uint64(utils.StartL2BlockFlag.Name)
genesisPath := ctx.String(utils.Genesis.Name)
genesis, err := utils.ReadGenesis(genesisPath)
if err != nil {
log.Crit("failed to read genesis", "genesis file", genesisPath, "error", err)
}
minCodecVersion := encoding.CodecVersion(ctx.Uint(utils.MinCodecVersionFlag.Name))
// sanity check config
if cfg.L2Config.BatchProposerConfig.MaxChunksPerBatch <= 0 {
log.Crit("cfg.L2Config.BatchProposerConfig.MaxChunksPerBatch must be greater than 0")
}
if cfg.L2Config.ChunkProposerConfig.MaxL2GasPerChunk <= 0 {
log.Crit("cfg.L2Config.ChunkProposerConfig.MaxL2GasPerChunk must be greater than 0")
}
proposerTool, err := watcher.NewProposerTool(subCtx, cancel, cfg, startL2BlockHeight, minCodecVersion, genesis.Config)
if err != nil {
log.Crit("failed to create proposer tool", "startL2BlockHeight", startL2BlockHeight, "minCodecVersion", minCodecVersion, "error", err)
}
proposerTool.Start()
log.Info("Start proposer-tool successfully", "version", version.Version)
// Catch CTRL-C to ensure a graceful shutdown.
interrupt := make(chan os.Signal, 1)
signal.Notify(interrupt, os.Interrupt)
// Wait until the interrupt signal is received from an OS signal.
<-interrupt
cancel()
proposerTool.Stop()
return nil
}
// Run proposer tool cmd instance.
func Run() {
if err := app.Run(os.Args); err != nil {
_, _ = fmt.Fprintln(os.Stderr, err)
os.Exit(1)
}
}

View File

@@ -0,0 +1,7 @@
package main
import "scroll-tech/rollup/cmd/proposer_tool/app"
func main() {
app.Run()
}

View File

@@ -0,0 +1,40 @@
version: '3'
services:
db:
image: postgres:14
environment:
- POSTGRES_USER=postgres
- POSTGRES_PASSWORD=postgres
- POSTGRES_DB=scroll
ports:
- "5432:5432"
volumes:
- postgres_data:/var/lib/postgresql/data
healthcheck:
test: ["CMD-SHELL", "pg_isready -U postgres"]
interval: 5s
timeout: 5s
retries: 5
proposer-tool:
build:
context: ..
dockerfile: ./rollup/proposer_tool.Dockerfile
depends_on:
db:
condition: service_healthy
command: [
"--config", "/app/conf/proposer-tool-config.json",
"--genesis", "/app/conf/proposer-tool-genesis.json",
"--min-codec-version", "4",
"--start-l2-block", "10000",
"--log.debug", "--verbosity", "3"
]
volumes:
- ./proposer-tool-config.json:/app/conf/proposer-tool-config.json
- ./proposer-tool-genesis.json:/app/conf/proposer-tool-genesis.json
restart: unless-stopped
volumes:
postgres_data:

View File

@@ -1,7 +1,10 @@
package config
import (
"encoding/json"
"fmt"
"os"
"path/filepath"
"reflect"
"strings"
@@ -20,6 +23,11 @@ type Config struct {
DBConfig *database.Config `json:"db_config"`
}
type ConfigForReplay struct {
Config
DBConfigForReplay *database.Config `json:"db_config_for_replay"`
}
// NewConfig returns a new instance of Config.
func NewConfig(file string) (*Config, error) {
v := viper.New()
@@ -87,3 +95,19 @@ func NewConfig(file string) (*Config, error) {
return cfg, nil
}
// NewConfigForReplay returns a new instance of ConfigForReplay.
func NewConfigForReplay(file string) (*ConfigForReplay, error) {
buf, err := os.ReadFile(filepath.Clean(file))
if err != nil {
return nil, err
}
cfg := &ConfigForReplay{}
err = json.Unmarshal(buf, cfg)
if err != nil {
return nil, err
}
return cfg, nil
}

View File

@@ -13,6 +13,8 @@ import (
"github.com/scroll-tech/go-ethereum/params"
"gorm.io/gorm"
"scroll-tech/common/types"
"scroll-tech/rollup/internal/config"
"scroll-tech/rollup/internal/orm"
"scroll-tech/rollup/internal/utils"
@@ -34,6 +36,7 @@ type BatchProposer struct {
maxUncompressedBatchBytesSize uint64
maxChunksPerBatch int
replayMode bool
minCodecVersion encoding.CodecVersion
chainCfg *params.ChainConfig
@@ -80,6 +83,7 @@ func NewBatchProposer(ctx context.Context, cfg *config.BatchProposerConfig, minC
gasCostIncreaseMultiplier: cfg.GasCostIncreaseMultiplier,
maxUncompressedBatchBytesSize: cfg.MaxUncompressedBatchBytesSize,
maxChunksPerBatch: cfg.MaxChunksPerBatch,
replayMode: false,
minCodecVersion: minCodecVersion,
chainCfg: chainCfg,
@@ -152,6 +156,14 @@ func NewBatchProposer(ctx context.Context, cfg *config.BatchProposerConfig, minC
return p
}
// SetReplayDB sets the replay database for the BatchProposer.
// This is used for the proposer tool only, to change the l2_block data source.
// This function is not thread-safe and should be called after initializing the BatchProposer and before starting to propose chunks.
func (p *BatchProposer) SetReplayDB(replayDB *gorm.DB) {
p.l2BlockOrm = orm.NewL2Block(replayDB)
p.replayMode = true
}
// TryProposeBatch tries to propose a new batches.
func (p *BatchProposer) TryProposeBatch() {
p.batchProposerCircleTotal.Inc()
@@ -226,6 +238,15 @@ func (p *BatchProposer) updateDBBatchInfo(batch *encoding.Batch, codecVersion en
log.Warn("BatchProposer.UpdateBatchHashInRange update the chunk's batch hash failure", "hash", dbBatch.Hash, "error", dbErr)
return dbErr
}
if p.replayMode {
// If replayMode is true, meaning the batch was proposed by the proposer tool,
// set batch status to types.RollupCommitted and assign a unique commit tx hash to enable new bundle proposals.
if dbErr = p.batchOrm.UpdateCommitTxHashAndRollupStatus(p.ctx, dbBatch.Hash, dbBatch.Hash, types.RollupCommitted, dbTX); dbErr != nil {
log.Warn("BatchProposer.UpdateCommitTxHashAndRollupStatus update the batch's commit tx hash failure", "hash", dbBatch.Hash, "error", dbErr)
return dbErr
}
}
return nil
})
if err != nil {

View File

@@ -199,7 +199,7 @@ func (p *BundleProposer) proposeBundle() error {
currentTimeSec := uint64(time.Now().Unix())
if firstChunk.StartBlockTime+p.bundleTimeoutSec < currentTimeSec {
log.Info("first block timeout", "batch count", len(batches), "start block number", firstChunk.StartBlockNumber, "start block timestamp", firstChunk.StartBlockTime, "current time", currentTimeSec)
log.Info("first block timeout", "batch count", len(batches), "start block number", firstChunk.StartBlockNumber, "start block timestamp", firstChunk.StartBlockTime, "bundle timeout", p.bundleTimeoutSec, "current time", currentTimeSec)
batches, err = p.allBatchesCommittedInSameTXIncluded(batches)
if err != nil {

View File

@@ -36,6 +36,7 @@ type ChunkProposer struct {
gasCostIncreaseMultiplier float64
maxUncompressedBatchBytesSize uint64
replayMode bool
minCodecVersion encoding.CodecVersion
chainCfg *params.ChainConfig
@@ -91,6 +92,7 @@ func NewChunkProposer(ctx context.Context, cfg *config.ChunkProposerConfig, minC
chunkTimeoutSec: cfg.ChunkTimeoutSec,
gasCostIncreaseMultiplier: cfg.GasCostIncreaseMultiplier,
maxUncompressedBatchBytesSize: cfg.MaxUncompressedBatchBytesSize,
replayMode: false,
minCodecVersion: minCodecVersion,
chainCfg: chainCfg,
@@ -175,6 +177,14 @@ func NewChunkProposer(ctx context.Context, cfg *config.ChunkProposerConfig, minC
return p
}
// SetReplayDB sets the replay database for the ChunkProposer.
// This is used for the proposer tool only, to change the l2_block data source.
// This function is not thread-safe and should be called after initializing the ChunkProposer and before starting to propose chunks.
func (p *ChunkProposer) SetReplayDB(replayDB *gorm.DB) {
p.l2BlockOrm = orm.NewL2Block(replayDB)
p.replayMode = true
}
// TryProposeChunk tries to propose a new chunk.
func (p *ChunkProposer) TryProposeChunk() {
p.chunkProposerCircleTotal.Inc()
@@ -241,10 +251,13 @@ func (p *ChunkProposer) updateDBChunkInfo(chunk *encoding.Chunk, codecVersion en
log.Warn("ChunkProposer.InsertChunk failed", "codec version", codecVersion, "err", err)
return err
}
// In replayMode we don't need to update chunk_hash in l2_block table.
if !p.replayMode {
if err := p.l2BlockOrm.UpdateChunkHashInRange(p.ctx, dbChunk.StartBlockNumber, dbChunk.EndBlockNumber, dbChunk.Hash, dbTX); err != nil {
log.Error("failed to update chunk_hash for l2_blocks", "chunk hash", dbChunk.Hash, "start block", dbChunk.StartBlockNumber, "end block", dbChunk.EndBlockNumber, "err", err)
log.Error("failed to update chunk_hash for l2_block", "chunk hash", dbChunk.Hash, "start block", dbChunk.StartBlockNumber, "end block", dbChunk.EndBlockNumber, "err", err)
return err
}
}
return nil
})
if err != nil {
@@ -436,6 +449,12 @@ func (p *ChunkProposer) recordTimerChunkMetrics(metrics *utils.ChunkMetrics) {
}
func (p *ChunkProposer) tryProposeEuclidTransitionChunk(blocks []*encoding.Block) (bool, error) {
// If we are in replay mode, there is a corner case when StartL2Block is set as 0 in this check,
// it needs to get genesis block, but in mainnet db there is no genesis block, so we need to bypass this check.
if p.replayMode {
return false, nil
}
if !p.chainCfg.IsEuclid(blocks[0].Header.Time) {
return false, nil
}

View File

@@ -0,0 +1,160 @@
package watcher
import (
"context"
"fmt"
"math/big"
"time"
"github.com/scroll-tech/da-codec/encoding"
"github.com/scroll-tech/go-ethereum/common"
gethTypes "github.com/scroll-tech/go-ethereum/core/types"
"github.com/scroll-tech/go-ethereum/ethclient"
"github.com/scroll-tech/go-ethereum/log"
"github.com/scroll-tech/go-ethereum/params"
"gorm.io/gorm"
"scroll-tech/common/database"
"scroll-tech/common/utils"
"scroll-tech/database/migrate"
"scroll-tech/rollup/internal/config"
"scroll-tech/rollup/internal/orm"
rutils "scroll-tech/rollup/internal/utils"
)
// ProposerTool is a tool for proposing chunks and bundles to the L1 chain.
type ProposerTool struct {
ctx context.Context
cancel context.CancelFunc
db *gorm.DB
dbForReplay *gorm.DB
client *ethclient.Client
chunkProposer *ChunkProposer
batchProposer *BatchProposer
bundleProposer *BundleProposer
}
// NewProposerTool creates a new ProposerTool instance.
func NewProposerTool(ctx context.Context, cancel context.CancelFunc, cfg *config.ConfigForReplay, startL2BlockHeight uint64, minCodecVersion encoding.CodecVersion, chainCfg *params.ChainConfig) (*ProposerTool, error) {
// Init db connection
db, err := database.InitDB(cfg.DBConfig)
if err != nil {
return nil, fmt.Errorf("failed to init db connection: %w", err)
}
sqlDB, err := db.DB()
if err != nil {
return nil, fmt.Errorf("failed to get db connection: %w", err)
}
if err = migrate.ResetDB(sqlDB); err != nil {
return nil, fmt.Errorf("failed to reset db: %w", err)
}
log.Info("successfully reset db")
// Init dbForReplay connection
dbForReplay, err := database.InitDB(cfg.DBConfigForReplay)
if err != nil {
return nil, fmt.Errorf("failed to init dbForReplay connection: %w", err)
}
client, err := ethclient.Dial(cfg.L2Config.Endpoint)
if err != nil {
return nil, fmt.Errorf("failed to connect to L2 geth, endpoint: %s, err: %w", cfg.L2Config.Endpoint, err)
}
prevChunk, err := orm.NewChunk(dbForReplay).GetParentChunkByBlockNumber(ctx, startL2BlockHeight)
if err != nil {
return nil, fmt.Errorf("failed to get previous chunk: %w", err)
}
var startQueueIndex uint64
if prevChunk != nil {
startQueueIndex = prevChunk.TotalL1MessagesPoppedBefore + prevChunk.TotalL1MessagesPoppedInChunk
}
startBlock := uint64(0)
if prevChunk != nil {
startBlock = prevChunk.EndBlockNumber + 1
}
var chunk *encoding.Chunk
for blockNum := startBlock; blockNum <= startL2BlockHeight; blockNum++ {
block, err := client.BlockByNumber(ctx, new(big.Int).SetUint64(blockNum))
if err != nil {
return nil, fmt.Errorf("failed to get block %d: %w", blockNum, err)
}
for _, tx := range block.Transactions() {
if tx.Type() == gethTypes.L1MessageTxType {
startQueueIndex++
}
}
if blockNum == startL2BlockHeight {
chunk = &encoding.Chunk{Blocks: []*encoding.Block{{Header: block.Header()}}}
}
}
// Setting empty hash as the post_l1_message_queue_hash of the first chunk,
// i.e., treating the first L1 message after this chunk as the first L1 message in message queue v2.
// Though this setting is different from mainnet, it's simple yet sufficient for data analysis usage.
_, err = orm.NewChunk(db).InsertTestChunkForProposerTool(ctx, chunk, minCodecVersion, startQueueIndex)
if err != nil {
return nil, fmt.Errorf("failed to insert chunk, minCodecVersion: %d, startQueueIndex: %d, err: %w", minCodecVersion, startQueueIndex, err)
}
batch := &encoding.Batch{
Index: 0,
TotalL1MessagePoppedBefore: 0,
ParentBatchHash: common.Hash{},
Chunks: []*encoding.Chunk{chunk},
}
var dbBatch *orm.Batch
dbBatch, err = orm.NewBatch(db).InsertBatch(ctx, batch, encoding.CodecV0, rutils.BatchMetrics{})
if err != nil {
return nil, fmt.Errorf("failed to insert batch: %w", err)
}
if err = orm.NewChunk(db).UpdateBatchHashInRange(ctx, 0, 0, dbBatch.Hash); err != nil {
return nil, fmt.Errorf("failed to update batch hash for chunks: %w", err)
}
chunkProposer := NewChunkProposer(ctx, cfg.L2Config.ChunkProposerConfig, minCodecVersion, chainCfg, db, nil)
chunkProposer.SetReplayDB(dbForReplay)
batchProposer := NewBatchProposer(ctx, cfg.L2Config.BatchProposerConfig, minCodecVersion, chainCfg, db, nil)
batchProposer.SetReplayDB(dbForReplay)
bundleProposer := NewBundleProposer(ctx, cfg.L2Config.BundleProposerConfig, minCodecVersion, chainCfg, db, nil)
return &ProposerTool{
ctx: ctx,
cancel: cancel,
db: db,
dbForReplay: dbForReplay,
client: client,
chunkProposer: chunkProposer,
batchProposer: batchProposer,
bundleProposer: bundleProposer,
}, nil
}
func (p *ProposerTool) Start() {
go utils.Loop(p.ctx, 100*time.Millisecond, p.chunkProposer.TryProposeChunk)
go utils.Loop(p.ctx, 100*time.Millisecond, p.batchProposer.TryProposeBatch)
go utils.Loop(p.ctx, 100*time.Millisecond, p.bundleProposer.TryProposeBundle)
}
func (p *ProposerTool) Stop() {
p.cancel()
if err := database.CloseDB(p.db); err != nil {
log.Error("failed to close db connection", "error", err)
}
if err := database.CloseDB(p.dbForReplay); err != nil {
log.Error("failed to close dbForReplay connection", "error", err)
}
p.client.Close()
}

View File

@@ -179,6 +179,25 @@ func (o *Chunk) GetChunksByBatchHash(ctx context.Context, batchHash string) ([]*
return chunks, nil
}
// GetParentChunkByBlockNumber retrieves the parent chunk by block number
// only used by proposer tool for analysis usage
func (o *Chunk) GetParentChunkByBlockNumber(ctx context.Context, blockNumber uint64) (*Chunk, error) {
db := o.db.WithContext(ctx)
db = db.Model(&Chunk{})
db = db.Where("end_block_number < ?", blockNumber)
db = db.Order("end_block_number DESC")
db = db.Limit(1)
var chunk Chunk
if err := db.First(&chunk).Error; err != nil {
if errors.Is(err, gorm.ErrRecordNotFound) {
return nil, nil
}
return nil, fmt.Errorf("Chunk.GetParentChunkByBlockNumber error: %w", err)
}
return &chunk, nil
}
// InsertChunk inserts a new chunk into the database.
func (o *Chunk) InsertChunk(ctx context.Context, chunk *encoding.Chunk, codecVersion encoding.CodecVersion, metrics rutils.ChunkMetrics, dbTX ...*gorm.DB) (*Chunk, error) {
if chunk == nil || len(chunk.Blocks) == 0 {
@@ -259,6 +278,51 @@ func (o *Chunk) InsertChunk(ctx context.Context, chunk *encoding.Chunk, codecVer
return &newChunk, nil
}
// InsertTestChunkForProposerTool inserts a new chunk into the database only for analysis usage by proposer tool.
func (o *Chunk) InsertTestChunkForProposerTool(ctx context.Context, chunk *encoding.Chunk, codecVersion encoding.CodecVersion, totalL1MessagePoppedBefore uint64, dbTX ...*gorm.DB) (*Chunk, error) {
if chunk == nil || len(chunk.Blocks) == 0 {
return nil, errors.New("invalid args")
}
chunkHash, err := rutils.GetChunkHash(chunk, totalL1MessagePoppedBefore, codecVersion)
if err != nil {
log.Error("failed to get chunk hash", "err", err)
return nil, fmt.Errorf("Chunk.InsertChunk error: %w", err)
}
numBlocks := len(chunk.Blocks)
firstBlock := chunk.Blocks[0]
lastBlock := chunk.Blocks[numBlocks-1]
newChunk := Chunk{
Index: 0,
Hash: chunkHash.Hex(),
StartBlockNumber: firstBlock.Header.Number.Uint64(),
StartBlockHash: firstBlock.Header.Hash().Hex(),
EndBlockNumber: lastBlock.Header.Number.Uint64(),
EndBlockHash: lastBlock.Header.Hash().Hex(),
TotalL2TxGas: chunk.TotalGasUsed(),
TotalL2TxNum: chunk.NumL2Transactions(),
StartBlockTime: firstBlock.Header.Time,
TotalL1MessagesPoppedBefore: totalL1MessagePoppedBefore,
StateRoot: lastBlock.Header.Root.Hex(),
WithdrawRoot: lastBlock.WithdrawRoot.Hex(),
CodecVersion: int16(codecVersion),
}
db := o.db
if len(dbTX) > 0 && dbTX[0] != nil {
db = dbTX[0]
}
db = db.WithContext(ctx)
db = db.Model(&Chunk{})
if err := db.Create(&newChunk).Error; err != nil {
return nil, fmt.Errorf("Chunk.InsertChunk error: %w, chunk hash: %v", err, newChunk.Hash)
}
return &newChunk, nil
}
// UpdateProvingStatus updates the proving status of a chunk.
func (o *Chunk) UpdateProvingStatus(ctx context.Context, hash string, status types.ProvingStatus, dbTX ...*gorm.DB) error {
updateFields := make(map[string]interface{})

View File

@@ -0,0 +1,40 @@
{
"l2_config": {
"endpoint": "https://rpc.scroll.io",
"chunk_proposer_config": {
"max_block_num_per_chunk": 100,
"max_tx_num_per_chunk": 100,
"max_l2_gas_per_chunk": 20000000,
"max_l1_commit_gas_per_chunk": 5000000,
"max_l1_commit_calldata_size_per_chunk": 123740,
"chunk_timeout_sec": 72000000000,
"max_row_consumption_per_chunk": 10000000000,
"gas_cost_increase_multiplier": 1.2,
"max_uncompressed_batch_bytes_size": 634693
},
"batch_proposer_config": {
"max_l1_commit_gas_per_batch": 5000000,
"max_l1_commit_calldata_size_per_batch": 123740,
"batch_timeout_sec": 72000000000,
"gas_cost_increase_multiplier": 1.2,
"max_uncompressed_batch_bytes_size": 634693,
"max_chunks_per_batch": 45
},
"bundle_proposer_config": {
"max_batch_num_per_bundle": 45,
"bundle_timeout_sec": 36000000000
}
},
"db_config": {
"driver_name": "postgres",
"dsn": "postgres://postgres:postgres@db:5432/scroll?sslmode=disable",
"maxOpenNum": 200,
"maxIdleNum": 20
},
"db_config_for_replay": {
"driver_name": "postgres",
"dsn": "<mainnet read db config>",
"maxOpenNum": 200,
"maxIdleNum": 20
}
}

View File

@@ -0,0 +1,19 @@
{
"config": {
"chainId": 534352,
"bernoulliBlock": 0,
"curieBlock": 0,
"darwinTime": 0,
"darwinV2Time": 0,
"euclidTime": 0,
"euclidV2Time": 0
},
"nonce": "0x0000000000000033",
"timestamp": "0x0",
"parentHash": "0x0000000000000000000000000000000000000000000000000000000000000000",
"gasLimit": "0x8000000",
"difficulty": "0x100",
"mixhash": "0x0000000000000000000000000000000000000000000000000000000000000000",
"coinbase": "0x3333333333333333333333333333333333333333",
"alloc": {}
}

View File

@@ -0,0 +1,30 @@
# Download Go dependencies
FROM scrolltech/go-rust-builder:go-1.22-rust-nightly-2023-12-03 as base
WORKDIR /src
COPY go.work* ./
COPY ./rollup/go.* ./rollup/
COPY ./common/go.* ./common/
COPY ./coordinator/go.* ./coordinator/
COPY ./database/go.* ./database/
COPY ./tests/integration-test/go.* ./tests/integration-test/
COPY ./bridge-history-api/go.* ./bridge-history-api/
RUN go mod download -x
# Build proposer_tool
FROM base as builder
RUN --mount=target=. \
--mount=type=cache,target=/root/.cache/go-build \
cd /src/rollup/cmd/proposer_tool/ && CGO_LDFLAGS="-ldl" go build -v -p 4 -o /bin/proposer_tool
# Pull proposer_tool into a second stage deploy ubuntu container
FROM ubuntu:20.04
RUN apt update && apt install vim netcat-openbsd net-tools curl ca-certificates -y
ENV CGO_LDFLAGS="-ldl"
COPY --from=builder /bin/proposer_tool /bin/
WORKDIR /app
ENTRYPOINT ["proposer_tool"]