From cd4b94a4d4747e9c1fe905f5571cb76feabe240d Mon Sep 17 00:00:00 2001 From: protolambda Date: Thu, 26 Jan 2023 17:27:48 +0100 Subject: [PATCH 1/9] op-node: EIP-4844 support Original rebased prototype by proto, plus changes from Roberto: - encapsulate data <-> blob conversion code, add unit tests - update 4844 code to be compatible with latest beacon node api - remove stray file, include one more invalid blob test - appropriate fee bumping & blob fee estimation for blob transactions - misc other improvements --- op-batcher/batcher/driver.go | 21 +- op-chain-ops/genesis/config.go | 16 ++ op-e2e/actions/l2_engine.go | 16 +- op-e2e/actions/l2_verifier.go | 3 +- op-e2e/e2eutils/geth/geth.go | 7 +- op-e2e/eip4844_test.go | 110 ++++++++ op-e2e/op_geth.go | 2 +- op-e2e/setup.go | 23 +- op-e2e/system_test.go | 4 +- op-node/flags/flags.go | 7 + op-node/node/client.go | 40 +++ op-node/node/config.go | 7 + op-node/node/node.go | 23 +- op-node/rollup/derive/blob_data_source.go | 129 +++++++++ op-node/rollup/derive/call_data_source.go | 86 ++++++ op-node/rollup/derive/calldata_source.go | 132 ---------- op-node/rollup/derive/data_source.go | 59 +++++ ...ata_source_test.go => data_source_test.go} | 8 +- op-node/rollup/derive/l1_retrieval.go | 6 +- op-node/rollup/derive/l1_retrieval_test.go | 15 +- op-node/rollup/derive/pipeline.go | 4 +- op-node/rollup/driver/driver.go | 4 +- op-node/rollup/types.go | 5 +- op-node/service.go | 11 + op-program/client/driver/driver.go | 2 +- op-service/client/http.go | 39 +++ op-service/eth/blob.go | 65 +++++ op-service/eth/blob_test.go | 78 ++++++ op-service/sources/l1_beacon_client.go | 135 ++++++++++ op-service/txmgr/price_bump_test.go | 2 +- op-service/txmgr/queue_test.go | 2 +- op-service/txmgr/txmgr.go | 245 +++++++++++++----- op-service/txmgr/txmgr_test.go | 150 ++++++++--- 33 files changed, 1171 insertions(+), 285 deletions(-) create mode 100644 op-e2e/eip4844_test.go create mode 100644 op-node/rollup/derive/blob_data_source.go create mode 100644 op-node/rollup/derive/call_data_source.go delete mode 100644 op-node/rollup/derive/calldata_source.go create mode 100644 op-node/rollup/derive/data_source.go rename op-node/rollup/derive/{calldata_source_test.go => data_source_test.go} (95%) create mode 100644 op-service/client/http.go create mode 100644 op-service/eth/blob_test.go create mode 100644 op-service/sources/l1_beacon_client.go diff --git a/op-batcher/batcher/driver.go b/op-batcher/batcher/driver.go index f55f647dbf17..541dd2919985 100644 --- a/op-batcher/batcher/driver.go +++ b/op-batcher/batcher/driver.go @@ -334,7 +334,7 @@ func (l *BatchSubmitter) publishTxToL1(ctx context.Context, queue *txmgr.Queue[t // send all available transactions l1tip, err := l.l1Tip(ctx) if err != nil { - l.Log.Error("Failed to query L1 tip", "error", err) + l.Log.Error("Failed to query L1 tip", "err", err) return err } l.recordL1Tip(l1tip) @@ -353,15 +353,29 @@ func (l *BatchSubmitter) publishTxToL1(ctx context.Context, queue *txmgr.Queue[t return nil } -// sendTransaction creates & submits a transaction to the batch inbox address with the given `data`. +// sendTransaction creates & submits a transaction to the batch inbox address with the given `txData`. // It currently uses the underlying `txmgr` to handle transaction sending & price management. // This is a blocking method. It should not be called concurrently. func (l *BatchSubmitter) sendTransaction(txdata txData, queue *txmgr.Queue[txData], receiptsCh chan txmgr.TxReceipt[txData]) { // Do the gas estimation offline. A value of 0 will cause the [txmgr] to estimate the gas limit. data := txdata.Bytes() + + var blobs []*eth.Blob + if l.RollupConfig.BlobsEnabledL1Timestamp != nil && *l.RollupConfig.BlobsEnabledL1Timestamp <= uint64(time.Now().Unix()) { + var b eth.Blob + if err := b.FromData(data); err != nil { + l.Log.Error("data could not be converted to blob", "err", err) + return + } + blobs = append(blobs, &b) + + // no calldata + data = []byte{} + } + intrinsicGas, err := core.IntrinsicGas(data, nil, false, true, true, false) if err != nil { - l.Log.Error("Failed to calculate intrinsic gas", "error", err) + l.Log.Error("Failed to calculate intrinsic gas", "err", err) return } @@ -369,6 +383,7 @@ func (l *BatchSubmitter) sendTransaction(txdata txData, queue *txmgr.Queue[txDat To: &l.RollupConfig.BatchInboxAddress, TxData: data, GasLimit: intrinsicGas, + Blobs: blobs, } queue.Send(txdata, candidate, receiptsCh) } diff --git a/op-chain-ops/genesis/config.go b/op-chain-ops/genesis/config.go index c703d66fe4e2..57d925c5ae91 100644 --- a/op-chain-ops/genesis/config.go +++ b/op-chain-ops/genesis/config.go @@ -230,6 +230,9 @@ type DeployConfig struct { // When Cancun activates. Relative to L1 genesis. L1CancunTimeOffset *uint64 `json:"l1CancunTimeOffset,omitempty"` + + // When 4844 blob-tx functionality for rollup DA actives. Relative to L2 genesis. + L2BlobsUpgradeTimeOffset *uint64 `json:"l2BlobsUpgradeTimeOffset,omitempty"` } // Copy will deeply copy the DeployConfig. This does a JSON roundtrip to copy @@ -522,6 +525,17 @@ func (d *DeployConfig) InteropTime(genesisTime uint64) *uint64 { return &v } +func (d *DeployConfig) BlobsUpgradeTime(genesisTime uint64) *uint64 { + if d.L2BlobsUpgradeTimeOffset == nil { + return nil + } + v := uint64(0) + if offset := *d.L2BlobsUpgradeTimeOffset; offset > 0 { + v = genesisTime + uint64(offset) + } + return &v +} + // RollupConfig converts a DeployConfig to a rollup.Config func (d *DeployConfig) RollupConfig(l1StartBlock *types.Block, l2GenesisBlockHash common.Hash, l2GenesisBlockNumber uint64) (*rollup.Config, error) { if d.OptimismPortalProxy == (common.Address{}) { @@ -564,6 +578,8 @@ func (d *DeployConfig) RollupConfig(l1StartBlock *types.Block, l2GenesisBlockHas EclipseTime: d.EclipseTime(l1StartBlock.Time()), FjordTime: d.FjordTime(l1StartBlock.Time()), InteropTime: d.InteropTime(l1StartBlock.Time()), + // 4844 blobs usage activation for rollup DA + BlobsEnabledL1Timestamp: d.BlobsUpgradeTime(l1StartBlock.Time()), }, nil } diff --git a/op-e2e/actions/l2_engine.go b/op-e2e/actions/l2_engine.go index d1230ec8b7c5..486ff6fafaf4 100644 --- a/op-e2e/actions/l2_engine.go +++ b/op-e2e/actions/l2_engine.go @@ -3,12 +3,11 @@ package actions import ( "errors" - "github.com/ethereum-optimism/optimism/op-e2e/e2eutils" - "github.com/ethereum-optimism/optimism/op-program/client/l2/engineapi" "github.com/stretchr/testify/require" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core" + "github.com/ethereum/go-ethereum/core/txpool/blobpool" "github.com/ethereum/go-ethereum/core/types" geth "github.com/ethereum/go-ethereum/eth" "github.com/ethereum/go-ethereum/eth/ethconfig" @@ -20,9 +19,11 @@ import ( "github.com/ethereum/go-ethereum/node" "github.com/ethereum/go-ethereum/rpc" + "github.com/ethereum-optimism/optimism/op-e2e/e2eutils" "github.com/ethereum-optimism/optimism/op-node/rollup" + "github.com/ethereum-optimism/optimism/op-program/client/l2/engineapi" "github.com/ethereum-optimism/optimism/op-service/client" - "github.com/ethereum-optimism/optimism/op-service/eth" + opeth "github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-service/sources" "github.com/ethereum-optimism/optimism/op-service/testutils" ) @@ -48,7 +49,7 @@ type L2Engine struct { type EngineOption func(ethCfg *ethconfig.Config, nodeCfg *node.Config) error -func NewL2Engine(t Testing, log log.Logger, genesis *core.Genesis, rollupGenesisL1 eth.BlockID, jwtPath string, options ...EngineOption) *L2Engine { +func NewL2Engine(t Testing, log log.Logger, genesis *core.Genesis, rollupGenesisL1 opeth.BlockID, jwtPath string, options ...EngineOption) *L2Engine { n, ethBackend, apiBackend := newBackend(t, genesis, jwtPath, options) engineApi := engineapi.NewL2EngineAPI(log, apiBackend) chain := ethBackend.BlockChain() @@ -59,7 +60,7 @@ func NewL2Engine(t Testing, log log.Logger, genesis *core.Genesis, rollupGenesis eth: ethBackend, rollupGenesis: &rollup.Genesis{ L1: rollupGenesisL1, - L2: eth.BlockID{Hash: genesisBlock.Hash(), Number: genesisBlock.NumberU64()}, + L2: opeth.BlockID{Hash: genesisBlock.Hash(), Number: genesisBlock.NumberU64()}, L2Time: genesis.Timestamp, }, l2Chain: chain, @@ -84,6 +85,11 @@ func newBackend(t e2eutils.TestingBase, genesis *core.Genesis, jwtPath string, o ethCfg := ðconfig.Config{ NetworkId: genesis.Config.ChainID.Uint64(), Genesis: genesis, + BlobPool: blobpool.Config{ + Datadir: t.TempDir(), + Datacap: blobpool.DefaultConfig.Datacap, + PriceBump: blobpool.DefaultConfig.PriceBump, + }, } nodeCfg := &node.Config{ Name: "l2-geth", diff --git a/op-e2e/actions/l2_verifier.go b/op-e2e/actions/l2_verifier.go index 229de1b77076..c7fe6eae83c7 100644 --- a/op-e2e/actions/l2_verifier.go +++ b/op-e2e/actions/l2_verifier.go @@ -59,7 +59,8 @@ type L2API interface { func NewL2Verifier(t Testing, log log.Logger, l1 derive.L1Fetcher, eng L2API, cfg *rollup.Config, syncCfg *sync.Config) *L2Verifier { metrics := &testutils.TestDerivationMetrics{} - pipeline := derive.NewDerivationPipeline(log, cfg, l1, eng, metrics, syncCfg) + // TODO blob testing + pipeline := derive.NewDerivationPipeline(log, cfg, l1, nil, eng, metrics, syncCfg) pipeline.Reset() rollupNode := &L2Verifier{ diff --git a/op-e2e/e2eutils/geth/geth.go b/op-e2e/e2eutils/geth/geth.go index fc8660acf943..c9502d183ed6 100644 --- a/op-e2e/e2eutils/geth/geth.go +++ b/op-e2e/e2eutils/geth/geth.go @@ -83,7 +83,7 @@ func defaultNodeConfig(name string, jwtPath string) *node.Config { type GethOption func(ethCfg *ethconfig.Config, nodeCfg *node.Config) error // InitL2 inits a L2 geth node. -func InitL2(name string, l2ChainID *big.Int, genesis *core.Genesis, jwtPath string, opts ...GethOption) (*node.Node, *eth.Ethereum, error) { +func InitL2(name string, l2ChainID *big.Int, genesis *core.Genesis, jwtPath string, blobPoolDir string, opts ...GethOption) (*node.Node, *eth.Ethereum, error) { ethConfig := ðconfig.Config{ NetworkId: l2ChainID.Uint64(), Genesis: genesis, @@ -96,6 +96,11 @@ func InitL2(name string, l2ChainID *big.Int, genesis *core.Genesis, jwtPath stri Recommit: 0, NewPayloadTimeout: 0, }, + BlobPool: blobpool.Config{ + Datadir: blobPoolDir, + Datacap: blobpool.DefaultConfig.Datacap, + PriceBump: blobpool.DefaultConfig.PriceBump, + }, } nodeConfig := defaultNodeConfig(fmt.Sprintf("l2-geth-%v", name), jwtPath) return createGethNode(true, nodeConfig, ethConfig, opts...) diff --git a/op-e2e/eip4844_test.go b/op-e2e/eip4844_test.go new file mode 100644 index 000000000000..e905c3ba92e5 --- /dev/null +++ b/op-e2e/eip4844_test.go @@ -0,0 +1,110 @@ +package op_e2e + +import ( + "context" + "math/big" + "testing" + "time" + + "github.com/stretchr/testify/require" + + "github.com/ethereum-optimism/optimism/op-service/client" + "github.com/ethereum-optimism/optimism/op-service/sources" + "github.com/ethereum-optimism/optimism/op-service/testlog" + "github.com/ethereum/go-ethereum/accounts/abi/bind" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/rpc" +) + +// TestSystem4844E2E runs the SystemE2E test with 4844 enabled on L1, +// and active on the rollup in the op-batcher and verifier. +func TestSystem4844E2E(t *testing.T) { + InitParallel(t) + + cfg := DefaultSystemConfig(t) + genesisActivation := uint64(0) + cfg.DeployConfig.L1CancunTimeOffset = &genesisActivation + cfg.DeployConfig.L2BlobsUpgradeTimeOffset = &genesisActivation + + sys, err := cfg.Start(t) + require.Nil(t, err, "Error starting up system") + defer sys.Close() + + log := testlog.Logger(t, log.LvlInfo) + log.Info("genesis", "l2", sys.RollupConfig.Genesis.L2, "l1", sys.RollupConfig.Genesis.L1, "l2_time", sys.RollupConfig.Genesis.L2Time) + + l1Client := sys.Clients["l1"] + l2Seq := sys.Clients["sequencer"] + l2Verif := sys.Clients["verifier"] + + // Transactor Account + ethPrivKey := cfg.Secrets.Alice + + // Send Transaction & wait for success + fromAddr := cfg.Secrets.Addresses().Alice + log.Info("alice", "addr", fromAddr) + + ctx, cancel := context.WithTimeout(context.Background(), 15*time.Second) + defer cancel() + startBalance, err := l2Verif.BalanceAt(ctx, fromAddr, nil) + require.Nil(t, err) + + // Send deposit transaction + opts, err := bind.NewKeyedTransactorWithChainID(ethPrivKey, cfg.L1ChainIDBig()) + require.Nil(t, err) + mintAmount := big.NewInt(1_000_000_000_000) + opts.Value = mintAmount + SendDepositTx(t, cfg, l1Client, l2Verif, opts, func(l2Opts *DepositTxOpts) {}) + + // Confirm balance + ctx, cancel = context.WithTimeout(context.Background(), 15*time.Second) + defer cancel() + endBalance, err := l2Verif.BalanceAt(ctx, fromAddr, nil) + require.Nil(t, err) + + diff := new(big.Int) + diff = diff.Sub(endBalance, startBalance) + require.Equal(t, mintAmount, diff, "Did not get expected balance change") + + // Submit TX to L2 sequencer node + receipt := SendL2Tx(t, cfg, l2Seq, ethPrivKey, func(opts *TxOpts) { + opts.Value = big.NewInt(1_000_000_000) + opts.Nonce = 1 // Already have deposit + opts.ToAddr = &common.Address{0xff, 0xff} + opts.VerifyOnClients(l2Verif) + }) + + // Verify blocks match after batch submission on verifiers and sequencers + verifBlock, err := l2Verif.BlockByNumber(context.Background(), receipt.BlockNumber) + require.Nil(t, err) + seqBlock, err := l2Seq.BlockByNumber(context.Background(), receipt.BlockNumber) + require.Nil(t, err) + require.Equal(t, verifBlock.NumberU64(), seqBlock.NumberU64(), "Verifier and sequencer blocks not the same after including a batch tx") + require.Equal(t, verifBlock.ParentHash(), seqBlock.ParentHash(), "Verifier and sequencer blocks parent hashes not the same after including a batch tx") + require.Equal(t, verifBlock.Hash(), seqBlock.Hash(), "Verifier and sequencer blocks not the same after including a batch tx") + + rollupRPCClient, err := rpc.DialContext(context.Background(), sys.RollupNodes["sequencer"].HTTPEndpoint()) + require.Nil(t, err) + rollupClient := sources.NewRollupClient(client.NewBaseRPCClient(rollupRPCClient)) + // basic check that sync status works + seqStatus, err := rollupClient.SyncStatus(context.Background()) + require.Nil(t, err) + require.LessOrEqual(t, seqBlock.NumberU64(), seqStatus.UnsafeL2.Number) + // basic check that version endpoint works + seqVersion, err := rollupClient.Version(context.Background()) + require.Nil(t, err) + require.NotEqual(t, "", seqVersion) + + // quick check that the batch submitter works + for i := 0; i < 10; i++ { + // wait for chain to be marked as "safe" (i.e. confirm batch-submission works) + stat, err := rollupClient.SyncStatus(context.Background()) + require.NoError(t, err) + if stat.SafeL2.Number > 0 { + return + } + time.Sleep(2 * time.Second) + } + t.Fatal("expected L2 to be batch-submitted and labeled as safe") +} diff --git a/op-e2e/op_geth.go b/op-e2e/op_geth.go index bd2ddfb0ff8f..51cc21aadfe3 100644 --- a/op-e2e/op_geth.go +++ b/op-e2e/op_geth.go @@ -76,7 +76,7 @@ func NewOpGeth(t *testing.T, ctx context.Context, cfg *SystemConfig) (*OpGeth, e var node EthInstance if cfg.ExternalL2Shim == "" { - gethNode, _, err := geth.InitL2("l2", big.NewInt(int64(cfg.DeployConfig.L2ChainID)), l2Genesis, cfg.JWTFilePath) + gethNode, _, err := geth.InitL2("l2", big.NewInt(int64(cfg.DeployConfig.L2ChainID)), l2Genesis, cfg.JWTFilePath, t.TempDir()) require.Nil(t, err) require.Nil(t, gethNode.Start()) node = gethNode diff --git a/op-e2e/setup.go b/op-e2e/setup.go index f9ebd17d875a..e43f727a662d 100644 --- a/op-e2e/setup.go +++ b/op-e2e/setup.go @@ -494,6 +494,8 @@ func (cfg SystemConfig) Start(t *testing.T, _opts ...SystemConfigOption) (*Syste FjordTime: cfg.DeployConfig.FjordTime(uint64(cfg.DeployConfig.L1GenesisBlockTimestamp)), InteropTime: cfg.DeployConfig.InteropTime(uint64(cfg.DeployConfig.L1GenesisBlockTimestamp)), ProtocolVersionsAddress: cfg.L1Deployments.ProtocolVersionsProxy, + // 4844 + BlobsEnabledL1Timestamp: cfg.DeployConfig.BlobsUpgradeTime(uint64(cfg.DeployConfig.L1GenesisBlockTimestamp)), } } defaultConfig := makeRollupConfig() @@ -509,8 +511,8 @@ func (cfg SystemConfig) Start(t *testing.T, _opts ...SystemConfigOption) (*Syste _ = bcn.Close() }) require.NoError(t, bcn.Start("127.0.0.1:0")) - beaconApiAddr := bcn.BeaconAddr() - require.NotEmpty(t, beaconApiAddr, "beacon API listener must be up") + sys.L1BeaconAPIAddr = bcn.BeaconAddr() + require.NotEmpty(t, sys.L1BeaconAPIAddr, "beacon API listener must be up") // Initialize nodes l1Node, l1Backend, err := geth.InitL1(cfg.DeployConfig.L1ChainID, cfg.DeployConfig.L1BlockTime, l1Genesis, c, @@ -529,8 +531,9 @@ func (cfg SystemConfig) Start(t *testing.T, _opts ...SystemConfigOption) (*Syste for name := range cfg.Nodes { var ethClient EthInstance + blobPoolPath := path.Join(cfg.BlobsPath, name) if cfg.ExternalL2Shim == "" { - node, backend, err := geth.InitL2(name, big.NewInt(int64(cfg.DeployConfig.L2ChainID)), l2Genesis, cfg.JWTFilePath, cfg.GethOptions[name]...) + node, backend, err := geth.InitL2(name, big.NewInt(int64(cfg.DeployConfig.L2ChainID)), l2Genesis, cfg.JWTFilePath, blobPoolPath, cfg.GethOptions[name]...) if err != nil { return nil, err } @@ -548,10 +551,11 @@ func (cfg SystemConfig) Start(t *testing.T, _opts ...SystemConfigOption) (*Syste t.Skip("External L2 nodes do not support configuration through GethOptions") } ethClient = (&ExternalRunner{ - Name: name, - BinPath: cfg.ExternalL2Shim, - Genesis: l2Genesis, - JWTPath: cfg.JWTFilePath, + Name: name, + BinPath: cfg.ExternalL2Shim, + Genesis: l2Genesis, + JWTPath: cfg.JWTFilePath, + BlobPoolPath: blobPoolPath, }).Run(t) } sys.EthInstances[name] = ethClient @@ -562,7 +566,7 @@ func (cfg SystemConfig) Start(t *testing.T, _opts ...SystemConfigOption) (*Syste // of only websockets (which are required for external eth client tests). for name, rollupCfg := range cfg.Nodes { configureL1(rollupCfg, sys.EthInstances["l1"]) - configureL2(rollupCfg, sys.EthInstances[name], cfg.JWTSecret) + configureL2(rollupCfg, sys.EthInstances[name], cfg.JWTSecret, sys.L1BeaconAPIAddr) } // Geth Clients @@ -865,7 +869,7 @@ type WSOrHTTPEndpoint interface { HTTPAuthEndpoint() string } -func configureL2(rollupNodeCfg *rollupNode.Config, l2Node WSOrHTTPEndpoint, jwtSecret [32]byte) { +func configureL2(rollupNodeCfg *rollupNode.Config, l2Node WSOrHTTPEndpoint, jwtSecret [32]byte, l1BeaconAPIAddr string) { l2EndpointConfig := l2Node.WSAuthEndpoint() if UseHTTP() { l2EndpointConfig = l2Node.HTTPAuthEndpoint() @@ -875,6 +879,7 @@ func configureL2(rollupNodeCfg *rollupNode.Config, l2Node WSOrHTTPEndpoint, jwtS L2EngineAddr: l2EndpointConfig, L2EngineJWTSecret: jwtSecret, } + rollupNodeCfg.Beacon = &rollupNode.L1BeaconEndpointConfig{BeaconAddr: l1BeaconAPIAddr} } func (cfg SystemConfig) L1ChainIDBig() *big.Int { diff --git a/op-e2e/system_test.go b/op-e2e/system_test.go index ea7f31c4fe7d..728f8d1bbea3 100644 --- a/op-e2e/system_test.go +++ b/op-e2e/system_test.go @@ -706,11 +706,11 @@ func TestSystemP2PAltSync(t *testing.T) { }, } configureL1(syncNodeCfg, sys.EthInstances["l1"]) - syncerL2Engine, _, err := geth.InitL2("syncer", big.NewInt(int64(cfg.DeployConfig.L2ChainID)), sys.L2GenesisCfg, cfg.JWTFilePath) + syncerL2Engine, _, err := geth.InitL2("syncer", big.NewInt(int64(cfg.DeployConfig.L2ChainID)), sys.L2GenesisCfg, cfg.JWTFilePath, t.TempDir()) require.NoError(t, err) require.NoError(t, syncerL2Engine.Start()) - configureL2(syncNodeCfg, syncerL2Engine, cfg.JWTSecret) + configureL2(syncNodeCfg, syncerL2Engine, cfg.JWTSecret, sys.L1BeaconAPIAddr) syncerNode, err := rollupNode.New(context.Background(), syncNodeCfg, cfg.Loggers["syncer"], snapLog, "", metrics.NewMetrics("")) require.NoError(t, err) diff --git a/op-node/flags/flags.go b/op-node/flags/flags.go index 2131eb216c6d..151b33755106 100644 --- a/op-node/flags/flags.go +++ b/op-node/flags/flags.go @@ -119,6 +119,12 @@ var ( EnvVars: prefixEnvVars("L1_HTTP_POLL_INTERVAL"), Value: time.Second * 12, } + BeaconAddr = &cli.StringFlag{ + Name: "beacon", + Usage: "Address of Beacon-node HTTP endpoint to use (beacon namespace required)", + Value: "http://127.0.0.1:4000", + EnvVars: prefixEnvVars("L1_BEACON_API"), + } VerifierL1Confs = &cli.Uint64Flag{ Name: "verifier.l1-confs", Usage: "Number of L1 blocks to keep distance from the L1 head before deriving L2 data from. Reorgs are supported, but may be slow to perform.", @@ -271,6 +277,7 @@ var optionalFlags = []cli.Flag{ SyncModeFlag, RPCListenAddr, RPCListenPort, + BeaconAddr, L1TrustRPC, L1RPCProviderKind, L1RPCRateLimit, diff --git a/op-node/node/client.go b/op-node/node/client.go index cec6b52865a6..0a35086f0dcb 100644 --- a/op-node/node/client.go +++ b/op-node/node/client.go @@ -29,6 +29,11 @@ type L1EndpointSetup interface { Check() error } +type L1BeaconEndpointSetup interface { + Setup(ctx context.Context, log log.Logger) (cl client.HTTP, err error) + Check() error +} + type L2EndpointConfig struct { L2EngineAddr string // Address of L2 Engine JSON-RPC endpoint to use (engine and eth namespace required) @@ -164,3 +169,38 @@ func (cfg *PreparedL1Endpoint) Check() error { return nil } + +type L1BeaconEndpointConfig struct { + BeaconAddr string // Address of L1 User Beacon-API endpoint to use (beacon namespace required) +} + +var _ L1BeaconEndpointSetup = (*L1BeaconEndpointConfig)(nil) + +func (cfg *L1BeaconEndpointConfig) Setup(ctx context.Context, log log.Logger) (cl client.HTTP, err error) { + return client.NewBasicHTTPClient(cfg.BeaconAddr, log), nil +} + +func (cfg *L1BeaconEndpointConfig) Check() error { + if cfg.BeaconAddr == "" { + return errors.New("expected beacon address, but got none") + } + return nil +} + +// PreparedL1BeaconEndpoint enables testing with an in-process pre-setup connection to a beacon node mock +type PreparedL1BeaconEndpoint struct { + Client client.HTTP +} + +var _ L1BeaconEndpointSetup = (*PreparedL1BeaconEndpoint)(nil) + +func (p *PreparedL1BeaconEndpoint) Setup(ctx context.Context, log log.Logger) (cl client.HTTP, err error) { + return p.Client, nil +} + +func (p *PreparedL1BeaconEndpoint) Check() error { + if p.Client == nil { + return errors.New("expected beacon client, but got none") + } + return nil +} diff --git a/op-node/node/config.go b/op-node/node/config.go index f169505530d9..9e4c29c1ea97 100644 --- a/op-node/node/config.go +++ b/op-node/node/config.go @@ -20,6 +20,8 @@ type Config struct { L1 L1EndpointSetup L2 L2EndpointSetup + Beacon L1BeaconEndpointSetup + Driver driver.Config Rollup rollup.Config @@ -124,6 +126,11 @@ func (cfg *Config) Check() error { if err := cfg.L2.Check(); err != nil { return fmt.Errorf("l2 endpoint config error: %w", err) } + if cfg.Beacon != nil { + if err := cfg.Beacon.Check(); err != nil { + return fmt.Errorf("beacon endpoint config error: %w", err) + } + } if err := cfg.Rollup.Check(); err != nil { return fmt.Errorf("rollup config error: %w", err) } diff --git a/op-node/node/node.go b/op-node/node/node.go index 586cd8df95c4..dc482a240753 100644 --- a/op-node/node/node.go +++ b/op-node/node/node.go @@ -58,6 +58,8 @@ type OpNode struct { pprofSrv *httputil.HTTPServer metricsSrv *httputil.HTTPServer + beacon *sources.L1BeaconClient + // some resources cannot be stopped directly, like the p2p gossipsub router (not our design), // and depend on this ctx to be closed. resourcesCtx context.Context @@ -114,6 +116,9 @@ func (n *OpNode) init(ctx context.Context, cfg *Config, snapshotLog log.Logger) if err := n.initL1(ctx, cfg); err != nil { return fmt.Errorf("failed to init L1: %w", err) } + if err := n.initL1BeaconAPI(ctx, cfg); err != nil { + return err + } if err := n.initL2(ctx, cfg, snapshotLog); err != nil { return fmt.Errorf("failed to init L2: %w", err) } @@ -288,6 +293,22 @@ func (n *OpNode) initRuntimeConfig(ctx context.Context, cfg *Config) error { return nil } +func (n *OpNode) initL1BeaconAPI(ctx context.Context, cfg *Config) error { + if cfg.Beacon == nil { + return nil + } + httpClient, err := cfg.Beacon.Setup(ctx, n.log) + if err != nil { + return fmt.Errorf("failed to setup L2 execution-engine RPC client: %w", err) + } + + // TODO: wrap http client with metrics + cl := sources.NewL1BeaconClient(httpClient) + n.beacon = cl + + return nil +} + func (n *OpNode) initL2(ctx context.Context, cfg *Config, snapshotLog log.Logger) error { rpcClient, rpcCfg, err := cfg.L2.Setup(ctx, n.log, &cfg.Rollup) if err != nil { @@ -305,7 +326,7 @@ func (n *OpNode) initL2(ctx context.Context, cfg *Config, snapshotLog log.Logger return err } - n.l2Driver = driver.NewDriver(&cfg.Driver, &cfg.Rollup, n.l2Source, n.l1Source, n, n, n.log, snapshotLog, n.metrics, cfg.ConfigPersistence, &cfg.Sync) + n.l2Driver = driver.NewDriver(&cfg.Driver, &cfg.Rollup, n.l2Source, n.l1Source, n.beacon, n, n, n.log, snapshotLog, n.metrics, cfg.ConfigPersistence, &cfg.Sync) return nil } diff --git a/op-node/rollup/derive/blob_data_source.go b/op-node/rollup/derive/blob_data_source.go new file mode 100644 index 000000000000..d9443d135b67 --- /dev/null +++ b/op-node/rollup/derive/blob_data_source.go @@ -0,0 +1,129 @@ +package derive + +import ( + "context" + "errors" + "fmt" + "io" + + "github.com/ethereum/go-ethereum" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/log" + + "github.com/ethereum-optimism/optimism/op-service/eth" +) + +// BlobDataSource fetches both call-data (backup) and blobs and transforms them into usable rollup data. +type BlobDataSource struct { + open bool + callData []eth.Data + blobDataHashes []eth.IndexedDataHash + + blobs []*eth.Blob + + ref eth.L1BlockRef + batcherAddr common.Address + + dsCfg DataSourceConfig + + fetcher L1TransactionFetcher + blobsFetcher L1BlobsFetcher + log log.Logger +} + +// NewBlobDataSource creates a new blob-data source. +func NewBlobDataSource(ctx context.Context, log log.Logger, dsCfg DataSourceConfig, fetcher L1TransactionFetcher, blobsFetcher L1BlobsFetcher, ref eth.L1BlockRef, batcherAddr common.Address) DataIter { + return &BlobDataSource{ + open: false, + ref: ref, + dsCfg: dsCfg, + fetcher: fetcher, + log: log.New("origin", ref), + batcherAddr: batcherAddr, + blobsFetcher: blobsFetcher, + } +} + +// Next returns the next piece of data if any remains. It returns ResetError if it cannot find the +// referenced block or a referenced blob, or TemporaryError for any other failure to fetch a block +// or blob. +func (ds *BlobDataSource) Next(ctx context.Context) (eth.Data, error) { + if !ds.open { + if _, txs, err := ds.fetcher.InfoAndTxsByHash(ctx, ds.ref.Hash); err == nil { + ds.open = true + ds.callData, ds.blobDataHashes = BlobDataFromEVMTransactions(ds.dsCfg, ds.batcherAddr, txs, ds.log) + } else if errors.Is(err, ethereum.NotFound) { + return nil, NewResetError(fmt.Errorf("failed to open blob-data source: %w", err)) + } else { + return nil, NewTemporaryError(fmt.Errorf("failed to open blob-data source: %w", err)) + } + } + // prioritize call-data + if len(ds.callData) != 0 { + data := ds.callData[0] + ds.callData = ds.callData[1:] + return data, nil + } + if len(ds.blobDataHashes) > 0 { // check if there is any blob data in this block we have opened. + if ds.blobs == nil { // fetch blobs if we haven't already + blobs, err := ds.blobsFetcher.BlobsByRefAndIndexedDataHashes(ctx, ds.ref, ds.blobDataHashes) + if errors.Is(err, ethereum.NotFound) { // if the L1 block was seen to be available, then the blobs should also be available + return nil, NewResetError(fmt.Errorf("failed to find blobs: %w", err)) + } else if err != nil { + return nil, NewTemporaryError(fmt.Errorf("failed to fetch blobs: %w", err)) + } + ds.blobs = blobs + } + if len(ds.blobs) > 0 { // parse the next blob, if any + b := ds.blobs[0] + ds.blobs = ds.blobs[1:] + data, err := b.ToData() + if err != nil { + ds.log.Error("ignoring blob due to parse failure", "err", err) + return ds.Next(ctx) + } + return data, nil + } + } + return nil, io.EOF +} + +// BlobDataFromEVMTransactions filters all of the transactions and returns the call-data and blob data-hashes +// from transactions that are sent to the batch inbox address from the batch sender address. +// This will return an empty array if no valid transactions are found. +// Call-data can be used as fallback in case blobs are overpriced or unstable. +func BlobDataFromEVMTransactions(dsCfg DataSourceConfig, batcherAddr common.Address, txs types.Transactions, log log.Logger) ([]eth.Data, []eth.IndexedDataHash) { + var callData []eth.Data + var indexedDataHashes []eth.IndexedDataHash + blobIndex := uint64(0) + for j, tx := range txs { + if to := tx.To(); to != nil && *to == dsCfg.batchInboxAddress { + seqDataSubmitter, err := dsCfg.l1Signer.Sender(tx) // optimization: only derive sender if To is correct + if err != nil { + log.Warn("tx in inbox with invalid signature", "index", j, "err", err) + blobIndex += uint64(len(tx.BlobHashes())) + continue // bad signature, ignore + } + // some random L1 user might have sent a transaction to our batch inbox, ignore them + if seqDataSubmitter != batcherAddr { + log.Warn("tx in inbox with unauthorized submitter", "index", j, "err", err) + blobIndex += uint64(len(tx.BlobHashes())) + continue // not an authorized batch submitter, ignore + } + if len(tx.Data()) > 0 { // ignore empty calldata + callData = append(callData, tx.Data()) + } + for _, h := range tx.BlobHashes() { + indexedDataHashes = append(indexedDataHashes, eth.IndexedDataHash{ + Index: blobIndex, + DataHash: h, + }) + blobIndex += 1 + } + } else { + blobIndex += uint64(len(tx.BlobHashes())) + } + } + return callData, indexedDataHashes +} diff --git a/op-node/rollup/derive/call_data_source.go b/op-node/rollup/derive/call_data_source.go new file mode 100644 index 000000000000..4b3bfb2824a8 --- /dev/null +++ b/op-node/rollup/derive/call_data_source.go @@ -0,0 +1,86 @@ +package derive + +import ( + "context" + "errors" + "fmt" + "io" + + "github.com/ethereum/go-ethereum" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/log" + + "github.com/ethereum-optimism/optimism/op-service/eth" +) + +// CallDataSource fetches call data (EVM inputs embedded in transactions) for a given block, +// filtered to a single batch submitter. +type CallDataSource struct { + open bool + callData []eth.Data + + ref eth.L1BlockRef + batcherAddr common.Address + + dsCfg DataSourceConfig + fetcher L1TransactionFetcher + log log.Logger +} + +// NewCallDataSource creates a new call-data source. +func NewCallDataSource(ctx context.Context, log log.Logger, dsCfg DataSourceConfig, fetcher L1TransactionFetcher, ref eth.L1BlockRef, batcherAddr common.Address) DataIter { + return &CallDataSource{ + open: false, + ref: ref, + dsCfg: dsCfg, + fetcher: fetcher, + log: log.New("origin", ref), + batcherAddr: batcherAddr, + } +} + +// Next returns the next piece of data if any remains. It returns ResetError if it cannot find the +// referenced block, or TemporaryError for any other failure to fetch the block. +func (ds *CallDataSource) Next(ctx context.Context) (eth.Data, error) { + if !ds.open { + if _, txs, err := ds.fetcher.InfoAndTxsByHash(ctx, ds.ref.Hash); err == nil { + ds.open = true + ds.callData = CallDataFromEVMTransactions(ds.dsCfg, ds.batcherAddr, txs, ds.log) + } else if errors.Is(err, ethereum.NotFound) { + return nil, NewResetError(fmt.Errorf("failed to open call-data source: %w", err)) + } else { + return nil, NewTemporaryError(fmt.Errorf("failed to open call-data source: %w", err)) + } + } + if len(ds.callData) == 0 { + return nil, io.EOF + } else { + data := ds.callData[0] + ds.callData = ds.callData[1:] + return data, nil + } +} + +// CallDataFromEVMTransactions filters all of the transactions and returns the call-data from transactions +// that are sent to the batch inbox address from the batch sender address. +// This will return an empty array if no valid transactions are found. +func CallDataFromEVMTransactions(dsCfg DataSourceConfig, batcherAddr common.Address, txs types.Transactions, log log.Logger) []eth.Data { + var out []eth.Data + for j, tx := range txs { + if to := tx.To(); to != nil && *to == dsCfg.batchInboxAddress { + seqDataSubmitter, err := dsCfg.l1Signer.Sender(tx) // optimization: only derive sender if To is correct + if err != nil { + log.Warn("tx in inbox with invalid signature", "index", j, "txHash", tx.Hash(), "err", err) + continue // bad signature, ignore + } + // some random L1 user might have sent a transaction to our batch inbox, ignore them + if seqDataSubmitter != batcherAddr { + log.Warn("tx in inbox with unauthorized submitter", "index", j, "txHash", tx.Hash(), "err", err) + continue // not an authorized batch submitter, ignore + } + out = append(out, tx.Data()) + } + } + return out +} diff --git a/op-node/rollup/derive/calldata_source.go b/op-node/rollup/derive/calldata_source.go deleted file mode 100644 index 3101aedc326c..000000000000 --- a/op-node/rollup/derive/calldata_source.go +++ /dev/null @@ -1,132 +0,0 @@ -package derive - -import ( - "context" - "errors" - "fmt" - "io" - - "github.com/ethereum/go-ethereum" - "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/core/types" - "github.com/ethereum/go-ethereum/log" - - "github.com/ethereum-optimism/optimism/op-node/rollup" - "github.com/ethereum-optimism/optimism/op-service/eth" -) - -type DataIter interface { - Next(ctx context.Context) (eth.Data, error) -} - -type L1TransactionFetcher interface { - InfoAndTxsByHash(ctx context.Context, hash common.Hash) (eth.BlockInfo, types.Transactions, error) -} - -// DataSourceFactory readers raw transactions from a given block & then filters for -// batch submitter transactions. -// This is not a stage in the pipeline, but a wrapper for another stage in the pipeline -type DataSourceFactory struct { - log log.Logger - dsCfg DataSourceConfig - fetcher L1TransactionFetcher -} - -func NewDataSourceFactory(log log.Logger, cfg *rollup.Config, fetcher L1TransactionFetcher) *DataSourceFactory { - return &DataSourceFactory{log: log, dsCfg: DataSourceConfig{l1Signer: cfg.L1Signer(), batchInboxAddress: cfg.BatchInboxAddress}, fetcher: fetcher} -} - -// OpenData returns a DataIter. This struct implements the `Next` function. -func (ds *DataSourceFactory) OpenData(ctx context.Context, id eth.BlockID, batcherAddr common.Address) DataIter { - return NewDataSource(ctx, ds.log, ds.dsCfg, ds.fetcher, id, batcherAddr) -} - -// DataSourceConfig regroups the mandatory rollup.Config fields needed for DataFromEVMTransactions. -type DataSourceConfig struct { - l1Signer types.Signer - batchInboxAddress common.Address -} - -// DataSource is a fault tolerant approach to fetching data. -// The constructor will never fail & it will instead re-attempt the fetcher -// at a later point. -type DataSource struct { - // Internal state + data - open bool - data []eth.Data - // Required to re-attempt fetching - id eth.BlockID - dsCfg DataSourceConfig - fetcher L1TransactionFetcher - log log.Logger - - batcherAddr common.Address -} - -// NewDataSource creates a new calldata source. It suppresses errors in fetching the L1 block if they occur. -// If there is an error, it will attempt to fetch the result on the next call to `Next`. -func NewDataSource(ctx context.Context, log log.Logger, dsCfg DataSourceConfig, fetcher L1TransactionFetcher, block eth.BlockID, batcherAddr common.Address) DataIter { - _, txs, err := fetcher.InfoAndTxsByHash(ctx, block.Hash) - - if err != nil { - return &DataSource{ - open: false, - id: block, - dsCfg: dsCfg, - fetcher: fetcher, - log: log, - batcherAddr: batcherAddr, - } - } else { - return &DataSource{ - open: true, - data: DataFromEVMTransactions(dsCfg, batcherAddr, txs, log.New("origin", block)), - } - } -} - -// Next returns the next piece of data if it has it. If the constructor failed, this -// will attempt to reinitialize itself. If it cannot find the block it returns a ResetError -// otherwise it returns a temporary error if fetching the block returns an error. -func (ds *DataSource) Next(ctx context.Context) (eth.Data, error) { - if !ds.open { - if _, txs, err := ds.fetcher.InfoAndTxsByHash(ctx, ds.id.Hash); err == nil { - ds.open = true - ds.data = DataFromEVMTransactions(ds.dsCfg, ds.batcherAddr, txs, log.New("origin", ds.id)) - } else if errors.Is(err, ethereum.NotFound) { - return nil, NewResetError(fmt.Errorf("failed to open calldata source: %w", err)) - } else { - return nil, NewTemporaryError(fmt.Errorf("failed to open calldata source: %w", err)) - } - } - if len(ds.data) == 0 { - return nil, io.EOF - } else { - data := ds.data[0] - ds.data = ds.data[1:] - return data, nil - } -} - -// DataFromEVMTransactions filters all of the transactions and returns the calldata from transactions -// that are sent to the batch inbox address from the batch sender address. -// This will return an empty array if no valid transactions are found. -func DataFromEVMTransactions(dsCfg DataSourceConfig, batcherAddr common.Address, txs types.Transactions, log log.Logger) []eth.Data { - var out []eth.Data - for j, tx := range txs { - if to := tx.To(); to != nil && *to == dsCfg.batchInboxAddress { - seqDataSubmitter, err := dsCfg.l1Signer.Sender(tx) // optimization: only derive sender if To is correct - if err != nil { - log.Warn("tx in inbox with invalid signature", "index", j, "txHash", tx.Hash(), "err", err) - continue // bad signature, ignore - } - // some random L1 user might have sent a transaction to our batch inbox, ignore them - if seqDataSubmitter != batcherAddr { - log.Warn("tx in inbox with unauthorized submitter", "index", j, "txHash", tx.Hash(), "err", err) - continue // not an authorized batch submitter, ignore - } - out = append(out, tx.Data()) - } - } - return out -} diff --git a/op-node/rollup/derive/data_source.go b/op-node/rollup/derive/data_source.go new file mode 100644 index 000000000000..67e198922e13 --- /dev/null +++ b/op-node/rollup/derive/data_source.go @@ -0,0 +1,59 @@ +package derive + +import ( + "context" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/log" + + "github.com/ethereum-optimism/optimism/op-node/rollup" + "github.com/ethereum-optimism/optimism/op-service/eth" +) + +type DataIter interface { + Next(ctx context.Context) (eth.Data, error) +} + +type L1TransactionFetcher interface { + InfoAndTxsByHash(ctx context.Context, hash common.Hash) (eth.BlockInfo, types.Transactions, error) +} + +type L1BlobsFetcher interface { + // BlobsByRefAndIndexedDataHashes fetches blobs that were confirmed in the given L1 block with the given indexed hashes. + BlobsByRefAndIndexedDataHashes(ctx context.Context, ref eth.L1BlockRef, dataHashes []eth.IndexedDataHash) ([]*eth.Blob, error) +} + +// DataSourceFactory reads raw transactions from a given block & then filters for +// batch submitter transactions. +// This is not a stage in the pipeline, but a wrapper for another stage in the pipeline +type DataSourceFactory struct { + log log.Logger + dsCfg DataSourceConfig + fetcher L1TransactionFetcher + blobsFetcher L1BlobsFetcher +} + +func NewDataSourceFactory(log log.Logger, cfg *rollup.Config, fetcher L1TransactionFetcher, blobsFetcher L1BlobsFetcher) *DataSourceFactory { + config := DataSourceConfig{ + l1Signer: cfg.L1Signer(), + batchInboxAddress: cfg.BatchInboxAddress, + blobsEnabledL1Timestamp: cfg.BlobsEnabledL1Timestamp, + } + return &DataSourceFactory{log: log, dsCfg: config, fetcher: fetcher, blobsFetcher: blobsFetcher} +} + +// OpenData returns the appropriate data source for the L1 block `ref`. +func (ds *DataSourceFactory) OpenData(ctx context.Context, ref eth.L1BlockRef, batcherAddr common.Address) DataIter { + if n := ds.dsCfg.blobsEnabledL1Timestamp; n != nil && *n <= ref.Time { + return NewBlobDataSource(ctx, ds.log, ds.dsCfg, ds.fetcher, ds.blobsFetcher, ref, batcherAddr) + } + return NewCallDataSource(ctx, ds.log, ds.dsCfg, ds.fetcher, ref, batcherAddr) +} + +// DataSourceConfig regroups the mandatory rollup.Config fields needed for DataFromEVMTransactions. +type DataSourceConfig struct { + l1Signer types.Signer + batchInboxAddress common.Address + blobsEnabledL1Timestamp *uint64 +} diff --git a/op-node/rollup/derive/calldata_source_test.go b/op-node/rollup/derive/data_source_test.go similarity index 95% rename from op-node/rollup/derive/calldata_source_test.go rename to op-node/rollup/derive/data_source_test.go index 213052b36e16..04ead1d8d8ac 100644 --- a/op-node/rollup/derive/calldata_source_test.go +++ b/op-node/rollup/derive/data_source_test.go @@ -120,8 +120,12 @@ func TestDataFromEVMTransactions(t *testing.T) { expectedData = append(expectedData, txs[i].Data()) } } - - out := DataFromEVMTransactions(DataSourceConfig{cfg.L1Signer(), cfg.BatchInboxAddress}, batcherAddr, txs, testlog.Logger(t, log.LvlCrit)) + config := DataSourceConfig{ + cfg.L1Signer(), + cfg.BatchInboxAddress, + nil, + } + out := CallDataFromEVMTransactions(config, batcherAddr, txs, testlog.Logger(t, log.LvlCrit)) require.ElementsMatch(t, expectedData, out) } diff --git a/op-node/rollup/derive/l1_retrieval.go b/op-node/rollup/derive/l1_retrieval.go index 586e125acc32..c2fba316aa1a 100644 --- a/op-node/rollup/derive/l1_retrieval.go +++ b/op-node/rollup/derive/l1_retrieval.go @@ -11,7 +11,7 @@ import ( ) type DataAvailabilitySource interface { - OpenData(ctx context.Context, id eth.BlockID, batcherAddr common.Address) DataIter + OpenData(ctx context.Context, ref eth.L1BlockRef, batcherAddr common.Address) DataIter } type NextBlockProvider interface { @@ -53,7 +53,7 @@ func (l1r *L1Retrieval) NextData(ctx context.Context) ([]byte, error) { } else if err != nil { return nil, err } - l1r.datas = l1r.dataSrc.OpenData(ctx, next.ID(), l1r.prev.SystemConfig().BatcherAddr) + l1r.datas = l1r.dataSrc.OpenData(ctx, next, l1r.prev.SystemConfig().BatcherAddr) } l1r.log.Debug("fetching next piece of data") @@ -73,7 +73,7 @@ func (l1r *L1Retrieval) NextData(ctx context.Context) ([]byte, error) { // Note that we open up the `l1r.datas` here because it is requires to maintain the // internal invariants that later propagate up the derivation pipeline. func (l1r *L1Retrieval) Reset(ctx context.Context, base eth.L1BlockRef, sysCfg eth.SystemConfig) error { - l1r.datas = l1r.dataSrc.OpenData(ctx, base.ID(), sysCfg.BatcherAddr) + l1r.datas = l1r.dataSrc.OpenData(ctx, base, sysCfg.BatcherAddr) l1r.log.Info("Reset of L1Retrieval done", "origin", base) return io.EOF } diff --git a/op-node/rollup/derive/l1_retrieval_test.go b/op-node/rollup/derive/l1_retrieval_test.go index e27cadbceb7f..16c45a596624 100644 --- a/op-node/rollup/derive/l1_retrieval_test.go +++ b/op-node/rollup/derive/l1_retrieval_test.go @@ -35,13 +35,13 @@ type MockDataSource struct { mock.Mock } -func (m *MockDataSource) OpenData(ctx context.Context, id eth.BlockID, batcherAddr common.Address) DataIter { - out := m.Mock.MethodCalled("OpenData", id, batcherAddr) +func (m *MockDataSource) OpenData(ctx context.Context, ref eth.L1BlockRef, batcherAddr common.Address) DataIter { + out := m.Mock.MethodCalled("OpenData", ref, batcherAddr) return out[0].(DataIter) } -func (m *MockDataSource) ExpectOpenData(id eth.BlockID, iter DataIter, batcherAddr common.Address) { - m.Mock.On("OpenData", id, batcherAddr).Return(iter) +func (m *MockDataSource) ExpectOpenData(ref eth.L1BlockRef, batcherAddr common.Address, iter DataIter) { + m.Mock.On("OpenData", ref, batcherAddr).Return(iter) } var _ DataAvailabilitySource = (*MockDataSource)(nil) @@ -79,8 +79,7 @@ func (m *MockL1Traversal) ExpectNextL1Block(block eth.L1BlockRef, err error) { var _ NextBlockProvider = (*MockL1Traversal)(nil) -// TestL1RetrievalReset tests the reset. The reset just opens up a new -// data for the specified block. +// TestL1RetrievalReset tests the reset. The reset prepares for retrieval, but does not retrieve data yet. func TestL1RetrievalReset(t *testing.T) { rng := rand.New(rand.NewSource(1234)) dataSrc := &MockDataSource{} @@ -89,7 +88,7 @@ func TestL1RetrievalReset(t *testing.T) { BatcherAddr: common.Address{42}, } - dataSrc.ExpectOpenData(a.ID(), &fakeDataIter{}, l1Cfg.BatcherAddr) + dataSrc.ExpectOpenData(a, l1Cfg.BatcherAddr, &fakeDataIter{}) defer dataSrc.AssertExpectations(t) l1r := NewL1Retrieval(testlog.Logger(t, log.LvlError), dataSrc, nil) @@ -147,7 +146,7 @@ func TestL1RetrievalNextData(t *testing.T) { l1t := &MockL1Traversal{} l1t.ExpectNextL1Block(test.prevBlock, test.prevErr) dataSrc := &MockDataSource{} - dataSrc.ExpectOpenData(test.prevBlock.ID(), &fakeDataIter{data: test.datas, errs: test.datasErrs}, test.sysCfg.BatcherAddr) + dataSrc.ExpectOpenData(test.prevBlock, test.sysCfg.BatcherAddr, &fakeDataIter{data: test.datas, errs: test.datasErrs}) ret := NewL1Retrieval(testlog.Logger(t, log.LvlCrit), dataSrc, l1t) diff --git a/op-node/rollup/derive/pipeline.go b/op-node/rollup/derive/pipeline.go index bafc67867ada..4cca26b9704d 100644 --- a/op-node/rollup/derive/pipeline.go +++ b/op-node/rollup/derive/pipeline.go @@ -83,11 +83,11 @@ type DerivationPipeline struct { } // NewDerivationPipeline creates a derivation pipeline, which should be reset before use. -func NewDerivationPipeline(log log.Logger, cfg *rollup.Config, l1Fetcher L1Fetcher, engine Engine, metrics Metrics, syncCfg *sync.Config) *DerivationPipeline { +func NewDerivationPipeline(log log.Logger, cfg *rollup.Config, l1Fetcher L1Fetcher, l1BlobsFetcher L1BlobsFetcher, engine Engine, metrics Metrics, syncCfg *sync.Config) *DerivationPipeline { // Pull stages l1Traversal := NewL1Traversal(log, cfg, l1Fetcher) - dataSrc := NewDataSourceFactory(log, cfg, l1Fetcher) // auxiliary stage for L1Retrieval + dataSrc := NewDataSourceFactory(log, cfg, l1Fetcher, l1BlobsFetcher) // auxiliary stage for L1Retrieval l1Src := NewL1Retrieval(log, dataSrc, l1Traversal) frameQueue := NewFrameQueue(log, l1Src) bank := NewChannelBank(log, cfg, frameQueue, l1Fetcher, metrics) diff --git a/op-node/rollup/driver/driver.go b/op-node/rollup/driver/driver.go index 67c952a99633..34ac25fe25f9 100644 --- a/op-node/rollup/driver/driver.go +++ b/op-node/rollup/driver/driver.go @@ -117,13 +117,13 @@ type SequencerStateListener interface { } // NewDriver composes an events handler that tracks L1 state, triggers L2 derivation, and optionally sequences new L2 blocks. -func NewDriver(driverCfg *Config, cfg *rollup.Config, l2 L2Chain, l1 L1Chain, altSync AltSync, network Network, log log.Logger, snapshotLog log.Logger, metrics Metrics, sequencerStateListener SequencerStateListener, syncCfg *sync.Config) *Driver { +func NewDriver(driverCfg *Config, cfg *rollup.Config, l2 L2Chain, l1 L1Chain, l1Blobs derive.L1BlobsFetcher, altSync AltSync, network Network, log log.Logger, snapshotLog log.Logger, metrics Metrics, sequencerStateListener SequencerStateListener, syncCfg *sync.Config) *Driver { l1 = NewMeteredL1Fetcher(l1, metrics) l1State := NewL1State(log, metrics) sequencerConfDepth := NewConfDepth(driverCfg.SequencerConfDepth, l1State.L1Head, l1) findL1Origin := NewL1OriginSelector(log, cfg, sequencerConfDepth) verifConfDepth := NewConfDepth(driverCfg.VerifierConfDepth, l1State.L1Head, l1) - derivationPipeline := derive.NewDerivationPipeline(log, cfg, verifConfDepth, l2, metrics, syncCfg) + derivationPipeline := derive.NewDerivationPipeline(log, cfg, verifConfDepth, l1Blobs, l2, metrics, syncCfg) attrBuilder := derive.NewFetchingAttributesBuilder(cfg, l1, l2) engine := derivationPipeline meteredEngine := NewMeteredEngine(cfg, engine, metrics, log) diff --git a/op-node/rollup/types.go b/op-node/rollup/types.go index 3ac192e83cbb..d18e25e0b11b 100644 --- a/op-node/rollup/types.go +++ b/op-node/rollup/types.go @@ -107,6 +107,9 @@ type Config struct { // L1 address that declares the protocol versions, optional (Beta feature) ProtocolVersionsAddress common.Address `json:"protocol_versions_address,omitempty"` + + // L1 block timestamp to start reading blobs as batch data-source. Optional. + BlobsEnabledL1Timestamp *uint64 `json:"blobs_data,omitempty"` } // ValidateL1Config checks L1 config variables for errors. @@ -275,7 +278,7 @@ func (cfg *Config) Check() error { } func (c *Config) L1Signer() types.Signer { - return types.NewLondonSigner(c.L1ChainID) + return types.NewCancunSigner(c.L1ChainID) } // IsRegolith returns true if the Regolith hardfork is active at or past the given timestamp. diff --git a/op-node/service.go b/op-node/service.go index 31987bfe1e16..f9151530c67f 100644 --- a/op-node/service.go +++ b/op-node/service.go @@ -72,10 +72,15 @@ func NewConfig(ctx *cli.Context, log log.Logger) (*node.Config, error) { if haltOption == "none" { haltOption = "" } + beaconEndpoint, err := NewL1BeaconEndpointConfig(ctx) + if err != nil { + return nil, fmt.Errorf("failed to load beacon endpoints info: %w", err) + } cfg := &node.Config{ L1: l1Endpoint, L2: l2Endpoint, + Beacon: beaconEndpoint, Rollup: *rollupConfig, Driver: *driverConfig, RPC: node.RPCConfig{ @@ -160,6 +165,12 @@ func NewL2EndpointConfig(ctx *cli.Context, log log.Logger) (*node.L2EndpointConf }, nil } +func NewL1BeaconEndpointConfig(ctx *cli.Context) (*node.L1BeaconEndpointConfig, error) { + return &node.L1BeaconEndpointConfig{ + BeaconAddr: ctx.String(flags.BeaconAddr.Name), + }, nil +} + func NewConfigPersistence(ctx *cli.Context) node.ConfigPersistence { stateFile := ctx.String(flags.RPCAdminPersistence.Name) if stateFile == "" { diff --git a/op-program/client/driver/driver.go b/op-program/client/driver/driver.go index 5487d8f19fd5..298587b407f6 100644 --- a/op-program/client/driver/driver.go +++ b/op-program/client/driver/driver.go @@ -36,7 +36,7 @@ type Driver struct { } func NewDriver(logger log.Logger, cfg *rollup.Config, l1Source derive.L1Fetcher, l2Source L2Source, targetBlockNum uint64) *Driver { - pipeline := derive.NewDerivationPipeline(logger, cfg, l1Source, l2Source, metrics.NoopMetrics, &sync.Config{}) + pipeline := derive.NewDerivationPipeline(logger, cfg, l1Source, nil, l2Source, metrics.NoopMetrics, &sync.Config{}) pipeline.Reset() return &Driver{ logger: logger, diff --git a/op-service/client/http.go b/op-service/client/http.go new file mode 100644 index 000000000000..a8abe6026f70 --- /dev/null +++ b/op-service/client/http.go @@ -0,0 +1,39 @@ +package client + +import ( + "context" + "fmt" + "net/http" + "time" + + "github.com/ethereum/go-ethereum/log" +) + +type HTTP interface { + Get(ctx context.Context, path string, headers http.Header) (*http.Response, error) +} + +type BasicHTTPClient struct { + endpoint string + log log.Logger + client http.Client +} + +func NewBasicHTTPClient(endpoint string, log log.Logger) *BasicHTTPClient { + return &BasicHTTPClient{endpoint: endpoint, log: log, client: http.Client{Timeout: 30 * time.Second}} +} + +func (cl *BasicHTTPClient) Get(ctx context.Context, p string, headers http.Header) (*http.Response, error) { + req, err := http.NewRequestWithContext(ctx, http.MethodGet, cl.endpoint+"/"+p, nil) + if err != nil { + return nil, fmt.Errorf("failed to construct request: %w", err) + } + for k, values := range headers { + for _, v := range values { + req.Header.Add(k, v) + } + } + return cl.client.Do(req) +} + +// TODO HTTP client wrapper to track response time and error metrics diff --git a/op-service/eth/blob.go b/op-service/eth/blob.go index b1426172077c..64c724e2bd61 100644 --- a/op-service/eth/blob.go +++ b/op-service/eth/blob.go @@ -2,6 +2,7 @@ package eth import ( "crypto/sha256" + "encoding/binary" "fmt" "reflect" @@ -66,3 +67,67 @@ func KZGToVersionedHash(commitment kzg4844.Commitment) (out common.Hash) { func VerifyBlobProof(blob *Blob, commitment kzg4844.Commitment, proof kzg4844.Proof) error { return kzg4844.VerifyBlobProof(*blob.KZGBlob(), commitment, proof) } + +// FromData encodes the given input data into this blob. The encoding scheme is as follows: +// +// First, field elements are encoded as big-endian uint256 in BLS modulus range. To avoid modulus +// overflow, we can't use the full 32 bytes, so we write data only to the topmost 31 bytes of each. +// TODO: we can optimize this to get a bit more data from the blobs by using the top byte +// partially. +// +// The first field element encodes the length of input data as a little endian uint32 in its +// topmost 4 (out of 31) bytes, and the first 27 bytes of the input data in its remaining 27 +// bytes. +// +// The remaining field elements each encode 31 bytes of the remaining input data, up until the end +// of the input. +// +// TODO: version the encoding format to allow for future encoding changes +func (b *Blob) FromData(data Data) error { + if len(data) > MaxBlobDataSize { + return fmt.Errorf("data is too large for blob. len=%v", len(data)) + } + b.Clear() + // encode 4-byte little-endian length value into topmost 4 bytes (out of 31) of first field + // element + binary.LittleEndian.PutUint32(b[1:5], uint32(len(data))) + // encode first 27 bytes of input data into remaining bytes of first field element + offset := copy(b[5:32], data) + // encode (up to) 31 bytes of remaining input data at a time into the subsequent field element + for i := 1; i < 4096; i++ { + offset += copy(b[i*32+1:i*32+32], data[offset:]) + if offset == len(data) { + break + } + } + if offset < len(data) { + return fmt.Errorf("failed to fit all data into blob. bytes remaining: %v", len(data)-offset) + } + return nil +} + +// ToData decodes the blob into raw byte data. See FromData above for details on the encoding +// format. +func (b *Blob) ToData() (Data, error) { + data := make(Data, 4096*32) + for i := 0; i < 4096; i++ { + if b[i*32] != 0 { + return nil, fmt.Errorf("invalid blob, found non-zero high order byte %x of field element %d", b[i*32], i) + } + copy(data[i*31:i*31+31], b[i*32+1:i*32+32]) + } + // extract the length prefix & trim the output accordingly + dataLen := binary.LittleEndian.Uint32(data[:4]) + data = data[4:] + if dataLen > uint32(len(data)) { + return nil, fmt.Errorf("invalid blob, length prefix out of range: %d", dataLen) + } + data = data[:dataLen] + return data, nil +} + +func (b *Blob) Clear() { + for i := 0; i < BlobSize; i++ { + b[i] = 0 + } +} diff --git a/op-service/eth/blob_test.go b/op-service/eth/blob_test.go new file mode 100644 index 000000000000..5f4fcd7fd64d --- /dev/null +++ b/op-service/eth/blob_test.go @@ -0,0 +1,78 @@ +package eth + +import ( + "testing" +) + +func TestBlobEncodeDecode(t *testing.T) { + cases := []string{ + "this is a test of blob encoding/decoding", + "short", + "\x00", + "\x00\x01\x00", + "\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00", + "\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00", + "\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00", + "", + } + + var b Blob + for _, c := range cases { + data := Data(c) + if err := b.FromData(data); err != nil { + t.Fatalf("failed to encode bytes: %v", err) + } + decoded, err := b.ToData() + if err != nil { + t.Fatalf("failed to decode blob: %v", err) + } + if string(decoded) != c { + t.Errorf("decoded != input. got: %v, want: %v", decoded, Data(c)) + } + } +} + +func TestBigBlobEncoding(t *testing.T) { + bigData := Data(make([]byte, MaxBlobDataSize)) + bigData[MaxBlobDataSize-1] = 0xFF + var b Blob + if err := b.FromData(bigData); err != nil { + t.Fatalf("failed to encode bytes: %v", err) + } + decoded, err := b.ToData() + if err != nil { + t.Fatalf("failed to decode blob: %v", err) + } + if string(decoded) != string(bigData) { + t.Errorf("decoded blob != big blob input") + } +} + +func TestInvalidBlobDecoding(t *testing.T) { + data := Data("this is a test of invalid blob decoding") + var b Blob + if err := b.FromData(data); err != nil { + t.Fatalf("failed to encode bytes: %v", err) + } + b[32] = 0x80 // field elements should never have their highest order bit set + if _, err := b.ToData(); err == nil { + t.Errorf("expected error, got none") + } + + b[32] = 0x00 + b[4] = 0xFF // encode an invalid (much too long) length prefix + if _, err := b.ToData(); err == nil { + t.Errorf("expected error, got none") + } +} + +func TestTooLongDataEncoding(t *testing.T) { + // should never be able to encode data that has size the same as that of the blob due to < 256 + // bit precision of each field element + data := Data(make([]byte, BlobSize)) + var b Blob + err := b.FromData(data) + if err == nil { + t.Errorf("expected error, got none") + } +} diff --git a/op-service/sources/l1_beacon_client.go b/op-service/sources/l1_beacon_client.go new file mode 100644 index 000000000000..75a6149108b3 --- /dev/null +++ b/op-service/sources/l1_beacon_client.go @@ -0,0 +1,135 @@ +package sources + +import ( + "context" + "encoding/json" + "fmt" + "io" + "net/http" + "strconv" + "strings" + "sync" + + "github.com/ethereum/go-ethereum/crypto/kzg4844" + + "github.com/ethereum-optimism/optimism/op-service/client" + "github.com/ethereum-optimism/optimism/op-service/eth" +) + +type L1BeaconClient struct { + cl client.HTTP + + initLock sync.Mutex + timeToSlotFn TimeToSlotFn +} + +func NewL1BeaconClient(cl client.HTTP) *L1BeaconClient { + return &L1BeaconClient{cl: cl} +} + +type TimeToSlotFn func(timestamp uint64) uint64 + +func (cl *L1BeaconClient) apiReq(ctx context.Context, dest any, method string) error { + headers := http.Header{} + headers.Add("Accept", "application/json") + resp, err := cl.cl.Get(ctx, method, headers) + if err != nil { + return fmt.Errorf("failed genesis details request: %w", err) + } + if resp.StatusCode != http.StatusOK { + errMsg, _ := io.ReadAll(resp.Body) + _ = resp.Body.Close() + return fmt.Errorf("failed request with status %d: %s", resp.StatusCode, string(errMsg)) + } + if err := json.NewDecoder(resp.Body).Decode(dest); err != nil { + _ = resp.Body.Close() + return err + } + if err := resp.Body.Close(); err != nil { + return fmt.Errorf("failed to close genesis response object: %w", err) + } + return nil +} + +func (cl *L1BeaconClient) GetTimeToSlotFn(ctx context.Context) (TimeToSlotFn, error) { + cl.initLock.Lock() + defer cl.initLock.Unlock() + if cl.timeToSlotFn != nil { + return cl.timeToSlotFn, nil + } + + var genesisResp eth.APIGenesisResponse + if err := cl.apiReq(ctx, &genesisResp, "eth/v1/beacon/genesis"); err != nil { + return nil, err + } + + var configResp eth.APIConfigResponse + if err := cl.apiReq(ctx, &configResp, "eth/v1/config/spec"); err != nil { + return nil, err + } + + cl.timeToSlotFn = func(timestamp uint64) uint64 { + return (timestamp - uint64(genesisResp.Data.GenesisTime)) / uint64(configResp.Data.SecondsPerSlot) + } + return cl.timeToSlotFn, nil +} + +// BlobsByRefAndIndexedDataHashes fetches blobs that were confirmed in the given L1 block with the +// given indexed hashes. The order of the returned blobs will match the order of `dataHashes`. +func (cl *L1BeaconClient) BlobsByRefAndIndexedDataHashes(ctx context.Context, ref eth.L1BlockRef, dataHashes []eth.IndexedDataHash) ([]*eth.Blob, error) { + slotFn, err := cl.GetTimeToSlotFn(ctx) + if err != nil { + return nil, fmt.Errorf("failed to get time to slot function: %w", err) + } + slot := slotFn(ref.Time) + + builder := strings.Builder{} + builder.WriteString("eth/v1/beacon/blob_sidecars/") + builder.WriteString(strconv.FormatUint(slot, 10)) + for i := range dataHashes { + if i == 0 { + builder.WriteString("?indices=") + } else { + builder.WriteString("&indices=") + } + builder.WriteString(strconv.FormatUint(dataHashes[i].Index, 10)) + } + + var resp eth.APIGetBlobSidecarsResponse + if err := cl.apiReq(ctx, &resp, builder.String()); err != nil { + return nil, fmt.Errorf("failed to fetch blob sidecars of slot %d (for block %s): %w", slot, ref, err) + } + if len(dataHashes) != len(resp.Data) { + return nil, fmt.Errorf("expected %v sidecars but got %v", len(dataHashes), len(resp.Data)) + } + + out := make([]*eth.Blob, len(dataHashes)) + for i, ih := range dataHashes { + // The beacon node api makes no guarantees on order of the returned blob sidecars, so + // search for the sidecar that matches the current indexed hash to ensure blobs are + // returned in the same order. + var sidecar *eth.BlobSidecar + for _, sc := range resp.Data { + if uint64(sc.Index) == ih.Index { + sidecar = sc + break + } + } + if sidecar == nil { + return nil, fmt.Errorf("no blob in response matches desired index: %v", ih.Index) + } + + // make sure the blob's kzg commitment hashes to the expected value + dataHash := eth.KZGToVersionedHash(kzg4844.Commitment(sidecar.KZGCommitment)) + if dataHash != ih.DataHash { + return nil, fmt.Errorf("expected datahash %s for blob at index %d in block %s but got %s", ih.DataHash, ih.Index, ref, dataHash) + } + + // confirm blob data is valid by verifying its proof against the commitment + if err := eth.VerifyBlobProof(&sidecar.Blob, kzg4844.Commitment(sidecar.KZGCommitment), kzg4844.Proof(sidecar.KZGProof)); err != nil { + return nil, fmt.Errorf("blob at index %v failed verification: %w", i, err) + } + out[i] = &sidecar.Blob + } + return out, nil +} diff --git a/op-service/txmgr/price_bump_test.go b/op-service/txmgr/price_bump_test.go index 5b32b12d0032..eb63a0f573f3 100644 --- a/op-service/txmgr/price_bump_test.go +++ b/op-service/txmgr/price_bump_test.go @@ -25,7 +25,7 @@ func (tc *priceBumpTest) run(t *testing.T) { prevFC := calcGasFeeCap(big.NewInt(tc.prevBasefee), big.NewInt(tc.prevGasTip)) lgr := testlog.Logger(t, log.LvlCrit) - tip, fc := updateFees(big.NewInt(tc.prevGasTip), prevFC, big.NewInt(tc.newGasTip), big.NewInt(tc.newBasefee), lgr) + tip, fc := updateFees(big.NewInt(tc.prevGasTip), prevFC, big.NewInt(tc.newGasTip), big.NewInt(tc.newBasefee), false, lgr) require.Equal(t, tc.expectedTip, tip.Int64(), "tip must be as expected") require.Equal(t, tc.expectedFC, fc.Int64(), "fee cap must be as expected") diff --git a/op-service/txmgr/queue_test.go b/op-service/txmgr/queue_test.go index 0f08e8199095..2823df4de3c0 100644 --- a/op-service/txmgr/queue_test.go +++ b/op-service/txmgr/queue_test.go @@ -195,7 +195,7 @@ func TestSend(t *testing.T) { return core.ErrNonceTooLow } txHash := tx.Hash() - backend.mine(&txHash, tx.GasFeeCap()) + backend.mine(&txHash, tx.GasFeeCap(), nil) return nil } backend.setTxSender(sendTx) diff --git a/op-service/txmgr/txmgr.go b/op-service/txmgr/txmgr.go index e77a43923db7..4e7c9f01def9 100644 --- a/op-service/txmgr/txmgr.go +++ b/op-service/txmgr/txmgr.go @@ -12,24 +12,31 @@ import ( "github.com/ethereum/go-ethereum" "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/consensus/misc/eip4844" "github.com/ethereum/go-ethereum/core" "github.com/ethereum/go-ethereum/core/txpool" "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/crypto/kzg4844" "github.com/ethereum/go-ethereum/log" + "github.com/holiman/uint256" + "github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-service/retry" "github.com/ethereum-optimism/optimism/op-service/txmgr/metrics" ) const ( - // Geth requires a minimum fee bump of 10% for tx resubmission - priceBump int64 = 10 + // Geth requires a minimum fee bump of 10% for regular tx resubmission, 100% for blob txs + priceBump int64 = 10 + blobPriceBump int64 = 100 ) // new = old * (100 + priceBump) / 100 var ( - priceBumpPercent = big.NewInt(100 + priceBump) - oneHundred = big.NewInt(100) + priceBumpPercent = big.NewInt(100 + priceBump) + oneHundred = big.NewInt(100) + blobPriceBumpPercent = big.NewInt(100 + blobPriceBump) + two = big.NewInt(2) ) // TxManager is an interface that allows callers to reliably publish txs, @@ -145,8 +152,10 @@ func (m *SimpleTxManager) Close() { // TxCandidate is a transaction candidate that can be submitted to ask the // [TxManager] to construct a transaction with gas price bounds. type TxCandidate struct { - // TxData is the transaction data to be used in the constructed tx. + // TxData is the transaction calldata to be used in the constructed tx. TxData []byte + // Blobs to send along in the tx (optional). + Blobs []*eth.Blob // To is the recipient of the constructed tx. Nil means contract creation. To *common.Address // GasLimit is the gas limit to be used in the constructed tx. @@ -202,44 +211,94 @@ func (m *SimpleTxManager) send(ctx context.Context, candidate TxCandidate) (*typ // NOTE: If the [TxCandidate.GasLimit] is non-zero, it will be used as the transaction's gas. // NOTE: Otherwise, the [SimpleTxManager] will query the specified backend for an estimate. func (m *SimpleTxManager) craftTx(ctx context.Context, candidate TxCandidate) (*types.Transaction, error) { - gasTipCap, basefee, err := m.suggestGasPriceCaps(ctx) + log := m.l.New("to", candidate.To, "from", m.cfg.From, "blobs", len(candidate.Blobs)) + log.Info("Creating tx") + + gasTipCap, basefee, blobFee, err := m.suggestGasPriceCaps(ctx) if err != nil { m.metr.RPCError() return nil, fmt.Errorf("failed to get gas price info: %w", err) } gasFeeCap := calcGasFeeCap(basefee, gasTipCap) - rawTx := &types.DynamicFeeTx{ - ChainID: m.chainID, - To: candidate.To, - GasTipCap: gasTipCap, - GasFeeCap: gasFeeCap, - Data: candidate.TxData, - Value: candidate.Value, - } - - m.l.Info("Creating tx", "to", rawTx.To, "from", m.cfg.From) + gasLimit := candidate.GasLimit // If the gas limit is set, we can use that as the gas - if candidate.GasLimit != 0 { - rawTx.Gas = candidate.GasLimit - } else { + if gasLimit == 0 { // Calculate the intrinsic gas for the transaction gas, err := m.backend.EstimateGas(ctx, ethereum.CallMsg{ From: m.cfg.From, To: candidate.To, - GasTipCap: gasTipCap, GasFeeCap: gasFeeCap, - Data: rawTx.Data, - Value: rawTx.Value, + Data: candidate.TxData, + Value: candidate.Value, }) if err != nil { return nil, fmt.Errorf("failed to estimate gas: %w", err) } - rawTx.Gas = gas + gasLimit = gas + log.Info("estimated gas", "gas", gasLimit) + } + + var sidecar *types.BlobTxSidecar + var blobHashes []common.Hash + if len(candidate.Blobs) > 0 { + if candidate.To == nil { + return nil, errors.New("blob txs cannot deploy contracts") + } + blobHashes = make([]common.Hash, 0, len(candidate.Blobs)) + sidecar = &types.BlobTxSidecar{} + for i, blob := range candidate.Blobs { + rawBlob := *blob.KZGBlob() + sidecar.Blobs = append(sidecar.Blobs, rawBlob) + commitment, err := kzg4844.BlobToCommitment(rawBlob) + if err != nil { + return nil, fmt.Errorf("cannot compute KZG commitment of blob %d in tx candidate: %w", i, err) + } + sidecar.Commitments = append(sidecar.Commitments, commitment) + proof, err := kzg4844.ComputeBlobProof(rawBlob, commitment) + if err != nil { + return nil, fmt.Errorf("cannot compute KZG proof for fast commitment verification of blob %d in tx candidate: %w", i, err) + } + sidecar.Proofs = append(sidecar.Proofs, proof) + blobHashes = append(blobHashes, eth.KZGToVersionedHash(commitment)) + } + log.Info("crafted blob tx sidecar") } - return m.signWithNextNonce(ctx, rawTx) + var txMessage types.TxData + if sidecar != nil { + if blobFee == nil { + return nil, fmt.Errorf("expected non-nil blobFee") + } + blobFeeCap := calcBlobFeeCap(blobFee) + txMessage = &types.BlobTx{ + ChainID: uint256.MustFromBig(m.chainID), + Nonce: 0, // set by signer + To: *candidate.To, + GasTipCap: uint256.MustFromBig(gasTipCap), + GasFeeCap: uint256.MustFromBig(gasFeeCap), + Value: uint256.MustFromBig(candidate.Value), + Data: candidate.TxData, + Gas: gasLimit, + BlobFeeCap: uint256.MustFromBig(blobFeeCap), + BlobHashes: blobHashes, + Sidecar: sidecar, + } + } else { + txMessage = &types.DynamicFeeTx{ + ChainID: m.chainID, + Nonce: 0, // set by signer + To: candidate.To, + GasTipCap: gasTipCap, + GasFeeCap: gasFeeCap, + Value: candidate.Value, + Data: candidate.TxData, + Gas: gasLimit, + } + } + + return m.signWithNextNonce(ctx, txMessage) } // signWithNextNonce returns a signed transaction with the next available nonce. @@ -247,7 +306,7 @@ func (m *SimpleTxManager) craftTx(ctx context.Context, candidate TxCandidate) (* // then subsequent calls simply increment this number. If the transaction manager // is reset, it will query the eth_getTransactionCount nonce again. If signing // fails, the nonce is not incremented. -func (m *SimpleTxManager) signWithNextNonce(ctx context.Context, rawTx *types.DynamicFeeTx) (*types.Transaction, error) { +func (m *SimpleTxManager) signWithNextNonce(ctx context.Context, txMessage types.TxData) (*types.Transaction, error) { m.nonceLock.Lock() defer m.nonceLock.Unlock() @@ -265,10 +324,17 @@ func (m *SimpleTxManager) signWithNextNonce(ctx context.Context, rawTx *types.Dy *m.nonce++ } - rawTx.Nonce = *m.nonce + switch x := txMessage.(type) { + case *types.DynamicFeeTx: + x.Nonce = *m.nonce + case *types.BlobTx: + x.Nonce = *m.nonce + default: + return nil, fmt.Errorf("unrecognized tx type: %T", x) + } ctx, cancel := context.WithTimeout(ctx, m.cfg.NetworkTimeout) defer cancel() - tx, err := m.cfg.Signer(ctx, m.cfg.From, types.NewTx(rawTx)) + tx, err := m.cfg.Signer(ctx, m.cfg.From, types.NewTx(txMessage)) if err != nil { // decrement the nonce, so we can retry signing with the same nonce next time // signWithNextNonce is called @@ -347,7 +413,11 @@ func (m *SimpleTxManager) sendTx(ctx context.Context, tx *types.Transaction) (*t // Returns the latest fee bumped tx, and a boolean indicating whether the tx was sent or not func (m *SimpleTxManager) publishTx(ctx context.Context, tx *types.Transaction, sendState *SendState, bumpFeesImmediately bool) (*types.Transaction, bool) { updateLogFields := func(tx *types.Transaction) log.Logger { - return m.l.New("hash", tx.Hash(), "nonce", tx.Nonce(), "gasTipCap", tx.GasTipCap(), "gasFeeCap", tx.GasFeeCap()) + logger := m.l.New("hash", tx.Hash(), "nonce", tx.Nonce(), "gasTipCap", tx.GasTipCap(), "gasFeeCap", tx.GasFeeCap()) + if sidecar := tx.BlobTxSidecar(); sidecar != nil { + logger = logger.New("blobs", len(sidecar.Blobs)) + } + return logger } l := updateLogFields(tx) @@ -499,19 +569,18 @@ func (m *SimpleTxManager) queryReceipt(ctx context.Context, txHash common.Hash, return nil } -// increaseGasPrice takes the previous transaction, clones it, and returns it with fee values that -// are at least `priceBump` percent higher than the previous ones to satisfy Geth's replacement -// rules, and no lower than the values returned by the fee suggestion algorithm to ensure it -// doesn't linger in the mempool. Finally to avoid runaway price increases, fees are capped at a -// `feeLimitMultiplier` multiple of the suggested values. +// increaseGasPrice returns a new transaction that is equivalent to the input transaction but with +// higher fees that should satisfy geth's tx replacement rules. It also computes an updated gas +// limit estimate. To avoid runaway price increases, fees are capped at a `feeLimitMultiplier` +// multiple of the suggested values. func (m *SimpleTxManager) increaseGasPrice(ctx context.Context, tx *types.Transaction) (*types.Transaction, error) { m.l.Info("bumping gas price for tx", "hash", tx.Hash(), "gasTipCap", tx.GasTipCap(), "gasFeeCap", tx.GasFeeCap(), "gaslimit", tx.Gas()) - tip, basefee, err := m.suggestGasPriceCaps(ctx) + tip, basefee, blobFee, err := m.suggestGasPriceCaps(ctx) if err != nil { m.l.Warn("failed to get suggested gas tip and basefee", "err", err) return nil, err } - bumpedTip, bumpedFee := updateFees(tx.GasTipCap(), tx.GasFeeCap(), tip, basefee, m.l) + bumpedTip, bumpedFee := updateFees(tx.GasTipCap(), tx.GasFeeCap(), tip, basefee, tx.Type() == types.BlobTxType, m.l) // Make sure increase is at most [FeeLimitMultiplier] the suggested values maxTip := new(big.Int).Mul(tip, big.NewInt(int64(m.cfg.FeeLimitMultiplier))) @@ -522,24 +591,14 @@ func (m *SimpleTxManager) increaseGasPrice(ctx context.Context, tx *types.Transa if bumpedFee.Cmp(maxFee) > 0 { return nil, fmt.Errorf("bumped fee cap %v is over %dx multiple of the suggested value", bumpedFee, m.cfg.FeeLimitMultiplier) } - rawTx := &types.DynamicFeeTx{ - ChainID: tx.ChainId(), - Nonce: tx.Nonce(), - GasTipCap: bumpedTip, - GasFeeCap: bumpedFee, - To: tx.To(), - Value: tx.Value(), - Data: tx.Data(), - AccessList: tx.AccessList(), - } - // Re-estimate gaslimit in case things have changed or a previous gaslimit estimate was wrong gas, err := m.backend.EstimateGas(ctx, ethereum.CallMsg{ From: m.cfg.From, - To: rawTx.To, + To: tx.To(), GasTipCap: bumpedTip, GasFeeCap: bumpedFee, - Data: rawTx.Data, + Data: tx.Data(), + Value: tx.Value(), }) if err != nil { // If this is a transaction resubmission, we sometimes see this outcome because the @@ -554,44 +613,91 @@ func (m *SimpleTxManager) increaseGasPrice(ctx context.Context, tx *types.Transa m.l.Info("re-estimated gas differs", "oldgas", tx.Gas(), "newgas", gas, "gasFeeCap", bumpedFee, "gasTipCap", bumpedTip) } - rawTx.Gas = gas + + var newTx *types.Transaction + if tx.Type() == types.BlobTxType { + bumpedBlobFee := calcThresholdValue(tx.BlobGasFeeCap(), true) + if bumpedBlobFee.Cmp(blobFee) < 0 { + bumpedBlobFee = blobFee + } + maxBlobFee := new(big.Int).Mul(calcBlobFeeCap(blobFee), big.NewInt(int64(m.cfg.FeeLimitMultiplier))) + if bumpedBlobFee.Cmp(maxBlobFee) > 0 { + m.l.Warn("bumped blob fee getting capped at multiple of the suggested value", "bumped", bumpedBlobFee, "suggestion", maxBlobFee) + bumpedBlobFee.Set(maxBlobFee) + } + newTx = types.NewTx(&types.BlobTx{ + ChainID: uint256.MustFromBig(tx.ChainId()), + Nonce: tx.Nonce(), + To: *tx.To(), + GasTipCap: uint256.MustFromBig(bumpedTip), + GasFeeCap: uint256.MustFromBig(bumpedFee), + Value: uint256.MustFromBig(tx.Value()), + Data: tx.Data(), + Gas: gas, + BlobFeeCap: uint256.MustFromBig(bumpedBlobFee), + BlobHashes: tx.BlobHashes(), + Sidecar: tx.BlobTxSidecar(), + }) + } else { + newTx = types.NewTx(&types.DynamicFeeTx{ + ChainID: tx.ChainId(), + Nonce: tx.Nonce(), + To: tx.To(), + GasTipCap: bumpedTip, + GasFeeCap: bumpedFee, + Value: tx.Value(), + Data: tx.Data(), + Gas: gas, + }) + } ctx, cancel := context.WithTimeout(ctx, m.cfg.NetworkTimeout) defer cancel() - newTx, err := m.cfg.Signer(ctx, m.cfg.From, types.NewTx(rawTx)) + signedTx, err := m.cfg.Signer(ctx, m.cfg.From, newTx) if err != nil { m.l.Warn("failed to sign new transaction", "err", err) return tx, nil } - return newTx, nil + return signedTx, nil } -// suggestGasPriceCaps suggests what the new tip & new basefee should be based on the current L1 conditions -func (m *SimpleTxManager) suggestGasPriceCaps(ctx context.Context) (*big.Int, *big.Int, error) { +// suggestGasPriceCaps suggests what the new tip, basefee, and blobfee should be based on the +// current L1 conditions. blobfee will be nil if 4844 is not yet active. +func (m *SimpleTxManager) suggestGasPriceCaps(ctx context.Context) (*big.Int, *big.Int, *big.Int, error) { cCtx, cancel := context.WithTimeout(ctx, m.cfg.NetworkTimeout) defer cancel() tip, err := m.backend.SuggestGasTipCap(cCtx) if err != nil { m.metr.RPCError() - return nil, nil, fmt.Errorf("failed to fetch the suggested gas tip cap: %w", err) + return nil, nil, nil, fmt.Errorf("failed to fetch the suggested gas tip cap: %w", err) } else if tip == nil { - return nil, nil, errors.New("the suggested tip was nil") + return nil, nil, nil, errors.New("the suggested tip was nil") } cCtx, cancel = context.WithTimeout(ctx, m.cfg.NetworkTimeout) defer cancel() head, err := m.backend.HeaderByNumber(cCtx, nil) if err != nil { m.metr.RPCError() - return nil, nil, fmt.Errorf("failed to fetch the suggested basefee: %w", err) + return nil, nil, nil, fmt.Errorf("failed to fetch the suggested basefee: %w", err) } else if head.BaseFee == nil { - return nil, nil, errors.New("txmgr does not support pre-london blocks that do not have a basefee") + return nil, nil, nil, errors.New("txmgr does not support pre-london blocks that do not have a basefee") + } + var blobFee *big.Int + if head.ExcessBlobGas != nil { + blobFee = eip4844.CalcBlobFee(*head.ExcessBlobGas) } - return tip, head.BaseFee, nil + return tip, head.BaseFee, blobFee, nil } // calcThresholdValue returns x * priceBumpPercent / 100 -func calcThresholdValue(x *big.Int) *big.Int { - threshold := new(big.Int).Mul(priceBumpPercent, x) +func calcThresholdValue(x *big.Int, isBlobTx bool) *big.Int { + var percent *big.Int + if isBlobTx { + percent = blobPriceBumpPercent + } else { + percent = priceBumpPercent + } + threshold := new(big.Int).Mul(percent, x) threshold = threshold.Div(threshold, oneHundred) return threshold } @@ -599,16 +705,15 @@ func calcThresholdValue(x *big.Int) *big.Int { // updateFees takes an old transaction's tip & fee cap plus a new tip & basefee, and returns // a suggested tip and fee cap such that: // -// (a) each satisfies geth's required tx-replacement fee bumps (we use a 10% increase), and +// (a) each satisfies geth's required tx-replacement fee bumps, and // (b) gasTipCap is no less than new tip, and // (c) gasFeeCap is no less than calcGasFee(newBaseFee, newTip) -func updateFees(oldTip, oldFeeCap, newTip, newBaseFee *big.Int, lgr log.Logger) (*big.Int, *big.Int) { +func updateFees(oldTip, oldFeeCap, newTip, newBaseFee *big.Int, isBlobTx bool, lgr log.Logger) (*big.Int, *big.Int) { newFeeCap := calcGasFeeCap(newBaseFee, newTip) lgr = lgr.New("old_gasTipCap", oldTip, "old_gasFeeCap", oldFeeCap, - "new_gasTipCap", newTip, "new_gasFeeCap", newFeeCap, - "new_basefee", newBaseFee) - thresholdTip := calcThresholdValue(oldTip) - thresholdFeeCap := calcThresholdValue(oldFeeCap) + "new_gasTipCap", newTip, "new_gasFeeCap", newFeeCap, "new_basefee", newBaseFee) + thresholdTip := calcThresholdValue(oldTip, isBlobTx) + thresholdFeeCap := calcThresholdValue(oldFeeCap, isBlobTx) if newTip.Cmp(thresholdTip) >= 0 && newFeeCap.Cmp(thresholdFeeCap) >= 0 { lgr.Debug("Using new tip and feecap") return newTip, newFeeCap @@ -637,10 +742,16 @@ func updateFees(oldTip, oldFeeCap, newTip, newBaseFee *big.Int, lgr log.Logger) func calcGasFeeCap(baseFee, gasTipCap *big.Int) *big.Int { return new(big.Int).Add( gasTipCap, - new(big.Int).Mul(baseFee, big.NewInt(2)), + new(big.Int).Mul(baseFee, two), ) } +// calcBlobFeeCap computes a suggested blob fee cap that is twice the current header's blob fee +// value. +func calcBlobFeeCap(blobFee *big.Int) *big.Int { + return new(big.Int).Mul(blobFee, two) +} + // errStringMatch returns true if err.Error() is a substring in target.Error() or if both are nil. // It can accept nil errors without issue. func errStringMatch(err, target error) bool { diff --git a/op-service/txmgr/txmgr_test.go b/op-service/txmgr/txmgr_test.go index e04966402946..55d1a894a276 100644 --- a/op-service/txmgr/txmgr_test.go +++ b/op-service/txmgr/txmgr_test.go @@ -9,16 +9,19 @@ import ( "testing" "time" + "github.com/holiman/uint256" "github.com/stretchr/testify/require" - "github.com/ethereum-optimism/optimism/op-service/testlog" - "github.com/ethereum-optimism/optimism/op-service/txmgr/metrics" - "github.com/ethereum/go-ethereum" "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/consensus/misc/eip4844" "github.com/ethereum/go-ethereum/core" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/params" + + "github.com/ethereum-optimism/optimism/op-service/testlog" + "github.com/ethereum-optimism/optimism/op-service/txmgr/metrics" ) type sendTransactionFunc func(ctx context.Context, tx *types.Transaction) error @@ -94,6 +97,7 @@ type gasPricer struct { mineAtEpoch int64 baseGasTipFee *big.Int baseBaseFee *big.Int + excessBlobGas uint64 err error mu sync.Mutex } @@ -103,24 +107,35 @@ func newGasPricer(mineAtEpoch int64) *gasPricer { mineAtEpoch: mineAtEpoch, baseGasTipFee: big.NewInt(5), baseBaseFee: big.NewInt(7), + excessBlobGas: 100 * (params.BlobTxBlobGasPerBlob), // simulate 100 excess blobs } } func (g *gasPricer) expGasFeeCap() *big.Int { - _, gasFeeCap := g.feesForEpoch(g.mineAtEpoch) + _, gasFeeCap, _ := g.feesForEpoch(g.mineAtEpoch) return gasFeeCap } +func (g *gasPricer) expBlobFeeCap() *big.Int { + _, _, excessBlobGas := g.feesForEpoch(g.mineAtEpoch) + return eip4844.CalcBlobFee(excessBlobGas) +} + func (g *gasPricer) shouldMine(gasFeeCap *big.Int) bool { - return g.expGasFeeCap().Cmp(gasFeeCap) == 0 + return g.expGasFeeCap().Cmp(gasFeeCap) <= 0 } -func (g *gasPricer) feesForEpoch(epoch int64) (*big.Int, *big.Int) { - epochBaseFee := new(big.Int).Mul(g.baseBaseFee, big.NewInt(epoch)) - epochGasTipCap := new(big.Int).Mul(g.baseGasTipFee, big.NewInt(epoch)) - epochGasFeeCap := calcGasFeeCap(epochBaseFee, epochGasTipCap) +func (g *gasPricer) shouldMineBlobTx(gasFeeCap, blobFeeCap *big.Int) bool { + return g.shouldMine(gasFeeCap) && g.expBlobFeeCap().Cmp(blobFeeCap) <= 0 +} - return epochGasTipCap, epochGasFeeCap +func (g *gasPricer) feesForEpoch(epoch int64) (*big.Int, *big.Int, uint64) { + e := big.NewInt(epoch) + epochBaseFee := new(big.Int).Mul(g.baseBaseFee, e) + epochGasTipCap := new(big.Int).Mul(g.baseGasTipFee, e) + epochGasFeeCap := calcGasFeeCap(epochBaseFee, epochGasTipCap) + epochExcessBlobGas := g.excessBlobGas * uint64(epoch) + return epochGasTipCap, epochGasFeeCap, epochExcessBlobGas } func (g *gasPricer) basefee() *big.Int { @@ -129,18 +144,25 @@ func (g *gasPricer) basefee() *big.Int { return new(big.Int).Mul(g.baseBaseFee, big.NewInt(g.epoch)) } -func (g *gasPricer) sample() (*big.Int, *big.Int) { +func (g *gasPricer) excessblobgas() uint64 { + g.mu.Lock() + defer g.mu.Unlock() + return g.excessBlobGas * uint64(g.epoch) +} + +func (g *gasPricer) sample() (*big.Int, *big.Int, uint64) { g.mu.Lock() defer g.mu.Unlock() g.epoch++ - epochGasTipCap, epochGasFeeCap := g.feesForEpoch(g.epoch) + epochGasTipCap, epochGasFeeCap, epochExcessBlobGas := g.feesForEpoch(g.epoch) - return epochGasTipCap, epochGasFeeCap + return epochGasTipCap, epochGasFeeCap, epochExcessBlobGas } type minedTxInfo struct { gasFeeCap *big.Int + blobFeeCap *big.Int blockNumber uint64 } @@ -175,7 +197,7 @@ func (b *mockBackend) setTxSender(s sendTransactionFunc) { // mine records a (txHash, gasFeeCap) as confirmed. Subsequent calls to // TransactionReceipt with a matching txHash will result in a non-nil receipt. // If a nil txHash is supplied this has the effect of mining an empty block. -func (b *mockBackend) mine(txHash *common.Hash, gasFeeCap *big.Int) { +func (b *mockBackend) mine(txHash *common.Hash, gasFeeCap, blobFeeCap *big.Int) { b.mu.Lock() defer b.mu.Unlock() @@ -183,6 +205,7 @@ func (b *mockBackend) mine(txHash *common.Hash, gasFeeCap *big.Int) { if txHash != nil { b.minedTxs[*txHash] = minedTxInfo{ gasFeeCap: gasFeeCap, + blobFeeCap: blobFeeCap, blockNumber: b.blockHeight, } } @@ -202,8 +225,10 @@ func (b *mockBackend) CallContract(ctx context.Context, call ethereum.CallMsg, b } func (b *mockBackend) HeaderByNumber(ctx context.Context, number *big.Int) (*types.Header, error) { + bg := b.g.excessblobgas() return &types.Header{ - BaseFee: b.g.basefee(), + BaseFee: b.g.basefee(), + ExcessBlobGas: &bg, }, nil } @@ -218,7 +243,7 @@ func (b *mockBackend) EstimateGas(ctx context.Context, msg ethereum.CallMsg) (ui } func (b *mockBackend) SuggestGasTipCap(ctx context.Context) (*big.Int, error) { - tip, _ := b.g.sample() + tip, _, _ := b.g.sample() return tip, nil } @@ -241,10 +266,10 @@ func (*mockBackend) ChainID(ctx context.Context) (*big.Int, error) { return big.NewInt(1), nil } -// TransactionReceipt queries the mockBackend for a mined txHash. If none is -// found, nil is returned for both return values. Otherwise, it returns a -// receipt containing the txHash and the gasFeeCap used in the GasUsed to make -// the value accessible from our test framework. +// TransactionReceipt queries the mockBackend for a mined txHash. If none is found, nil is returned +// for both return values. Otherwise, it returns a receipt containing the txHash, the gasFeeCap +// used in GasUsed, and the blobFeeCap in CumuluativeGasUsed to make the values accessible from our +// test framework. func (b *mockBackend) TransactionReceipt(ctx context.Context, txHash common.Hash) (*types.Receipt, error) { b.mu.RLock() @@ -257,10 +282,15 @@ func (b *mockBackend) TransactionReceipt(ctx context.Context, txHash common.Hash // Return the gas fee cap for the transaction in the GasUsed field so that // we can assert the proper tx confirmed in our tests. + var blobFeeCap uint64 + if txInfo.blobFeeCap != nil { + blobFeeCap = txInfo.blobFeeCap.Uint64() + } return &types.Receipt{ - TxHash: txHash, - GasUsed: txInfo.gasFeeCap.Uint64(), - BlockNumber: big.NewInt(int64(txInfo.blockNumber)), + TxHash: txHash, + GasUsed: txInfo.gasFeeCap.Uint64(), + CumulativeGasUsed: blobFeeCap, + BlockNumber: big.NewInt(int64(txInfo.blockNumber)), }, nil } @@ -276,7 +306,7 @@ func TestTxMgrConfirmAtMinGasPrice(t *testing.T) { gasPricer := newGasPricer(1) - gasTipCap, gasFeeCap := gasPricer.sample() + gasTipCap, gasFeeCap, _ := gasPricer.sample() tx := types.NewTx(&types.DynamicFeeTx{ GasTipCap: gasTipCap, GasFeeCap: gasFeeCap, @@ -285,7 +315,7 @@ func TestTxMgrConfirmAtMinGasPrice(t *testing.T) { sendTx := func(ctx context.Context, tx *types.Transaction) error { if gasPricer.shouldMine(tx.GasFeeCap()) { txHash := tx.Hash() - h.backend.mine(&txHash, tx.GasFeeCap()) + h.backend.mine(&txHash, tx.GasFeeCap(), nil) } return nil } @@ -307,7 +337,7 @@ func TestTxMgrNeverConfirmCancel(t *testing.T) { h := newTestHarness(t) - gasTipCap, gasFeeCap := h.gasPricer.sample() + gasTipCap, gasFeeCap, _ := h.gasPricer.sample() tx := types.NewTx(&types.DynamicFeeTx{ GasTipCap: gasTipCap, GasFeeCap: gasFeeCap, @@ -333,7 +363,7 @@ func TestTxMgrConfirmsAtHigherGasPrice(t *testing.T) { h := newTestHarness(t) - gasTipCap, gasFeeCap := h.gasPricer.sample() + gasTipCap, gasFeeCap, _ := h.gasPricer.sample() tx := types.NewTx(&types.DynamicFeeTx{ GasTipCap: gasTipCap, GasFeeCap: gasFeeCap, @@ -341,7 +371,7 @@ func TestTxMgrConfirmsAtHigherGasPrice(t *testing.T) { sendTx := func(ctx context.Context, tx *types.Transaction) error { if h.gasPricer.shouldMine(tx.GasFeeCap()) { txHash := tx.Hash() - h.backend.mine(&txHash, tx.GasFeeCap()) + h.backend.mine(&txHash, tx.GasFeeCap(), nil) } return nil } @@ -356,6 +386,42 @@ func TestTxMgrConfirmsAtHigherGasPrice(t *testing.T) { require.Equal(t, h.gasPricer.expGasFeeCap().Uint64(), receipt.GasUsed) } +// TestTxMgrConfirmsBlobTxAtMaxGasPrice asserts that Send properly returns the max gas price +// receipt if none of the lower gas price txs were mined when attempting to send a blob tx. +func TestTxMgrConfirmsBlobTxAtHigherGasPrice(t *testing.T) { + t.Parallel() + + h := newTestHarness(t) + + gasTipCap, gasFeeCap, excessBlobGas := h.gasPricer.sample() + blobFeeCap := eip4844.CalcBlobFee(excessBlobGas) + + tx := types.NewTx(&types.BlobTx{ + GasTipCap: uint256.MustFromBig(gasTipCap), + GasFeeCap: uint256.MustFromBig(gasFeeCap), + BlobFeeCap: uint256.MustFromBig(blobFeeCap), + }) + sendTx := func(ctx context.Context, tx *types.Transaction) error { + if h.gasPricer.shouldMineBlobTx(tx.GasFeeCap(), tx.BlobGasFeeCap()) { + txHash := tx.Hash() + h.backend.mine(&txHash, tx.GasFeeCap(), tx.BlobGasFeeCap()) + } + return nil + } + h.backend.setTxSender(sendTx) + + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + + receipt, err := h.mgr.sendTx(ctx, tx) + require.Nil(t, err) + require.NotNil(t, receipt) + // the fee cap for the blob tx at epoch == 3 should end up higher than the min required gas + // (expFeeCap()) since blob tx fee caps are bumped 100% with each epoch. + require.Less(t, h.gasPricer.expGasFeeCap().Uint64(), receipt.GasUsed) + require.Equal(t, h.gasPricer.expBlobFeeCap().Uint64(), receipt.CumulativeGasUsed) +} + // errRpcFailure is a sentinel error used in testing to fail publications. var errRpcFailure = errors.New("rpc failure") @@ -367,7 +433,7 @@ func TestTxMgrBlocksOnFailingRpcCalls(t *testing.T) { h := newTestHarness(t) - gasTipCap, gasFeeCap := h.gasPricer.sample() + gasTipCap, gasFeeCap, _ := h.gasPricer.sample() tx := types.NewTx(&types.DynamicFeeTx{ GasTipCap: gasTipCap, GasFeeCap: gasFeeCap, @@ -393,7 +459,7 @@ func TestTxMgr_CraftTx(t *testing.T) { candidate := h.createTxCandidate() // Craft the transaction. - gasTipCap, gasFeeCap := h.gasPricer.feesForEpoch(h.gasPricer.epoch + 1) + gasTipCap, gasFeeCap, _ := h.gasPricer.feesForEpoch(h.gasPricer.epoch + 1) tx, err := h.mgr.craftTx(context.Background(), candidate) require.Nil(t, err) require.NotNil(t, tx) @@ -498,7 +564,7 @@ func TestTxMgrOnlyOnePublicationSucceeds(t *testing.T) { h := newTestHarness(t) - gasTipCap, gasFeeCap := h.gasPricer.sample() + gasTipCap, gasFeeCap, _ := h.gasPricer.sample() tx := types.NewTx(&types.DynamicFeeTx{ GasTipCap: gasTipCap, GasFeeCap: gasFeeCap, @@ -511,7 +577,7 @@ func TestTxMgrOnlyOnePublicationSucceeds(t *testing.T) { } txHash := tx.Hash() - h.backend.mine(&txHash, tx.GasFeeCap()) + h.backend.mine(&txHash, tx.GasFeeCap(), nil) return nil } h.backend.setTxSender(sendTx) @@ -526,14 +592,14 @@ func TestTxMgrOnlyOnePublicationSucceeds(t *testing.T) { } // TestTxMgrConfirmsMinGasPriceAfterBumping delays the mining of the initial tx -// with the minimum gas price, and asserts that it's receipt is returned even +// with the minimum gas price, and asserts that its receipt is returned even // though if the gas price has been bumped in other goroutines. func TestTxMgrConfirmsMinGasPriceAfterBumping(t *testing.T) { t.Parallel() h := newTestHarness(t) - gasTipCap, gasFeeCap := h.gasPricer.sample() + gasTipCap, gasFeeCap, _ := h.gasPricer.sample() tx := types.NewTx(&types.DynamicFeeTx{ GasTipCap: gasTipCap, GasFeeCap: gasFeeCap, @@ -544,7 +610,7 @@ func TestTxMgrConfirmsMinGasPriceAfterBumping(t *testing.T) { if h.gasPricer.shouldMine(tx.GasFeeCap()) { time.AfterFunc(5*time.Second, func() { txHash := tx.Hash() - h.backend.mine(&txHash, tx.GasFeeCap()) + h.backend.mine(&txHash, tx.GasFeeCap(), nil) }) } return nil @@ -565,7 +631,7 @@ func TestTxMgrDoesntAbortNonceTooLowAfterMiningTx(t *testing.T) { h := newTestHarnessWithConfig(t, configWithNumConfs(2)) - gasTipCap, gasFeeCap := h.gasPricer.sample() + gasTipCap, gasFeeCap, _ := h.gasPricer.sample() tx := types.NewTx(&types.DynamicFeeTx{ GasTipCap: gasTipCap, GasFeeCap: gasFeeCap, @@ -582,9 +648,9 @@ func TestTxMgrDoesntAbortNonceTooLowAfterMiningTx(t *testing.T) { // Accept and mine the actual txn we expect to confirm. case h.gasPricer.shouldMine(tx.GasFeeCap()): txHash := tx.Hash() - h.backend.mine(&txHash, tx.GasFeeCap()) + h.backend.mine(&txHash, tx.GasFeeCap(), nil) time.AfterFunc(5*time.Second, func() { - h.backend.mine(nil, nil) + h.backend.mine(nil, nil, nil) }) return nil @@ -614,7 +680,7 @@ func TestWaitMinedReturnsReceiptOnFirstSuccess(t *testing.T) { // Create a tx and mine it immediately using the default backend. tx := types.NewTx(&types.LegacyTx{}) txHash := tx.Hash() - h.backend.mine(&txHash, new(big.Int)) + h.backend.mine(&txHash, new(big.Int), nil) ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) defer cancel() @@ -656,7 +722,7 @@ func TestWaitMinedMultipleConfs(t *testing.T) { // Create an unimined tx. tx := types.NewTx(&types.LegacyTx{}) txHash := tx.Hash() - h.backend.mine(&txHash, new(big.Int)) + h.backend.mine(&txHash, new(big.Int), nil) receipt, err := h.mgr.waitMined(ctx, tx, NewSendState(10, time.Hour)) require.Equal(t, err, context.DeadlineExceeded) @@ -666,7 +732,7 @@ func TestWaitMinedMultipleConfs(t *testing.T) { defer cancel() // Mine an empty block, tx should now be confirmed. - h.backend.mine(nil, nil) + h.backend.mine(nil, nil, nil) receipt, err = h.mgr.waitMined(ctx, tx, NewSendState(10, time.Hour)) require.Nil(t, err) require.NotNil(t, receipt) @@ -982,7 +1048,7 @@ func TestNonceReset(t *testing.T) { return core.ErrNonceTooLow } txHash := tx.Hash() - h.backend.mine(&txHash, tx.GasFeeCap()) + h.backend.mine(&txHash, tx.GasFeeCap(), nil) return nil } h.backend.setTxSender(sendTx) From 899346cf515fbd0ef2d3596d28acfa06f78d838b Mon Sep 17 00:00:00 2001 From: Roberto Bayardo Date: Fri, 15 Dec 2023 17:19:58 -0800 Subject: [PATCH 2/9] disallow blob calldata from being used for batches, factor out isValidBatchTx code --- op-node/rollup/derive/blob_data_source.go | 31 ++++++----------------- op-node/rollup/derive/call_data_source.go | 13 +++------- op-node/rollup/derive/data_source.go | 14 ++++++++++ op-service/txmgr/txmgr.go | 7 ++--- 4 files changed, 29 insertions(+), 36 deletions(-) diff --git a/op-node/rollup/derive/blob_data_source.go b/op-node/rollup/derive/blob_data_source.go index d9443d135b67..08faff54a128 100644 --- a/op-node/rollup/derive/blob_data_source.go +++ b/op-node/rollup/derive/blob_data_source.go @@ -17,7 +17,6 @@ import ( // BlobDataSource fetches both call-data (backup) and blobs and transforms them into usable rollup data. type BlobDataSource struct { open bool - callData []eth.Data blobDataHashes []eth.IndexedDataHash blobs []*eth.Blob @@ -52,7 +51,7 @@ func (ds *BlobDataSource) Next(ctx context.Context) (eth.Data, error) { if !ds.open { if _, txs, err := ds.fetcher.InfoAndTxsByHash(ctx, ds.ref.Hash); err == nil { ds.open = true - ds.callData, ds.blobDataHashes = BlobDataFromEVMTransactions(ds.dsCfg, ds.batcherAddr, txs, ds.log) + ds.blobDataHashes = BlobDataFromEVMTransactions(ds.dsCfg, ds.batcherAddr, txs, ds.log) } else if errors.Is(err, ethereum.NotFound) { return nil, NewResetError(fmt.Errorf("failed to open blob-data source: %w", err)) } else { @@ -60,11 +59,6 @@ func (ds *BlobDataSource) Next(ctx context.Context) (eth.Data, error) { } } // prioritize call-data - if len(ds.callData) != 0 { - data := ds.callData[0] - ds.callData = ds.callData[1:] - return data, nil - } if len(ds.blobDataHashes) > 0 { // check if there is any blob data in this block we have opened. if ds.blobs == nil { // fetch blobs if we haven't already blobs, err := ds.blobsFetcher.BlobsByRefAndIndexedDataHashes(ctx, ds.ref, ds.blobDataHashes) @@ -93,26 +87,17 @@ func (ds *BlobDataSource) Next(ctx context.Context) (eth.Data, error) { // from transactions that are sent to the batch inbox address from the batch sender address. // This will return an empty array if no valid transactions are found. // Call-data can be used as fallback in case blobs are overpriced or unstable. -func BlobDataFromEVMTransactions(dsCfg DataSourceConfig, batcherAddr common.Address, txs types.Transactions, log log.Logger) ([]eth.Data, []eth.IndexedDataHash) { - var callData []eth.Data +func BlobDataFromEVMTransactions(dsCfg DataSourceConfig, batcherAddr common.Address, txs types.Transactions, log log.Logger) []eth.IndexedDataHash { var indexedDataHashes []eth.IndexedDataHash blobIndex := uint64(0) - for j, tx := range txs { + for _, tx := range txs { if to := tx.To(); to != nil && *to == dsCfg.batchInboxAddress { - seqDataSubmitter, err := dsCfg.l1Signer.Sender(tx) // optimization: only derive sender if To is correct - if err != nil { - log.Warn("tx in inbox with invalid signature", "index", j, "err", err) - blobIndex += uint64(len(tx.BlobHashes())) - continue // bad signature, ignore - } - // some random L1 user might have sent a transaction to our batch inbox, ignore them - if seqDataSubmitter != batcherAddr { - log.Warn("tx in inbox with unauthorized submitter", "index", j, "err", err) + if !isValidBatchTx(tx, dsCfg.l1Signer, batcherAddr) { blobIndex += uint64(len(tx.BlobHashes())) - continue // not an authorized batch submitter, ignore + continue } - if len(tx.Data()) > 0 { // ignore empty calldata - callData = append(callData, tx.Data()) + if len(tx.Data()) > 0 { // ignore calldata since + log.Warn("blob tx has calldata, which will be ignored") } for _, h := range tx.BlobHashes() { indexedDataHashes = append(indexedDataHashes, eth.IndexedDataHash{ @@ -125,5 +110,5 @@ func BlobDataFromEVMTransactions(dsCfg DataSourceConfig, batcherAddr common.Addr blobIndex += uint64(len(tx.BlobHashes())) } } - return callData, indexedDataHashes + return indexedDataHashes } diff --git a/op-node/rollup/derive/call_data_source.go b/op-node/rollup/derive/call_data_source.go index 4b3bfb2824a8..bc983ddbcbfc 100644 --- a/op-node/rollup/derive/call_data_source.go +++ b/op-node/rollup/derive/call_data_source.go @@ -67,17 +67,10 @@ func (ds *CallDataSource) Next(ctx context.Context) (eth.Data, error) { // This will return an empty array if no valid transactions are found. func CallDataFromEVMTransactions(dsCfg DataSourceConfig, batcherAddr common.Address, txs types.Transactions, log log.Logger) []eth.Data { var out []eth.Data - for j, tx := range txs { + for _, tx := range txs { if to := tx.To(); to != nil && *to == dsCfg.batchInboxAddress { - seqDataSubmitter, err := dsCfg.l1Signer.Sender(tx) // optimization: only derive sender if To is correct - if err != nil { - log.Warn("tx in inbox with invalid signature", "index", j, "txHash", tx.Hash(), "err", err) - continue // bad signature, ignore - } - // some random L1 user might have sent a transaction to our batch inbox, ignore them - if seqDataSubmitter != batcherAddr { - log.Warn("tx in inbox with unauthorized submitter", "index", j, "txHash", tx.Hash(), "err", err) - continue // not an authorized batch submitter, ignore + if !isValidBatchTx(tx, dsCfg.l1Signer, batcherAddr) { + continue } out = append(out, tx.Data()) } diff --git a/op-node/rollup/derive/data_source.go b/op-node/rollup/derive/data_source.go index 67e198922e13..14786c3819e6 100644 --- a/op-node/rollup/derive/data_source.go +++ b/op-node/rollup/derive/data_source.go @@ -57,3 +57,17 @@ type DataSourceConfig struct { batchInboxAddress common.Address blobsEnabledL1Timestamp *uint64 } + +func isValidBatchTx(tx *types.Transaction, l1Signer types.Signer, batcherAddr common.Address) bool { + seqDataSubmitter, err := l1Signer.Sender(tx) // optimization: only derive sender if To is correct + if err != nil { + log.Warn("tx in inbox with invalid signature", "hash", tx.Hash(), "err", err) + return false + } + // some random L1 user might have sent a transaction to our batch inbox, ignore them + if seqDataSubmitter != batcherAddr { + log.Warn("tx in inbox with unauthorized submitter", "hash", tx.Hash(), "err", err) + return false + } + return true +} diff --git a/op-service/txmgr/txmgr.go b/op-service/txmgr/txmgr.go index 4e7c9f01def9..0274b0a2c14e 100644 --- a/op-service/txmgr/txmgr.go +++ b/op-service/txmgr/txmgr.go @@ -31,12 +31,13 @@ const ( blobPriceBump int64 = 100 ) -// new = old * (100 + priceBump) / 100 var ( + // new = old * (100 + [blob]priceBump) / 100 priceBumpPercent = big.NewInt(100 + priceBump) - oneHundred = big.NewInt(100) blobPriceBumpPercent = big.NewInt(100 + blobPriceBump) - two = big.NewInt(2) + + oneHundred = big.NewInt(100) + two = big.NewInt(2) ) // TxManager is an interface that allows callers to reliably publish txs, From 2a262b108e050936ffc27ac04c4032d643a25b6e Mon Sep 17 00:00:00 2001 From: Roberto Bayardo Date: Tue, 19 Dec 2023 15:48:33 -0800 Subject: [PATCH 3/9] Eclipse -> Ecotone --- op-chain-ops/genesis/config.go | 30 +++++++----------------------- op-e2e/e2eutils/setup.go | 2 +- op-e2e/eip4844_test.go | 4 +++- op-e2e/setup.go | 4 +--- op-node/rollup/superchain.go | 2 +- op-node/rollup/types.go | 16 ++++++++-------- 6 files changed, 21 insertions(+), 37 deletions(-) diff --git a/op-chain-ops/genesis/config.go b/op-chain-ops/genesis/config.go index 57d925c5ae91..9c659cf19f77 100644 --- a/op-chain-ops/genesis/config.go +++ b/op-chain-ops/genesis/config.go @@ -115,9 +115,9 @@ type DeployConfig struct { // L2GenesisDeltaTimeOffset is the number of seconds after genesis block that Delta hard fork activates. // Set it to 0 to activate at genesis. Nil to disable Delta. L2GenesisDeltaTimeOffset *hexutil.Uint64 `json:"l2GenesisDeltaTimeOffset,omitempty"` - // L2GenesisEclipseTimeOffset is the number of seconds after genesis block that Eclipse hard fork activates. - // Set it to 0 to activate at genesis. Nil to disable Eclipse. - L2GenesisEclipseTimeOffset *hexutil.Uint64 `json:"l2GenesisEclipseTimeOffset,omitempty"` + // L2GenesisEcotoneTimeOffset is the number of seconds after genesis block that Ecotone hard fork activates. + // Set it to 0 to activate at genesis. Nil to disable Ecotone. + L2GenesisEcotoneTimeOffset *hexutil.Uint64 `json:"l2GenesisEcotoneTimeOffset,omitempty"` // L2GenesisFjordTimeOffset is the number of seconds after genesis block that Fjord hard fork activates. // Set it to 0 to activate at genesis. Nil to disable Fjord. L2GenesisFjordTimeOffset *hexutil.Uint64 `json:"l2GenesisFjordTimeOffset,omitempty"` @@ -230,9 +230,6 @@ type DeployConfig struct { // When Cancun activates. Relative to L1 genesis. L1CancunTimeOffset *uint64 `json:"l1CancunTimeOffset,omitempty"` - - // When 4844 blob-tx functionality for rollup DA actives. Relative to L2 genesis. - L2BlobsUpgradeTimeOffset *uint64 `json:"l2BlobsUpgradeTimeOffset,omitempty"` } // Copy will deeply copy the DeployConfig. This does a JSON roundtrip to copy @@ -492,12 +489,12 @@ func (d *DeployConfig) DeltaTime(genesisTime uint64) *uint64 { return &v } -func (d *DeployConfig) EclipseTime(genesisTime uint64) *uint64 { - if d.L2GenesisEclipseTimeOffset == nil { +func (d *DeployConfig) EcotoneTime(genesisTime uint64) *uint64 { + if d.L2GenesisEcotoneTimeOffset == nil { return nil } v := uint64(0) - if offset := *d.L2GenesisEclipseTimeOffset; offset > 0 { + if offset := *d.L2GenesisEcotoneTimeOffset; offset > 0 { v = genesisTime + uint64(offset) } return &v @@ -525,17 +522,6 @@ func (d *DeployConfig) InteropTime(genesisTime uint64) *uint64 { return &v } -func (d *DeployConfig) BlobsUpgradeTime(genesisTime uint64) *uint64 { - if d.L2BlobsUpgradeTimeOffset == nil { - return nil - } - v := uint64(0) - if offset := *d.L2BlobsUpgradeTimeOffset; offset > 0 { - v = genesisTime + uint64(offset) - } - return &v -} - // RollupConfig converts a DeployConfig to a rollup.Config func (d *DeployConfig) RollupConfig(l1StartBlock *types.Block, l2GenesisBlockHash common.Hash, l2GenesisBlockNumber uint64) (*rollup.Config, error) { if d.OptimismPortalProxy == (common.Address{}) { @@ -575,11 +561,9 @@ func (d *DeployConfig) RollupConfig(l1StartBlock *types.Block, l2GenesisBlockHas RegolithTime: d.RegolithTime(l1StartBlock.Time()), CanyonTime: d.CanyonTime(l1StartBlock.Time()), DeltaTime: d.DeltaTime(l1StartBlock.Time()), - EclipseTime: d.EclipseTime(l1StartBlock.Time()), + EcotoneTime: d.EcotoneTime(l1StartBlock.Time()), FjordTime: d.FjordTime(l1StartBlock.Time()), InteropTime: d.InteropTime(l1StartBlock.Time()), - // 4844 blobs usage activation for rollup DA - BlobsEnabledL1Timestamp: d.BlobsUpgradeTime(l1StartBlock.Time()), }, nil } diff --git a/op-e2e/e2eutils/setup.go b/op-e2e/e2eutils/setup.go index 164f890a1178..867fb335199d 100644 --- a/op-e2e/e2eutils/setup.go +++ b/op-e2e/e2eutils/setup.go @@ -159,7 +159,7 @@ func Setup(t require.TestingT, deployParams *DeployParams, alloc *AllocParams) * RegolithTime: deployConf.RegolithTime(uint64(deployConf.L1GenesisBlockTimestamp)), CanyonTime: deployConf.CanyonTime(uint64(deployConf.L1GenesisBlockTimestamp)), DeltaTime: deployConf.DeltaTime(uint64(deployConf.L1GenesisBlockTimestamp)), - EclipseTime: deployConf.EclipseTime(uint64(deployConf.L1GenesisBlockTimestamp)), + EcotoneTime: deployConf.EcotoneTime(uint64(deployConf.L1GenesisBlockTimestamp)), FjordTime: deployConf.FjordTime(uint64(deployConf.L1GenesisBlockTimestamp)), InteropTime: deployConf.InteropTime(uint64(deployConf.L1GenesisBlockTimestamp)), } diff --git a/op-e2e/eip4844_test.go b/op-e2e/eip4844_test.go index e905c3ba92e5..ccea327ca62d 100644 --- a/op-e2e/eip4844_test.go +++ b/op-e2e/eip4844_test.go @@ -13,6 +13,7 @@ import ( "github.com/ethereum-optimism/optimism/op-service/testlog" "github.com/ethereum/go-ethereum/accounts/abi/bind" "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/common/hexutil" "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/rpc" ) @@ -25,7 +26,8 @@ func TestSystem4844E2E(t *testing.T) { cfg := DefaultSystemConfig(t) genesisActivation := uint64(0) cfg.DeployConfig.L1CancunTimeOffset = &genesisActivation - cfg.DeployConfig.L2BlobsUpgradeTimeOffset = &genesisActivation + hu := hexutil.Uint64(0) + cfg.DeployConfig.L2GenesisEcotoneTimeOffset = &hu sys, err := cfg.Start(t) require.Nil(t, err, "Error starting up system") diff --git a/op-e2e/setup.go b/op-e2e/setup.go index e43f727a662d..8be5f289e78c 100644 --- a/op-e2e/setup.go +++ b/op-e2e/setup.go @@ -490,12 +490,10 @@ func (cfg SystemConfig) Start(t *testing.T, _opts ...SystemConfigOption) (*Syste RegolithTime: cfg.DeployConfig.RegolithTime(uint64(cfg.DeployConfig.L1GenesisBlockTimestamp)), CanyonTime: cfg.DeployConfig.CanyonTime(uint64(cfg.DeployConfig.L1GenesisBlockTimestamp)), DeltaTime: cfg.DeployConfig.DeltaTime(uint64(cfg.DeployConfig.L1GenesisBlockTimestamp)), - EclipseTime: cfg.DeployConfig.EclipseTime(uint64(cfg.DeployConfig.L1GenesisBlockTimestamp)), + EcotoneTime: cfg.DeployConfig.EcotoneTime(uint64(cfg.DeployConfig.L1GenesisBlockTimestamp)), FjordTime: cfg.DeployConfig.FjordTime(uint64(cfg.DeployConfig.L1GenesisBlockTimestamp)), InteropTime: cfg.DeployConfig.InteropTime(uint64(cfg.DeployConfig.L1GenesisBlockTimestamp)), ProtocolVersionsAddress: cfg.L1Deployments.ProtocolVersionsProxy, - // 4844 - BlobsEnabledL1Timestamp: cfg.DeployConfig.BlobsUpgradeTime(uint64(cfg.DeployConfig.L1GenesisBlockTimestamp)), } } defaultConfig := makeRollupConfig() diff --git a/op-node/rollup/superchain.go b/op-node/rollup/superchain.go index 953abb35af2f..dba51b991d4d 100644 --- a/op-node/rollup/superchain.go +++ b/op-node/rollup/superchain.go @@ -100,7 +100,7 @@ func LoadOPStackRollupConfig(chainID uint64) (*Config, error) { RegolithTime: ®olithTime, CanyonTime: superChain.Config.CanyonTime, DeltaTime: superChain.Config.DeltaTime, - EclipseTime: superChain.Config.EclipseTime, + EcotoneTime: superChain.Config.EclipseTime, FjordTime: superChain.Config.FjordTime, BatchInboxAddress: common.Address(chConfig.BatchInboxAddr), DepositContractAddress: depositContractAddress, diff --git a/op-node/rollup/types.go b/op-node/rollup/types.go index d18e25e0b11b..a6f539850aad 100644 --- a/op-node/rollup/types.go +++ b/op-node/rollup/types.go @@ -83,9 +83,9 @@ type Config struct { // Active if DeltaTime != nil && L2 block timestamp >= *DeltaTime, inactive otherwise. DeltaTime *uint64 `json:"delta_time,omitempty"` - // EclipseTime sets the activation time of the Eclipse network upgrade. - // Active if EclipseTime != nil && L2 block timestamp >= *EclipseTime, inactive otherwise. - EclipseTime *uint64 `json:"eclipse_time,omitempty"` + // EcotoneTime sets the activation time of the Ecotone network upgrade. + // Active if EcotoneTime != nil && L2 block timestamp >= *EcotoneTime, inactive otherwise. + EcotoneTime *uint64 `json:"ecotone_time,omitempty"` // FjordTime sets the activation time of the Fjord network upgrade. // Active if FjordTime != nil && L2 block timestamp >= *FjordTime, inactive otherwise. @@ -296,9 +296,9 @@ func (c *Config) IsDelta(timestamp uint64) bool { return c.DeltaTime != nil && timestamp >= *c.DeltaTime } -// IsEclipse returns true if the Eclipse hardfork is active at or past the given timestamp. -func (c *Config) IsEclipse(timestamp uint64) bool { - return c.EclipseTime != nil && timestamp >= *c.EclipseTime +// IsEcotone returns true if the Ecotone hardfork is active at or past the given timestamp. +func (c *Config) IsEcotone(timestamp uint64) bool { + return c.EcotoneTime != nil && timestamp >= *c.EcotoneTime } // IsFjord returns true if the Fjord hardfork is active at or past the given timestamp. @@ -340,7 +340,7 @@ func (c *Config) Description(l2Chains map[string]string) string { banner += fmt.Sprintf(" - Regolith: %s\n", fmtForkTimeOrUnset(c.RegolithTime)) banner += fmt.Sprintf(" - Canyon: %s\n", fmtForkTimeOrUnset(c.CanyonTime)) banner += fmt.Sprintf(" - Delta: %s\n", fmtForkTimeOrUnset(c.DeltaTime)) - banner += fmt.Sprintf(" - Eclipse: %s\n", fmtForkTimeOrUnset(c.EclipseTime)) + banner += fmt.Sprintf(" - Ecotone: %s\n", fmtForkTimeOrUnset(c.EcotoneTime)) banner += fmt.Sprintf(" - Fjord: %s\n", fmtForkTimeOrUnset(c.FjordTime)) banner += fmt.Sprintf(" - Interop: %s\n", fmtForkTimeOrUnset(c.InteropTime)) // Report the protocol version @@ -370,7 +370,7 @@ func (c *Config) LogDescription(log log.Logger, l2Chains map[string]string) { "l1_block_number", c.Genesis.L1.Number, "regolith_time", fmtForkTimeOrUnset(c.RegolithTime), "canyon_time", fmtForkTimeOrUnset(c.CanyonTime), "delta_time", fmtForkTimeOrUnset(c.DeltaTime), - "eclipse_time", fmtForkTimeOrUnset(c.EclipseTime), + "ecotone_time", fmtForkTimeOrUnset(c.EcotoneTime), "fjord_time", fmtForkTimeOrUnset(c.FjordTime), "interop_time", fmtForkTimeOrUnset(c.InteropTime), ) From 5ab7c46f6a75399a1e7cc746f9d3f2ccc79168a7 Mon Sep 17 00:00:00 2001 From: Roberto Bayardo Date: Mon, 11 Dec 2023 16:18:54 -0800 Subject: [PATCH 4/9] update L1Block contract with new fields for 4844 --- packages/contracts-bedrock/src/L2/L1Block.sol | 49 ++++++++++++++++++- 1 file changed, 47 insertions(+), 2 deletions(-) diff --git a/packages/contracts-bedrock/src/L2/L1Block.sol b/packages/contracts-bedrock/src/L2/L1Block.sol index 7ed45ce0d4b6..56e1f9649e38 100644 --- a/packages/contracts-bedrock/src/L2/L1Block.sol +++ b/packages/contracts-bedrock/src/L2/L1Block.sol @@ -38,8 +38,17 @@ contract L1Block is ISemver { /// @notice The scalar value applied to the L1 portion of the transaction fee. uint256 public l1FeeScalar; - /// @custom:semver 1.1.0 - string public constant version = "1.1.0"; + /// @notice The scalar value applied to the L1 base fee portion of the blob-capable L1 cost func + uint256 public basefeeScalar; + + /// @notice The scalar value applied to the L1 blob base fee portion of the blob-capable L1 cost func + uint256 public blobBasefeeScalar; + + /// @notice The latest L1 blob basefee. + uint256 public blobBasefee; + + /// @custom:semver 1.2.0 + string public constant version = "1.2.0"; /// @notice Updates the L1 block values. /// @param _number L1 blocknumber. @@ -73,4 +82,40 @@ contract L1Block is ISemver { l1FeeOverhead = _l1FeeOverhead; l1FeeScalar = _l1FeeScalar; } + + /// @notice Updates the L1 block values for a post-blob activated chain. + /// @param _number L1 blocknumber. + /// @param _timestamp L1 timestamp. + /// @param _basefee L1 basefee. + /// @param _blobBasefee L1 blobBasefee. + /// @param _hash L1 blockhash. + /// @param _sequenceNumber Number of L2 blocks since epoch start. + /// @param _batcherHash Versioned hash to authenticate batcher by. + /// @param _basefeeScalar L1 base fee scalar + /// @param _blobBasefeeScalar L1 blob base fee scalar + function setL1BlockValuesV2( + uint64 _number, + uint64 _timestamp, + uint256 _basefee, + uint256 _blobBasefee, + bytes32 _hash, + uint64 _sequenceNumber, + bytes32 _batcherHash, + uint256 _basefeeScalar, + uint256 _blobBasefeeScalar + ) + external + { + require(msg.sender == DEPOSITOR_ACCOUNT, "L1Block: only the depositor account can set L1 block values"); + + number = _number; + timestamp = _timestamp; + basefee = _basefee; + blobBasefee = _blobBasefee; + hash = _hash; + sequenceNumber = _sequenceNumber; + batcherHash = _batcherHash; + basefeeScalar = _basefeeScalar; + blobBasefeeScalar = _blobBasefeeScalar; + } } From f51594f076cfba7b4a20bdf517ea6b592abd8737 Mon Sep 17 00:00:00 2001 From: anikaraghu Date: Tue, 12 Dec 2023 15:59:16 -0800 Subject: [PATCH 5/9] Update pricing function --- .../src/L2/GasPriceOracle.sol | 39 ++++++++++++++----- packages/contracts-bedrock/src/L2/L1Block.sol | 27 +++++++------ 2 files changed, 44 insertions(+), 22 deletions(-) diff --git a/packages/contracts-bedrock/src/L2/GasPriceOracle.sol b/packages/contracts-bedrock/src/L2/GasPriceOracle.sol index 30ab88c19d24..2cfab8444f63 100644 --- a/packages/contracts-bedrock/src/L2/GasPriceOracle.sol +++ b/packages/contracts-bedrock/src/L2/GasPriceOracle.sol @@ -24,8 +24,8 @@ contract GasPriceOracle is ISemver { uint256 public constant DECIMALS = 6; /// @notice Semantic version. - /// @custom:semver 1.1.0 - string public constant version = "1.1.0"; + /// @custom:semver 1.2.0 + string public constant version = "1.2.0"; /// @notice Computes the L1 portion of the fee based on the size of the rlp encoded input /// transaction, the current L1 base fee, and the various dynamic parameters. @@ -33,9 +33,10 @@ contract GasPriceOracle is ISemver { /// @return L1 fee that should be paid for the tx function getL1Fee(bytes memory _data) external view returns (uint256) { uint256 l1GasUsed = getL1GasUsed(_data); - uint256 l1Fee = l1GasUsed * l1BaseFee(); + uint256 scaledBaseFee = baseFeeScalar() * l1BaseFee(); + uint256 scaledBlobBaseFee = blobBaseFeeScalar() * blobBaseFee(); uint256 divisor = 10 ** DECIMALS; - uint256 unscaled = l1Fee * scalar(); + uint256 unscaled = l1GasUsed * (scaledBaseFee + scaledBlobBaseFee); uint256 scaled = unscaled / divisor; return scaled; } @@ -52,12 +53,14 @@ contract GasPriceOracle is ISemver { return block.basefee; } + /// @custom:legacy /// @notice Retrieves the current fee overhead. /// @return Current fee overhead. function overhead() public view returns (uint256) { return L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).l1FeeOverhead(); } + /// @custom:legacy /// @notice Retrieves the current fee scalar. /// @return Current fee scalar. function scalar() public view returns (uint256) { @@ -70,6 +73,24 @@ contract GasPriceOracle is ISemver { return L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).basefee(); } + /// @notice Retrieves the current blob base fee. + /// @return Current blob base fee. + function blobBaseFee() public view returns (uint256) { + return L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).blobBaseFee(); + } + + /// @notice Retrieves the current base fee scalar. + /// @return Current base fee scalar. + function baseFeeScalar() public view returns (uint256) { + return L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).baseFeeScalar(); + } + + /// @notice Retrieves the current blob base fee scalar. + /// @return Current blob base fee scalar. + function blobBaseFeeScalar() public view returns (uint256) { + return L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).blobBaseFeeScalar(); + } + /// @custom:legacy /// @notice Retrieves the number of decimals used in the scalar. /// @return Number of decimals used in the scalar. @@ -77,10 +98,8 @@ contract GasPriceOracle is ISemver { return DECIMALS; } - /// @notice Computes the amount of L1 gas used for a transaction. Adds the overhead which - /// represents the per-transaction gas overhead of posting the transaction and state - /// roots to L1. Adds 68 bytes of padding to account for the fact that the input does - /// not have a signature. + /// @notice Computes the amount of L1 gas used for a transaction. Adds 68 bytes + /// of padding to account for the fact that the input does not have a signature. /// @param _data Unsigned fully RLP-encoded transaction to get the L1 gas for. /// @return Amount of L1 gas used to publish the transaction. function getL1GasUsed(bytes memory _data) public view returns (uint256) { @@ -93,7 +112,7 @@ contract GasPriceOracle is ISemver { total += 16; } } - uint256 unsigned = total + overhead(); - return unsigned + (68 * 16); + uint256 compressedTxSize = total / 16; + return compressedTxSize + (68 * 16); } } diff --git a/packages/contracts-bedrock/src/L2/L1Block.sol b/packages/contracts-bedrock/src/L2/L1Block.sol index 56e1f9649e38..13ea588c27ef 100644 --- a/packages/contracts-bedrock/src/L2/L1Block.sol +++ b/packages/contracts-bedrock/src/L2/L1Block.sol @@ -33,23 +33,26 @@ contract L1Block is ISemver { bytes32 public batcherHash; /// @notice The overhead value applied to the L1 portion of the transaction fee. + /// @custom:legacy uint256 public l1FeeOverhead; /// @notice The scalar value applied to the L1 portion of the transaction fee. + /// @custom:legacy uint256 public l1FeeScalar; /// @notice The scalar value applied to the L1 base fee portion of the blob-capable L1 cost func - uint256 public basefeeScalar; + uint256 public baseFeeScalar; /// @notice The scalar value applied to the L1 blob base fee portion of the blob-capable L1 cost func - uint256 public blobBasefeeScalar; + uint256 public blobBaseFeeScalar; /// @notice The latest L1 blob basefee. - uint256 public blobBasefee; + uint256 public blobBaseFee; /// @custom:semver 1.2.0 string public constant version = "1.2.0"; + /// @custom:legacy /// @notice Updates the L1 block values. /// @param _number L1 blocknumber. /// @param _timestamp L1 timestamp. @@ -87,22 +90,22 @@ contract L1Block is ISemver { /// @param _number L1 blocknumber. /// @param _timestamp L1 timestamp. /// @param _basefee L1 basefee. - /// @param _blobBasefee L1 blobBasefee. + /// @param _blobBaseFee L1 blobBaseFee. /// @param _hash L1 blockhash. /// @param _sequenceNumber Number of L2 blocks since epoch start. /// @param _batcherHash Versioned hash to authenticate batcher by. - /// @param _basefeeScalar L1 base fee scalar - /// @param _blobBasefeeScalar L1 blob base fee scalar + /// @param _baseFeeScalar L1 base fee scalar + /// @param _blobBaseFeeScalar L1 blob base fee scalar function setL1BlockValuesV2( uint64 _number, uint64 _timestamp, uint256 _basefee, - uint256 _blobBasefee, + uint256 _blobBaseFee, bytes32 _hash, uint64 _sequenceNumber, bytes32 _batcherHash, - uint256 _basefeeScalar, - uint256 _blobBasefeeScalar + uint256 _baseFeeScalar, + uint256 _blobBaseFeeScalar ) external { @@ -111,11 +114,11 @@ contract L1Block is ISemver { number = _number; timestamp = _timestamp; basefee = _basefee; - blobBasefee = _blobBasefee; + blobBasefee = _blobBaseFee; hash = _hash; sequenceNumber = _sequenceNumber; batcherHash = _batcherHash; - basefeeScalar = _basefeeScalar; - blobBasefeeScalar = _blobBasefeeScalar; + basefeeScalar = _baseFeeScalar; + blobBasefeeScalar = _blobBaseFeeScalar; } } From d7ea07879141cebcd411df4ec9c5be106f2c9013 Mon Sep 17 00:00:00 2001 From: anikaraghu Date: Fri, 15 Dec 2023 15:33:35 -0800 Subject: [PATCH 6/9] Comments, untested --- .../src/L2/GasPriceOracle.sol | 63 +++++++++++++++---- packages/contracts-bedrock/src/L2/L1Block.sol | 48 +++++++------- 2 files changed, 71 insertions(+), 40 deletions(-) diff --git a/packages/contracts-bedrock/src/L2/GasPriceOracle.sol b/packages/contracts-bedrock/src/L2/GasPriceOracle.sol index 2cfab8444f63..71a24f1371f4 100644 --- a/packages/contracts-bedrock/src/L2/GasPriceOracle.sol +++ b/packages/contracts-bedrock/src/L2/GasPriceOracle.sol @@ -27,20 +27,38 @@ contract GasPriceOracle is ISemver { /// @custom:semver 1.2.0 string public constant version = "1.2.0"; + /// @notice Flag that indicates whether the network is in eclipse mode. + bool public isEclipse; + /// @notice Computes the L1 portion of the fee based on the size of the rlp encoded input /// transaction, the current L1 base fee, and the various dynamic parameters. /// @param _data Unsigned fully RLP-encoded transaction to get the L1 fee for. /// @return L1 fee that should be paid for the tx function getL1Fee(bytes memory _data) external view returns (uint256) { - uint256 l1GasUsed = getL1GasUsed(_data); - uint256 scaledBaseFee = baseFeeScalar() * l1BaseFee(); - uint256 scaledBlobBaseFee = blobBaseFeeScalar() * blobBaseFee(); + uint256 unscaled; + if (isEclipse) { + uint256 l1GasUsed = getL1GasUsed(_data); + uint256 scaledBaseFee = baseFeeScalar() * l1BaseFee(); + uint256 scaledBlobBaseFee = blobBaseFeeScalar() * blobBaseFee(); + unscaled = l1GasUsed * (scaledBaseFee + scaledBlobBaseFee); + } else { + uint256 l1GasUsed = _getL1GasUsedPreEclipse(_data); + uint256 l1Fee = l1GasUsed * l1BaseFee(); + unscaled = l1Fee * L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).l1FeeScalar(); + } uint256 divisor = 10 ** DECIMALS; - uint256 unscaled = l1GasUsed * (scaledBaseFee + scaledBlobBaseFee); uint256 scaled = unscaled / divisor; return scaled; } + function setIsEclipse(bool _isEclipse) external { + require( + msg.sender == L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).DEPOSITOR_ACCOUNT(), + "GasPriceOracle: only the depositor account can set isEclipse flag" + ); + isEclipse = _isEclipse; + } + /// @notice Retrieves the current gas price (base fee). /// @return Current L2 gas price (base fee). function gasPrice() public view returns (uint256) { @@ -56,15 +74,15 @@ contract GasPriceOracle is ISemver { /// @custom:legacy /// @notice Retrieves the current fee overhead. /// @return Current fee overhead. - function overhead() public view returns (uint256) { - return L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).l1FeeOverhead(); + function overhead() public pure returns (uint256) { + revert("GasPriceOracle: overhead() is deprecated"); } /// @custom:legacy /// @notice Retrieves the current fee scalar. /// @return Current fee scalar. - function scalar() public view returns (uint256) { - return L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).l1FeeScalar(); + function scalar() public pure returns (uint256) { + revert("GasPriceOracle: scalar() is deprecated"); } /// @notice Retrieves the latest known L1 base fee. @@ -81,13 +99,13 @@ contract GasPriceOracle is ISemver { /// @notice Retrieves the current base fee scalar. /// @return Current base fee scalar. - function baseFeeScalar() public view returns (uint256) { + function baseFeeScalar() public view returns (uint32) { return L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).baseFeeScalar(); } /// @notice Retrieves the current blob base fee scalar. /// @return Current blob base fee scalar. - function blobBaseFeeScalar() public view returns (uint256) { + function blobBaseFeeScalar() public view returns (uint32) { return L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).blobBaseFeeScalar(); } @@ -98,11 +116,31 @@ contract GasPriceOracle is ISemver { return DECIMALS; } + /// @notice Computes the amount of L1 gas used for a transaction. Adds the overhead which + /// represents the per-transaction gas overhead of posting the transaction and state + /// roots to L1. Adds 68 bytes of padding to account for the fact that the input does + /// not have a signature. + /// @param _data Unsigned fully RLP-encoded transaction to get the L1 gas for. + /// @return Amount of L1 gas used to publish the transaction. + function _getL1GasUsedPreEclipse(bytes memory _data) internal view returns (uint256) { + uint256 total = 0; + uint256 length = _data.length; + for (uint256 i = 0; i < length; i++) { + if (_data[i] == 0) { + total += 4; + } else { + total += 16; + } + } + uint256 unsigned = total + L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).l1FeeOverhead(); + return unsigned + (68 * 16); + } + /// @notice Computes the amount of L1 gas used for a transaction. Adds 68 bytes /// of padding to account for the fact that the input does not have a signature. /// @param _data Unsigned fully RLP-encoded transaction to get the L1 gas for. /// @return Amount of L1 gas used to publish the transaction. - function getL1GasUsed(bytes memory _data) public view returns (uint256) { + function getL1GasUsed(bytes memory _data) public pure returns (uint256) { uint256 total = 0; uint256 length = _data.length; for (uint256 i = 0; i < length; i++) { @@ -112,7 +150,6 @@ contract GasPriceOracle is ISemver { total += 16; } } - uint256 compressedTxSize = total / 16; - return compressedTxSize + (68 * 16); + return (total + (68 * 16 * 16)) / 16; } } diff --git a/packages/contracts-bedrock/src/L2/L1Block.sol b/packages/contracts-bedrock/src/L2/L1Block.sol index 13ea588c27ef..cf986110fc0a 100644 --- a/packages/contracts-bedrock/src/L2/L1Block.sol +++ b/packages/contracts-bedrock/src/L2/L1Block.sol @@ -41,10 +41,10 @@ contract L1Block is ISemver { uint256 public l1FeeScalar; /// @notice The scalar value applied to the L1 base fee portion of the blob-capable L1 cost func - uint256 public baseFeeScalar; + uint32 public baseFeeScalar; /// @notice The scalar value applied to the L1 blob base fee portion of the blob-capable L1 cost func - uint256 public blobBaseFeeScalar; + uint32 public blobBaseFeeScalar; /// @notice The latest L1 blob basefee. uint256 public blobBaseFee; @@ -87,38 +87,32 @@ contract L1Block is ISemver { } /// @notice Updates the L1 block values for a post-blob activated chain. - /// @param _number L1 blocknumber. - /// @param _timestamp L1 timestamp. - /// @param _basefee L1 basefee. - /// @param _blobBaseFee L1 blobBaseFee. - /// @param _hash L1 blockhash. - /// @param _sequenceNumber Number of L2 blocks since epoch start. - /// @param _batcherHash Versioned hash to authenticate batcher by. - /// @param _baseFeeScalar L1 base fee scalar - /// @param _blobBaseFeeScalar L1 blob base fee scalar - function setL1BlockValuesV2( - uint64 _number, - uint64 _timestamp, - uint256 _basefee, - uint256 _blobBaseFee, - bytes32 _hash, - uint64 _sequenceNumber, - bytes32 _batcherHash, - uint256 _baseFeeScalar, - uint256 _blobBaseFeeScalar - ) - external - { + /// Params are passed in as part of msg.data in order to compress the calldata. + /// Params should be passed in in the following order: + /// 1. _number L1 blocknumber. + /// 2. _timestamp L1 timestamp. + /// 3. _basefee L1 basefee. + /// 4. _blobBaseFee L1 blobBaseFee. + /// 5. _hash L1 blockhash. + /// 6. _sequenceNumber Number of L2 blocks since epoch start. + /// 7. _batcherHash Versioned hash to authenticate batcher by. + /// 8. _baseFeeScalar L1 base fee scalar + /// 9. _blobBaseFeeScalar L1 blob base fee scalar + function setL1BlockValuesV2() external { require(msg.sender == DEPOSITOR_ACCOUNT, "L1Block: only the depositor account can set L1 block values"); + (uint64 _number, uint64 _timestamp, uint256 _basefee, uint256 _blobBaseFee, bytes32 _hash, + uint64 _sequenceNumber, bytes32 _batcherHash, uint32 _baseFeeScalar, uint32 _blobBaseFeeScalar) = + abi.decode(msg.data[4:], (uint64, uint64, uint256, uint256, bytes32, uint64, bytes32, uint32, uint32)); + number = _number; timestamp = _timestamp; basefee = _basefee; - blobBasefee = _blobBaseFee; + blobBaseFee = _blobBaseFee; hash = _hash; sequenceNumber = _sequenceNumber; batcherHash = _batcherHash; - basefeeScalar = _baseFeeScalar; - blobBasefeeScalar = _blobBaseFeeScalar; + baseFeeScalar = _baseFeeScalar; + blobBaseFeeScalar = _blobBaseFeeScalar; } } From 91edb9b3e4a383ddae0462f752bfb912b4da9e08 Mon Sep 17 00:00:00 2001 From: anikaraghu Date: Mon, 18 Dec 2023 19:18:50 -0800 Subject: [PATCH 7/9] WIP packing logic --- .../src/L2/GasPriceOracle.sol | 41 ++++++++++++------ packages/contracts-bedrock/src/L2/L1Block.sol | 43 ++++++++++++++++--- 2 files changed, 64 insertions(+), 20 deletions(-) diff --git a/packages/contracts-bedrock/src/L2/GasPriceOracle.sol b/packages/contracts-bedrock/src/L2/GasPriceOracle.sol index 71a24f1371f4..de0b45819d74 100644 --- a/packages/contracts-bedrock/src/L2/GasPriceOracle.sol +++ b/packages/contracts-bedrock/src/L2/GasPriceOracle.sol @@ -37,7 +37,7 @@ contract GasPriceOracle is ISemver { function getL1Fee(bytes memory _data) external view returns (uint256) { uint256 unscaled; if (isEclipse) { - uint256 l1GasUsed = getL1GasUsed(_data); + uint256 l1GasUsed = _getL1GasUsed(_data); uint256 scaledBaseFee = baseFeeScalar() * l1BaseFee(); uint256 scaledBlobBaseFee = blobBaseFeeScalar() * blobBaseFee(); unscaled = l1GasUsed * (scaledBaseFee + scaledBlobBaseFee); @@ -51,12 +51,12 @@ contract GasPriceOracle is ISemver { return scaled; } - function setIsEclipse(bool _isEclipse) external { + function setEclipse() external { require( msg.sender == L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).DEPOSITOR_ACCOUNT(), "GasPriceOracle: only the depositor account can set isEclipse flag" ); - isEclipse = _isEclipse; + isEclipse = true; } /// @notice Retrieves the current gas price (base fee). @@ -74,15 +74,21 @@ contract GasPriceOracle is ISemver { /// @custom:legacy /// @notice Retrieves the current fee overhead. /// @return Current fee overhead. - function overhead() public pure returns (uint256) { - revert("GasPriceOracle: overhead() is deprecated"); + function overhead() public view returns (uint256) { + if (isEclipse) { + revert("GasPriceOracle: overhead() is deprecated"); + } + return L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).l1FeeOverhead(); } /// @custom:legacy /// @notice Retrieves the current fee scalar. /// @return Current fee scalar. - function scalar() public pure returns (uint256) { - revert("GasPriceOracle: scalar() is deprecated"); + function scalar() public view returns (uint256) { + if (isEclipse) { + revert("GasPriceOracle: scalar() is deprecated"); + } + return L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).l1FeeScalar(); } /// @notice Retrieves the latest known L1 base fee. @@ -116,10 +122,18 @@ contract GasPriceOracle is ISemver { return DECIMALS; } - /// @notice Computes the amount of L1 gas used for a transaction. Adds the overhead which - /// represents the per-transaction gas overhead of posting the transaction and state - /// roots to L1. Adds 68 bytes of padding to account for the fact that the input does - /// not have a signature. + /// @notice Computes the amount of L1 gas used for a transaction. Adds 68 bytes + /// of padding to account for the fact that the input does not have a signature. + /// @param _data Unsigned fully RLP-encoded transaction to get the L1 gas for. + /// @return Amount of L1 gas used to publish the transaction. + function getL1GasUsed(bytes memory _data) public view returns (uint256) { + if (isEclipse) { + return _getL1GasUsed(_data); + } + return _getL1GasUsedPreEclipse(_data); + } + + /// @notice Pre-eclipse L1 gas estimation calculation. /// @param _data Unsigned fully RLP-encoded transaction to get the L1 gas for. /// @return Amount of L1 gas used to publish the transaction. function _getL1GasUsedPreEclipse(bytes memory _data) internal view returns (uint256) { @@ -136,11 +150,10 @@ contract GasPriceOracle is ISemver { return unsigned + (68 * 16); } - /// @notice Computes the amount of L1 gas used for a transaction. Adds 68 bytes - /// of padding to account for the fact that the input does not have a signature. + /// @notice Post-eclipse L1 gas estimation calculation. /// @param _data Unsigned fully RLP-encoded transaction to get the L1 gas for. /// @return Amount of L1 gas used to publish the transaction. - function getL1GasUsed(bytes memory _data) public pure returns (uint256) { + function _getL1GasUsed(bytes memory _data) internal pure returns (uint256) { uint256 total = 0; uint256 length = _data.length; for (uint256 i = 0; i < length; i++) { diff --git a/packages/contracts-bedrock/src/L2/L1Block.sol b/packages/contracts-bedrock/src/L2/L1Block.sol index cf986110fc0a..82aab1000d97 100644 --- a/packages/contracts-bedrock/src/L2/L1Block.sol +++ b/packages/contracts-bedrock/src/L2/L1Block.sol @@ -40,15 +40,15 @@ contract L1Block is ISemver { /// @custom:legacy uint256 public l1FeeScalar; + /// @notice The latest L1 blob basefee. + uint256 public blobBaseFee; + /// @notice The scalar value applied to the L1 base fee portion of the blob-capable L1 cost func uint32 public baseFeeScalar; /// @notice The scalar value applied to the L1 blob base fee portion of the blob-capable L1 cost func uint32 public blobBaseFeeScalar; - /// @notice The latest L1 blob basefee. - uint256 public blobBaseFee; - /// @custom:semver 1.2.0 string public constant version = "1.2.0"; @@ -84,6 +84,12 @@ contract L1Block is ISemver { batcherHash = _batcherHash; l1FeeOverhead = _l1FeeOverhead; l1FeeScalar = _l1FeeScalar; + + (uint256 _blobBaseFee) = abi.decode(msg.data[260:], (uint256)); + blobBaseFeeScalar = 1; + baseFeeScalar = 1; + blobBaseFee = _blobBaseFee; + } /// @notice Updates the L1 block values for a post-blob activated chain. @@ -101,9 +107,34 @@ contract L1Block is ISemver { function setL1BlockValuesV2() external { require(msg.sender == DEPOSITOR_ACCOUNT, "L1Block: only the depositor account can set L1 block values"); - (uint64 _number, uint64 _timestamp, uint256 _basefee, uint256 _blobBaseFee, bytes32 _hash, - uint64 _sequenceNumber, bytes32 _batcherHash, uint32 _baseFeeScalar, uint32 _blobBaseFeeScalar) = - abi.decode(msg.data[4:], (uint64, uint64, uint256, uint256, bytes32, uint64, bytes32, uint32, uint32)); + bytes memory _data = msg.data[4:]; + // if (_msgData.length != XXX) { // TODO: configure + // revert("L1Block: invalid msg.data length"); + // } + + uint64 _number; + uint64 _timestamp; + uint256 _basefee; + uint256 _blobBaseFee; + bytes32 _hash; + uint64 _sequenceNumber; + bytes32 _batcherHash; + uint32 _baseFeeScalar; + uint32 _blobBaseFeeScalar; + + assembly { + _number := mload(add(_data, 0x8)) + _timestamp := shr(48, mload(add(_data, 0x16))) + _basefee := shr(192, mload(add(_data, 0x48))) + } + + // TODO calculations for rest + // _blobBaseFee := + // _hash := + // _sequenceNumber := + // _batcherHash := + // _baseFeeScalar := + // _blobBaseFeeScalar := number = _number; timestamp = _timestamp; From 62aac267386efdfac9d1236a66c83d2dda8a7635 Mon Sep 17 00:00:00 2001 From: anikaraghu Date: Tue, 19 Dec 2023 10:57:44 -0800 Subject: [PATCH 8/9] Add tests, not yet passing --- .../src/L2/GasPriceOracle.sol | 60 +++++---- packages/contracts-bedrock/src/L2/L1Block.sol | 64 +++++----- .../test/L2/GasPriceOracle.t.sol | 100 ++++++++++++++- .../contracts-bedrock/test/L2/L1Block.t.sol | 118 +++++++++++++++--- 4 files changed, 270 insertions(+), 72 deletions(-) diff --git a/packages/contracts-bedrock/src/L2/GasPriceOracle.sol b/packages/contracts-bedrock/src/L2/GasPriceOracle.sol index de0b45819d74..b3e41fb143e8 100644 --- a/packages/contracts-bedrock/src/L2/GasPriceOracle.sol +++ b/packages/contracts-bedrock/src/L2/GasPriceOracle.sol @@ -35,22 +35,13 @@ contract GasPriceOracle is ISemver { /// @param _data Unsigned fully RLP-encoded transaction to get the L1 fee for. /// @return L1 fee that should be paid for the tx function getL1Fee(bytes memory _data) external view returns (uint256) { - uint256 unscaled; if (isEclipse) { - uint256 l1GasUsed = _getL1GasUsed(_data); - uint256 scaledBaseFee = baseFeeScalar() * l1BaseFee(); - uint256 scaledBlobBaseFee = blobBaseFeeScalar() * blobBaseFee(); - unscaled = l1GasUsed * (scaledBaseFee + scaledBlobBaseFee); - } else { - uint256 l1GasUsed = _getL1GasUsedPreEclipse(_data); - uint256 l1Fee = l1GasUsed * l1BaseFee(); - unscaled = l1Fee * L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).l1FeeScalar(); + _getL1FeeEclipse(_data); } - uint256 divisor = 10 ** DECIMALS; - uint256 scaled = unscaled / divisor; - return scaled; + return _getL1FeeBedrock(_data); } + /// @notice Set chain to be Eclipse chain (callable by depositor account) function setEclipse() external { require( msg.sender == L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).DEPOSITOR_ACCOUNT(), @@ -99,20 +90,20 @@ contract GasPriceOracle is ISemver { /// @notice Retrieves the current blob base fee. /// @return Current blob base fee. - function blobBaseFee() public view returns (uint256) { - return L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).blobBaseFee(); + function blobBasefee() public view returns (uint256) { + return L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).blobBasefee(); } /// @notice Retrieves the current base fee scalar. /// @return Current base fee scalar. - function baseFeeScalar() public view returns (uint32) { - return L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).baseFeeScalar(); + function basefeeScalar() public view returns (uint32) { + return L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).basefeeScalar(); } /// @notice Retrieves the current blob base fee scalar. /// @return Current blob base fee scalar. - function blobBaseFeeScalar() public view returns (uint32) { - return L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).blobBaseFeeScalar(); + function blobBasefeeScalar() public view returns (uint32) { + return L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).blobBasefeeScalar(); } /// @custom:legacy @@ -128,15 +119,40 @@ contract GasPriceOracle is ISemver { /// @return Amount of L1 gas used to publish the transaction. function getL1GasUsed(bytes memory _data) public view returns (uint256) { if (isEclipse) { - return _getL1GasUsed(_data); + return _getL1GasUsedEclipse(_data); } - return _getL1GasUsedPreEclipse(_data); + return _getL1GasUsedBedrock(_data); + } + + /// @notice Pre-eclipse computation of the L1 portion of the fee. + /// @param _data Unsigned fully RLP-encoded transaction to get the L1 fee for. + /// @return L1 fee that should be paid for the tx + function _getL1FeeBedrock(bytes memory _data) internal view returns (uint256) { + uint256 l1GasUsed = _getL1GasUsedBedrock(_data); + uint256 l1Fee = l1GasUsed * l1BaseFee(); + uint256 unscaled = l1Fee * L1Block(Predeploys.L1_BLOCK_ATTRIBUTES).l1FeeScalar(); + uint256 divisor = 10 ** DECIMALS; + uint256 scaled = unscaled / divisor; + return scaled; + } + + /// @notice Post-eclipse computation of the L1 portion of the fee. + /// @param _data Unsigned fully RLP-encoded transaction to get the L1 fee for. + /// @return L1 fee that should be paid for the tx + function _getL1FeeEclipse(bytes memory _data) internal view returns (uint256) { + uint256 l1GasUsed = _getL1GasUsedEclipse(_data); + uint256 scaledBasefee = basefeeScalar() * l1BaseFee(); + uint256 scaledBlobBasefee = blobBasefeeScalar() * blobBasefee(); + uint256 unscaled = l1GasUsed * (scaledBasefee + scaledBlobBasefee); + uint256 divisor = 10 ** DECIMALS; + uint256 scaled = unscaled / divisor; + return scaled; } /// @notice Pre-eclipse L1 gas estimation calculation. /// @param _data Unsigned fully RLP-encoded transaction to get the L1 gas for. /// @return Amount of L1 gas used to publish the transaction. - function _getL1GasUsedPreEclipse(bytes memory _data) internal view returns (uint256) { + function _getL1GasUsedBedrock(bytes memory _data) internal view returns (uint256) { uint256 total = 0; uint256 length = _data.length; for (uint256 i = 0; i < length; i++) { @@ -153,7 +169,7 @@ contract GasPriceOracle is ISemver { /// @notice Post-eclipse L1 gas estimation calculation. /// @param _data Unsigned fully RLP-encoded transaction to get the L1 gas for. /// @return Amount of L1 gas used to publish the transaction. - function _getL1GasUsed(bytes memory _data) internal pure returns (uint256) { + function _getL1GasUsedEclipse(bytes memory _data) internal pure returns (uint256) { uint256 total = 0; uint256 length = _data.length; for (uint256 i = 0; i < length; i++) { diff --git a/packages/contracts-bedrock/src/L2/L1Block.sol b/packages/contracts-bedrock/src/L2/L1Block.sol index 82aab1000d97..96f7119431ad 100644 --- a/packages/contracts-bedrock/src/L2/L1Block.sol +++ b/packages/contracts-bedrock/src/L2/L1Block.sol @@ -41,13 +41,13 @@ contract L1Block is ISemver { uint256 public l1FeeScalar; /// @notice The latest L1 blob basefee. - uint256 public blobBaseFee; + uint256 public blobBasefee; /// @notice The scalar value applied to the L1 base fee portion of the blob-capable L1 cost func - uint32 public baseFeeScalar; + uint32 public basefeeScalar; /// @notice The scalar value applied to the L1 blob base fee portion of the blob-capable L1 cost func - uint32 public blobBaseFeeScalar; + uint32 public blobBasefeeScalar; /// @custom:semver 1.2.0 string public constant version = "1.2.0"; @@ -84,12 +84,6 @@ contract L1Block is ISemver { batcherHash = _batcherHash; l1FeeOverhead = _l1FeeOverhead; l1FeeScalar = _l1FeeScalar; - - (uint256 _blobBaseFee) = abi.decode(msg.data[260:], (uint256)); - blobBaseFeeScalar = 1; - baseFeeScalar = 1; - blobBaseFee = _blobBaseFee; - } /// @notice Updates the L1 block values for a post-blob activated chain. @@ -98,52 +92,54 @@ contract L1Block is ISemver { /// 1. _number L1 blocknumber. /// 2. _timestamp L1 timestamp. /// 3. _basefee L1 basefee. - /// 4. _blobBaseFee L1 blobBaseFee. + /// 4. _blobBasefee L1 blobBasefee. /// 5. _hash L1 blockhash. /// 6. _sequenceNumber Number of L2 blocks since epoch start. /// 7. _batcherHash Versioned hash to authenticate batcher by. - /// 8. _baseFeeScalar L1 base fee scalar - /// 9. _blobBaseFeeScalar L1 blob base fee scalar + /// 8. _basefeeScalar L1 base fee scalar + /// 9. _blobBasefeeScalar L1 blob base fee scalar function setL1BlockValuesV2() external { require(msg.sender == DEPOSITOR_ACCOUNT, "L1Block: only the depositor account can set L1 block values"); - bytes memory _data = msg.data[4:]; - // if (_msgData.length != XXX) { // TODO: configure - // revert("L1Block: invalid msg.data length"); - // } - uint64 _number; uint64 _timestamp; uint256 _basefee; - uint256 _blobBaseFee; + uint256 _blobBasefee; bytes32 _hash; uint64 _sequenceNumber; bytes32 _batcherHash; - uint32 _baseFeeScalar; - uint32 _blobBaseFeeScalar; + uint32 _basefeeScalar; + uint32 _blobBasefeeScalar; assembly { - _number := mload(add(_data, 0x8)) - _timestamp := shr(48, mload(add(_data, 0x16))) - _basefee := shr(192, mload(add(_data, 0x48))) + let offset := 0x4 + _number := shr(24, calldataload(offset)) // uint64 + offset := add(offset, 0x20) + _timestamp := shr(24, calldataload(offset)) // uint64 + offset := add(offset, 0x20) + _basefee := calldataload(offset) // uint256 + offset := add(offset, 0x20) + _blobBasefee := calldataload(offset) // uint256 + offset := add(offset, 0x20) + _hash := calldataload(offset) // bytes32 + offset := add(offset, 0x20) + _sequenceNumber := shr(24, calldataload(offset)) // uint64 + offset := add(offset, 0x20) + _batcherHash := calldataload(offset) // bytes32 + offset := add(offset, 0x20) + _basefeeScalar := shr(24, calldataload(offset)) // uint32 + offset := add(offset, 0x20) + _blobBasefeeScalar := shr(24, calldataload(offset)) // uint32 } - // TODO calculations for rest - // _blobBaseFee := - // _hash := - // _sequenceNumber := - // _batcherHash := - // _baseFeeScalar := - // _blobBaseFeeScalar := - number = _number; timestamp = _timestamp; basefee = _basefee; - blobBaseFee = _blobBaseFee; + blobBasefee = _blobBasefee; hash = _hash; sequenceNumber = _sequenceNumber; batcherHash = _batcherHash; - baseFeeScalar = _baseFeeScalar; - blobBaseFeeScalar = _blobBaseFeeScalar; + basefeeScalar = _basefeeScalar; + blobBasefeeScalar = _blobBasefeeScalar; } } diff --git a/packages/contracts-bedrock/test/L2/GasPriceOracle.t.sol b/packages/contracts-bedrock/test/L2/GasPriceOracle.t.sol index 5e9d71e7baa5..2a2b357c532d 100644 --- a/packages/contracts-bedrock/test/L2/GasPriceOracle.t.sol +++ b/packages/contracts-bedrock/test/L2/GasPriceOracle.t.sol @@ -3,28 +3,38 @@ pragma solidity 0.8.15; // Testing utilities import { CommonTest } from "test/setup/CommonTest.sol"; +import {console} from "forge-std/console.sol"; contract GasPriceOracle_Test is CommonTest { event OverheadUpdated(uint256); event ScalarUpdated(uint256); event DecimalsUpdated(uint256); - address depositor; // The initial L1 context values uint64 constant number = 10; uint64 constant timestamp = 11; uint256 constant basefee = 100; + uint256 constant blobBasefee = 101; bytes32 constant hash = bytes32(uint256(64)); uint64 constant sequenceNumber = 0; bytes32 constant batcherHash = bytes32(uint256(777)); uint256 constant l1FeeOverhead = 310; uint256 constant l1FeeScalar = 10; + uint32 constant blobBasefeeScalar = 15; + uint32 constant basefeeScalar = 20; /// @dev Sets up the test suite. function setUp() public virtual override { super.setUp(); + depositor = l1Block.DEPOSITOR_ACCOUNT(); + } +} +contract GasPriceOracleBedrock_Test is GasPriceOracle_Test { + /// @dev Sets up the test suite. + function setUp() public virtual override { + super.setUp(); depositor = l1Block.DEPOSITOR_ACCOUNT(); vm.prank(depositor); @@ -93,3 +103,91 @@ contract GasPriceOracle_Test is CommonTest { assertEq(returndata, hex""); } } + +contract GasPriceOracleEclipse_Test is GasPriceOracle_Test { + /// @dev Sets up the test suite. + function setUp() public virtual override { + super.setUp(); + + // Define the function signature + bytes4 functionSignature = bytes4(keccak256("setL1BlockValuesV2()")); + + // Encode the function signature and extra data + bytes memory callDataPacked = abi.encodePacked( + number, + timestamp, + basefee, + blobBasefee, + hash, + sequenceNumber, + batcherHash, + basefeeScalar, + blobBasefeeScalar + ); + // console.logBytes(functionCallData); + bytes memory functionCallData = abi.encodePacked(functionSignature, callDataPacked); + + // Execute the function call + vm.prank(depositor); + (bool success, ) = address(l1Block).call(functionCallData); + require(success, "Function call failed"); + + vm.prank(depositor); + gasPriceOracle.setEclipse(); + } + + /// @dev Tests that `l1BaseFee` is set correctly. + function test_l1BaseFee_succeeds() external { + assertEq(gasPriceOracle.l1BaseFee(), basefee); + } + + /// @dev Tests that `gasPrice` is set correctly. + function test_gasPrice_succeeds() external { + vm.fee(100); + uint256 gasPrice = gasPriceOracle.gasPrice(); + assertEq(gasPrice, 100); + } + + /// @dev Tests that `baseFee` is set correctly. + function test_baseFee_succeeds() external { + vm.fee(64); + uint256 gasPrice = gasPriceOracle.baseFee(); + assertEq(gasPrice, 64); + } + + /// @dev Tests that `decimals` is set correctly. + function test_decimals_succeeds() external { + assertEq(gasPriceOracle.decimals(), 6); + assertEq(gasPriceOracle.DECIMALS(), 6); + } + + /// @dev Tests that `scalar` reverts since it was removed in eclipse. + function test_scalar_reverts() external { + vm.expectRevert("GasPriceOracle: scalar() is deprecated"); + gasPriceOracle.scalar(); + } + + /// @dev Tests that `overhead` reverts since it was removed in eclipse. + function test_overhead_reverts() external { + vm.expectRevert("GasPriceOracle: overhead() is deprecated"); + gasPriceOracle.overhead(); + } + + /// @dev Tests that `setGasPrice` reverts since it was removed in bedrock. + function test_setGasPrice_doesNotExist_reverts() external { + (bool success, bytes memory returndata) = + address(gasPriceOracle).call(abi.encodeWithSignature("setGasPrice(uint256)", 1)); + + assertEq(success, false); + assertEq(returndata, hex""); + } + + /// @dev Tests that `setL1BaseFee` reverts since it was removed in bedrock. + function test_setL1BaseFee_doesNotExist_reverts() external { + (bool success, bytes memory returndata) = + address(gasPriceOracle).call(abi.encodeWithSignature("setL1BaseFee(uint256)", 1)); + + assertEq(success, false); + assertEq(returndata, hex""); + } +} \ No newline at end of file diff --git a/packages/contracts-bedrock/test/L2/L1Block.t.sol b/packages/contracts-bedrock/test/L2/L1Block.t.sol index c62a32e446f9..1c52d441308d 100644 --- a/packages/contracts-bedrock/test/L2/L1Block.t.sol +++ b/packages/contracts-bedrock/test/L2/L1Block.t.sol @@ -6,29 +6,51 @@ import { CommonTest } from "test/setup/CommonTest.sol"; // Target contract import { L1Block } from "src/L2/L1Block.sol"; +import "forge-std/console.sol"; + contract L1BlockTest is CommonTest { address depositor; + // The initial L1 context values + uint64 constant number = 123; + uint64 constant timestamp = 456; + uint256 constant basefee = 789; + uint256 constant blobBasefee = 1011; + bytes32 constant hash = bytes32(uint256(1213)); + uint64 constant sequenceNumber = 14; + bytes32 constant batcherHash = bytes32(uint256(1516)); + uint256 constant l1FeeOverhead = 1718; + uint256 constant l1FeeScalar = 1920; + uint32 constant blobBasefeeScalar = 21; + uint32 constant basefeeScalar = 22; + /// @dev Sets up the test suite. function setUp() public virtual override { super.setUp(); - depositor = l1Block.DEPOSITOR_ACCOUNT(); + } +} + +contract L1BlockBedrock_Test is L1BlockTest { + /// @dev Sets up the test suite. + function setUp() public virtual override { + super.setUp(); + vm.prank(depositor); l1Block.setL1BlockValues({ - _number: uint64(1), - _timestamp: uint64(2), - _basefee: 3, - _hash: keccak256(abi.encode(block.number)), - _sequenceNumber: uint64(4), - _batcherHash: bytes32(0), - _l1FeeOverhead: 2, - _l1FeeScalar: 3 + _number: number, + _timestamp: timestamp, + _basefee: basefee, + _hash: hash, + _sequenceNumber: sequenceNumber, + _batcherHash: batcherHash, + _l1FeeOverhead: l1FeeOverhead, + _l1FeeScalar: l1FeeScalar }); } - /// @dev Tests that `setL1BlockValues` updates the values correctly. + // @dev Tests that `setL1BlockValues` updates the values correctly. function testFuzz_updatesValues_succeeds( uint64 n, uint64 t, @@ -55,27 +77,27 @@ contract L1BlockTest is CommonTest { /// @dev Tests that `number` returns the correct value. function test_number_succeeds() external { - assertEq(l1Block.number(), uint64(1)); + assertEq(l1Block.number(), number); } /// @dev Tests that `timestamp` returns the correct value. function test_timestamp_succeeds() external { - assertEq(l1Block.timestamp(), uint64(2)); + assertEq(l1Block.timestamp(), timestamp); } /// @dev Tests that `basefee` returns the correct value. function test_basefee_succeeds() external { - assertEq(l1Block.basefee(), 3); + assertEq(l1Block.basefee(), basefee); } /// @dev Tests that `hash` returns the correct value. function test_hash_succeeds() external { - assertEq(l1Block.hash(), keccak256(abi.encode(block.number))); + assertEq(l1Block.hash(), hash); } /// @dev Tests that `sequenceNumber` returns the correct value. function test_sequenceNumber_succeeds() external { - assertEq(l1Block.sequenceNumber(), uint64(4)); + assertEq(l1Block.sequenceNumber(), sequenceNumber); } /// @dev Tests that `setL1BlockValues` can set max values. @@ -93,3 +115,69 @@ contract L1BlockTest is CommonTest { }); } } + +contract L1BlockEclipse_Test is L1BlockTest { + /// @dev Sets up the test suite. + function setUp() public virtual override { + super.setUp(); + + bytes4 functionSignature = bytes4(keccak256("setL1BlockValuesV2()")); + bytes memory callDataPacked = abi.encodePacked( + number, + timestamp, + basefee, + blobBasefee, + hash, + sequenceNumber, + batcherHash, + basefeeScalar, + blobBasefeeScalar + ); + + bytes memory functionCallDataPacked = abi.encodePacked(functionSignature, callDataPacked); + + vm.prank(depositor); + (bool success, ) = address(l1Block).call(functionCallDataPacked); + require(success, "Function call failed"); + } + + /// @dev Tests that `number` returns the correct value. + function test_number_succeeds() external { + assertEq(l1Block.number(), number); + } + + /// @dev Tests that `timestamp` returns the correct value. + function test_timestamp_succeeds() external { + assertEq(l1Block.timestamp(), timestamp); + } + + /// @dev Tests that `blobBasefee` returns the correct value. + function test_blobBaseFee_succeeds() external { + assertEq(l1Block.blobBasefee(), blobBasefee); + } + + /// @dev Tests that `basefeeScalar` returns the correct value. + function test_baseFeeScalar_succeeds() external { + assertEq(l1Block.basefeeScalar(), basefeeScalar); + } + + /// @dev Tests that `blobBasefeeScalar` returns the correct value. + function test_blobBaseFeeScalar_succeeds() external { + assertEq(l1Block.blobBasefeeScalar(), blobBasefeeScalar); + } + + /// @dev Tests that `basefee` returns the correct value. + function test_basefee_succeeds() external { + assertEq(l1Block.basefee(), basefee); + } + + /// @dev Tests that `hash` returns the correct value. + function test_hash_succeeds() external { + assertEq(l1Block.hash(), hash); + } + + /// @dev Tests that `sequenceNumber` returns the correct value. + function test_sequenceNumber_succeeds() external { + assertEq(l1Block.sequenceNumber(), sequenceNumber); + } +} \ No newline at end of file From 07278b7976ca42a6a46a62c7d0cf45749c895497 Mon Sep 17 00:00:00 2001 From: anikaraghu Date: Tue, 19 Dec 2023 17:15:27 -0800 Subject: [PATCH 9/9] Tests passing --- packages/contracts-bedrock/src/L2/L1Block.sol | 46 +++++++++---------- .../test/L2/GasPriceOracle.t.sol | 2 - .../contracts-bedrock/test/L2/L1Block.t.sol | 8 ++-- 3 files changed, 26 insertions(+), 30 deletions(-) diff --git a/packages/contracts-bedrock/src/L2/L1Block.sol b/packages/contracts-bedrock/src/L2/L1Block.sol index 96f7119431ad..d862b17d804a 100644 --- a/packages/contracts-bedrock/src/L2/L1Block.sol +++ b/packages/contracts-bedrock/src/L2/L1Block.sol @@ -101,45 +101,45 @@ contract L1Block is ISemver { function setL1BlockValuesV2() external { require(msg.sender == DEPOSITOR_ACCOUNT, "L1Block: only the depositor account can set L1 block values"); - uint64 _number; - uint64 _timestamp; + uint256 _number; + uint256 _timestamp; uint256 _basefee; uint256 _blobBasefee; bytes32 _hash; - uint64 _sequenceNumber; + uint256 _sequenceNumber; bytes32 _batcherHash; - uint32 _basefeeScalar; - uint32 _blobBasefeeScalar; + uint256 _basefeeScalar; + uint256 _blobBasefeeScalar; assembly { let offset := 0x4 - _number := shr(24, calldataload(offset)) // uint64 + _number := shr(192, calldataload(offset)) // uint64 + offset := add(offset, 0x8) + _timestamp := shr(192, calldataload(offset)) // uint64 + offset := add(offset, 0x18) + _basefee := shr(128, calldataload(offset)) // uint256 offset := add(offset, 0x20) - _timestamp := shr(24, calldataload(offset)) // uint64 - offset := add(offset, 0x20) - _basefee := calldataload(offset) // uint256 - offset := add(offset, 0x20) - _blobBasefee := calldataload(offset) // uint256 - offset := add(offset, 0x20) - _hash := calldataload(offset) // bytes32 - offset := add(offset, 0x20) - _sequenceNumber := shr(24, calldataload(offset)) // uint64 + _blobBasefee := shr(128, calldataload(offset)) // uint256 offset := add(offset, 0x20) + _hash := shr(128, calldataload(offset)) // bytes32 + offset := add(offset, 0x10) + _sequenceNumber := shr(192, calldataload(offset)) // uint64 + offset := add(offset, 0x8) _batcherHash := calldataload(offset) // bytes32 offset := add(offset, 0x20) - _basefeeScalar := shr(24, calldataload(offset)) // uint32 - offset := add(offset, 0x20) - _blobBasefeeScalar := shr(24, calldataload(offset)) // uint32 + _basefeeScalar := shr(224, calldataload(offset)) // uint32 + offset := add(offset, 0x4) + _blobBasefeeScalar := shr(224, calldataload(offset)) // uint32 } - number = _number; - timestamp = _timestamp; + number = uint64(_number); + timestamp = uint64(_timestamp); basefee = _basefee; blobBasefee = _blobBasefee; hash = _hash; - sequenceNumber = _sequenceNumber; + sequenceNumber = uint64(_sequenceNumber); batcherHash = _batcherHash; - basefeeScalar = _basefeeScalar; - blobBasefeeScalar = _blobBasefeeScalar; + basefeeScalar = uint32(_basefeeScalar); + blobBasefeeScalar = uint32(_blobBasefeeScalar); } } diff --git a/packages/contracts-bedrock/test/L2/GasPriceOracle.t.sol b/packages/contracts-bedrock/test/L2/GasPriceOracle.t.sol index 2a2b357c532d..05423a94e6b1 100644 --- a/packages/contracts-bedrock/test/L2/GasPriceOracle.t.sol +++ b/packages/contracts-bedrock/test/L2/GasPriceOracle.t.sol @@ -3,7 +3,6 @@ pragma solidity 0.8.15; // Testing utilities import { CommonTest } from "test/setup/CommonTest.sol"; -import {console} from "forge-std/console.sol"; contract GasPriceOracle_Test is CommonTest { event OverheadUpdated(uint256); @@ -124,7 +123,6 @@ contract GasPriceOracleEclipse_Test is GasPriceOracle_Test { basefeeScalar, blobBasefeeScalar ); - // console.logBytes(functionCallData); bytes memory functionCallData = abi.encodePacked(functionSignature, callDataPacked); // Execute the function call diff --git a/packages/contracts-bedrock/test/L2/L1Block.t.sol b/packages/contracts-bedrock/test/L2/L1Block.t.sol index 1c52d441308d..03551cabcc65 100644 --- a/packages/contracts-bedrock/test/L2/L1Block.t.sol +++ b/packages/contracts-bedrock/test/L2/L1Block.t.sol @@ -6,8 +6,6 @@ import { CommonTest } from "test/setup/CommonTest.sol"; // Target contract import { L1Block } from "src/L2/L1Block.sol"; -import "forge-std/console.sol"; - contract L1BlockTest is CommonTest { address depositor; @@ -22,8 +20,8 @@ contract L1BlockTest is CommonTest { bytes32 constant batcherHash = bytes32(uint256(1516)); uint256 constant l1FeeOverhead = 1718; uint256 constant l1FeeScalar = 1920; - uint32 constant blobBasefeeScalar = 21; - uint32 constant basefeeScalar = 22; + uint32 constant basefeeScalar = 21; + uint32 constant blobBasefeeScalar = 22; /// @dev Sets up the test suite. function setUp() public virtual override { @@ -179,5 +177,5 @@ contract L1BlockEclipse_Test is L1BlockTest { /// @dev Tests that `sequenceNumber` returns the correct value. function test_sequenceNumber_succeeds() external { assertEq(l1Block.sequenceNumber(), sequenceNumber); - } + }åå } \ No newline at end of file