optimism

diff: ignored:
+5367
-103
+1866
-624

This is an overview of the changes in SWC optimism, a fork of official optimism, part of the SWC OP-stack.

The OP-stack is extended to to allow the batch inbox to be a contract, enabling customized batch submission conditions such as:

  • Requiring the batch transaction to be signed by a quorum of sequencers in a decentralized sequencing network; or

  • Mandating that the batch transaction call a BLOB storage contract (e.g., EthStorage) with a long-term storage fee, which is then distributed to data nodes that prove BLOB storage over time.

diff --git official optimism/op-batcher/batcher/driver.go SWC optimism/op-batcher/batcher/driver.go index 4630df4a6f3bdd88aaa6e9a1f71ec43323b95150..308c1abbc73f9a2aa7495578afd2d0e15d2fb34e 100644 --- official optimism/op-batcher/batcher/driver.go +++ SWC optimism/op-batcher/batcher/driver.go @@ -32,8 +32,9 @@ "github.com/ethereum-optimism/optimism/op-service/txmgr" )   var ( - ErrBatcherNotRunning = errors.New("batcher is not running") - emptyTxData = txData{ + ErrBatcherNotRunning = errors.New("batcher is not running") + ErrInboxTransactionFailed = errors.New("inbox transaction failed") + emptyTxData = txData{ frames: []frameData{ { data: []byte{}, @@ -71,6 +72,7 @@ type L1Client interface { HeaderByNumber(ctx context.Context, number *big.Int) (*types.Header, error) NonceAt(ctx context.Context, account common.Address, blockNumber *big.Int) (uint64, error) + CodeAt(ctx context.Context, account common.Address, blockNumber *big.Int) ([]byte, error) }   type L2Client interface { @@ -985,6 +987,10 @@ } candidate = l.calldataTxCandidate(txdata.CallData()) }   + if *candidate.To != l.RollupConfig.BatchInboxAddress { + return fmt.Errorf("candidate.To is not inbox") + } + l.sendTx(txdata, false, candidate, queue, receiptsCh) return nil } @@ -996,12 +1002,15 @@ // sendTx uses the txmgr queue to send the given transaction candidate after setting its // gaslimit. It will block if the txmgr queue has reached its MaxPendingTransactions limit. func (l *BatchSubmitter) sendTx(txdata txData, isCancel bool, candidate *txmgr.TxCandidate, queue TxSender[txRef], receiptsCh chan txmgr.TxReceipt[txRef]) { - floorDataGas, err := core.FloorDataGas(candidate.TxData) - if err != nil { - // We log instead of return an error here because the txmgr will do its own gas estimation. - l.Log.Warn("Failed to calculate floor data gas", "err", err) - } else { - candidate.GasLimit = floorDataGas + // Leave GasLimit unset when inbox is contract so that later on `EstimateGas` will be called + if !l.RollupConfig.UseInboxContract() { + floorDataGas, err := core.FloorDataGas(candidate.TxData) + if err != nil { + // We log instead of return an error here because the txmgr will do its own gas estimation. + l.Log.Warn("Failed to calculate floor data gas", "err", err) + } else { + candidate.GasLimit = floorDataGas + } }   queue.Send(txRef{id: txdata.ID(), isCancel: isCancel, isBlob: txdata.asBlob}, *candidate, receiptsCh) @@ -1036,6 +1045,12 @@ // Record TX Status if r.Err != nil { l.recordFailedTx(r.ID.id, r.Err) } else if r.Receipt != nil { + // check tx status + if l.RollupConfig.UseInboxContract() && r.Receipt.Status == types.ReceiptStatusFailed { + l.recordFailedTx(r.ID.id, ErrInboxTransactionFailed) + return + } + l.recordConfirmedTx(r.ID.id, r.Receipt) } // Both r.Err and r.Receipt can be nil, in which case we do nothing. @@ -1053,6 +1068,7 @@ func (l *BatchSubmitter) recordFailedTx(id txID, err error) { l.channelMgrMutex.Lock() defer l.channelMgrMutex.Unlock() + l.Log.Warn("Transaction failed to send", logFields(id, err)...) l.channelMgr.TxFailed(id) }
diff --git official optimism/op-chain-ops/genesis/config.go SWC optimism/op-chain-ops/genesis/config.go index 99b07787624fd518de9e503ffae436424839ba93..2683204d242fad69264406fe351f93498819db68 100644 --- official optimism/op-chain-ops/genesis/config.go +++ SWC optimism/op-chain-ops/genesis/config.go @@ -378,6 +378,11 @@ // When Cancun activates. Relative to L1 genesis. L1CancunTimeOffset *hexutil.Uint64 `json:"l1CancunTimeOffset,omitempty"` // When Prague activates. Relative to L1 genesis. L1PragueTimeOffset *hexutil.Uint64 `json:"l1PragueTimeOffset,omitempty"` + + // QKC changes: + // L2GenesisBlobTimeOffset is the number of seconds after genesis block that the L2Blob hard fork activates. + // Set it to 0 to activate at genesis. Nil to disable L2Blob. + L2GenesisBlobTimeOffset *hexutil.Uint64 `json:"l2GenesisBlobTimeOffset,omitempty"` // When Osaka activates. Relative to L1 genesis. L1OsakaTimeOffset *hexutil.Uint64 `json:"l1OsakaTimeOffset,omitempty"` // When BPO1 activates. Relative to L1 genesis. @@ -532,6 +537,10 @@ func (d *UpgradeScheduleDeployConfig) InteropTime(genesisTime uint64) *uint64 { return offsetToUpgradeTime(d.L2GenesisInteropTimeOffset, genesisTime) }   +func (d *UpgradeScheduleDeployConfig) L2BlobTime(genesisTime uint64) *uint64 { + return offsetToUpgradeTime(d.L2GenesisBlobTimeOffset, genesisTime) +} + func (d *UpgradeScheduleDeployConfig) AllocMode(genesisTime uint64) L2AllocsMode { forks := d.forks() for i := len(forks) - 1; i >= 0; i-- { @@ -743,6 +752,9 @@ UpgradeScheduleDeployConfig L2CoreDeployConfig FeeMarketConfig AltDADeployConfig + SoulGasTokenConfig + InboxContractConfig + L1ScalarMultiplierConfig }   func (d *L2InitializationConfig) Check(log log.Logger) error { @@ -915,6 +927,33 @@ ProtocolVersionsProxy common.Address `json:"protocolVersionsProxy"` }   +// SoulGasTokenConfig configures the SoulGasToken deployment to L2. +type SoulGasTokenConfig struct { + // DeploySoulGasToken is a flag that indicates if the SGT contract will be deploy at genesis. + DeploySoulGasToken bool `json:"deploySoulGasToken,omitempty"` + // The time offset of the block at which the SoulGasToken is activated. + SoulGasTokenTimeOffset *hexutil.Uint64 `json:"soulGasTokenTimeOffset,omitempty"` + // IsSoulBackedByNative is a flag that indicates if the SoulGasToken is backed by native. + // Only effective when DeploySoulGasToken is true. + IsSoulBackedByNative bool `json:"isSoulBackedByNative,omitempty"` +} + +func (c *SoulGasTokenConfig) SoulGasTokenTime(genesisTime uint64) *uint64 { + return offsetToUpgradeTime(c.SoulGasTokenTimeOffset, genesisTime) +} + +// InboxContractConfig configures whether inbox contract is enabled. +// If enabled, the batcher tx will be further filtered by tx status. +type InboxContractConfig struct { + UseInboxContract bool `json:"useInboxContract,omitempty"` +} + +// L1ScalarMultiplierConfig configures the scalar multipliers for L1 base fee and blob base fee. +type L1ScalarMultiplierConfig struct { + L1BaseFeeScalarMultiplier uint64 `json:"l1BaseFeeScalarMultiplier,omitempty"` + L1BlobBaseFeeScalarMultiplier uint64 `json:"l1BlobBaseFeeScalarMultiplier,omitempty"` +} + // DependencyContext is the contextual configuration needed to verify the L1 dependencies, // used by DeployConfig.CheckAddresses. type DependencyContext struct { @@ -1069,10 +1108,22 @@ if d.SystemConfigProxy == (common.Address{}) { return nil, errors.New("SystemConfigProxy cannot be address(0)") }   + l1StartTime := l1StartBlock.Time + + soulGasTokenTime := d.SoulGasTokenTime(l1StartTime) + // The SGT contract will only be deployed if DeploySoulGasToken is true. + if !d.DeploySoulGasToken && soulGasTokenTime != nil { + return nil, fmt.Errorf("soulGasTokenTimeOffset is set, but DeploySoulGasToken is false") + } chainOpConfig := &params.OptimismConfig{ - EIP1559Elasticity: d.EIP1559Elasticity, - EIP1559Denominator: d.EIP1559Denominator, - EIP1559DenominatorCanyon: &d.EIP1559DenominatorCanyon, + EIP1559Elasticity: d.EIP1559Elasticity, + EIP1559Denominator: d.EIP1559Denominator, + EIP1559DenominatorCanyon: &d.EIP1559DenominatorCanyon, + L2BlobTime: d.L2BlobTime(l1StartTime), + SoulGasTokenTime: soulGasTokenTime, + IsSoulBackedByNative: d.IsSoulBackedByNative, + L1BaseFeeScalarMultiplier: d.L1BaseFeeScalarMultiplier, + L1BlobBaseFeeScalarMultiplier: d.L1BlobBaseFeeScalarMultiplier, }   var altDA *rollup.AltDAConfig @@ -1085,7 +1136,10 @@ DAResolveWindow: d.DAResolveWindow, } }   - l1StartTime := l1StartBlock.Time + var inboxContractConfig *rollup.InboxContractConfig + if d.UseInboxContract { + inboxContractConfig = &rollup.InboxContractConfig{UseInboxContract: true} + }   return &rollup.Config{ Genesis: rollup.Genesis{ @@ -1122,6 +1176,7 @@ JovianTime: d.JovianTime(l1StartTime), InteropTime: d.InteropTime(l1StartTime), ProtocolVersionsAddress: d.ProtocolVersionsProxy, AltDAConfig: altDA, + InboxContractConfig: inboxContractConfig, ChainOpConfig: chainOpConfig, }, nil }
diff --git official optimism/op-node/rollup/derive/blob_data_source.go SWC optimism/op-node/rollup/derive/blob_data_source.go index 2c4626941b8b51cb9477c9c08dd9f6e3fb01208a..fd2f7cef1236ea9e8e7c886bf7e78a934ae4b98a 100644 --- official optimism/op-node/rollup/derive/blob_data_source.go +++ SWC optimism/op-node/rollup/derive/blob_data_source.go @@ -27,13 +27,13 @@ data []blobOrCalldata ref eth.L1BlockRef batcherAddr common.Address dsCfg DataSourceConfig - fetcher L1TransactionFetcher + fetcher L1Fetcher 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 { +func NewBlobDataSource(ctx context.Context, log log.Logger, dsCfg DataSourceConfig, fetcher L1Fetcher, blobsFetcher L1BlobsFetcher, ref eth.L1BlockRef, batcherAddr common.Address) DataIter { return &BlobDataSource{ ref: ref, dsCfg: dsCfg, @@ -73,6 +73,51 @@ } return data, nil }   +// getTxSucceedMap returns a map indicating whether tx status is successful if useInboxContract; +// if !useInboxContract, nil map is returned to indicate that no status check is needed. +func getTxSucceedMap(ctx context.Context, useInboxContract bool, fetcher L1Fetcher, hash common.Hash) (txSucceeded map[common.Hash]bool, err error) { + if !useInboxContract { + return + } + _, receipts, err := fetcher.FetchReceipts(ctx, hash) + if err != nil { + return nil, NewTemporaryError(fmt.Errorf("failed to fetch L1 block info and receipts: %w", err)) + } + txSucceeded = make(map[common.Hash]bool) + for _, receipt := range receipts { + if receipt.Status == types.ReceiptStatusSuccessful { + txSucceeded[receipt.TxHash] = true + } + } + return +} + +// getTxSucceed returns all successful txs +func getTxSucceed(ctx context.Context, useInboxContract bool, fetcher L1Fetcher, hash common.Hash, txs types.Transactions) (successTxs types.Transactions, err error) { + if !useInboxContract { + // if !useInboxContract, all txs are considered successful + return txs, nil + } + _, receipts, err := fetcher.FetchReceipts(ctx, hash) + if err != nil { + return nil, NewTemporaryError(fmt.Errorf("failed to fetch L1 block info and receipts: %w", err)) + } + + txSucceeded := make(map[common.Hash]bool) + for _, receipt := range receipts { + if receipt.Status == types.ReceiptStatusSuccessful { + txSucceeded[receipt.TxHash] = true + } + } + successTxs = make(types.Transactions, 0) + for _, tx := range txs { + if _, ok := txSucceeded[tx.Hash()]; ok { + successTxs = append(successTxs, tx) + } + } + return successTxs, nil +} + // open fetches and returns the blob or calldata (as appropriate) from all valid batcher // transactions in the referenced block. Returns an empty (non-nil) array if no batcher // transactions are found. It returns ResetError if it cannot find the referenced block or a @@ -85,8 +130,12 @@ return nil, NewResetError(fmt.Errorf("failed to open blob data source: %w", err)) } return nil, NewTemporaryError(fmt.Errorf("failed to open blob data source: %w", err)) } + txSucceedMap, err := getTxSucceedMap(ctx, ds.dsCfg.useInboxContract, ds.fetcher, ds.ref.Hash) + if err != nil { + return nil, err + }   - data, hashes := dataAndHashesFromTxs(txs, &ds.dsCfg, ds.batcherAddr, ds.log) + data, hashes := dataAndHashesFromTxs(txs, &ds.dsCfg, ds.batcherAddr, ds.log, txSucceedMap)   if len(hashes) == 0 { // there are no blobs to fetch so we can return immediately @@ -115,13 +164,15 @@ // dataAndHashesFromTxs extracts calldata and datahashes from the input transactions and returns them. It // creates a placeholder blobOrCalldata element for each returned blob hash that must be populated // by fillBlobPointers after blob bodies are retrieved. -func dataAndHashesFromTxs(txs types.Transactions, config *DataSourceConfig, batcherAddr common.Address, logger log.Logger) ([]blobOrCalldata, []eth.IndexedBlobHash) { +func dataAndHashesFromTxs(txs types.Transactions, config *DataSourceConfig, batcherAddr common.Address, logger log.Logger, txSucceedMap map[common.Hash]bool) ([]blobOrCalldata, []eth.IndexedBlobHash) { data := []blobOrCalldata{} var hashes []eth.IndexedBlobHash blobIndex := 0 // index of each blob in the block's blob sidecar for _, tx := range txs { - // skip any non-batcher transactions - if !isValidBatchTx(tx, config.l1Signer, config.batchInboxAddress, batcherAddr, logger) { + // skip any non-batcher transactions or failed transactions + // blobIndex needs to be incremented for both invalid batch tx and failed tx + // if txSucceedMap is nil, it means no status check is needed. + if (!isValidBatchTx(tx, config.l1Signer, config.batchInboxAddress, batcherAddr, logger)) || (txSucceedMap != nil && !txSucceedMap[tx.Hash()]) { blobIndex += len(tx.BlobHashes()) continue }
diff --git official optimism/op-node/rollup/derive/blob_data_source_test.go SWC optimism/op-node/rollup/derive/blob_data_source_test.go index a20205544c478c35fe088d9bb782f504f5c6bf18..beb2aeadc5550b4f322c4b8418d1b8cbfec49591 100644 --- official optimism/op-node/rollup/derive/blob_data_source_test.go +++ SWC optimism/op-node/rollup/derive/blob_data_source_test.go @@ -45,7 +45,7 @@ Data: testutils.RandomData(rng, rng.Intn(1000)), } calldataTx, _ := types.SignNewTx(privateKey, signer, txData) txs := types.Transactions{calldataTx} - data, blobHashes := dataAndHashesFromTxs(txs, &config, batcherAddr, logger) + data, blobHashes := dataAndHashesFromTxs(txs, &config, batcherAddr, logger, nil) require.Equal(t, 1, len(data)) require.Equal(t, 0, len(blobHashes))   @@ -60,14 +60,14 @@ BlobHashes: []common.Hash{blobHash}, } blobTx, _ := types.SignNewTx(privateKey, signer, blobTxData) txs = types.Transactions{blobTx} - data, blobHashes = dataAndHashesFromTxs(txs, &config, batcherAddr, logger) + data, blobHashes = dataAndHashesFromTxs(txs, &config, batcherAddr, logger, nil) require.Equal(t, 1, len(data)) require.Equal(t, 1, len(blobHashes)) require.Nil(t, data[0].calldata)   // try again with both the blob & calldata transactions and make sure both are picked up txs = types.Transactions{blobTx, calldataTx} - data, blobHashes = dataAndHashesFromTxs(txs, &config, batcherAddr, logger) + data, blobHashes = dataAndHashesFromTxs(txs, &config, batcherAddr, logger, nil) require.Equal(t, 2, len(data)) require.Equal(t, 1, len(blobHashes)) require.NotNil(t, data[1].calldata) @@ -75,7 +75,7 @@ // make sure blob tx to the batch inbox is ignored if not signed by the batcher blobTx, _ = types.SignNewTx(testutils.RandomKey(), signer, blobTxData) txs = types.Transactions{blobTx} - data, blobHashes = dataAndHashesFromTxs(txs, &config, batcherAddr, logger) + data, blobHashes = dataAndHashesFromTxs(txs, &config, batcherAddr, logger, nil) require.Equal(t, 0, len(data)) require.Equal(t, 0, len(blobHashes))   @@ -84,7 +84,7 @@ // signature is valid. blobTxData.To = testutils.RandomAddress(rng) blobTx, _ = types.SignNewTx(privateKey, signer, blobTxData) txs = types.Transactions{blobTx} - data, blobHashes = dataAndHashesFromTxs(txs, &config, batcherAddr, logger) + data, blobHashes = dataAndHashesFromTxs(txs, &config, batcherAddr, logger, nil) require.Equal(t, 0, len(data)) require.Equal(t, 0, len(blobHashes))   @@ -98,9 +98,47 @@ } setCodeTx, err := types.SignNewTx(privateKey, signer, setCodeTxData) require.NoError(t, err) txs = types.Transactions{setCodeTx} - data, blobHashes = dataAndHashesFromTxs(txs, &config, batcherAddr, logger) + data, blobHashes = dataAndHashesFromTxs(txs, &config, batcherAddr, logger, nil) require.Equal(t, 0, len(data)) require.Equal(t, 0, len(blobHashes)) +} + +func TestBlockWithFailedBlobTx(t *testing.T) { + // test setup + rng := rand.New(rand.NewSource(12345)) + privateKey := testutils.InsecureRandomKey(rng) + publicKey, _ := privateKey.Public().(*ecdsa.PublicKey) + batcherAddr := crypto.PubkeyToAddress(*publicKey) + batchInboxAddr := testutils.RandomAddress(rng) + logger := testlog.Logger(t, log.LvlInfo) + + chainId := new(big.Int).SetUint64(rng.Uint64()) + signer := types.NewCancunSigner(chainId) + config := DataSourceConfig{ + l1Signer: signer, + batchInboxAddress: batchInboxAddr, + } + + // create two valid blob batcher txs + var txs types.Transactions + for i := 0; i < 2; i++ { + blobHash := testutils.RandomHash(rng) + blobTxData := &types.BlobTx{ + Nonce: rng.Uint64(), + Gas: 2_000_000, + To: batchInboxAddr, + Data: testutils.RandomData(rng, rng.Intn(1000)), + BlobHashes: []common.Hash{blobHash}, + } + blobTx, _ := types.SignNewTx(privateKey, signer, blobTxData) + txs = append(txs, blobTx) + } + + // mark the first blob tx as failed + txSucceedMap := map[common.Hash]bool{txs[1].Hash(): true} + _, blobHashes := dataAndHashesFromTxs(txs, &config, batcherAddr, logger, txSucceedMap) + // check the returned blob index is 1 + require.True(t, len(blobHashes) == 1 && blobHashes[0].Index == 1) }   func TestFillBlobPointers(t *testing.T) {
diff --git official optimism/op-node/rollup/derive/calldata_source.go SWC optimism/op-node/rollup/derive/calldata_source.go index 0e8147261e93e4235c33b93f1d56be2285d96297..776428dce37b8a9099177efb526185a60e1272c0 100644 --- official optimism/op-node/rollup/derive/calldata_source.go +++ SWC optimism/op-node/rollup/derive/calldata_source.go @@ -24,7 +24,7 @@ data []eth.Data // Required to re-attempt fetching ref eth.L1BlockRef dsCfg DataSourceConfig - fetcher L1TransactionFetcher + fetcher L1Fetcher log log.Logger   batcherAddr common.Address @@ -32,7 +32,7 @@ }   // NewCalldataSource 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 NewCalldataSource(ctx context.Context, log log.Logger, dsCfg DataSourceConfig, fetcher L1TransactionFetcher, ref eth.L1BlockRef, batcherAddr common.Address) DataIter { +func NewCalldataSource(ctx context.Context, log log.Logger, dsCfg DataSourceConfig, fetcher L1Fetcher, ref eth.L1BlockRef, batcherAddr common.Address) DataIter { _, txs, err := fetcher.InfoAndTxsByHash(ctx, ref.Hash) if err != nil { return &CalldataSource{ @@ -44,6 +44,17 @@ log: log, batcherAddr: batcherAddr, } } + txs, err = getTxSucceed(ctx, dsCfg.useInboxContract, fetcher, ref.Hash, txs) + if err != nil { + return &CalldataSource{ + open: false, + ref: ref, + dsCfg: dsCfg, + fetcher: fetcher, + log: log, + batcherAddr: batcherAddr, + } + } return &CalldataSource{ open: true, data: DataFromEVMTransactions(dsCfg, batcherAddr, txs, log.New("origin", ref)), @@ -56,6 +67,10 @@ // otherwise it returns a temporary error if fetching the block returns an error. 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 { + txs, err := getTxSucceed(ctx, ds.dsCfg.useInboxContract, ds.fetcher, ds.ref.Hash, txs) + if err != nil { + return nil, err + } ds.open = true ds.data = DataFromEVMTransactions(ds.dsCfg, ds.batcherAddr, txs, ds.log) } else if errors.Is(err, ethereum.NotFound) {
diff --git official optimism/op-node/rollup/derive/calldata_source_test.go SWC optimism/op-node/rollup/derive/calldata_source_test.go index 01b2616cca3fa66be0beeff9b4e9de814f492120..31555996ddbe354a81ba2982585c18be1344c654 100644 --- official optimism/op-node/rollup/derive/calldata_source_test.go +++ SWC optimism/op-node/rollup/derive/calldata_source_test.go @@ -121,7 +121,7 @@ expectedData = append(expectedData, txs[i].Data()) } }   - out := DataFromEVMTransactions(DataSourceConfig{cfg.L1Signer(), cfg.BatchInboxAddress, false}, batcherAddr, txs, testlog.Logger(t, log.LevelCrit)) + out := DataFromEVMTransactions(DataSourceConfig{cfg.L1Signer(), cfg.BatchInboxAddress, false, false}, batcherAddr, txs, testlog.Logger(t, log.LevelCrit)) require.ElementsMatch(t, expectedData, out) }
diff --git official optimism/op-node/rollup/derive/data_source.go SWC optimism/op-node/rollup/derive/data_source.go index dfeda599501a1970dca6a52a15f99e867d0fb992..a86e0de4b98eafcb68f6bd883f2c21f5d8b2b46f 100644 --- official optimism/op-node/rollup/derive/data_source.go +++ SWC optimism/op-node/rollup/derive/data_source.go @@ -52,6 +52,7 @@ config := DataSourceConfig{ l1Signer: cfg.L1Signer(), batchInboxAddress: cfg.BatchInboxAddress, altDAEnabled: cfg.AltDAEnabled(), + useInboxContract: cfg.UseInboxContract(), } return &DataSourceFactory{ log: log, @@ -88,6 +89,7 @@ type DataSourceConfig struct { l1Signer types.Signer batchInboxAddress common.Address altDAEnabled bool + useInboxContract bool }   // isValidBatchTx returns true if:
diff --git official optimism/op-node/rollup/types.go SWC optimism/op-node/rollup/types.go index 0d64e0331187f70fe710ec40d31ecac121bc3fe2..2fc832445d96861c98b9d76157d91c9d6c9d3faf 100644 --- official optimism/op-node/rollup/types.go +++ SWC optimism/op-node/rollup/types.go @@ -152,6 +152,8 @@ // parameters to the protocol values, like the execution layer does. // If missing, it is loaded by the op-node from the embedded superchain config at startup. ChainOpConfig *params.OptimismConfig `json:"chain_op_config,omitempty"`   + InboxContractConfig *InboxContractConfig `json:"inbox_contract_config,omitempty"` + // Optional Features   // AltDAConfig. We are in the process of migrating to the AltDAConfig from these legacy top level values @@ -164,6 +166,29 @@ // blob parameters for the first L1 Prague block, as was intended. // This feature (de)activates by L1 origin timestamp, to keep a consistent L1 block info per L2 // epoch. PectraBlobScheduleTime *uint64 `json:"pectra_blob_schedule_time,omitempty"` +} + +type L2BlobConfig struct { + L2BlobTime *uint64 `json:"l2BlobTime,omitempty"` +} + +type InboxContractConfig struct { + UseInboxContract bool `json:"use_inbox_contract,omitempty"` +} + +// IsL2Blob returns whether l2 blob is enabled +func (cfg *Config) IsL2Blob(timestamp uint64) bool { + return cfg.IsL2BlobTimeSet() && *cfg.ChainOpConfig.L2BlobTime <= timestamp +} + +// UseInboxContract returns whether inbox contract is enabled +func (cfg *Config) UseInboxContract() bool { + return cfg.InboxContractConfig != nil && cfg.InboxContractConfig.UseInboxContract +} + +// IsL2BlobTimeSet returns whether l2 blob activation time is set +func (cfg *Config) IsL2BlobTimeSet() bool { + return cfg.ChainOpConfig != nil && cfg.ChainOpConfig.L2BlobTime != nil }   // ValidateL1Config checks L1 config variables for errors. @@ -773,6 +798,12 @@ banner += "Post-Bedrock Network Upgrades (timestamp based):\n" c.forEachFork(func(name string, _ string, time *uint64) { banner += fmt.Sprintf(" - %v: %s\n", name, fmtForkTimeOrUnset(time)) }) + var l2BlobTime *uint64 + if c.ChainOpConfig != nil { + l2BlobTime = c.ChainOpConfig.L2BlobTime + } + banner += fmt.Sprintf(" - L2Blob: %s\n", fmtForkTimeOrUnset(l2BlobTime)) + banner += fmt.Sprintf(" - Use inbox contract: %v\n", c.UseInboxContract()) // Report the protocol version banner += fmt.Sprintf("Node supports up to OP-Stack Protocol Version: %s\n", OPStackSupport) if c.AltDAConfig != nil { @@ -815,6 +846,13 @@ }) if c.AltDAConfig != nil { ctx = append(ctx, "alt_da", *c.AltDAConfig) } + + var l2BlobTime *uint64 + if c.ChainOpConfig != nil { + l2BlobTime = c.ChainOpConfig.L2BlobTime + } + ctx = append(ctx, "l2_blob_config", fmtForkTimeOrUnset(l2BlobTime)) + ctx = append(ctx, "use_inbox_contract", c.UseInboxContract()) log.Info("Rollup Config", ctx...) }

To bridge the gap between traditional web users and the growing world of Web3, we propose a non-transferable gas token named Soul Gas Token (SGT), where the gas token is the native token of a Rollup.

The concept revolves around facilitating Web2 users’ entry into Web3 by airdropping them with SGT.

This token will enable users to pay for transaction gas fees without the immediate selling pressure of the airdropped token.

This initiative is particularly aimed at those new to Web3, providing a seamless transition without the upfront cost of acquiring a gas token.

diff --git official optimism/op-chain-ops/genesis/genesis.go SWC optimism/op-chain-ops/genesis/genesis.go index 306f70df2dd5d999f5601c36f16cb9c69cb0e497..94f02286d53acc62d067d23fad2d66ddb3d4cff2 100644 --- official optimism/op-chain-ops/genesis/genesis.go +++ SWC optimism/op-chain-ops/genesis/genesis.go @@ -46,6 +46,12 @@ }   l1StartTime := l1StartHeader.Time   + soulGasTokenTime := config.SoulGasTokenTime(l1StartTime) + // The SGT contract will only be deployed if DeploySoulGasToken is true. + if !config.DeploySoulGasToken && soulGasTokenTime != nil { + return nil, fmt.Errorf("soulGasTokenTimeOffset is set, but DeploySoulGasToken is false") + } + optimismChainConfig := params.ChainConfig{ ChainID: new(big.Int).SetUint64(config.L2ChainID), HomesteadBlock: big.NewInt(0), @@ -79,9 +85,14 @@ JovianTime: config.JovianTime(l1StartTime), PragueTime: config.IsthmusTime(l1StartTime), InteropTime: config.InteropTime(l1StartTime), Optimism: &params.OptimismConfig{ - EIP1559Denominator: eip1559Denom, - EIP1559Elasticity: eip1559Elasticity, - EIP1559DenominatorCanyon: &eip1559DenomCanyon, + EIP1559Denominator: eip1559Denom, + EIP1559Elasticity: eip1559Elasticity, + EIP1559DenominatorCanyon: &eip1559DenomCanyon, + L2BlobTime: config.L2BlobTime(l1StartTime), + IsSoulBackedByNative: config.IsSoulBackedByNative, + SoulGasTokenTime: soulGasTokenTime, + L1BaseFeeScalarMultiplier: config.L1BaseFeeScalarMultiplier, + L1BlobBaseFeeScalarMultiplier: config.L1BlobBaseFeeScalarMultiplier, }, }
diff --git official optimism/op-service/predeploys/addresses.go SWC optimism/op-service/predeploys/addresses.go index e461f0c86faaf596bbd3ec9914e8b5ee6dd21b93..de20083e3d0aa4c675d82936e82525baab74e15d 100644 --- official optimism/op-service/predeploys/addresses.go +++ SWC optimism/op-service/predeploys/addresses.go @@ -26,6 +26,7 @@ L1FeeVault = "0x420000000000000000000000000000000000001a" OperatorFeeVault = "0x420000000000000000000000000000000000001b" SchemaRegistry = "0x4200000000000000000000000000000000000020" EAS = "0x4200000000000000000000000000000000000021" + SoulGasToken = "0x4200000000000000000000000000000000000800" CrossL2Inbox = "0x4200000000000000000000000000000000000022" L2toL2CrossDomainMessenger = "0x4200000000000000000000000000000000000023" SuperchainETHBridge = "0x4200000000000000000000000000000000000024" @@ -67,6 +68,7 @@ L1FeeVaultAddr = common.HexToAddress(L1FeeVault) OperatorFeeVaultAddr = common.HexToAddress(OperatorFeeVault) SchemaRegistryAddr = common.HexToAddress(SchemaRegistry) EASAddr = common.HexToAddress(EAS) + SoulGasTokenAddr = common.HexToAddress(SoulGasToken) CrossL2InboxAddr = common.HexToAddress(CrossL2Inbox) L2toL2CrossDomainMessengerAddr = common.HexToAddress(L2toL2CrossDomainMessenger) SuperchainETHBridgeAddr = common.HexToAddress(SuperchainETHBridge) @@ -122,6 +124,7 @@ Predeploys["L1FeeVault"] = &Predeploy{Address: L1FeeVaultAddr} Predeploys["OperatorFeeVault"] = &Predeploy{Address: OperatorFeeVaultAddr} Predeploys["SchemaRegistry"] = &Predeploy{Address: SchemaRegistryAddr} Predeploys["EAS"] = &Predeploy{Address: EASAddr} + Predeploys["SoulGasToken"] = &Predeploy{Address: SoulGasTokenAddr} Predeploys["Create2Deployer"] = &Predeploy{ Address: Create2DeployerAddr, ProxyDisabled: true,
diff --git official optimism/packages/contracts-bedrock/interfaces/L2/ISoulGasToken.sol SWC optimism/packages/contracts-bedrock/interfaces/L2/ISoulGasToken.sol new file mode 100644 index 0000000000000000000000000000000000000000..096b42f7a41172bb0aacba7d2121bfa4d1f9939d --- /dev/null +++ SWC optimism/packages/contracts-bedrock/interfaces/L2/ISoulGasToken.sol @@ -0,0 +1,15 @@ +// SPDX-License-Identifier: MIT +pragma solidity ^0.8.0; + +/// @title ISoulGasToken +/// @notice The interface for the SoulGasToken. +interface ISoulGasToken { + function initialize(string memory _name, string memory _symbol, address _owner) external; + + function __constructor__(bool _isBackedByNative) external; + + function name() external view returns (string memory); + function symbol() external view returns (string memory); + function owner() external view returns (address); + function admin() external view returns (address); +}
diff --git official optimism/packages/contracts-bedrock/scripts/Artifacts.s.sol SWC optimism/packages/contracts-bedrock/scripts/Artifacts.s.sol index 5858c8aa4c59705788b859231522aac25ede2e79..7d5471ba49b73c35e4b9d55640403a86094bc572 100644 --- official optimism/packages/contracts-bedrock/scripts/Artifacts.s.sol +++ SWC optimism/packages/contracts-bedrock/scripts/Artifacts.s.sol @@ -80,6 +80,8 @@ bytes32 digest = keccak256(bytes(_name)); if (digest == keccak256(bytes("L2CrossDomainMessenger"))) { return payable(Predeploys.L2_CROSS_DOMAIN_MESSENGER); + } else if (digest == keccak256(bytes("SoulGasToken"))) { + return payable(Predeploys.SOUL_GAS_TOKEN); } else if (digest == keccak256(bytes("L2ToL1MessagePasser"))) { return payable(Predeploys.L2_TO_L1_MESSAGE_PASSER); } else if (digest == keccak256(bytes("L2StandardBridge"))) {
diff --git official optimism/packages/contracts-bedrock/scripts/L2Genesis.s.sol SWC optimism/packages/contracts-bedrock/scripts/L2Genesis.s.sol index 5708dfe33c7a16c76bdff5bbbd7798d8d422182e..ba05e3632f630642c2bf0414524483a0978543a0 100644 --- official optimism/packages/contracts-bedrock/scripts/L2Genesis.s.sol +++ SWC optimism/packages/contracts-bedrock/scripts/L2Genesis.s.sol @@ -31,6 +31,8 @@ import { IL2CrossDomainMessenger } from "interfaces/L2/IL2CrossDomainMessenger.sol"; import { IGasPriceOracle } from "interfaces/L2/IGasPriceOracle.sol"; import { IL1Block } from "interfaces/L2/IL1Block.sol";   +import { SoulGasToken } from "src/L2/SoulGasToken.sol"; + /// @title L2Genesis /// @notice Generates the genesis state for the L2 network. /// The following safety invariants are used when setting state: @@ -60,6 +62,8 @@ uint256 fork; bool deployCrossL2Inbox; bool enableGovernance; bool fundDevAccounts; + bool deploySoulGasToken; + bool isSoulBackedByNative; }   using ForkUtils for Fork; @@ -231,6 +235,7 @@ setOperatorFeeVault(); // 1B // 1C,1D,1E,1F: not used. setSchemaRegistry(); // 20 setEAS(); // 21 + if (_input.deploySoulGasToken) setSoulGasToken(_input); // 800 setGovernanceToken(_input); // 42: OP (not behind a proxy) if (_input.fork >= uint256(Fork.INTEROP)) { if (_input.deployCrossL2Inbox) { @@ -256,6 +261,25 @@ }   function setL2ToL1MessagePasser() internal { _setImplementationCode(Predeploys.L2_TO_L1_MESSAGE_PASSER); + } + + /// @notice This predeploy is following the safety invariant #2. + function setSoulGasToken(Input memory _input) public { + address impl = Predeploys.predeployToCodeNamespace(Predeploys.SOUL_GAS_TOKEN); + + SoulGasToken token = new SoulGasToken({ _isBackedByNative: _input.isSoulBackedByNative }); + vm.etch(impl, address(token).code); + + /// Reset so its not included state dump + vm.etch(address(token), ""); + vm.resetNonce(address(token)); + + SoulGasToken(impl).initialize({ _name: "", _symbol: "", _owner: _input.opChainProxyAdminOwner }); + SoulGasToken(Predeploys.SOUL_GAS_TOKEN).initialize({ + _name: "SoulQKC", + _symbol: "SoulQKC", + _owner: _input.opChainProxyAdminOwner + }); }   /// @notice This predeploy is following the safety invariant #1.
diff --git official optimism/packages/contracts-bedrock/scripts/deploy/DeployConfig.s.sol SWC optimism/packages/contracts-bedrock/scripts/deploy/DeployConfig.s.sol index 928faad8129bc2cc60915e1aea2a68f8b6e9e995..bfda16442e540dfd91d493441d8b45d598cc9e80 100644 --- official optimism/packages/contracts-bedrock/scripts/deploy/DeployConfig.s.sol +++ SWC optimism/packages/contracts-bedrock/scripts/deploy/DeployConfig.s.sol @@ -83,6 +83,8 @@ uint256 public faultGameV2ClockExtension; uint256 public faultGameV2MaxClockDuration;   bool public useInterop; + bool public deploySoulGasToken; + bool public isSoulBackedByNative; bool public useUpgradedFork; bytes32 public devFeatureBitmap;   @@ -158,6 +160,8 @@ daBondSize = _readOr(_json, "$.daBondSize", 1000000000); daResolverRefundPercentage = _readOr(_json, "$.daResolverRefundPercentage", 0);   useInterop = _readOr(_json, "$.useInterop", false); + deploySoulGasToken = _readOr(_json, "$.deploySoulGasToken", false); + isSoulBackedByNative = _readOr(_json, "$.isSoulBackedByNative", false); devFeatureBitmap = bytes32(_readOr(_json, "$.devFeatureBitmap", 0)); useUpgradedFork; faultGameV2MaxGameDepth = _readOr(_json, "$.faultGameV2MaxGameDepth", 73); @@ -213,6 +217,16 @@ /// @notice Allow the `useInterop` config to be overridden in testing environments function setUseInterop(bool _useInterop) public { useInterop = _useInterop; + } + + /// @notice Allow the `deploySoulGasToken` config to be overridden in testing environments + function setDeploySoulGasToken(bool _deploySoulGasToken) public { + deploySoulGasToken = _deploySoulGasToken; + } + + /// @notice Allow the `isSoulBackedByNative` config to be overridden in testing environments + function setIsSoulBackedByNative(bool _isSoulBackedByNative) public { + isSoulBackedByNative = _isSoulBackedByNative; }   /// @notice Allow the `fundDevAccounts` config to be overridden.
diff --git official optimism/packages/contracts-bedrock/src/L2/SoulGasToken.sol SWC optimism/packages/contracts-bedrock/src/L2/SoulGasToken.sol new file mode 100644 index 0000000000000000000000000000000000000000..5a23f83bcd0d1935ed3599f98a2435008ec6a4ed --- /dev/null +++ SWC optimism/packages/contracts-bedrock/src/L2/SoulGasToken.sol @@ -0,0 +1,266 @@ +// SPDX-License-Identifier: MIT +pragma solidity 0.8.15; + +import { ERC20Upgradeable } from "@openzeppelin/contracts-upgradeable/token/ERC20/ERC20Upgradeable.sol"; +import { OwnableUpgradeable } from "@openzeppelin/contracts-upgradeable/access/OwnableUpgradeable.sol"; + +/// @title SoulGasToken +/// @notice The SoulGasToken is a soul-bounded ERC20 contract which can be used to pay gas on L2. +/// It has 2 modes: +/// 1. when IS_BACKED_BY_NATIVE(or in other words: SoulQKC mode), the token can be minted by +/// anyone depositing native token into the contract. +/// 2. when !IS_BACKED_BY_NATIVE(or in other words: SoulETH mode), the token can only be +/// minted by whitelist minters specified by contract owner. +contract SoulGasToken is ERC20Upgradeable, OwnableUpgradeable { + /// @custom:storage-location erc7201:openzeppelin.storage.SoulGasToken + struct SoulGasTokenStorage { + // minters are whitelist EOAs, only used when !IS_BACKED_BY_NATIVE + mapping(address => bool) minters; + // burners are whitelist EOAs to burn/withdraw SoulGasToken + mapping(address => bool) burners; + // allowSgtValue are whitelist contracts to consume sgt as msg.value + // when IS_BACKED_BY_NATIVE + mapping(address => bool) allowSgtValue; + } + + /// @notice Emitted when sgt as msg.value is enabled for a contract. + /// @param from Address of the contract for which sgt as msg.value is enabled. + event AllowSgtValue(address indexed from); + /// @notice Emitted when sgt as msg.value is disabled for a contract. + /// @param from Address of the contract for which sgt as msg.value is disabled. + event DisallowSgtValue(address indexed from); + + event BurnerAdded(address indexed burner); + event BurnerDeleted(address indexed burner); + event MinterAdded(address indexed minter); + event MinterDeleted(address indexed minter); + + // keccak256(abi.encode(uint256(keccak256("openzeppelin.storage.SoulGasToken")) - 1)) & ~bytes32(uint256(0xff)) + bytes32 private constant _SOULGASTOKEN_STORAGE_LOCATION = + 0x135c38e215d95c59dcdd8fe622dccc30d04cacb8c88c332e4e7441bac172dd00; + + bool internal immutable IS_BACKED_BY_NATIVE; + + function _getSoulGasTokenStorage() private pure returns (SoulGasTokenStorage storage $) { + assembly { + $.slot := _SOULGASTOKEN_STORAGE_LOCATION + } + } + + constructor(bool _isBackedByNative) { + IS_BACKED_BY_NATIVE = _isBackedByNative; + initialize("", "", msg.sender); + } + + /// @notice Initializer. + function initialize(string memory _name, string memory _symbol, address _owner) public initializer { + __Ownable_init(); + transferOwnership(_owner); + + // initialize the inherited ERC20Upgradeable + __ERC20_init(_name, _symbol); + } + + /// @notice deposit can be called by anyone to deposit native token for SoulGasToken when + /// IS_BACKED_BY_NATIVE. + function deposit() external payable { + require(IS_BACKED_BY_NATIVE, "SGT: deposit should only be called when IS_BACKED_BY_NATIVE"); + + _mint(_msgSender(), msg.value); + } + + /// @notice batchDepositFor can be called by anyone to deposit native token for SoulGasToken in batch when + /// IS_BACKED_BY_NATIVE. + function batchDepositFor(address[] calldata _accounts, uint256[] calldata _values) external payable { + require(_accounts.length == _values.length, "SGT: invalid arguments"); + + require(IS_BACKED_BY_NATIVE, "SGT: batchDepositFor should only be called when IS_BACKED_BY_NATIVE"); + + uint256 totalValue = 0; + for (uint256 i = 0; i < _accounts.length; i++) { + _mint(_accounts[i], _values[i]); + totalValue += _values[i]; + } + require(msg.value == totalValue, "SGT: unexpected msg.value"); + } + + /// @notice batchDepositForAll is similar to batchDepositFor, but the value is the same for all accounts. + function batchDepositForAll(address[] calldata _accounts, uint256 _value) external payable { + require(IS_BACKED_BY_NATIVE, "SGT: batchDepositForAll should only be called when IS_BACKED_BY_NATIVE"); + + for (uint256 i = 0; i < _accounts.length; i++) { + _mint(_accounts[i], _value); + } + require(msg.value == _value * _accounts.length, "SGT: unexpected msg.value"); + } + + /// @notice withdrawFrom is called by the burner to burn SoulGasToken and return the native token when + /// IS_BACKED_BY_NATIVE. + function withdrawFrom(address _account, uint256 _value) external { + require(IS_BACKED_BY_NATIVE, "SGT: withdrawFrom should only be called when IS_BACKED_BY_NATIVE"); + + SoulGasTokenStorage storage $ = _getSoulGasTokenStorage(); + require($.burners[_msgSender()], "SGT: not the burner"); + + _burn(_account, _value); + payable(_msgSender()).transfer(_value); + } + + /// @notice batchWithdrawFrom is the batch version of withdrawFrom. + function batchWithdrawFrom(address[] calldata _accounts, uint256[] calldata _values) external { + require(_accounts.length == _values.length, "SGT: invalid arguments"); + + require(IS_BACKED_BY_NATIVE, "SGT: batchWithdrawFrom should only be called when IS_BACKED_BY_NATIVE"); + + SoulGasTokenStorage storage $ = _getSoulGasTokenStorage(); + require($.burners[_msgSender()], "SGT: not the burner"); + + uint256 totalValue = 0; + for (uint256 i = 0; i < _accounts.length; i++) { + _burn(_accounts[i], _values[i]); + totalValue += _values[i]; + } + + payable(_msgSender()).transfer(totalValue); + } + + /// @notice batchMint is called: + /// 1. by EOA minters to mint SoulGasToken in batch when !IS_BACKED_BY_NATIVE. + function batchMint(address[] calldata _accounts, uint256[] calldata _values) external { + // we don't explicitly check !IS_BACKED_BY_NATIVE here, because if IS_BACKED_BY_NATIVE, + // there's no way to add a minter. + require(_accounts.length == _values.length, "SGT: invalid arguments"); + + SoulGasTokenStorage storage $ = _getSoulGasTokenStorage(); + require($.minters[_msgSender()], "SGT: not a minter"); + + for (uint256 i = 0; i < _accounts.length; i++) { + _mint(_accounts[i], _values[i]); + } + } + + /// @notice addMinters is called by the owner to add minters when !IS_BACKED_BY_NATIVE. + function addMinters(address[] calldata _minters) external onlyOwner { + require(!IS_BACKED_BY_NATIVE, "SGT: addMinters should only be called when !IS_BACKED_BY_NATIVE"); + SoulGasTokenStorage storage $ = _getSoulGasTokenStorage(); + uint256 i; + for (i = 0; i < _minters.length; i++) { + $.minters[_minters[i]] = true; + emit MinterAdded(_minters[i]); + } + } + + /// @notice delMinters is called by the owner to delete minters when !IS_BACKED_BY_NATIVE. + function delMinters(address[] calldata _minters) external onlyOwner { + require(!IS_BACKED_BY_NATIVE, "SGT: delMinters should only be called when !IS_BACKED_BY_NATIVE"); + SoulGasTokenStorage storage $ = _getSoulGasTokenStorage(); + uint256 i; + for (i = 0; i < _minters.length; i++) { + delete $.minters[_minters[i]]; + emit MinterDeleted(_minters[i]); + } + } + + /// @notice addBurners is called by the owner to add burners. + function addBurners(address[] calldata _burners) external onlyOwner { + SoulGasTokenStorage storage $ = _getSoulGasTokenStorage(); + uint256 i; + for (i = 0; i < _burners.length; i++) { + $.burners[_burners[i]] = true; + emit BurnerAdded(_burners[i]); + } + } + + /// @notice delBurners is called by the owner to delete burners. + function delBurners(address[] calldata _burners) external onlyOwner { + SoulGasTokenStorage storage $ = _getSoulGasTokenStorage(); + uint256 i; + for (i = 0; i < _burners.length; i++) { + delete $.burners[_burners[i]]; + emit BurnerDeleted(_burners[i]); + } + } + + /// @notice allowSgtValue is called by the owner to enable whitelist contracts to consume sgt as msg.value + function allowSgtValue(address[] calldata _contracts) external onlyOwner { + require(IS_BACKED_BY_NATIVE, "SGT: allowSgtValue should only be called when IS_BACKED_BY_NATIVE"); + SoulGasTokenStorage storage $ = _getSoulGasTokenStorage(); + uint256 i; + for (i = 0; i < _contracts.length; i++) { + $.allowSgtValue[_contracts[i]] = true; + emit AllowSgtValue(_contracts[i]); + } + } + + /// @notice allowSgtValue is called by the owner to disable whitelist contracts to consume sgt as msg.value + function disallowSgtValue(address[] calldata _contracts) external onlyOwner { + require(IS_BACKED_BY_NATIVE, "SGT: disallowSgtValue should only be called when IS_BACKED_BY_NATIVE"); + SoulGasTokenStorage storage $ = _getSoulGasTokenStorage(); + uint256 i; + for (i = 0; i < _contracts.length; i++) { + $.allowSgtValue[_contracts[i]] = false; + emit DisallowSgtValue(_contracts[i]); + } + } + + /// @notice chargeFromOrigin is called when IS_BACKED_BY_NATIVE to charge for native balance + /// from tx.origin if caller is whitelisted. + function chargeFromOrigin(uint256 _amount) external returns (uint256 amountCharged_) { + require(IS_BACKED_BY_NATIVE, "SGT: chargeFromOrigin should only be called when IS_BACKED_BY_NATIVE"); + SoulGasTokenStorage storage $ = _getSoulGasTokenStorage(); + require($.allowSgtValue[_msgSender()], "SGT: caller is not whitelisted"); + uint256 balance = balanceOf(tx.origin); + if (balance == 0) { + amountCharged_ = 0; + return amountCharged_; + } + if (balance >= _amount) { + amountCharged_ = _amount; + } else { + amountCharged_ = balance; + } + _burn(tx.origin, amountCharged_); + payable(_msgSender()).transfer(amountCharged_); + } + + /// @notice burnFrom is called when !IS_BACKED_BY_NATIVE: + /// 1. by the burner to burn SoulGasToken. + function burnFrom(address _account, uint256 _value) external { + require(!IS_BACKED_BY_NATIVE, "SGT: burnFrom should only be called when !IS_BACKED_BY_NATIVE"); + SoulGasTokenStorage storage $ = _getSoulGasTokenStorage(); + require($.burners[_msgSender()], "SGT: not the burner"); + _burn(_account, _value); + } + + /// @notice batchBurnFrom is the batch version of burnFrom. + function batchBurnFrom(address[] calldata _accounts, uint256[] calldata _values) external { + require(_accounts.length == _values.length, "SGT: invalid arguments"); + require(!IS_BACKED_BY_NATIVE, "SGT: batchBurnFrom should only be called when !IS_BACKED_BY_NATIVE"); + SoulGasTokenStorage storage $ = _getSoulGasTokenStorage(); + require($.burners[_msgSender()], "SGT: not the burner"); + + for (uint256 i = 0; i < _accounts.length; i++) { + _burn(_accounts[i], _values[i]); + } + } + + /// @notice transferFrom is disabled for SoulGasToken. + function transfer(address, uint256) public virtual override returns (bool) { + revert("SGT: transfer is disabled for SoulGasToken"); + } + + /// @notice transferFrom is disabled for SoulGasToken. + function transferFrom(address, address, uint256) public virtual override returns (bool) { + revert("SGT: transferFrom is disabled for SoulGasToken"); + } + + /// @notice approve is disabled for SoulGasToken. + function approve(address, uint256) public virtual override returns (bool) { + revert("SGT: approve is disabled for SoulGasToken"); + } + + /// @notice Returns whether SoulGasToken is backed by native token. + function isBackedByNative() external view returns (bool) { + return IS_BACKED_BY_NATIVE; + } +}
diff --git official optimism/packages/contracts-bedrock/src/libraries/Predeploys.sol SWC optimism/packages/contracts-bedrock/src/libraries/Predeploys.sol index baeb6a143575c6bcf59876f160edf7a21169914c..859a26cbac2508e4652af200816d469a623a59f0 100644 --- official optimism/packages/contracts-bedrock/src/libraries/Predeploys.sol +++ SWC optimism/packages/contracts-bedrock/src/libraries/Predeploys.sol @@ -8,7 +8,7 @@ /// @notice Contains constant addresses for protocol contracts that are pre-deployed to the L2 system. // This excludes the preinstalls (non-protocol contracts). library Predeploys { /// @notice Number of predeploy-namespace addresses reserved for protocol usage. - uint256 internal constant PREDEPLOY_COUNT = 2048; + uint256 internal constant PREDEPLOY_COUNT = 4096;   /// @custom:legacy /// @notice Address of the LegacyMessagePasser predeploy. Deprecate. Use the updated @@ -78,6 +78,9 @@ address internal constant SCHEMA_REGISTRY = 0x4200000000000000000000000000000000000020;   /// @notice Address of the EAS predeploy. address internal constant EAS = 0x4200000000000000000000000000000000000021; + + /// @notice Address of the SOUL_GAS_TOKEN predeploy. + address internal constant SOUL_GAS_TOKEN = 0x4200000000000000000000000000000000000800;   /// @notice Address of the GovernanceToken predeploy. address internal constant GOVERNANCE_TOKEN = 0x4200000000000000000000000000000000000042; @@ -145,6 +148,7 @@ if (_addr == ETH_LIQUIDITY) return "ETHLiquidity"; if (_addr == OPTIMISM_SUPERCHAIN_ERC20_FACTORY) return "OptimismSuperchainERC20Factory"; if (_addr == OPTIMISM_SUPERCHAIN_ERC20_BEACON) return "OptimismSuperchainERC20Beacon"; if (_addr == SUPERCHAIN_TOKEN_BRIDGE) return "SuperchainTokenBridge"; + if (_addr == SOUL_GAS_TOKEN) return "SoulGasToken"; revert("Predeploys: unnamed predeploy"); }   @@ -171,11 +175,11 @@ || _addr == OPTIMISM_MINTABLE_ERC721_FACTORY || _addr == PROXY_ADMIN || _addr == BASE_FEE_VAULT || _addr == L1_FEE_VAULT || _addr == OPERATOR_FEE_VAULT || _addr == SCHEMA_REGISTRY || _addr == EAS || _addr == GOVERNANCE_TOKEN || (_fork >= uint256(Fork.INTEROP) && _enableCrossL2Inbox && _addr == CROSS_L2_INBOX) - || (_fork >= uint256(Fork.INTEROP) && _addr == L2_TO_L2_CROSS_DOMAIN_MESSENGER); + || (_fork >= uint256(Fork.INTEROP) && _addr == L2_TO_L2_CROSS_DOMAIN_MESSENGER) || _addr == SOUL_GAS_TOKEN; }   function isPredeployNamespace(address _addr) internal pure returns (bool) { - return uint160(_addr) >> 11 == uint160(0x4200000000000000000000000000000000000000) >> 11; + return uint160(_addr) >> 12 == uint160(0x4200000000000000000000000000000000000000) >> 12; }   /// @notice Function to compute the expected address of the predeploy implementation

The Ethereum Cancun upgrade has significantly reduced Layer 2 (L2) data uploading costs by introducing BLOB transactions to Layer 1 (L1). This innovation has also enabled a variety of additional applications based on the BLOBs due to their low cost, such as blob.fm, EthStorage, and Ethscriptions. However, while the data upload costs have decreased, the execution costs on L1 remain high compared to L2, leading to high costs for L2 state proposals and non-financial applications that rely on BLOBs.

To address these challenges, the L2 BLOB feature introduces support for BLOB transactions on L2. enabling L3 solutions that settle on L2 to use an enshrined 4844-compatible DA layer without needing to integrate third-party DA providers or deal with the security risks associated with DA bridges. Additionally, the applications mentioned above could migrate to L2 with minimal costs.

diff --git official optimism/op-node/flags/flags.go SWC optimism/op-node/flags/flags.go index e72383527234ae3fb69939ad5b5022ab3216ba22..8a9abf056e31bd26eb1533750005cc98c2b87c24 100644 --- official optimism/op-node/flags/flags.go +++ SWC optimism/op-node/flags/flags.go @@ -368,6 +368,12 @@ EnvVars: prefixEnvVars("CONDUCTOR_RPC_TIMEOUT"), Value: time.Second * 1, Category: SequencerCategory, } + DACUrlsFlag = &cli.StringFlag{ + Name: "dac.urls", + Usage: "dac urls for sequencer when l2 blob is enabled", + EnvVars: prefixEnvVars("DAC_URLS"), + Category: SequencerCategory, + } /* Interop flags, experimental. */ InteropRPCAddr = &cli.StringFlag{ Name: "interop.rpc.addr", @@ -465,6 +471,7 @@ ConductorRpcTimeoutFlag, SafeDBPath, L1ChainConfig, L2EngineKind, + DACUrlsFlag, L2EngineRpcTimeout, InteropRPCAddr, InteropRPCPort,
diff --git official optimism/op-node/node/node.go SWC optimism/op-node/node/node.go index b0ea946651d8e1a742fb64a44c50ecf208a65b95..13dd821b7746e0cac8018537c145259017401712 100644 --- official optimism/op-node/node/node.go +++ SWC optimism/op-node/node/node.go @@ -583,12 +583,13 @@ } else { safeDB = safedb.Disabled }   + dacClient := cfg.DACConfig.Client() if cfg.Rollup.ChainOpConfig == nil { return nil, nil, nil, nil, fmt.Errorf("cfg.Rollup.ChainOpConfig is nil. Please see https://github.com/ethereum-optimism/optimism/releases/tag/op-node/v1.11.0: %w", err) }   l2Driver := driver.NewDriver(node.eventSys, node.eventDrain, &cfg.Driver, &cfg.Rollup, cfg.L1ChainConfig, cfg.DependencySet, l2Source, node.l1Source, - node.beacon, node, node, node.log, node.metrics, cfg.ConfigPersistence, safeDB, &cfg.Sync, sequencerConductor, altDA, indexingMode) + node.beacon, node, node, node.log, node.metrics, cfg.ConfigPersistence, safeDB, &cfg.Sync, sequencerConductor, altDA, indexingMode, dacClient)   // Wire up IndexingMode to engine controller for direct procedure call if sys != nil {
diff --git official optimism/op-node/p2p/gossip.go SWC optimism/op-node/p2p/gossip.go index 2a8442a9d238f931145889ae920742c902f896c1..9d4b2a5c704904a2c7ff2b1071ff31bca4b93c14 100644 --- official optimism/op-node/p2p/gossip.go +++ SWC optimism/op-node/p2p/gossip.go @@ -334,7 +334,7 @@ now := uint64(time.Now().Unix())   // [REJECT] if the `payload.timestamp` is older than the configured threshold threshold := uint64(gossipConf.GetGossipTimestampThreshold().Seconds()) - if uint64(payload.Timestamp) < now-threshold { + if uint64(payload.Timestamp) < now-threshold || uint64(payload.Timestamp) < cfg.BlockTime /* ensure timestamp>=BlockTime since we'll do subtraction below */ { log.Warn("payload is too old", "timestamp", uint64(payload.Timestamp), "threshold_seconds", threshold) return pubsub.ValidationReject } @@ -387,14 +387,15 @@ if payload.BlobGasUsed == nil { log.Warn("payload is on v3 topic, but has nil blob gas used", "bad_hash", payload.BlockHash.String()) return pubsub.ValidationReject // [REJECT] if the block is on a topic >= V3 and has a non-zero blob gas used field pre-Jovian - } else if !cfg.IsDAFootprintBlockLimit(uint64(payload.Timestamp)) && *payload.BlobGasUsed != 0 { + } else if !(cfg.IsDAFootprintBlockLimit(uint64(payload.Timestamp)) || cfg.IsL2Blob(uint64(payload.Timestamp)-cfg.BlockTime)) && *payload.BlobGasUsed != 0 { log.Warn("payload is on v3 topic, but has non-zero blob gas used", "bad_hash", payload.BlockHash.String(), "blob_gas_used", *payload.BlobGasUsed) return pubsub.ValidationReject }   // [REJECT] if the block is on a topic >= V3 and has an excess blob gas value that is not zero - if payload.ExcessBlobGas == nil || *payload.ExcessBlobGas != 0 { + + if payload.ExcessBlobGas == nil || (!cfg.IsL2Blob(uint64(payload.Timestamp)-cfg.BlockTime) && *payload.ExcessBlobGas != 0) { log.Warn("payload is on v3 topic, but has non-zero excess blob gas", "bad_hash", payload.BlockHash.String(), "excess_blob_gas", ptr.Str(payload.ExcessBlobGas)) return pubsub.ValidationReject
diff --git official optimism/op-node/rollup/derive/span_batch_tx.go SWC optimism/op-node/rollup/derive/span_batch_tx.go index 876fbd9347f27566b571b3f0d09e1f600f6ba0ec..3d8e5a01962120470eff853522b23838ee287f3c 100644 --- official optimism/op-node/rollup/derive/span_batch_tx.go +++ SWC optimism/op-node/rollup/derive/span_batch_tx.go @@ -46,6 +46,18 @@ }   func (txData *spanBatchDynamicFeeTxData) txType() byte { return types.DynamicFeeTxType }   +type spanBatchBlobTxData struct { + Value *uint256.Int + GasTipCap *uint256.Int // a.k.a. maxPriorityFeePerGas + GasFeeCap *uint256.Int // a.k.a. maxFeePerGas + Data []byte + AccessList types.AccessList + BlobFeeCap *uint256.Int // a.k.a. maxFeePerBlobGas + BlobHashes []common.Hash +} + +func (txData *spanBatchBlobTxData) txType() byte { return types.BlobTxType } + type spanBatchSetCodeTxData struct { Value *uint256.Int GasTipCap *uint256.Int // a.k.a. maxPriorityFeePerGas @@ -103,6 +115,13 @@ var inner spanBatchDynamicFeeTxData err := rlp.DecodeBytes(b[1:], &inner) if err != nil { return nil, fmt.Errorf("failed to decode spanBatchDynamicFeeTxData: %w", err) + } + return &inner, nil + case types.BlobTxType: + var inner spanBatchBlobTxData + err := rlp.DecodeBytes(b[1:], &inner) + if err != nil { + return nil, fmt.Errorf("failed to decode spanBatchBlobTxData: %w", err) } return &inner, nil case types.SetCodeTxType: @@ -187,6 +206,39 @@ V: V, R: R, S: S, } + case types.BlobTxType: + if to == nil { + return nil, fmt.Errorf("invalid blob tx: to can't be nil") + } + VU256, overflow := uint256.FromBig(V) + if overflow { + return nil, fmt.Errorf("invalid blob tx: V overflow:%v", V) + } + RU256, overflow := uint256.FromBig(R) + if overflow { + return nil, fmt.Errorf("invalid blob tx: R overflow:%v", R) + } + SU256, overflow := uint256.FromBig(S) + if overflow { + return nil, fmt.Errorf("invalid blob tx: S overflow:%v", R) + } + batchTxInner := tx.inner.(*spanBatchBlobTxData) + inner = &types.BlobTx{ + ChainID: uint256.MustFromBig(chainID), + Nonce: nonce, + GasTipCap: batchTxInner.GasTipCap, + GasFeeCap: batchTxInner.GasFeeCap, + Gas: gas, + To: *to, + Value: batchTxInner.Value, + Data: batchTxInner.Data, + AccessList: batchTxInner.AccessList, + BlobFeeCap: batchTxInner.BlobFeeCap, + BlobHashes: batchTxInner.BlobHashes, + V: VU256, + R: RU256, + S: SU256, + } case types.SetCodeTxType: if to == nil { return nil, fmt.Errorf("to address is required for SetCodeTx") @@ -238,6 +290,32 @@ GasFeeCap: tx.GasFeeCap(), Value: tx.Value(), Data: tx.Data(), AccessList: tx.AccessList(), + } + case types.BlobTxType: + gasTipCap, overflow := uint256.FromBig(tx.GasTipCap()) + if overflow { + return nil, fmt.Errorf("tx.GasTipCap() overflow: %v", tx.GasTipCap()) + } + gasFeeCap, overflow := uint256.FromBig(tx.GasFeeCap()) + if overflow { + return nil, fmt.Errorf("tx.GasFeeCap() overflow: %v", tx.GasFeeCap()) + } + value, overflow := uint256.FromBig(tx.Value()) + if overflow { + return nil, fmt.Errorf("tx.Value() overflow: %v", tx.Value()) + } + blobFeeCap, overflow := uint256.FromBig(tx.BlobGasFeeCap()) + if overflow { + return nil, fmt.Errorf("tx.BlobGasFeeCap() overflow: %v", tx.BlobGasFeeCap()) + } + inner = &spanBatchBlobTxData{ + Value: value, + GasTipCap: gasTipCap, + GasFeeCap: gasFeeCap, + Data: tx.Data(), + AccessList: tx.AccessList(), + BlobFeeCap: blobFeeCap, + BlobHashes: tx.BlobHashes(), } case types.SetCodeTxType: inner = &spanBatchSetCodeTxData{
diff --git official optimism/op-node/rollup/derive/span_batch_txs.go SWC optimism/op-node/rollup/derive/span_batch_txs.go index 7c354430fc30e5a61f2fbbdcb36df607f311e677..159a9eb14506010fbc23980912d5bea288af0e15 100644 --- official optimism/op-node/rollup/derive/span_batch_txs.go +++ SWC optimism/op-node/rollup/derive/span_batch_txs.go @@ -271,6 +271,8 @@ case types.AccessListTxType: v = bit case types.DynamicFeeTxType: v = bit + case types.BlobTxType: + v = bit case types.SetCodeTxType: v = bit default: @@ -387,6 +389,8 @@ } case types.AccessListTxType: yParityBit = uint(v) case types.DynamicFeeTxType: + yParityBit = uint(v) + case types.BlobTxType: yParityBit = uint(v) case types.SetCodeTxType: yParityBit = uint(v)
diff --git official optimism/op-node/rollup/derive/span_batch_txs_test.go SWC optimism/op-node/rollup/derive/span_batch_txs_test.go index 1a752f75f47baab50442081154e40b49b048729f..7ccd1b2761a60a00ee5c533d837591d9c2f060e6 100644 --- official optimism/op-node/rollup/derive/span_batch_txs_test.go +++ SWC optimism/op-node/rollup/derive/span_batch_txs_test.go @@ -333,7 +333,8 @@ func TestSpanBatchTxsRecoverV(t *testing.T) { rng := rand.New(rand.NewSource(0x123))   chainID := big.NewInt(rng.Int63n(1000)) - isthmusSigner := types.NewIsthmusSigner(chainID) + isthmusSigner := types.NewIsthmusSigner(chainID, false) + cancunSigner := types.NewCancunSigner(chainID) totalblockTxCount := 20 + rng.Intn(100)   cases := []txTypeTest{ @@ -342,6 +343,7 @@ {"legacy tx", testutils.RandomLegacyTx, isthmusSigner}, {"access list tx", testutils.RandomAccessListTx, isthmusSigner}, {"dynamic fee tx", testutils.RandomDynamicFeeTx, isthmusSigner}, {"setcode tx", testutils.RandomSetCodeTx, isthmusSigner}, + {"blob tx", testutils.RandomBlobTx, cancunSigner}, }   for _, testCase := range cases { @@ -425,7 +427,7 @@ func TestSpanBatchTxsRoundTripFullTxs(t *testing.T) { rng := rand.New(rand.NewSource(0x13377331)) chainID := big.NewInt(rng.Int63n(1000)) - isthmusSigner := types.NewIsthmusSigner(chainID) + isthmusSigner := types.NewIsthmusSigner(chainID, false)   cases := []txTypeTest{ {"unprotected legacy tx", testutils.RandomLegacyTx, types.HomesteadSigner{}}, @@ -476,7 +478,7 @@ func TestSpanBatchTxsFullTxNotEnoughTxTos(t *testing.T) { rng := rand.New(rand.NewSource(0x13572468)) chainID := big.NewInt(rng.Int63n(1000)) - isthmusSigner := types.NewIsthmusSigner(chainID) + isthmusSigner := types.NewIsthmusSigner(chainID, false)   cases := []txTypeTest{ {"unprotected legacy tx", testutils.RandomLegacyTx, types.HomesteadSigner{}},
diff --git official optimism/op-node/rollup/driver/driver.go SWC optimism/op-node/rollup/driver/driver.go index cc402cecab8ac44860bf99b53d84446962e39632..c006dcd9529546bd00a2f157cd40acf41360ba65 100644 --- official optimism/op-node/rollup/driver/driver.go +++ SWC optimism/op-node/rollup/driver/driver.go @@ -48,6 +48,7 @@ syncCfg *sync.Config, sequencerConductor conductor.SequencerConductor, altDA AltDAIface, indexingMode bool, + dacClient engine.DACClient, ) *Driver { driverCtx, driverCancel := context.WithCancel(context.Background())   @@ -119,7 +120,7 @@ // Connect origin selector to the engine controller for force reset notifications ec.SetOriginSelectorResetter(findL1Origin)   sequencer = sequencing.NewSequencer(driverCtx, log, cfg, attrBuilder, findL1Origin, - sequencerStateListener, sequencerConductor, asyncGossiper, metrics, ec) + sequencerStateListener, sequencerConductor, asyncGossiper, metrics, ec, dacClient) sys.Register("sequencer", sequencer) } else { sequencer = sequencing.DisabledSequencer{}
diff --git official optimism/op-node/rollup/engine/engine_controller.go SWC optimism/op-node/rollup/engine/engine_controller.go index cca666a28a93754b83390f60a6455faed5ed5a39..3a6b871b741ffefc121010333870652d365d5084 100644 --- official optimism/op-node/rollup/engine/engine_controller.go +++ SWC optimism/op-node/rollup/engine/engine_controller.go @@ -92,6 +92,10 @@ OnCrossUnsafeUpdate(ctx context.Context, crossUnsafe eth.L2BlockRef, localUnsafe eth.L2BlockRef) OnCrossSafeUpdate(ctx context.Context, crossSafe eth.L2BlockRef, localSafe eth.L2BlockRef) }   +type DACClient interface { + UploadBlobs(context.Context, *eth.ExecutionPayloadEnvelope) error +} + type EngineController struct { engine ExecEngine // Underlying execution engine RPC log log.Logger
diff --git official optimism/op-node/rollup/sequencing/sequencer.go SWC optimism/op-node/rollup/sequencing/sequencer.go index c536c5bc8b276391260710ad1d88279ae35fc4fa..8887cc8cf73e04d0a232ba8c223c8ab0edfa6fcc 100644 --- official optimism/op-node/rollup/sequencing/sequencer.go +++ SWC optimism/op-node/rollup/sequencing/sequencer.go @@ -126,6 +126,8 @@ latestHeadSet chan struct{}   // toBlockRef converts a payload to a block-ref, and is only configurable for test-purposes toBlockRef func(rollupCfg *rollup.Config, payload *eth.ExecutionPayload) (eth.L2BlockRef, error) + + dacClient engine.DACClient }   var _ SequencerIface = (*Sequencer)(nil) @@ -138,6 +140,7 @@ conductor conductor.SequencerConductor, asyncGossip AsyncGossiper, metrics Metrics, eng attributes.EngineController, + dacClient engine.DACClient, ) *Sequencer { return &Sequencer{ ctx: driverCtx, @@ -153,6 +156,7 @@ metrics: metrics, eng: eng, timeNow: time.Now, toBlockRef: derive.PayloadToBlockRef, + dacClient: dacClient, } }   @@ -274,6 +278,28 @@ "block", x.Envelope.ExecutionPayload.ID(), "parent", x.Envelope.ExecutionPayload.ParentID(), "txs", len(x.Envelope.ExecutionPayload.Transactions), "time", uint64(x.Envelope.ExecutionPayload.Timestamp)) + + { + envelope := x.Envelope + if envelope.BlobsBundle != nil && len(envelope.BlobsBundle.Blobs) > 0 { + // Deriving is based on onchain-data which doesn't contain L2 blob. + if x.DerivedFrom != (eth.L1BlockRef{}) { + d.emitter.Emit(d.ctx, rollup.EngineTemporaryErrorEvent{ + Err: fmt.Errorf("got blobs when deriving")}) + return + } + if d.dacClient != nil { + ctx, cancel := context.WithTimeout(d.ctx, time.Second*5) + defer cancel() + err := d.dacClient.UploadBlobs(ctx, envelope) + if err != nil { + d.emitter.Emit(d.ctx, rollup.EngineTemporaryErrorEvent{ + Err: fmt.Errorf("UploadBlobs failed: %w", err)}) + return + } + } + } + }   // generous timeout, the conductor is important ctx, cancel := context.WithTimeout(d.ctx, time.Second*30)
diff --git official optimism/op-node/rollup/sequencing/sequencer_test.go SWC optimism/op-node/rollup/sequencing/sequencer_test.go index bcdd6cdd79060bb80afe9309007147469451cc30..2148ebd33e312a1362945159bf1d196a98e50dad 100644 --- official optimism/op-node/rollup/sequencing/sequencer_test.go +++ SWC optimism/op-node/rollup/sequencing/sequencer_test.go @@ -733,7 +733,7 @@ asyncGossip: &FakeAsyncGossip{}, } seq := NewSequencer(context.Background(), log, cfg, deps.attribBuilder, deps.l1OriginSelector, deps.seqState, deps.conductor, - deps.asyncGossip, metrics.NoopMetrics, fakeEngController{}) + deps.asyncGossip, metrics.NoopMetrics, fakeEngController{}, nil) // We create mock payloads, with the epoch-id as tx[0], rather than proper L1Block-info deposit tx. seq.toBlockRef = func(rollupCfg *rollup.Config, payload *eth.ExecutionPayload) (eth.L2BlockRef, error) { return eth.L2BlockRef{
diff --git official optimism/op-node/service.go SWC optimism/op-node/service.go index c87f61af6f74f5eb36bd328f0503cf92a9dc8a59..38380e05b4a22935f2a23b4cafe2f122533b696c 100644 --- official optimism/op-node/service.go +++ SWC optimism/op-node/service.go @@ -124,7 +124,8 @@ return conductorRPCEndpoint, nil }, ConductorRpcTimeout: ctx.Duration(flags.ConductorRpcTimeoutFlag.Name),   - AltDA: altda.ReadCLIConfig(ctx), + AltDA: altda.ReadCLIConfig(ctx), + DACConfig: config.ReadDACConfigFromCLI(ctx),   IgnoreMissingPectraBlobSchedule: ctx.Bool(flags.IgnoreMissingPectraBlobSchedule.Name), FetchWithdrawalRootFromState: ctx.Bool(flags.FetchWithdrawalRootFromState.Name),
diff --git official optimism/op-program/client/l2/engineapi/block_processor.go SWC optimism/op-program/client/l2/engineapi/block_processor.go index be7178b32ed0ed0d6bb79dd01bc32093ca8a4d1b..c4dbcaf44358bdd8772387ec54ad67fd855c9a8d 100644 --- official optimism/op-program/client/l2/engineapi/block_processor.go +++ SWC optimism/op-program/client/l2/engineapi/block_processor.go @@ -10,6 +10,7 @@ "github.com/ethereum-optimism/optimism/op-service/predeploys" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/consensus" "github.com/ethereum/go-ethereum/consensus/misc/eip1559" + "github.com/ethereum/go-ethereum/consensus/misc/eip4844" "github.com/ethereum/go-ethereum/core" "github.com/ethereum/go-ethereum/core/state" "github.com/ethereum/go-ethereum/core/types" @@ -105,7 +106,8 @@ if provider.Config().IsCancun(header.Number, header.Time) { // Blob tx not supported on optimism chains but fields must be set when Cancun is active. zero := uint64(0) header.BlobGasUsed = &zero - header.ExcessBlobGas = &zero + excessBlobGas := eip4844.CalcExcessBlobGas(provider.Config(), parentHeader, header.Time) + header.ExcessBlobGas = &excessBlobGas } // core.NewEVMBlockContext need to be called after the blob gas fields are set vmenv = mkEVM() @@ -153,6 +155,9 @@ return nil, fmt.Errorf("failed to apply transaction to L2 block (tx %d): %w", txIndex, err) } b.receipts = append(b.receipts, receipt) b.transactions = append(b.transactions, tx) + if b.header.BlobGasUsed != nil { + *b.header.BlobGasUsed += receipt.BlobGasUsed + } return receipt, nil }
diff --git official optimism/op-service/eth/types.go SWC optimism/op-service/eth/types.go index 8d192085de0739699e33faaa039cfe40bda922e9..aebacca74ff6d887329b8066da94349fa5c71300 100644 --- official optimism/op-service/eth/types.go +++ SWC optimism/op-service/eth/types.go @@ -231,8 +231,9 @@ Timestamp uint64 `json:"timestamp"` }   type ExecutionPayloadEnvelope struct { - ParentBeaconBlockRoot *common.Hash `json:"parentBeaconBlockRoot,omitempty"` - ExecutionPayload *ExecutionPayload `json:"executionPayload"` + ParentBeaconBlockRoot *common.Hash `json:"parentBeaconBlockRoot,omitempty"` + ExecutionPayload *ExecutionPayload `json:"executionPayload"` + BlobsBundle *engine.BlobsBundleV1 `json:"blobsBundle"` }   func (env *ExecutionPayloadEnvelope) ID() BlockID {
diff --git official optimism/op-service/sources/engine_client.go SWC optimism/op-service/sources/engine_client.go index c16629b77d2bbd040188d1b4f6cd6747b6bacbec..755bcf5a0070061d856c0c7e42af4e22f3d6adf9 100644 --- official optimism/op-service/sources/engine_client.go +++ SWC optimism/op-service/sources/engine_client.go @@ -107,6 +107,8 @@ switch method := s.evp.NewPayloadVersion(uint64(payload.Timestamp)); method { case eth.NewPayloadV4: err = s.RPC.CallContext(ctx, &result, string(method), payload, []common.Hash{}, parentBeaconBlockRoot, []hexutil.Bytes{}) case eth.NewPayloadV3: + // now we pass empty array to skip checking versionedHashes + // TODO: sync with OP upstream once they support L2 blob tx err = s.RPC.CallContext(ctx, &result, string(method), payload, []common.Hash{}, parentBeaconBlockRoot) case eth.NewPayloadV2: err = s.RPC.CallContext(ctx, &result, string(method), payload)
diff --git official optimism/op-service/testutils/random.go SWC optimism/op-service/testutils/random.go index e685b97c11b3947d73a25a3b714371e9e40ca6fe..87f50f666699c98931b575233d1373f56be453a9 100644 --- official optimism/op-service/testutils/random.go +++ SWC optimism/op-service/testutils/random.go @@ -144,7 +144,7 @@ return &to }   func isIsthmusSigner(signer types.Signer) bool { - isthusSigner := types.NewIsthmusSigner(signer.ChainID()) + isthusSigner := types.NewIsthmusSigner(signer.ChainID(), false) return signer.Equal(isthusSigner) }   @@ -253,6 +253,30 @@ baseFee := new(big.Int).SetUint64(rng.Uint64()) return RandomDynamicFeeTxWithBaseFee(rng, baseFee, signer) }   +func RandomBlobTx(rng *rand.Rand, signer types.Signer) *types.Transaction { + baseFee := new(big.Int).SetUint64(rng.Uint64()) + key := InsecureRandomKey(rng) + tip := big.NewInt(rng.Int63n(10 * params.GWei)) + txData := &types.BlobTx{ + ChainID: uint256.MustFromBig(signer.ChainID()), + Nonce: rng.Uint64(), + GasTipCap: uint256.MustFromBig(tip), + GasFeeCap: uint256.MustFromBig(new(big.Int).Add(baseFee, tip)), + Gas: params.TxGas + uint64(rng.Int63n(2_000_000)), + To: RandomAddress(rng), + Value: uint256.MustFromBig(RandomETH(rng, 10)), + Data: RandomData(rng, rng.Intn(1000)), + AccessList: nil, + BlobFeeCap: uint256.MustFromBig(baseFee), + BlobHashes: []common.Hash{RandomHash(rng)}, + } + tx, err := types.SignNewTx(key, signer, txData) + if err != nil { + panic(err) + } + return tx +} + func RandomSetCodeAuth(rng *rand.Rand) types.SetCodeAuthorization { key := InsecureRandomKey(rng)   @@ -360,7 +384,7 @@ if chainIDInt == 0 { // Zero chainID is invalid. chainIDInt++ } chainID := big.NewInt(chainIDInt) - signer := types.NewIsthmusSigner(chainID) + signer := types.NewIsthmusSigner(chainID, false) txs := make([]*types.Transaction, 0, txCount+len(ptxs)) txs = append(txs, ptxs...) for i := 0; i < txCount; i++ {

In order to simplify the integration with EthStorage as L3, The L1Block contract is extended to allow fetching 8191 history L1 blockhash not including the current one.

diff --git official optimism/op-node/bindings/l1block.go SWC optimism/op-node/bindings/l1block.go index 41a24ec38878128d02ced08b4b14abca2aa378e6..5793ce341edb0a499c698b9fbe84f3669ff9d499 100644 --- official optimism/op-node/bindings/l1block.go +++ SWC optimism/op-node/bindings/l1block.go @@ -30,8 +30,8 @@ )   // L1BlockMetaData contains all meta data concerning the L1Block contract. var L1BlockMetaData = &bind.MetaData{ - ABI: "[{\"type\":\"function\",\"name\":\"DEPOSITOR_ACCOUNT\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"address\",\"internalType\":\"address\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"baseFeeScalar\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint32\",\"internalType\":\"uint32\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"basefee\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"batcherHash\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"blobBaseFee\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"blobBaseFeeScalar\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint32\",\"internalType\":\"uint32\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"hash\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"l1FeeOverhead\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"l1FeeScalar\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"number\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint64\",\"internalType\":\"uint64\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"sequenceNumber\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint64\",\"internalType\":\"uint64\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"setL1BlockValues\",\"inputs\":[{\"name\":\"_number\",\"type\":\"uint64\",\"internalType\":\"uint64\"},{\"name\":\"_timestamp\",\"type\":\"uint64\",\"internalType\":\"uint64\"},{\"name\":\"_basefee\",\"type\":\"uint256\",\"internalType\":\"uint256\"},{\"name\":\"_hash\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"},{\"name\":\"_sequenceNumber\",\"type\":\"uint64\",\"internalType\":\"uint64\"},{\"name\":\"_batcherHash\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"},{\"name\":\"_l1FeeOverhead\",\"type\":\"uint256\",\"internalType\":\"uint256\"},{\"name\":\"_l1FeeScalar\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"setL1BlockValuesEcotone\",\"inputs\":[],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"timestamp\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint64\",\"internalType\":\"uint64\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"version\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"string\",\"internalType\":\"string\"}],\"stateMutability\":\"view\"}]", - Bin: "0x608060405234801561001057600080fd5b5061053e806100206000396000f3fe608060405234801561001057600080fd5b50600436106100f55760003560e01c80638381f58a11610097578063c598591811610066578063c598591814610229578063e591b28214610249578063e81b2c6d14610289578063f82061401461029257600080fd5b80638381f58a146101e35780638b239f73146101f75780639e8c496614610200578063b80777ea1461020957600080fd5b806354fd4d50116100d357806354fd4d50146101335780635cf249691461017c57806364ca23ef1461018557806368d5dca6146101b257600080fd5b8063015d8eb9146100fa57806309bd5a601461010f578063440a5e201461012b575b600080fd5b61010d61010836600461044c565b61029b565b005b61011860025481565b6040519081526020015b60405180910390f35b61010d6103da565b61016f6040518060400160405280600581526020017f312e322e3000000000000000000000000000000000000000000000000000000081525081565b60405161012291906104be565b61011860015481565b6003546101999067ffffffffffffffff1681565b60405167ffffffffffffffff9091168152602001610122565b6003546101ce9068010000000000000000900463ffffffff1681565b60405163ffffffff9091168152602001610122565b6000546101999067ffffffffffffffff1681565b61011860055481565b61011860065481565b6000546101999068010000000000000000900467ffffffffffffffff1681565b6003546101ce906c01000000000000000000000000900463ffffffff1681565b61026473deaddeaddeaddeaddeaddeaddeaddeaddead000181565b60405173ffffffffffffffffffffffffffffffffffffffff9091168152602001610122565b61011860045481565b61011860075481565b3373deaddeaddeaddeaddeaddeaddeaddeaddead000114610342576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152603b60248201527f4c31426c6f636b3a206f6e6c7920746865206465706f7369746f72206163636f60448201527f756e742063616e20736574204c3120626c6f636b2076616c7565730000000000606482015260840160405180910390fd5b6000805467ffffffffffffffff98891668010000000000000000027fffffffffffffffffffffffffffffffff00000000000000000000000000000000909116998916999099179890981790975560019490945560029290925560038054919094167fffffffffffffffffffffffffffffffffffffffffffffffff00000000000000009190911617909255600491909155600555600655565b3373deaddeaddeaddeaddeaddeaddeaddeaddead00011461040357633cc50b456000526004601cfd5b60043560801c60035560143560801c600055602435600155604435600755606435600255608435600455565b803567ffffffffffffffff8116811461044757600080fd5b919050565b600080600080600080600080610100898b03121561046957600080fd5b6104728961042f565b975061048060208a0161042f565b9650604089013595506060890135945061049c60808a0161042f565b979a969950949793969560a0850135955060c08501359460e001359350915050565b600060208083528351808285015260005b818110156104eb578581018301518582016040015282016104cf565b818111156104fd576000604083870101525b50601f017fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffe01692909201604001939250505056fea164736f6c634300080f000a", + ABI: "[{\"type\":\"function\",\"name\":\"DEPOSITOR_ACCOUNT\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"address\",\"internalType\":\"address\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"HISTORY_SIZE\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"baseFeeScalar\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint32\",\"internalType\":\"uint32\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"basefee\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"batcherHash\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"blobBaseFee\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"blobBaseFeeScalar\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint32\",\"internalType\":\"uint32\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"blockHash\",\"inputs\":[{\"name\":\"_historyNumber\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[{\"name\":\"\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"hash\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"historyHashes\",\"inputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[{\"name\":\"\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"l1FeeOverhead\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"l1FeeScalar\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"number\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint64\",\"internalType\":\"uint64\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"sequenceNumber\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint64\",\"internalType\":\"uint64\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"setL1BlockValues\",\"inputs\":[{\"name\":\"_number\",\"type\":\"uint64\",\"internalType\":\"uint64\"},{\"name\":\"_timestamp\",\"type\":\"uint64\",\"internalType\":\"uint64\"},{\"name\":\"_basefee\",\"type\":\"uint256\",\"internalType\":\"uint256\"},{\"name\":\"_hash\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"},{\"name\":\"_sequenceNumber\",\"type\":\"uint64\",\"internalType\":\"uint64\"},{\"name\":\"_batcherHash\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"},{\"name\":\"_l1FeeOverhead\",\"type\":\"uint256\",\"internalType\":\"uint256\"},{\"name\":\"_l1FeeScalar\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"setL1BlockValuesEcotone\",\"inputs\":[],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"timestamp\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint64\",\"internalType\":\"uint64\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"version\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"string\",\"internalType\":\"string\"}],\"stateMutability\":\"view\"}]", + Bin: "0x608060405234801561001057600080fd5b5061075e806100206000396000f3fe608060405234801561001057600080fd5b50600436106101365760003560e01c806385df51fd116100b2578063b80777ea11610081578063e591b28211610066578063e591b282146102b9578063e81b2c6d146102f9578063f82061401461030257600080fd5b8063b80777ea14610279578063c59859181461029957600080fd5b806385df51fd146102415780638b239f731461025457806392abaa421461025d5780639e8c49661461027057600080fd5b80635cf2496911610109578063652c462e116100ee578063652c462e146101f357806368d5dca6146101fc5780638381f58a1461022d57600080fd5b80635cf24969146101bd57806364ca23ef146101c657600080fd5b8063015d8eb91461013b57806309bd5a6014610150578063440a5e201461016c57806354fd4d5014610174575b600080fd5b61014e61014936600461058b565b61030b565b005b61015960025481565b6040519081526020015b60405180910390f35b61014e61044a565b6101b06040518060400160405280600581526020017f312e322e3000000000000000000000000000000000000000000000000000000081525081565b60405161016391906105fd565b61015960015481565b6003546101da9067ffffffffffffffff1681565b60405167ffffffffffffffff9091168152602001610163565b61015961200081565b6003546102189068010000000000000000900463ffffffff1681565b60405163ffffffff9091168152602001610163565b6000546101da9067ffffffffffffffff1681565b61015961024f366004610670565b6104d2565b61015960055481565b61015961026b366004610670565b610556565b61015960065481565b6000546101da9068010000000000000000900467ffffffffffffffff1681565b600354610218906c01000000000000000000000000900463ffffffff1681565b6102d473deaddeaddeaddeaddeaddeaddeaddeaddead000181565b60405173ffffffffffffffffffffffffffffffffffffffff9091168152602001610163565b61015960045481565b61015960075481565b3373deaddeaddeaddeaddeaddeaddeaddeaddead0001146103b2576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152603b60248201527f4c31426c6f636b3a206f6e6c7920746865206465706f7369746f72206163636f60448201527f756e742063616e20736574204c3120626c6f636b2076616c7565730000000000606482015260840160405180910390fd5b6000805467ffffffffffffffff98891668010000000000000000027fffffffffffffffffffffffffffffffff00000000000000000000000000000000909116998916999099179890981790975560019490945560029290925560038054919094167fffffffffffffffffffffffffffffffffffffffffffffffff00000000000000009190911617909255600491909155600555600655565b3373deaddeaddeaddeaddeaddeaddeaddeaddead00011461047357633cc50b456000526004601cfd5b60048035608090811c600355601435901c600081905560243560015560443560075560643560028190556084359092556008906104bd906120009067ffffffffffffffff16610689565b61200081106104ce576104ce6106c4565b0155565b60008054819067ffffffffffffffff166120008110156104f5576000915061050f565b61050161200082610722565b61050c906001610739565b91505b81841015801561051e57508084105b1561054c57600861053161200086610689565b6120008110610542576105426106c4565b0154949350505050565b5060009392505050565b600881612000811061056757600080fd5b0154905081565b803567ffffffffffffffff8116811461058657600080fd5b919050565b600080600080600080600080610100898b0312156105a857600080fd5b6105b18961056e565b97506105bf60208a0161056e565b965060408901359550606089013594506105db60808a0161056e565b979a969950949793969560a0850135955060c08501359460e001359350915050565b600060208083528351808285015260005b8181101561062a5785810183015185820160400152820161060e565b8181111561063c576000604083870101525b50601f017fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffe016929092016040019392505050565b60006020828403121561068257600080fd5b5035919050565b6000826106bf577f4e487b7100000000000000000000000000000000000000000000000000000000600052601260045260246000fd5b500690565b7f4e487b7100000000000000000000000000000000000000000000000000000000600052603260045260246000fd5b7f4e487b7100000000000000000000000000000000000000000000000000000000600052601160045260246000fd5b600082821015610734576107346106f3565b500390565b6000821982111561074c5761074c6106f3565b50019056fea164736f6c634300080f000a", }   // L1BlockABI is the input ABI used to generate the binding from. @@ -232,6 +232,37 @@ func (_L1Block *L1BlockCallerSession) DEPOSITORACCOUNT() (common.Address, error) { return _L1Block.Contract.DEPOSITORACCOUNT(&_L1Block.CallOpts) }   +// HISTORYSIZE is a free data retrieval call binding the contract method 0x652c462e. +// +// Solidity: function HISTORY_SIZE() view returns(uint256) +func (_L1Block *L1BlockCaller) HISTORYSIZE(opts *bind.CallOpts) (*big.Int, error) { + var out []interface{} + err := _L1Block.contract.Call(opts, &out, "HISTORY_SIZE") + + if err != nil { + return *new(*big.Int), err + } + + out0 := *abi.ConvertType(out[0], new(*big.Int)).(**big.Int) + + return out0, err + +} + +// HISTORYSIZE is a free data retrieval call binding the contract method 0x652c462e. +// +// Solidity: function HISTORY_SIZE() view returns(uint256) +func (_L1Block *L1BlockSession) HISTORYSIZE() (*big.Int, error) { + return _L1Block.Contract.HISTORYSIZE(&_L1Block.CallOpts) +} + +// HISTORYSIZE is a free data retrieval call binding the contract method 0x652c462e. +// +// Solidity: function HISTORY_SIZE() view returns(uint256) +func (_L1Block *L1BlockCallerSession) HISTORYSIZE() (*big.Int, error) { + return _L1Block.Contract.HISTORYSIZE(&_L1Block.CallOpts) +} + // BaseFeeScalar is a free data retrieval call binding the contract method 0xc5985918. // // Solidity: function baseFeeScalar() view returns(uint32) @@ -387,6 +418,37 @@ func (_L1Block *L1BlockCallerSession) BlobBaseFeeScalar() (uint32, error) { return _L1Block.Contract.BlobBaseFeeScalar(&_L1Block.CallOpts) }   +// BlockHash is a free data retrieval call binding the contract method 0x85df51fd. +// +// Solidity: function blockHash(uint256 _historyNumber) view returns(bytes32) +func (_L1Block *L1BlockCaller) BlockHash(opts *bind.CallOpts, _historyNumber *big.Int) ([32]byte, error) { + var out []interface{} + err := _L1Block.contract.Call(opts, &out, "blockHash", _historyNumber) + + if err != nil { + return *new([32]byte), err + } + + out0 := *abi.ConvertType(out[0], new([32]byte)).(*[32]byte) + + return out0, err + +} + +// BlockHash is a free data retrieval call binding the contract method 0x85df51fd. +// +// Solidity: function blockHash(uint256 _historyNumber) view returns(bytes32) +func (_L1Block *L1BlockSession) BlockHash(_historyNumber *big.Int) ([32]byte, error) { + return _L1Block.Contract.BlockHash(&_L1Block.CallOpts, _historyNumber) +} + +// BlockHash is a free data retrieval call binding the contract method 0x85df51fd. +// +// Solidity: function blockHash(uint256 _historyNumber) view returns(bytes32) +func (_L1Block *L1BlockCallerSession) BlockHash(_historyNumber *big.Int) ([32]byte, error) { + return _L1Block.Contract.BlockHash(&_L1Block.CallOpts, _historyNumber) +} + // Hash is a free data retrieval call binding the contract method 0x09bd5a60. // // Solidity: function hash() view returns(bytes32) @@ -416,6 +478,37 @@ // // Solidity: function hash() view returns(bytes32) func (_L1Block *L1BlockCallerSession) Hash() ([32]byte, error) { return _L1Block.Contract.Hash(&_L1Block.CallOpts) +} + +// HistoryHashes is a free data retrieval call binding the contract method 0x92abaa42. +// +// Solidity: function historyHashes(uint256 ) view returns(bytes32) +func (_L1Block *L1BlockCaller) HistoryHashes(opts *bind.CallOpts, arg0 *big.Int) ([32]byte, error) { + var out []interface{} + err := _L1Block.contract.Call(opts, &out, "historyHashes", arg0) + + if err != nil { + return *new([32]byte), err + } + + out0 := *abi.ConvertType(out[0], new([32]byte)).(*[32]byte) + + return out0, err + +} + +// HistoryHashes is a free data retrieval call binding the contract method 0x92abaa42. +// +// Solidity: function historyHashes(uint256 ) view returns(bytes32) +func (_L1Block *L1BlockSession) HistoryHashes(arg0 *big.Int) ([32]byte, error) { + return _L1Block.Contract.HistoryHashes(&_L1Block.CallOpts, arg0) +} + +// HistoryHashes is a free data retrieval call binding the contract method 0x92abaa42. +// +// Solidity: function historyHashes(uint256 ) view returns(bytes32) +func (_L1Block *L1BlockCallerSession) HistoryHashes(arg0 *big.Int) ([32]byte, error) { + return _L1Block.Contract.HistoryHashes(&_L1Block.CallOpts, arg0) }   // L1FeeOverhead is a free data retrieval call binding the contract method 0x8b239f73.
diff --git official optimism/packages/contracts-bedrock/interfaces/L2/IL1Block.sol SWC optimism/packages/contracts-bedrock/interfaces/L2/IL1Block.sol index 65b9ef0f2029b1eaad9e6547af2c8fd514d0885d..19024bbee5d68de0d4fbfebb5c442d22f45da41f 100644 --- official optimism/packages/contracts-bedrock/interfaces/L2/IL1Block.sol +++ SWC optimism/packages/contracts-bedrock/interfaces/L2/IL1Block.sol @@ -38,4 +38,7 @@ function timestamp() external view returns (uint64); function version() external pure returns (string memory);   function __constructor__() external; + + function historySize() external pure returns (uint256); + function blockHash(uint256 _historyNumber) external view returns (bytes32); }
diff --git official optimism/packages/contracts-bedrock/src/L2/L1Block.sol SWC optimism/packages/contracts-bedrock/src/L2/L1Block.sol index fffce12bb5115efcfd172aaeeb8a90f7e7396ae9..57fa2a10076bb152881211698770188277e30b81 100644 --- official optimism/packages/contracts-bedrock/src/L2/L1Block.sol +++ SWC optimism/packages/contracts-bedrock/src/L2/L1Block.sol @@ -95,6 +95,25 @@ function isCustomGasToken() public pure returns (bool is_) { is_ = false; }   + // keccak256(abi.encode(uint256(keccak256("openzeppelin.storage.L1Block.HistoryHashesStorage")) - 1)) & + // ~bytes32(uint256(0xff)) + bytes32 private constant _L1BLOCK_HISTORY_HASHES_LOCATION = + 0xb6e35d777715793a0808adb4ab3d51fc56065457feaf3a18bd958afef33bef00; + /// @notice size of historyHashes. + uint256 internal constant HISTORY_SIZE = 8192; + /// @custom:storage-location erc7201:openzeppelin.storage.L1Block.HistoryHashesStorage + + struct HistoryHashesStorage { + /// @notice The 8191 history L1 blockhashes and 1 latest L1 blockhash. + bytes32[HISTORY_SIZE] historyHashes; + } + + function _getHistoryHashesStorage() private pure returns (HistoryHashesStorage storage $) { + assembly { + $.slot := _L1BLOCK_HISTORY_HASHES_LOCATION + } + } + /// @custom:legacy /// @notice Updates the L1 block values. /// @param _number L1 blocknumber. @@ -158,6 +177,36 @@ sstore(blobBaseFee.slot, calldataload(68)) // uint256 sstore(hash.slot, calldataload(100)) // bytes32 sstore(batcherHash.slot, calldataload(132)) // bytes32 } + + HistoryHashesStorage storage $ = _getHistoryHashesStorage(); + $.historyHashes[number % HISTORY_SIZE] = hash; + } + + /// @notice Returns the L1 block hash at the requested L1 blocknumber. + /// Only the most recent 8191 L1 block hashes are available, excluding the current one. + /// @param _historyNumber L1 blocknumber. + function blockHash(uint256 _historyNumber) external view returns (bytes32) { + // translated from + // [opBlockhash](https://github.com/ethereum/go-ethereum/blob/e31709db6570e302557a9bccd681034ea0dcc246/core/vm/instructions.go#L434) + // with 256 => HISTORY_SIZE-1 + uint256 lower; + uint256 upper = number; + if (upper < HISTORY_SIZE) { + lower = 0; + } else { + lower = upper - HISTORY_SIZE + 1; + } + if (_historyNumber >= lower && _historyNumber < upper) { + HistoryHashesStorage storage $ = _getHistoryHashesStorage(); + return $.historyHashes[_historyNumber % HISTORY_SIZE]; + } else { + return bytes32(0); + } + } + + /// @notice Returns the size of history hashes. + function historySize() external pure returns (uint256) { + return HISTORY_SIZE; }   /// @notice Updates the L1 block values for an Isthmus upgraded chain.
diff --git official optimism/packages/contracts-bedrock/test/L2/L1Block.t.sol SWC optimism/packages/contracts-bedrock/test/L2/L1Block.t.sol index 8a7d22cae8de1b0b470134fcbf5cfe8ada7679e6..15a279d195caeaf7aaeeb948bb343e43862ab339 100644 --- official optimism/packages/contracts-bedrock/test/L2/L1Block.t.sol +++ SWC optimism/packages/contracts-bedrock/test/L2/L1Block.t.sol @@ -209,6 +209,62 @@ // make sure return value is the expected function selector for "NotDepositor()" bytes memory expReturn = hex"3cc50b45"; assertEq(data, expReturn); } + + /// @dev Tests that `blockHash` works for block range [n-256, n) where n is the latest + /// L1 block number known by the L2 system. + function testFuzz_blockHash_succeeds( + uint32 baseFeeScalar, + uint32 blobBaseFeeScalar, + uint64 sequenceNumber, + uint64 timestamp, + uint64 number, + uint256 baseFee, + uint256 blobBaseFee, + bytes32 hash, + bytes32 batcherHash + ) + external + { + if (number > type(uint64).max - uint64(l1Block.historySize()) - 1) { + number = type(uint64).max - uint64(l1Block.historySize()) - 1; + } + if (uint256(hash) > type(uint256).max - l1Block.historySize() - 1) { + hash = bytes32(type(uint256).max - l1Block.historySize() - 1); + } + + for (uint256 i = 1; i <= l1Block.historySize() + 1; i++) { + bytes memory functionCallDataPacked = Encoding.encodeSetL1BlockValuesEcotone( + baseFeeScalar, + blobBaseFeeScalar, + sequenceNumber, + timestamp, + number + uint64(i), + baseFee, + blobBaseFee, + bytes32(uint256(hash) + i), + batcherHash + ); + + vm.prank(depositor); + (bool success,) = address(l1Block).call(functionCallDataPacked); + assertTrue(success, "function call failed"); + + assertEq(l1Block.number(), number + uint64(i)); + assertEq(l1Block.hash(), bytes32(uint256(hash) + i)); + } + + assertTrue( + l1Block.blockHash(number + l1Block.historySize() + 1) == bytes32(0), + "should return bytes32(0) for the latest L1 block" + ); + assertTrue(l1Block.blockHash(number + 1) == bytes32(0), "should return bytes32(0) for blocks out of range"); + for (uint256 i = 2; i <= l1Block.historySize(); i++) { + assertTrue( + l1Block.blockHash(number + i) == bytes32(uint256(hash) + i), + "blockHash's return value should match the value set" + ); + } + } }   /// @title L1Block_SetL1BlockValuesIsthmus_Test

In order to facilitate token migration of existing L1 token to L2 native token, we disable the native deposit and enable minting L2 native token on L1. Refer to this PR for details.

diff --git official optimism/packages/contracts-bedrock/interfaces/L1/IOptimismPortal2.sol SWC optimism/packages/contracts-bedrock/interfaces/L1/IOptimismPortal2.sol index 987450481d6b6f56eea6d12a65437123964357e8..f22191cff3d574872ca5173fa0371cf5fb0f2320 100644 --- official optimism/packages/contracts-bedrock/interfaces/L1/IOptimismPortal2.sol +++ SWC optimism/packages/contracts-bedrock/interfaces/L1/IOptimismPortal2.sol @@ -33,11 +33,16 @@ error OptimismPortal_NoReentrancy(); error OptimismPortal_ProofNotOldEnough(); error OptimismPortal_Unproven(); error OptimismPortal_InvalidLockboxState(); + error OptimismPortal_Unauthorized(); + error OptimismPortal_NativeDepositForbidden(); error OutOfGas(); error UnexpectedList(); error UnexpectedString();   event Initialized(uint8 version); + event NativeDepositEnabled(); + event NativeDepositDisabled(); + event MinterSet(address indexed minter); event TransactionDeposited(address indexed from, address indexed to, uint256 indexed version, bytes opaqueData); event WithdrawalFinalized(bytes32 indexed withdrawalHash, bool success); event WithdrawalProven(bytes32 indexed withdrawalHash, address indexed from, address indexed to); @@ -101,6 +106,10 @@ function respectedGameType() external view returns (GameType); function respectedGameTypeUpdatedAt() external view returns (uint64); function systemConfig() external view returns (ISystemConfig); function version() external pure returns (string memory); + + function setMinter(address _minter) external; + function mintTransaction(address _to, uint256 _value) external; + function setNativeDeposit(bool _disable) external;   function __constructor__(uint256 _proofMaturityDelaySeconds) external; }
diff --git official optimism/packages/contracts-bedrock/interfaces/L2/IL2ToL1MessagePasser.sol SWC optimism/packages/contracts-bedrock/interfaces/L2/IL2ToL1MessagePasser.sol index 4629dbaba8d09fcacb9ba8b369ec1e2716a0b51b..7526983ea28dc7b7dd352a79692b15c17bd40d88 100644 --- official optimism/packages/contracts-bedrock/interfaces/L2/IL2ToL1MessagePasser.sol +++ SWC optimism/packages/contracts-bedrock/interfaces/L2/IL2ToL1MessagePasser.sol @@ -2,6 +2,10 @@ // SPDX-License-Identifier: MIT pragma solidity ^0.8.0;   interface IL2ToL1MessagePasser { + event NativeDepositEnabled(); + event NativeDepositDisabled(); + error L2ToL1MessagePasser_NativeDepositDisabled(); + event MessagePassed( uint256 indexed nonce, address indexed sender, @@ -21,6 +25,7 @@ function initiateWithdrawal(address _target, uint256 _gasLimit, bytes memory _data) external payable; function messageNonce() external view returns (uint256); function sentMessages(bytes32) external view returns (bool); function version() external view returns (string memory); + function setNativeDeposit(bool _disable) external;   function __constructor__() external; }
diff --git official optimism/packages/contracts-bedrock/src/L1/OptimismPortal2.sol SWC optimism/packages/contracts-bedrock/src/L1/OptimismPortal2.sol index 071bf55a8892230103bbcf552105b0ac6431f585..50b1d35e407da4bf2cff405051e4fd7c90a18b7b 100644 --- official optimism/packages/contracts-bedrock/src/L1/OptimismPortal2.sol +++ SWC optimism/packages/contracts-bedrock/src/L1/OptimismPortal2.sol @@ -11,6 +11,7 @@ // Libraries import { EOA } from "src/libraries/EOA.sol"; import { SafeCall } from "src/libraries/SafeCall.sol"; import { Constants } from "src/libraries/Constants.sol"; +import { Predeploys } from "src/libraries/Predeploys.sol"; import { Types } from "src/libraries/Types.sol"; import { Hashing } from "src/libraries/Hashing.sol"; import { SecureMerkleTrie } from "src/libraries/trie/SecureMerkleTrie.sol"; @@ -24,6 +25,7 @@ import { ISystemConfig } from "interfaces/L1/ISystemConfig.sol"; import { IResourceMetering } from "interfaces/L1/IResourceMetering.sol"; import { IDisputeGameFactory } from "interfaces/dispute/IDisputeGameFactory.sol"; import { IDisputeGame } from "interfaces/dispute/IDisputeGame.sol"; +import { IL2ToL1MessagePasser } from "interfaces/L2/IL2ToL1MessagePasser.sol"; import { IAnchorStateRegistry } from "interfaces/dispute/IAnchorStateRegistry.sol"; import { IETHLockbox } from "interfaces/L1/IETHLockbox.sol"; import { ISuperchainConfig } from "interfaces/L1/ISuperchainConfig.sol"; @@ -50,6 +52,9 @@ uint256 internal constant DEPOSIT_VERSION = 0;   /// @notice The L2 gas limit set when eth is deposited using the receive() function. uint64 internal constant RECEIVE_DEFAULT_GAS_LIMIT = 100_000; + + /// @notice The L2 gas limit for system deposit transactions that are initiated from L1. + uint32 internal constant SYSTEM_DEPOSIT_GAS_LIMIT = 200_000;   /// @notice Address of the L2 account which initiated a withdrawal in this transaction. /// If the value of this variable is the default L2 sender address, then we are NOT @@ -130,6 +135,24 @@ /// @custom:legacy /// @custom:spacer superRootsActive bool private spacer_63_20_1;   + // keccak256(abi.encode(uint256(keccak256("openzeppelin.storage.OptimismPortal2.QKCConfigStorage")) - 1)) & + // ~bytes32(uint256(0xff)) + bytes32 private constant _QKC_CONFIG_STORAGE_LOCATION = + 0xb42b8bfdf1143b9dfcdc891f15a039d3c36301d501f5a44f62223d852a602a00; + /// @custom:storage-location erc7201:openzeppelin.storage.OptimismPortal2.QKCConfigStorage + + struct QKCConfigStorage { + /// @notice The minter for migrating existing L1 token to L2 native token. + address minter; + bool disableNativeDeposit; + } + + function _getQKCConfigStorage() private pure returns (QKCConfigStorage storage $) { + assembly { + $.slot := _QKC_CONFIG_STORAGE_LOCATION + } + } + /// @notice Emitted when a transaction is deposited from L1 to L2. The parameters of this event /// are read by the rollup node and used to derive deposit transactions on L2. /// @param from Address that triggered the deposit transaction. @@ -156,6 +179,18 @@ /// @param withdrawalHash Hash of the withdrawal transaction. /// @param success Whether the withdrawal transaction was successful. event WithdrawalFinalized(bytes32 indexed withdrawalHash, bool success);   + /// QKC changes: + + /// @notice added back by QKC + error OptimismPortal_Unauthorized(); + /// @notice Emitted when a minter is set. + + event MinterSet(address indexed minter); + /// @notice Emitted when native deposit is disabled. + event NativeDepositDisabled(); + /// @notice Emitted when native deposit is enabled. + event NativeDepositEnabled(); + /// @notice Thrown when a withdrawal has already been finalized. error OptimismPortal_AlreadyFinalized();   @@ -173,6 +208,10 @@ error OptimismPortal_GasEstimation();   /// @notice Thrown when the gas limit for a deposit is too low. error OptimismPortal_GasLimitTooLow(); + + // @notice Thrown when native token is deposited to the portal contract when disabled. + // For swc, the native token is actually qkc so we need to disable ETH deposits. + error OptimismPortal_NativeDepositForbidden();   /// @notice Thrown when the target of a withdrawal is not a proper dispute game. error OptimismPortal_ImproperDisputeGame(); @@ -538,6 +577,66 @@ revert OptimismPortal_InvalidRootClaim(); } }   + /// @notice Add a minter to the OptimismPortal contract. To disable, set an empty value. + function setMinter(address _minter) external { + if (msg.sender != proxyAdminOwner()) { + revert OptimismPortal_Unauthorized(); + } + QKCConfigStorage storage $ = _getQKCConfigStorage(); + $.minter = _minter; + emit MinterSet(_minter); + } + + /// @notice Mint a specific amount of L2 native token to an address. + function mintTransaction(address _to, uint256 _value) external metered(RECEIVE_DEFAULT_GAS_LIMIT) { + QKCConfigStorage storage $ = _getQKCConfigStorage(); + if (msg.sender != $.minter) { + revert OptimismPortal_Unauthorized(); + } + + if (_to == address(0)) { + revert OptimismPortal_BadTarget(); + } + // Compute the opaque data that will be emitted as part of the TransactionDeposited event. + // We use opaque data so that we can update the TransactionDeposited event in the future + // without breaking the current interface. + bytes memory opaqueData = abi.encodePacked(_value, _value, RECEIVE_DEFAULT_GAS_LIMIT, false, bytes("")); + + // Emit a TransactionDeposited event so that the rollup node can derive a deposit + // transaction for this deposit. + emit TransactionDeposited(Constants.QKC_DEPOSITOR_ACCOUNT, _to, DEPOSIT_VERSION, opaqueData); + } + + /// @notice set native deposit flag. Pass true to disable. + function setNativeDeposit(bool _disable) external { + if (msg.sender != proxyAdminOwner()) { + revert OptimismPortal_Unauthorized(); + } + QKCConfigStorage storage $ = _getQKCConfigStorage(); + $.disableNativeDeposit = _disable; + if (_disable) { + emit NativeDepositDisabled(); + } else { + emit NativeDepositEnabled(); + } + + // Compute the opaque data that will be emitted as part of the TransactionDeposited event. + // We use opaque data so that we can update the TransactionDeposited event in the future + // without breaking the current interface. + bytes memory opaqueData = abi.encodePacked( + uint256(0), + uint256(0), + uint64(SYSTEM_DEPOSIT_GAS_LIMIT), + false, + abi.encodeCall(IL2ToL1MessagePasser.setNativeDeposit, (_disable)) + ); + // Emit a TransactionDeposited event so that the rollup node can derive a deposit + // transaction for this deposit. + emit TransactionDeposited( + Constants.QKC_DEPOSITOR_ACCOUNT, Predeploys.L2_TO_L1_MESSAGE_PASSER, DEPOSIT_VERSION, opaqueData + ); + } + /// @notice Accepts deposits of ETH and data, and emits a TransactionDeposited event for use in /// deriving deposit transactions. Note that if a deposit is made by a contract, its /// address will be aliased when retrieved using `tx.origin` or `msg.sender`. Consider @@ -560,6 +659,12 @@ public payable metered(_gasLimit) { + if (msg.value > 0) { + QKCConfigStorage storage $ = _getQKCConfigStorage(); + if ($.disableNativeDeposit) { + revert OptimismPortal_NativeDepositForbidden(); + } + } // If using ETHLockbox, lock the ETH in the ETHLockbox. if (_isUsingLockbox()) { if (msg.value > 0) ethLockbox.lockETH{ value: msg.value }();
diff --git official optimism/packages/contracts-bedrock/src/L2/L2ToL1MessagePasser.sol SWC optimism/packages/contracts-bedrock/src/L2/L2ToL1MessagePasser.sol index b25a2a1248bc5ce4570284f841b0b86776d4b433..3908f6de3a6de8efe658ac9291bcb4066f503913 100644 --- official optimism/packages/contracts-bedrock/src/L2/L2ToL1MessagePasser.sol +++ SWC optimism/packages/contracts-bedrock/src/L2/L2ToL1MessagePasser.sol @@ -6,6 +6,7 @@ import { Types } from "src/libraries/Types.sol"; import { Hashing } from "src/libraries/Hashing.sol"; import { Encoding } from "src/libraries/Encoding.sol"; import { Burn } from "src/libraries/Burn.sol"; +import { Constants } from "src/libraries/Constants.sol";   // Interfaces import { ISemver } from "interfaces/universal/ISemver.sol"; @@ -29,6 +30,22 @@ /// @notice A unique value hashed with each withdrawal. uint240 internal msgNonce;   + // keccak256(abi.encode(uint256(keccak256("openzeppelin.storage.L2ToL1MessagePasser.QKCConfigStorage")) - 1)) & + // ~bytes32(uint256(0xff)) + bytes32 private constant _QKC_CONFIG_STORAGE_LOCATION = + 0x750f1ab2ed0ba2a4405b31f7b30e394ba3545975e71082ba3e508022a159f900; + /// @custom:storage-location erc7201:openzeppelin.storage.L2ToL1MessagePasser.QKCConfigStorage + + struct QKCConfigStorage { + bool disableNativeDeposit; + } + + function _getQKCConfigStorage() private pure returns (QKCConfigStorage storage $) { + assembly { + $.slot := _QKC_CONFIG_STORAGE_LOCATION + } + } + /// @notice Emitted any time a withdrawal is initiated. /// @param nonce Unique value corresponding to each withdrawal. /// @param sender The L2 account address which initiated the withdrawal. @@ -51,6 +68,14 @@ /// @notice Emitted when the balance of this contract is burned. /// @param amount Amount of ETh that was burned. event WithdrawerBalanceBurnt(uint256 indexed amount);   + /// @notice Emitted when native deposit is disabled. + event NativeDepositDisabled(); + /// @notice Emitted when native deposit is enabled. + event NativeDepositEnabled(); + + /// @notice Error for disabled native deposit/withdraw. + error L2ToL1MessagePasser_NativeDepositDisabled(); + /// @custom:semver 1.1.2 string public constant version = "1.1.2";   @@ -74,6 +99,10 @@ /// @param _target Address to call on L1 execution. /// @param _gasLimit Minimum gas limit for executing the message on L1. /// @param _data Data to forward to L1 target. function initiateWithdrawal(address _target, uint256 _gasLimit, bytes memory _data) public payable { + QKCConfigStorage storage $ = _getQKCConfigStorage(); + if ($.disableNativeDeposit && msg.value > 0) { + revert L2ToL1MessagePasser_NativeDepositDisabled(); + } bytes32 withdrawalHash = Hashing.hashWithdrawal( Types.WithdrawalTransaction({ nonce: messageNonce(), @@ -100,5 +129,19 @@ /// different structures. /// @return Nonce of the next message to be sent, with added message version. function messageNonce() public view returns (uint256) { return Encoding.encodeVersionedNonce(msgNonce, MESSAGE_VERSION); + } + + /// @notice set native deposit flag. Pass true to disable. + function setNativeDeposit(bool _disable) external { + if (msg.sender != Constants.QKC_DEPOSITOR_ACCOUNT) { + revert("L2ToL1MessagePasser: Only the depositor #2 can enable/disable native deposits"); + } + QKCConfigStorage storage $ = _getQKCConfigStorage(); + $.disableNativeDeposit = _disable; + if (_disable) { + emit NativeDepositDisabled(); + } else { + emit NativeDepositEnabled(); + } } }
diff --git official optimism/packages/contracts-bedrock/src/libraries/Constants.sol SWC optimism/packages/contracts-bedrock/src/libraries/Constants.sol index 50ad716697486e11a6d802db9a5b9f53e7991a99..f0c5d085205e6cadec511952b82ce38d3f89f752 100644 --- official optimism/packages/contracts-bedrock/src/libraries/Constants.sol +++ SWC optimism/packages/contracts-bedrock/src/libraries/Constants.sol @@ -41,6 +41,8 @@ /// @notice The address that represents the system caller responsible for L1 attributes /// transactions. address internal constant DEPOSITOR_ACCOUNT = 0xDeaDDEaDDeAdDeAdDEAdDEaddeAddEAdDEAd0001; + /// @notice Another fixed address to avoid potential risky calls for deposit tx on L2. + address internal constant QKC_DEPOSITOR_ACCOUNT = 0xDEAdDeaddEAdDeadDEaddeAdDeaddEAd00514b43;   /// @notice Returns the default values for the ResourceConfig. These are the recommended values /// for a production network.

Based on these PRs to enable fault proof for custom chains:

We enabled fault proof for the beta testnet.

diff --git official optimism/op-program/Makefile SWC optimism/op-program/Makefile index e12c8e5a1d3b7f75a60c32bdb9826932f795e840..5f27a9af07141a8d53eb4ff2f020f1ba1bcab646 100644 --- official optimism/op-program/Makefile +++ SWC optimism/op-program/Makefile @@ -74,6 +74,10 @@ test: go test -v ./...   + +verify-beta-testnet: op-program-host op-program-client + env GO111MODULE=on go run ./verify/beta-testnet/cmd/beta-testnet.go --l1 $$SEPOLIA_L1URL --l1.beacon $$SEPOLIA_BEACON_URL --l2 $$SEPOLIA_L2URL --datadir /tmp/test-beta-testnet + capture-mainnet-genesis: op-program-host op-program-client rm -rf "$(COMPAT_DIR)/mainnet-genesis" "$(COMPAT_DIR)/mainnet-genesis.tar.bz" env GO111MODULE=on go run ./verify/mainnet/cmd/mainnet.go --l1 $$MAINNET_L1URL --l1.beacon $$MAINNET_BEACON_URL --l2 $$MAINNET_L2URL --datadir "$(COMPAT_DIR)/mainnet-genesis" --l1.head "0x4903424f6cc2cfba7c2bf8c8f48ca46721c963fa64b411cfee3697b781e3e5f1" --l2.start "105235063" --l2.end "105235064"
diff --git official optimism/op-program/scripts/build-prestates.sh SWC optimism/op-program/scripts/build-prestates.sh index aa77ae0be5ded669665d35776d44bca34f9cffad..ce098f583e67a5ea98ef3cb8bc30e78b49b0e582 100755 --- official optimism/op-program/scripts/build-prestates.sh +++ SWC optimism/op-program/scripts/build-prestates.sh @@ -12,7 +12,7 @@ cd "${TMP_DIR}"   # Need to check out a fresh copy of the monorepo so we can switch to specific tags without it also affecting the # contents of this script (which is checked into the repo). -git clone https://github.com/ethereum-optimism/optimism --recurse-submodules +git clone https://github.com/ethstorage/optimism --recurse-submodules   STATES_DIR="${SCRIPTS_DIR}/../temp/states" LOGS_DIR="${SCRIPTS_DIR}/../temp/logs"
diff --git official optimism/op-program/verify/beta-testnet/cmd/beta-testnet.go SWC optimism/op-program/verify/beta-testnet/cmd/beta-testnet.go new file mode 100644 index 0000000000000000000000000000000000000000..79fdda66394c34e8a8650ee11b10f10937de6fac --- /dev/null +++ SWC optimism/op-program/verify/beta-testnet/cmd/beta-testnet.go @@ -0,0 +1,68 @@ +package main + +import ( + "context" + "flag" + "fmt" + "os" + + "github.com/ethereum-optimism/optimism/op-program/verify" + "github.com/ethereum-optimism/optimism/op-service/eth" + "github.com/ethereum/go-ethereum/common" +) + +func main() { + var l1RpcUrl string + var l1RpcKind string + var l1BeaconUrl string + var l2RpcUrl string + var dataDir string + var l1HashStr string + var l2Start uint64 + var l2End uint64 + flag.StringVar(&l1RpcUrl, "l1", "", "L1 RPC URL to use") + flag.StringVar(&l1BeaconUrl, "l1.beacon", "", "L1 Beacon URL to use") + flag.StringVar(&l1RpcKind, "l1-rpckind", "", "L1 RPC kind") + flag.StringVar(&l2RpcUrl, "l2", "", "L2 RPC URL to use") + flag.StringVar(&dataDir, "datadir", "", + "Directory to use for storing pre-images. If not set a temporary directory will be used.") + flag.StringVar(&l1HashStr, "l1.head", "", "Hash of L1 block to use") + flag.Uint64Var(&l2Start, "l2.start", 0, "Block number of agreed L2 block") + flag.Uint64Var(&l2End, "l2.end", 0, "Block number of claimed L2 block") + flag.Parse() + + if l1RpcUrl == "" { + _, _ = fmt.Fprintln(os.Stderr, "Must specify --l1 RPC URL") + os.Exit(2) + } + if l1BeaconUrl == "" { + _, _ = fmt.Fprintln(os.Stderr, "Must specify --l1.beacon URL") + os.Exit(2) + } + if l2RpcUrl == "" { + _, _ = fmt.Fprintln(os.Stderr, "Must specify --l2 RPC URL") + os.Exit(2) + } + + // Apply the custom configs by running op-program + runner, err := verify.NewRunner(l1RpcUrl, l1RpcKind, l1BeaconUrl, l2RpcUrl, dataDir, "3335", eth.ChainIDFromUInt64(3335), false) + if err != nil { + _, _ = fmt.Fprintf(os.Stderr, "Failed to create runner: %v\n", err.Error()) + os.Exit(1) + } + + if l1HashStr == "" && l2Start == 0 && l2End == 0 { + err = runner.RunToFinalized(context.Background()) + } else { + l1Hash := common.HexToHash(l1HashStr) + if l1Hash == (common.Hash{}) { + _, _ = fmt.Fprintf(os.Stderr, "Invalid --l1.head: %v\n", l1HashStr) + os.Exit(2) + } + err = runner.RunBetweenBlocks(context.Background(), l1Hash, l2Start, l2End) + } + if err != nil { + _, _ = fmt.Fprintf(os.Stderr, "Failed: %v\n", err.Error()) + os.Exit(1) + } +}
diff --git official optimism/op-e2e/bindings/batchinbox.go SWC optimism/op-e2e/bindings/batchinbox.go new file mode 100644 index 0000000000000000000000000000000000000000..24f67a1efa2d7b378c4817c191e2daa4cd5b3a29 --- /dev/null +++ SWC optimism/op-e2e/bindings/batchinbox.go @@ -0,0 +1,351 @@ +// Code generated - DO NOT EDIT. +// This file is a generated binding and any manual changes will be lost. + +// The contract is located at https://github.com/ethstorage/es-op-batchinbox/blob/main/src/BatchInbox.sol + +package bindings + +import ( + "errors" + "math/big" + "strings" + + ethereum "github.com/ethereum/go-ethereum" + "github.com/ethereum/go-ethereum/accounts/abi" + "github.com/ethereum/go-ethereum/accounts/abi/bind" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/event" +) + +// Reference imports to suppress errors if they are not otherwise used. +var ( + _ = errors.New + _ = big.NewInt + _ = strings.NewReader + _ = ethereum.NotFound + _ = bind.Bind + _ = common.Big1 + _ = types.BloomLookup + _ = event.NewSubscription + _ = abi.ConvertType +) + +// BatchInboxMetaData contains all meta data concerning the BatchInbox contract. +var BatchInboxMetaData = &bind.MetaData{ + ABI: "[{\"inputs\":[{\"internalType\":\"address\",\"name\":\"_esStorageContract\",\"type\":\"address\"}],\"stateMutability\":\"nonpayable\",\"type\":\"constructor\"},{\"inputs\":[],\"name\":\"BalanceNotEnough\",\"type\":\"error\"},{\"stateMutability\":\"payable\",\"type\":\"fallback\"},{\"inputs\":[{\"internalType\":\"address\",\"name\":\"\",\"type\":\"address\"}],\"name\":\"balances\",\"outputs\":[{\"internalType\":\"uint256\",\"name\":\"\",\"type\":\"uint256\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"address\",\"name\":\"_to\",\"type\":\"address\"}],\"name\":\"deposit\",\"outputs\":[],\"stateMutability\":\"payable\",\"type\":\"function\"},{\"inputs\":[],\"name\":\"esStorageContract\",\"outputs\":[{\"internalType\":\"contractStorageContract\",\"name\":\"\",\"type\":\"address\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"address\",\"name\":\"_to\",\"type\":\"address\"},{\"internalType\":\"uint256\",\"name\":\"_amount\",\"type\":\"uint256\"}],\"name\":\"withdraw\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"stateMutability\":\"payable\",\"type\":\"receive\"}]", + Bin: "0x60a060405234801561000f575f5ffd5b50604051610984380380610984833981810160405281019061003191906100c9565b8073ffffffffffffffffffffffffffffffffffffffff1660808173ffffffffffffffffffffffffffffffffffffffff1681525050506100f4565b5f5ffd5b5f73ffffffffffffffffffffffffffffffffffffffff82169050919050565b5f6100988261006f565b9050919050565b6100a88161008e565b81146100b2575f5ffd5b50565b5f815190506100c38161009f565b92915050565b5f602082840312156100de576100dd61006b565b5b5f6100eb848285016100b5565b91505092915050565b60805161086a61011a5f395f81816101990152818161022901526103b3015261086a5ff3fe608060405260043610610042575f3560e01c806322eb767d1461006f57806327e235e314610099578063f340fa01146100d5578063f3fef3a3146100f15761005b565b3661005b576100513334610119565b610059610178565b005b6100653334610119565b61006d610178565b005b34801561007a575f5ffd5b506100836103b1565b604051610090919061057f565b60405180910390f35b3480156100a4575f5ffd5b506100bf60048036038101906100ba91906105d7565b6103d5565b6040516100cc919061061a565b60405180910390f35b6100ef60048036038101906100ea91906105d7565b6103e9565b005b3480156100fc575f5ffd5b506101176004803603810190610112919061065d565b6103f6565b005b5f81031561017457805f5f8473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019081526020015f205f82825461016c91906106c8565b925050819055505b5050565b5f5f90505f5f90505f5b824990505f5f1b8103156102c5575f8203610227577f000000000000000000000000000000000000000000000000000000000000000073ffffffffffffffffffffffffffffffffffffffff16631ccbc6da6040518163ffffffff1660e01b8152600401602060405180830381865afa158015610200573d5f5f3e3d5ffd5b505050506040513d601f19601f82011682018060405250810190610224919061070f565b91505b7f000000000000000000000000000000000000000000000000000000000000000073ffffffffffffffffffffffffffffffffffffffff16634581a920838386620200006040518563ffffffff1660e01b81526004016102889392919061078b565b5f604051808303818588803b15801561029f575f5ffd5b505af11580156102b1573d5f5f3e3d5ffd5b505050505082806001019350506001610182575b5f83036102d4575050506103af565b5f83836102e191906107c0565b90505f5f5f3373ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019081526020015f205490508181101561035d576040517f9882883500000000000000000000000000000000000000000000000000000000815260040160405180910390fd5b81816103699190610801565b5f5f3373ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019081526020015f208190555050505050505b565b7f000000000000000000000000000000000000000000000000000000000000000081565b5f602052805f5260405f205f915090505481565b6103f38134610119565b50565b5f5f5f3373ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019081526020015f2054905081811015610470576040517f9882883500000000000000000000000000000000000000000000000000000000815260040160405180910390fd5b818161047c9190610801565b5f5f3373ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019081526020015f20819055508273ffffffffffffffffffffffffffffffffffffffff166108fc8390811502906040515f60405180830381858888f193505050501580156104ff573d5f5f3e3d5ffd5b50505050565b5f73ffffffffffffffffffffffffffffffffffffffff82169050919050565b5f819050919050565b5f61054761054261053d84610505565b610524565b610505565b9050919050565b5f6105588261052d565b9050919050565b5f6105698261054e565b9050919050565b6105798161055f565b82525050565b5f6020820190506105925f830184610570565b92915050565b5f5ffd5b5f6105a682610505565b9050919050565b6105b68161059c565b81146105c0575f5ffd5b50565b5f813590506105d1816105ad565b92915050565b5f602082840312156105ec576105eb610598565b5b5f6105f9848285016105c3565b91505092915050565b5f819050919050565b61061481610602565b82525050565b5f60208201905061062d5f83018461060b565b92915050565b61063c81610602565b8114610646575f5ffd5b50565b5f8135905061065781610633565b92915050565b5f5f6040838503121561067357610672610598565b5b5f610680858286016105c3565b925050602061069185828601610649565b9150509250929050565b7f4e487b71000000000000000000000000000000000000000000000000000000005f52601160045260245ffd5b5f6106d282610602565b91506106dd83610602565b92508282019050808211156106f5576106f461069b565b5b92915050565b5f8151905061070981610633565b92915050565b5f6020828403121561072457610723610598565b5b5f610731848285016106fb565b91505092915050565b5f819050919050565b61074c8161073a565b82525050565b5f819050919050565b5f61077561077061076b84610752565b610524565b610602565b9050919050565b6107858161075b565b82525050565b5f60608201905061079e5f830186610743565b6107ab602083018561060b565b6107b8604083018461077c565b949350505050565b5f6107ca82610602565b91506107d583610602565b92508282026107e381610602565b915082820484148315176107fa576107f961069b565b5b5092915050565b5f61080b82610602565b915061081683610602565b925082820390508181111561082e5761082d61069b565b5b9291505056fea2646970667358221220ad769cb754fdad1ddf58c9c13e0a15d938b6139b499ebc6a85cd815b50b05f8b64736f6c634300081c0033", +} + +// BatchInboxABI is the input ABI used to generate the binding from. +// Deprecated: Use BatchInboxMetaData.ABI instead. +var BatchInboxABI = BatchInboxMetaData.ABI + +// BatchInboxBin is the compiled bytecode used for deploying new contracts. +// Deprecated: Use BatchInboxMetaData.Bin instead. +var BatchInboxBin = BatchInboxMetaData.Bin + +// DeployBatchInbox deploys a new Ethereum contract, binding an instance of BatchInbox to it. +func DeployBatchInbox(auth *bind.TransactOpts, backend bind.ContractBackend, _esStorageContract common.Address) (common.Address, *types.Transaction, *BatchInbox, error) { + parsed, err := BatchInboxMetaData.GetAbi() + if err != nil { + return common.Address{}, nil, nil, err + } + if parsed == nil { + return common.Address{}, nil, nil, errors.New("GetABI returned nil") + } + + address, tx, contract, err := bind.DeployContract(auth, *parsed, common.FromHex(BatchInboxBin), backend, _esStorageContract) + if err != nil { + return common.Address{}, nil, nil, err + } + return address, tx, &BatchInbox{BatchInboxCaller: BatchInboxCaller{contract: contract}, BatchInboxTransactor: BatchInboxTransactor{contract: contract}, BatchInboxFilterer: BatchInboxFilterer{contract: contract}}, nil +} + +// BatchInbox is an auto generated Go binding around an Ethereum contract. +type BatchInbox struct { + BatchInboxCaller // Read-only binding to the contract + BatchInboxTransactor // Write-only binding to the contract + BatchInboxFilterer // Log filterer for contract events +} + +// BatchInboxCaller is an auto generated read-only Go binding around an Ethereum contract. +type BatchInboxCaller struct { + contract *bind.BoundContract // Generic contract wrapper for the low level calls +} + +// BatchInboxTransactor is an auto generated write-only Go binding around an Ethereum contract. +type BatchInboxTransactor struct { + contract *bind.BoundContract // Generic contract wrapper for the low level calls +} + +// BatchInboxFilterer is an auto generated log filtering Go binding around an Ethereum contract events. +type BatchInboxFilterer struct { + contract *bind.BoundContract // Generic contract wrapper for the low level calls +} + +// BatchInboxSession is an auto generated Go binding around an Ethereum contract, +// with pre-set call and transact options. +type BatchInboxSession struct { + Contract *BatchInbox // Generic contract binding to set the session for + CallOpts bind.CallOpts // Call options to use throughout this session + TransactOpts bind.TransactOpts // Transaction auth options to use throughout this session +} + +// BatchInboxCallerSession is an auto generated read-only Go binding around an Ethereum contract, +// with pre-set call options. +type BatchInboxCallerSession struct { + Contract *BatchInboxCaller // Generic contract caller binding to set the session for + CallOpts bind.CallOpts // Call options to use throughout this session +} + +// BatchInboxTransactorSession is an auto generated write-only Go binding around an Ethereum contract, +// with pre-set transact options. +type BatchInboxTransactorSession struct { + Contract *BatchInboxTransactor // Generic contract transactor binding to set the session for + TransactOpts bind.TransactOpts // Transaction auth options to use throughout this session +} + +// BatchInboxRaw is an auto generated low-level Go binding around an Ethereum contract. +type BatchInboxRaw struct { + Contract *BatchInbox // Generic contract binding to access the raw methods on +} + +// BatchInboxCallerRaw is an auto generated low-level read-only Go binding around an Ethereum contract. +type BatchInboxCallerRaw struct { + Contract *BatchInboxCaller // Generic read-only contract binding to access the raw methods on +} + +// BatchInboxTransactorRaw is an auto generated low-level write-only Go binding around an Ethereum contract. +type BatchInboxTransactorRaw struct { + Contract *BatchInboxTransactor // Generic write-only contract binding to access the raw methods on +} + +// NewBatchInbox creates a new instance of BatchInbox, bound to a specific deployed contract. +func NewBatchInbox(address common.Address, backend bind.ContractBackend) (*BatchInbox, error) { + contract, err := bindBatchInbox(address, backend, backend, backend) + if err != nil { + return nil, err + } + return &BatchInbox{BatchInboxCaller: BatchInboxCaller{contract: contract}, BatchInboxTransactor: BatchInboxTransactor{contract: contract}, BatchInboxFilterer: BatchInboxFilterer{contract: contract}}, nil +} + +// NewBatchInboxCaller creates a new read-only instance of BatchInbox, bound to a specific deployed contract. +func NewBatchInboxCaller(address common.Address, caller bind.ContractCaller) (*BatchInboxCaller, error) { + contract, err := bindBatchInbox(address, caller, nil, nil) + if err != nil { + return nil, err + } + return &BatchInboxCaller{contract: contract}, nil +} + +// NewBatchInboxTransactor creates a new write-only instance of BatchInbox, bound to a specific deployed contract. +func NewBatchInboxTransactor(address common.Address, transactor bind.ContractTransactor) (*BatchInboxTransactor, error) { + contract, err := bindBatchInbox(address, nil, transactor, nil) + if err != nil { + return nil, err + } + return &BatchInboxTransactor{contract: contract}, nil +} + +// NewBatchInboxFilterer creates a new log filterer instance of BatchInbox, bound to a specific deployed contract. +func NewBatchInboxFilterer(address common.Address, filterer bind.ContractFilterer) (*BatchInboxFilterer, error) { + contract, err := bindBatchInbox(address, nil, nil, filterer) + if err != nil { + return nil, err + } + return &BatchInboxFilterer{contract: contract}, nil +} + +// bindBatchInbox binds a generic wrapper to an already deployed contract. +func bindBatchInbox(address common.Address, caller bind.ContractCaller, transactor bind.ContractTransactor, filterer bind.ContractFilterer) (*bind.BoundContract, error) { + parsed, err := BatchInboxMetaData.GetAbi() + if err != nil { + return nil, err + } + return bind.NewBoundContract(address, *parsed, caller, transactor, filterer), nil +} + +// Call invokes the (constant) contract method with params as input values and +// sets the output to result. The result type might be a single field for simple +// returns, a slice of interfaces for anonymous returns and a struct for named +// returns. +func (_BatchInbox *BatchInboxRaw) Call(opts *bind.CallOpts, result *[]interface{}, method string, params ...interface{}) error { + return _BatchInbox.Contract.BatchInboxCaller.contract.Call(opts, result, method, params...) +} + +// Transfer initiates a plain transaction to move funds to the contract, calling +// its default method if one is available. +func (_BatchInbox *BatchInboxRaw) Transfer(opts *bind.TransactOpts) (*types.Transaction, error) { + return _BatchInbox.Contract.BatchInboxTransactor.contract.Transfer(opts) +} + +// Transact invokes the (paid) contract method with params as input values. +func (_BatchInbox *BatchInboxRaw) Transact(opts *bind.TransactOpts, method string, params ...interface{}) (*types.Transaction, error) { + return _BatchInbox.Contract.BatchInboxTransactor.contract.Transact(opts, method, params...) +} + +// Call invokes the (constant) contract method with params as input values and +// sets the output to result. The result type might be a single field for simple +// returns, a slice of interfaces for anonymous returns and a struct for named +// returns. +func (_BatchInbox *BatchInboxCallerRaw) Call(opts *bind.CallOpts, result *[]interface{}, method string, params ...interface{}) error { + return _BatchInbox.Contract.contract.Call(opts, result, method, params...) +} + +// Transfer initiates a plain transaction to move funds to the contract, calling +// its default method if one is available. +func (_BatchInbox *BatchInboxTransactorRaw) Transfer(opts *bind.TransactOpts) (*types.Transaction, error) { + return _BatchInbox.Contract.contract.Transfer(opts) +} + +// Transact invokes the (paid) contract method with params as input values. +func (_BatchInbox *BatchInboxTransactorRaw) Transact(opts *bind.TransactOpts, method string, params ...interface{}) (*types.Transaction, error) { + return _BatchInbox.Contract.contract.Transact(opts, method, params...) +} + +// Balances is a free data retrieval call binding the contract method 0x27e235e3. +// +// Solidity: function balances(address ) view returns(uint256) +func (_BatchInbox *BatchInboxCaller) Balances(opts *bind.CallOpts, arg0 common.Address) (*big.Int, error) { + var out []interface{} + err := _BatchInbox.contract.Call(opts, &out, "balances", arg0) + + if err != nil { + return *new(*big.Int), err + } + + out0 := *abi.ConvertType(out[0], new(*big.Int)).(**big.Int) + + return out0, err + +} + +// Balances is a free data retrieval call binding the contract method 0x27e235e3. +// +// Solidity: function balances(address ) view returns(uint256) +func (_BatchInbox *BatchInboxSession) Balances(arg0 common.Address) (*big.Int, error) { + return _BatchInbox.Contract.Balances(&_BatchInbox.CallOpts, arg0) +} + +// Balances is a free data retrieval call binding the contract method 0x27e235e3. +// +// Solidity: function balances(address ) view returns(uint256) +func (_BatchInbox *BatchInboxCallerSession) Balances(arg0 common.Address) (*big.Int, error) { + return _BatchInbox.Contract.Balances(&_BatchInbox.CallOpts, arg0) +} + +// EsStorageContract is a free data retrieval call binding the contract method 0x22eb767d. +// +// Solidity: function esStorageContract() view returns(address) +func (_BatchInbox *BatchInboxCaller) EsStorageContract(opts *bind.CallOpts) (common.Address, error) { + var out []interface{} + err := _BatchInbox.contract.Call(opts, &out, "esStorageContract") + + if err != nil { + return *new(common.Address), err + } + + out0 := *abi.ConvertType(out[0], new(common.Address)).(*common.Address) + + return out0, err + +} + +// EsStorageContract is a free data retrieval call binding the contract method 0x22eb767d. +// +// Solidity: function esStorageContract() view returns(address) +func (_BatchInbox *BatchInboxSession) EsStorageContract() (common.Address, error) { + return _BatchInbox.Contract.EsStorageContract(&_BatchInbox.CallOpts) +} + +// EsStorageContract is a free data retrieval call binding the contract method 0x22eb767d. +// +// Solidity: function esStorageContract() view returns(address) +func (_BatchInbox *BatchInboxCallerSession) EsStorageContract() (common.Address, error) { + return _BatchInbox.Contract.EsStorageContract(&_BatchInbox.CallOpts) +} + +// Deposit is a paid mutator transaction binding the contract method 0xf340fa01. +// +// Solidity: function deposit(address _to) payable returns() +func (_BatchInbox *BatchInboxTransactor) Deposit(opts *bind.TransactOpts, _to common.Address) (*types.Transaction, error) { + return _BatchInbox.contract.Transact(opts, "deposit", _to) +} + +// Deposit is a paid mutator transaction binding the contract method 0xf340fa01. +// +// Solidity: function deposit(address _to) payable returns() +func (_BatchInbox *BatchInboxSession) Deposit(_to common.Address) (*types.Transaction, error) { + return _BatchInbox.Contract.Deposit(&_BatchInbox.TransactOpts, _to) +} + +// Deposit is a paid mutator transaction binding the contract method 0xf340fa01. +// +// Solidity: function deposit(address _to) payable returns() +func (_BatchInbox *BatchInboxTransactorSession) Deposit(_to common.Address) (*types.Transaction, error) { + return _BatchInbox.Contract.Deposit(&_BatchInbox.TransactOpts, _to) +} + +// Withdraw is a paid mutator transaction binding the contract method 0xf3fef3a3. +// +// Solidity: function withdraw(address _to, uint256 _amount) returns() +func (_BatchInbox *BatchInboxTransactor) Withdraw(opts *bind.TransactOpts, _to common.Address, _amount *big.Int) (*types.Transaction, error) { + return _BatchInbox.contract.Transact(opts, "withdraw", _to, _amount) +} + +// Withdraw is a paid mutator transaction binding the contract method 0xf3fef3a3. +// +// Solidity: function withdraw(address _to, uint256 _amount) returns() +func (_BatchInbox *BatchInboxSession) Withdraw(_to common.Address, _amount *big.Int) (*types.Transaction, error) { + return _BatchInbox.Contract.Withdraw(&_BatchInbox.TransactOpts, _to, _amount) +} + +// Withdraw is a paid mutator transaction binding the contract method 0xf3fef3a3. +// +// Solidity: function withdraw(address _to, uint256 _amount) returns() +func (_BatchInbox *BatchInboxTransactorSession) Withdraw(_to common.Address, _amount *big.Int) (*types.Transaction, error) { + return _BatchInbox.Contract.Withdraw(&_BatchInbox.TransactOpts, _to, _amount) +} + +// Fallback is a paid mutator transaction binding the contract fallback function. +// +// Solidity: fallback() payable returns() +func (_BatchInbox *BatchInboxTransactor) Fallback(opts *bind.TransactOpts, calldata []byte) (*types.Transaction, error) { + return _BatchInbox.contract.RawTransact(opts, calldata) +} + +// Fallback is a paid mutator transaction binding the contract fallback function. +// +// Solidity: fallback() payable returns() +func (_BatchInbox *BatchInboxSession) Fallback(calldata []byte) (*types.Transaction, error) { + return _BatchInbox.Contract.Fallback(&_BatchInbox.TransactOpts, calldata) +} + +// Fallback is a paid mutator transaction binding the contract fallback function. +// +// Solidity: fallback() payable returns() +func (_BatchInbox *BatchInboxTransactorSession) Fallback(calldata []byte) (*types.Transaction, error) { + return _BatchInbox.Contract.Fallback(&_BatchInbox.TransactOpts, calldata) +} + +// Receive is a paid mutator transaction binding the contract receive function. +// +// Solidity: receive() payable returns() +func (_BatchInbox *BatchInboxTransactor) Receive(opts *bind.TransactOpts) (*types.Transaction, error) { + return _BatchInbox.contract.RawTransact(opts, nil) // calldata is disallowed for receive function +} + +// Receive is a paid mutator transaction binding the contract receive function. +// +// Solidity: receive() payable returns() +func (_BatchInbox *BatchInboxSession) Receive() (*types.Transaction, error) { + return _BatchInbox.Contract.Receive(&_BatchInbox.TransactOpts) +} + +// Receive is a paid mutator transaction binding the contract receive function. +// +// Solidity: receive() payable returns() +func (_BatchInbox *BatchInboxTransactorSession) Receive() (*types.Transaction, error) { + return _BatchInbox.Contract.Receive(&_BatchInbox.TransactOpts) +}
diff --git official optimism/op-e2e/bindings/mockstorage.go SWC optimism/op-e2e/bindings/mockstorage.go new file mode 100644 index 0000000000000000000000000000000000000000..2fd02f79fb217ad7689ce957ebb45e2860b79d2b --- /dev/null +++ SWC optimism/op-e2e/bindings/mockstorage.go @@ -0,0 +1,450 @@ +// Code generated - DO NOT EDIT. +// This file is a generated binding and any manual changes will be lost. + +// The contract is located at ./op-e2e/inbox/testcontract/mockstorage.sol + +package bindings + +import ( + "errors" + "math/big" + "strings" + + ethereum "github.com/ethereum/go-ethereum" + "github.com/ethereum/go-ethereum/accounts/abi" + "github.com/ethereum/go-ethereum/accounts/abi/bind" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/event" +) + +// Reference imports to suppress errors if they are not otherwise used. +var ( + _ = errors.New + _ = big.NewInt + _ = strings.NewReader + _ = ethereum.NotFound + _ = bind.Bind + _ = common.Big1 + _ = types.BloomLookup + _ = event.NewSubscription + _ = abi.ConvertType +) + +// MockEthStorageMetaData contains all meta data concerning the MockEthStorage contract. +var MockEthStorageMetaData = &bind.MetaData{ + ABI: "[{\"inputs\":[{\"internalType\":\"uint256\",\"name\":\"_cost\",\"type\":\"uint256\"}],\"stateMutability\":\"nonpayable\",\"type\":\"constructor\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":true,\"internalType\":\"uint256\",\"name\":\"kvIdx\",\"type\":\"uint256\"},{\"indexed\":true,\"internalType\":\"uint256\",\"name\":\"kvSize\",\"type\":\"uint256\"},{\"indexed\":true,\"internalType\":\"bytes32\",\"name\":\"dataHash\",\"type\":\"bytes32\"}],\"name\":\"PutBlob\",\"type\":\"event\"},{\"inputs\":[],\"name\":\"kvEntryCount\",\"outputs\":[{\"internalType\":\"uint256\",\"name\":\"\",\"type\":\"uint256\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"bytes32\",\"name\":\"_key\",\"type\":\"bytes32\"},{\"internalType\":\"uint256\",\"name\":\"_blobIdx\",\"type\":\"uint256\"},{\"internalType\":\"uint256\",\"name\":\"_length\",\"type\":\"uint256\"}],\"name\":\"putBlob\",\"outputs\":[],\"stateMutability\":\"payable\",\"type\":\"function\"},{\"inputs\":[],\"name\":\"upfrontPayment\",\"outputs\":[{\"internalType\":\"uint256\",\"name\":\"\",\"type\":\"uint256\"}],\"stateMutability\":\"view\",\"type\":\"function\"}]", + Bin: "0x60a0604052348015600e575f5ffd5b506040516105d43803806105d48339818101604052810190602e9190606d565b8060808181525050506093565b5f5ffd5b5f819050919050565b604f81603f565b81146058575f5ffd5b50565b5f815190506067816048565b92915050565b5f60208284031215607f57607e603b565b5b5f608a84828501605b565b91505092915050565b60805161052a6100aa5f395f60aa015261052a5ff3fe608060405260043610610033575f3560e01c80631ccbc6da146100375780634581a92014610061578063638ba9e91461007d575b5f5ffd5b348015610042575f5ffd5b5061004b6100a7565b6040516100589190610210565b60405180910390f35b61007b6004803603810190610076919061028a565b6100ce565b005b348015610088575f5ffd5b506100916101f3565b60405161009e9190610210565b60405180910390f35b5f7f0000000000000000000000000000000000000000000000000000000000000000905090565b5f824990505f5f1b8103610117576040517f08c379a000000000000000000000000000000000000000000000000000000000815260040161010e9061035a565b60405180910390fd5b5f5f1b840361015b576040517f08c379a0000000000000000000000000000000000000000000000000000000008152600401610152906103e8565b60405180910390fd5b6101636100a7565b3410156101a5576040517f08c379a000000000000000000000000000000000000000000000000000000000815260040161019c90610476565b60405180910390fd5b5f5f54905060015f546101b891906104c1565b5f819055508183827f8b7a21215282409938287ae262331bfe6411d35d3d46aa7e505ef02000524ac260405160405180910390a45050505050565b5f5481565b5f819050919050565b61020a816101f8565b82525050565b5f6020820190506102235f830184610201565b92915050565b5f5ffd5b5f819050919050565b61023f8161022d565b8114610249575f5ffd5b50565b5f8135905061025a81610236565b92915050565b610269816101f8565b8114610273575f5ffd5b50565b5f8135905061028481610260565b92915050565b5f5f5f606084860312156102a1576102a0610229565b5b5f6102ae8682870161024c565b93505060206102bf86828701610276565b92505060406102d086828701610276565b9150509250925092565b5f82825260208201905092915050565b7f45746853746f72616765436f6e74726163743a206661696c656420746f2067655f8201527f7420626c6f622068617368000000000000000000000000000000000000000000602082015250565b5f610344602b836102da565b915061034f826102ea565b604082019050919050565b5f6020820190508181035f83015261037181610338565b9050919050565b7f45746853746f72616765436f6e74726163743a206661696c656420746f2067655f8201527f7420626c6f62206b657900000000000000000000000000000000000000000000602082015250565b5f6103d2602a836102da565b91506103dd82610378565b604082019050919050565b5f6020820190508181035f8301526103ff816103c6565b9050919050565b7f446563656e7472616c697a65644b563a206e6f7420656e6f75676820626174635f8201527f68207061796d656e740000000000000000000000000000000000000000000000602082015250565b5f6104606029836102da565b915061046b82610406565b604082019050919050565b5f6020820190508181035f83015261048d81610454565b9050919050565b7f4e487b71000000000000000000000000000000000000000000000000000000005f52601160045260245ffd5b5f6104cb826101f8565b91506104d6836101f8565b92508282019050808211156104ee576104ed610494565b5b9291505056fea2646970667358221220d4d1a376c0d170d3fa44c75c4fba21e612513babbea0199bfa331bc05827b17464736f6c634300081c0033", +} + +// MockEthStorageABI is the input ABI used to generate the binding from. +// Deprecated: Use MockEthStorageMetaData.ABI instead. +var MockEthStorageABI = MockEthStorageMetaData.ABI + +// MockEthStorageBin is the compiled bytecode used for deploying new contracts. +// Deprecated: Use MockEthStorageMetaData.Bin instead. +var MockEthStorageBin = MockEthStorageMetaData.Bin + +// DeployMockEthStorage deploys a new Ethereum contract, binding an instance of MockEthStorage to it. +func DeployMockEthStorage(auth *bind.TransactOpts, backend bind.ContractBackend, _cost *big.Int) (common.Address, *types.Transaction, *MockEthStorage, error) { + parsed, err := MockEthStorageMetaData.GetAbi() + if err != nil { + return common.Address{}, nil, nil, err + } + if parsed == nil { + return common.Address{}, nil, nil, errors.New("GetABI returned nil") + } + + address, tx, contract, err := bind.DeployContract(auth, *parsed, common.FromHex(MockEthStorageBin), backend, _cost) + if err != nil { + return common.Address{}, nil, nil, err + } + return address, tx, &MockEthStorage{MockEthStorageCaller: MockEthStorageCaller{contract: contract}, MockEthStorageTransactor: MockEthStorageTransactor{contract: contract}, MockEthStorageFilterer: MockEthStorageFilterer{contract: contract}}, nil +} + +// MockEthStorage is an auto generated Go binding around an Ethereum contract. +type MockEthStorage struct { + MockEthStorageCaller // Read-only binding to the contract + MockEthStorageTransactor // Write-only binding to the contract + MockEthStorageFilterer // Log filterer for contract events +} + +// MockEthStorageCaller is an auto generated read-only Go binding around an Ethereum contract. +type MockEthStorageCaller struct { + contract *bind.BoundContract // Generic contract wrapper for the low level calls +} + +// MockEthStorageTransactor is an auto generated write-only Go binding around an Ethereum contract. +type MockEthStorageTransactor struct { + contract *bind.BoundContract // Generic contract wrapper for the low level calls +} + +// MockEthStorageFilterer is an auto generated log filtering Go binding around an Ethereum contract events. +type MockEthStorageFilterer struct { + contract *bind.BoundContract // Generic contract wrapper for the low level calls +} + +// MockEthStorageSession is an auto generated Go binding around an Ethereum contract, +// with pre-set call and transact options. +type MockEthStorageSession struct { + Contract *MockEthStorage // Generic contract binding to set the session for + CallOpts bind.CallOpts // Call options to use throughout this session + TransactOpts bind.TransactOpts // Transaction auth options to use throughout this session +} + +// MockEthStorageCallerSession is an auto generated read-only Go binding around an Ethereum contract, +// with pre-set call options. +type MockEthStorageCallerSession struct { + Contract *MockEthStorageCaller // Generic contract caller binding to set the session for + CallOpts bind.CallOpts // Call options to use throughout this session +} + +// MockEthStorageTransactorSession is an auto generated write-only Go binding around an Ethereum contract, +// with pre-set transact options. +type MockEthStorageTransactorSession struct { + Contract *MockEthStorageTransactor // Generic contract transactor binding to set the session for + TransactOpts bind.TransactOpts // Transaction auth options to use throughout this session +} + +// MockEthStorageRaw is an auto generated low-level Go binding around an Ethereum contract. +type MockEthStorageRaw struct { + Contract *MockEthStorage // Generic contract binding to access the raw methods on +} + +// MockEthStorageCallerRaw is an auto generated low-level read-only Go binding around an Ethereum contract. +type MockEthStorageCallerRaw struct { + Contract *MockEthStorageCaller // Generic read-only contract binding to access the raw methods on +} + +// MockEthStorageTransactorRaw is an auto generated low-level write-only Go binding around an Ethereum contract. +type MockEthStorageTransactorRaw struct { + Contract *MockEthStorageTransactor // Generic write-only contract binding to access the raw methods on +} + +// NewMockEthStorage creates a new instance of MockEthStorage, bound to a specific deployed contract. +func NewMockEthStorage(address common.Address, backend bind.ContractBackend) (*MockEthStorage, error) { + contract, err := bindMockEthStorage(address, backend, backend, backend) + if err != nil { + return nil, err + } + return &MockEthStorage{MockEthStorageCaller: MockEthStorageCaller{contract: contract}, MockEthStorageTransactor: MockEthStorageTransactor{contract: contract}, MockEthStorageFilterer: MockEthStorageFilterer{contract: contract}}, nil +} + +// NewMockEthStorageCaller creates a new read-only instance of MockEthStorage, bound to a specific deployed contract. +func NewMockEthStorageCaller(address common.Address, caller bind.ContractCaller) (*MockEthStorageCaller, error) { + contract, err := bindMockEthStorage(address, caller, nil, nil) + if err != nil { + return nil, err + } + return &MockEthStorageCaller{contract: contract}, nil +} + +// NewMockEthStorageTransactor creates a new write-only instance of MockEthStorage, bound to a specific deployed contract. +func NewMockEthStorageTransactor(address common.Address, transactor bind.ContractTransactor) (*MockEthStorageTransactor, error) { + contract, err := bindMockEthStorage(address, nil, transactor, nil) + if err != nil { + return nil, err + } + return &MockEthStorageTransactor{contract: contract}, nil +} + +// NewMockEthStorageFilterer creates a new log filterer instance of MockEthStorage, bound to a specific deployed contract. +func NewMockEthStorageFilterer(address common.Address, filterer bind.ContractFilterer) (*MockEthStorageFilterer, error) { + contract, err := bindMockEthStorage(address, nil, nil, filterer) + if err != nil { + return nil, err + } + return &MockEthStorageFilterer{contract: contract}, nil +} + +// bindMockEthStorage binds a generic wrapper to an already deployed contract. +func bindMockEthStorage(address common.Address, caller bind.ContractCaller, transactor bind.ContractTransactor, filterer bind.ContractFilterer) (*bind.BoundContract, error) { + parsed, err := MockEthStorageMetaData.GetAbi() + if err != nil { + return nil, err + } + return bind.NewBoundContract(address, *parsed, caller, transactor, filterer), nil +} + +// Call invokes the (constant) contract method with params as input values and +// sets the output to result. The result type might be a single field for simple +// returns, a slice of interfaces for anonymous returns and a struct for named +// returns. +func (_MockEthStorage *MockEthStorageRaw) Call(opts *bind.CallOpts, result *[]interface{}, method string, params ...interface{}) error { + return _MockEthStorage.Contract.MockEthStorageCaller.contract.Call(opts, result, method, params...) +} + +// Transfer initiates a plain transaction to move funds to the contract, calling +// its default method if one is available. +func (_MockEthStorage *MockEthStorageRaw) Transfer(opts *bind.TransactOpts) (*types.Transaction, error) { + return _MockEthStorage.Contract.MockEthStorageTransactor.contract.Transfer(opts) +} + +// Transact invokes the (paid) contract method with params as input values. +func (_MockEthStorage *MockEthStorageRaw) Transact(opts *bind.TransactOpts, method string, params ...interface{}) (*types.Transaction, error) { + return _MockEthStorage.Contract.MockEthStorageTransactor.contract.Transact(opts, method, params...) +} + +// Call invokes the (constant) contract method with params as input values and +// sets the output to result. The result type might be a single field for simple +// returns, a slice of interfaces for anonymous returns and a struct for named +// returns. +func (_MockEthStorage *MockEthStorageCallerRaw) Call(opts *bind.CallOpts, result *[]interface{}, method string, params ...interface{}) error { + return _MockEthStorage.Contract.contract.Call(opts, result, method, params...) +} + +// Transfer initiates a plain transaction to move funds to the contract, calling +// its default method if one is available. +func (_MockEthStorage *MockEthStorageTransactorRaw) Transfer(opts *bind.TransactOpts) (*types.Transaction, error) { + return _MockEthStorage.Contract.contract.Transfer(opts) +} + +// Transact invokes the (paid) contract method with params as input values. +func (_MockEthStorage *MockEthStorageTransactorRaw) Transact(opts *bind.TransactOpts, method string, params ...interface{}) (*types.Transaction, error) { + return _MockEthStorage.Contract.contract.Transact(opts, method, params...) +} + +// KvEntryCount is a free data retrieval call binding the contract method 0x638ba9e9. +// +// Solidity: function kvEntryCount() view returns(uint256) +func (_MockEthStorage *MockEthStorageCaller) KvEntryCount(opts *bind.CallOpts) (*big.Int, error) { + var out []interface{} + err := _MockEthStorage.contract.Call(opts, &out, "kvEntryCount") + + if err != nil { + return *new(*big.Int), err + } + + out0 := *abi.ConvertType(out[0], new(*big.Int)).(**big.Int) + + return out0, err + +} + +// KvEntryCount is a free data retrieval call binding the contract method 0x638ba9e9. +// +// Solidity: function kvEntryCount() view returns(uint256) +func (_MockEthStorage *MockEthStorageSession) KvEntryCount() (*big.Int, error) { + return _MockEthStorage.Contract.KvEntryCount(&_MockEthStorage.CallOpts) +} + +// KvEntryCount is a free data retrieval call binding the contract method 0x638ba9e9. +// +// Solidity: function kvEntryCount() view returns(uint256) +func (_MockEthStorage *MockEthStorageCallerSession) KvEntryCount() (*big.Int, error) { + return _MockEthStorage.Contract.KvEntryCount(&_MockEthStorage.CallOpts) +} + +// UpfrontPayment is a free data retrieval call binding the contract method 0x1ccbc6da. +// +// Solidity: function upfrontPayment() view returns(uint256) +func (_MockEthStorage *MockEthStorageCaller) UpfrontPayment(opts *bind.CallOpts) (*big.Int, error) { + var out []interface{} + err := _MockEthStorage.contract.Call(opts, &out, "upfrontPayment") + + if err != nil { + return *new(*big.Int), err + } + + out0 := *abi.ConvertType(out[0], new(*big.Int)).(**big.Int) + + return out0, err + +} + +// UpfrontPayment is a free data retrieval call binding the contract method 0x1ccbc6da. +// +// Solidity: function upfrontPayment() view returns(uint256) +func (_MockEthStorage *MockEthStorageSession) UpfrontPayment() (*big.Int, error) { + return _MockEthStorage.Contract.UpfrontPayment(&_MockEthStorage.CallOpts) +} + +// UpfrontPayment is a free data retrieval call binding the contract method 0x1ccbc6da. +// +// Solidity: function upfrontPayment() view returns(uint256) +func (_MockEthStorage *MockEthStorageCallerSession) UpfrontPayment() (*big.Int, error) { + return _MockEthStorage.Contract.UpfrontPayment(&_MockEthStorage.CallOpts) +} + +// PutBlob is a paid mutator transaction binding the contract method 0x4581a920. +// +// Solidity: function putBlob(bytes32 _key, uint256 _blobIdx, uint256 _length) payable returns() +func (_MockEthStorage *MockEthStorageTransactor) PutBlob(opts *bind.TransactOpts, _key [32]byte, _blobIdx *big.Int, _length *big.Int) (*types.Transaction, error) { + return _MockEthStorage.contract.Transact(opts, "putBlob", _key, _blobIdx, _length) +} + +// PutBlob is a paid mutator transaction binding the contract method 0x4581a920. +// +// Solidity: function putBlob(bytes32 _key, uint256 _blobIdx, uint256 _length) payable returns() +func (_MockEthStorage *MockEthStorageSession) PutBlob(_key [32]byte, _blobIdx *big.Int, _length *big.Int) (*types.Transaction, error) { + return _MockEthStorage.Contract.PutBlob(&_MockEthStorage.TransactOpts, _key, _blobIdx, _length) +} + +// PutBlob is a paid mutator transaction binding the contract method 0x4581a920. +// +// Solidity: function putBlob(bytes32 _key, uint256 _blobIdx, uint256 _length) payable returns() +func (_MockEthStorage *MockEthStorageTransactorSession) PutBlob(_key [32]byte, _blobIdx *big.Int, _length *big.Int) (*types.Transaction, error) { + return _MockEthStorage.Contract.PutBlob(&_MockEthStorage.TransactOpts, _key, _blobIdx, _length) +} + +// MockEthStoragePutBlobIterator is returned from FilterPutBlob and is used to iterate over the raw logs and unpacked data for PutBlob events raised by the MockEthStorage contract. +type MockEthStoragePutBlobIterator struct { + Event *MockEthStoragePutBlob // Event containing the contract specifics and raw log + + contract *bind.BoundContract // Generic contract to use for unpacking event data + event string // Event name to use for unpacking event data + + logs chan types.Log // Log channel receiving the found contract events + sub ethereum.Subscription // Subscription for errors, completion and termination + done bool // Whether the subscription completed delivering logs + fail error // Occurred error to stop iteration +} + +// Next advances the iterator to the subsequent event, returning whether there +// are any more events found. In case of a retrieval or parsing error, false is +// returned and Error() can be queried for the exact failure. +func (it *MockEthStoragePutBlobIterator) Next() bool { + // If the iterator failed, stop iterating + if it.fail != nil { + return false + } + // If the iterator completed, deliver directly whatever's available + if it.done { + select { + case log := <-it.logs: + it.Event = new(MockEthStoragePutBlob) + if err := it.contract.UnpackLog(it.Event, it.event, log); err != nil { + it.fail = err + return false + } + it.Event.Raw = log + return true + + default: + return false + } + } + // Iterator still in progress, wait for either a data or an error event + select { + case log := <-it.logs: + it.Event = new(MockEthStoragePutBlob) + if err := it.contract.UnpackLog(it.Event, it.event, log); err != nil { + it.fail = err + return false + } + it.Event.Raw = log + return true + + case err := <-it.sub.Err(): + it.done = true + it.fail = err + return it.Next() + } +} + +// Error returns any retrieval or parsing error occurred during filtering. +func (it *MockEthStoragePutBlobIterator) Error() error { + return it.fail +} + +// Close terminates the iteration process, releasing any pending underlying +// resources. +func (it *MockEthStoragePutBlobIterator) Close() error { + it.sub.Unsubscribe() + return nil +} + +// MockEthStoragePutBlob represents a PutBlob event raised by the MockEthStorage contract. +type MockEthStoragePutBlob struct { + KvIdx *big.Int + KvSize *big.Int + DataHash [32]byte + Raw types.Log // Blockchain specific contextual infos +} + +// FilterPutBlob is a free log retrieval operation binding the contract event 0x8b7a21215282409938287ae262331bfe6411d35d3d46aa7e505ef02000524ac2. +// +// Solidity: event PutBlob(uint256 indexed kvIdx, uint256 indexed kvSize, bytes32 indexed dataHash) +func (_MockEthStorage *MockEthStorageFilterer) FilterPutBlob(opts *bind.FilterOpts, kvIdx []*big.Int, kvSize []*big.Int, dataHash [][32]byte) (*MockEthStoragePutBlobIterator, error) { + + var kvIdxRule []interface{} + for _, kvIdxItem := range kvIdx { + kvIdxRule = append(kvIdxRule, kvIdxItem) + } + var kvSizeRule []interface{} + for _, kvSizeItem := range kvSize { + kvSizeRule = append(kvSizeRule, kvSizeItem) + } + var dataHashRule []interface{} + for _, dataHashItem := range dataHash { + dataHashRule = append(dataHashRule, dataHashItem) + } + + logs, sub, err := _MockEthStorage.contract.FilterLogs(opts, "PutBlob", kvIdxRule, kvSizeRule, dataHashRule) + if err != nil { + return nil, err + } + return &MockEthStoragePutBlobIterator{contract: _MockEthStorage.contract, event: "PutBlob", logs: logs, sub: sub}, nil +} + +// WatchPutBlob is a free log subscription operation binding the contract event 0x8b7a21215282409938287ae262331bfe6411d35d3d46aa7e505ef02000524ac2. +// +// Solidity: event PutBlob(uint256 indexed kvIdx, uint256 indexed kvSize, bytes32 indexed dataHash) +func (_MockEthStorage *MockEthStorageFilterer) WatchPutBlob(opts *bind.WatchOpts, sink chan<- *MockEthStoragePutBlob, kvIdx []*big.Int, kvSize []*big.Int, dataHash [][32]byte) (event.Subscription, error) { + + var kvIdxRule []interface{} + for _, kvIdxItem := range kvIdx { + kvIdxRule = append(kvIdxRule, kvIdxItem) + } + var kvSizeRule []interface{} + for _, kvSizeItem := range kvSize { + kvSizeRule = append(kvSizeRule, kvSizeItem) + } + var dataHashRule []interface{} + for _, dataHashItem := range dataHash { + dataHashRule = append(dataHashRule, dataHashItem) + } + + logs, sub, err := _MockEthStorage.contract.WatchLogs(opts, "PutBlob", kvIdxRule, kvSizeRule, dataHashRule) + if err != nil { + return nil, err + } + return event.NewSubscription(func(quit <-chan struct{}) error { + defer sub.Unsubscribe() + for { + select { + case log := <-logs: + // New log arrived, parse the event and forward to the user + event := new(MockEthStoragePutBlob) + if err := _MockEthStorage.contract.UnpackLog(event, "PutBlob", log); err != nil { + return err + } + event.Raw = log + + select { + case sink <- event: + case err := <-sub.Err(): + return err + case <-quit: + return nil + } + case err := <-sub.Err(): + return err + case <-quit: + return nil + } + } + }), nil +} + +// ParsePutBlob is a log parse operation binding the contract event 0x8b7a21215282409938287ae262331bfe6411d35d3d46aa7e505ef02000524ac2. +// +// Solidity: event PutBlob(uint256 indexed kvIdx, uint256 indexed kvSize, bytes32 indexed dataHash) +func (_MockEthStorage *MockEthStorageFilterer) ParsePutBlob(log types.Log) (*MockEthStoragePutBlob, error) { + event := new(MockEthStoragePutBlob) + if err := _MockEthStorage.contract.UnpackLog(event, "PutBlob", log); err != nil { + return nil, err + } + event.Raw = log + return event, nil +}
diff --git official optimism/op-e2e/inbox/inbox_test.go SWC optimism/op-e2e/inbox/inbox_test.go new file mode 100644 index 0000000000000000000000000000000000000000..f391043878b726104d13da324b7a6746cd01d828 --- /dev/null +++ SWC optimism/op-e2e/inbox/inbox_test.go @@ -0,0 +1,148 @@ +package inbox + +import ( + "context" + "math/big" + "testing" + "time" + + batcherFlags "github.com/ethereum-optimism/optimism/op-batcher/flags" + op_e2e "github.com/ethereum-optimism/optimism/op-e2e" + "github.com/ethereum-optimism/optimism/op-e2e/bindings" + "github.com/ethereum-optimism/optimism/op-e2e/e2eutils/wait" + "github.com/ethereum-optimism/optimism/op-e2e/system/e2esys" + "github.com/ethereum/go-ethereum/accounts/abi/bind" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/ethclient" + "github.com/stretchr/testify/require" +) + +var ( + ctx, _ = context.WithTimeout(context.Background(), 20*time.Second) + cost = big.NewInt(1500000000000000) +) + +func TestBatchInboxFunctionSuccess(t *testing.T) { + op_e2e.InitParallel(t) + + sys, l1Client := startSystemWithBatchInboxContract(t) + t.Cleanup(sys.Close) + + // Wait for batch submitted and check event + requireEventualBatcherTx(t, &sys.Cfg, l1Client, 8*time.Second) +} + +func startSystemWithBatchInboxContract(t *testing.T) (*e2esys.System, *ethclient.Client) { + cfg := e2esys.DefaultSystemConfig(t) + cfg.DataAvailabilityType = batcherFlags.BlobsType + cfg.BatcherTargetNumFrames = 6 + cfg.DeployConfig.UseInboxContract = true + c, ok := cfg.Nodes["sequencer"] + require.True(t, ok, "sequencer is required") + c.Driver.SequencerEnabled = true + + sys, err := cfg.Start(t, e2esys.StartOption{ + Key: "afterL1Start", + Action: func(cfg *e2esys.SystemConfig, s *e2esys.System) { + l1Client := s.NodeClient(e2esys.RoleL1) + // Deploy mock storage contract + mockStorageAddr := deployContract(t, cfg, l1Client, bindings.MockEthStorageMetaData, cost) + // Deploy BatchInbox.sol contract + batchInboxAddr := deployContract(t, cfg, l1Client, bindings.BatchInboxMetaData, mockStorageAddr) + t.Logf("mock storage %s, batchInbox %s", mockStorageAddr.Hex(), batchInboxAddr.Hex()) + // Set BatchInboxAddress + cfg.DeployConfig.BatchInboxAddress = batchInboxAddr + // Deposit token + transferNativeTokenToBatchInboxAddress(t, cfg, l1Client, new(big.Int).Mul(cost, big.NewInt(1000))) + }, + }) + require.Nil(t, err, "Error starting up system") + return sys, sys.NodeClient(e2esys.RoleL1) +} + +func requireEventualBatcherTx(t *testing.T, cfg *e2esys.SystemConfig, l1Client *ethclient.Client, timeout time.Duration) { + require.Eventually(t, func() bool { + b, err := l1Client.BlockByNumber(ctx, nil) + require.NoError(t, err) + for _, tx := range b.Transactions() { + if tx.To() == nil || tx.To().Cmp(cfg.DeployConfig.BatchInboxAddress) != 0 { + continue + } + receipt, err := l1Client.TransactionReceipt(ctx, tx.Hash()) + require.NoError(t, err) + if len(receipt.Logs) == 0 { + continue + } + balanceBefore, err := l1Client.BalanceAt(ctx, receipt.Logs[0].Address, new(big.Int).Add(receipt.BlockNumber, big.NewInt(-1))) + require.NoError(t, err) + balanceAfter, err := l1Client.BalanceAt(ctx, receipt.Logs[0].Address, receipt.BlockNumber) + require.NoError(t, err) + require.True(t, balanceAfter.Uint64()-balanceBefore.Uint64() == cost.Uint64()*uint64(len(receipt.Logs)), "Cost is mismatch") + return true + } + return false + }, timeout, time.Second, "expected batcher tx type didn't arrive") +} + +func deployContract(t *testing.T, cfg *e2esys.SystemConfig, client *ethclient.Client, meta *bind.MetaData, + params ...interface{}) common.Address { + ethPrivKey := cfg.Secrets.Batcher + fromAddr := cfg.Secrets.Addresses().Batcher + + nonce, err := client.PendingNonceAt(context.Background(), fromAddr) + require.NoError(t, err) + gasPrice, err := client.SuggestGasPrice(context.Background()) + require.NoError(t, err) + + auth, err := bind.NewKeyedTransactorWithChainID(ethPrivKey, cfg.L1ChainIDBig()) + require.NoError(t, err) + auth.Nonce = big.NewInt(int64(nonce)) + auth.Value = big.NewInt(0) + auth.GasLimit = uint64(3000000) + auth.GasPrice = gasPrice + + parsed, err := meta.GetAbi() + require.NoError(t, err) + + address, tx, _, err := bind.DeployContract(auth, *parsed, common.FromHex(meta.Bin), client, params...) + require.NoError(t, err) + + _, err = wait.ForReceiptOK(ctx, client, tx.Hash()) + require.NoError(t, err) + return address +} + +func transferNativeTokenToBatchInboxAddress(t *testing.T, cfg *e2esys.SystemConfig, client *ethclient.Client, amount *big.Int) { + ethPrivKey := cfg.Secrets.Batcher + fromAddr := cfg.Secrets.Addresses().Batcher + + gasTipCap, err := client.SuggestGasTipCap(ctx) + require.NoError(t, err) + head, err := client.HeaderByNumber(ctx, nil) + require.NoError(t, err) + gasFeeCap := new(big.Int).Add( + gasTipCap, + new(big.Int).Mul(head.BaseFee, big.NewInt(2)), + ) + + nonce, err := client.NonceAt(ctx, fromAddr, nil) + require.NoError(t, err) + tx := types.MustSignNewTx(ethPrivKey, types.LatestSignerForChainID(cfg.L1ChainIDBig()), &types.DynamicFeeTx{ + ChainID: cfg.L1ChainIDBig(), + Nonce: nonce, + To: &cfg.DeployConfig.BatchInboxAddress, + Value: amount, + GasTipCap: gasTipCap, + GasFeeCap: gasFeeCap, + Gas: 1000000, + }) + err = client.SendTransaction(ctx, tx) + require.NoError(t, err) + _, err = wait.ForReceiptOK(context.Background(), client, tx.Hash()) + require.NoError(t, err) + + balance, err := client.BalanceAt(ctx, cfg.DeployConfig.BatchInboxAddress, nil) + require.NoError(t, err) + require.True(t, balance.Uint64() == amount.Uint64(), "balance no match") +}
diff --git official optimism/op-e2e/inbox/testcontract/mockstorage.sol SWC optimism/op-e2e/inbox/testcontract/mockstorage.sol new file mode 100644 index 0000000000000000000000000000000000000000..f3316089c47a51ed4f10b8648cdf6a495f4358b8 --- /dev/null +++ SWC optimism/op-e2e/inbox/testcontract/mockstorage.sol @@ -0,0 +1,39 @@ +// SPDX-License-Identifier: MIT +pragma solidity ^0.8.24; + +contract EthStorageContract { + uint256 internal immutable COST; + uint256 public kvEntryCount; + + /// @notice Emitted when a BLOB is appended. + /// @param kvIdx The index of the KV pair + /// @param kvSize The size of the KV pair + /// @param dataHash The hash of the data + event PutBlob(uint256 indexed kvIdx, uint256 indexed kvSize, bytes32 indexed dataHash); + + constructor(uint256 _cost) { + COST = _cost; + } + + /// @notice Write a large value to KV store. If the KV pair exists, overrides it. + /// Otherwise, will append the KV to the KV array. + /// @param _key The key of the KV pair + /// @param _blobIdx The index of the blob + /// @param _length The length of the blob + function putBlob(bytes32 _key, uint256 _blobIdx, uint256 _length) public payable virtual { + bytes32 dataHash = blobhash(_blobIdx); + require(dataHash != 0, "EthStorageContract: failed to get blob hash"); + require(_key != 0, "EthStorageContract: failed to get blob key"); + require(msg.value >= upfrontPayment(), "DecentralizedKV: not enough batch payment"); + + uint256 kvIndex = kvEntryCount; + kvEntryCount = kvEntryCount + 1; + + emit PutBlob(kvIndex, _length, dataHash); + } + + /// @notice Evaluate the storage cost of a single put(). + function upfrontPayment() public view virtual returns (uint256) { + return COST ; + } +}
diff --git official optimism/op-e2e/system/e2esys/setup.go SWC optimism/op-e2e/system/e2esys/setup.go index a79bb9b5384b12fe848d35c9a9c0373c532e6ede..9dccd5ae22a3bc87b248e119e3c9316d2be17158 100644 --- official optimism/op-e2e/system/e2esys/setup.go +++ SWC optimism/op-e2e/system/e2esys/setup.go @@ -117,6 +117,7 @@ secrets := secrets.DefaultSecrets deployConfig := config.DeployConfig(sco.AllocType) require.Nil(t, deployConfig.L2GenesisJovianTimeOffset, "jovian not supported yet") deployConfig.L1GenesisBlockTimestamp = hexutil.Uint64(time.Now().Unix()) + deployConfig.L2GenesisBlobTimeOffset = nil e2eutils.ApplyDeployConfigForks(deployConfig) require.NoError(t, deployConfig.Check(testlog.Logger(t, log.LevelInfo)), "Deploy config is invalid, do you need to run make devnet-allocs?") @@ -684,6 +685,11 @@ CommitmentType: altda.GenericCommitmentString, } }   + var inboxContractConfig *rollup.InboxContractConfig + if cfg.DeployConfig.UseInboxContract { + inboxContractConfig = &rollup.InboxContractConfig{UseInboxContract: true} + } + makeRollupConfig := func() rollup.Config { return rollup.Config{ Genesis: rollup.Genesis{ @@ -720,10 +726,16 @@ JovianTime: cfg.DeployConfig.JovianTime(uint64(cfg.DeployConfig.L1GenesisBlockTimestamp)), InteropTime: cfg.DeployConfig.InteropTime(uint64(cfg.DeployConfig.L1GenesisBlockTimestamp)), ProtocolVersionsAddress: cfg.L1Deployments.ProtocolVersionsProxy, AltDAConfig: rollupAltDAConfig, + InboxContractConfig: inboxContractConfig, ChainOpConfig: &params.OptimismConfig{ - EIP1559Elasticity: cfg.DeployConfig.EIP1559Elasticity, - EIP1559Denominator: cfg.DeployConfig.EIP1559Denominator, - EIP1559DenominatorCanyon: &cfg.DeployConfig.EIP1559DenominatorCanyon, + EIP1559Elasticity: cfg.DeployConfig.EIP1559Elasticity, + EIP1559Denominator: cfg.DeployConfig.EIP1559Denominator, + EIP1559DenominatorCanyon: &cfg.DeployConfig.EIP1559DenominatorCanyon, + L2BlobTime: cfg.DeployConfig.L2BlobTime(l1Block.Time()), + SoulGasTokenTime: cfg.DeployConfig.SoulGasTokenTime(l1Block.Time()), + IsSoulBackedByNative: cfg.DeployConfig.IsSoulBackedByNative, + L1BaseFeeScalarMultiplier: cfg.DeployConfig.L1BaseFeeScalarMultiplier, + L1BlobBaseFeeScalarMultiplier: cfg.DeployConfig.L1BlobBaseFeeScalarMultiplier, }, } } @@ -809,6 +821,11 @@ l1Client := sys.NodeClient(RoleL1) _, err = geth.WaitForBlock(big.NewInt(2), l1Client) if err != nil { return nil, fmt.Errorf("waiting for blocks: %w", err) + } + + // exec func which needs to run after L1 node starts and before L2 nodes start. + if action, ok := parsedStartOpts.Get("afterL1Start", ""); ok { + action(&cfg, sys) }   sys.Mocknet = mocknet.New()
diff --git official optimism/op-e2e/actions/sgt/sgt_test.go SWC optimism/op-e2e/actions/sgt/sgt_test.go new file mode 100644 index 0000000000000000000000000000000000000000..aaae0d3ce7e983adf3a88a9b34c090932ddbe6fa --- /dev/null +++ SWC optimism/op-e2e/actions/sgt/sgt_test.go @@ -0,0 +1,114 @@ +package sgt + +import ( + "math/big" + "testing" + + "github.com/ethereum-optimism/optimism/op-e2e/actions/helpers" + "github.com/ethereum-optimism/optimism/op-e2e/bindings" + "github.com/ethereum-optimism/optimism/op-e2e/e2eutils" + "github.com/ethereum-optimism/optimism/op-service/predeploys" + "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/core/types" + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/params" + "github.com/stretchr/testify/require" +) + +func TestDynamicSGT(gt *testing.T) { + t := helpers.NewDefaultTesting(gt) + dp := e2eutils.MakeDeployParams(t, helpers.DefaultRollupTestParams()) + dp.DeployConfig.DeploySoulGasToken = true + timeOffset := new(hexutil.Uint64) + *timeOffset = 100 + dp.DeployConfig.SoulGasTokenTimeOffset = timeOffset + sd := e2eutils.Setup(t, dp, helpers.DefaultAlloc) + log := testlog.Logger(t, log.LevelDebug) + miner, engine, sequencer := helpers.SetupSequencerTest(t, sd, log) + + cl := engine.EthClient() + depositSGT(gt, engine, sd, dp, dp.Addresses.Alice, e2eutils.Ether(2)) + + sequencer.ActL2PipelineFull(t) + + genesisTime := sequencer.L2Unsafe().Time + + // Make L2 block + sequencer.ActL2StartBlock(t) + engine.ActL2IncludeTx(dp.Addresses.Alice)(t) + sequencer.ActL2EndBlock(t) + + // Alice makes a L2 tx + + balance1, err := cl.BalanceAt(t.Ctx(), dp.Addresses.Alice, nil) + require.NoError(t, err) + n, err := cl.PendingNonceAt(t.Ctx(), dp.Addresses.Alice) + require.NoError(t, err) + signer := types.LatestSigner(sd.L2Cfg.Config) + tx := types.MustSignNewTx(dp.Secrets.Alice, signer, &types.DynamicFeeTx{ + ChainID: sd.L2Cfg.Config.ChainID, + Nonce: n, + GasTipCap: big.NewInt(2 * params.GWei), + GasFeeCap: new(big.Int).Add(miner.L1Chain().CurrentBlock().BaseFee, big.NewInt(2*params.GWei)), + Gas: params.TxGas, + To: &dp.Addresses.Bob, + }) + require.NoError(t, cl.SendTransaction(t.Ctx(), tx)) + + // Make L2 block + sequencer.ActL2StartBlock(t) + engine.ActL2IncludeTx(dp.Addresses.Alice)(t) + sequencer.ActL2EndBlock(t) + + balance2, err := cl.BalanceAt(t.Ctx(), dp.Addresses.Alice, nil) + require.NoError(t, err) + // Check that the balance is different + // because the SGT is not active yet + require.True(t, balance2.Cmp(balance1) < 0) + + // advance to the block where the SGT is active + sequencer.ActBuildL2ToTime(t, genesisTime+(uint64)(*dp.DeployConfig.SoulGasTokenTimeOffset)) + + // Alice makes a L2 tx + n, err = cl.PendingNonceAt(t.Ctx(), dp.Addresses.Alice) + require.NoError(t, err) + tx = types.MustSignNewTx(dp.Secrets.Alice, signer, &types.DynamicFeeTx{ + ChainID: sd.L2Cfg.Config.ChainID, + Nonce: n, + GasTipCap: big.NewInt(2 * params.GWei), + GasFeeCap: new(big.Int).Add(miner.L1Chain().CurrentBlock().BaseFee, big.NewInt(2*params.GWei)), + Gas: params.TxGas, + To: &dp.Addresses.Bob, + }) + require.NoError(t, cl.SendTransaction(t.Ctx(), tx)) + + // Make L2 block + sequencer.ActL2StartBlock(t) + engine.ActL2IncludeTx(dp.Addresses.Alice)(t) + sequencer.ActL2EndBlock(t) + + balance3, err := cl.BalanceAt(t.Ctx(), dp.Addresses.Alice, nil) + require.NoError(t, err) + + // Check that the balance is the same as before + // because the SGT is active + require.True(t, balance3.Cmp(balance2) == 0) +} + +func depositSGT(t *testing.T, engine *helpers.L2Engine, sd *e2eutils.SetupData, dp *e2eutils.DeployParams, target common.Address, depositSgtValue *big.Int) { + + sgtAddr := predeploys.SoulGasTokenAddr + sgtContract, err := bindings.NewSoulGasToken(sgtAddr, engine.EthClient()) + require.NoError(t, err) + + txOpts, err := bind.NewKeyedTransactorWithChainID(dp.Secrets.Alice, sd.L2Cfg.Config.ChainID) + require.NoError(t, err) + txOpts.Value = depositSgtValue + + _, err = sgtContract.BatchDepositForAll(txOpts, []common.Address{target}, depositSgtValue) + require.NoError(t, err) + +}
diff --git official optimism/op-e2e/bindings/soulgastoken.go SWC optimism/op-e2e/bindings/soulgastoken.go new file mode 100644 index 0000000000000000000000000000000000000000..4930a2bba88c0a3d625b47ba0069aa0764f49194 --- /dev/null +++ SWC optimism/op-e2e/bindings/soulgastoken.go @@ -0,0 +1,1795 @@ +// Code generated - DO NOT EDIT. +// This file is a generated binding and any manual changes will be lost. + +package bindings + +import ( + "errors" + "math/big" + "strings" + + ethereum "github.com/ethereum/go-ethereum" + "github.com/ethereum/go-ethereum/accounts/abi" + "github.com/ethereum/go-ethereum/accounts/abi/bind" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/event" +) + +// Reference imports to suppress errors if they are not otherwise used. +var ( + _ = errors.New + _ = big.NewInt + _ = strings.NewReader + _ = ethereum.NotFound + _ = bind.Bind + _ = common.Big1 + _ = types.BloomLookup + _ = event.NewSubscription + _ = abi.ConvertType +) + +// SoulGasTokenMetaData contains all meta data concerning the SoulGasToken contract. +var SoulGasTokenMetaData = &bind.MetaData{ + ABI: "[{\"type\":\"constructor\",\"inputs\":[{\"name\":\"_isBackedByNative\",\"type\":\"bool\",\"internalType\":\"bool\"}],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"addBurners\",\"inputs\":[{\"name\":\"_burners\",\"type\":\"address[]\",\"internalType\":\"address[]\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"addMinters\",\"inputs\":[{\"name\":\"_minters\",\"type\":\"address[]\",\"internalType\":\"address[]\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"allowSgtValue\",\"inputs\":[{\"name\":\"_contracts\",\"type\":\"address[]\",\"internalType\":\"address[]\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"allowance\",\"inputs\":[{\"name\":\"owner\",\"type\":\"address\",\"internalType\":\"address\"},{\"name\":\"spender\",\"type\":\"address\",\"internalType\":\"address\"}],\"outputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"approve\",\"inputs\":[{\"name\":\"\",\"type\":\"address\",\"internalType\":\"address\"},{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[{\"name\":\"\",\"type\":\"bool\",\"internalType\":\"bool\"}],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"balanceOf\",\"inputs\":[{\"name\":\"account\",\"type\":\"address\",\"internalType\":\"address\"}],\"outputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"batchBurnFrom\",\"inputs\":[{\"name\":\"_accounts\",\"type\":\"address[]\",\"internalType\":\"address[]\"},{\"name\":\"_values\",\"type\":\"uint256[]\",\"internalType\":\"uint256[]\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"batchDepositFor\",\"inputs\":[{\"name\":\"_accounts\",\"type\":\"address[]\",\"internalType\":\"address[]\"},{\"name\":\"_values\",\"type\":\"uint256[]\",\"internalType\":\"uint256[]\"}],\"outputs\":[],\"stateMutability\":\"payable\"},{\"type\":\"function\",\"name\":\"batchDepositForAll\",\"inputs\":[{\"name\":\"_accounts\",\"type\":\"address[]\",\"internalType\":\"address[]\"},{\"name\":\"_value\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[],\"stateMutability\":\"payable\"},{\"type\":\"function\",\"name\":\"batchMint\",\"inputs\":[{\"name\":\"_accounts\",\"type\":\"address[]\",\"internalType\":\"address[]\"},{\"name\":\"_values\",\"type\":\"uint256[]\",\"internalType\":\"uint256[]\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"batchWithdrawFrom\",\"inputs\":[{\"name\":\"_accounts\",\"type\":\"address[]\",\"internalType\":\"address[]\"},{\"name\":\"_values\",\"type\":\"uint256[]\",\"internalType\":\"uint256[]\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"burnFrom\",\"inputs\":[{\"name\":\"_account\",\"type\":\"address\",\"internalType\":\"address\"},{\"name\":\"_value\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"chargeFromOrigin\",\"inputs\":[{\"name\":\"_amount\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[{\"name\":\"amountCharged_\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"decimals\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint8\",\"internalType\":\"uint8\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"decreaseAllowance\",\"inputs\":[{\"name\":\"spender\",\"type\":\"address\",\"internalType\":\"address\"},{\"name\":\"subtractedValue\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[{\"name\":\"\",\"type\":\"bool\",\"internalType\":\"bool\"}],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"delBurners\",\"inputs\":[{\"name\":\"_burners\",\"type\":\"address[]\",\"internalType\":\"address[]\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"delMinters\",\"inputs\":[{\"name\":\"_minters\",\"type\":\"address[]\",\"internalType\":\"address[]\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"deposit\",\"inputs\":[],\"outputs\":[],\"stateMutability\":\"payable\"},{\"type\":\"function\",\"name\":\"disallowSgtValue\",\"inputs\":[{\"name\":\"_contracts\",\"type\":\"address[]\",\"internalType\":\"address[]\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"increaseAllowance\",\"inputs\":[{\"name\":\"spender\",\"type\":\"address\",\"internalType\":\"address\"},{\"name\":\"addedValue\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[{\"name\":\"\",\"type\":\"bool\",\"internalType\":\"bool\"}],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"initialize\",\"inputs\":[{\"name\":\"_name\",\"type\":\"string\",\"internalType\":\"string\"},{\"name\":\"_symbol\",\"type\":\"string\",\"internalType\":\"string\"},{\"name\":\"_owner\",\"type\":\"address\",\"internalType\":\"address\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"isBackedByNative\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"bool\",\"internalType\":\"bool\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"name\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"string\",\"internalType\":\"string\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"owner\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"address\",\"internalType\":\"address\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"renounceOwnership\",\"inputs\":[],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"symbol\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"string\",\"internalType\":\"string\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"totalSupply\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"transfer\",\"inputs\":[{\"name\":\"\",\"type\":\"address\",\"internalType\":\"address\"},{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[{\"name\":\"\",\"type\":\"bool\",\"internalType\":\"bool\"}],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"transferFrom\",\"inputs\":[{\"name\":\"\",\"type\":\"address\",\"internalType\":\"address\"},{\"name\":\"\",\"type\":\"address\",\"internalType\":\"address\"},{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[{\"name\":\"\",\"type\":\"bool\",\"internalType\":\"bool\"}],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"transferOwnership\",\"inputs\":[{\"name\":\"newOwner\",\"type\":\"address\",\"internalType\":\"address\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"withdrawFrom\",\"inputs\":[{\"name\":\"_account\",\"type\":\"address\",\"internalType\":\"address\"},{\"name\":\"_value\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"event\",\"name\":\"AllowSgtValue\",\"inputs\":[{\"name\":\"from\",\"type\":\"address\",\"indexed\":true,\"internalType\":\"address\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"Approval\",\"inputs\":[{\"name\":\"owner\",\"type\":\"address\",\"indexed\":true,\"internalType\":\"address\"},{\"name\":\"spender\",\"type\":\"address\",\"indexed\":true,\"internalType\":\"address\"},{\"name\":\"value\",\"type\":\"uint256\",\"indexed\":false,\"internalType\":\"uint256\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"DisallowSgtValue\",\"inputs\":[{\"name\":\"from\",\"type\":\"address\",\"indexed\":true,\"internalType\":\"address\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"Initialized\",\"inputs\":[{\"name\":\"version\",\"type\":\"uint8\",\"indexed\":false,\"internalType\":\"uint8\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"OwnershipTransferred\",\"inputs\":[{\"name\":\"previousOwner\",\"type\":\"address\",\"indexed\":true,\"internalType\":\"address\"},{\"name\":\"newOwner\",\"type\":\"address\",\"indexed\":true,\"internalType\":\"address\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"Transfer\",\"inputs\":[{\"name\":\"from\",\"type\":\"address\",\"indexed\":true,\"internalType\":\"address\"},{\"name\":\"to\",\"type\":\"address\",\"indexed\":true,\"internalType\":\"address\"},{\"name\":\"value\",\"type\":\"uint256\",\"indexed\":false,\"internalType\":\"uint256\"}],\"anonymous\":false}]", +} + +// SoulGasTokenABI is the input ABI used to generate the binding from. +// Deprecated: Use SoulGasTokenMetaData.ABI instead. +var SoulGasTokenABI = SoulGasTokenMetaData.ABI + +// SoulGasToken is an auto generated Go binding around an Ethereum contract. +type SoulGasToken struct { + SoulGasTokenCaller // Read-only binding to the contract + SoulGasTokenTransactor // Write-only binding to the contract + SoulGasTokenFilterer // Log filterer for contract events +} + +// SoulGasTokenCaller is an auto generated read-only Go binding around an Ethereum contract. +type SoulGasTokenCaller struct { + contract *bind.BoundContract // Generic contract wrapper for the low level calls +} + +// SoulGasTokenTransactor is an auto generated write-only Go binding around an Ethereum contract. +type SoulGasTokenTransactor struct { + contract *bind.BoundContract // Generic contract wrapper for the low level calls +} + +// SoulGasTokenFilterer is an auto generated log filtering Go binding around an Ethereum contract events. +type SoulGasTokenFilterer struct { + contract *bind.BoundContract // Generic contract wrapper for the low level calls +} + +// SoulGasTokenSession is an auto generated Go binding around an Ethereum contract, +// with pre-set call and transact options. +type SoulGasTokenSession struct { + Contract *SoulGasToken // Generic contract binding to set the session for + CallOpts bind.CallOpts // Call options to use throughout this session + TransactOpts bind.TransactOpts // Transaction auth options to use throughout this session +} + +// SoulGasTokenCallerSession is an auto generated read-only Go binding around an Ethereum contract, +// with pre-set call options. +type SoulGasTokenCallerSession struct { + Contract *SoulGasTokenCaller // Generic contract caller binding to set the session for + CallOpts bind.CallOpts // Call options to use throughout this session +} + +// SoulGasTokenTransactorSession is an auto generated write-only Go binding around an Ethereum contract, +// with pre-set transact options. +type SoulGasTokenTransactorSession struct { + Contract *SoulGasTokenTransactor // Generic contract transactor binding to set the session for + TransactOpts bind.TransactOpts // Transaction auth options to use throughout this session +} + +// SoulGasTokenRaw is an auto generated low-level Go binding around an Ethereum contract. +type SoulGasTokenRaw struct { + Contract *SoulGasToken // Generic contract binding to access the raw methods on +} + +// SoulGasTokenCallerRaw is an auto generated low-level read-only Go binding around an Ethereum contract. +type SoulGasTokenCallerRaw struct { + Contract *SoulGasTokenCaller // Generic read-only contract binding to access the raw methods on +} + +// SoulGasTokenTransactorRaw is an auto generated low-level write-only Go binding around an Ethereum contract. +type SoulGasTokenTransactorRaw struct { + Contract *SoulGasTokenTransactor // Generic write-only contract binding to access the raw methods on +} + +// NewSoulGasToken creates a new instance of SoulGasToken, bound to a specific deployed contract. +func NewSoulGasToken(address common.Address, backend bind.ContractBackend) (*SoulGasToken, error) { + contract, err := bindSoulGasToken(address, backend, backend, backend) + if err != nil { + return nil, err + } + return &SoulGasToken{SoulGasTokenCaller: SoulGasTokenCaller{contract: contract}, SoulGasTokenTransactor: SoulGasTokenTransactor{contract: contract}, SoulGasTokenFilterer: SoulGasTokenFilterer{contract: contract}}, nil +} + +// NewSoulGasTokenCaller creates a new read-only instance of SoulGasToken, bound to a specific deployed contract. +func NewSoulGasTokenCaller(address common.Address, caller bind.ContractCaller) (*SoulGasTokenCaller, error) { + contract, err := bindSoulGasToken(address, caller, nil, nil) + if err != nil { + return nil, err + } + return &SoulGasTokenCaller{contract: contract}, nil +} + +// NewSoulGasTokenTransactor creates a new write-only instance of SoulGasToken, bound to a specific deployed contract. +func NewSoulGasTokenTransactor(address common.Address, transactor bind.ContractTransactor) (*SoulGasTokenTransactor, error) { + contract, err := bindSoulGasToken(address, nil, transactor, nil) + if err != nil { + return nil, err + } + return &SoulGasTokenTransactor{contract: contract}, nil +} + +// NewSoulGasTokenFilterer creates a new log filterer instance of SoulGasToken, bound to a specific deployed contract. +func NewSoulGasTokenFilterer(address common.Address, filterer bind.ContractFilterer) (*SoulGasTokenFilterer, error) { + contract, err := bindSoulGasToken(address, nil, nil, filterer) + if err != nil { + return nil, err + } + return &SoulGasTokenFilterer{contract: contract}, nil +} + +// bindSoulGasToken binds a generic wrapper to an already deployed contract. +func bindSoulGasToken(address common.Address, caller bind.ContractCaller, transactor bind.ContractTransactor, filterer bind.ContractFilterer) (*bind.BoundContract, error) { + parsed, err := SoulGasTokenMetaData.GetAbi() + if err != nil { + return nil, err + } + return bind.NewBoundContract(address, *parsed, caller, transactor, filterer), nil +} + +// Call invokes the (constant) contract method with params as input values and +// sets the output to result. The result type might be a single field for simple +// returns, a slice of interfaces for anonymous returns and a struct for named +// returns. +func (_SoulGasToken *SoulGasTokenRaw) Call(opts *bind.CallOpts, result *[]interface{}, method string, params ...interface{}) error { + return _SoulGasToken.Contract.SoulGasTokenCaller.contract.Call(opts, result, method, params...) +} + +// Transfer initiates a plain transaction to move funds to the contract, calling +// its default method if one is available. +func (_SoulGasToken *SoulGasTokenRaw) Transfer(opts *bind.TransactOpts) (*types.Transaction, error) { + return _SoulGasToken.Contract.SoulGasTokenTransactor.contract.Transfer(opts) +} + +// Transact invokes the (paid) contract method with params as input values. +func (_SoulGasToken *SoulGasTokenRaw) Transact(opts *bind.TransactOpts, method string, params ...interface{}) (*types.Transaction, error) { + return _SoulGasToken.Contract.SoulGasTokenTransactor.contract.Transact(opts, method, params...) +} + +// Call invokes the (constant) contract method with params as input values and +// sets the output to result. The result type might be a single field for simple +// returns, a slice of interfaces for anonymous returns and a struct for named +// returns. +func (_SoulGasToken *SoulGasTokenCallerRaw) Call(opts *bind.CallOpts, result *[]interface{}, method string, params ...interface{}) error { + return _SoulGasToken.Contract.contract.Call(opts, result, method, params...) +} + +// Transfer initiates a plain transaction to move funds to the contract, calling +// its default method if one is available. +func (_SoulGasToken *SoulGasTokenTransactorRaw) Transfer(opts *bind.TransactOpts) (*types.Transaction, error) { + return _SoulGasToken.Contract.contract.Transfer(opts) +} + +// Transact invokes the (paid) contract method with params as input values. +func (_SoulGasToken *SoulGasTokenTransactorRaw) Transact(opts *bind.TransactOpts, method string, params ...interface{}) (*types.Transaction, error) { + return _SoulGasToken.Contract.contract.Transact(opts, method, params...) +} + +// Allowance is a free data retrieval call binding the contract method 0xdd62ed3e. +// +// Solidity: function allowance(address owner, address spender) view returns(uint256) +func (_SoulGasToken *SoulGasTokenCaller) Allowance(opts *bind.CallOpts, owner common.Address, spender common.Address) (*big.Int, error) { + var out []interface{} + err := _SoulGasToken.contract.Call(opts, &out, "allowance", owner, spender) + + if err != nil { + return *new(*big.Int), err + } + + out0 := *abi.ConvertType(out[0], new(*big.Int)).(**big.Int) + + return out0, err + +} + +// Allowance is a free data retrieval call binding the contract method 0xdd62ed3e. +// +// Solidity: function allowance(address owner, address spender) view returns(uint256) +func (_SoulGasToken *SoulGasTokenSession) Allowance(owner common.Address, spender common.Address) (*big.Int, error) { + return _SoulGasToken.Contract.Allowance(&_SoulGasToken.CallOpts, owner, spender) +} + +// Allowance is a free data retrieval call binding the contract method 0xdd62ed3e. +// +// Solidity: function allowance(address owner, address spender) view returns(uint256) +func (_SoulGasToken *SoulGasTokenCallerSession) Allowance(owner common.Address, spender common.Address) (*big.Int, error) { + return _SoulGasToken.Contract.Allowance(&_SoulGasToken.CallOpts, owner, spender) +} + +// BalanceOf is a free data retrieval call binding the contract method 0x70a08231. +// +// Solidity: function balanceOf(address account) view returns(uint256) +func (_SoulGasToken *SoulGasTokenCaller) BalanceOf(opts *bind.CallOpts, account common.Address) (*big.Int, error) { + var out []interface{} + err := _SoulGasToken.contract.Call(opts, &out, "balanceOf", account) + + if err != nil { + return *new(*big.Int), err + } + + out0 := *abi.ConvertType(out[0], new(*big.Int)).(**big.Int) + + return out0, err + +} + +// BalanceOf is a free data retrieval call binding the contract method 0x70a08231. +// +// Solidity: function balanceOf(address account) view returns(uint256) +func (_SoulGasToken *SoulGasTokenSession) BalanceOf(account common.Address) (*big.Int, error) { + return _SoulGasToken.Contract.BalanceOf(&_SoulGasToken.CallOpts, account) +} + +// BalanceOf is a free data retrieval call binding the contract method 0x70a08231. +// +// Solidity: function balanceOf(address account) view returns(uint256) +func (_SoulGasToken *SoulGasTokenCallerSession) BalanceOf(account common.Address) (*big.Int, error) { + return _SoulGasToken.Contract.BalanceOf(&_SoulGasToken.CallOpts, account) +} + +// Decimals is a free data retrieval call binding the contract method 0x313ce567. +// +// Solidity: function decimals() view returns(uint8) +func (_SoulGasToken *SoulGasTokenCaller) Decimals(opts *bind.CallOpts) (uint8, error) { + var out []interface{} + err := _SoulGasToken.contract.Call(opts, &out, "decimals") + + if err != nil { + return *new(uint8), err + } + + out0 := *abi.ConvertType(out[0], new(uint8)).(*uint8) + + return out0, err + +} + +// Decimals is a free data retrieval call binding the contract method 0x313ce567. +// +// Solidity: function decimals() view returns(uint8) +func (_SoulGasToken *SoulGasTokenSession) Decimals() (uint8, error) { + return _SoulGasToken.Contract.Decimals(&_SoulGasToken.CallOpts) +} + +// Decimals is a free data retrieval call binding the contract method 0x313ce567. +// +// Solidity: function decimals() view returns(uint8) +func (_SoulGasToken *SoulGasTokenCallerSession) Decimals() (uint8, error) { + return _SoulGasToken.Contract.Decimals(&_SoulGasToken.CallOpts) +} + +// IsBackedByNative is a free data retrieval call binding the contract method 0xbbd10120. +// +// Solidity: function isBackedByNative() view returns(bool) +func (_SoulGasToken *SoulGasTokenCaller) IsBackedByNative(opts *bind.CallOpts) (bool, error) { + var out []interface{} + err := _SoulGasToken.contract.Call(opts, &out, "isBackedByNative") + + if err != nil { + return *new(bool), err + } + + out0 := *abi.ConvertType(out[0], new(bool)).(*bool) + + return out0, err + +} + +// IsBackedByNative is a free data retrieval call binding the contract method 0xbbd10120. +// +// Solidity: function isBackedByNative() view returns(bool) +func (_SoulGasToken *SoulGasTokenSession) IsBackedByNative() (bool, error) { + return _SoulGasToken.Contract.IsBackedByNative(&_SoulGasToken.CallOpts) +} + +// IsBackedByNative is a free data retrieval call binding the contract method 0xbbd10120. +// +// Solidity: function isBackedByNative() view returns(bool) +func (_SoulGasToken *SoulGasTokenCallerSession) IsBackedByNative() (bool, error) { + return _SoulGasToken.Contract.IsBackedByNative(&_SoulGasToken.CallOpts) +} + +// Name is a free data retrieval call binding the contract method 0x06fdde03. +// +// Solidity: function name() view returns(string) +func (_SoulGasToken *SoulGasTokenCaller) Name(opts *bind.CallOpts) (string, error) { + var out []interface{} + err := _SoulGasToken.contract.Call(opts, &out, "name") + + if err != nil { + return *new(string), err + } + + out0 := *abi.ConvertType(out[0], new(string)).(*string) + + return out0, err + +} + +// Name is a free data retrieval call binding the contract method 0x06fdde03. +// +// Solidity: function name() view returns(string) +func (_SoulGasToken *SoulGasTokenSession) Name() (string, error) { + return _SoulGasToken.Contract.Name(&_SoulGasToken.CallOpts) +} + +// Name is a free data retrieval call binding the contract method 0x06fdde03. +// +// Solidity: function name() view returns(string) +func (_SoulGasToken *SoulGasTokenCallerSession) Name() (string, error) { + return _SoulGasToken.Contract.Name(&_SoulGasToken.CallOpts) +} + +// Owner is a free data retrieval call binding the contract method 0x8da5cb5b. +// +// Solidity: function owner() view returns(address) +func (_SoulGasToken *SoulGasTokenCaller) Owner(opts *bind.CallOpts) (common.Address, error) { + var out []interface{} + err := _SoulGasToken.contract.Call(opts, &out, "owner") + + if err != nil { + return *new(common.Address), err + } + + out0 := *abi.ConvertType(out[0], new(common.Address)).(*common.Address) + + return out0, err + +} + +// Owner is a free data retrieval call binding the contract method 0x8da5cb5b. +// +// Solidity: function owner() view returns(address) +func (_SoulGasToken *SoulGasTokenSession) Owner() (common.Address, error) { + return _SoulGasToken.Contract.Owner(&_SoulGasToken.CallOpts) +} + +// Owner is a free data retrieval call binding the contract method 0x8da5cb5b. +// +// Solidity: function owner() view returns(address) +func (_SoulGasToken *SoulGasTokenCallerSession) Owner() (common.Address, error) { + return _SoulGasToken.Contract.Owner(&_SoulGasToken.CallOpts) +} + +// Symbol is a free data retrieval call binding the contract method 0x95d89b41. +// +// Solidity: function symbol() view returns(string) +func (_SoulGasToken *SoulGasTokenCaller) Symbol(opts *bind.CallOpts) (string, error) { + var out []interface{} + err := _SoulGasToken.contract.Call(opts, &out, "symbol") + + if err != nil { + return *new(string), err + } + + out0 := *abi.ConvertType(out[0], new(string)).(*string) + + return out0, err + +} + +// Symbol is a free data retrieval call binding the contract method 0x95d89b41. +// +// Solidity: function symbol() view returns(string) +func (_SoulGasToken *SoulGasTokenSession) Symbol() (string, error) { + return _SoulGasToken.Contract.Symbol(&_SoulGasToken.CallOpts) +} + +// Symbol is a free data retrieval call binding the contract method 0x95d89b41. +// +// Solidity: function symbol() view returns(string) +func (_SoulGasToken *SoulGasTokenCallerSession) Symbol() (string, error) { + return _SoulGasToken.Contract.Symbol(&_SoulGasToken.CallOpts) +} + +// TotalSupply is a free data retrieval call binding the contract method 0x18160ddd. +// +// Solidity: function totalSupply() view returns(uint256) +func (_SoulGasToken *SoulGasTokenCaller) TotalSupply(opts *bind.CallOpts) (*big.Int, error) { + var out []interface{} + err := _SoulGasToken.contract.Call(opts, &out, "totalSupply") + + if err != nil { + return *new(*big.Int), err + } + + out0 := *abi.ConvertType(out[0], new(*big.Int)).(**big.Int) + + return out0, err + +} + +// TotalSupply is a free data retrieval call binding the contract method 0x18160ddd. +// +// Solidity: function totalSupply() view returns(uint256) +func (_SoulGasToken *SoulGasTokenSession) TotalSupply() (*big.Int, error) { + return _SoulGasToken.Contract.TotalSupply(&_SoulGasToken.CallOpts) +} + +// TotalSupply is a free data retrieval call binding the contract method 0x18160ddd. +// +// Solidity: function totalSupply() view returns(uint256) +func (_SoulGasToken *SoulGasTokenCallerSession) TotalSupply() (*big.Int, error) { + return _SoulGasToken.Contract.TotalSupply(&_SoulGasToken.CallOpts) +} + +// AddBurners is a paid mutator transaction binding the contract method 0x3ab84dd9. +// +// Solidity: function addBurners(address[] _burners) returns() +func (_SoulGasToken *SoulGasTokenTransactor) AddBurners(opts *bind.TransactOpts, _burners []common.Address) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "addBurners", _burners) +} + +// AddBurners is a paid mutator transaction binding the contract method 0x3ab84dd9. +// +// Solidity: function addBurners(address[] _burners) returns() +func (_SoulGasToken *SoulGasTokenSession) AddBurners(_burners []common.Address) (*types.Transaction, error) { + return _SoulGasToken.Contract.AddBurners(&_SoulGasToken.TransactOpts, _burners) +} + +// AddBurners is a paid mutator transaction binding the contract method 0x3ab84dd9. +// +// Solidity: function addBurners(address[] _burners) returns() +func (_SoulGasToken *SoulGasTokenTransactorSession) AddBurners(_burners []common.Address) (*types.Transaction, error) { + return _SoulGasToken.Contract.AddBurners(&_SoulGasToken.TransactOpts, _burners) +} + +// AddMinters is a paid mutator transaction binding the contract method 0x71e2a657. +// +// Solidity: function addMinters(address[] _minters) returns() +func (_SoulGasToken *SoulGasTokenTransactor) AddMinters(opts *bind.TransactOpts, _minters []common.Address) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "addMinters", _minters) +} + +// AddMinters is a paid mutator transaction binding the contract method 0x71e2a657. +// +// Solidity: function addMinters(address[] _minters) returns() +func (_SoulGasToken *SoulGasTokenSession) AddMinters(_minters []common.Address) (*types.Transaction, error) { + return _SoulGasToken.Contract.AddMinters(&_SoulGasToken.TransactOpts, _minters) +} + +// AddMinters is a paid mutator transaction binding the contract method 0x71e2a657. +// +// Solidity: function addMinters(address[] _minters) returns() +func (_SoulGasToken *SoulGasTokenTransactorSession) AddMinters(_minters []common.Address) (*types.Transaction, error) { + return _SoulGasToken.Contract.AddMinters(&_SoulGasToken.TransactOpts, _minters) +} + +// AllowSgtValue is a paid mutator transaction binding the contract method 0x674e29ea. +// +// Solidity: function allowSgtValue(address[] _contracts) returns() +func (_SoulGasToken *SoulGasTokenTransactor) AllowSgtValue(opts *bind.TransactOpts, _contracts []common.Address) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "allowSgtValue", _contracts) +} + +// AllowSgtValue is a paid mutator transaction binding the contract method 0x674e29ea. +// +// Solidity: function allowSgtValue(address[] _contracts) returns() +func (_SoulGasToken *SoulGasTokenSession) AllowSgtValue(_contracts []common.Address) (*types.Transaction, error) { + return _SoulGasToken.Contract.AllowSgtValue(&_SoulGasToken.TransactOpts, _contracts) +} + +// AllowSgtValue is a paid mutator transaction binding the contract method 0x674e29ea. +// +// Solidity: function allowSgtValue(address[] _contracts) returns() +func (_SoulGasToken *SoulGasTokenTransactorSession) AllowSgtValue(_contracts []common.Address) (*types.Transaction, error) { + return _SoulGasToken.Contract.AllowSgtValue(&_SoulGasToken.TransactOpts, _contracts) +} + +// Approve is a paid mutator transaction binding the contract method 0x095ea7b3. +// +// Solidity: function approve(address , uint256 ) returns(bool) +func (_SoulGasToken *SoulGasTokenTransactor) Approve(opts *bind.TransactOpts, arg0 common.Address, arg1 *big.Int) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "approve", arg0, arg1) +} + +// Approve is a paid mutator transaction binding the contract method 0x095ea7b3. +// +// Solidity: function approve(address , uint256 ) returns(bool) +func (_SoulGasToken *SoulGasTokenSession) Approve(arg0 common.Address, arg1 *big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.Approve(&_SoulGasToken.TransactOpts, arg0, arg1) +} + +// Approve is a paid mutator transaction binding the contract method 0x095ea7b3. +// +// Solidity: function approve(address , uint256 ) returns(bool) +func (_SoulGasToken *SoulGasTokenTransactorSession) Approve(arg0 common.Address, arg1 *big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.Approve(&_SoulGasToken.TransactOpts, arg0, arg1) +} + +// BatchBurnFrom is a paid mutator transaction binding the contract method 0x1b9a7529. +// +// Solidity: function batchBurnFrom(address[] _accounts, uint256[] _values) returns() +func (_SoulGasToken *SoulGasTokenTransactor) BatchBurnFrom(opts *bind.TransactOpts, _accounts []common.Address, _values []*big.Int) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "batchBurnFrom", _accounts, _values) +} + +// BatchBurnFrom is a paid mutator transaction binding the contract method 0x1b9a7529. +// +// Solidity: function batchBurnFrom(address[] _accounts, uint256[] _values) returns() +func (_SoulGasToken *SoulGasTokenSession) BatchBurnFrom(_accounts []common.Address, _values []*big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.BatchBurnFrom(&_SoulGasToken.TransactOpts, _accounts, _values) +} + +// BatchBurnFrom is a paid mutator transaction binding the contract method 0x1b9a7529. +// +// Solidity: function batchBurnFrom(address[] _accounts, uint256[] _values) returns() +func (_SoulGasToken *SoulGasTokenTransactorSession) BatchBurnFrom(_accounts []common.Address, _values []*big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.BatchBurnFrom(&_SoulGasToken.TransactOpts, _accounts, _values) +} + +// BatchDepositFor is a paid mutator transaction binding the contract method 0x299f8170. +// +// Solidity: function batchDepositFor(address[] _accounts, uint256[] _values) payable returns() +func (_SoulGasToken *SoulGasTokenTransactor) BatchDepositFor(opts *bind.TransactOpts, _accounts []common.Address, _values []*big.Int) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "batchDepositFor", _accounts, _values) +} + +// BatchDepositFor is a paid mutator transaction binding the contract method 0x299f8170. +// +// Solidity: function batchDepositFor(address[] _accounts, uint256[] _values) payable returns() +func (_SoulGasToken *SoulGasTokenSession) BatchDepositFor(_accounts []common.Address, _values []*big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.BatchDepositFor(&_SoulGasToken.TransactOpts, _accounts, _values) +} + +// BatchDepositFor is a paid mutator transaction binding the contract method 0x299f8170. +// +// Solidity: function batchDepositFor(address[] _accounts, uint256[] _values) payable returns() +func (_SoulGasToken *SoulGasTokenTransactorSession) BatchDepositFor(_accounts []common.Address, _values []*big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.BatchDepositFor(&_SoulGasToken.TransactOpts, _accounts, _values) +} + +// BatchDepositForAll is a paid mutator transaction binding the contract method 0x84e08810. +// +// Solidity: function batchDepositForAll(address[] _accounts, uint256 _value) payable returns() +func (_SoulGasToken *SoulGasTokenTransactor) BatchDepositForAll(opts *bind.TransactOpts, _accounts []common.Address, _value *big.Int) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "batchDepositForAll", _accounts, _value) +} + +// BatchDepositForAll is a paid mutator transaction binding the contract method 0x84e08810. +// +// Solidity: function batchDepositForAll(address[] _accounts, uint256 _value) payable returns() +func (_SoulGasToken *SoulGasTokenSession) BatchDepositForAll(_accounts []common.Address, _value *big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.BatchDepositForAll(&_SoulGasToken.TransactOpts, _accounts, _value) +} + +// BatchDepositForAll is a paid mutator transaction binding the contract method 0x84e08810. +// +// Solidity: function batchDepositForAll(address[] _accounts, uint256 _value) payable returns() +func (_SoulGasToken *SoulGasTokenTransactorSession) BatchDepositForAll(_accounts []common.Address, _value *big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.BatchDepositForAll(&_SoulGasToken.TransactOpts, _accounts, _value) +} + +// BatchMint is a paid mutator transaction binding the contract method 0x68573107. +// +// Solidity: function batchMint(address[] _accounts, uint256[] _values) returns() +func (_SoulGasToken *SoulGasTokenTransactor) BatchMint(opts *bind.TransactOpts, _accounts []common.Address, _values []*big.Int) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "batchMint", _accounts, _values) +} + +// BatchMint is a paid mutator transaction binding the contract method 0x68573107. +// +// Solidity: function batchMint(address[] _accounts, uint256[] _values) returns() +func (_SoulGasToken *SoulGasTokenSession) BatchMint(_accounts []common.Address, _values []*big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.BatchMint(&_SoulGasToken.TransactOpts, _accounts, _values) +} + +// BatchMint is a paid mutator transaction binding the contract method 0x68573107. +// +// Solidity: function batchMint(address[] _accounts, uint256[] _values) returns() +func (_SoulGasToken *SoulGasTokenTransactorSession) BatchMint(_accounts []common.Address, _values []*big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.BatchMint(&_SoulGasToken.TransactOpts, _accounts, _values) +} + +// BatchWithdrawFrom is a paid mutator transaction binding the contract method 0xb3e2a832. +// +// Solidity: function batchWithdrawFrom(address[] _accounts, uint256[] _values) returns() +func (_SoulGasToken *SoulGasTokenTransactor) BatchWithdrawFrom(opts *bind.TransactOpts, _accounts []common.Address, _values []*big.Int) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "batchWithdrawFrom", _accounts, _values) +} + +// BatchWithdrawFrom is a paid mutator transaction binding the contract method 0xb3e2a832. +// +// Solidity: function batchWithdrawFrom(address[] _accounts, uint256[] _values) returns() +func (_SoulGasToken *SoulGasTokenSession) BatchWithdrawFrom(_accounts []common.Address, _values []*big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.BatchWithdrawFrom(&_SoulGasToken.TransactOpts, _accounts, _values) +} + +// BatchWithdrawFrom is a paid mutator transaction binding the contract method 0xb3e2a832. +// +// Solidity: function batchWithdrawFrom(address[] _accounts, uint256[] _values) returns() +func (_SoulGasToken *SoulGasTokenTransactorSession) BatchWithdrawFrom(_accounts []common.Address, _values []*big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.BatchWithdrawFrom(&_SoulGasToken.TransactOpts, _accounts, _values) +} + +// BurnFrom is a paid mutator transaction binding the contract method 0x79cc6790. +// +// Solidity: function burnFrom(address _account, uint256 _value) returns() +func (_SoulGasToken *SoulGasTokenTransactor) BurnFrom(opts *bind.TransactOpts, _account common.Address, _value *big.Int) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "burnFrom", _account, _value) +} + +// BurnFrom is a paid mutator transaction binding the contract method 0x79cc6790. +// +// Solidity: function burnFrom(address _account, uint256 _value) returns() +func (_SoulGasToken *SoulGasTokenSession) BurnFrom(_account common.Address, _value *big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.BurnFrom(&_SoulGasToken.TransactOpts, _account, _value) +} + +// BurnFrom is a paid mutator transaction binding the contract method 0x79cc6790. +// +// Solidity: function burnFrom(address _account, uint256 _value) returns() +func (_SoulGasToken *SoulGasTokenTransactorSession) BurnFrom(_account common.Address, _value *big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.BurnFrom(&_SoulGasToken.TransactOpts, _account, _value) +} + +// ChargeFromOrigin is a paid mutator transaction binding the contract method 0xce25c861. +// +// Solidity: function chargeFromOrigin(uint256 _amount) returns(uint256 amountCharged_) +func (_SoulGasToken *SoulGasTokenTransactor) ChargeFromOrigin(opts *bind.TransactOpts, _amount *big.Int) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "chargeFromOrigin", _amount) +} + +// ChargeFromOrigin is a paid mutator transaction binding the contract method 0xce25c861. +// +// Solidity: function chargeFromOrigin(uint256 _amount) returns(uint256 amountCharged_) +func (_SoulGasToken *SoulGasTokenSession) ChargeFromOrigin(_amount *big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.ChargeFromOrigin(&_SoulGasToken.TransactOpts, _amount) +} + +// ChargeFromOrigin is a paid mutator transaction binding the contract method 0xce25c861. +// +// Solidity: function chargeFromOrigin(uint256 _amount) returns(uint256 amountCharged_) +func (_SoulGasToken *SoulGasTokenTransactorSession) ChargeFromOrigin(_amount *big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.ChargeFromOrigin(&_SoulGasToken.TransactOpts, _amount) +} + +// DecreaseAllowance is a paid mutator transaction binding the contract method 0xa457c2d7. +// +// Solidity: function decreaseAllowance(address spender, uint256 subtractedValue) returns(bool) +func (_SoulGasToken *SoulGasTokenTransactor) DecreaseAllowance(opts *bind.TransactOpts, spender common.Address, subtractedValue *big.Int) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "decreaseAllowance", spender, subtractedValue) +} + +// DecreaseAllowance is a paid mutator transaction binding the contract method 0xa457c2d7. +// +// Solidity: function decreaseAllowance(address spender, uint256 subtractedValue) returns(bool) +func (_SoulGasToken *SoulGasTokenSession) DecreaseAllowance(spender common.Address, subtractedValue *big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.DecreaseAllowance(&_SoulGasToken.TransactOpts, spender, subtractedValue) +} + +// DecreaseAllowance is a paid mutator transaction binding the contract method 0xa457c2d7. +// +// Solidity: function decreaseAllowance(address spender, uint256 subtractedValue) returns(bool) +func (_SoulGasToken *SoulGasTokenTransactorSession) DecreaseAllowance(spender common.Address, subtractedValue *big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.DecreaseAllowance(&_SoulGasToken.TransactOpts, spender, subtractedValue) +} + +// DelBurners is a paid mutator transaction binding the contract method 0xb8de86b4. +// +// Solidity: function delBurners(address[] _burners) returns() +func (_SoulGasToken *SoulGasTokenTransactor) DelBurners(opts *bind.TransactOpts, _burners []common.Address) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "delBurners", _burners) +} + +// DelBurners is a paid mutator transaction binding the contract method 0xb8de86b4. +// +// Solidity: function delBurners(address[] _burners) returns() +func (_SoulGasToken *SoulGasTokenSession) DelBurners(_burners []common.Address) (*types.Transaction, error) { + return _SoulGasToken.Contract.DelBurners(&_SoulGasToken.TransactOpts, _burners) +} + +// DelBurners is a paid mutator transaction binding the contract method 0xb8de86b4. +// +// Solidity: function delBurners(address[] _burners) returns() +func (_SoulGasToken *SoulGasTokenTransactorSession) DelBurners(_burners []common.Address) (*types.Transaction, error) { + return _SoulGasToken.Contract.DelBurners(&_SoulGasToken.TransactOpts, _burners) +} + +// DelMinters is a paid mutator transaction binding the contract method 0xe04b8180. +// +// Solidity: function delMinters(address[] _minters) returns() +func (_SoulGasToken *SoulGasTokenTransactor) DelMinters(opts *bind.TransactOpts, _minters []common.Address) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "delMinters", _minters) +} + +// DelMinters is a paid mutator transaction binding the contract method 0xe04b8180. +// +// Solidity: function delMinters(address[] _minters) returns() +func (_SoulGasToken *SoulGasTokenSession) DelMinters(_minters []common.Address) (*types.Transaction, error) { + return _SoulGasToken.Contract.DelMinters(&_SoulGasToken.TransactOpts, _minters) +} + +// DelMinters is a paid mutator transaction binding the contract method 0xe04b8180. +// +// Solidity: function delMinters(address[] _minters) returns() +func (_SoulGasToken *SoulGasTokenTransactorSession) DelMinters(_minters []common.Address) (*types.Transaction, error) { + return _SoulGasToken.Contract.DelMinters(&_SoulGasToken.TransactOpts, _minters) +} + +// Deposit is a paid mutator transaction binding the contract method 0xd0e30db0. +// +// Solidity: function deposit() payable returns() +func (_SoulGasToken *SoulGasTokenTransactor) Deposit(opts *bind.TransactOpts) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "deposit") +} + +// Deposit is a paid mutator transaction binding the contract method 0xd0e30db0. +// +// Solidity: function deposit() payable returns() +func (_SoulGasToken *SoulGasTokenSession) Deposit() (*types.Transaction, error) { + return _SoulGasToken.Contract.Deposit(&_SoulGasToken.TransactOpts) +} + +// Deposit is a paid mutator transaction binding the contract method 0xd0e30db0. +// +// Solidity: function deposit() payable returns() +func (_SoulGasToken *SoulGasTokenTransactorSession) Deposit() (*types.Transaction, error) { + return _SoulGasToken.Contract.Deposit(&_SoulGasToken.TransactOpts) +} + +// DisallowSgtValue is a paid mutator transaction binding the contract method 0xdc270eb8. +// +// Solidity: function disallowSgtValue(address[] _contracts) returns() +func (_SoulGasToken *SoulGasTokenTransactor) DisallowSgtValue(opts *bind.TransactOpts, _contracts []common.Address) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "disallowSgtValue", _contracts) +} + +// DisallowSgtValue is a paid mutator transaction binding the contract method 0xdc270eb8. +// +// Solidity: function disallowSgtValue(address[] _contracts) returns() +func (_SoulGasToken *SoulGasTokenSession) DisallowSgtValue(_contracts []common.Address) (*types.Transaction, error) { + return _SoulGasToken.Contract.DisallowSgtValue(&_SoulGasToken.TransactOpts, _contracts) +} + +// DisallowSgtValue is a paid mutator transaction binding the contract method 0xdc270eb8. +// +// Solidity: function disallowSgtValue(address[] _contracts) returns() +func (_SoulGasToken *SoulGasTokenTransactorSession) DisallowSgtValue(_contracts []common.Address) (*types.Transaction, error) { + return _SoulGasToken.Contract.DisallowSgtValue(&_SoulGasToken.TransactOpts, _contracts) +} + +// IncreaseAllowance is a paid mutator transaction binding the contract method 0x39509351. +// +// Solidity: function increaseAllowance(address spender, uint256 addedValue) returns(bool) +func (_SoulGasToken *SoulGasTokenTransactor) IncreaseAllowance(opts *bind.TransactOpts, spender common.Address, addedValue *big.Int) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "increaseAllowance", spender, addedValue) +} + +// IncreaseAllowance is a paid mutator transaction binding the contract method 0x39509351. +// +// Solidity: function increaseAllowance(address spender, uint256 addedValue) returns(bool) +func (_SoulGasToken *SoulGasTokenSession) IncreaseAllowance(spender common.Address, addedValue *big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.IncreaseAllowance(&_SoulGasToken.TransactOpts, spender, addedValue) +} + +// IncreaseAllowance is a paid mutator transaction binding the contract method 0x39509351. +// +// Solidity: function increaseAllowance(address spender, uint256 addedValue) returns(bool) +func (_SoulGasToken *SoulGasTokenTransactorSession) IncreaseAllowance(spender common.Address, addedValue *big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.IncreaseAllowance(&_SoulGasToken.TransactOpts, spender, addedValue) +} + +// Initialize is a paid mutator transaction binding the contract method 0x077f224a. +// +// Solidity: function initialize(string _name, string _symbol, address _owner) returns() +func (_SoulGasToken *SoulGasTokenTransactor) Initialize(opts *bind.TransactOpts, _name string, _symbol string, _owner common.Address) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "initialize", _name, _symbol, _owner) +} + +// Initialize is a paid mutator transaction binding the contract method 0x077f224a. +// +// Solidity: function initialize(string _name, string _symbol, address _owner) returns() +func (_SoulGasToken *SoulGasTokenSession) Initialize(_name string, _symbol string, _owner common.Address) (*types.Transaction, error) { + return _SoulGasToken.Contract.Initialize(&_SoulGasToken.TransactOpts, _name, _symbol, _owner) +} + +// Initialize is a paid mutator transaction binding the contract method 0x077f224a. +// +// Solidity: function initialize(string _name, string _symbol, address _owner) returns() +func (_SoulGasToken *SoulGasTokenTransactorSession) Initialize(_name string, _symbol string, _owner common.Address) (*types.Transaction, error) { + return _SoulGasToken.Contract.Initialize(&_SoulGasToken.TransactOpts, _name, _symbol, _owner) +} + +// RenounceOwnership is a paid mutator transaction binding the contract method 0x715018a6. +// +// Solidity: function renounceOwnership() returns() +func (_SoulGasToken *SoulGasTokenTransactor) RenounceOwnership(opts *bind.TransactOpts) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "renounceOwnership") +} + +// RenounceOwnership is a paid mutator transaction binding the contract method 0x715018a6. +// +// Solidity: function renounceOwnership() returns() +func (_SoulGasToken *SoulGasTokenSession) RenounceOwnership() (*types.Transaction, error) { + return _SoulGasToken.Contract.RenounceOwnership(&_SoulGasToken.TransactOpts) +} + +// RenounceOwnership is a paid mutator transaction binding the contract method 0x715018a6. +// +// Solidity: function renounceOwnership() returns() +func (_SoulGasToken *SoulGasTokenTransactorSession) RenounceOwnership() (*types.Transaction, error) { + return _SoulGasToken.Contract.RenounceOwnership(&_SoulGasToken.TransactOpts) +} + +// Transfer is a paid mutator transaction binding the contract method 0xa9059cbb. +// +// Solidity: function transfer(address , uint256 ) returns(bool) +func (_SoulGasToken *SoulGasTokenTransactor) Transfer(opts *bind.TransactOpts, arg0 common.Address, arg1 *big.Int) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "transfer", arg0, arg1) +} + +// Transfer is a paid mutator transaction binding the contract method 0xa9059cbb. +// +// Solidity: function transfer(address , uint256 ) returns(bool) +func (_SoulGasToken *SoulGasTokenSession) Transfer(arg0 common.Address, arg1 *big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.Transfer(&_SoulGasToken.TransactOpts, arg0, arg1) +} + +// Transfer is a paid mutator transaction binding the contract method 0xa9059cbb. +// +// Solidity: function transfer(address , uint256 ) returns(bool) +func (_SoulGasToken *SoulGasTokenTransactorSession) Transfer(arg0 common.Address, arg1 *big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.Transfer(&_SoulGasToken.TransactOpts, arg0, arg1) +} + +// TransferFrom is a paid mutator transaction binding the contract method 0x23b872dd. +// +// Solidity: function transferFrom(address , address , uint256 ) returns(bool) +func (_SoulGasToken *SoulGasTokenTransactor) TransferFrom(opts *bind.TransactOpts, arg0 common.Address, arg1 common.Address, arg2 *big.Int) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "transferFrom", arg0, arg1, arg2) +} + +// TransferFrom is a paid mutator transaction binding the contract method 0x23b872dd. +// +// Solidity: function transferFrom(address , address , uint256 ) returns(bool) +func (_SoulGasToken *SoulGasTokenSession) TransferFrom(arg0 common.Address, arg1 common.Address, arg2 *big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.TransferFrom(&_SoulGasToken.TransactOpts, arg0, arg1, arg2) +} + +// TransferFrom is a paid mutator transaction binding the contract method 0x23b872dd. +// +// Solidity: function transferFrom(address , address , uint256 ) returns(bool) +func (_SoulGasToken *SoulGasTokenTransactorSession) TransferFrom(arg0 common.Address, arg1 common.Address, arg2 *big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.TransferFrom(&_SoulGasToken.TransactOpts, arg0, arg1, arg2) +} + +// TransferOwnership is a paid mutator transaction binding the contract method 0xf2fde38b. +// +// Solidity: function transferOwnership(address newOwner) returns() +func (_SoulGasToken *SoulGasTokenTransactor) TransferOwnership(opts *bind.TransactOpts, newOwner common.Address) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "transferOwnership", newOwner) +} + +// TransferOwnership is a paid mutator transaction binding the contract method 0xf2fde38b. +// +// Solidity: function transferOwnership(address newOwner) returns() +func (_SoulGasToken *SoulGasTokenSession) TransferOwnership(newOwner common.Address) (*types.Transaction, error) { + return _SoulGasToken.Contract.TransferOwnership(&_SoulGasToken.TransactOpts, newOwner) +} + +// TransferOwnership is a paid mutator transaction binding the contract method 0xf2fde38b. +// +// Solidity: function transferOwnership(address newOwner) returns() +func (_SoulGasToken *SoulGasTokenTransactorSession) TransferOwnership(newOwner common.Address) (*types.Transaction, error) { + return _SoulGasToken.Contract.TransferOwnership(&_SoulGasToken.TransactOpts, newOwner) +} + +// WithdrawFrom is a paid mutator transaction binding the contract method 0x9470b0bd. +// +// Solidity: function withdrawFrom(address _account, uint256 _value) returns() +func (_SoulGasToken *SoulGasTokenTransactor) WithdrawFrom(opts *bind.TransactOpts, _account common.Address, _value *big.Int) (*types.Transaction, error) { + return _SoulGasToken.contract.Transact(opts, "withdrawFrom", _account, _value) +} + +// WithdrawFrom is a paid mutator transaction binding the contract method 0x9470b0bd. +// +// Solidity: function withdrawFrom(address _account, uint256 _value) returns() +func (_SoulGasToken *SoulGasTokenSession) WithdrawFrom(_account common.Address, _value *big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.WithdrawFrom(&_SoulGasToken.TransactOpts, _account, _value) +} + +// WithdrawFrom is a paid mutator transaction binding the contract method 0x9470b0bd. +// +// Solidity: function withdrawFrom(address _account, uint256 _value) returns() +func (_SoulGasToken *SoulGasTokenTransactorSession) WithdrawFrom(_account common.Address, _value *big.Int) (*types.Transaction, error) { + return _SoulGasToken.Contract.WithdrawFrom(&_SoulGasToken.TransactOpts, _account, _value) +} + +// SoulGasTokenAllowSgtValueIterator is returned from FilterAllowSgtValue and is used to iterate over the raw logs and unpacked data for AllowSgtValue events raised by the SoulGasToken contract. +type SoulGasTokenAllowSgtValueIterator struct { + Event *SoulGasTokenAllowSgtValue // Event containing the contract specifics and raw log + + contract *bind.BoundContract // Generic contract to use for unpacking event data + event string // Event name to use for unpacking event data + + logs chan types.Log // Log channel receiving the found contract events + sub ethereum.Subscription // Subscription for errors, completion and termination + done bool // Whether the subscription completed delivering logs + fail error // Occurred error to stop iteration +} + +// Next advances the iterator to the subsequent event, returning whether there +// are any more events found. In case of a retrieval or parsing error, false is +// returned and Error() can be queried for the exact failure. +func (it *SoulGasTokenAllowSgtValueIterator) Next() bool { + // If the iterator failed, stop iterating + if it.fail != nil { + return false + } + // If the iterator completed, deliver directly whatever's available + if it.done { + select { + case log := <-it.logs: + it.Event = new(SoulGasTokenAllowSgtValue) + if err := it.contract.UnpackLog(it.Event, it.event, log); err != nil { + it.fail = err + return false + } + it.Event.Raw = log + return true + + default: + return false + } + } + // Iterator still in progress, wait for either a data or an error event + select { + case log := <-it.logs: + it.Event = new(SoulGasTokenAllowSgtValue) + if err := it.contract.UnpackLog(it.Event, it.event, log); err != nil { + it.fail = err + return false + } + it.Event.Raw = log + return true + + case err := <-it.sub.Err(): + it.done = true + it.fail = err + return it.Next() + } +} + +// Error returns any retrieval or parsing error occurred during filtering. +func (it *SoulGasTokenAllowSgtValueIterator) Error() error { + return it.fail +} + +// Close terminates the iteration process, releasing any pending underlying +// resources. +func (it *SoulGasTokenAllowSgtValueIterator) Close() error { + it.sub.Unsubscribe() + return nil +} + +// SoulGasTokenAllowSgtValue represents a AllowSgtValue event raised by the SoulGasToken contract. +type SoulGasTokenAllowSgtValue struct { + From common.Address + Raw types.Log // Blockchain specific contextual infos +} + +// FilterAllowSgtValue is a free log retrieval operation binding the contract event 0xf135aca2ee4483470b8f44f38ab676fc36fc67437777f3c520e5fbeb3706009f. +// +// Solidity: event AllowSgtValue(address indexed from) +func (_SoulGasToken *SoulGasTokenFilterer) FilterAllowSgtValue(opts *bind.FilterOpts, from []common.Address) (*SoulGasTokenAllowSgtValueIterator, error) { + + var fromRule []interface{} + for _, fromItem := range from { + fromRule = append(fromRule, fromItem) + } + + logs, sub, err := _SoulGasToken.contract.FilterLogs(opts, "AllowSgtValue", fromRule) + if err != nil { + return nil, err + } + return &SoulGasTokenAllowSgtValueIterator{contract: _SoulGasToken.contract, event: "AllowSgtValue", logs: logs, sub: sub}, nil +} + +// WatchAllowSgtValue is a free log subscription operation binding the contract event 0xf135aca2ee4483470b8f44f38ab676fc36fc67437777f3c520e5fbeb3706009f. +// +// Solidity: event AllowSgtValue(address indexed from) +func (_SoulGasToken *SoulGasTokenFilterer) WatchAllowSgtValue(opts *bind.WatchOpts, sink chan<- *SoulGasTokenAllowSgtValue, from []common.Address) (event.Subscription, error) { + + var fromRule []interface{} + for _, fromItem := range from { + fromRule = append(fromRule, fromItem) + } + + logs, sub, err := _SoulGasToken.contract.WatchLogs(opts, "AllowSgtValue", fromRule) + if err != nil { + return nil, err + } + return event.NewSubscription(func(quit <-chan struct{}) error { + defer sub.Unsubscribe() + for { + select { + case log := <-logs: + // New log arrived, parse the event and forward to the user + event := new(SoulGasTokenAllowSgtValue) + if err := _SoulGasToken.contract.UnpackLog(event, "AllowSgtValue", log); err != nil { + return err + } + event.Raw = log + + select { + case sink <- event: + case err := <-sub.Err(): + return err + case <-quit: + return nil + } + case err := <-sub.Err(): + return err + case <-quit: + return nil + } + } + }), nil +} + +// ParseAllowSgtValue is a log parse operation binding the contract event 0xf135aca2ee4483470b8f44f38ab676fc36fc67437777f3c520e5fbeb3706009f. +// +// Solidity: event AllowSgtValue(address indexed from) +func (_SoulGasToken *SoulGasTokenFilterer) ParseAllowSgtValue(log types.Log) (*SoulGasTokenAllowSgtValue, error) { + event := new(SoulGasTokenAllowSgtValue) + if err := _SoulGasToken.contract.UnpackLog(event, "AllowSgtValue", log); err != nil { + return nil, err + } + event.Raw = log + return event, nil +} + +// SoulGasTokenApprovalIterator is returned from FilterApproval and is used to iterate over the raw logs and unpacked data for Approval events raised by the SoulGasToken contract. +type SoulGasTokenApprovalIterator struct { + Event *SoulGasTokenApproval // Event containing the contract specifics and raw log + + contract *bind.BoundContract // Generic contract to use for unpacking event data + event string // Event name to use for unpacking event data + + logs chan types.Log // Log channel receiving the found contract events + sub ethereum.Subscription // Subscription for errors, completion and termination + done bool // Whether the subscription completed delivering logs + fail error // Occurred error to stop iteration +} + +// Next advances the iterator to the subsequent event, returning whether there +// are any more events found. In case of a retrieval or parsing error, false is +// returned and Error() can be queried for the exact failure. +func (it *SoulGasTokenApprovalIterator) Next() bool { + // If the iterator failed, stop iterating + if it.fail != nil { + return false + } + // If the iterator completed, deliver directly whatever's available + if it.done { + select { + case log := <-it.logs: + it.Event = new(SoulGasTokenApproval) + if err := it.contract.UnpackLog(it.Event, it.event, log); err != nil { + it.fail = err + return false + } + it.Event.Raw = log + return true + + default: + return false + } + } + // Iterator still in progress, wait for either a data or an error event + select { + case log := <-it.logs: + it.Event = new(SoulGasTokenApproval) + if err := it.contract.UnpackLog(it.Event, it.event, log); err != nil { + it.fail = err + return false + } + it.Event.Raw = log + return true + + case err := <-it.sub.Err(): + it.done = true + it.fail = err + return it.Next() + } +} + +// Error returns any retrieval or parsing error occurred during filtering. +func (it *SoulGasTokenApprovalIterator) Error() error { + return it.fail +} + +// Close terminates the iteration process, releasing any pending underlying +// resources. +func (it *SoulGasTokenApprovalIterator) Close() error { + it.sub.Unsubscribe() + return nil +} + +// SoulGasTokenApproval represents a Approval event raised by the SoulGasToken contract. +type SoulGasTokenApproval struct { + Owner common.Address + Spender common.Address + Value *big.Int + Raw types.Log // Blockchain specific contextual infos +} + +// FilterApproval is a free log retrieval operation binding the contract event 0x8c5be1e5ebec7d5bd14f71427d1e84f3dd0314c0f7b2291e5b200ac8c7c3b925. +// +// Solidity: event Approval(address indexed owner, address indexed spender, uint256 value) +func (_SoulGasToken *SoulGasTokenFilterer) FilterApproval(opts *bind.FilterOpts, owner []common.Address, spender []common.Address) (*SoulGasTokenApprovalIterator, error) { + + var ownerRule []interface{} + for _, ownerItem := range owner { + ownerRule = append(ownerRule, ownerItem) + } + var spenderRule []interface{} + for _, spenderItem := range spender { + spenderRule = append(spenderRule, spenderItem) + } + + logs, sub, err := _SoulGasToken.contract.FilterLogs(opts, "Approval", ownerRule, spenderRule) + if err != nil { + return nil, err + } + return &SoulGasTokenApprovalIterator{contract: _SoulGasToken.contract, event: "Approval", logs: logs, sub: sub}, nil +} + +// WatchApproval is a free log subscription operation binding the contract event 0x8c5be1e5ebec7d5bd14f71427d1e84f3dd0314c0f7b2291e5b200ac8c7c3b925. +// +// Solidity: event Approval(address indexed owner, address indexed spender, uint256 value) +func (_SoulGasToken *SoulGasTokenFilterer) WatchApproval(opts *bind.WatchOpts, sink chan<- *SoulGasTokenApproval, owner []common.Address, spender []common.Address) (event.Subscription, error) { + + var ownerRule []interface{} + for _, ownerItem := range owner { + ownerRule = append(ownerRule, ownerItem) + } + var spenderRule []interface{} + for _, spenderItem := range spender { + spenderRule = append(spenderRule, spenderItem) + } + + logs, sub, err := _SoulGasToken.contract.WatchLogs(opts, "Approval", ownerRule, spenderRule) + if err != nil { + return nil, err + } + return event.NewSubscription(func(quit <-chan struct{}) error { + defer sub.Unsubscribe() + for { + select { + case log := <-logs: + // New log arrived, parse the event and forward to the user + event := new(SoulGasTokenApproval) + if err := _SoulGasToken.contract.UnpackLog(event, "Approval", log); err != nil { + return err + } + event.Raw = log + + select { + case sink <- event: + case err := <-sub.Err(): + return err + case <-quit: + return nil + } + case err := <-sub.Err(): + return err + case <-quit: + return nil + } + } + }), nil +} + +// ParseApproval is a log parse operation binding the contract event 0x8c5be1e5ebec7d5bd14f71427d1e84f3dd0314c0f7b2291e5b200ac8c7c3b925. +// +// Solidity: event Approval(address indexed owner, address indexed spender, uint256 value) +func (_SoulGasToken *SoulGasTokenFilterer) ParseApproval(log types.Log) (*SoulGasTokenApproval, error) { + event := new(SoulGasTokenApproval) + if err := _SoulGasToken.contract.UnpackLog(event, "Approval", log); err != nil { + return nil, err + } + event.Raw = log + return event, nil +} + +// SoulGasTokenDisallowSgtValueIterator is returned from FilterDisallowSgtValue and is used to iterate over the raw logs and unpacked data for DisallowSgtValue events raised by the SoulGasToken contract. +type SoulGasTokenDisallowSgtValueIterator struct { + Event *SoulGasTokenDisallowSgtValue // Event containing the contract specifics and raw log + + contract *bind.BoundContract // Generic contract to use for unpacking event data + event string // Event name to use for unpacking event data + + logs chan types.Log // Log channel receiving the found contract events + sub ethereum.Subscription // Subscription for errors, completion and termination + done bool // Whether the subscription completed delivering logs + fail error // Occurred error to stop iteration +} + +// Next advances the iterator to the subsequent event, returning whether there +// are any more events found. In case of a retrieval or parsing error, false is +// returned and Error() can be queried for the exact failure. +func (it *SoulGasTokenDisallowSgtValueIterator) Next() bool { + // If the iterator failed, stop iterating + if it.fail != nil { + return false + } + // If the iterator completed, deliver directly whatever's available + if it.done { + select { + case log := <-it.logs: + it.Event = new(SoulGasTokenDisallowSgtValue) + if err := it.contract.UnpackLog(it.Event, it.event, log); err != nil { + it.fail = err + return false + } + it.Event.Raw = log + return true + + default: + return false + } + } + // Iterator still in progress, wait for either a data or an error event + select { + case log := <-it.logs: + it.Event = new(SoulGasTokenDisallowSgtValue) + if err := it.contract.UnpackLog(it.Event, it.event, log); err != nil { + it.fail = err + return false + } + it.Event.Raw = log + return true + + case err := <-it.sub.Err(): + it.done = true + it.fail = err + return it.Next() + } +} + +// Error returns any retrieval or parsing error occurred during filtering. +func (it *SoulGasTokenDisallowSgtValueIterator) Error() error { + return it.fail +} + +// Close terminates the iteration process, releasing any pending underlying +// resources. +func (it *SoulGasTokenDisallowSgtValueIterator) Close() error { + it.sub.Unsubscribe() + return nil +} + +// SoulGasTokenDisallowSgtValue represents a DisallowSgtValue event raised by the SoulGasToken contract. +type SoulGasTokenDisallowSgtValue struct { + From common.Address + Raw types.Log // Blockchain specific contextual infos +} + +// FilterDisallowSgtValue is a free log retrieval operation binding the contract event 0x42d3350598a4a2ec6e60463e0bffa1aab494a9e8d4484b017270dde628b4edb1. +// +// Solidity: event DisallowSgtValue(address indexed from) +func (_SoulGasToken *SoulGasTokenFilterer) FilterDisallowSgtValue(opts *bind.FilterOpts, from []common.Address) (*SoulGasTokenDisallowSgtValueIterator, error) { + + var fromRule []interface{} + for _, fromItem := range from { + fromRule = append(fromRule, fromItem) + } + + logs, sub, err := _SoulGasToken.contract.FilterLogs(opts, "DisallowSgtValue", fromRule) + if err != nil { + return nil, err + } + return &SoulGasTokenDisallowSgtValueIterator{contract: _SoulGasToken.contract, event: "DisallowSgtValue", logs: logs, sub: sub}, nil +} + +// WatchDisallowSgtValue is a free log subscription operation binding the contract event 0x42d3350598a4a2ec6e60463e0bffa1aab494a9e8d4484b017270dde628b4edb1. +// +// Solidity: event DisallowSgtValue(address indexed from) +func (_SoulGasToken *SoulGasTokenFilterer) WatchDisallowSgtValue(opts *bind.WatchOpts, sink chan<- *SoulGasTokenDisallowSgtValue, from []common.Address) (event.Subscription, error) { + + var fromRule []interface{} + for _, fromItem := range from { + fromRule = append(fromRule, fromItem) + } + + logs, sub, err := _SoulGasToken.contract.WatchLogs(opts, "DisallowSgtValue", fromRule) + if err != nil { + return nil, err + } + return event.NewSubscription(func(quit <-chan struct{}) error { + defer sub.Unsubscribe() + for { + select { + case log := <-logs: + // New log arrived, parse the event and forward to the user + event := new(SoulGasTokenDisallowSgtValue) + if err := _SoulGasToken.contract.UnpackLog(event, "DisallowSgtValue", log); err != nil { + return err + } + event.Raw = log + + select { + case sink <- event: + case err := <-sub.Err(): + return err + case <-quit: + return nil + } + case err := <-sub.Err(): + return err + case <-quit: + return nil + } + } + }), nil +} + +// ParseDisallowSgtValue is a log parse operation binding the contract event 0x42d3350598a4a2ec6e60463e0bffa1aab494a9e8d4484b017270dde628b4edb1. +// +// Solidity: event DisallowSgtValue(address indexed from) +func (_SoulGasToken *SoulGasTokenFilterer) ParseDisallowSgtValue(log types.Log) (*SoulGasTokenDisallowSgtValue, error) { + event := new(SoulGasTokenDisallowSgtValue) + if err := _SoulGasToken.contract.UnpackLog(event, "DisallowSgtValue", log); err != nil { + return nil, err + } + event.Raw = log + return event, nil +} + +// SoulGasTokenInitializedIterator is returned from FilterInitialized and is used to iterate over the raw logs and unpacked data for Initialized events raised by the SoulGasToken contract. +type SoulGasTokenInitializedIterator struct { + Event *SoulGasTokenInitialized // Event containing the contract specifics and raw log + + contract *bind.BoundContract // Generic contract to use for unpacking event data + event string // Event name to use for unpacking event data + + logs chan types.Log // Log channel receiving the found contract events + sub ethereum.Subscription // Subscription for errors, completion and termination + done bool // Whether the subscription completed delivering logs + fail error // Occurred error to stop iteration +} + +// Next advances the iterator to the subsequent event, returning whether there +// are any more events found. In case of a retrieval or parsing error, false is +// returned and Error() can be queried for the exact failure. +func (it *SoulGasTokenInitializedIterator) Next() bool { + // If the iterator failed, stop iterating + if it.fail != nil { + return false + } + // If the iterator completed, deliver directly whatever's available + if it.done { + select { + case log := <-it.logs: + it.Event = new(SoulGasTokenInitialized) + if err := it.contract.UnpackLog(it.Event, it.event, log); err != nil { + it.fail = err + return false + } + it.Event.Raw = log + return true + + default: + return false + } + } + // Iterator still in progress, wait for either a data or an error event + select { + case log := <-it.logs: + it.Event = new(SoulGasTokenInitialized) + if err := it.contract.UnpackLog(it.Event, it.event, log); err != nil { + it.fail = err + return false + } + it.Event.Raw = log + return true + + case err := <-it.sub.Err(): + it.done = true + it.fail = err + return it.Next() + } +} + +// Error returns any retrieval or parsing error occurred during filtering. +func (it *SoulGasTokenInitializedIterator) Error() error { + return it.fail +} + +// Close terminates the iteration process, releasing any pending underlying +// resources. +func (it *SoulGasTokenInitializedIterator) Close() error { + it.sub.Unsubscribe() + return nil +} + +// SoulGasTokenInitialized represents a Initialized event raised by the SoulGasToken contract. +type SoulGasTokenInitialized struct { + Version uint8 + Raw types.Log // Blockchain specific contextual infos +} + +// FilterInitialized is a free log retrieval operation binding the contract event 0x7f26b83ff96e1f2b6a682f133852f6798a09c465da95921460cefb3847402498. +// +// Solidity: event Initialized(uint8 version) +func (_SoulGasToken *SoulGasTokenFilterer) FilterInitialized(opts *bind.FilterOpts) (*SoulGasTokenInitializedIterator, error) { + + logs, sub, err := _SoulGasToken.contract.FilterLogs(opts, "Initialized") + if err != nil { + return nil, err + } + return &SoulGasTokenInitializedIterator{contract: _SoulGasToken.contract, event: "Initialized", logs: logs, sub: sub}, nil +} + +// WatchInitialized is a free log subscription operation binding the contract event 0x7f26b83ff96e1f2b6a682f133852f6798a09c465da95921460cefb3847402498. +// +// Solidity: event Initialized(uint8 version) +func (_SoulGasToken *SoulGasTokenFilterer) WatchInitialized(opts *bind.WatchOpts, sink chan<- *SoulGasTokenInitialized) (event.Subscription, error) { + + logs, sub, err := _SoulGasToken.contract.WatchLogs(opts, "Initialized") + if err != nil { + return nil, err + } + return event.NewSubscription(func(quit <-chan struct{}) error { + defer sub.Unsubscribe() + for { + select { + case log := <-logs: + // New log arrived, parse the event and forward to the user + event := new(SoulGasTokenInitialized) + if err := _SoulGasToken.contract.UnpackLog(event, "Initialized", log); err != nil { + return err + } + event.Raw = log + + select { + case sink <- event: + case err := <-sub.Err(): + return err + case <-quit: + return nil + } + case err := <-sub.Err(): + return err + case <-quit: + return nil + } + } + }), nil +} + +// ParseInitialized is a log parse operation binding the contract event 0x7f26b83ff96e1f2b6a682f133852f6798a09c465da95921460cefb3847402498. +// +// Solidity: event Initialized(uint8 version) +func (_SoulGasToken *SoulGasTokenFilterer) ParseInitialized(log types.Log) (*SoulGasTokenInitialized, error) { + event := new(SoulGasTokenInitialized) + if err := _SoulGasToken.contract.UnpackLog(event, "Initialized", log); err != nil { + return nil, err + } + event.Raw = log + return event, nil +} + +// SoulGasTokenOwnershipTransferredIterator is returned from FilterOwnershipTransferred and is used to iterate over the raw logs and unpacked data for OwnershipTransferred events raised by the SoulGasToken contract. +type SoulGasTokenOwnershipTransferredIterator struct { + Event *SoulGasTokenOwnershipTransferred // Event containing the contract specifics and raw log + + contract *bind.BoundContract // Generic contract to use for unpacking event data + event string // Event name to use for unpacking event data + + logs chan types.Log // Log channel receiving the found contract events + sub ethereum.Subscription // Subscription for errors, completion and termination + done bool // Whether the subscription completed delivering logs + fail error // Occurred error to stop iteration +} + +// Next advances the iterator to the subsequent event, returning whether there +// are any more events found. In case of a retrieval or parsing error, false is +// returned and Error() can be queried for the exact failure. +func (it *SoulGasTokenOwnershipTransferredIterator) Next() bool { + // If the iterator failed, stop iterating + if it.fail != nil { + return false + } + // If the iterator completed, deliver directly whatever's available + if it.done { + select { + case log := <-it.logs: + it.Event = new(SoulGasTokenOwnershipTransferred) + if err := it.contract.UnpackLog(it.Event, it.event, log); err != nil { + it.fail = err + return false + } + it.Event.Raw = log + return true + + default: + return false + } + } + // Iterator still in progress, wait for either a data or an error event + select { + case log := <-it.logs: + it.Event = new(SoulGasTokenOwnershipTransferred) + if err := it.contract.UnpackLog(it.Event, it.event, log); err != nil { + it.fail = err + return false + } + it.Event.Raw = log + return true + + case err := <-it.sub.Err(): + it.done = true + it.fail = err + return it.Next() + } +} + +// Error returns any retrieval or parsing error occurred during filtering. +func (it *SoulGasTokenOwnershipTransferredIterator) Error() error { + return it.fail +} + +// Close terminates the iteration process, releasing any pending underlying +// resources. +func (it *SoulGasTokenOwnershipTransferredIterator) Close() error { + it.sub.Unsubscribe() + return nil +} + +// SoulGasTokenOwnershipTransferred represents a OwnershipTransferred event raised by the SoulGasToken contract. +type SoulGasTokenOwnershipTransferred struct { + PreviousOwner common.Address + NewOwner common.Address + Raw types.Log // Blockchain specific contextual infos +} + +// FilterOwnershipTransferred is a free log retrieval operation binding the contract event 0x8be0079c531659141344cd1fd0a4f28419497f9722a3daafe3b4186f6b6457e0. +// +// Solidity: event OwnershipTransferred(address indexed previousOwner, address indexed newOwner) +func (_SoulGasToken *SoulGasTokenFilterer) FilterOwnershipTransferred(opts *bind.FilterOpts, previousOwner []common.Address, newOwner []common.Address) (*SoulGasTokenOwnershipTransferredIterator, error) { + + var previousOwnerRule []interface{} + for _, previousOwnerItem := range previousOwner { + previousOwnerRule = append(previousOwnerRule, previousOwnerItem) + } + var newOwnerRule []interface{} + for _, newOwnerItem := range newOwner { + newOwnerRule = append(newOwnerRule, newOwnerItem) + } + + logs, sub, err := _SoulGasToken.contract.FilterLogs(opts, "OwnershipTransferred", previousOwnerRule, newOwnerRule) + if err != nil { + return nil, err + } + return &SoulGasTokenOwnershipTransferredIterator{contract: _SoulGasToken.contract, event: "OwnershipTransferred", logs: logs, sub: sub}, nil +} + +// WatchOwnershipTransferred is a free log subscription operation binding the contract event 0x8be0079c531659141344cd1fd0a4f28419497f9722a3daafe3b4186f6b6457e0. +// +// Solidity: event OwnershipTransferred(address indexed previousOwner, address indexed newOwner) +func (_SoulGasToken *SoulGasTokenFilterer) WatchOwnershipTransferred(opts *bind.WatchOpts, sink chan<- *SoulGasTokenOwnershipTransferred, previousOwner []common.Address, newOwner []common.Address) (event.Subscription, error) { + + var previousOwnerRule []interface{} + for _, previousOwnerItem := range previousOwner { + previousOwnerRule = append(previousOwnerRule, previousOwnerItem) + } + var newOwnerRule []interface{} + for _, newOwnerItem := range newOwner { + newOwnerRule = append(newOwnerRule, newOwnerItem) + } + + logs, sub, err := _SoulGasToken.contract.WatchLogs(opts, "OwnershipTransferred", previousOwnerRule, newOwnerRule) + if err != nil { + return nil, err + } + return event.NewSubscription(func(quit <-chan struct{}) error { + defer sub.Unsubscribe() + for { + select { + case log := <-logs: + // New log arrived, parse the event and forward to the user + event := new(SoulGasTokenOwnershipTransferred) + if err := _SoulGasToken.contract.UnpackLog(event, "OwnershipTransferred", log); err != nil { + return err + } + event.Raw = log + + select { + case sink <- event: + case err := <-sub.Err(): + return err + case <-quit: + return nil + } + case err := <-sub.Err(): + return err + case <-quit: + return nil + } + } + }), nil +} + +// ParseOwnershipTransferred is a log parse operation binding the contract event 0x8be0079c531659141344cd1fd0a4f28419497f9722a3daafe3b4186f6b6457e0. +// +// Solidity: event OwnershipTransferred(address indexed previousOwner, address indexed newOwner) +func (_SoulGasToken *SoulGasTokenFilterer) ParseOwnershipTransferred(log types.Log) (*SoulGasTokenOwnershipTransferred, error) { + event := new(SoulGasTokenOwnershipTransferred) + if err := _SoulGasToken.contract.UnpackLog(event, "OwnershipTransferred", log); err != nil { + return nil, err + } + event.Raw = log + return event, nil +} + +// SoulGasTokenTransferIterator is returned from FilterTransfer and is used to iterate over the raw logs and unpacked data for Transfer events raised by the SoulGasToken contract. +type SoulGasTokenTransferIterator struct { + Event *SoulGasTokenTransfer // Event containing the contract specifics and raw log + + contract *bind.BoundContract // Generic contract to use for unpacking event data + event string // Event name to use for unpacking event data + + logs chan types.Log // Log channel receiving the found contract events + sub ethereum.Subscription // Subscription for errors, completion and termination + done bool // Whether the subscription completed delivering logs + fail error // Occurred error to stop iteration +} + +// Next advances the iterator to the subsequent event, returning whether there +// are any more events found. In case of a retrieval or parsing error, false is +// returned and Error() can be queried for the exact failure. +func (it *SoulGasTokenTransferIterator) Next() bool { + // If the iterator failed, stop iterating + if it.fail != nil { + return false + } + // If the iterator completed, deliver directly whatever's available + if it.done { + select { + case log := <-it.logs: + it.Event = new(SoulGasTokenTransfer) + if err := it.contract.UnpackLog(it.Event, it.event, log); err != nil { + it.fail = err + return false + } + it.Event.Raw = log + return true + + default: + return false + } + } + // Iterator still in progress, wait for either a data or an error event + select { + case log := <-it.logs: + it.Event = new(SoulGasTokenTransfer) + if err := it.contract.UnpackLog(it.Event, it.event, log); err != nil { + it.fail = err + return false + } + it.Event.Raw = log + return true + + case err := <-it.sub.Err(): + it.done = true + it.fail = err + return it.Next() + } +} + +// Error returns any retrieval or parsing error occurred during filtering. +func (it *SoulGasTokenTransferIterator) Error() error { + return it.fail +} + +// Close terminates the iteration process, releasing any pending underlying +// resources. +func (it *SoulGasTokenTransferIterator) Close() error { + it.sub.Unsubscribe() + return nil +} + +// SoulGasTokenTransfer represents a Transfer event raised by the SoulGasToken contract. +type SoulGasTokenTransfer struct { + From common.Address + To common.Address + Value *big.Int + Raw types.Log // Blockchain specific contextual infos +} + +// FilterTransfer is a free log retrieval operation binding the contract event 0xddf252ad1be2c89b69c2b068fc378daa952ba7f163c4a11628f55a4df523b3ef. +// +// Solidity: event Transfer(address indexed from, address indexed to, uint256 value) +func (_SoulGasToken *SoulGasTokenFilterer) FilterTransfer(opts *bind.FilterOpts, from []common.Address, to []common.Address) (*SoulGasTokenTransferIterator, error) { + + var fromRule []interface{} + for _, fromItem := range from { + fromRule = append(fromRule, fromItem) + } + var toRule []interface{} + for _, toItem := range to { + toRule = append(toRule, toItem) + } + + logs, sub, err := _SoulGasToken.contract.FilterLogs(opts, "Transfer", fromRule, toRule) + if err != nil { + return nil, err + } + return &SoulGasTokenTransferIterator{contract: _SoulGasToken.contract, event: "Transfer", logs: logs, sub: sub}, nil +} + +// WatchTransfer is a free log subscription operation binding the contract event 0xddf252ad1be2c89b69c2b068fc378daa952ba7f163c4a11628f55a4df523b3ef. +// +// Solidity: event Transfer(address indexed from, address indexed to, uint256 value) +func (_SoulGasToken *SoulGasTokenFilterer) WatchTransfer(opts *bind.WatchOpts, sink chan<- *SoulGasTokenTransfer, from []common.Address, to []common.Address) (event.Subscription, error) { + + var fromRule []interface{} + for _, fromItem := range from { + fromRule = append(fromRule, fromItem) + } + var toRule []interface{} + for _, toItem := range to { + toRule = append(toRule, toItem) + } + + logs, sub, err := _SoulGasToken.contract.WatchLogs(opts, "Transfer", fromRule, toRule) + if err != nil { + return nil, err + } + return event.NewSubscription(func(quit <-chan struct{}) error { + defer sub.Unsubscribe() + for { + select { + case log := <-logs: + // New log arrived, parse the event and forward to the user + event := new(SoulGasTokenTransfer) + if err := _SoulGasToken.contract.UnpackLog(event, "Transfer", log); err != nil { + return err + } + event.Raw = log + + select { + case sink <- event: + case err := <-sub.Err(): + return err + case <-quit: + return nil + } + case err := <-sub.Err(): + return err + case <-quit: + return nil + } + } + }), nil +} + +// ParseTransfer is a log parse operation binding the contract event 0xddf252ad1be2c89b69c2b068fc378daa952ba7f163c4a11628f55a4df523b3ef. +// +// Solidity: event Transfer(address indexed from, address indexed to, uint256 value) +func (_SoulGasToken *SoulGasTokenFilterer) ParseTransfer(log types.Log) (*SoulGasTokenTransfer, error) { + event := new(SoulGasTokenTransfer) + if err := _SoulGasToken.contract.UnpackLog(event, "Transfer", log); err != nil { + return nil, err + } + event.Raw = log + return event, nil +}
diff --git official optimism/op-e2e/config/init.go SWC optimism/op-e2e/config/init.go index 48867df80a642f488e57e8f5d9d96d02f62ee2c9..c2bc3b4f9088dec56acb96356880d53b2822e97e 100644 --- official optimism/op-e2e/config/init.go +++ SWC optimism/op-e2e/config/init.go @@ -254,6 +254,10 @@ "l2GenesisGraniteTimeOffset": nil, "l2GenesisHoloceneTimeOffset": nil, "l2GenesisIsthmusTimeOffset": nil, "l2GenesisJovianTimeOffset": nil, + // SWC changes + "deploySoulGasToken": true, + "isSoulBackedByNative": true, + "soulGasTokenTimeOffset": "0x0", }   upgradeSchedule := new(genesis.UpgradeScheduleDeployConfig) @@ -310,6 +314,9 @@ dc.FundDevAccounts = true // Speed up the in memory tests dc.L1BlockTime = 2 dc.L2BlockTime = 1 + if !(dc.DeploySoulGasToken && dc.IsSoulBackedByNative) { + panic("Soul gas token must be deployed and backed by native") + } dc.SetContracts(l1Contracts) mtx.Lock() deployConfigsByType[allocType] = dc
diff --git official optimism/op-e2e/sgt/helper.go SWC optimism/op-e2e/sgt/helper.go new file mode 100644 index 0000000000000000000000000000000000000000..d39dd4e436202c5afb2c8206379a03e58e3ce140 --- /dev/null +++ SWC optimism/op-e2e/sgt/helper.go @@ -0,0 +1,92 @@ +package sgt + +import ( + "context" + "crypto/ecdsa" + "math/big" + "testing" + "time" + + "github.com/ethereum-optimism/optimism/op-e2e/bindings" + "github.com/ethereum-optimism/optimism/op-e2e/e2eutils/wait" + "github.com/ethereum-optimism/optimism/op-e2e/system/e2esys" + "github.com/ethereum-optimism/optimism/op-service/predeploys" + "github.com/ethereum/go-ethereum/accounts/abi/bind" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/crypto" + "github.com/ethereum/go-ethereum/ethclient" + "github.com/stretchr/testify/require" +) + +type SgtHelper struct { + T *testing.T + L2Client *ethclient.Client + SysCfg e2esys.SystemConfig + SgtContract *bindings.SoulGasToken + ChainID *big.Int + sys *e2esys.System +} + +func NewSgtHelper(t *testing.T, ctx context.Context, sys *e2esys.System) *SgtHelper { + // use sequencer's L2 client + client := sys.NodeClient(e2esys.RoleSeq) + chainID, err := client.ChainID(ctx) + require.NoError(t, err) + + sgtAddr := predeploys.SoulGasTokenAddr + sgtContract, err := bindings.NewSoulGasToken(sgtAddr, client) + require.NoError(t, err) + + return &SgtHelper{ + T: t, + L2Client: client, + SysCfg: sys.Cfg, + SgtContract: sgtContract, + ChainID: chainID, + sys: sys, + } +} + +func (s *SgtHelper) GetTestAccount(idx int) *ecdsa.PrivateKey { + return s.sys.TestAccount(idx) +} + +func (s *SgtHelper) depositSgtAndNativeFromGenesisAccountToAccount(t *testing.T, ctx context.Context, toAddr common.Address, sgtValue *big.Int, l2Value *big.Int) { + privKey := s.GetTestAccount(0) // Genesis Account with lots of native balances + // deposit some sgt and native tokens first + txOpts, err := bind.NewKeyedTransactorWithChainID(privKey, s.ChainID) + require.NoError(t, err) + txOpts.Value = sgtValue + sgtTx, err := s.SgtContract.BatchDepositForAll(txOpts, []common.Address{toAddr}, sgtValue) + require.NoError(t, err) + _, err = wait.ForReceiptOK(ctx, s.L2Client, sgtTx.Hash()) + require.NoError(t, err) + nativeTx, err := s.transferNativeToken(t, ctx, privKey, toAddr, l2Value) + require.NoError(t, err) + _, err = wait.ForReceiptOK(ctx, s.L2Client, nativeTx.Hash()) + require.NoError(t, err) +} + +func (s *SgtHelper) transferNativeToken(t *testing.T, ctx context.Context, sender *ecdsa.PrivateKey, toAddr common.Address, amount *big.Int) (*types.Transaction, error) { + chainID, err := s.L2Client.ChainID(ctx) + require.NoError(t, err) + gasFeeCap := big.NewInt(200) + gasTipCap := big.NewInt(10) + + nonce, err := s.L2Client.NonceAt(ctx, crypto.PubkeyToAddress(sender.PublicKey), nil) + require.NoError(t, err) + tx := types.MustSignNewTx(sender, types.LatestSignerForChainID(chainID), &types.DynamicFeeTx{ + ChainID: chainID, + Nonce: nonce, + To: &toAddr, + Value: amount, + GasTipCap: gasTipCap, + GasFeeCap: gasFeeCap, + Gas: 21000, + }) + ctx, cancel := context.WithTimeout(ctx, 5*time.Second) + defer cancel() + err = s.L2Client.SendTransaction(ctx, tx) + return tx, err +}
diff --git official optimism/op-e2e/sgt/sgt_test.go SWC optimism/op-e2e/sgt/sgt_test.go new file mode 100644 index 0000000000000000000000000000000000000000..93fd2372f3cd594443dbd26d95ee724cc0ba21c1 --- /dev/null +++ SWC optimism/op-e2e/sgt/sgt_test.go @@ -0,0 +1,396 @@ +package sgt + +import ( + "context" + "crypto/ecdsa" + "math/big" + "math/rand" + "testing" + + op_e2e "github.com/ethereum-optimism/optimism/op-e2e" + "github.com/ethereum-optimism/optimism/op-e2e/e2eutils/wait" + "github.com/ethereum-optimism/optimism/op-e2e/system/e2esys" + "github.com/ethereum-optimism/optimism/op-service/predeploys" + "github.com/ethereum-optimism/optimism/op-service/testutils" + + "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/core/types" + "github.com/ethereum/go-ethereum/crypto" + "github.com/stretchr/testify/require" +) + +var ( + seqVault = predeploys.SequencerFeeVaultAddr + baseVault = predeploys.BaseFeeVaultAddr + l1Vault = predeploys.L1FeeVaultAddr + dummyAddr = common.Address{0xff, 0xff} +) + +func TestSGTDepositFunctionSuccess(t *testing.T) { + op_e2e.InitParallel(t) + sgtTimeOffset := uint64(0) + sys := startSystemWithSGT(&sgtTimeOffset, t) + t.Cleanup(sys.Close) + ctx := context.Background() + + sgt := NewSgtHelper(t, ctx, sys) + depositSgtValue := big.NewInt(10000) + _, _, _ = setUpTestAccount(t, ctx, 0, sgt, depositSgtValue, big.NewInt(0)) +} + +func startSystemWithSGT(sgtTimeOffset *uint64, t *testing.T) *e2esys.System { + cfg := e2esys.DefaultSystemConfig(t) + delete(cfg.Nodes, "verifier") + _, ok := cfg.Nodes["sequencer"] + require.True(t, ok, "sequencer is required") + + if sgtTimeOffset != nil { + cfg.DeployConfig.DeploySoulGasToken = true + cfg.DeployConfig.SoulGasTokenTimeOffset = (*hexutil.Uint64)(sgtTimeOffset) + } else { + cfg.DeployConfig.DeploySoulGasToken = false + } + // Disable proposer creating fast games automatically - required games are manually created + cfg.DisableProposer = true + sys, err := cfg.Start(t) + require.Nil(t, err, "Error starting up system") + return sys +} + +// Diverse test scenarios to verify that the SoulGasToken(sgt) is utilized for gas payment firstly, +// unless there is insufficient sgt balance, in which case the native balance will be used instead. +func TestSGTAsGasPayment(t *testing.T) { + op_e2e.InitParallel(t) + sgtTimeOffset := uint64(0) + sys := startSystemWithSGT(&sgtTimeOffset, t) + t.Cleanup(sys.Close) + ctx := context.Background() + + sgt := NewSgtHelper(t, ctx, sys) + // 1. setup a test account and deposit specified amount of sgt tokens (`depositSgtValue`) and native tokens (`depositL2Value`) into it. + // 2. execute a token transfer tx with `txValue` to `dummyAddr` and validate that the gas payment behavior using sgt is as anticipated. + tests := []struct { + name string + action func(t *testing.T, ctx context.Context, index int64, sgt *SgtHelper) + }{ + { + name: "NativaGasPaymentWithoutSGTSuccess", + action: nativaGasPaymentWithoutSGTSuccess, + }, + { + name: "FullSGTGasPaymentWithoutNativeBalanceSuccess", + action: fullSGTGasPaymentWithoutNativeBalanceSuccess, + }, + { + name: "FullSGTGasPaymentWithNativeBalanceSuccess", + action: fullSGTGasPaymentWithNativeBalanceSuccess, + }, + { + name: "PartialSGTGasPaymentSuccess", + action: partialSGTGasPaymentSuccess, + }, + { + name: "FullSGTGasPaymentAndNonZeroTxValueWithSufficientNativeBalanceSuccess", + action: fullSGTGasPaymentAndNonZeroTxValueWithSufficientNativeBalanceSuccess, + }, + { + name: "PartialSGTGasPaymentAndNonZeroTxValueWithSufficientNativeBalanceSuccess", + action: partialSGTGasPaymentAndNonZeroTxValueWithSufficientNativeBalanceSuccess, + }, + { + name: "FullSGTInsufficientGasPaymentFail", + action: fullSGTInsufficientGasPaymentFail, + }, + { + name: "FullNativeInsufficientGasPaymentFail", + action: fullNativeInsufficientGasPaymentFail, + }, + { + name: "PartialSGTInsufficientGasPaymentFail", + action: partialSGTInsufficientGasPaymentFail, + }, + { + name: "FullSGTGasPaymentAndNonZeroTxValueWithInsufficientNativeBalanceFail", + action: fullSGTGasPaymentAndNonZeroTxValueWithInsufficientNativeBalanceFail, + }, + { + name: "PartialSGTGasPaymentAndNonZeroTxValueWithInsufficientNativeBalanceFail", + action: partialSGTGasPaymentAndNonZeroTxValueWithInsufficientNativeBalanceFail, + }, + } + + for index, tCase := range tests { + t.Run(tCase.name, func(t *testing.T) { + tCase.action(t, ctx, int64(index), sgt) + }) + } +} + +func setUpTestAccount(t *testing.T, ctx context.Context, index int64, sgt *SgtHelper, depositSgtValue *big.Int, depositL2Value *big.Int) (*ecdsa.PrivateKey, common.Address, *big.Int) { + opts := &bind.CallOpts{Context: ctx} + rng := rand.New(rand.NewSource(index)) + testPrivKey := testutils.InsecureRandomKey(rng) + testAddr := crypto.PubkeyToAddress(testPrivKey.PublicKey) + + // check it's a fresh account + sgtBalance, err := sgt.SgtContract.BalanceOf(opts, testAddr) + require.NoError(t, err) + require.Equal(t, int64(0), sgtBalance.Int64()) + l2Balance, err := sgt.L2Client.BalanceAt(ctx, testAddr, nil) + require.NoError(t, err) + require.Equal(t, int64(0), l2Balance.Int64()) + + // deposit initial sgt and native(L2) balance to the test account + sgt.depositSgtAndNativeFromGenesisAccountToAccount(t, ctx, testAddr, depositSgtValue, depositL2Value) + // ensure that sgt and native balance of testAccount are correctly initialized + preSgtBalance, err := sgt.SgtContract.BalanceOf(opts, testAddr) + require.NoError(t, err) + require.Equal(t, depositSgtValue.Cmp(preSgtBalance), 0) + preL2Balance, err := sgt.L2Client.BalanceAt(ctx, testAddr, nil) + require.NoError(t, err) + require.Equal(t, depositL2Value.Cmp(preL2Balance), 0) + + return testPrivKey, testAddr, calcVaultBalance(t, ctx, sgt) +} + +// balance invariant check: preTotalBalance = postTotalBalance + gasCost + txValue +func invariantBalanceCheck(t *testing.T, ctx context.Context, sgt *SgtHelper, addr common.Address, gasCost *big.Int, txValue *big.Int, preSgtBalance *big.Int, preL2Balance *big.Int, postSgtBalance *big.Int, sgtShouldChange bool) { + if sgtShouldChange { + require.True(t, preSgtBalance.Cmp(postSgtBalance) != 0) + } else { + require.True(t, preSgtBalance.Cmp(postSgtBalance) == 0) + } + postL2Balance, err := sgt.L2Client.BalanceAt(ctx, addr, nil) + require.NoError(t, err) + preBalance := new(big.Int).Add(preSgtBalance, preL2Balance) + postBalance := new(big.Int).Add(postSgtBalance, gasCost) + postBalance = postBalance.Add(postBalance, txValue) + postBalance = postBalance.Add(postBalance, postL2Balance) + require.Equal(t, 0, preBalance.Cmp(postBalance)) +} + +func nativaGasPaymentWithoutSGTSuccess(t *testing.T, ctx context.Context, index int64, sgt *SgtHelper) { + depositSgtValue := big.NewInt(0) + // 10000000000000 is a random chosen value that is far bigger than the gas cos (~1225000231000) of the following `transferNativeToken` tx + depositL2Value := big.NewInt(10000000000000) + txValue := big.NewInt(0) + testAccount, testAddr, vaultBalanceBefore := setUpTestAccount(t, ctx, index, sgt, depositSgtValue, depositL2Value) + + // make a simple tx with the testAccount: transfer txValue from testAccount to dummyAddr + tx, err := sgt.transferNativeToken(t, ctx, testAccount, dummyAddr, txValue) + require.NoError(t, err) + receipt, err := wait.ForReceiptOK(ctx, sgt.L2Client, tx.Hash()) + require.NoError(t, err) + gasCost := calcGasFee(receipt) + vaultBalanceAfter := calcVaultBalance(t, ctx, sgt) + + // gasCost == vaultBalanceDiff check + require.Equal(t, new(big.Int).Sub(vaultBalanceAfter, vaultBalanceBefore).Cmp(gasCost), 0) + // post sgt balance check: it should be 0 + opts := &bind.CallOpts{Context: ctx} + postSgtBalance, err := sgt.SgtContract.BalanceOf(opts, testAddr) + require.NoError(t, err) + require.Equal(t, common.Big0.Cmp(postSgtBalance), 0) + // balance invariant check + invariantBalanceCheck(t, ctx, sgt, testAddr, gasCost, txValue, depositSgtValue, depositL2Value, postSgtBalance, false) +} + +func fullSGTGasPaymentWithoutNativeBalanceSuccess(t *testing.T, ctx context.Context, index int64, sgt *SgtHelper) { + depositSgtValue := big.NewInt(10000000000000) + depositL2Value := big.NewInt(0) + txValue := big.NewInt(0) + testAccount, testAddr, vaultBalanceBefore := setUpTestAccount(t, ctx, index, sgt, depositSgtValue, depositL2Value) + + // make a simple tx with the testAccount: transfer txValue from testAccount to dummyAddr + tx, err := sgt.transferNativeToken(t, ctx, testAccount, dummyAddr, txValue) + require.NoError(t, err) + receipt, err := wait.ForReceiptOK(ctx, sgt.L2Client, tx.Hash()) + require.NoError(t, err) + gasCost := calcGasFee(receipt) + vaultBalanceAfter := calcVaultBalance(t, ctx, sgt) + + // gasCost == vaultBalanceDiff check + require.Equal(t, new(big.Int).Sub(vaultBalanceAfter, vaultBalanceBefore).Cmp(gasCost), 0) + // post sgt balance check: sgt should be used as gas first + opts := &bind.CallOpts{Context: ctx} + postSgtBalance, err := sgt.SgtContract.BalanceOf(opts, testAddr) + require.NoError(t, err) + require.Equal(t, new(big.Int).Add(postSgtBalance, gasCost).Cmp(depositSgtValue), 0) + // balance invariant check + invariantBalanceCheck(t, ctx, sgt, testAddr, gasCost, txValue, depositSgtValue, depositL2Value, postSgtBalance, true) +} + +func fullSGTGasPaymentWithNativeBalanceSuccess(t *testing.T, ctx context.Context, index int64, sgt *SgtHelper) { + depositSgtValue := big.NewInt(10000000000000) + depositL2Value := big.NewInt(10000000000000) + txValue := big.NewInt(0) + testAccount, testAddr, vaultBalanceBefore := setUpTestAccount(t, ctx, index, sgt, depositSgtValue, depositL2Value) + + // make a simple tx with the testAccount: transfer txValue from testAccount to dummyAddr + tx, err := sgt.transferNativeToken(t, ctx, testAccount, dummyAddr, txValue) + require.NoError(t, err) + receipt, err := wait.ForReceiptOK(ctx, sgt.L2Client, tx.Hash()) + require.NoError(t, err) + gasCost := calcGasFee(receipt) + vaultBalanceAfter := calcVaultBalance(t, ctx, sgt) + + // gasCost == vaultBalanceDiff check + require.Equal(t, new(big.Int).Sub(vaultBalanceAfter, vaultBalanceBefore).Cmp(gasCost), 0) + // post sgt balance check: sgt should be used as gas first + opts := &bind.CallOpts{Context: ctx} + postSgtBalance, err := sgt.SgtContract.BalanceOf(opts, testAddr) + require.NoError(t, err) + require.Equal(t, new(big.Int).Add(postSgtBalance, gasCost).Cmp(depositSgtValue), 0) + // balance invariant check + invariantBalanceCheck(t, ctx, sgt, testAddr, gasCost, txValue, depositSgtValue, depositL2Value, postSgtBalance, true) +} + +func partialSGTGasPaymentSuccess(t *testing.T, ctx context.Context, index int64, sgt *SgtHelper) { + // 1000 is a random chosen value that is far less than the gas cos (~1225000231000) of the following `transferNativeToken` tx + depositSgtValue := big.NewInt(1000) + depositL2Value := big.NewInt(10000000000000) + txValue := big.NewInt(0) + testAccount, testAddr, vaultBalanceBefore := setUpTestAccount(t, ctx, index, sgt, depositSgtValue, depositL2Value) + + // make a simple tx with the testAccount: transfer txValue from testAccount to dummyAddr + tx, err := sgt.transferNativeToken(t, ctx, testAccount, dummyAddr, txValue) + require.NoError(t, err) + receipt, err := wait.ForReceiptOK(ctx, sgt.L2Client, tx.Hash()) + require.NoError(t, err) + gasCost := calcGasFee(receipt) + vaultBalanceAfter := calcVaultBalance(t, ctx, sgt) + + // gasCost == vaultBalanceDiff check + require.Equal(t, new(big.Int).Sub(vaultBalanceAfter, vaultBalanceBefore).Cmp(gasCost), 0) + // post sgt balance check: sgt should be used as gas first and should be spent all + opts := &bind.CallOpts{Context: ctx} + postSgtBalance, err := sgt.SgtContract.BalanceOf(opts, testAddr) + require.NoError(t, err) + require.Equal(t, common.Big0.Cmp(postSgtBalance), 0) + // balance invariant check + invariantBalanceCheck(t, ctx, sgt, testAddr, gasCost, txValue, depositSgtValue, depositL2Value, postSgtBalance, true) +} + +func fullSGTGasPaymentAndNonZeroTxValueWithSufficientNativeBalanceSuccess(t *testing.T, ctx context.Context, index int64, sgt *SgtHelper) { + depositSgtValue := big.NewInt(10000000000000) + depositL2Value := big.NewInt(10000000000000) + txValue := big.NewInt(10000) + testAccount, testAddr, vaultBalanceBefore := setUpTestAccount(t, ctx, index, sgt, depositSgtValue, depositL2Value) + + // make a simple tx with the testAccount: transfer txValue from testAccount to dummyAddr + tx, err := sgt.transferNativeToken(t, ctx, testAccount, dummyAddr, txValue) + require.NoError(t, err) + receipt, err := wait.ForReceiptOK(ctx, sgt.L2Client, tx.Hash()) + require.NoError(t, err) + gasCost := calcGasFee(receipt) + vaultBalanceAfter := calcVaultBalance(t, ctx, sgt) + + // gasCost == vaultBalanceDiff check + require.Equal(t, new(big.Int).Sub(vaultBalanceAfter, vaultBalanceBefore).Cmp(gasCost), 0) + // post sgt balance check: sgt should be used as gas first + opts := &bind.CallOpts{Context: ctx} + postSgtBalance, err := sgt.SgtContract.BalanceOf(opts, testAddr) + require.NoError(t, err) + require.Equal(t, new(big.Int).Add(postSgtBalance, gasCost).Cmp(depositSgtValue), 0) + // balance invariant check + invariantBalanceCheck(t, ctx, sgt, testAddr, gasCost, txValue, depositSgtValue, depositL2Value, postSgtBalance, true) +} + +func partialSGTGasPaymentAndNonZeroTxValueWithSufficientNativeBalanceSuccess(t *testing.T, ctx context.Context, index int64, sgt *SgtHelper) { + depositSgtValue := big.NewInt(1000) + depositL2Value := big.NewInt(10000000000000) + txValue := big.NewInt(10000) + testAccount, testAddr, vaultBalanceBefore := setUpTestAccount(t, ctx, index, sgt, depositSgtValue, depositL2Value) + + // make a simple tx with the testAccount: transfer txValue from testAccount to dummyAddr + tx, err := sgt.transferNativeToken(t, ctx, testAccount, dummyAddr, txValue) + require.NoError(t, err) + receipt, err := wait.ForReceiptOK(ctx, sgt.L2Client, tx.Hash()) + require.NoError(t, err) + gasCost := calcGasFee(receipt) + vaultBalanceAfter := calcVaultBalance(t, ctx, sgt) + + // gasCost == vaultBalanceDiff check + require.Equal(t, new(big.Int).Sub(vaultBalanceAfter, vaultBalanceBefore).Cmp(gasCost), 0) + // post sgt balance check: sgt should be used as gas first and should be spent all + opts := &bind.CallOpts{Context: ctx} + postSgtBalance, err := sgt.SgtContract.BalanceOf(opts, testAddr) + require.NoError(t, err) + require.Equal(t, common.Big0.Cmp(postSgtBalance), 0) + // balance invariant check + invariantBalanceCheck(t, ctx, sgt, testAddr, gasCost, txValue, depositSgtValue, depositL2Value, postSgtBalance, true) +} + +func fullSGTInsufficientGasPaymentFail(t *testing.T, ctx context.Context, index int64, sgt *SgtHelper) { + depositSgtValue := big.NewInt(10000) + depositL2Value := big.NewInt(0) + txValue := big.NewInt(0) + testAccount, _, _ := setUpTestAccount(t, ctx, index, sgt, depositSgtValue, depositL2Value) + + // make a simple tx with the testAccount: transfer txValue from testAccount to dummyAddr + _, err := sgt.transferNativeToken(t, ctx, testAccount, dummyAddr, txValue) + require.Error(t, err) +} + +func fullNativeInsufficientGasPaymentFail(t *testing.T, ctx context.Context, index int64, sgt *SgtHelper) { + depositSgtValue := big.NewInt(0) + depositL2Value := big.NewInt(10000) + txValue := big.NewInt(0) + testAccount, _, _ := setUpTestAccount(t, ctx, index, sgt, depositSgtValue, depositL2Value) + + // make a simple tx with the testAccount: transfer txValue from testAccount to dummyAddr + _, err := sgt.transferNativeToken(t, ctx, testAccount, dummyAddr, txValue) + require.Error(t, err) +} + +func partialSGTInsufficientGasPaymentFail(t *testing.T, ctx context.Context, index int64, sgt *SgtHelper) { + depositSgtValue := big.NewInt(10000) + depositL2Value := big.NewInt(10000) + txValue := big.NewInt(0) + testAccount, _, _ := setUpTestAccount(t, ctx, index, sgt, depositSgtValue, depositL2Value) + + // make a simple tx with the testAccount: transfer txValue from testAccount to dummyAddr + _, err := sgt.transferNativeToken(t, ctx, testAccount, dummyAddr, txValue) + require.Error(t, err) +} + +func fullSGTGasPaymentAndNonZeroTxValueWithInsufficientNativeBalanceFail(t *testing.T, ctx context.Context, index int64, sgt *SgtHelper) { + depositSgtValue := big.NewInt(10000000000000) + depositL2Value := big.NewInt(10000) + txValue := big.NewInt(10001) + testAccount, _, _ := setUpTestAccount(t, ctx, index, sgt, depositSgtValue, depositL2Value) + + // make a simple tx with the testAccount: transfer txValue from testAccount to dummyAddr + _, err := sgt.transferNativeToken(t, ctx, testAccount, dummyAddr, txValue) + require.Error(t, err) +} + +func partialSGTGasPaymentAndNonZeroTxValueWithInsufficientNativeBalanceFail(t *testing.T, ctx context.Context, index int64, sgt *SgtHelper) { + depositSgtValue := big.NewInt(10000) + depositL2Value := big.NewInt(10000000000000) + txValue := new(big.Int).Sub(depositL2Value, depositSgtValue) + testAccount, _, _ := setUpTestAccount(t, ctx, index, sgt, depositSgtValue, depositL2Value) + + // make a simple tx with the testAccount: transfer txValue from testAccount to dummyAddr + _, err := sgt.transferNativeToken(t, ctx, testAccount, dummyAddr, txValue) + require.Error(t, err) +} + +func calcGasFee(receipt *types.Receipt) *big.Int { + // OPStackTxFee = L2ExecutionGasFee + L1DataFee + fees := new(big.Int).Mul(receipt.EffectiveGasPrice, big.NewInt(int64(receipt.GasUsed))) + fees = fees.Add(fees, receipt.L1Fee) + return fees +} + +func calcVaultBalance(t *testing.T, ctx context.Context, sgt *SgtHelper) *big.Int { + sequencerFee, err := sgt.L2Client.BalanceAt(ctx, seqVault, nil) + require.NoError(t, err) + baseFee, err := sgt.L2Client.BalanceAt(ctx, baseVault, nil) + require.NoError(t, err) + l1Fee, err := sgt.L2Client.BalanceAt(ctx, l1Vault, nil) + require.NoError(t, err) + return sequencerFee.Add(sequencerFee, baseFee.Add(baseFee, l1Fee)) +}
diff --git official optimism/op-e2e/l2blob/l2blob_test.go SWC optimism/op-e2e/l2blob/l2blob_test.go new file mode 100644 index 0000000000000000000000000000000000000000..fe183466743730904e44e541b15f0883801f6714 --- /dev/null +++ SWC optimism/op-e2e/l2blob/l2blob_test.go @@ -0,0 +1,175 @@ +package l2blob + +import ( + "bytes" + "context" + "crypto/ecdsa" + "errors" + "fmt" + "math/big" + mrand "math/rand" + "testing" + "time" + + op_e2e "github.com/ethereum-optimism/optimism/op-e2e" + "github.com/ethereum-optimism/optimism/op-e2e/e2eutils/wait" + "github.com/ethereum-optimism/optimism/op-e2e/system/e2esys" + "github.com/ethereum-optimism/optimism/op-node/config" + "github.com/ethereum-optimism/optimism/op-service/eth" + "github.com/ethereum-optimism/optimism/op-service/testutils" + "github.com/ethereum-optimism/optimism/op-service/txmgr" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/common/hexutil" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/crypto" + "github.com/ethereum/go-ethereum/ethclient" + "github.com/ethstorage/da-server/pkg/da" + "github.com/ethstorage/da-server/pkg/da/client" + "github.com/holiman/uint256" + "github.com/stretchr/testify/require" +) + +const ( + dacPort = 37777 +) + +var ( + ctx, _ = context.WithTimeout(context.Background(), 10*time.Second) + dacUrl = fmt.Sprintf("http://127.0.0.1:%d", dacPort) +) + +func TestSubmitTXWithBlobsFunctionSuccess(t *testing.T) { + op_e2e.InitParallel(t) + dacServer := startDACServer(t) + defer func() { + if err := dacServer.Stop(ctx); err != nil { + t.Errorf("Failed to stop DAC server: %v", err) + } + }() + + sys, l2Client := startSystemWithDAC(t) + t.Cleanup(sys.Close) + + var ( + toAddress = testutils.RandomAddress(mrand.New(mrand.NewSource(dacPort))) + blobs = make([]*eth.Blob, 3) + ) + for i := range blobs { + b := getRandBlob(t, int64(i)) + blobs[i] = &b + } + + tx, err := sendTransactionWithBlobs(t, ctx, l2Client, sys.TestAccount(0), toAddress, blobs) + require.NoError(t, err) + _, err = wait.ForReceiptOK(ctx, l2Client, tx.Hash()) + require.NoError(t, err) + + dblobs, err := downloadBlobs(dacUrl, tx.BlobHashes()) + require.NoError(t, err) + require.True(t, len(dblobs) == len(tx.BlobHashes()), "blobs downloaded is not equal to blob hashes") + + for i, blob := range dblobs { + require.True(t, len(blob) == eth.BlobSize, fmt.Sprintf("invalid downloaded blob, index %d; len %d", i, len(blob))) + require.True(t, bytes.Equal(blob, blobs[i][:]), fmt.Sprintf("blob content diff: %s vs %s", + common.Bytes2Hex(blob[:32]), common.Bytes2Hex(blobs[i][:32]))) + } +} + +func startSystemWithDAC(t *testing.T) (*e2esys.System, *ethclient.Client) { + cfg := e2esys.IsthmusSystemConfig(t, new(hexutil.Uint64)) + delete(cfg.Nodes, "verifier") + c, ok := cfg.Nodes["sequencer"] + require.True(t, ok, "sequencer is required") + c.DACConfig = &config.DACConfig{URLS: []string{dacUrl}} + c.Driver.SequencerEnabled = true + cfg.DeployConfig.L2GenesisBlobTimeOffset = new(hexutil.Uint64) + // Disable proposer creating fast games automatically - required games are manually created + cfg.DisableProposer = true + sys, err := cfg.Start(t) + require.Nil(t, err, "Error starting up system") + return sys, sys.NodeClient(e2esys.RoleSeq) +} + +func sendTransactionWithBlobs(t *testing.T, ctx context.Context, l2Client *ethclient.Client, sender *ecdsa.PrivateKey, + toAddr common.Address, blobs []*eth.Blob) (*types.Transaction, error) { + chainID, err := l2Client.ChainID(ctx) + require.NoError(t, err) + gasTipCap, gasFeeCap, blobFeeCap, err := gasPriceEstimator(ctx, l2Client) + require.NoError(t, err) + nonce, err := l2Client.NonceAt(ctx, crypto.PubkeyToAddress(sender.PublicKey), nil) + require.NoError(t, err) + sidecar, blobHashes, err := txmgr.MakeSidecar(blobs, false) + require.NoError(t, err) + tx := types.MustSignNewTx(sender, types.LatestSignerForChainID(chainID), &types.BlobTx{ + ChainID: uint256.NewInt(chainID.Uint64()), + Nonce: nonce, + GasFeeCap: uint256.NewInt(gasFeeCap.Uint64()), + GasTipCap: uint256.NewInt(gasTipCap.Uint64()), + Gas: uint64(22000), + To: toAddr, + Value: uint256.NewInt(0), + BlobFeeCap: uint256.NewInt(blobFeeCap.Uint64()), + BlobHashes: blobHashes, + Sidecar: sidecar, + }) + ctx, cancel := context.WithTimeout(ctx, 5*time.Second) + defer cancel() + err = l2Client.SendTransaction(ctx, tx) + return tx, err +} + +func gasPriceEstimator(ctx context.Context, client *ethclient.Client) (*big.Int, *big.Int, *big.Int, error) { + tip, err := client.SuggestGasTipCap(ctx) + if err != nil { + return nil, nil, nil, err + } + + head, err := client.HeaderByNumber(ctx, nil) + if err != nil { + return nil, nil, nil, err + } + if head.BaseFee == nil { + return nil, nil, nil, errors.New("head BaseFee is nil") + } + + var blobFee *big.Int + if head.ExcessBlobGas != nil { + blobFee = eth.CalcBlobFeeCancun(*head.ExcessBlobGas) + } + + gasFeeCap := new(big.Int).Add( + tip, + new(big.Int).Mul(head.BaseFee, big.NewInt(2)), + ) + return tip, gasFeeCap, blobFee, nil +} + +func getRandBlob(t *testing.T, seed int64) eth.Blob { + r := mrand.New(mrand.NewSource(seed)) + bigData := eth.Data(make([]byte, eth.MaxBlobDataSize)) + _, err := r.Read(bigData) + require.NoError(t, err) + var b eth.Blob + err = b.FromData(bigData) + require.NoError(t, err) + return b +} + +func startDACServer(t *testing.T) *da.Server { + config := da.Config{ + SequencerIP: "127.0.0.1", + ListenAddr: fmt.Sprintf("0.0.0.0:%d", dacPort), + StorePath: t.TempDir(), + } + server := da.NewServer(&config) + err := server.Start(ctx) + require.NoError(t, err) + + return server +} + +func downloadBlobs(dacUrl string, blobHashes []common.Hash) (blobs []hexutil.Bytes, err error) { + client := client.New([]string{dacUrl}) + blobs, err = client.GetBlobs(ctx, blobHashes) + return +}

This script is used to run the overall local test for the OP-stack. For details, please refer to here.

diff --git official optimism/.mise-tasks/dev-test.sh SWC optimism/.mise-tasks/dev-test.sh new file mode 100755 index 0000000000000000000000000000000000000000..0445739d61852784a24e4050d695f2be82a457a0 --- /dev/null +++ SWC optimism/.mise-tasks/dev-test.sh @@ -0,0 +1,100 @@ +#!/usr/bin/env bash + +#MISE description="Developers' local tests" +#MISE alias="dt" + +set -e +SECONDS=0 + +error_handler() { + echo "Execution time: ${SECONDS} seconds" + exit 1 +} + +trap 'error_handler' ERR + +# Environment tests + +forge --version + +for var in SEPOLIA_RPC_URL MAINNET_RPC_URL; do + if [ -z "${!var}" ]; then + echo "Error: $var is not set." + exit 1 + fi +done + +STATUS=$(kurtosis engine status) +if echo "$STATUS" | grep -q "1.4.3"; then + echo "Kurtosis engine is running." +else + echo "The Kurtosis engine is not running, or there is a version mismatch." + exit 1 +fi + +# Runs semgrep tests on the entire monorepo + +just semgrep +just semgrep-test + +# Solidity + +cd packages/contracts-bedrock +just lint-check +just pre-pr +just test + +# Go + +cd ../.. +make lint-go +make build-go + +cd op-program && make op-program-client && cd .. +cd cannon && make elf && cd .. +cd op-e2e && make pre-test && cd .. + +make devnet-allocs + +export ENABLE_KURTOSIS=true +export OP_E2E_CANNON_ENABLED="false" +export OP_E2E_SKIP_SLOW_TEST=true +export OP_E2E_USE_HTTP=true +export ENABLE_ANVIL=true + +# Note: not all packages are tested. +# For example the test `TestFinalization` in `op-alt-da` package fails even in upstream. +packages=( + op-batcher + op-chain-ops + op-node + op-proposer + op-challenger + op-dispute-mon + op-conductor + op-program + op-service + op-supervisor + op-deployer + op-e2e/system + op-e2e/e2eutils + op-e2e/opgeth + op-e2e/interop + op-e2e/actions + op-e2e/faultproofs + op-e2e/l2blob + op-e2e/inbox + op-e2e/sgt + packages/contracts-bedrock/scripts/checks +) +formatted_packages="" +for package in "${packages[@]}"; do + formatted_packages="$formatted_packages ./$package/..." +done + +gotestsum --no-summary=skipped,output \ + --packages="$formatted_packages" \ + --format=short-verbose \ + --rerun-fails=2 + +echo "Execution time: $((SECONDS / 60)) minute(s) and $((SECONDS % 60)) second(s)" \ No newline at end of file

This script is used to upgrade the SoulGasToken contract on L2. It allows for the deployment of a new version of the contract, ensuring that users can benefit from the latest features and improvements.

diff --git official optimism/packages/contracts-bedrock/scripts/deploy/UpgradeSoulGasToken.s.sol SWC optimism/packages/contracts-bedrock/scripts/deploy/UpgradeSoulGasToken.s.sol new file mode 100644 index 0000000000000000000000000000000000000000..3850cfecb31d7b75fd32ee8cb474d6314cc06117 --- /dev/null +++ SWC optimism/packages/contracts-bedrock/scripts/deploy/UpgradeSoulGasToken.s.sol @@ -0,0 +1,94 @@ +// SPDX-License-Identifier: MIT +pragma solidity ^0.8.15; + +// Forge +import { Script } from "forge-std/Script.sol"; + +// Scripts +import { DeployUtils } from "scripts/libraries/DeployUtils.sol"; + +// Interfaces +import { IProxyAdmin } from "interfaces/universal/IProxyAdmin.sol"; +import { IStorageSetter } from "interfaces/universal/IStorageSetter.sol"; +import { ISoulGasToken } from "interfaces/L2/ISoulGasToken.sol"; + +/// @title UpgradeSoulGasToken +contract UpgradeSoulGasToken is Script { + IProxyAdmin constant proxyAdmin = IProxyAdmin(0x4200000000000000000000000000000000000018); + ISoulGasToken constant soulGasToken = ISoulGasToken(0x4200000000000000000000000000000000000800); + + function run(address _storageSetter) public { + address sgtOwner = getSGTOwner(); + preCheck(); + + vm.startBroadcast(); + upgradeSoulGasTokenImpl(_storageSetter); + vm.stopBroadcast(); + postCheck(sgtOwner); + } + + function upgradeSoulGasTokenImpl(address _storageSetter) internal { + if (_storageSetter == address(0)) { + _storageSetter = DeployUtils.create1({ + _name: "StorageSetter", + _args: DeployUtils.encodeConstructor(abi.encodeCall(IStorageSetter.__constructor__, ())) + }); + } + + address sgtOwner = ISoulGasToken(soulGasToken).owner(); + + address impl = proxyAdmin.getProxyImplementation(address(soulGasToken)); + + bytes memory data; + data = encodeStorageSetterZeroOutInitializedSlot(); + upgradeAndCall(proxyAdmin, address(soulGasToken), _storageSetter, data); + data = encodeSoulGasTokenInitializer(sgtOwner); + upgradeAndCall(proxyAdmin, address(soulGasToken), impl, data); + } + + function encodeStorageSetterZeroOutInitializedSlot() internal pure returns (bytes memory) { + return abi.encodeCall(IStorageSetter.setBytes32, (0, 0)); + } + + function encodeSoulGasTokenInitializer(address _sgtOwner) internal view virtual returns (bytes memory) { + return abi.encodeCall(ISoulGasToken.initialize, ("SoulQKC", "SoulQKC", _sgtOwner)); + } + + /// @notice Makes an external call to the target to initialize the proxy with the specified data. + /// First performs safety checks to ensure the target, implementation, and proxy admin are valid. + function upgradeAndCall( + IProxyAdmin _proxyAdmin, + address _target, + address _implementation, + bytes memory _data + ) + internal + { + DeployUtils.assertValidContractAddress(address(_proxyAdmin)); + DeployUtils.assertValidContractAddress(_target); + DeployUtils.assertValidContractAddress(_implementation); + + _proxyAdmin.upgradeAndCall(payable(address(_target)), _implementation, _data); + } + + function getSGTOwner() public view returns (address) { + return soulGasToken.owner(); + } + + function preCheck() public view { + address sgtAdmin = proxyAdmin.getProxyAdmin(payable(address(soulGasToken))); + require(sgtAdmin == address(proxyAdmin), "UpgradeSoulGasToken: admin not match"); + } + + function postCheck(address _sgtOwner) public view { + require( + keccak256(abi.encodePacked(soulGasToken.name())) == keccak256("SoulQKC"), + "UpgradeSoulGasToken: name not match" + ); + require( + keccak256(abi.encodePacked(soulGasToken.symbol())) == keccak256("SoulQKC"), + "UpgradeSoulGasToken: symbol not match" + ); + require(soulGasToken.owner() == _sgtOwner, "UpgradeSoulGasToken: owner not match"); + } +}

This script is used to update the AnchorStateRegistry with a new anchor state when switching from permissioned FDG to permissionless FDG on L1. Fore details, please refer to here.

diff --git official optimism/packages/contracts-bedrock/scripts/deploy/UpgradeAnchorStateRegistry.s.sol SWC optimism/packages/contracts-bedrock/scripts/deploy/UpgradeAnchorStateRegistry.s.sol new file mode 100644 index 0000000000000000000000000000000000000000..3bfad8ac0af169c0fcafa549558f4bbfa111be5c --- /dev/null +++ SWC optimism/packages/contracts-bedrock/scripts/deploy/UpgradeAnchorStateRegistry.s.sol @@ -0,0 +1,161 @@ +// SPDX-License-Identifier: MIT +pragma solidity ^0.8.15; + +// Forge +import { Script } from "forge-std/Script.sol"; +import { console } from "forge-std/console.sol"; + +// Scripts +import { DeployUtils } from "scripts/libraries/DeployUtils.sol"; + +// Libraries +import { GameType, Hash, Proposal } from "src/dispute/lib/Types.sol"; +// Contracts +import { IStorageSetter } from "interfaces/universal/IStorageSetter.sol"; +// Interfaces +import { IAnchorStateRegistry } from "interfaces/dispute/IAnchorStateRegistry.sol"; +import { IDisputeGameFactory } from "interfaces/dispute/IDisputeGameFactory.sol"; +import { IAnchorStateRegistry } from "interfaces/dispute/IAnchorStateRegistry.sol"; +import { IProxyAdmin } from "interfaces/universal/IProxyAdmin.sol"; +import { ISystemConfig } from "interfaces/L1/ISystemConfig.sol"; + +/// @title UpgradeAnchorStateRegistry +contract UpgradeAnchorStateRegistry is Script { + function run( + address _disputeGameFactoryProxy, + address _opChainProxyAdmin, + address _anchorStateRegistryProxy, + address _systemConfig, + address _storageSetter, + uint32 _type, + Proposal memory _startingAnchorRoot + ) + public + { + console.log("_disputeGameFactoryProxy: %s", _disputeGameFactoryProxy); + console.log("_opChainProxyAdmin: %s", _opChainProxyAdmin); + console.log("_anchorStateRegistryProxy: %s", _anchorStateRegistryProxy); + console.log("_systemConfig: %s", _systemConfig); + console.log("_storageSetter: %s", _storageSetter); + console.log("_type: %s", _type); + console.log("_l2BlockNumber: %s", _startingAnchorRoot.l2SequenceNumber); + console.log("_outputRoot: %s", bytes32ToHex(Hash.unwrap(_startingAnchorRoot.root))); + + vm.startBroadcast(); + upgradeAnchorStateRegistryImpl( + IDisputeGameFactory(_disputeGameFactoryProxy), + IProxyAdmin(_opChainProxyAdmin), + IAnchorStateRegistry(_anchorStateRegistryProxy), + ISystemConfig(_systemConfig), + _storageSetter, + GameType.wrap(_type), + _startingAnchorRoot + ); + vm.stopBroadcast(); + checkOutput(IAnchorStateRegistry(_anchorStateRegistryProxy), GameType.wrap(_type), _startingAnchorRoot); + } + + function upgradeAnchorStateRegistryImpl( + IDisputeGameFactory _disputeGameFactoryProxy, + IProxyAdmin _opChainProxyAdmin, + IAnchorStateRegistry _anchorStateRegistryProxy, + ISystemConfig _systemConfig, + address _storageSetter, + GameType _type, + Proposal memory _startingAnchorRoot + ) + internal + { + address anchorStateRegistryImpl = DeployUtils.create1({ + _name: "AnchorStateRegistry", + _args: DeployUtils.encodeConstructor( + abi.encodeCall( + IAnchorStateRegistry.__constructor__, (_anchorStateRegistryProxy.disputeGameFinalityDelaySeconds()) + ) + ) + }); + + if (_storageSetter == address(0)) { + _storageSetter = DeployUtils.create1({ + _name: "StorageSetter", + _args: DeployUtils.encodeConstructor(abi.encodeCall(IStorageSetter.__constructor__, ())) + }); + } + + bytes memory data; + data = encodeStorageSetterZeroOutInitializedSlot(); + upgradeAndCall(_opChainProxyAdmin, address(_anchorStateRegistryProxy), _storageSetter, data); + data = encodeAnchorStateRegistryInitializer(_disputeGameFactoryProxy, _type, _startingAnchorRoot, _systemConfig); + upgradeAndCall(_opChainProxyAdmin, address(_anchorStateRegistryProxy), anchorStateRegistryImpl, data); + } + + function encodeStorageSetterZeroOutInitializedSlot() internal pure returns (bytes memory) { + return abi.encodeCall(IStorageSetter.setBytes32, (0, 0)); + } + + function encodeAnchorStateRegistryInitializer( + IDisputeGameFactory _disputeGameFactory, + GameType _type, + Proposal memory _startingAnchorRoot, + ISystemConfig _systemConfig + ) + internal + view + virtual + returns (bytes memory) + { + return abi.encodeCall( + IAnchorStateRegistry.initialize, (_systemConfig, _disputeGameFactory, _startingAnchorRoot, _type) + ); + } + + /// @notice Makes an external call to the target to initialize the proxy with the specified data. + /// First performs safety checks to ensure the target, implementation, and proxy admin are valid. + function upgradeAndCall( + IProxyAdmin _proxyAdmin, + address _target, + address _implementation, + bytes memory _data + ) + internal + { + DeployUtils.assertValidContractAddress(address(_proxyAdmin)); + DeployUtils.assertValidContractAddress(_target); + DeployUtils.assertValidContractAddress(_implementation); + + _proxyAdmin.upgradeAndCall(payable(address(_target)), _implementation, _data); + } + + function checkOutput( + IAnchorStateRegistry _anchorStateRegistryProxy, + GameType _type, + Proposal memory _startingAnchorRoot + ) + public + view + { + (Hash root, uint256 l2BlockNumber) = IAnchorStateRegistry(_anchorStateRegistryProxy).anchors(_type); + require( + Hash.unwrap(root) == Hash.unwrap(_startingAnchorRoot.root), + "UpgradeAnchorStateRegistryOutput: root mismatch" + ); + require( + l2BlockNumber == _startingAnchorRoot.l2SequenceNumber, + "UpgradeAnchorStateRegistryOutput: l2BlockNumber mismatch" + ); + } + + function bytes32ToHex(bytes32 _data) internal pure returns (string memory) { + bytes memory result = new bytes(64); + for (uint256 i = 0; i < 32; i++) { + uint8 byteValue = uint8(_data[i]); + result[i * 2] = toHexChar(byteValue / 16); + result[i * 2 + 1] = toHexChar(byteValue % 16); + } + return string(result); + } + + function toHexChar(uint8 _b) internal pure returns (bytes1) { + return _b < 10 ? bytes1(_b + 0x30) : bytes1(_b + 0x57); + } +}
diff --git official optimism/.gitmodules SWC optimism/.gitmodules index 1591d79b2a8af7c1865e8ae49e8f10709b1e0462..c80d071a1d4d7a9ea3ef5d569d0308da88c309eb 100644 --- official optimism/.gitmodules +++ SWC optimism/.gitmodules @@ -29,6 +29,13 @@ url = https://github.com/OpenZeppelin/openzeppelin-contracts [submodule "packages/contracts-bedrock/lib/solady-v0.0.245"] path = packages/contracts-bedrock/lib/solady-v0.0.245 url = https://github.com/vectorized/solady +[submodule "da-server"] + path = da-server + url = https://github.com/ethstorage/da-server [submodule "packages/contracts-bedrock/lib/superchain-registry"] path = packages/contracts-bedrock/lib/superchain-registry url = https://github.com/ethereum-optimism/superchain-registry +[submodule "op-geth"] + path = op-geth + url = https://github.com/QuarkChain/op-geth + branch = op-es
diff --git official optimism/.semgrep/rules/sol-rules.yaml SWC optimism/.semgrep/rules/sol-rules.yaml index bcd5616170a5c9995d0fd027fcce0bb25cbf1097..31cacaf334ee067f9ead4856cd636db7787aee38 100644 --- official optimism/.semgrep/rules/sol-rules.yaml +++ SWC optimism/.semgrep/rules/sol-rules.yaml @@ -324,6 +324,7 @@ - packages/contracts-bedrock/src/L1/OptimismPortalInterop.sol - packages/contracts-bedrock/src/L2/FeeVault.sol - packages/contracts-bedrock/src/L2/OptimismMintableERC721.sol - packages/contracts-bedrock/src/L2/OptimismMintableERC721Factory.sol + - packages/contracts-bedrock/src/L2/SoulGasToken.sol - packages/contracts-bedrock/src/cannon/MIPS64.sol - packages/contracts-bedrock/src/cannon/PreimageOracle.sol - packages/contracts-bedrock/src/dispute/AnchorStateRegistry.sol @@ -338,7 +339,6 @@ - packages/contracts-bedrock/src/governance/MintManager.sol - packages/contracts-bedrock/src/periphery/TransferOnion.sol - packages/contracts-bedrock/src/periphery/faucet/Faucet.sol - packages/contracts-bedrock/src/periphery/faucet/authmodules/AdminFaucetAuthModule.sol - - packages/contracts-bedrock/src/safe/DeputyGuardianModule.sol - packages/contracts-bedrock/src/safe/DeputyPauseModule.sol - packages/contracts-bedrock/src/safe/LivenessGuard.sol - packages/contracts-bedrock/src/safe/LivenessModule.sol
diff --git official optimism/Makefile SWC optimism/Makefile index a05239f14c354e9db134beef567308a2ee751e6f..9b83651f98698bf83ca66ede02fd9d85392224ee 100644 --- official optimism/Makefile +++ SWC optimism/Makefile @@ -257,11 +257,15 @@ export ENABLE_ANVIL=true && \ export PARALLEL=$$(nproc 2>/dev/null || sysctl -n hw.ncpu 2>/dev/null || echo 4) endef   +# QKC: The following are removed to use environment variables for RPC URLs in CircleCI +# export SEPOLIA_RPC_URL="https://ci-sepolia-l1-archive.optimism.io" && \ +# export MAINNET_RPC_URL="https://ci-mainnet-l1-archive.optimism.io" && \ + # Additional CI-specific environment variables define CI_ENV_VARS export OP_TESTLOG_FILE_LOGGER_OUTDIR=$$(realpath ./tmp/testlogs) && \ -export SEPOLIA_RPC_URL="https://ci-sepolia-l1-archive.optimism.io" && \ -export MAINNET_RPC_URL="https://ci-mainnet-l1-archive.optimism.io" && \ +if [ -z "$$SEPOLIA_RPC_URL" ]; then echo "ERROR: SEPOLIA_RPC_URL is not set"; exit 1; fi && \ +if [ -z "$$MAINNET_RPC_URL" ]; then echo "ERROR: MAINNET_RPC_URL is not set"; exit 1; fi && \ export NAT_INTEROP_LOADTEST_TARGET=10 && \ export NAT_INTEROP_LOADTEST_TIMEOUT=30s endef
diff --git official optimism/devnet-sdk/system/txprocessor.go SWC optimism/devnet-sdk/system/txprocessor.go index 1279aadffc1b6e289c4a6d57cc5bb6a0abcf95e4..0d5f9c40a5feabc18a42cb6b8d8044025feb86e3 100644 --- official optimism/devnet-sdk/system/txprocessor.go +++ SWC optimism/devnet-sdk/system/txprocessor.go @@ -45,7 +45,7 @@ var signer types.Signer switch tx.Type() { case types.SetCodeTxType: - signer = types.NewIsthmusSigner(p.chainID) + signer = types.NewIsthmusSigner(p.chainID, false) case types.DynamicFeeTxType: signer = types.NewLondonSigner(p.chainID) case types.AccessListTxType:
diff --git official optimism/devnet-sdk/system/wallet.go SWC optimism/devnet-sdk/system/wallet.go index 147aea26d470f16bde1c627e79b6cf8c7737faa3..1851fd3893a98320b21b7a60fd7d2c18777eea2f 100644 --- official optimism/devnet-sdk/system/wallet.go +++ SWC optimism/devnet-sdk/system/wallet.go @@ -284,7 +284,7 @@ var signer coreTypes.Signer switch tx.Type() { case coreTypes.SetCodeTxType: - signer = coreTypes.NewIsthmusSigner(w.chain.ID()) + signer = coreTypes.NewIsthmusSigner(w.chain.ID(), false) case coreTypes.DynamicFeeTxType: signer = coreTypes.NewLondonSigner(w.chain.ID()) case coreTypes.AccessListTxType:
diff --git official optimism/kurtosis-devnet/optimism-package-trampoline/kurtosis.yml SWC optimism/kurtosis-devnet/optimism-package-trampoline/kurtosis.yml index 06e9db8afd93d243f1a50e36ebe254b20846c6d8..58f2cc557a9f554f3ad178f520e17666fc11e42e 100644 --- official optimism/kurtosis-devnet/optimism-package-trampoline/kurtosis.yml +++ SWC optimism/kurtosis-devnet/optimism-package-trampoline/kurtosis.yml @@ -2,7 +2,7 @@ name: github.com/ethereum-optimism/optimism/kurtosis-devnet/optimism-package-trampoline description: |- A trampoline package for optimism-package. This one is reproducible, due to the replace directives below. replace: - github.com/ethpandaops/optimism-package: github.com/ethpandaops/optimism-package@89e0b8cacab9f7e9f74d53b72d4870092825d577 + github.com/QuarkChain/optimism-package: github.com/QuarkChain/optimism-package@9553f5f6a70068139d1709b77665314c5ebbb78e github.com/ethpandaops/ethereum-package: github.com/ethpandaops/ethereum-package@83830d44823767af65eda7dfe6b26c87c536c4cf github.com/kurtosis-tech/prometheus-package: github.com/kurtosis-tech/prometheus-package@637c9dea933be18e47f96cadc0d9bb0e3a5aa9d6 # v1.0.0 github.com/kurtosis-tech/postgres-package: github.com/kurtosis-tech/postgres-package@9cbdde2c55e8d1656deb87821465a2ad244d8b33 # v1.0.0
diff --git official optimism/kurtosis-devnet/optimism-package-trampoline/main.star SWC optimism/kurtosis-devnet/optimism-package-trampoline/main.star index d1b2d34b0331ff62fd68ac85443bc21824454ab5..626771cda5d112e1b57923dc5968893579bad0d2 100644 --- official optimism/kurtosis-devnet/optimism-package-trampoline/main.star +++ SWC optimism/kurtosis-devnet/optimism-package-trampoline/main.star @@ -1,4 +1,4 @@ -optimism_package = import_module("github.com/ethpandaops/optimism-package/main.star") +optimism_package = import_module("github.com/QuarkChain/optimism-package/main.star")   def run(plan, args): # just delegate to optimism-package
diff --git official optimism/kurtosis-devnet/simple.yaml SWC optimism/kurtosis-devnet/simple.yaml index 351ac0230ac5c18ee3deed94d7c90171caa09747..6c7836f62b986af71ea89c77a4bb76c2e3f79d7c 100644 --- official optimism/kurtosis-devnet/simple.yaml +++ SWC optimism/kurtosis-devnet/simple.yaml @@ -22,6 +22,7 @@ max_mem: 0 cl: type: op-node image: {{ localDockerImage "op-node" }} + is_qkc: true log_level: "" extra_env_vars: {} extra_labels: {} @@ -66,6 +67,10 @@ l1_artifacts_locator: {{ localContractArtifacts "l1" }} l2_artifacts_locator: {{ localContractArtifacts "l2" }} overrides: faultGameAbsolutePrestate: {{ localPrestate.Hashes.prestate_mt64 }} + deploySoulGasToken: true + soulGasTokenTimeOffset: "0x0" + isSoulBackedByNative: true + l2GenesisBlobTimeOffset: "0x0" global_log_level: "info" global_node_selectors: {} global_tolerations: []
diff --git official optimism/mise.toml SWC optimism/mise.toml index be52c50066e561e9d9325e74c38a2c532fda8111..7d75b84e95d9831cbbcd25e7f9ddd640755a251a 100644 --- official optimism/mise.toml +++ SWC optimism/mise.toml @@ -37,7 +37,7 @@ # Other dependencies codecov-uploader = "0.8.0" goreleaser-pro = "2.11.2" -kurtosis = "1.8.1" +kurtosis = "1.11.1" op-acceptor = "op-acceptor/v3.6.1"   # Fake dependencies
diff --git official optimism/op-acceptance-tests/tests/fjord/check_scripts_test.go SWC optimism/op-acceptance-tests/tests/fjord/check_scripts_test.go index 2572332b7df7cf5c6fa5ce767240c08619eb8552..a9fe15fe1a126aa1d0b14955b754500ab76cdf26 100644 --- official optimism/op-acceptance-tests/tests/fjord/check_scripts_test.go +++ SWC optimism/op-acceptance-tests/tests/fjord/check_scripts_test.go @@ -162,12 +162,15 @@ gpoFee, err := dsl.ReadGasPriceOracleL1FeeAt(ctx, l2Client, gasPriceOracle, txUnsigned, receipt.BlockHash) require.NoError(err)   + baseFeeScalarMultiplierBig := common.Big1 + blobBaseFeeScalarMultiplierBig := common.Big1 + fastLzSize := uint64(types.FlzCompressLen(txUnsigned) + 68) - gethGPOFee, err := dsl.CalculateFjordL1Cost(ctx, l2Client, types.RollupCostData{FastLzSize: fastLzSize}, receipt.BlockHash) + gethGPOFee, err := dsl.CalculateFjordL1Cost(ctx, l2Client, types.RollupCostData{FastLzSize: fastLzSize}, receipt.BlockHash, baseFeeScalarMultiplierBig, blobBaseFeeScalarMultiplierBig) require.NoError(err) require.Equalf(gethGPOFee.Uint64(), gpoFee.Uint64(), "GPO L1 fee mismatch (expected=%d actual=%d)", gethGPOFee.Uint64(), gpoFee.Uint64())   - expectedFee, err := dsl.CalculateFjordL1Cost(ctx, l2Client, signedTx.RollupCostData(), receipt.BlockHash) + expectedFee, err := dsl.CalculateFjordL1Cost(ctx, l2Client, signedTx.RollupCostData(), receipt.BlockHash, baseFeeScalarMultiplierBig, blobBaseFeeScalarMultiplierBig) require.NoError(err) require.NotNil(receipt.L1Fee) dsl.ValidateL1FeeMatches(t, expectedFee, receipt.L1Fee) @@ -176,7 +179,7 @@ upperBound, err := dsl.ReadGasPriceOracleL1FeeUpperBoundAt(ctx, l2Client, gasPriceOracle, len(txUnsigned), receipt.BlockHash) require.NoError(err) txLenGPO := len(txUnsigned) + 68 flzUpperBound := uint64(txLenGPO + txLenGPO/255 + 16) - upperBoundCost, err := dsl.CalculateFjordL1Cost(ctx, l2Client, types.RollupCostData{FastLzSize: flzUpperBound}, receipt.BlockHash) + upperBoundCost, err := dsl.CalculateFjordL1Cost(ctx, l2Client, types.RollupCostData{FastLzSize: flzUpperBound}, receipt.BlockHash, baseFeeScalarMultiplierBig, blobBaseFeeScalarMultiplierBig) require.NoError(err) require.Equalf(upperBoundCost.Uint64(), upperBound.Uint64(), "GPO L1 upper bound mismatch (expected=%d actual=%d)", upperBoundCost.Uint64(), upperBound.Uint64())
diff --git official optimism/op-chain-ops/cmd/check-derivation/main.go SWC optimism/op-chain-ops/cmd/check-derivation/main.go index 026e821c01effaf7d4764c44636590b7a836927b..6ce6a3ce9801ca1d5ac8b8bebcdf9da5aaebb52c 100644 --- official optimism/op-chain-ops/cmd/check-derivation/main.go +++ SWC optimism/op-chain-ops/cmd/check-derivation/main.go @@ -302,7 +302,7 @@ }   tx := types.NewTx(txData)   - signer := types.NewIsthmusSigner(chainId) + signer := types.NewIsthmusSigner(chainId, false) if !protected { if txType == types.LegacyTxType { signer = types.HomesteadSigner{}
diff --git official optimism/op-chain-ops/cmd/check-ecotone/main.go SWC optimism/op-chain-ops/cmd/check-ecotone/main.go index 10d5f72e7ad74e7fe4d0330d1b0f2e7b019fc587..5e52cc70f798b7a1eb1de91fa7003040b80473e8 100644 --- official optimism/op-chain-ops/cmd/check-ecotone/main.go +++ SWC optimism/op-chain-ops/cmd/check-ecotone/main.go @@ -786,7 +786,7 @@ Value: big.NewInt(3 * params.GWei), Data: []byte("hello"), AccessList: nil, } - tx, err := types.SignNewTx(env.key, types.NewIsthmusSigner(txData.ChainID), txData) + tx, err := types.SignNewTx(env.key, types.NewIsthmusSigner(txData.ChainID, false), txData) if err != nil { return fmt.Errorf("failed to sign test tx: %w", err) }
diff --git official optimism/op-chain-ops/cmd/check-fjord/checks/checks.go SWC optimism/op-chain-ops/cmd/check-fjord/checks/checks.go index a7063bbde9436164e6bf34267d48dd7d192c1f9e..6220067cf57225c89cf81381d1c6fab968f0309b 100644 --- official optimism/op-chain-ops/cmd/check-fjord/checks/checks.go +++ SWC optimism/op-chain-ops/cmd/check-fjord/checks/checks.go @@ -283,7 +283,8 @@ costFunc := types.NewL1CostFuncFjord( l1BaseFee, blobBaseFee, new(big.Int).SetUint64(uint64(baseFeeScalar)), - new(big.Int).SetUint64(uint64(blobBaseFeeScalar))) + new(big.Int).SetUint64(uint64(blobBaseFeeScalar)), + big.NewInt(1), big.NewInt(1))   fee, _ := costFunc(types.RollupCostData{FastLzSize: fastLzSize}) return fee, nil
diff --git official optimism/op-chain-ops/interopgen/deploy.go SWC optimism/op-chain-ops/interopgen/deploy.go index d18d78a29c23675237a894c89eee781adb129738..e681ce3438359be3cc43fb8b7a46fa7a8323aa43 100644 --- official optimism/op-chain-ops/interopgen/deploy.go +++ SWC optimism/op-chain-ops/interopgen/deploy.go @@ -334,6 +334,8 @@ Fork: big.NewInt(cfg.SolidityForkNumber(1)), DeployCrossL2Inbox: multichainDepSet, EnableGovernance: cfg.EnableGovernance, FundDevAccounts: cfg.FundDevAccounts, + DeploySoulGasToken: cfg.DeploySoulGasToken, + IsSoulBackedByNative: cfg.IsSoulBackedByNative, }); err != nil { return fmt.Errorf("failed L2 genesis: %w", err) }
diff --git official optimism/op-deployer/pkg/deployer/opcm/l2genesis.go SWC optimism/op-deployer/pkg/deployer/opcm/l2genesis.go index 3f91196cc85fca54a81f0c670b275255be5b33b2..1ea2e41cb0a6a63c3b6462b81cd135977ecf8f33 100644 --- official optimism/op-deployer/pkg/deployer/opcm/l2genesis.go +++ SWC optimism/op-deployer/pkg/deployer/opcm/l2genesis.go @@ -28,6 +28,8 @@ Fork *big.Int DeployCrossL2Inbox bool EnableGovernance bool FundDevAccounts bool + DeploySoulGasToken bool + IsSoulBackedByNative bool }   type L2GenesisScript script.DeployScriptWithoutOutput[L2GenesisInput]
diff --git official optimism/op-deployer/pkg/deployer/pipeline/l2genesis.go SWC optimism/op-deployer/pkg/deployer/pipeline/l2genesis.go index bb3f514358bcef96f3613a093120bc0936aa8a79..c977d0a903354654c90e3b68ff4088785e528cb0 100644 --- official optimism/op-deployer/pkg/deployer/pipeline/l2genesis.go +++ SWC optimism/op-deployer/pkg/deployer/pipeline/l2genesis.go @@ -31,6 +31,8 @@ L1FeeVaultWithdrawalNetwork genesis.WithdrawalNetwork `json:"l1FeeVaultWithdrawalNetwork"` SequencerFeeVaultWithdrawalNetwork genesis.WithdrawalNetwork `json:"sequencerFeeVaultWithdrawalNetwork"` EnableGovernance bool `json:"enableGovernance"` GovernanceTokenOwner common.Address `json:"governanceTokenOwner"` + DeploySoulGasToken bool `json:"deploySoulGasToken"` + IsSoulBackedByNative bool `json:"isSoulBackedByNative"` }   func GenerateL2Genesis(pEnv *Env, intent *state.Intent, bundle ArtifactsBundle, st *state.State, chainID common.Hash) error { @@ -94,6 +96,8 @@ Fork: big.NewInt(schedule.SolidityForkNumber(1)), DeployCrossL2Inbox: len(intent.Chains) > 1, EnableGovernance: overrides.EnableGovernance, FundDevAccounts: overrides.FundDevAccounts, + DeploySoulGasToken: overrides.DeploySoulGasToken, + IsSoulBackedByNative: overrides.IsSoulBackedByNative, }); err != nil { return fmt.Errorf("failed to call L2Genesis script: %w", err) }
diff --git official optimism/op-devstack/dsl/fjord_fees.go SWC optimism/op-devstack/dsl/fjord_fees.go index b27566634c71c1416ebd0137c6d7aba83b2be178..f72a33539911e0dea6c500baa43f42fa8782f9a7 100644 --- official optimism/op-devstack/dsl/fjord_fees.go +++ SWC optimism/op-devstack/dsl/fjord_fees.go @@ -196,7 +196,8 @@ ff.t.Logf("L1 fee is nil in receipt, skipping L1 fee validation") return fastLzSizeSigned, nil }   - expectedFee, err := CalculateFjordL1Cost(ff.ctx, client, signedTx.RollupCostData(), receipt.BlockHash) + scalar1, scalar2 := ff.l2Network.L1ScalarMultiplierConfig() + expectedFee, err := CalculateFjordL1Cost(ff.ctx, client, signedTx.RollupCostData(), receipt.BlockHash, scalar1, scalar2) ff.require.NoError(err, "should calculate L1 fee")   ff.require.Equalf(expectedFee, receiptL1Fee, "Calculated L1 fee should match receipt L1 fee (expected=%s actual=%s)", expectedFee.String(), receiptL1Fee.String()) @@ -329,7 +330,7 @@ require.Equalf(calculatedFee.Uint64(), receiptFee.Uint64(), "L1 fee mismatch (expected=%d actual=%d)", calculatedFee.Uint64(), receiptFee.Uint64()) }   // CalculateFjordL1Cost calculates L1 cost using Fjord formula with block-specific L1 state -func CalculateFjordL1Cost(ctx context.Context, client apis.EthClient, rollupCostData types.RollupCostData, blockHash common.Hash) (*big.Int, error) { +func CalculateFjordL1Cost(ctx context.Context, client apis.EthClient, rollupCostData types.RollupCostData, blockHash common.Hash, l1BaseFeeScalarMultiplier, l1BlobBaseFeeScalarMultiplier *big.Int) (*big.Int, error) { l1Block := bindings.NewL1Block( bindings.WithClient(client), bindings.WithTo(predeploys.L1BlockAddr), @@ -362,7 +363,9 @@ costFunc := types.NewL1CostFuncFjord( l1BaseFee, blobBaseFee, new(big.Int).SetUint64(uint64(baseFeeScalar)), - new(big.Int).SetUint64(uint64(blobBaseFeeScalar))) + new(big.Int).SetUint64(uint64(blobBaseFeeScalar)), + l1BaseFeeScalarMultiplier, + l1BlobBaseFeeScalarMultiplier)   fee, _ := costFunc(rollupCostData) return fee, nil
diff --git official optimism/op-devstack/dsl/l2_network.go SWC optimism/op-devstack/dsl/l2_network.go index adde64d626e1fca245cfdb1af73859021e17cb14..df348bbea8a0018e8305e858ccaf70bd0e1b7b38 100644 --- official optimism/op-devstack/dsl/l2_network.go +++ SWC optimism/op-devstack/dsl/l2_network.go @@ -3,6 +3,7 @@ import ( "fmt" "math" + "math/big" "time"   "github.com/davecgh/go-spew/spew" @@ -254,3 +255,7 @@ func (n *L2Network) DepositContractAddr() common.Address { return n.inner.RollupConfig().DepositContractAddress } + +func (n *L2Network) L1ScalarMultiplierConfig() (*big.Int, *big.Int) { + return n.inner.RollupConfig().ChainOpConfig.L1ScalarMultipliers(n.unsafeHeadRef().Time) +}
diff --git official optimism/op-e2e/actions/helpers/l2_sequencer.go SWC optimism/op-e2e/actions/helpers/l2_sequencer.go index 97d8100b1a5d694ad70fc048fe3070c7357ed7b8..b735ab85c13c2f4f54e5132538ab74caa9c3b722 100644 --- official optimism/op-e2e/actions/helpers/l2_sequencer.go +++ SWC optimism/op-e2e/actions/helpers/l2_sequencer.go @@ -73,7 +73,7 @@ seqStateListener := config.DisabledConfigPersistence{} conduc := &conductor.NoOpConductor{} asyncGossip := async.NoOpGossiper{} seq := sequencing.NewSequencer(t.Ctx(), log, cfg, attrBuilder, l1OriginSelector, - seqStateListener, conduc, asyncGossip, metr, ver.engine) + seqStateListener, conduc, asyncGossip, metr, ver.engine, nil) opts := event.WithEmitLimiter( // TestSyncBatchType/DerivationWithFlakyL1RPC does *a lot* of quick retries // TestL2BatcherBatchType/ExtendedTimeWithoutL1Batches as well.
diff --git official optimism/op-e2e/actions/proofs/isthmus_setcode_tx_test.go SWC optimism/op-e2e/actions/proofs/isthmus_setcode_tx_test.go index d05694ec8cd243ccb8791366086c6e9e2ae37ea6..9af8566aed5ebd90a2dbd5b7f30578a126670e36 100644 --- official optimism/op-e2e/actions/proofs/isthmus_setcode_tx_test.go +++ SWC optimism/op-e2e/actions/proofs/isthmus_setcode_tx_test.go @@ -98,7 +98,7 @@ GasFeeCap: uint256.NewInt(5000000000), GasTipCap: uint256.NewInt(2), AuthList: []types.SetCodeAuthorization{auth1, auth2}, } - signer := types.NewIsthmusSigner(chainID) + signer := types.NewIsthmusSigner(chainID, false) tx := types.MustSignNewTx(aliceSecret, signer, txdata)   err = cl.SendTransaction(t.Ctx(), tx) @@ -244,7 +244,7 @@ u1 := sequencer.L2Unsafe()   sequencer.ActL2EmptyBlock(t) // we'll inject the setcode tx in this block's batch   - signer := types.NewIsthmusSigner(chainID) + signer := types.NewIsthmusSigner(chainID, false)   rng := rand.New(rand.NewSource(0)) tx := testutils.RandomSetCodeTx(rng, signer)
diff --git official optimism/op-e2e/actions/proofs/operator_fee_test.go SWC optimism/op-e2e/actions/proofs/operator_fee_test.go index b024be821a9eb4ee2666fd8ff67813084dc1b21e..4f1f733afe77275c261f3f493d0461f452a2fc19 100644 --- official optimism/op-e2e/actions/proofs/operator_fee_test.go +++ SWC optimism/op-e2e/actions/proofs/operator_fee_test.go @@ -212,7 +212,7 @@ l1BlockInfo, err := derive.L1BlockInfoFromBytes(env.Sd.RollupCfg, unsafeHeader.Time, unsafeBlock.Transactions()[0].Data()) require.NoError(t, err)   - daCost := fjordL1Cost(l1BlockInfo, types.NewRollupCostData(rlp)) + daCost := fjordL1Cost(l1BlockInfo, types.NewRollupCostData(rlp, len(tx.BlobHashes()))) expectedFeePreIsthmus := nextBaseFee.Mul(nextBaseFee, big.NewInt(int64(params.TxGas))) expectedFeePreIsthmus.Add(expectedFeePreIsthmus, daCost)   @@ -380,7 +380,9 @@ // Ensure that the logs contain a mention of the block being replaced _due to the signer not having enough // balance_. require.NotNil(t, env.Logs.FindLog(testlog.NewAttributesContainsFilter("err", "insufficient funds for gas * price + value"))) - require.NotNil(t, env.Logs.FindLog(testlog.NewAttributesContainsFilter("err", "have 1400000021000 want 1400000086535"))) + // log diff is caused by sgt, which is enabled by default + require.NotNil(t, env.Logs.FindLog(testlog.NewAttributesContainsFilter("err", "have total balance 1400000021000 want 1400000086535"))) + // require.NotNil(t, env.Logs.FindLog(testlog.NewAttributesContainsFilter("err", "have 1400000021000 want 1400000086535"))) } else { require.Equal(t, len(safeHeadBlock.Transactions()), 2) } @@ -405,7 +407,10 @@ costFunc := types.NewL1CostFuncFjord( l1BlockInfo.BaseFee, l1BlockInfo.BlobBaseFee, new(big.Int).SetUint64(uint64(l1BlockInfo.BaseFeeScalar)), - new(big.Int).SetUint64(uint64(l1BlockInfo.BlobBaseFeeScalar))) + new(big.Int).SetUint64(uint64(l1BlockInfo.BlobBaseFeeScalar)), + new(big.Int).SetUint64(1), + new(big.Int).SetUint64(1), + )   fee, _ := costFunc(rollupCostData) return fee
diff --git official optimism/op-e2e/actions/upgrades/fjord_fork_test.go SWC optimism/op-e2e/actions/upgrades/fjord_fork_test.go index e7a4a8af8aab22d2df589f659bf6ac5e639f9315..813281be10da3ea9d6ef9d3437d67aca0de02721 100644 --- official optimism/op-e2e/actions/upgrades/fjord_fork_test.go +++ SWC optimism/op-e2e/actions/upgrades/fjord_fork_test.go @@ -134,7 +134,10 @@ costFunc := types.NewL1CostFuncFjord( l1BaseFee, blobBaseFee, new(big.Int).SetUint64(uint64(baseFeeScalar)), - new(big.Int).SetUint64(uint64(blobBaseFeeScalar))) + new(big.Int).SetUint64(uint64(blobBaseFeeScalar)), + new(big.Int).SetUint64(1), + new(big.Int).SetUint64(1), + )   fee, _ := costFunc(rollupCostData) return fee
diff --git official optimism/op-e2e/opgeth/fastlz_test.go SWC optimism/op-e2e/opgeth/fastlz_test.go index 5b03ca38eca3c1fa3356e12d5213ff5c185bcc82..3b910d41f816df5f3a223cfa4e03563b4636244c 100644 --- official optimism/op-e2e/opgeth/fastlz_test.go +++ SWC optimism/op-e2e/opgeth/fastlz_test.go @@ -173,7 +173,7 @@ l1FeeSolidity.Mul(l1FeeSolidity, feeScaled) l1FeeSolidity.Div(l1FeeSolidity, big.NewInt(1e12))   - costData := types.NewRollupCostData(fuzzedData) + costData := types.NewRollupCostData(fuzzedData, 0)   l1FeeGeth := costFunc(costData, zeroTime)
diff --git official optimism/op-e2e/system/bridge/deposit_test.go SWC optimism/op-e2e/system/bridge/deposit_test.go index 70d06c1c33b685f493af4aeee81a91467a146f36..4007e8766d1373e583c44f2f08d562e927ed836d 100644 --- official optimism/op-e2e/system/bridge/deposit_test.go +++ SWC optimism/op-e2e/system/bridge/deposit_test.go @@ -143,7 +143,7 @@ // Simple constructor that is prefixed to the actual contract code // Results in the contract code being returned as the code for the new contract deployData := append(deployPrefix, sstoreContract...) - signer := types.NewIsthmusSigner(cfg.L2ChainIDBig()) + signer := types.NewIsthmusSigner(cfg.L2ChainIDBig(), false)   tx := types.MustSignNewTx(cfg.Secrets.Alice, signer, &types.DynamicFeeTx{ ChainID: cfg.L2ChainIDBig(),
diff --git official optimism/op-e2e/system/runcfg/protocol_versions_test.go SWC optimism/op-e2e/system/runcfg/protocol_versions_test.go index 85daeff6c43657ae3a51ee500d1b638e92ecebdc..87a234711b7e01e3f30a0c94800e8bfc2354b221 100644 --- official optimism/op-e2e/system/runcfg/protocol_versions_test.go +++ SWC optimism/op-e2e/system/runcfg/protocol_versions_test.go @@ -128,6 +128,12 @@ t.Log("verified that op-node closed!") // Checking if the engine is down is not trivial in op-e2e. // In op-geth we have halting tests covering the Engine API, in op-e2e we instead check if the API stops. _, err = retry.Do(context.Background(), 10, retry.Fixed(time.Second*10), func() (struct{}, error) { + url := sys.NodeEndpoint("verifier").(endpoint.HttpRPC).HttpRPC() + // The URL of the verifier op-geth becomes empty after it is stopped. + // In this case, IsURLAvailable() returns an incorrect value when port 80 happens to be occupied by another process. + if url == "http://" { + return struct{}{}, nil + } available := client.IsURLAvailable(context.Background(), sys.NodeEndpoint("verifier").(endpoint.HttpRPC).HttpRPC(), 5*time.Second) if !available { // waiting for client to stop responding to RPC requests (slow dials with timeout don't count) return struct{}{}, nil
diff --git official optimism/op-node/config/config.go SWC optimism/op-node/config/config.go index 4aaa32ccf970aaea42693a6e4fac867de003802b..9de04fbe9390e05a06f80978357f5ce5e5a132f0 100644 --- official optimism/op-node/config/config.go +++ SWC optimism/op-node/config/config.go @@ -4,6 +4,9 @@ import ( "context" "errors" "fmt" + + "strings" + "time"   "github.com/ethereum/go-ethereum/log" @@ -14,13 +17,16 @@ "github.com/ethereum-optimism/optimism/op-node/node/tracer" "github.com/ethereum-optimism/optimism/op-node/p2p" "github.com/ethereum-optimism/optimism/op-node/rollup" "github.com/ethereum-optimism/optimism/op-node/rollup/driver" + "github.com/ethereum-optimism/optimism/op-node/rollup/engine" "github.com/ethereum-optimism/optimism/op-node/rollup/interop" "github.com/ethereum-optimism/optimism/op-node/rollup/sync" + "github.com/ethereum-optimism/optimism/op-service/cliiface" opmetrics "github.com/ethereum-optimism/optimism/op-service/metrics" "github.com/ethereum-optimism/optimism/op-service/oppprof" oprpc "github.com/ethereum-optimism/optimism/op-service/rpc" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/depset" "github.com/ethereum/go-ethereum/params" + "github.com/ethstorage/da-server/pkg/da/client" )   type Config struct { @@ -84,6 +90,8 @@ ConductorRpcTimeout time.Duration   // AltDA config AltDA altda.CLIConfig + // DACConfig for sequencer when l2 blob is enabled + DACConfig *DACConfig   IgnoreMissingPectraBlobSchedule bool FetchWithdrawalRootFromState bool @@ -92,6 +100,27 @@ // Experimental. Enables new opstack RPC namespace. Used by op-test-sequencer. ExperimentalOPStackAPI bool }   +func ReadDACConfigFromCLI(c cliiface.Context) *DACConfig { + urls := c.String(flags.DACUrlsFlag.Name) + if urls == "" { + return nil + } + return &DACConfig{ + URLS: strings.Split(urls, ","), + } +} + +type DACConfig struct { + URLS []string +} + +func (dacConfig *DACConfig) Client() engine.DACClient { + if dacConfig == nil || len(dacConfig.URLS) == 0 { + return nil + } + return client.New(dacConfig.URLS) +} + // ConductorRPCFunc retrieves the endpoint. The RPC may not immediately be available. type ConductorRPCFunc func(ctx context.Context) (string, error)   @@ -180,6 +209,12 @@ return fmt.Errorf("altDA config error: %w", err) } if cfg.AltDA.Enabled { log.Warn("Alt-DA Mode is a Beta feature of the MIT licensed OP Stack. While it has received initial review from core contributors, it is still undergoing testing, and may have bugs or other issues.") + } + if cfg.Driver.SequencerEnabled && cfg.Rollup.IsL2BlobTimeSet() && cfg.DACConfig == nil { + return fmt.Errorf("dac.urls must be set for sequencer when l2 blob time is set") + } + if (!cfg.Driver.SequencerEnabled || !cfg.Rollup.IsL2BlobTimeSet()) && cfg.DACConfig != nil { + return fmt.Errorf("dac.urls can only be set for sequencer when l2 blob time is set") } return nil }
diff --git official optimism/op-node/rollup/derive/batch_test.go SWC optimism/op-node/rollup/derive/batch_test.go index c206739c979aeb9a9e92536d137266a7a0d7bf6f..4f67ef2e30e515554e720f29edaecf5adc213b7a 100644 --- official optimism/op-node/rollup/derive/batch_test.go +++ SWC optimism/op-node/rollup/derive/batch_test.go @@ -34,7 +34,7 @@ blockTxCount := 1 + uint64(rng.Intn(16)) blockTxCounts = append(blockTxCounts, blockTxCount) totalblockTxCounts += blockTxCount } - signer := types.NewIsthmusSigner(chainId) + signer := types.NewIsthmusSigner(chainId, false) var txs [][]byte for i := 0; i < int(totalblockTxCounts); i++ { var tx *types.Transaction
diff --git official optimism/op-node/rollup/derive/batch_test_utils.go SWC optimism/op-node/rollup/derive/batch_test_utils.go index bf28c1e00e656fb3afc5a2d2ad455bde71ce325b..a1007af5b62fc61edfb4e3b21d566bdc28f16545 100644 --- official optimism/op-node/rollup/derive/batch_test_utils.go +++ SWC optimism/op-node/rollup/derive/batch_test_utils.go @@ -11,7 +11,7 @@ "github.com/ethereum/go-ethereum/core/types" )   func RandomSingularBatch(rng *rand.Rand, txCount int, chainID *big.Int) *SingularBatch { - signer := types.NewIsthmusSigner(chainID) + signer := types.NewIsthmusSigner(chainID, false) baseFee := big.NewInt(rng.Int63n(300_000_000_000)) txsEncoded := make([]hexutil.Bytes, 0, txCount) // force each tx to have equal chainID
diff --git official optimism/op-node/rollup/derive/batches.go SWC optimism/op-node/rollup/derive/batches.go index 2e8c255d7fb2d421d3774a156d952f2579a74780..6974e807919015f3af09b630ab2b5f1608e8cef2 100644 --- official optimism/op-node/rollup/derive/batches.go +++ SWC optimism/op-node/rollup/derive/batches.go @@ -170,6 +170,7 @@ } }   isIsthmus := cfg.IsIsthmus(batch.Timestamp) + isL2Blob := cfg.IsL2Blob(batch.Timestamp)   // We can do this check earlier, but it's a more intensive one, so we do this last. for i, txBytes := range batch.Transactions { @@ -183,6 +184,10 @@ return BatchDrop } if !isIsthmus && txBytes[0] == types.SetCodeTxType { log.Warn("sequencers may not embed any SetCode transactions before Isthmus", "tx_index", i) + return BatchDrop + } + if !isL2Blob && txBytes[0] == types.BlobTxType { + log.Warn("sequencers may not embed any Blob transactions before L2Blob", "tx_index", i) return BatchDrop } }
diff --git official optimism/op-node/rollup/derive/batches_test.go SWC optimism/op-node/rollup/derive/batches_test.go index ade8d9df4c3cd255a1b9666f3aa15b9b9e0bf9eb..ae656c6368941dbc790cc7d0bf6c4cab0b7b518a 100644 --- official optimism/op-node/rollup/derive/batches_test.go +++ SWC optimism/op-node/rollup/derive/batches_test.go @@ -75,7 +75,7 @@ rng := rand.New(rand.NewSource(1234))   chainId := new(big.Int).SetUint64(rng.Uint64()) - signer := types.NewIsthmusSigner(chainId) + signer := types.NewIsthmusSigner(chainId, false) randTx := testutils.RandomTx(rng, new(big.Int).SetUint64(rng.Uint64()), signer) randTxData, _ := randTx.MarshalBinary()
diff --git official optimism/op-node/rollup/derive/span_batch_test.go SWC optimism/op-node/rollup/derive/span_batch_test.go index 96628c92c10ba213a1fe16524be4cf532ee94efc..e691fa58059966032a7f3cb3d300139fd38fb1be 100644 --- official optimism/op-node/rollup/derive/span_batch_test.go +++ SWC optimism/op-node/rollup/derive/span_batch_test.go @@ -485,7 +485,7 @@ for i, testCase := range cases { t.Run(testCase.name, func(t *testing.T) { rng := rand.New(rand.NewSource(int64(0x109550 + i))) chainID := new(big.Int).SetUint64(rng.Uint64()) - signer := types.NewIsthmusSigner(chainID) + signer := types.NewIsthmusSigner(chainID, false) if !testCase.protected { signer = types.HomesteadSigner{} }
diff --git official optimism/op-node/rollup/derive/span_batch_tx_test.go SWC optimism/op-node/rollup/derive/span_batch_tx_test.go index d215a0f2c6be01a30b3421e44c3d34b768941687..d39278856869f6385426f4c5fc99dbffe6642798 100644 --- official optimism/op-node/rollup/derive/span_batch_tx_test.go +++ SWC optimism/op-node/rollup/derive/span_batch_tx_test.go @@ -31,7 +31,7 @@ for i, testCase := range cases { t.Run(testCase.name, func(t *testing.T) { rng := rand.New(rand.NewSource(int64(0x1331 + i))) chainID := big.NewInt(rng.Int63n(1000)) - signer := types.NewIsthmusSigner(chainID) + signer := types.NewIsthmusSigner(chainID, false) if !testCase.protected { signer = types.HomesteadSigner{} } @@ -71,7 +71,7 @@ for i, testCase := range cases { t.Run(testCase.name, func(t *testing.T) { rng := rand.New(rand.NewSource(int64(0x1332 + i))) chainID := big.NewInt(rng.Int63n(1000)) - signer := types.NewIsthmusSigner(chainID) + signer := types.NewIsthmusSigner(chainID, false) if !testCase.protected { signer = types.HomesteadSigner{} }
diff --git official optimism/op-node/rollup/derive/test/random.go SWC optimism/op-node/rollup/derive/test/random.go index 24f28bb174a37ee9cb5a10c6e730b87b89400a9b..1ccece081f1fa501bbda19fc16edc7fa9b0cffcd 100644 --- official optimism/op-node/rollup/derive/test/random.go +++ SWC optimism/op-node/rollup/derive/test/random.go @@ -41,7 +41,7 @@ return RandomL2BlockWithChainIdAndTime(rng, txCount, chainId, time.Time{}) }   func RandomL2BlockWithChainIdAndTime(rng *rand.Rand, txCount int, chainId *big.Int, t time.Time) *types.Block { - signer := types.NewIsthmusSigner(chainId) + signer := types.NewIsthmusSigner(chainId, false) block, _ := RandomL2Block(rng, 0, t) txs := []*types.Transaction{block.Transactions()[0]} // L1 info deposit TX for i := 0; i < txCount; i++ {
diff --git official optimism/op-node/rollup/superchain.go SWC optimism/op-node/rollup/superchain.go index ec3da88143097204db2a705e192fa9c19d6bb844..b71c9aa2981462eaa6dae51ff27e13203e6203f0 100644 --- official optimism/op-node/rollup/superchain.go +++ SWC optimism/op-node/rollup/superchain.go @@ -12,6 +12,14 @@ )   var OPStackSupport = params.ProtocolVersionV0{Build: [8]byte{}, Major: 9, Minor: 0, Patch: 0, PreRelease: 0}.Encode()   +func copyuint64ptr(ptr *uint64) *uint64 { + if ptr == nil { + return nil + } + val := *ptr + return &val +} + // LoadOPStackRollupConfig loads the rollup configuration of the requested chain ID from the superchain-registry. // Some chains may require a SystemConfigProvider to retrieve any values not part of the registry. func LoadOPStackRollupConfig(chainID uint64) (*Config, error) { @@ -25,9 +33,14 @@ if err != nil { return nil, fmt.Errorf("unable to retrieve chain %d config: %w", chainID, err) } chOpConfig := &params.OptimismConfig{ - EIP1559Elasticity: chConfig.Optimism.EIP1559Elasticity, - EIP1559Denominator: chConfig.Optimism.EIP1559Denominator, - EIP1559DenominatorCanyon: chConfig.Optimism.EIP1559DenominatorCanyon, + EIP1559Elasticity: chConfig.Optimism.EIP1559Elasticity, + EIP1559Denominator: chConfig.Optimism.EIP1559Denominator, + EIP1559DenominatorCanyon: chConfig.Optimism.EIP1559DenominatorCanyon, + L2BlobTime: copyuint64ptr(chConfig.Optimism.L2BlobTime), + SoulGasTokenTime: copyuint64ptr(chConfig.Optimism.SoulGasTokenTime), + IsSoulBackedByNative: chConfig.Optimism.IsSoulBackedByNative, + L1BaseFeeScalarMultiplier: chConfig.Optimism.L1BaseFeeScalarMultiplier, + L1BlobBaseFeeScalarMultiplier: chConfig.Optimism.L1BlobBaseFeeScalarMultiplier, }   superConfig, err := superchain.GetSuperchain(chain.Network) @@ -45,6 +58,13 @@ GasLimit: sysCfg.GasLimit, }   addrs := chConfig.Addresses + + var inboxContractConfig *InboxContractConfig + if chConfig.InboxContractConfig != nil { + inboxContractConfig = &InboxContractConfig{ + UseInboxContract: chConfig.InboxContractConfig.UseInboxContract, + } + }   var altDA *AltDAConfig if chConfig.AltDA != nil { @@ -80,6 +100,7 @@ ChannelTimeoutBedrock: 300, L1ChainID: new(big.Int).SetUint64(superConfig.L1.ChainID), L2ChainID: new(big.Int).SetUint64(chConfig.ChainID), BatchInboxAddress: chConfig.BatchInboxAddr, + InboxContractConfig: inboxContractConfig, DepositContractAddress: *addrs.OptimismPortalProxy, L1SystemConfigAddress: *addrs.SystemConfigProxy, AltDAConfig: altDA,
diff --git official optimism/op-service/testutils/devnet/proxy.go SWC optimism/op-service/testutils/devnet/proxy.go index 0f1ab2c89ee131a7de303f8bcc429f660d41a28a..6d494389c855e6050fb2aaa3698a80dfea82e982 100644 --- official optimism/op-service/testutils/devnet/proxy.go +++ SWC optimism/op-service/testutils/devnet/proxy.go @@ -162,6 +162,7 @@ req, err := http.NewRequestWithContext(ctx, http.MethodPost, p.upstream, bytes.NewReader(body)) if err != nil { panic(fmt.Errorf("failed to create request: %w", err)) } + req.Header.Set("Content-Type", "application/json") res, err := p.client.Do(req) if err != nil { p.lgr.Warn("failed to proxy request", "err", err)
diff --git official optimism/op-service/txinclude/isthmus_cost_oracle.go SWC optimism/op-service/txinclude/isthmus_cost_oracle.go index 4083b87328c38af80a135ae4c76a1c06339d1f18..cdaba0155024a6995806b0d875d5f4ecb2b791e6 100644 --- official optimism/op-service/txinclude/isthmus_cost_oracle.go +++ SWC optimism/op-service/txinclude/isthmus_cost_oracle.go @@ -86,7 +86,7 @@ func (i *IsthmusCostOracle) OPCost(tx *types.Transaction) *big.Int { params := i.costParams.Load()   - l1CostFunc := types.NewL1CostFuncFjord(params.L1BaseFee, params.L1BlobBaseFee, params.L1BaseFeeScalar, params.L1BlobBaseFeeScalar) + l1CostFunc := types.NewL1CostFuncFjord(params.L1BaseFee, params.L1BlobBaseFee, params.L1BaseFeeScalar, params.L1BlobBaseFeeScalar, big.NewInt(1), big.NewInt(1)) l1Cost, _ := l1CostFunc(tx.RollupCostData())   operatorCost := new(big.Int).SetUint64(tx.Gas())
diff --git official optimism/op-service/txinclude/isthmus_cost_oracle_test.go SWC optimism/op-service/txinclude/isthmus_cost_oracle_test.go index a6dafba52a5660c884d404dc79fb52b86a1a6622..7993cbcdca6ba2d51fab60f544e97a4df5895520 100644 --- official optimism/op-service/txinclude/isthmus_cost_oracle_test.go +++ SWC optimism/op-service/txinclude/isthmus_cost_oracle_test.go @@ -100,7 +100,7 @@ oracle := txinclude.NewIsthmusCostOracle(mock, time.Millisecond) require.NoError(t, oracle.SetParams(context.Background())) tx := types.NewTx(&types.DynamicFeeTx{}) got := oracle.OPCost(tx) - want, _ := types.NewL1CostFuncFjord(big.NewInt(102), big.NewInt(104), big.NewInt(103), big.NewInt(105))(tx.RollupCostData()) + want, _ := types.NewL1CostFuncFjord(big.NewInt(102), big.NewInt(104), big.NewInt(103), big.NewInt(105), big.NewInt(1), big.NewInt(1))(tx.RollupCostData()) require.Equal(t, want, got) }) }
diff --git official optimism/op-service/txinclude/txbudget.go SWC optimism/op-service/txinclude/txbudget.go index 4582b90853afa3f8a6e052e4398de59751a83286..d0dcecb47124953c07227e0720bc0e6b0e539281 100644 --- official optimism/op-service/txinclude/txbudget.go +++ SWC optimism/op-service/txinclude/txbudget.go @@ -88,7 +88,7 @@ // l1Cost if receipt.L1BaseFeeScalar != nil { l1BaseFeeScalar := new(big.Int).SetUint64(*receipt.L1BaseFeeScalar) l1BlobBaseFeeScalar := new(big.Int).SetUint64(*receipt.L1BlobBaseFeeScalar) - costFunc := types.NewL1CostFuncFjord(receipt.L1GasPrice, receipt.L1BlobBaseFee, l1BaseFeeScalar, l1BlobBaseFeeScalar) + costFunc := types.NewL1CostFuncFjord(receipt.L1GasPrice, receipt.L1BlobBaseFee, l1BaseFeeScalar, l1BlobBaseFeeScalar, big.NewInt(1), big.NewInt(1)) l1Cost, _ := costFunc(tx.Transaction.RollupCostData()) actualCost.Add(actualCost, l1Cost) }
diff --git official optimism/op-service/txinclude/txbudget_test.go SWC optimism/op-service/txinclude/txbudget_test.go index a658fc8085258148bc7ac2da4d35d493e388d8a6..600a18186bb852a8006eadbb1124f712166582b4 100644 --- official optimism/op-service/txinclude/txbudget_test.go +++ SWC optimism/op-service/txinclude/txbudget_test.go @@ -91,8 +91,7 @@ BlobFeeCap: uint256.NewInt(1), BlobHashes: []common.Hash{{}}, })   - l1Cost, _ := types.NewL1CostFuncFjord(big.NewInt(1), big.NewInt(1), big.NewInt(1), big.NewInt(1))(tx.RollupCostData()) - l1Cost.Add(l1Cost, big.NewInt(1)) // operator fee + l1Cost, _ := types.NewL1CostFuncFjord(big.NewInt(1), big.NewInt(1), big.NewInt(1), big.NewInt(1), big.NewInt(1), big.NewInt(1))(tx.RollupCostData()) oracle := mockOPCostOracle{ cost: l1Cost, } @@ -104,7 +103,7 @@ budgetedCost := eth.WeiBig(cost)   receipt := &types.Receipt{ EffectiveGasPrice: eth.WeiU64(1).ToBig(), - GasUsed: budgetedCost.ToBig().Uint64(), + GasUsed: budgetedCost.ToBig().Uint64() - l1Cost.Uint64(), Type: types.DynamicFeeTxType,   L1GasPrice: big.NewInt(1),
diff --git official optimism/ops/docker/op-stack-go/Dockerfile SWC optimism/ops/docker/op-stack-go/Dockerfile index 0b40d26cc005d1203f58c5684f4a6efbe08fd6c6..da60ffd6bf455e1f04d83fa02733ac6c37f846d8 100644 --- official optimism/ops/docker/op-stack-go/Dockerfile +++ SWC optimism/ops/docker/op-stack-go/Dockerfile @@ -14,7 +14,7 @@ ARG UBUNTU_TARGET_BASE_IMAGE=ubuntu:22.04   # The version of kona to use. # The only build that uses this is `op-challenger-target`. -ARG KONA_VERSION=none +ARG KONA_VERSION=kona-client-v0.1.0-beta.5   # The version of asterisc to use. # The only build that uses this is `op-challenger-target`. @@ -187,6 +187,15 @@ RUN --mount=type=cache,target=/go/pkg/mod --mount=type=cache,target=/root/.cache/go-build just \ GOOS=$TARGETOS GOARCH=$TARGETARCH GITCOMMIT=$GIT_COMMIT GITDATE=$GIT_DATE VERSION="$OP_DEPLOYER_VERSION" \ op-deployer/build-go   + +FROM --platform=$BUILDPLATFORM builder AS dac-server-builder +ARG OP_NODE_VERSION=v0.0.0 +RUN --mount=type=cache,target=/go/pkg/mod --mount=type=cache,target=/root/.cache/go-build cd da-server && go build -o da-server main.go + +FROM --platform=$BUILDPLATFORM builder AS op-geth-builder +ARG OP_NODE_VERSION=v0.0.0 +RUN --mount=type=cache,target=/go/pkg/mod --mount=type=cache,target=/root/.cache/go-build cd op-geth && CGO_ENABLED=0 make geth + FROM --platform=$BUILDPLATFORM builder AS op-dripper-builder ARG OP_DRIPPER_VERSION=v0.0.0 RUN --mount=type=cache,target=/go/pkg/mod --mount=type=cache,target=/root/.cache/go-build cd op-dripper && make op-dripper \ @@ -219,6 +228,9 @@ CMD ["op-wheel"]   FROM $TARGET_BASE_IMAGE AS op-node-target COPY --from=op-node-builder /app/op-node/bin/op-node /usr/local/bin/ +COPY --from=dac-server-builder /app/da-server/da-server /usr/local/bin/ +COPY --from=dac-server-builder /app/da-server/default.json /usr/local/bin/ +COPY --from=op-geth-builder /app/op-geth/build/bin/geth /usr/local/bin/ CMD ["op-node"]   # Make the kona docker image published by upstream available as a source to copy kona from.
diff --git official optimism/ops/docker/op-stack-go/Dockerfile.dockerignore SWC optimism/ops/docker/op-stack-go/Dockerfile.dockerignore index 280b1603456263ad79bb54e770eec0f72f135464..85aadf567d6e1a963fe8cdff1784094d972f0580 100644 --- official optimism/ops/docker/op-stack-go/Dockerfile.dockerignore +++ SWC optimism/ops/docker/op-stack-go/Dockerfile.dockerignore @@ -25,6 +25,8 @@ !/op-wheel !/op-alt-da !/op-faucet !/op-interop-mon +!/da-server +!/op-geth !/go.mod !/go.sum !/justfiles
diff --git official optimism/packages/contracts-bedrock/interfaces/universal/IStorageSetter.sol SWC optimism/packages/contracts-bedrock/interfaces/universal/IStorageSetter.sol new file mode 100644 index 0000000000000000000000000000000000000000..fe339c38a14ee1d27eb68cd3d2c5bd72badf8adb --- /dev/null +++ SWC optimism/packages/contracts-bedrock/interfaces/universal/IStorageSetter.sol @@ -0,0 +1,8 @@ +// SPDX-License-Identifier: MIT +pragma solidity ^0.8.0; + +interface IStorageSetter { + function setBytes32(bytes32, bytes32) external; + + function __constructor__() external; +}
diff --git official optimism/packages/contracts-bedrock/scripts/checks/check-semver-diff.sh SWC optimism/packages/contracts-bedrock/scripts/checks/check-semver-diff.sh index 6fa03c5f729f91c7ed44e8fb314bf0c9d08799cf..157a69ec10c01fd5c19ebcb23b58e8431e92c76d 100755 --- official optimism/packages/contracts-bedrock/scripts/checks/check-semver-diff.sh +++ SWC optimism/packages/contracts-bedrock/scripts/checks/check-semver-diff.sh @@ -32,7 +32,7 @@ temp_dir=$(mktemp -d) trap 'rm -rf "$temp_dir"' EXIT   # Exit early if semver-lock.json has not changed. -TARGET_BRANCH="${TARGET_BRANCH:-develop}" +TARGET_BRANCH="${TARGET_BRANCH:-op-es}" UPSTREAM_REF="origin/${TARGET_BRANCH}" if ! { git diff "$UPSTREAM_REF"...HEAD --name-only; git diff --name-only; git diff --cached --name-only; } | grep -q "$SEMVER_LOCK"; then echo "No changes detected in semver-lock.json"
diff --git official optimism/packages/contracts-bedrock/scripts/checks/interfaces/main.go SWC optimism/packages/contracts-bedrock/scripts/checks/interfaces/main.go index fb2bdf4051201dd0b99839987cd2039ffc72950f..d7138e5f98b655a4105f53484dad507785f869fa 100644 --- official optimism/packages/contracts-bedrock/scripts/checks/interfaces/main.go +++ SWC optimism/packages/contracts-bedrock/scripts/checks/interfaces/main.go @@ -31,8 +31,10 @@ // Misc stuff that can be ignored "IOPContractsManagerLegacyUpgrade",   // TODO: Interfaces that need to be fixed + "IInitializable", "IOptimismMintableERC20", "ILegacyMintableERC20", "KontrolCheatsBase", "IResolvedDelegateProxy", + "IERC20Upgradeable", "ISoulGasToken", "IStorageSetter", }   // excludeSourceContracts is a list of contracts that are allowed to not have interfaces
diff --git official optimism/packages/contracts-bedrock/src/dispute/AnchorStateRegistry.sol SWC optimism/packages/contracts-bedrock/src/dispute/AnchorStateRegistry.sol index 87f74740048e61b0a495031b0d722696f7847c6a..b70734d9028bc27995861240c5524ac76cc23e01 100644 --- official optimism/packages/contracts-bedrock/src/dispute/AnchorStateRegistry.sol +++ SWC optimism/packages/contracts-bedrock/src/dispute/AnchorStateRegistry.sol @@ -344,7 +344,8 @@ /// @notice Asserts that the caller is the Guardian. function _assertOnlyGuardian() internal view { if (msg.sender != systemConfig.guardian()) { - revert AnchorStateRegistry_Unauthorized(); + /// QKC changes: we also allow the ProxyAdmin or ProxyAdmin owner to perform guardian actions + _assertOnlyProxyAdminOrProxyAdminOwner(); } } }
diff --git official optimism/packages/contracts-bedrock/test/libraries/Predeploys.t.sol SWC optimism/packages/contracts-bedrock/test/libraries/Predeploys.t.sol index 548c178050fd919ed982ed9a9a5a76930ab14dd4..93d862b27ef853599a11127627badfc7e7b1bc46 100644 --- official optimism/packages/contracts-bedrock/test/libraries/Predeploys.t.sol +++ SWC optimism/packages/contracts-bedrock/test/libraries/Predeploys.t.sol @@ -31,7 +31,8 @@ /// @notice Returns true if the predeploy is initializable. function _isInitializable(address _addr) internal pure returns (bool) { return _addr == Predeploys.L2_CROSS_DOMAIN_MESSENGER || _addr == Predeploys.L2_STANDARD_BRIDGE - || _addr == Predeploys.L2_ERC721_BRIDGE || _addr == Predeploys.OPTIMISM_MINTABLE_ERC20_FACTORY; + || _addr == Predeploys.L2_ERC721_BRIDGE || _addr == Predeploys.OPTIMISM_MINTABLE_ERC20_FACTORY + || _addr == Predeploys.SOUL_GAS_TOKEN; }   /// @notice Returns true if the predeploy uses immutables.
diff --git official optimism/packages/contracts-bedrock/test/scripts/L2Genesis.t.sol SWC optimism/packages/contracts-bedrock/test/scripts/L2Genesis.t.sol index ffd6a60d4d1268ec875a93a01a6857a2ffc84a66..31ffa61826fd00680eddbad6a9e63f90425ff6b3 100644 --- official optimism/packages/contracts-bedrock/test/scripts/L2Genesis.t.sol +++ SWC optimism/packages/contracts-bedrock/test/scripts/L2Genesis.t.sol @@ -128,7 +128,9 @@ governanceTokenOwner: address(0x0000000000000000000000000000000000000008), fork: uint256(LATEST_FORK), deployCrossL2Inbox: true, enableGovernance: true, - fundDevAccounts: true + fundDevAccounts: true, + deploySoulGasToken: true, + isSoulBackedByNative: false }); genesis.run(input);
diff --git official optimism/packages/contracts-bedrock/test/setup/CommonTest.sol SWC optimism/packages/contracts-bedrock/test/setup/CommonTest.sol index 71f55ec386702a5209772384a6780fb102181548..9b811c9de40d3945d76cc0439eb915edb9016d8d 100644 --- official optimism/packages/contracts-bedrock/test/setup/CommonTest.sol +++ SWC optimism/packages/contracts-bedrock/test/setup/CommonTest.sol @@ -34,6 +34,8 @@ FFIInterface constant ffi = FFIInterface(address(uint160(uint256(keccak256(abi.encode("optimism.ffi"))))));   bool useAltDAOverride; bool useInteropOverride; + bool deploySoulGasToken; + bool isSoulBackedByNative;   /// @dev This value is only used in forked tests. During forked tests, the default is to perform the upgrade before /// running the tests. @@ -71,6 +73,14 @@ } if (useInteropOverride) { deploy.cfg().setUseInterop(true); } + + if (deploySoulGasToken) { + deploy.cfg().setDeploySoulGasToken(true); + if (isSoulBackedByNative) { + deploy.cfg().setIsSoulBackedByNative(true); + } + } + if (useUpgradedFork) { deploy.cfg().setUseUpgradedFork(true); } @@ -198,6 +208,13 @@ /// @dev Enables interoperability mode for testing function enableInterop() public { _checkNotDeployed("interop"); useInteropOverride = true; + } + + function enableSoulGasToken() public { + _checkNotDeployed("sgt"); + + deploySoulGasToken = true; + isSoulBackedByNative = true; }   /// @dev Disables upgrade mode for testing. By default the fork testing env will be upgraded to the latest
diff --git official optimism/packages/contracts-bedrock/test/setup/Setup.sol SWC optimism/packages/contracts-bedrock/test/setup/Setup.sol index 92fb55b6485fd3a35cbb3f316212f4bdb14ed57f..8ae2af02279a4028899e99488dbd23aeecf9c346 100644 --- official optimism/packages/contracts-bedrock/test/setup/Setup.sol +++ SWC optimism/packages/contracts-bedrock/test/setup/Setup.sol @@ -59,6 +59,7 @@ import { IWETH98 } from "interfaces/universal/IWETH98.sol"; import { IGovernanceToken } from "interfaces/governance/IGovernanceToken.sol"; import { ILegacyMessagePasser } from "interfaces/legacy/ILegacyMessagePasser.sol"; import { ISuperchainTokenBridge } from "interfaces/L2/ISuperchainTokenBridge.sol"; +import { ISoulGasToken } from "interfaces/L2/ISoulGasToken.sol"; import { IPermissionedDisputeGame } from "interfaces/dispute/IPermissionedDisputeGame.sol"; import { IFaultDisputeGame } from "interfaces/dispute/IFaultDisputeGame.sol"; import { ICrossL2Inbox } from "interfaces/L2/ICrossL2Inbox.sol"; @@ -146,6 +147,7 @@ IETHLiquidity ethLiquidity = IETHLiquidity(Predeploys.ETH_LIQUIDITY); ISuperchainTokenBridge superchainTokenBridge = ISuperchainTokenBridge(Predeploys.SUPERCHAIN_TOKEN_BRIDGE); IOptimismSuperchainERC20Factory l2OptimismSuperchainERC20Factory = IOptimismSuperchainERC20Factory(Predeploys.OPTIMISM_SUPERCHAIN_ERC20_FACTORY); + ISoulGasToken soulGasToken = ISoulGasToken(Predeploys.SOUL_GAS_TOKEN);   /// @notice Indicates whether a test is running against a forked production network. function isForkTest() public view returns (bool) { @@ -326,7 +328,9 @@ governanceTokenOwner: deploy.cfg().governanceTokenOwner(), fork: uint256(l2Fork), deployCrossL2Inbox: deploy.cfg().useInterop(), enableGovernance: deploy.cfg().enableGovernance(), - fundDevAccounts: deploy.cfg().fundDevAccounts() + fundDevAccounts: deploy.cfg().fundDevAccounts(), + deploySoulGasToken: deploy.cfg().deploySoulGasToken(), + isSoulBackedByNative: deploy.cfg().isSoulBackedByNative() }) );   @@ -358,6 +362,7 @@ labelPredeploy(Predeploys.ETH_LIQUIDITY); labelPredeploy(Predeploys.OPTIMISM_SUPERCHAIN_ERC20_FACTORY); labelPredeploy(Predeploys.OPTIMISM_SUPERCHAIN_ERC20_BEACON); labelPredeploy(Predeploys.SUPERCHAIN_TOKEN_BRIDGE); + labelPredeploy(Predeploys.SOUL_GAS_TOKEN);   // L2 Preinstalls labelPreinstall(Preinstalls.MultiCall3);
diff --git official optimism/packages/contracts-bedrock/test/universal/BenchmarkTest.t.sol SWC optimism/packages/contracts-bedrock/test/universal/BenchmarkTest.t.sol index d78aeb37ae5054f810c163917b70cd371b633c8e..0b210ab26a11ef1b125b1eec3b601ee186bd8d63 100644 --- official optimism/packages/contracts-bedrock/test/universal/BenchmarkTest.t.sol +++ SWC optimism/packages/contracts-bedrock/test/universal/BenchmarkTest.t.sol @@ -64,7 +64,7 @@ // Test SafeCall.call({ _target: address(l1Block), _calldata: setValuesCalldata });   // Assert - assertLt(vm.lastCallGas().gasTotalUsed, 160_000); + assertLt(vm.lastCallGas().gasTotalUsed, 161_000); } }
diff --git official optimism/packages/contracts-bedrock/test/vendor/Initializable.t.sol SWC optimism/packages/contracts-bedrock/test/vendor/Initializable.t.sol index 5318d7d15fdcab64a6c3face290d6015a09d9a3e..0497aec028b528bede863c96a33daf606e99f132 100644 --- official optimism/packages/contracts-bedrock/test/vendor/Initializable.t.sol +++ SWC optimism/packages/contracts-bedrock/test/vendor/Initializable.t.sol @@ -47,6 +47,7 @@ mapping(string => string) nicknames;   function setUp() public override { super.enableAltDA(); + super.enableSoulGasToken(); super.setUp();   // Initialize the `contracts` array with the addresses of the contracts to test, the @@ -253,6 +254,22 @@ target: address(protocolVersions), initCalldata: abi.encodeCall( protocolVersions.initialize, (address(0), ProtocolVersion.wrap(1), ProtocolVersion.wrap(2)) ) + }) + ); + // SoulGasTokenImpl + contracts.push( + InitializeableContract({ + name: "SoulGasTokenImpl", + target: EIP1967Helper.getImplementation(address(soulGasToken)), + initCalldata: abi.encodeCall(soulGasToken.initialize, ("SoulGasToken", "SGT", address(0))) + }) + ); + // SoulGasTokenProxy + contracts.push( + InitializeableContract({ + name: "SoulGasTokenProxy", + target: address(soulGasToken), + initCalldata: abi.encodeCall(soulGasToken.initialize, ("SoulGasToken", "SGT", address(0))) }) ); // L1StandardBridgeImpl
diff --git official optimism/.circleci/config.yml SWC optimism/.circleci/config.yml index c08ea0f326a25a8d8fdfcf8cab4248dc2157a2cb..ec063b93a57d8651c8aca06145e3a11745e9808b 100644 --- official optimism/.circleci/config.yml +++ SWC optimism/.circleci/config.yml @@ -94,7 +94,7 @@ gcp-cli: circleci/gcp-cli@3.0.1 slack: circleci/slack@6.0.0 shellcheck: circleci/shellcheck@3.2.0 codecov: codecov/codecov@5.0.3 - utils: ethereum-optimism/circleci-utils@1.0.22 + utils: qkc/circleci-utils@0.0.2 docker: circleci/docker@2.8.2 github-cli: circleci/github-cli@2.7.0   @@ -205,7 +205,7 @@ - slack/notify: channel: << parameters.channel >> event: fail template: basic_fail_1 - branch_pattern: develop + branch_pattern: op-es mentions: "<< parameters.mentions >>"   get-target-branch: @@ -462,7 +462,7 @@ - "."   cannon-go-lint-and-test: machine: true - resource_class: ethereum-optimism/latitude-1 + resource_class: qkc/ax101 parameters: skip_slow_tests: type: boolean @@ -522,7 +522,7 @@ mentions: "@proofs-team"   diff-asterisc-bytecode: machine: true - resource_class: ethereum-optimism/latitude-1 + resource_class: qkc/ax101 steps: - checkout-from-workspace - run: @@ -560,9 +560,8 @@ - notify-failures-on-develop: mentions: "@clabby @proofs-team"   contracts-bedrock-build: - docker: - - image: <<pipeline.parameters.default_docker_image>> - resource_class: xlarge + machine: true + resource_class: qkc/ax101 parameters: build_args: description: Forge build arguments @@ -791,7 +790,7 @@ - "<<parameters.publish>>" - "<<parameters.release>>" - and: - "<<parameters.publish>>" - - equal: [develop, << pipeline.git.branch >>] + - equal: [op-es, << pipeline.git.branch >>] steps: - gcp-oidc-authenticate: service_account_email: GCP_SERVICE_ATTESTOR_ACCOUNT_EMAIL @@ -899,7 +898,7 @@ command: | TEST_FILES=$(<<parameters.test_list>>) if [ -z "$TEST_FILES" ]; then echo "No test files to run. Exiting early." - circleci-agent step halt + exit 0 fi working_directory: packages/contracts-bedrock - check-changed: @@ -930,7 +929,10 @@ - run: name: Run tests command: | TEST_FILES=$(<<parameters.test_list>>) - TEST_FILES=$(echo "$TEST_FILES" | circleci tests split --split-by=timings) + if [ -z "$TEST_FILES" ]; then + echo "No test files to run. Skipping forge test." + exit 0 + fi TEST_FILES=$(echo "$TEST_FILES" | sed 's|^test/||') MATCH_PATH="./test/{$(echo "$TEST_FILES" | paste -sd "," -)}" forge <<parameters.test_command>> <<parameters.test_flags>> --match-path "$MATCH_PATH" @@ -1043,7 +1045,7 @@ contracts-bedrock-coverage: circleci_ip_ranges: true docker: - image: <<pipeline.parameters.default_docker_image>> - resource_class: 2xlarge + resource_class: xlarge parameters: test_flags: description: Additional flags to pass to the test command @@ -1225,7 +1227,7 @@ - notify-failures-on-develop   contracts-bedrock-checks: machine: true - resource_class: ethereum-optimism/latitude-1 + resource_class: qkc/ax101 steps: - checkout-from-workspace - check-changed: @@ -1292,9 +1294,8 @@ uses_artifacts: description: should load in foundry artifacts type: boolean default: false - docker: - - image: <<pipeline.parameters.default_docker_image>> - resource_class: xlarge + machine: true + resource_class: qkc/ax101 steps: - checkout-from-workspace - check-changed: @@ -1323,9 +1324,8 @@ path: ./fuzzdata when: always   go-lint: - docker: - - image: <<pipeline.parameters.default_docker_image>> - resource_class: large + machine: true + resource_class: qkc/ax101 steps: - checkout-from-workspace - restore_cache: @@ -1349,6 +1349,10 @@ mentions: description: Slack user or group to mention when notifying of failures type: string default: "" + resource_class: + description: Machine resource class + type: string + default: qkc/ax101 no_output_timeout: description: Timeout for when CircleCI kills the job if there's no output type: string @@ -1870,9 +1874,8 @@ retry_delay: 3 template: SLACK_TEMPLATE   sanitize-op-program: - docker: - - image: <<pipeline.parameters.default_docker_image>> - resource_class: large + machine: true + resource_class: qkc/ax101 steps: - checkout-from-workspace - run: @@ -1893,7 +1896,7 @@ working_directory: cannon   cannon-prestate-quick: machine: true - resource_class: ethereum-optimism/latitude-1 + resource_class: qkc/ax101 steps: - checkout-from-workspace - restore_cache: @@ -1934,9 +1937,8 @@ - "op-program/bin/prestate*" - "op-program/bin/meta*"   publish-cannon-prestates: - resource_class: medium - docker: - - image: <<pipeline.parameters.default_docker_image>> + machine: true + resource_class: qkc/ax101 steps: - utils/checkout-with-mise - attach_workspace: @@ -2028,7 +2030,7 @@ semgrep-scan: parameters: diff_branch: type: string - default: develop + default: op-es scan_command: type: string default: semgrep ci --timeout=100 @@ -2044,7 +2046,7 @@ steps: - checkout # no need to use mise here since the docker image contains the only dependency - unless: condition: - equal: ["develop", << pipeline.git.branch >>] + equal: [ "op-es", << pipeline.git.branch >> ] steps: - run: # Scan changed files in PRs, block on new issues only (existing issues ignored) @@ -2090,7 +2092,7 @@ working_directory: op-program   op-program-compat: machine: true - resource_class: ethereum-optimism/latitude-1 + resource_class: qkc/ax101 steps: - checkout-from-workspace - run: @@ -2101,7 +2103,7 @@ working_directory: op-program   check-generated-mocks-op-node: machine: true - resource_class: ethereum-optimism/latitude-1 + resource_class: qkc/ax101 steps: - checkout-from-workspace - check-changed: @@ -2112,7 +2114,7 @@ command: make generate-mocks-op-node && git diff --exit-code   check-generated-mocks-op-service: machine: true - resource_class: ethereum-optimism/latitude-1 + resource_class: qkc/ax101 steps: - checkout-from-workspace - check-changed: @@ -2164,7 +2166,7 @@ - notify-failures-on-develop   publish-contract-artifacts: machine: true - resource_class: ethereum-optimism/latitude-1 + resource_class: qkc/ax101 steps: - gcp-cli/install - gcp-oidc-authenticate: @@ -2395,15 +2397,13 @@ - contracts-bedrock-build - cannon-prestate-quick main: when: - or: - - equal: ["webhook", << pipeline.trigger_source >>] - - and: - - equal: [true, <<pipeline.parameters.main_dispatch>>] - - equal: ["api", << pipeline.trigger_source >>] - - equal: [ - << pipeline.parameters.github-event-type >>, - "__not_set__", - ] #this is to prevent triggering this workflow as the default value is always set for main_dispatch + and: + - equal: [true, <<pipeline.parameters.main_dispatch>>] + - equal: ["api", << pipeline.trigger_source >>] + - equal: [ + << pipeline.parameters.github-event-type >>, + "__not_set__", + ] #this is to prevent triggering this workflow as the default value is always set for main_dispatch jobs: - initialize: context: @@ -2424,8 +2424,8 @@ - circleci-repo-readonly-authenticated-github-token - contracts-bedrock-tests: # Heavily fuzz any fuzz tests within added or modified test files. name: contracts-bedrock-tests-heavy-fuzz-modified - test_list: git diff origin/develop...HEAD --name-only --diff-filter=AM -- './test/**/*.t.sol' | sed 's|packages/contracts-bedrock/||' - test_timeout: 1h + test_list: git diff origin/op-es...HEAD --name-only --diff-filter=AM -- './test/**/*.t.sol' | sed 's|packages/contracts-bedrock/||' + test_timeout: 3h test_profile: ciheavy context: - circleci-repo-readonly-authenticated-github-token @@ -2574,9 +2574,10 @@ parallelism: 16 no_output_timeout: 89m # Longer timeout for full tests test_timeout: 90m notify: true - filters: - branches: - only: develop # Only runs on develop branch (post-merge) + # Always run full tests on qkc + # filters: + # branches: + # only: develop # Only runs on develop branch (post-merge) requires: - contracts-bedrock-build - cannon-prestate-quick @@ -2678,590 +2679,590 @@ ignore-dirs: ./packages/contracts-bedrock/lib context: - circleci-repo-readonly-authenticated-github-token   - go-release-op-deployer: - jobs: - - initialize: - filters: - tags: - only: /^op-deployer.*/ - branches: - ignore: /.*/ - context: - - circleci-repo-readonly-authenticated-github-token - - contracts-bedrock-build: - name: build-contracts-go-release-op-deployer - filters: - tags: - only: /^op-deployer.*/ - branches: - ignore: /.*/ - build_args: --skip test - context: - - circleci-repo-readonly-authenticated-github-token - requires: - - initialize - - go-release: - filters: - tags: - only: /^op-deployer.*/ - branches: - ignore: /.*/ - module: op-deployer - context: - - oplabs-gcr-release - - circleci-repo-readonly-authenticated-github-token - requires: - - build-contracts-go-release-op-deployer + # go-release-op-deployer: + # jobs: + # - initialize: + # filters: + # tags: + # only: /^op-deployer.*/ + # branches: + # ignore: /.*/ + # context: + # - circleci-repo-readonly-authenticated-github-token + # - contracts-bedrock-build: + # name: build-contracts-go-release-op-deployer + # filters: + # tags: + # only: /^op-deployer.*/ + # branches: + # ignore: /.*/ + # build_args: --skip test + # context: + # - circleci-repo-readonly-authenticated-github-token + # requires: + # - initialize + # - go-release: + # filters: + # tags: + # only: /^op-deployer.*/ + # branches: + # ignore: /.*/ + # module: op-deployer + # context: + # - oplabs-gcr-release + # - circleci-repo-readonly-authenticated-github-token + # requires: + # - build-contracts-go-release-op-deployer   - go-release-op-up: - jobs: - - initialize: - filters: - tags: - only: /^op-up.*/ - branches: - ignore: /.*/ - context: - - circleci-repo-readonly-authenticated-github-token - - contracts-bedrock-build: - name: build-contracts-go-release-op-up - filters: - tags: - only: /^op-up.*/ - branches: - ignore: /.*/ - build_args: --skip test - context: - - circleci-repo-readonly-authenticated-github-token - requires: - - initialize - - go-release: - filters: - tags: - only: /^op-up.*/ - branches: - ignore: /.*/ - module: op-up - context: - - oplabs-gcr-release - - circleci-repo-readonly-authenticated-github-token - requires: - - build-contracts-go-release-op-up + # go-release-op-up: + # jobs: + # - initialize: + # filters: + # tags: + # only: /^op-up.*/ + # branches: + # ignore: /.*/ + # context: + # - circleci-repo-readonly-authenticated-github-token + # - contracts-bedrock-build: + # name: build-contracts-go-release-op-up + # filters: + # tags: + # only: /^op-up.*/ + # branches: + # ignore: /.*/ + # build_args: --skip test + # context: + # - circleci-repo-readonly-authenticated-github-token + # requires: + # - initialize + # - go-release: + # filters: + # tags: + # only: /^op-up.*/ + # branches: + # ignore: /.*/ + # module: op-up + # context: + # - oplabs-gcr-release + # - circleci-repo-readonly-authenticated-github-token + # requires: + # - build-contracts-go-release-op-up   - release: - when: - not: - equal: [scheduled_pipeline, << pipeline.trigger_source >>] - jobs: - - initialize: - context: - - circleci-repo-readonly-authenticated-github-token - filters: - tags: - only: /^(da-server|cannon|ufm-[a-z0-9\-]*|op-[a-z0-9\-]*)\/v.*/ - branches: - ignore: /.*/ - - contracts-bedrock-build: - context: - - circleci-repo-readonly-authenticated-github-token - requires: - - initialize - filters: - tags: - only: /^op-deployer.*/ # ensure contract artifacts are embedded in op-deployer binary - branches: - ignore: /.*/ - # Standard (medium) cross-platform docker images go here - - docker-build: - matrix: - parameters: - docker_name: - - op-node - - op-batcher - - op-deployer - - op-faucet - - op-proposer - - op-challenger - - op-dispute-mon - - op-conductor - - da-server - - op-ufm - - op-supervisor - - op-supernode - - op-test-sequencer - - cannon - - op-dripper - - op-interop-mon - name: <<matrix.docker_name>>-docker-release - docker_tags: <<pipeline.git.revision>> - platforms: "linux/amd64,linux/arm64" - publish: true - release: true - filters: - tags: - only: /^<<matrix.docker_name>>\/v.*/ - branches: - ignore: /.*/ - context: - - oplabs-gcr-release - requires: - - initialize - - contracts-bedrock-build - # Checks for cross-platform images go here - - check-cross-platform: - matrix: - parameters: - op_component: - - op-node - - op-batcher - - op-deployer - - op-faucet - - op-proposer - - op-challenger - - op-dispute-mon - - op-conductor - - da-server - - op-ufm - - op-supervisor - - op-supernode - - op-test-sequencer - - op-deployer - - cannon - - op-dripper - - op-interop-mon - name: <<matrix.op_component>>-cross-platform - requires: - - op-node-docker-release - - op-batcher-docker-release - - op-deployer-docker-release - - op-faucet-docker-release - - op-proposer-docker-release - - op-challenger-docker-release - - op-dispute-mon-docker-release - - op-conductor-docker-release - - da-server-docker-release - - op-ufm-docker-release - - op-supervisor-docker-release - - op-supernode-docker-release - - op-test-sequencer-docker-release - - cannon-docker-release - - op-dripper-docker-release - - op-interop-mon-docker-release - context: - - circleci-repo-readonly-authenticated-github-token - - cannon-prestate: - filters: - tags: - only: /^op-program\/v.*/ - branches: - ignore: /.*/ - context: - - circleci-repo-readonly-authenticated-github-token - requires: - - initialize - - publish-cannon-prestates: - context: - - circleci-repo-readonly-authenticated-github-token - - slack - - oplabs-network-optimism-io-bucket - requires: - - cannon-prestate - filters: - tags: - only: /^op-program\/v.*/ - branches: - ignore: /.*/ + # release: + # when: + # not: + # equal: [scheduled_pipeline, << pipeline.trigger_source >>] + # jobs: + # - initialize: + # context: + # - circleci-repo-readonly-authenticated-github-token + # filters: + # tags: + # only: /^(da-server|cannon|ufm-[a-z0-9\-]*|op-[a-z0-9\-]*)\/v.*/ + # branches: + # ignore: /.*/ + # - contracts-bedrock-build: + # context: + # - circleci-repo-readonly-authenticated-github-token + # requires: + # - initialize + # filters: + # tags: + # only: /^op-deployer.*/ # ensure contract artifacts are embedded in op-deployer binary + # branches: + # ignore: /.*/ + # # Standard (medium) cross-platform docker images go here + # - docker-build: + # matrix: + # parameters: + # docker_name: + # - op-node + # - op-batcher + # - op-deployer + # - op-faucet + # - op-proposer + # - op-challenger + # - op-dispute-mon + # - op-conductor + # - da-server + # - op-ufm + # - op-supervisor + # - op-supernode + # - op-test-sequencer + # - cannon + # - op-dripper + # - op-interop-mon + # name: <<matrix.docker_name>>-docker-release + # docker_tags: <<pipeline.git.revision>> + # platforms: "linux/amd64,linux/arm64" + # publish: true + # release: true + # filters: + # tags: + # only: /^<<matrix.docker_name>>\/v.*/ + # branches: + # ignore: /.*/ + # context: + # - oplabs-gcr-release + # requires: + # - initialize + # - contracts-bedrock-build + # # Checks for cross-platform images go here + # - check-cross-platform: + # matrix: + # parameters: + # op_component: + # - op-node + # - op-batcher + # - op-deployer + # - op-faucet + # - op-proposer + # - op-challenger + # - op-dispute-mon + # - op-conductor + # - da-server + # - op-ufm + # - op-supervisor + # - op-supernode + # - op-test-sequencer + # - op-deployer + # - cannon + # - op-dripper + # - op-interop-mon + # name: <<matrix.op_component>>-cross-platform + # requires: + # - op-node-docker-release + # - op-batcher-docker-release + # - op-deployer-docker-release + # - op-faucet-docker-release + # - op-proposer-docker-release + # - op-challenger-docker-release + # - op-dispute-mon-docker-release + # - op-conductor-docker-release + # - da-server-docker-release + # - op-ufm-docker-release + # - op-supervisor-docker-release + # - op-supernode-docker-release + # - op-test-sequencer-docker-release + # - cannon-docker-release + # - op-dripper-docker-release + # - op-interop-mon-docker-release + # context: + # - circleci-repo-readonly-authenticated-github-token + # - cannon-prestate: + # filters: + # tags: + # only: /^op-program\/v.*/ + # branches: + # ignore: /.*/ + # context: + # - circleci-repo-readonly-authenticated-github-token + # requires: + # - initialize + # - publish-cannon-prestates: + # context: + # - circleci-repo-readonly-authenticated-github-token + # - slack + # - oplabs-network-optimism-io-bucket + # requires: + # - cannon-prestate + # filters: + # tags: + # only: /^op-program\/v.*/ + # branches: + # ignore: /.*/   - scheduled-todo-issues: - when: - equal: [build_four_hours, <<pipeline.schedule.name>>] - jobs: - - todo-issues: - name: todo-issue-checks - context: - - slack - - circleci-repo-readonly-authenticated-github-token + # scheduled-todo-issues: + # when: + # equal: [build_four_hours, <<pipeline.schedule.name>>] + # jobs: + # - todo-issues: + # name: todo-issue-checks + # context: + # - slack + # - circleci-repo-readonly-authenticated-github-token   - develop-publish-contract-artifacts: - when: - or: - - and: - - equal: ["develop", <<pipeline.git.branch>>] - - equal: ["webhook", << pipeline.trigger_source >>] - - and: - - equal: - [ - true, - <<pipeline.parameters.publish_contract_artifacts_dispatch>>, - ] - - equal: ["api", << pipeline.trigger_source >>] - jobs: - - publish-contract-artifacts: - context: - - circleci-repo-readonly-authenticated-github-token + # develop-publish-contract-artifacts: + # when: + # or: + # - and: + # - equal: ["develop", <<pipeline.git.branch>>] + # - equal: ["webhook", << pipeline.trigger_source >>] + # - and: + # - equal: + # [ + # true, + # <<pipeline.parameters.publish_contract_artifacts_dispatch>>, + # ] + # - equal: ["api", << pipeline.trigger_source >>] + # jobs: + # - publish-contract-artifacts: + # context: + # - circleci-repo-readonly-authenticated-github-token   - develop-fault-proofs: - when: - or: - - and: - - equal: ["develop", <<pipeline.git.branch>>] - - equal: ["webhook", << pipeline.trigger_source >>] - - and: - - equal: [true, <<pipeline.parameters.fault_proofs_dispatch>>] - - equal: ["api", << pipeline.trigger_source >>] - jobs: - - initialize: - context: - - circleci-repo-readonly-authenticated-github-token - - cannon-prestate: - context: - - circleci-repo-readonly-authenticated-github-token - requires: - - initialize - - cannon-stf-verify: - context: - - slack - - circleci-repo-readonly-authenticated-github-token - - contracts-bedrock-build: - build_args: --deny-warnings --skip test - context: - - slack - - circleci-repo-readonly-authenticated-github-token - requires: - - initialize - - go-tests-with-fault-proof-deps: - name: op-e2e-cannon-tests - notify: true - mentions: "@proofs-team" - no_output_timeout: 90m - test_timeout: 480m - resource_class: ethereum-optimism/latitude-fps-1 - context: - - slack - - circleci-repo-readonly-authenticated-github-token - requires: - - contracts-bedrock-build - - cannon-prestate - - publish-cannon-prestates: - context: - - slack - - oplabs-network-optimism-io-bucket - - circleci-repo-readonly-authenticated-github-token - requires: - - cannon-prestate - - op-e2e-cannon-tests + # develop-fault-proofs: + # when: + # or: + # - and: + # - equal: ["develop", <<pipeline.git.branch>>] + # - equal: ["webhook", << pipeline.trigger_source >>] + # - and: + # - equal: [true, <<pipeline.parameters.fault_proofs_dispatch>>] + # - equal: ["api", << pipeline.trigger_source >>] + # jobs: + # - initialize: + # context: + # - circleci-repo-readonly-authenticated-github-token + # - cannon-prestate: + # context: + # - circleci-repo-readonly-authenticated-github-token + # requires: + # - initialize + # - cannon-stf-verify: + # context: + # - slack + # - circleci-repo-readonly-authenticated-github-token + # - contracts-bedrock-build: + # build_args: --deny-warnings --skip test + # context: + # - slack + # - circleci-repo-readonly-authenticated-github-token + # requires: + # - initialize + # - go-tests-with-fault-proof-deps: + # name: op-e2e-cannon-tests + # notify: true + # mentions: "@proofs-team" + # no_output_timeout: 3h + # test_timeout: 3h + # resource_class: qkc/ax101 + # context: + # - slack + # - circleci-repo-readonly-authenticated-github-token + # requires: + # - contracts-bedrock-build + # - cannon-prestate + # - publish-cannon-prestates: + # context: + # - slack + # - oplabs-network-optimism-io-bucket + # - circleci-repo-readonly-authenticated-github-token + # requires: + # - cannon-prestate + # - op-e2e-cannon-tests   - develop-kontrol-tests: - when: - or: - - and: - - equal: ["develop", <<pipeline.git.branch>>] - - equal: ["webhook", << pipeline.trigger_source >>] - - and: - - equal: [true, <<pipeline.parameters.kontrol_dispatch>>] - - equal: ["api", << pipeline.trigger_source >>] - jobs: - - kontrol-tests: - context: - - slack - - runtimeverification - - circleci-repo-readonly-authenticated-github-token + # develop-kontrol-tests: + # when: + # or: + # - and: + # - equal: ["develop", <<pipeline.git.branch>>] + # - equal: ["webhook", << pipeline.trigger_source >>] + # - and: + # - equal: [true, <<pipeline.parameters.kontrol_dispatch>>] + # - equal: ["api", << pipeline.trigger_source >>] + # jobs: + # - kontrol-tests: + # context: + # - slack + # - runtimeverification + # - circleci-repo-readonly-authenticated-github-token   - scheduled-cannon-full-tests: - when: - or: - - equal: [build_four_hours, <<pipeline.schedule.name>>] - - equal: [true, << pipeline.parameters.cannon_full_test_dispatch >>] - jobs: - - initialize: - context: - - circleci-repo-readonly-authenticated-github-token - - contracts-bedrock-build: - build_args: --deny-warnings --skip test - context: - - circleci-repo-readonly-authenticated-github-token - requires: - - initialize - - cannon-go-lint-and-test: - requires: - - contracts-bedrock-build - skip_slow_tests: false - no_output_timeout: 30m - notify: true - context: - - slack - - circleci-repo-readonly-authenticated-github-token + # scheduled-cannon-full-tests: + # when: + # or: + # - equal: [build_four_hours, <<pipeline.schedule.name>>] + # - equal: [true, << pipeline.parameters.cannon_full_test_dispatch >>] + # jobs: + # - initialize: + # context: + # - circleci-repo-readonly-authenticated-github-token + # - contracts-bedrock-build: + # build_args: --deny-warnings --skip test + # context: + # - circleci-repo-readonly-authenticated-github-token + # requires: + # - initialize + # - cannon-go-lint-and-test: + # requires: + # - contracts-bedrock-build + # skip_slow_tests: false + # no_output_timeout: 30m + # notify: true + # context: + # - slack + # - circleci-repo-readonly-authenticated-github-token   - scheduled-docker-publish: - when: - or: - - equal: [build_daily, <<pipeline.schedule.name>>] - # Trigger on manual triggers if explicitly requested - - equal: [true, << pipeline.parameters.docker_publish_dispatch >>] - jobs: - - initialize: - context: - - circleci-repo-readonly-authenticated-github-token - - contracts-bedrock-build: - context: - - circleci-repo-readonly-authenticated-github-token - requires: - - initialize - - docker-build: - matrix: - parameters: - docker_name: - - op-node - - op-batcher - - op-deployer - - op-faucet - - op-program - - op-proposer - - op-challenger - - op-dispute-mon - - op-conductor - - op-supervisor - - op-supernode - - op-test-sequencer - - cannon - - op-dripper - - op-interop-mon - name: <<matrix.docker_name>>-docker-publish - docker_tags: <<pipeline.git.revision>>,<<pipeline.git.branch>> - platforms: "linux/amd64,linux/arm64" - publish: true - context: - - oplabs-gcr - - slack - - circleci-repo-readonly-authenticated-github-token - requires: - - initialize - - contracts-bedrock-build - - check-cross-platform: - matrix: - parameters: - op_component: - - op-node - - op-batcher - - op-deployer - - op-faucet - - op-program - - op-proposer - - op-challenger - - op-dispute-mon - - op-conductor - - op-supervisor - - op-supernode - - op-test-sequencer - - cannon - - op-dripper - - op-interop-mon - name: <<matrix.op_component>>-cross-platform - requires: - - <<matrix.op_component>>-docker-publish - context: - - circleci-repo-readonly-authenticated-github-token + # scheduled-docker-publish: + # when: + # or: + # - equal: [build_daily, <<pipeline.schedule.name>>] + # # Trigger on manual triggers if explicitly requested + # - equal: [true, << pipeline.parameters.docker_publish_dispatch >>] + # jobs: + # - initialize: + # context: + # - circleci-repo-readonly-authenticated-github-token + # - contracts-bedrock-build: + # context: + # - circleci-repo-readonly-authenticated-github-token + # requires: + # - initialize + # - docker-build: + # matrix: + # parameters: + # docker_name: + # - op-node + # - op-batcher + # - op-deployer + # - op-faucet + # - op-program + # - op-proposer + # - op-challenger + # - op-dispute-mon + # - op-conductor + # - op-supervisor + # - op-supernode + # - op-test-sequencer + # - cannon + # - op-dripper + # - op-interop-mon + # name: <<matrix.docker_name>>-docker-publish + # docker_tags: <<pipeline.git.revision>>,<<pipeline.git.branch>> + # platforms: "linux/amd64,linux/arm64" + # publish: true + # context: + # - oplabs-gcr + # - slack + # - circleci-repo-readonly-authenticated-github-token + # requires: + # - initialize + # - contracts-bedrock-build + # - check-cross-platform: + # matrix: + # parameters: + # op_component: + # - op-node + # - op-batcher + # - op-deployer + # - op-faucet + # - op-program + # - op-proposer + # - op-challenger + # - op-dispute-mon + # - op-conductor + # - op-supervisor + # - op-supernode + # - op-test-sequencer + # - cannon + # - op-dripper + # - op-interop-mon + # name: <<matrix.op_component>>-cross-platform + # requires: + # - <<matrix.op_component>>-docker-publish + # context: + # - circleci-repo-readonly-authenticated-github-token   - scheduled-flake-shake: - when: - or: - - equal: [build_daily, << pipeline.schedule.name >>] - - and: - - equal: [true, << pipeline.parameters.flake-shake-dispatch >>] - - equal: ["api", << pipeline.trigger_source >>] - jobs: - - initialize: - context: - - circleci-repo-readonly-authenticated-github-token - - contracts-bedrock-build: - build_args: --skip test - context: - - circleci-repo-readonly-authenticated-github-token - requires: - - initialize - - cannon-prestate-quick: - context: - - circleci-repo-readonly-authenticated-github-token - requires: - - initialize - - op-acceptance-tests-flake-shake: - context: - - circleci-repo-readonly-authenticated-github-token - requires: - - contracts-bedrock-build - - cannon-prestate-quick - - op-acceptance-tests-flake-shake-report: - requires: - - op-acceptance-tests-flake-shake - - op-acceptance-tests-flake-shake-promote: - requires: - - op-acceptance-tests-flake-shake-report - context: - - circleci-repo-readonly-authenticated-github-token - - circleci-repo-optimism - - circleci-api-token - - slack + # scheduled-flake-shake: + # when: + # or: + # - equal: [build_daily, << pipeline.schedule.name >>] + # - and: + # - equal: [true, << pipeline.parameters.flake-shake-dispatch >>] + # - equal: ["api", << pipeline.trigger_source >>] + # jobs: + # - initialize: + # context: + # - circleci-repo-readonly-authenticated-github-token + # - contracts-bedrock-build: + # build_args: --skip test + # context: + # - circleci-repo-readonly-authenticated-github-token + # requires: + # - initialize + # - cannon-prestate-quick: + # context: + # - circleci-repo-readonly-authenticated-github-token + # requires: + # - initialize + # - op-acceptance-tests-flake-shake: + # context: + # - circleci-repo-readonly-authenticated-github-token + # requires: + # - contracts-bedrock-build + # - cannon-prestate-quick + # - op-acceptance-tests-flake-shake-report: + # requires: + # - op-acceptance-tests-flake-shake + # - op-acceptance-tests-flake-shake-promote: + # requires: + # - op-acceptance-tests-flake-shake-report + # context: + # - circleci-repo-readonly-authenticated-github-token + # - circleci-repo-optimism + # - circleci-api-token + # - slack   - scheduled-preimage-reproducibility: - when: - or: - - equal: [build_daily, <<pipeline.schedule.name>>] - # Trigger on manual triggers if explicitly requested - - equal: [true, << pipeline.parameters.reproducibility_dispatch >>] - jobs: - - preimage-reproducibility: - context: - - slack - - circleci-repo-readonly-authenticated-github-token + # scheduled-preimage-reproducibility: + # when: + # or: + # - equal: [build_daily, <<pipeline.schedule.name>>] + # # Trigger on manual triggers if explicitly requested + # - equal: [true, << pipeline.parameters.reproducibility_dispatch >>] + # jobs: + # - preimage-reproducibility: + # context: + # - slack + # - circleci-repo-readonly-authenticated-github-token   - scheduled-stale-check: - when: - or: - - equal: [build_daily, <<pipeline.schedule.name>>] - # Trigger on manual triggers if explicitly requested - - equal: [true, << pipeline.parameters.stale_check_dispatch >>] - jobs: - - stale-check: - context: - - circleci-repo-optimism + # scheduled-stale-check: + # when: + # or: + # - equal: [build_daily, <<pipeline.schedule.name>>] + # # Trigger on manual triggers if explicitly requested + # - equal: [true, << pipeline.parameters.stale_check_dispatch >>] + # jobs: + # - stale-check: + # context: + # - circleci-repo-optimism   - scheduled-sync-test-op-node: - when: - or: - - equal: [build_daily, <<pipeline.schedule.name>>] - # Trigger on manual triggers if explicitly requested - - equal: [true, << pipeline.parameters.sync_test_op_node_dispatch >>] - jobs: - - initialize: - context: - - circleci-repo-readonly-authenticated-github-token - - contracts-bedrock-build: # needed for sysgo tests - build_args: --skip test - context: - - circleci-repo-readonly-authenticated-github-token - requires: - - initialize - - cannon-prestate-quick: # needed for sysgo tests - context: - - circleci-repo-readonly-authenticated-github-token - requires: - - initialize - - op-acceptance-sync-tests-docker: - name: "sync-test-<<matrix.network_preset>>-daily-<<matrix.l2_cl_syncmode>>" - gate: sync-test-op-node - no_output_timeout: 30m - context: - - circleci-repo-readonly-authenticated-github-token - - discord - requires: - - contracts-bedrock-build - - cannon-prestate-quick - matrix: - parameters: - network_preset: ["op-sepolia", "base-sepolia", "unichain-sepolia", "op-mainnet", "base-mainnet"] - l2_cl_syncmode: ["consensus-layer", "execution-layer"] + # scheduled-sync-test-op-node: + # when: + # or: + # - equal: [build_daily, <<pipeline.schedule.name>>] + # # Trigger on manual triggers if explicitly requested + # - equal: [true, << pipeline.parameters.sync_test_op_node_dispatch >>] + # jobs: + # - initialize: + # context: + # - circleci-repo-readonly-authenticated-github-token + # - contracts-bedrock-build: # needed for sysgo tests + # build_args: --skip test + # context: + # - circleci-repo-readonly-authenticated-github-token + # requires: + # - initialize + # - cannon-prestate-quick: # needed for sysgo tests + # context: + # - circleci-repo-readonly-authenticated-github-token + # requires: + # - initialize + # - op-acceptance-sync-tests-docker: + # name: "sync-test-<<matrix.network_preset>>-daily-<<matrix.l2_cl_syncmode>>" + # gate: sync-test-op-node + # no_output_timeout: 30m + # context: + # - circleci-repo-readonly-authenticated-github-token + # - discord + # requires: + # - contracts-bedrock-build + # - cannon-prestate-quick + # matrix: + # parameters: + # network_preset: ["op-sepolia", "base-sepolia", "unichain-sepolia", "op-mainnet", "base-mainnet"] + # l2_cl_syncmode: ["consensus-layer", "execution-layer"]   - scheduled-heavy-fuzz-tests: - when: - or: - - equal: [build_daily, <<pipeline.schedule.name>>] - - equal: [true, << pipeline.parameters.heavy_fuzz_dispatch >>] - jobs: - - initialize: - context: - - circleci-repo-readonly-authenticated-github-token - - contracts-bedrock-heavy-fuzz-nightly: - context: - - slack - - circleci-repo-readonly-authenticated-github-token - requires: - - initialize + # scheduled-heavy-fuzz-tests: + # when: + # or: + # - equal: [build_daily, <<pipeline.schedule.name>>] + # - equal: [true, << pipeline.parameters.heavy_fuzz_dispatch >>] + # jobs: + # - initialize: + # context: + # - circleci-repo-readonly-authenticated-github-token + # - contracts-bedrock-heavy-fuzz-nightly: + # context: + # - slack + # - circleci-repo-readonly-authenticated-github-token + # requires: + # - initialize   - # Acceptance tests - acceptance-tests: - when: - or: - - equal: ["webhook", << pipeline.trigger_source >>] - # Manual dispatch - - and: - - equal: [true, <<pipeline.parameters.acceptance_tests_dispatch>>] - - equal: ["api", << pipeline.trigger_source >>] - jobs: - - initialize: - context: - - circleci-repo-readonly-authenticated-github-token - - contracts-bedrock-build: # needed for sysgo tests - build_args: --skip test - context: - - circleci-repo-readonly-authenticated-github-token - requires: - - initialize - - cannon-prestate-quick: # needed for sysgo tests - context: - - circleci-repo-readonly-authenticated-github-token - requires: - - initialize - # IN-MEMORY (all) - - op-acceptance-tests: - name: memory-all - gate: "" # Empty gate = gateless mode - no_output_timeout: 90m - context: - - circleci-repo-readonly-authenticated-github-token - - discord - requires: - - contracts-bedrock-build - - cannon-prestate-quick - # Generate flaky test report - - generate-flaky-report: - name: generate-flaky-tests-report - context: - - circleci-repo-readonly-authenticated-github-token - - circleci-api-token + # # Acceptance tests + # acceptance-tests: + # when: + # or: + # - equal: ["webhook", << pipeline.trigger_source >>] + # # Manual dispatch + # - and: + # - equal: [true, <<pipeline.parameters.acceptance_tests_dispatch>>] + # - equal: ["api", << pipeline.trigger_source >>] + # jobs: + # - initialize: + # context: + # - circleci-repo-readonly-authenticated-github-token + # - contracts-bedrock-build: # needed for sysgo tests + # build_args: --skip test + # context: + # - circleci-repo-readonly-authenticated-github-token + # requires: + # - initialize + # - cannon-prestate-quick: # needed for sysgo tests + # context: + # - circleci-repo-readonly-authenticated-github-token + # requires: + # - initialize + # # IN-MEMORY (all) + # - op-acceptance-tests: + # name: memory-all + # gate: "" # Empty gate = gateless mode + # no_output_timeout: 90m + # context: + # - circleci-repo-readonly-authenticated-github-token + # - discord + # requires: + # - contracts-bedrock-build + # - cannon-prestate-quick + # # Generate flaky test report + # - generate-flaky-report: + # name: generate-flaky-tests-report + # context: + # - circleci-repo-readonly-authenticated-github-token + # - circleci-api-token   - close-issue-workflow: - when: - and: - - equal: [<< pipeline.trigger_source >>, "api"] - - equal: [<< pipeline.parameters.github-event-type >>, "pull_request"] - - equal: [<< pipeline.parameters.github-event-action >>, "labeled"] - jobs: - - close-issue: - label_name: "auto-close-trivial-contribution" - message: "Thank you for your interest in contributing! - At this time, we are not accepting contributions that primarily fix spelling, stylistic, or grammatical errors in documentation, code, or elsewhere. - Please check our [contribution guidelines](https://github.com/ethereum-optimism/optimism/blob/develop/CONTRIBUTING.md#contributions-related-to-spelling-and-grammar) for more information. - This issue will be closed now." - context: - - circleci-repo-optimism + # close-issue-workflow: + # when: + # and: + # - equal: [<< pipeline.trigger_source >>, "api"] + # - equal: [<< pipeline.parameters.github-event-type >>, "pull_request"] + # - equal: [<< pipeline.parameters.github-event-action >>, "labeled"] + # jobs: + # - close-issue: + # label_name: "auto-close-trivial-contribution" + # message: "Thank you for your interest in contributing! + # At this time, we are not accepting contributions that primarily fix spelling, stylistic, or grammatical errors in documentation, code, or elsewhere. + # Please check our [contribution guidelines](https://github.com/ethereum-optimism/optimism/blob/develop/CONTRIBUTING.md#contributions-related-to-spelling-and-grammar) for more information. + # This issue will be closed now." + # context: + # - circleci-repo-optimism   - devnet-metrics-collect: - when: - or: - - equal: [<< pipeline.trigger_source >>, "webhook"] - - and: - - equal: [true, << pipeline.parameters.devnet-metrics-collect >>] - - equal: [<< pipeline.trigger_source >>, "api"] - jobs: - - devnet-metrics-collect-authorship: - context: - - circleci-repo-readonly-authenticated-github-token - - oplabs-tools-data-public-metrics-bucket + # devnet-metrics-collect: + # when: + # or: + # - equal: [<< pipeline.trigger_source >>, "webhook"] + # - and: + # - equal: [true, << pipeline.parameters.devnet-metrics-collect >>] + # - equal: [<< pipeline.trigger_source >>, "api"] + # jobs: + # - devnet-metrics-collect-authorship: + # context: + # - circleci-repo-readonly-authenticated-github-token + # - oplabs-tools-data-public-metrics-bucket   - ai-contracts-test-workflow: - when: - or: - - equal: [build_mon_thu, <<pipeline.schedule.name>>] - - equal: [true, << pipeline.parameters.ai_contracts_test_dispatch >>] - jobs: - - initialize: - context: - - circleci-repo-readonly-authenticated-github-token - - ai-contracts-test: - context: - - circleci-repo-readonly-authenticated-github-token - - circleci-api-token - - devin-api - - slack - requires: - - initialize + # ai-contracts-test-workflow: + # when: + # or: + # - equal: [build_mon_thu, <<pipeline.schedule.name>>] + # - equal: [true, << pipeline.parameters.ai_contracts_test_dispatch >>] + # jobs: + # - initialize: + # context: + # - circleci-repo-readonly-authenticated-github-token + # - ai-contracts-test: + # context: + # - circleci-repo-readonly-authenticated-github-token + # - circleci-api-token + # - devin-api + # - slack + # requires: + # - initialize
diff --git official optimism/.github/workflows/publish-op-challenger.yml SWC optimism/.github/workflows/publish-op-challenger.yml new file mode 100644 index 0000000000000000000000000000000000000000..e7bf2f6a000b0e138bb17d421a002f0fea1d7379 --- /dev/null +++ SWC optimism/.github/workflows/publish-op-challenger.yml @@ -0,0 +1,71 @@ +# This workflow will publish a github release for op-challenger + +name: Publish +run-name: ${{ github.actor }} is publishing an op-challenger release 🚀 +on: + push: + tags: + - 'op-challenger/v*' + +# Always wait for previous release to finish before releasing again +concurrency: ${{ github.workflow }}-${{ github.ref }} + + +jobs: + build: + runs-on: ubuntu-latest + strategy: + matrix: + os: [ubuntu-latest, macos-latest] + arch: [amd64, arm64] + exclude: + - os: ubuntu-latest + arch: arm64 + env: + BUILD_DIR: ${{ github.ref_name }} + BIN_DIR: ${{ github.ref_name }}/build/bin + steps: + - name: Set file name + run: | + # replace "op-challenger/v*" with "op-challenger.v*" since `actions/upload-artifact@v4` doesn't support slashes in file names + FILE_NAME="op-challenger.${GITHUB_REF_NAME:14}.${{ matrix.os == 'ubuntu-latest' && 'linux' || 'darwin' }}-${{ matrix.arch }}.tar.gz" + echo "FILE_NAME=$FILE_NAME" >> $GITHUB_ENV + - name: Checkout code + uses: actions/checkout@v4 + + - name: Setup go + uses: actions/setup-go@v5 + with: + go-version: '1.22.7' + + - uses: extractions/setup-just@v2 + + - name: Build + run: | + TARGETOS=${{ matrix.os == 'ubuntu-latest' && 'linux' || 'darwin' }} TARGETARCH=${{ matrix.arch }} make op-challenger + mkdir -p ${{ env.BIN_DIR }} + mv op-challenger/bin/op-challenger ${{ env.BIN_DIR }}/ + tar -czvf ${{ env.FILE_NAME }} ${{ env.BUILD_DIR }} + + - name: Upload artifact + uses: actions/upload-artifact@v4 + with: + name: ${{ env.FILE_NAME }} + path: ${{ env.FILE_NAME }} + + release: + needs: build + runs-on: ubuntu-latest + steps: + - name: Download artifact + uses: actions/download-artifact@v4 + + - name: Create release + uses: softprops/action-gh-release@v2 + with: + tag_name: ${{ github.ref }} + name: Release ${{ github.ref_name }} + files: | + **/* + fail_on_unmatched_files: true + generate_release_notes: true
diff --git official optimism/.github/workflows/publish-op-node.yml SWC optimism/.github/workflows/publish-op-node.yml new file mode 100644 index 0000000000000000000000000000000000000000..6b0a2e9d91e6cd8c9a5b2f4e73fcc0017f11fba2 --- /dev/null +++ SWC optimism/.github/workflows/publish-op-node.yml @@ -0,0 +1,71 @@ +# This workflow will publish a github release for op-node + +name: Publish +run-name: ${{ github.actor }} is publishing an op-node release 🚀 +on: + push: + tags: + - 'op-node/v*' + +# Always wait for previous release to finish before releasing again +concurrency: ${{ github.workflow }}-${{ github.ref }} + + +jobs: + build: + runs-on: ubuntu-latest + strategy: + matrix: + os: [ubuntu-latest, macos-latest] + arch: [amd64, arm64] + exclude: + - os: ubuntu-latest + arch: arm64 + env: + BUILD_DIR: ${{ github.ref_name }} + BIN_DIR: ${{ github.ref_name }}/build/bin + steps: + - name: Set file name + run: | + # replace "op-node/v*" with "op-node.v*" since `actions/upload-artifact@v4` doesn't support slashes in file names + FILE_NAME="op-node.${GITHUB_REF_NAME:8}.${{ matrix.os == 'ubuntu-latest' && 'linux' || 'darwin' }}-${{ matrix.arch }}.tar.gz" + echo "FILE_NAME=$FILE_NAME" >> $GITHUB_ENV + - name: Checkout code + uses: actions/checkout@v4 + + - name: Setup go + uses: actions/setup-go@v5 + with: + go-version: '1.22.7' + + - uses: extractions/setup-just@v2 + + - name: Build + run: | + TARGETOS=${{ matrix.os == 'ubuntu-latest' && 'linux' || 'darwin' }} TARGETARCH=${{ matrix.arch }} make op-node + mkdir -p ${{ env.BIN_DIR }} + mv op-node/bin/op-node ${{ env.BIN_DIR }}/ + tar -czvf ${{ env.FILE_NAME }} ${{ env.BUILD_DIR }} + + - name: Upload artifact + uses: actions/upload-artifact@v4 + with: + name: ${{ env.FILE_NAME }} + path: ${{ env.FILE_NAME }} + + release: + needs: build + runs-on: ubuntu-latest + steps: + - name: Download artifact + uses: actions/download-artifact@v4 + + - name: Create release + uses: softprops/action-gh-release@v2 + with: + tag_name: ${{ github.ref }} + name: Release ${{ github.ref_name }} + files: | + **/* + fail_on_unmatched_files: true + generate_release_notes: true
diff --git official optimism/go.mod SWC optimism/go.mod index 057eeba7802eb965861dd4503d7860ae0f8d3568..4b561a7d3c3517dcfe736b5a957cfef9478d70c5 100644 --- official optimism/go.mod +++ SWC optimism/go.mod @@ -23,6 +23,7 @@ github.com/docker/go-connections v0.5.0 github.com/ethereum-optimism/go-ethereum-hdwallet v0.1.4-0.20251001155152-4eb15ccedf7e github.com/ethereum-optimism/superchain-registry/validation v0.0.0-20251009180028-9b4658b9b7af github.com/ethereum/go-ethereum v1.16.3 + github.com/ethstorage/da-server v0.0.0-20241224013916-2bd2256b6a70 github.com/fatih/color v1.18.0 github.com/fsnotify/fsnotify v1.9.0 github.com/go-task/slim-sprig/v3 v3.0.0 @@ -308,7 +309,7 @@ gotest.tools/v3 v3.5.2 // indirect lukechampine.com/blake3 v1.3.0 // indirect )   -replace github.com/ethereum/go-ethereum => github.com/ethereum-optimism/op-geth v1.101603.4-rc.1 +replace github.com/ethereum/go-ethereum => github.com/Quarkchain/op-geth v0.0.0-20251111051036-0f7e8b6a9a0e   // replace github.com/ethereum/go-ethereum => ../op-geth
diff --git official optimism/go.sum SWC optimism/go.sum index daec5bd6f99446dadaadc334e77666f35c4994cb..82a0a013238f1f661ea1a97acb00c823546477e3 100644 --- official optimism/go.sum +++ SWC optimism/go.sum @@ -39,6 +39,8 @@ github.com/Microsoft/go-winio v0.6.2/go.mod h1:yd8OoFMLzJbo9gZq8j5qaps8bJ9aShtEA8Ipt1oGCvU= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= github.com/ProtonMail/go-crypto v0.0.0-20230217124315-7d5c6f04bbb8 h1:wPbRQzjjwFc0ih8puEVAOFGELsn1zoIIYdxvML7mDxA= github.com/ProtonMail/go-crypto v0.0.0-20230217124315-7d5c6f04bbb8/go.mod h1:I0gYDMZ6Z5GRU7l58bNFSkPTFN6Yl12dsUlAZ8xy98g= +github.com/Quarkchain/op-geth v0.0.0-20251111051036-0f7e8b6a9a0e h1:bY2Ie0loZkBSpne/2utW4M1YMq8JMkNH1pXvhjLQVwU= +github.com/Quarkchain/op-geth v0.0.0-20251111051036-0f7e8b6a9a0e/go.mod h1:cnGR2M8zX91+rRQxXyNTEOEpw/IwdR8P11FQX7Xaqwk= github.com/VictoriaMetrics/fastcache v1.12.2 h1:N0y9ASrJ0F6h0QaC3o6uJb3NIZ9VKLjCM7NQbSmF7WI= github.com/VictoriaMetrics/fastcache v1.12.2/go.mod h1:AmC+Nzz1+3G2eCPapF6UcsnkThDcMsQicp4xDukwJYI= github.com/adrg/xdg v0.4.0 h1:RzRqFcjH4nE5C6oTAxhBtoE2IRyjBSa62SCbyPidvls= @@ -238,14 +240,14 @@ github.com/emicklei/dot v1.6.2 h1:08GN+DD79cy/tzN6uLCT84+2Wk9u+wvqP+Hkx/dIR8A= github.com/emicklei/dot v1.6.2/go.mod h1:DeV7GvQtIw4h2u73RKBkkFdvVAz0D9fzeJrgPW6gy/s= github.com/ethereum-optimism/go-ethereum-hdwallet v0.1.4-0.20251001155152-4eb15ccedf7e h1:iy1vBIzACYUyOVyoADUwvAiq2eOPC0yVsDUdolPwQjk= github.com/ethereum-optimism/go-ethereum-hdwallet v0.1.4-0.20251001155152-4eb15ccedf7e/go.mod h1:DYj7+vYJ4cIB7zera9mv4LcAynCL5u4YVfoeUu6Wa+w= -github.com/ethereum-optimism/op-geth v1.101603.4-rc.1 h1:5ByBDUKBY/IHibukF/C9kjzaTk4SxUoXH449VuDrMn4= -github.com/ethereum-optimism/op-geth v1.101603.4-rc.1/go.mod h1:cnGR2M8zX91+rRQxXyNTEOEpw/IwdR8P11FQX7Xaqwk= github.com/ethereum-optimism/superchain-registry/validation v0.0.0-20251009180028-9b4658b9b7af h1:WWz0gJM/boaUImtJnROecPirAerKCLpAU4m6Tx0ArOg= github.com/ethereum-optimism/superchain-registry/validation v0.0.0-20251009180028-9b4658b9b7af/go.mod h1:NZ816PzLU1TLv1RdAvYAb6KWOj4Zm5aInT0YpDVml2Y= github.com/ethereum/c-kzg-4844/v2 v2.1.5 h1:aVtoLK5xwJ6c5RiqO8g8ptJ5KU+2Hdquf6G3aXiHh5s= github.com/ethereum/c-kzg-4844/v2 v2.1.5/go.mod h1:u59hRTTah4Co6i9fDWtiCjTrblJv0UwsqZKCc0GfgUs= github.com/ethereum/go-verkle v0.2.2 h1:I2W0WjnrFUIzzVPwm8ykY+7pL2d4VhlsePn4j7cnFk8= github.com/ethereum/go-verkle v0.2.2/go.mod h1:M3b90YRnzqKyyzBEWJGqj8Qff4IDeXnzFw0P9bFw3uk= +github.com/ethstorage/da-server v0.0.0-20241224013916-2bd2256b6a70 h1:g3H1N+WC/nDFL1H+rpKwsSKT0cCWu1fTHjKDqEMyuq0= +github.com/ethstorage/da-server v0.0.0-20241224013916-2bd2256b6a70/go.mod h1:4VnU8W5POGWvaUKd+SDns3cooFGU5wWpIu9M1KhEe1s= github.com/fatih/color v1.13.0/go.mod h1:kLAiJbzzSOZDVNGyDpeOxJ47H46qBXwg5ILebYFFOfk= github.com/fatih/color v1.18.0 h1:S8gINlzdQ840/4pfAwic/ZE0djQEH3wM94VfqLTZcOM= github.com/fatih/color v1.18.0/go.mod h1:4FelSpRwEGDpQ12mAdzqdOukCy4u8WUtOY6lkT/6HfU=
diff --git official optimism/packages/contracts-bedrock/snapshots/abi/L1Block.json SWC optimism/packages/contracts-bedrock/snapshots/abi/L1Block.json index 5f3ebef9dba14c4fd498765ad90a7c97841d2c67..f6dc3b436d05fc6d8baee155656462f875e15057 100644 --- official optimism/packages/contracts-bedrock/snapshots/abi/L1Block.json +++ SWC optimism/packages/contracts-bedrock/snapshots/abi/L1Block.json @@ -78,6 +78,25 @@ "stateMutability": "view", "type": "function" }, { + "inputs": [ + { + "internalType": "uint256", + "name": "_historyNumber", + "type": "uint256" + } + ], + "name": "blockHash", + "outputs": [ + { + "internalType": "bytes32", + "name": "", + "type": "bytes32" + } + ], + "stateMutability": "view", + "type": "function" + }, + { "inputs": [], "name": "daFootprintGasScalar", "outputs": [ @@ -145,6 +164,19 @@ "type": "bytes32" } ], "stateMutability": "view", + "type": "function" + }, + { + "inputs": [], + "name": "historySize", + "outputs": [ + { + "internalType": "uint256", + "name": "", + "type": "uint256" + } + ], + "stateMutability": "pure", "type": "function" }, {
diff --git official optimism/packages/contracts-bedrock/snapshots/abi/L2ToL1MessagePasser.json SWC optimism/packages/contracts-bedrock/snapshots/abi/L2ToL1MessagePasser.json index 77e1cf7596b35abe9699d06f505a98a448cfac62..998a46e7aa5dc2b6eea12c5d822758d4da0200be 100644 --- official optimism/packages/contracts-bedrock/snapshots/abi/L2ToL1MessagePasser.json +++ SWC optimism/packages/contracts-bedrock/snapshots/abi/L2ToL1MessagePasser.json @@ -79,6 +79,19 @@ "stateMutability": "view", "type": "function" }, { + "inputs": [ + { + "internalType": "bool", + "name": "_disable", + "type": "bool" + } + ], + "name": "setNativeDeposit", + "outputs": [], + "stateMutability": "nonpayable", + "type": "function" + }, + { "inputs": [], "name": "version", "outputs": [ @@ -142,6 +155,18 @@ "type": "event" }, { "anonymous": false, + "inputs": [], + "name": "NativeDepositDisabled", + "type": "event" + }, + { + "anonymous": false, + "inputs": [], + "name": "NativeDepositEnabled", + "type": "event" + }, + { + "anonymous": false, "inputs": [ { "indexed": true, @@ -152,5 +177,10 @@ } ], "name": "WithdrawerBalanceBurnt", "type": "event" + }, + { + "inputs": [], + "name": "L2ToL1MessagePasser_NativeDepositDisabled", + "type": "error" } ] \ No newline at end of file
diff --git official optimism/packages/contracts-bedrock/snapshots/abi/OptimismPortal2.json SWC optimism/packages/contracts-bedrock/snapshots/abi/OptimismPortal2.json index 49ae551310b014279feb63e6c281ed1739f4f357..e82e5e1f8e69940a6044534d099c34e710d7377e 100644 --- official optimism/packages/contracts-bedrock/snapshots/abi/OptimismPortal2.json +++ SWC optimism/packages/contracts-bedrock/snapshots/abi/OptimismPortal2.json @@ -336,6 +336,24 @@ }, { "inputs": [ { + "internalType": "address", + "name": "_to", + "type": "address" + }, + { + "internalType": "uint256", + "name": "_value", + "type": "uint256" + } + ], + "name": "mintTransaction", + "outputs": [], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [ + { "internalType": "bytes32", "name": "_withdrawalHash", "type": "bytes32" @@ -589,6 +607,32 @@ "stateMutability": "view", "type": "function" }, { + "inputs": [ + { + "internalType": "address", + "name": "_minter", + "type": "address" + } + ], + "name": "setMinter", + "outputs": [], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "bool", + "name": "_disable", + "type": "bool" + } + ], + "name": "setNativeDeposit", + "outputs": [], + "stateMutability": "nonpayable", + "type": "function" + }, + { "inputs": [], "name": "superchainConfig", "outputs": [ @@ -638,6 +682,31 @@ "type": "uint8" } ], "name": "Initialized", + "type": "event" + }, + { + "anonymous": false, + "inputs": [ + { + "indexed": true, + "internalType": "address", + "name": "minter", + "type": "address" + } + ], + "name": "MinterSet", + "type": "event" + }, + { + "anonymous": false, + "inputs": [], + "name": "NativeDepositDisabled", + "type": "event" + }, + { + "anonymous": false, + "inputs": [], + "name": "NativeDepositEnabled", "type": "event" }, { @@ -821,12 +890,22 @@ "type": "error" }, { "inputs": [], + "name": "OptimismPortal_NativeDepositForbidden", + "type": "error" + }, + { + "inputs": [], "name": "OptimismPortal_NoReentrancy", "type": "error" }, { "inputs": [], "name": "OptimismPortal_ProofNotOldEnough", + "type": "error" + }, + { + "inputs": [], + "name": "OptimismPortal_Unauthorized", "type": "error" }, {
diff --git official optimism/packages/contracts-bedrock/snapshots/abi/SoulGasToken.json SWC optimism/packages/contracts-bedrock/snapshots/abi/SoulGasToken.json new file mode 100644 index 0000000000000000000000000000000000000000..2b1da1119daa26dbde6e428c6196120ec44241b4 --- /dev/null +++ SWC optimism/packages/contracts-bedrock/snapshots/abi/SoulGasToken.json @@ -0,0 +1,692 @@ +[ + { + "inputs": [ + { + "internalType": "bool", + "name": "_isBackedByNative", + "type": "bool" + } + ], + "stateMutability": "nonpayable", + "type": "constructor" + }, + { + "inputs": [ + { + "internalType": "address[]", + "name": "_burners", + "type": "address[]" + } + ], + "name": "addBurners", + "outputs": [], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "address[]", + "name": "_minters", + "type": "address[]" + } + ], + "name": "addMinters", + "outputs": [], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "address[]", + "name": "_contracts", + "type": "address[]" + } + ], + "name": "allowSgtValue", + "outputs": [], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "address", + "name": "owner", + "type": "address" + }, + { + "internalType": "address", + "name": "spender", + "type": "address" + } + ], + "name": "allowance", + "outputs": [ + { + "internalType": "uint256", + "name": "", + "type": "uint256" + } + ], + "stateMutability": "view", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "address", + "name": "", + "type": "address" + }, + { + "internalType": "uint256", + "name": "", + "type": "uint256" + } + ], + "name": "approve", + "outputs": [ + { + "internalType": "bool", + "name": "", + "type": "bool" + } + ], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "address", + "name": "account", + "type": "address" + } + ], + "name": "balanceOf", + "outputs": [ + { + "internalType": "uint256", + "name": "", + "type": "uint256" + } + ], + "stateMutability": "view", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "address[]", + "name": "_accounts", + "type": "address[]" + }, + { + "internalType": "uint256[]", + "name": "_values", + "type": "uint256[]" + } + ], + "name": "batchBurnFrom", + "outputs": [], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "address[]", + "name": "_accounts", + "type": "address[]" + }, + { + "internalType": "uint256[]", + "name": "_values", + "type": "uint256[]" + } + ], + "name": "batchDepositFor", + "outputs": [], + "stateMutability": "payable", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "address[]", + "name": "_accounts", + "type": "address[]" + }, + { + "internalType": "uint256", + "name": "_value", + "type": "uint256" + } + ], + "name": "batchDepositForAll", + "outputs": [], + "stateMutability": "payable", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "address[]", + "name": "_accounts", + "type": "address[]" + }, + { + "internalType": "uint256[]", + "name": "_values", + "type": "uint256[]" + } + ], + "name": "batchMint", + "outputs": [], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "address[]", + "name": "_accounts", + "type": "address[]" + }, + { + "internalType": "uint256[]", + "name": "_values", + "type": "uint256[]" + } + ], + "name": "batchWithdrawFrom", + "outputs": [], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "address", + "name": "_account", + "type": "address" + }, + { + "internalType": "uint256", + "name": "_value", + "type": "uint256" + } + ], + "name": "burnFrom", + "outputs": [], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "uint256", + "name": "_amount", + "type": "uint256" + } + ], + "name": "chargeFromOrigin", + "outputs": [ + { + "internalType": "uint256", + "name": "amountCharged_", + "type": "uint256" + } + ], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [], + "name": "decimals", + "outputs": [ + { + "internalType": "uint8", + "name": "", + "type": "uint8" + } + ], + "stateMutability": "view", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "address", + "name": "spender", + "type": "address" + }, + { + "internalType": "uint256", + "name": "subtractedValue", + "type": "uint256" + } + ], + "name": "decreaseAllowance", + "outputs": [ + { + "internalType": "bool", + "name": "", + "type": "bool" + } + ], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "address[]", + "name": "_burners", + "type": "address[]" + } + ], + "name": "delBurners", + "outputs": [], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "address[]", + "name": "_minters", + "type": "address[]" + } + ], + "name": "delMinters", + "outputs": [], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [], + "name": "deposit", + "outputs": [], + "stateMutability": "payable", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "address[]", + "name": "_contracts", + "type": "address[]" + } + ], + "name": "disallowSgtValue", + "outputs": [], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "address", + "name": "spender", + "type": "address" + }, + { + "internalType": "uint256", + "name": "addedValue", + "type": "uint256" + } + ], + "name": "increaseAllowance", + "outputs": [ + { + "internalType": "bool", + "name": "", + "type": "bool" + } + ], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "string", + "name": "_name", + "type": "string" + }, + { + "internalType": "string", + "name": "_symbol", + "type": "string" + }, + { + "internalType": "address", + "name": "_owner", + "type": "address" + } + ], + "name": "initialize", + "outputs": [], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [], + "name": "isBackedByNative", + "outputs": [ + { + "internalType": "bool", + "name": "", + "type": "bool" + } + ], + "stateMutability": "view", + "type": "function" + }, + { + "inputs": [], + "name": "name", + "outputs": [ + { + "internalType": "string", + "name": "", + "type": "string" + } + ], + "stateMutability": "view", + "type": "function" + }, + { + "inputs": [], + "name": "owner", + "outputs": [ + { + "internalType": "address", + "name": "", + "type": "address" + } + ], + "stateMutability": "view", + "type": "function" + }, + { + "inputs": [], + "name": "renounceOwnership", + "outputs": [], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [], + "name": "symbol", + "outputs": [ + { + "internalType": "string", + "name": "", + "type": "string" + } + ], + "stateMutability": "view", + "type": "function" + }, + { + "inputs": [], + "name": "totalSupply", + "outputs": [ + { + "internalType": "uint256", + "name": "", + "type": "uint256" + } + ], + "stateMutability": "view", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "address", + "name": "", + "type": "address" + }, + { + "internalType": "uint256", + "name": "", + "type": "uint256" + } + ], + "name": "transfer", + "outputs": [ + { + "internalType": "bool", + "name": "", + "type": "bool" + } + ], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "address", + "name": "", + "type": "address" + }, + { + "internalType": "address", + "name": "", + "type": "address" + }, + { + "internalType": "uint256", + "name": "", + "type": "uint256" + } + ], + "name": "transferFrom", + "outputs": [ + { + "internalType": "bool", + "name": "", + "type": "bool" + } + ], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "address", + "name": "newOwner", + "type": "address" + } + ], + "name": "transferOwnership", + "outputs": [], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "address", + "name": "_account", + "type": "address" + }, + { + "internalType": "uint256", + "name": "_value", + "type": "uint256" + } + ], + "name": "withdrawFrom", + "outputs": [], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "anonymous": false, + "inputs": [ + { + "indexed": true, + "internalType": "address", + "name": "from", + "type": "address" + } + ], + "name": "AllowSgtValue", + "type": "event" + }, + { + "anonymous": false, + "inputs": [ + { + "indexed": true, + "internalType": "address", + "name": "owner", + "type": "address" + }, + { + "indexed": true, + "internalType": "address", + "name": "spender", + "type": "address" + }, + { + "indexed": false, + "internalType": "uint256", + "name": "value", + "type": "uint256" + } + ], + "name": "Approval", + "type": "event" + }, + { + "anonymous": false, + "inputs": [ + { + "indexed": true, + "internalType": "address", + "name": "burner", + "type": "address" + } + ], + "name": "BurnerAdded", + "type": "event" + }, + { + "anonymous": false, + "inputs": [ + { + "indexed": true, + "internalType": "address", + "name": "burner", + "type": "address" + } + ], + "name": "BurnerDeleted", + "type": "event" + }, + { + "anonymous": false, + "inputs": [ + { + "indexed": true, + "internalType": "address", + "name": "from", + "type": "address" + } + ], + "name": "DisallowSgtValue", + "type": "event" + }, + { + "anonymous": false, + "inputs": [ + { + "indexed": false, + "internalType": "uint8", + "name": "version", + "type": "uint8" + } + ], + "name": "Initialized", + "type": "event" + }, + { + "anonymous": false, + "inputs": [ + { + "indexed": true, + "internalType": "address", + "name": "minter", + "type": "address" + } + ], + "name": "MinterAdded", + "type": "event" + }, + { + "anonymous": false, + "inputs": [ + { + "indexed": true, + "internalType": "address", + "name": "minter", + "type": "address" + } + ], + "name": "MinterDeleted", + "type": "event" + }, + { + "anonymous": false, + "inputs": [ + { + "indexed": true, + "internalType": "address", + "name": "previousOwner", + "type": "address" + }, + { + "indexed": true, + "internalType": "address", + "name": "newOwner", + "type": "address" + } + ], + "name": "OwnershipTransferred", + "type": "event" + }, + { + "anonymous": false, + "inputs": [ + { + "indexed": true, + "internalType": "address", + "name": "from", + "type": "address" + }, + { + "indexed": true, + "internalType": "address", + "name": "to", + "type": "address" + }, + { + "indexed": false, + "internalType": "uint256", + "name": "value", + "type": "uint256" + } + ], + "name": "Transfer", + "type": "event" + } +] \ No newline at end of file
diff --git official optimism/packages/contracts-bedrock/snapshots/abi_loader.go SWC optimism/packages/contracts-bedrock/snapshots/abi_loader.go index d014c6604e05051df13a8b14d8d8c088260c93bb..4783bbd4c6a3a544b4f847aeed2f3c290f1eb4f5 100644 --- official optimism/packages/contracts-bedrock/snapshots/abi_loader.go +++ SWC optimism/packages/contracts-bedrock/snapshots/abi_loader.go @@ -31,6 +31,9 @@ //go:embed abi/CrossL2Inbox.json var crossL2Inbox []byte   +//go:embed abi/SoulGasToken.json +var solGasToken []byte + func LoadDisputeGameFactoryABI() *abi.ABI { return loadABI(disputeGameFactory) } @@ -56,6 +59,10 @@ }   func LoadCrossL2InboxABI() *abi.ABI { return loadABI(crossL2Inbox) +} + +func LoadSoulGasTokenABI() *abi.ABI { + return loadABI(solGasToken) }   func loadABI(json []byte) *abi.ABI {
diff --git official optimism/packages/contracts-bedrock/snapshots/abi_loader_test.go SWC optimism/packages/contracts-bedrock/snapshots/abi_loader_test.go index dc9d0884cb094f1960e944086a3f787ee110a532..ef27d28838cace1dd76bedede464c7dc8c4105d5 100644 --- official optimism/packages/contracts-bedrock/snapshots/abi_loader_test.go +++ SWC optimism/packages/contracts-bedrock/snapshots/abi_loader_test.go @@ -18,6 +18,7 @@ {"SuperFaultDisputeGame", LoadSuperFaultDisputeGameABI}, {"PreimageOracle", LoadPreimageOracleABI}, {"MIPS", LoadMIPSABI}, {"DelayedWETH", LoadDelayedWETHABI}, + {"SoulGasToken", LoadSoulGasTokenABI}, } for _, test := range tests { test := test
diff --git official optimism/packages/contracts-bedrock/snapshots/semver-lock.json SWC optimism/packages/contracts-bedrock/snapshots/semver-lock.json index 5fe766f632d1d1b80276125476370b7b7975cf7b..2b48211f9a7e0e704b0b2f52d250b5cd96fac535 100644 --- official optimism/packages/contracts-bedrock/snapshots/semver-lock.json +++ SWC optimism/packages/contracts-bedrock/snapshots/semver-lock.json @@ -28,8 +28,8 @@ "initCodeHash": "0x0c8b15453d0f0bc5d9af07f104505e0bbb2b358f0df418289822fb73a8652b30", "sourceCodeHash": "0x8c156f9f46ae60d928dcc49355519281d019cafabb327103db3094f28ed03537" }, "src/L1/OptimismPortal2.sol:OptimismPortal2": { - "initCodeHash": "0x5bf576ea7f566e402a997204988471fc9b971410aa9dff8fe810b10baf6b7456", - "sourceCodeHash": "0xcde7f2a838d13bf3bd2140a686d0c31472b637ab84d18ee46963977f036f80eb" + "initCodeHash": "0xe0b9780a472f5706c92f45bd6846e4e2a843221d8151458aa03d6e98e1448c9c", + "sourceCodeHash": "0x4b66477a38b5508c03be2d859d72ff025a63cdea4907826689125c7ba464a5b8" }, "src/L1/OptimismPortalInterop.sol:OptimismPortalInterop": { "initCodeHash": "0x087281cd2a48e882648c09fa90bfcca7487d222e16300f9372deba6b2b8ccfad", @@ -64,8 +64,8 @@ "initCodeHash": "0xf72c23d9c3775afd7b645fde429d09800622d329116feb5ff9829634655123ca", "sourceCodeHash": "0xb4d1bf3669ba87bbeaf4373145c7e1490478c4a05ba4838a524aa6f0ce7348a6" }, "src/L2/L1Block.sol:L1Block": { - "initCodeHash": "0x1f054ff228ecad7f51772dd25084469192f7a33c522b87cd46ec5558d3c46aec", - "sourceCodeHash": "0xa45ec2f83b73ac7792d193098240e19e4b26e4102bd1e3c64bb7d0f91074be82" + "initCodeHash": "0x05d13c532bab81f374eaec6d0bac8009c66545701c5c72bb303f7b0740b24f86", + "sourceCodeHash": "0x5de4f14b576e23eb568ba4f4b5693d6dc948efc294635cbf0c00a5fabbc3bbbc" }, "src/L2/L1FeeVault.sol:L1FeeVault": { "initCodeHash": "0x9b664e3d84ad510091337b4aacaa494b142512e2f6f7fbcdb6210ed62ca9b885", @@ -88,8 +88,8 @@ "initCodeHash": "0xa7a2e7efe8116ebb21f47ee06c1e62d3b2f5a046478094611a2ab4b714154030", "sourceCodeHash": "0xde724da82ecf3c96b330c2876a7285b6e2b933ac599241eaa3174c443ebbe33a" }, "src/L2/L2ToL1MessagePasser.sol:L2ToL1MessagePasser": { - "initCodeHash": "0x88f7b25f956eceeab9ad84c17e66cded6a1acbb933054ac2c8b336641f70f875", - "sourceCodeHash": "0x83396cbd12a0c5c02e09a4d99c4b62ab4e9d9eb762745e63283e2e818a78a39c" + "initCodeHash": "0x7169999087d4dfca3dac4993c7452fa56bb2a8986ca62f09c91523bb9e6bd2e1", + "sourceCodeHash": "0x242bf5fc497c9da521c0be0fa4046fae86587ccb6aecb37fa2681a44c4f4a39e" }, "src/L2/L2ToL2CrossDomainMessenger.sol:L2ToL2CrossDomainMessenger": { "initCodeHash": "0x975fd33a3a386310d54dbb01b56f3a6a8350f55a3b6bd7781e5ccc2166ddf2e6",
diff --git official optimism/packages/contracts-bedrock/snapshots/storageLayout/SoulGasToken.json SWC optimism/packages/contracts-bedrock/snapshots/storageLayout/SoulGasToken.json new file mode 100644 index 0000000000000000000000000000000000000000..ce8662ad00f0554231a033169dbb984b0a4c150b --- /dev/null +++ SWC optimism/packages/contracts-bedrock/snapshots/storageLayout/SoulGasToken.json @@ -0,0 +1,79 @@ +[ + { + "bytes": "1", + "label": "_initialized", + "offset": 0, + "slot": "0", + "type": "uint8" + }, + { + "bytes": "1", + "label": "_initializing", + "offset": 1, + "slot": "0", + "type": "bool" + }, + { + "bytes": "1600", + "label": "__gap", + "offset": 0, + "slot": "1", + "type": "uint256[50]" + }, + { + "bytes": "32", + "label": "_balances", + "offset": 0, + "slot": "51", + "type": "mapping(address => uint256)" + }, + { + "bytes": "32", + "label": "_allowances", + "offset": 0, + "slot": "52", + "type": "mapping(address => mapping(address => uint256))" + }, + { + "bytes": "32", + "label": "_totalSupply", + "offset": 0, + "slot": "53", + "type": "uint256" + }, + { + "bytes": "32", + "label": "_name", + "offset": 0, + "slot": "54", + "type": "string" + }, + { + "bytes": "32", + "label": "_symbol", + "offset": 0, + "slot": "55", + "type": "string" + }, + { + "bytes": "1440", + "label": "__gap", + "offset": 0, + "slot": "56", + "type": "uint256[45]" + }, + { + "bytes": "20", + "label": "_owner", + "offset": 0, + "slot": "101", + "type": "address" + }, + { + "bytes": "1568", + "label": "__gap", + "offset": 0, + "slot": "102", + "type": "uint256[49]" + } +] \ No newline at end of file
diff --git official optimism/qkc-fork.yaml SWC optimism/qkc-fork.yaml new file mode 100644 index 0000000000000000000000000000000000000000..ab834a77feb9591519dcedbf4e3477f8aa25ec31 --- /dev/null +++ SWC optimism/qkc-fork.yaml @@ -0,0 +1,178 @@ +title: "SWC-optimism - official-optimism fork diff overview" +footer: | + Fork-diff overview of [`SWC optimism`](https://github.com/Quarkchain/optimism), a fork of [`official optimism`](https://github.com/ethereum-optimism/optimism). + and consensus layer of the `SWC OP-stack`. +base: + name: official optimism + url: https://github.com/ethereum-optimism/optimism + hash: 94706ec5072b13030600d1b45ae10b673b660c0d +fork: + name: SWC optimism + url: https://github.com/Quarkchain/optimism + ref: refs/heads/op-es +def: + title: "optimism" + description: | + This is an overview of the changes in [`SWC optimism`](https://github.com/Quarkchain/optimism), + a fork of [`official optimism`](https://github.com/ethereum-optimism/optimism), part of the `SWC OP-stack`. + sub: + - title: "Inbox contract modifications" + description: | + The OP-stack is extended to to allow the batch inbox to be a contract, enabling customized batch submission conditions such as: + + - Requiring the batch transaction to be signed by a quorum of sequencers in a decentralized sequencing network; or + + - Mandating that the batch transaction call a BLOB storage contract (e.g., EthStorage) with a long-term storage fee, which is then distributed to data nodes that prove BLOB storage over time. + globs: + - "op-batcher/batcher/driver.go" + - "op-chain-ops/genesis/config.go" + - "op-node/rollup/derive/blob_data_source.go" + - "op-node/rollup/derive/calldata_source.go" + - "op-node/rollup/derive/calldata_source_test.go" + - "op-node/rollup/derive/blob_data_source_test.go" + - "op-node/rollup/derive/data_source.go" + - "op-node/rollup/types.go" + - title: "Soul Gas Token modifications" + description: | + To bridge the gap between traditional web users and the growing world of Web3, + we propose a **non-transferable gas token** named Soul Gas Token (SGT), where the gas token is the native token of a Rollup. + + The concept revolves around facilitating Web2 users' entry into Web3 by airdropping them with SGT. + + This token will enable users to pay for transaction gas fees without the immediate selling pressure of the airdropped token. + + This initiative is particularly aimed at those new to Web3, providing a seamless transition without the upfront cost of acquiring a gas token. + globs: + - "op-chain-ops/genesis/genesis.go" + - "op-service/predeploys/addresses.go" + - "packages/contracts-bedrock/scripts/Artifacts.s.sol" + - "packages/contracts-bedrock/scripts/L2Genesis.s.sol" + - "packages/contracts-bedrock/scripts/deploy/DeployConfig.s.sol" + - "packages/contracts-bedrock/src/L2/SoulGasToken.sol" + - "packages/contracts-bedrock/src/libraries/Predeploys.sol" + - "packages/contracts-bedrock/interfaces/L2/ISoulGasToken.sol" + - "packages/contracts-bedrock/deploy-config/devnetL1-template.json" + - title: "L2 Blob modifications" + description: | + The Ethereum Cancun upgrade has significantly reduced Layer 2 (L2) data uploading costs by introducing BLOB + transactions to Layer 1 (L1). This innovation has also enabled a variety of additional applications based on + the BLOBs due to their low cost, such as [blob.fm](https://blob.fm/), [EthStorage](https://ethstorage.io), and + [Ethscriptions](https://ethscriptions.com/). However, while the data upload costs have decreased, the execution + costs on L1 remain high compared to L2, leading to high costs for L2 state proposals and non-financial applications + that rely on BLOBs. + + To address these challenges, the L2 BLOB feature introduces support for BLOB transactions on L2. enabling L3 solutions that settle on L2 to use an enshrined 4844-compatible DA layer without needing to + integrate third-party DA providers or deal with the security risks associated with DA bridges. Additionally, the + applications mentioned above could migrate to L2 with minimal costs. + globs: + - "op-node/flags/flags.go" + - "op-node/node/config.go" + - "op-node/node/node.go" + - "op-node/p2p/gossip.go" + - "op-node/rollup/derive/span_batch_tx.go" + - "op-node/rollup/derive/span_batch_txs.go" + - "op-node/rollup/derive/span_batch_txs_test.go" + - "op-node/rollup/driver/driver.go" + - "op-node/rollup/engine/engine_controller.go" + - "op-node/rollup/sequencing/sequencer.go" + - "op-node/rollup/sequencing/sequencer_test.go" + - "op-node/rollup/types.go" + - "op-node/service.go" + - "op-service/eth/types.go" + - "op-service/sources/engine_client.go" + - "op-service/testutils/random.go" + - "op-program/client/l2/engineapi/block_processor.go" + - "packages/contracts-bedrock/deploy-config/devnetL1-template.json" + - title: "L1 Header Hash History modifications" + description: | + In order to simplify the integration with EthStorage as L3, The `L1Block` contract is extended to allow + fetching 8191 history L1 blockhash not including the current one. + globs: + - "packages/contracts-bedrock/src/L2/L1Block.sol" + - "packages/contracts-bedrock/interfaces/L2/IL1Block.sol" + - "packages/contracts-bedrock/interfaces/L2/IL1BlockInterop.sol" + - "op-node/bindings/l1block.go" + - "packages/contracts-bedrock/test/L2/L1Block.t.sol" + - title: "Disable native deposit & enable minting" + description: | + In order to facilitate token migration of existing L1 token to L2 native token, we disable the native deposit + and enable minting L2 native token on L1. Refer to this [PR](https://github.com/QuarkChain/optimism/pull/49) for details. + globs: + - "packages/contracts-bedrock/interfaces/L1/IOptimismPortal2.sol" + - "packages/contracts-bedrock/interfaces/L2/IL2ToL1MessagePasser.sol" + - "packages/contracts-bedrock/src/L1/OptimismPortal2.sol" + - "packages/contracts-bedrock/src/L2/L2ToL1MessagePasser.sol" + - "packages/contracts-bedrock/src/libraries/Constants.sol" + - title: "Fault Proof for beta testnet" + description: | + Based on these PRs to enable fault proof for custom chains: + + - https://github.com/ethereum-optimism/optimism/pull/12503 + - https://github.com/ethereum-optimism/optimism/pull/12310 + + We enabled fault proof for the beta testnet. + globs: + - "op-program/chainconfig/configs/3335-genesis-l2.json" + - "op-program/chainconfig/configs/3335-rollup.json" + - "op-program/client/l2/engineapi/block_processor.go" + - "op-program/verify/beta-testnet/cmd/beta-testnet.go" + - "op-program/Makefile" + - "op-program/scripts/build-prestates.sh" + - "op-program/prestates/releases.json" + - title: "Testing" + sub: + - title: "Inbox contract e2e test" + description: "" + globs: + - "op-e2e/inbox/inbox_test.go" + - "op-e2e/bindings/batchinbox.go" + - "op-e2e/bindings/mockstorage.go" + - "op-e2e/inbox/testcontract/mockstorage.sol" + - "op-e2e/system/e2esys/setup.go" + - title: "Soul Gas Token e2e test" + description: "" + globs: + - "op-e2e/bindings/soulgastoken.go" + - "op-e2e/sgt/sgt_test.go" + - "op-e2e/sgt/helper.go" + - "op-e2e/actions/sgt/sgt_test.go" + - "op-e2e/config/init.go" + - title: "L2 Blob e2e test" + description: "" + globs: + - "op-e2e/l2blob/l2blob_test.go" + - title: "Script to run overall local test" + description: | + This script is used to run the overall local test for the OP-stack. + For details, please refer to [here](https://github.com/ethstorage/optimism/pull/161). + globs: + - ".mise-tasks/dev-test.sh" + - title: "Tools" + sub: + - title: "Script to upgrade SoulGasToken" + description: | + This script is used to upgrade the SoulGasToken contract on L2. + It allows for the deployment of a new version of the contract, ensuring that + users can benefit from the latest features and improvements. + globs: + - "packages/contracts-bedrock/scripts/deploy/UpgradeSoulGasToken.s.sol" + - title: "Script to update the AnchorStateRegistry" + description: | + This script is used to update the AnchorStateRegistry with a new anchor state when + switching from permissioned FDG to permissionless FDG on L1. + Fore details, please refer to [here](https://github.com/ethstorage/pm/blob/main/L2/beta_testnet_fdg_upgrade.md). + globs: + - "packages/contracts-bedrock/scripts/deploy/UpgradeAnchorStateRegistry.s.sol" + + +# ignored globally, does not count towards line count +ignore: + - ".circleci/*" + - "*.sum" + - "go.mod" + - "qkc-fork.yaml" + - ".github/**" + - "**/*.gob" # data asset, not code + - "core/vm/testdata/precompiles/p256Verify.json" # data asset, not code + - "eth/tracers/internal/tracetest/testdata/**/*.json" + - "packages/contracts-bedrock/snapshots/**"