diff --git a/core/block_validator.go b/core/block_validator.go index c7dfe8db0d56..2038bd4f0d9b 100644 --- a/core/block_validator.go +++ b/core/block_validator.go @@ -132,6 +132,8 @@ func (v *BlockValidator) ValidateL1Messages(block *types.Block) error { return nil } + blockHash := block.Hash() + if v.config.Scroll.L1Config == nil { // TODO: should we allow follower nodes to skip L1 message verification? panic("Running on L1Message-enabled network but no l1Config was provided") @@ -168,13 +170,16 @@ func (v *BlockValidator) ValidateL1Messages(block *types.Block) error { // skipped messages // TODO: consider verifying that skipped messages overflow for index := queueIndex; index < txQueueIndex; index++ { - log.Debug("Skipped L1 message", "queueIndex", index, "tx", tx.Hash().String(), "block", block.Hash().String()) - if exists := it.Next(); !exists { // the message in this block is not available in our local db. // we'll reprocess this block at a later time. return consensus.ErrMissingL1MessageData } + + l1msg := it.L1Message() + skippedTx := types.NewTx(&l1msg) + log.Debug("Skipped L1 message", "queueIndex", index, "tx", skippedTx.Hash().String(), "block", blockHash.String()) + rawdb.WriteSkippedTransaction(v.db, skippedTx, "unknown", block.NumberU64(), &blockHash) } queueIndex = txQueueIndex + 1 diff --git a/core/rawdb/accessors_l1_message.go b/core/rawdb/accessors_l1_message.go index 5d0eebb11020..c2c19f90561e 100644 --- a/core/rawdb/accessors_l1_message.go +++ b/core/rawdb/accessors_l1_message.go @@ -98,7 +98,7 @@ type L1MessageIterator struct { // IterateL1MessagesFrom creates an L1MessageIterator that iterates over // all L1 message in the database starting at the provided enqueue index. func IterateL1MessagesFrom(db ethdb.Iteratee, fromQueueIndex uint64) L1MessageIterator { - start := encodeQueueIndex(fromQueueIndex) + start := encodeBigEndian(fromQueueIndex) it := db.NewIterator(l1MessagePrefix, start) keyLength := len(l1MessagePrefix) + 8 @@ -180,7 +180,7 @@ func ReadL1MessagesFrom(db ethdb.Iteratee, startIndex, maxCount uint64) []types. // The L2 block is identified by its block hash. If the L2 block contains zero // L1 messages, this value MUST equal its parent's value. func WriteFirstQueueIndexNotInL2Block(db ethdb.KeyValueWriter, l2BlockHash common.Hash, queueIndex uint64) { - if err := db.Put(FirstQueueIndexNotInL2BlockKey(l2BlockHash), encodeQueueIndex(queueIndex)); err != nil { + if err := db.Put(FirstQueueIndexNotInL2BlockKey(l2BlockHash), encodeBigEndian(queueIndex)); err != nil { log.Crit("Failed to store first L1 message not in L2 block", "l2BlockHash", l2BlockHash, "err", err) } } diff --git a/core/rawdb/accessors_skipped_txs.go b/core/rawdb/accessors_skipped_txs.go new file mode 100644 index 000000000000..4d26031b30dc --- /dev/null +++ b/core/rawdb/accessors_skipped_txs.go @@ -0,0 +1,202 @@ +package rawdb + +import ( + "bytes" + "encoding/binary" + "math/big" + "sync" + + "github.com/scroll-tech/go-ethereum/common" + "github.com/scroll-tech/go-ethereum/core/types" + "github.com/scroll-tech/go-ethereum/ethdb" + "github.com/scroll-tech/go-ethereum/log" + "github.com/scroll-tech/go-ethereum/rlp" +) + +// mutex used to avoid concurrent updates of NumSkippedTransactions +var mu sync.Mutex + +// writeNumSkippedTransactions writes the number of skipped transactions to the database. +func writeNumSkippedTransactions(db ethdb.KeyValueWriter, numSkipped uint64) { + value := big.NewInt(0).SetUint64(numSkipped).Bytes() + + if err := db.Put(numSkippedTransactionsKey, value); err != nil { + log.Crit("Failed to update the number of skipped transactions", "err", err) + } +} + +// ReadNumSkippedTransactions retrieves the number of skipped transactions. +func ReadNumSkippedTransactions(db ethdb.Reader) uint64 { + data, err := db.Get(numSkippedTransactionsKey) + if err != nil && isNotFoundErr(err) { + return 0 + } + if err != nil { + log.Crit("Failed to read number of skipped transactions from database", "err", err) + } + if len(data) == 0 { + return 0 + } + + number := new(big.Int).SetBytes(data) + if !number.IsUint64() { + log.Crit("Unexpected number of skipped transactions in database", "number", number) + } + return number.Uint64() +} + +// SkippedTransaction stores the transaction object, along with the skip reason and block context. +type SkippedTransaction struct { + // Tx is the skipped transaction. + // We store the tx itself because otherwise geth will discard it after skipping. + Tx *types.Transaction + + // Reason is the skip reason. + Reason string + + // BlockNumber is the number of the block in which this transaction was skipped. + BlockNumber uint64 + + // BlockNumber is the hash of the block in which this transaction was skipped or nil. + BlockHash *common.Hash +} + +// writeSkippedTransaction writes a skipped transaction to the database. +func writeSkippedTransaction(db ethdb.KeyValueWriter, tx *types.Transaction, reason string, blockNumber uint64, blockHash *common.Hash) { + // workaround: RLP decoding fails if this is nil + if blockHash == nil { + blockHash = &common.Hash{} + } + stx := SkippedTransaction{Tx: tx, Reason: reason, BlockNumber: blockNumber, BlockHash: blockHash} + bytes, err := rlp.EncodeToBytes(stx) + if err != nil { + log.Crit("Failed to RLP encode skipped transaction", "err", err) + } + if err := db.Put(SkippedTransactionKey(tx.Hash()), bytes); err != nil { + log.Crit("Failed to store skipped transaction", "hash", tx.Hash().String(), "err", err) + } +} + +// readSkippedTransactionRLP retrieves a skipped transaction in its raw RLP database encoding. +func readSkippedTransactionRLP(db ethdb.Reader, txHash common.Hash) rlp.RawValue { + data, err := db.Get(SkippedTransactionKey(txHash)) + if err != nil && isNotFoundErr(err) { + return nil + } + if err != nil { + log.Crit("Failed to load skipped transaction", "hash", txHash.String(), "err", err) + } + return data +} + +// ReadSkippedTransaction retrieves a skipped transaction by its hash, along with its skipped reason. +func ReadSkippedTransaction(db ethdb.Reader, txHash common.Hash) *SkippedTransaction { + data := readSkippedTransactionRLP(db, txHash) + if len(data) == 0 { + return nil + } + var stx SkippedTransaction + if err := rlp.Decode(bytes.NewReader(data), &stx); err != nil { + log.Crit("Invalid skipped transaction RLP", "hash", txHash.String(), "data", data, "err", err) + } + if stx.BlockHash != nil && *stx.BlockHash == (common.Hash{}) { + stx.BlockHash = nil + } + return &stx +} + +// writeSkippedTransactionHash writes the hash of a skipped transaction to the database. +func writeSkippedTransactionHash(db ethdb.KeyValueWriter, index uint64, txHash common.Hash) { + if err := db.Put(SkippedTransactionHashKey(index), txHash[:]); err != nil { + log.Crit("Failed to store skipped transaction hash", "index", index, "hash", txHash.String(), "err", err) + } +} + +// ReadSkippedTransactionHash retrieves the hash of a skipped transaction by its index. +func ReadSkippedTransactionHash(db ethdb.Reader, index uint64) *common.Hash { + data, err := db.Get(SkippedTransactionHashKey(index)) + if err != nil && isNotFoundErr(err) { + return nil + } + if err != nil { + log.Crit("Failed to load skipped transaction hash", "index", index, "err", err) + } + hash := common.BytesToHash(data) + return &hash +} + +// WriteSkippedTransaction writes a skipped transaction to the database and also updates the count and lookup index. +// Note: The lookup index and count will include duplicates if there are chain reorgs. +func WriteSkippedTransaction(db ethdb.Database, tx *types.Transaction, reason string, blockNumber uint64, blockHash *common.Hash) { + // this method is not accessed concurrently, but just to be sure... + mu.Lock() + defer mu.Unlock() + + index := ReadNumSkippedTransactions(db) + + // update in a batch + batch := db.NewBatch() + writeSkippedTransaction(db, tx, reason, blockNumber, blockHash) + writeSkippedTransactionHash(db, index, tx.Hash()) + writeNumSkippedTransactions(db, index+1) + + // write to DB + if err := batch.Write(); err != nil { + log.Crit("Failed to store skipped transaction", "hash", tx.Hash().String(), "err", err) + } +} + +// SkippedTransactionIterator is a wrapper around ethdb.Iterator that +// allows us to iterate over skipped transaction hashes in the database. +// It implements an interface similar to ethdb.Iterator. +type SkippedTransactionIterator struct { + inner ethdb.Iterator + db ethdb.Reader + keyLength int +} + +// IterateSkippedTransactionsFrom creates a SkippedTransactionIterator that iterates +// over all skipped transaction hashes in the database starting at the provided index. +func IterateSkippedTransactionsFrom(db ethdb.Database, index uint64) SkippedTransactionIterator { + start := encodeBigEndian(index) + it := db.NewIterator(skippedTransactionHashPrefix, start) + keyLength := len(skippedTransactionHashPrefix) + 8 + + return SkippedTransactionIterator{ + inner: it, + db: db, + keyLength: keyLength, + } +} + +// Next moves the iterator to the next key/value pair. +// It returns false when the iterator is exhausted. +// TODO: Consider reading items in batches. +func (it *SkippedTransactionIterator) Next() bool { + for it.inner.Next() { + key := it.inner.Key() + if len(key) == it.keyLength { + return true + } + } + return false +} + +// Index returns the index of the current skipped transaction hash. +func (it *SkippedTransactionIterator) Index() uint64 { + key := it.inner.Key() + raw := key[len(skippedTransactionHashPrefix) : len(skippedTransactionHashPrefix)+8] + index := binary.BigEndian.Uint64(raw) + return index +} + +// TransactionHash returns the current skipped transaction hash. +func (it *SkippedTransactionIterator) TransactionHash() common.Hash { + data := it.inner.Value() + return common.BytesToHash(data) +} + +// Release releases the associated resources. +func (it *SkippedTransactionIterator) Release() { + it.inner.Release() +} diff --git a/core/rawdb/accessors_skipped_txs_test.go b/core/rawdb/accessors_skipped_txs_test.go new file mode 100644 index 000000000000..926d0d9c9662 --- /dev/null +++ b/core/rawdb/accessors_skipped_txs_test.go @@ -0,0 +1,134 @@ +package rawdb + +import ( + "math/big" + "sync" + "testing" + + "github.com/scroll-tech/go-ethereum/common" + "github.com/scroll-tech/go-ethereum/core/types" +) + +func TestReadWriteNumSkippedTransactions(t *testing.T) { + blockNumbers := []uint64{ + 1, + 1 << 2, + 1 << 8, + 1 << 16, + 1 << 32, + } + + db := NewMemoryDatabase() + for _, num := range blockNumbers { + writeNumSkippedTransactions(db, num) + got := ReadNumSkippedTransactions(db) + + if got != num { + t.Fatal("Num skipped transactions mismatch", "expected", num, "got", got) + } + } +} + +func newTestTransaction(queueIndex uint64) *types.Transaction { + l1msg := types.L1MessageTx{ + QueueIndex: queueIndex, + Gas: 0, + To: &common.Address{}, + Value: big.NewInt(0), + Data: nil, + Sender: common.Address{}, + } + return types.NewTx(&l1msg) +} + +func TestReadWriteSkippedTransactionNoIndex(t *testing.T) { + tx := newTestTransaction(123) + db := NewMemoryDatabase() + writeSkippedTransaction(db, tx, "random reason", 1, &common.Hash{1}) + got := ReadSkippedTransaction(db, tx.Hash()) + if got == nil || got.Tx.Hash() != tx.Hash() || got.Reason != "random reason" || got.BlockNumber != 1 || got.BlockHash == nil || *got.BlockHash != (common.Hash{1}) { + t.Fatal("Skipped transaction mismatch", "got", got) + } +} + +func TestReadWriteSkippedTransaction(t *testing.T) { + tx := newTestTransaction(123) + db := NewMemoryDatabase() + WriteSkippedTransaction(db, tx, "random reason", 1, &common.Hash{1}) + got := ReadSkippedTransaction(db, tx.Hash()) + if got == nil || got.Tx.Hash() != tx.Hash() || got.Reason != "random reason" || got.BlockNumber != 1 || got.BlockHash == nil || *got.BlockHash != (common.Hash{1}) { + t.Fatal("Skipped transaction mismatch", "got", got) + } + count := ReadNumSkippedTransactions(db) + if count != 1 { + t.Fatal("Skipped transaction count mismatch", "expected", 1, "got", count) + } + hash := ReadSkippedTransactionHash(db, 0) + if hash == nil || *hash != tx.Hash() { + t.Fatal("Skipped L1 message hash mismatch", "expected", tx.Hash(), "got", hash) + } +} + +func TestSkippedTransactionConcurrentUpdate(t *testing.T) { + count := 20 + tx := newTestTransaction(123) + db := NewMemoryDatabase() + var wg sync.WaitGroup + for ii := 0; ii < count; ii++ { + wg.Add(1) + go func() { + defer wg.Done() + WriteSkippedTransaction(db, tx, "random reason", 1, &common.Hash{1}) + }() + } + wg.Wait() + got := ReadNumSkippedTransactions(db) + if got != uint64(count) { + t.Fatal("Skipped transaction count mismatch", "expected", count, "got", got) + } +} + +func TestIterateSkippedTransactions(t *testing.T) { + db := NewMemoryDatabase() + + txs := []*types.Transaction{ + newTestTransaction(1), + newTestTransaction(2), + newTestTransaction(3), + newTestTransaction(4), + newTestTransaction(5), + } + + for _, tx := range txs { + WriteSkippedTransaction(db, tx, "random reason", 1, &common.Hash{1}) + } + + // simulate skipped L2 tx that's not included in the index + l2tx := newTestTransaction(6) + writeSkippedTransaction(db, l2tx, "random reason", 1, &common.Hash{1}) + + it := IterateSkippedTransactionsFrom(db, 2) + defer it.Release() + + for ii := 2; ii < len(txs); ii++ { + finished := !it.Next() + if finished { + t.Fatal("Iterator terminated early", "ii", ii) + } + + index := it.Index() + if index != uint64(ii) { + t.Fatal("Invalid skipped transaction index", "expected", ii, "got", index) + } + + hash := it.TransactionHash() + if hash != txs[ii].Hash() { + t.Fatal("Invalid skipped transaction hash", "expected", txs[ii].Hash(), "got", hash) + } + } + + finished := !it.Next() + if !finished { + t.Fatal("Iterator did not terminate") + } +} diff --git a/core/rawdb/schema.go b/core/rawdb/schema.go index 4b7ac8ab7015..960e8cbcf6c4 100644 --- a/core/rawdb/schema.go +++ b/core/rawdb/schema.go @@ -110,6 +110,11 @@ var ( // Row consumption rowConsumptionPrefix = []byte("rc") // rowConsumptionPrefix + hash -> row consumption by block + + // Skipped transactions + numSkippedTransactionsKey = []byte("NumberOfSkippedTransactions") + skippedTransactionPrefix = []byte("skip") // skippedTransactionPrefix + tx hash -> skipped transaction + skippedTransactionHashPrefix = []byte("sh") // skippedTransactionHashPrefix + index -> tx hash ) const ( @@ -243,8 +248,8 @@ func configKey(hash common.Hash) []byte { return append(configPrefix, hash.Bytes()...) } -// encodeQueueIndex encodes an L1 enqueue index as big endian uint64 -func encodeQueueIndex(index uint64) []byte { +// encodeBigEndian encodes an index as big endian uint64 +func encodeBigEndian(index uint64) []byte { enc := make([]byte, 8) binary.BigEndian.PutUint64(enc, index) return enc @@ -252,7 +257,7 @@ func encodeQueueIndex(index uint64) []byte { // L1MessageKey = l1MessagePrefix + queueIndex (uint64 big endian) func L1MessageKey(queueIndex uint64) []byte { - return append(l1MessagePrefix, encodeQueueIndex(queueIndex)...) + return append(l1MessagePrefix, encodeBigEndian(queueIndex)...) } // FirstQueueIndexNotInL2BlockKey = firstQueueIndexNotInL2BlockPrefix + L2 block hash @@ -268,3 +273,13 @@ func rowConsumptionKey(hash common.Hash) []byte { func isNotFoundErr(err error) bool { return errors.Is(err, leveldb.ErrNotFound) || errors.Is(err, memorydb.ErrMemorydbNotFound) } + +// SkippedTransactionKey = skippedTransactionPrefix + tx hash +func SkippedTransactionKey(txHash common.Hash) []byte { + return append(skippedTransactionPrefix, txHash.Bytes()...) +} + +// SkippedTransactionHashKey = skippedTransactionHashPrefix + index (uint64 big endian) +func SkippedTransactionHashKey(index uint64) []byte { + return append(skippedTransactionHashPrefix, encodeBigEndian(index)...) +} diff --git a/eth/api.go b/eth/api.go index 49e5bfcf676b..c22425f9bd8e 100644 --- a/eth/api.go +++ b/eth/api.go @@ -708,3 +708,47 @@ func (api *ScrollAPI) GetBlockByNumber(ctx context.Context, number rpc.BlockNumb } return nil, err } + +// GetNumSkippedTransactions returns the number of skipped transactions. +func (api *ScrollAPI) GetNumSkippedTransactions(ctx context.Context) (uint64, error) { + return rawdb.ReadNumSkippedTransactions(api.eth.ChainDb()), nil +} + +// RPCTransaction is the standard RPC transaction return type with some additional skip-related fields. +type RPCTransaction struct { + ethapi.RPCTransaction + SkipReason string `json:"skipReason"` + SkipBlockNumber *hexutil.Big `json:"skipBlockNumber"` + SkipBlockHash *common.Hash `json:"skipBlockHash,omitempty"` +} + +// GetSkippedTransaction returns a skipped transaction by its hash. +func (api *ScrollAPI) GetSkippedTransaction(ctx context.Context, hash common.Hash) (*RPCTransaction, error) { + stx := rawdb.ReadSkippedTransaction(api.eth.ChainDb(), hash) + if stx == nil { + return nil, nil + } + var rpcTx RPCTransaction + rpcTx.RPCTransaction = *ethapi.NewRPCTransaction(stx.Tx, common.Hash{}, 0, 0, nil, api.eth.blockchain.Config()) + rpcTx.SkipReason = stx.Reason + rpcTx.SkipBlockNumber = (*hexutil.Big)(new(big.Int).SetUint64(stx.BlockNumber)) + rpcTx.SkipBlockHash = stx.BlockHash + return &rpcTx, nil +} + +// GetSkippedTransactionHashes returns a list of skipped transaction hashes between the two indices provided (inclusive). +func (api *ScrollAPI) GetSkippedTransactionHashes(ctx context.Context, from uint64, to uint64) ([]common.Hash, error) { + it := rawdb.IterateSkippedTransactionsFrom(api.eth.ChainDb(), from) + defer it.Release() + + var hashes []common.Hash + + for it.Next() { + if it.Index() > to { + break + } + hashes = append(hashes, it.TransactionHash()) + } + + return hashes, nil +} diff --git a/internal/ethapi/api.go b/internal/ethapi/api.go index dede12698b24..2a553b0c1e41 100644 --- a/internal/ethapi/api.go +++ b/internal/ethapi/api.go @@ -1337,9 +1337,9 @@ type RPCTransaction struct { QueueIndex *hexutil.Uint64 `json:"queueIndex,omitempty"` } -// newRPCTransaction returns a transaction that will serialize to the RPC +// NewRPCTransaction returns a transaction that will serialize to the RPC // representation, with the given location metadata set (if available). -func newRPCTransaction(tx *types.Transaction, blockHash common.Hash, blockNumber uint64, index uint64, baseFee *big.Int, config *params.ChainConfig) *RPCTransaction { +func NewRPCTransaction(tx *types.Transaction, blockHash common.Hash, blockNumber uint64, index uint64, baseFee *big.Int, config *params.ChainConfig) *RPCTransaction { signer := types.MakeSigner(config, big.NewInt(0).SetUint64(blockNumber)) from, _ := types.Sender(signer, tx) v, r, s := tx.RawSignatureValues() @@ -1397,7 +1397,7 @@ func newRPCPendingTransaction(tx *types.Transaction, current *types.Header, conf baseFee = misc.CalcBaseFee(config, current) blockNumber = current.Number.Uint64() } - return newRPCTransaction(tx, common.Hash{}, blockNumber, 0, baseFee, config) + return NewRPCTransaction(tx, common.Hash{}, blockNumber, 0, baseFee, config) } // newRPCTransactionFromBlockIndex returns a transaction that will serialize to the RPC representation. @@ -1406,7 +1406,7 @@ func newRPCTransactionFromBlockIndex(b *types.Block, index uint64, config *param if index >= uint64(len(txs)) { return nil } - return newRPCTransaction(txs[index], b.Hash(), b.NumberU64(), index, b.BaseFee(), config) + return NewRPCTransaction(txs[index], b.Hash(), b.NumberU64(), index, b.BaseFee(), config) } // newRPCRawTransactionFromBlockIndex returns the bytes of a transaction given a block and a transaction index. @@ -1629,7 +1629,7 @@ func (s *PublicTransactionPoolAPI) GetTransactionByHash(ctx context.Context, has if err != nil { return nil, err } - return newRPCTransaction(tx, blockHash, blockNumber, index, header.BaseFee, s.b.ChainConfig()), nil + return NewRPCTransaction(tx, blockHash, blockNumber, index, header.BaseFee, s.b.ChainConfig()), nil } // No finalized transaction, try to retrieve it from the pool if tx := s.b.GetPoolTransaction(hash); tx != nil { diff --git a/internal/web3ext/web3ext.go b/internal/web3ext/web3ext.go index 7c65cb0f537b..afe5a63b315a 100644 --- a/internal/web3ext/web3ext.go +++ b/internal/web3ext/web3ext.go @@ -887,6 +887,16 @@ web3._extend({ params: 2, inputFormatter: [null, function (val) { return !!val; }] }), + new web3._extend.Method({ + name: 'getSkippedTransaction', + call: 'scroll_getSkippedTransaction', + params: 1 + }), + new web3._extend.Method({ + name: 'getSkippedTransactionHashes', + call: 'scroll_getSkippedTransactionHashes', + params: 2 + }), ], properties: [ @@ -897,7 +907,11 @@ web3._extend({ new web3._extend.Property({ name: 'latestRelayedQueueIndex', getter: 'scroll_getLatestRelayedQueueIndex' - }) + }), + new web3._extend.Property({ + name: 'numSkippedTransactions', + getter: 'scroll_getNumSkippedTransactions' + }), ] }); ` diff --git a/miner/worker.go b/miner/worker.go index 24f15bbaf788..5c99c2be6f55 100644 --- a/miner/worker.go +++ b/miner/worker.go @@ -19,6 +19,7 @@ package miner import ( "bytes" "errors" + "fmt" "math/big" "sync" "sync/atomic" @@ -1027,6 +1028,7 @@ loop: log.Info("Skipping L1 message", "queueIndex", queueIndex, "tx", tx.Hash().String(), "block", w.current.header.Number, "reason", "gas limit exceeded") w.current.nextL1MsgIndex = queueIndex + 1 txs.Shift() + rawdb.WriteSkippedTransaction(w.eth.ChainDb(), tx, "gas limit exceeded", w.current.header.Number.Uint64(), nil) case errors.Is(err, core.ErrGasLimitReached): // Pop the current out-of-gas transaction without shifting in the next from the account @@ -1095,6 +1097,9 @@ loop: w.circuitCapacityChecker.Reset() log.Trace("Worker reset ccc", "id", w.circuitCapacityChecker.ID) circuitCapacityReached = false + + // Store skipped transaction in local db + rawdb.WriteSkippedTransaction(w.eth.ChainDb(), tx, "row consumption overflow", w.current.header.Number.Uint64(), nil) } case (errors.Is(err, circuitcapacitychecker.ErrUnknown) && tx.IsL1MessageTx()): @@ -1104,6 +1109,8 @@ loop: log.Trace("Unknown circuit capacity checker error for L1MessageTx", "tx", tx.Hash().String(), "queueIndex", queueIndex) log.Info("Skipping L1 message", "queueIndex", queueIndex, "tx", tx.Hash().String(), "block", w.current.header.Number, "reason", "unknown row consumption error") w.current.nextL1MsgIndex = queueIndex + 1 + // TODO: propagate more info about the error from CCC + rawdb.WriteSkippedTransaction(w.eth.ChainDb(), tx, "unknown circuit capacity checker error", w.current.header.Number.Uint64(), nil) // Normally we would do `txs.Shift()` here. // However, after `ErrUnknown`, ccc might remain in an @@ -1129,6 +1136,7 @@ loop: queueIndex := tx.AsL1MessageTx().QueueIndex log.Info("Skipping L1 message", "queueIndex", queueIndex, "tx", tx.Hash().String(), "block", w.current.header.Number, "reason", "strange error", "err", err) w.current.nextL1MsgIndex = queueIndex + 1 + rawdb.WriteSkippedTransaction(w.eth.ChainDb(), tx, fmt.Sprintf("strange error: %v", err), w.current.header.Number.Uint64(), nil) } txs.Shift() } diff --git a/miner/worker_test.go b/miner/worker_test.go index 6a95372641e6..a215a60ec9e0 100644 --- a/miner/worker_test.go +++ b/miner/worker_test.go @@ -1046,3 +1046,63 @@ func TestOversizedTxThenNormal(t *testing.T) { } }) } + +func TestSkippedTransactionDatabaseEntries(t *testing.T) { + assert := assert.New(t) + + msgs := []types.L1MessageTx{ + {QueueIndex: 0, Gas: 10000000, To: &common.Address{1}, Data: []byte{0x01}, Sender: common.Address{2}}, // over gas limit + {QueueIndex: 1, Gas: 21016, To: &common.Address{1}, Data: []byte{0x01}, Sender: common.Address{2}}, + } + + l1MessageTest(t, msgs, false, func(blockNum int, block *types.Block, db ethdb.Database, w *worker) bool { + switch blockNum { + case 0: + return false + case 1: + // skip #0, include #1 + assert.Equal(1, len(block.Transactions())) + + assert.True(block.Transactions()[0].IsL1MessageTx()) + assert.Equal(uint64(1), block.Transactions()[0].AsL1MessageTx().QueueIndex) + + // db is updated correctly + queueIndex := rawdb.ReadFirstQueueIndexNotInL2Block(db, block.Hash()) + assert.NotNil(queueIndex) + + assert.Equal(uint64(2), *queueIndex) + + l1msg := rawdb.ReadL1Message(db, 0) + assert.NotNil(l1msg) + hash := types.NewTx(l1msg).Hash() + + stx := rawdb.ReadSkippedTransaction(db, hash) + assert.NotNil(stx) + assert.True(stx.Tx.IsL1MessageTx()) + assert.Equal(uint64(0), stx.Tx.AsL1MessageTx().QueueIndex) + assert.Equal("gas limit exceeded", stx.Reason) + assert.Equal(block.NumberU64(), stx.BlockNumber) + assert.Nil(stx.BlockHash) + + numSkipped := rawdb.ReadNumSkippedTransactions(db) + assert.Equal(uint64(1), numSkipped) + + hash2 := rawdb.ReadSkippedTransactionHash(db, 0) + assert.NotNil(hash2) + assert.Equal(&hash, hash2) + + // iterator API + it := rawdb.IterateSkippedTransactionsFrom(db, 0) + hasMore := it.Next() + assert.True(hasMore) + assert.Equal(uint64(0), it.Index()) + hash3 := it.TransactionHash() + assert.Equal(hash, hash3) + hasMore = it.Next() + assert.False(hasMore) + return true + default: + return true + } + }) +} diff --git a/params/version.go b/params/version.go index cd538c3ce1d6..4b67f0af2632 100644 --- a/params/version.go +++ b/params/version.go @@ -24,7 +24,7 @@ import ( const ( VersionMajor = 4 // Major version component of the current release VersionMinor = 3 // Minor version component of the current release - VersionPatch = 52 // Patch version component of the current release + VersionPatch = 53 // Patch version component of the current release VersionMeta = "sepolia" // Version metadata to append to the version string ) diff --git a/rollup/circuitcapacitychecker/types.go b/rollup/circuitcapacitychecker/types.go index 8309a0274a22..f57152745465 100644 --- a/rollup/circuitcapacitychecker/types.go +++ b/rollup/circuitcapacitychecker/types.go @@ -8,7 +8,7 @@ import ( var ( ErrUnknown = errors.New("unknown circuit capacity checker error") - ErrBlockRowConsumptionOverflow = errors.New("block row consumption oveflow") + ErrBlockRowConsumptionOverflow = errors.New("block row consumption overflow") ) type WrappedRowUsage struct {