From 01cba0d0fdfa5c4f270faf17e627134963a43734 Mon Sep 17 00:00:00 2001 From: Vlad Date: Mon, 22 Sep 2025 11:37:11 -0400 Subject: [PATCH 01/47] wip: set up local journal --- mempool/mempool.go | 58 ++++-- mempool/txpool/locals/errors.go | 46 +++++ mempool/txpool/locals/journal.go | 186 +++++++++++++++++++ mempool/txpool/locals/tx_tracker.go | 216 +++++++++++++++++++++++ mempool/txpool/locals/tx_tracker_test.go | 165 +++++++++++++++++ 5 files changed, 656 insertions(+), 15 deletions(-) create mode 100644 mempool/txpool/locals/errors.go create mode 100644 mempool/txpool/locals/journal.go create mode 100644 mempool/txpool/locals/tx_tracker.go create mode 100644 mempool/txpool/locals/tx_tracker_test.go diff --git a/mempool/mempool.go b/mempool/mempool.go index a3a7aa62c..6143a1f1b 100644 --- a/mempool/mempool.go +++ b/mempool/mempool.go @@ -4,7 +4,9 @@ import ( "context" "errors" "fmt" + "github.com/cosmos/evm/mempool/txpool/locals" "sync" + "time" ethtypes "github.com/ethereum/go-ethereum/core/types" "github.com/holiman/uint256" @@ -46,9 +48,10 @@ type ( vmKeeper VMKeeperI /** Mempools **/ - txPool *txpool.TxPool - legacyTxPool *legacypool.LegacyPool - cosmosPool sdkmempool.ExtMempool + txPool *txpool.TxPool + legacyTxPool *legacypool.LegacyPool + localTxTracker *locals.TxTracker + cosmosPool sdkmempool.ExtMempool /** Utils **/ logger log.Logger @@ -127,6 +130,7 @@ func NewExperimentalEVMMempool(getCtxCallback func(height int64, prove bool) (sd // from queued into pending, noting their readiness to be executed. legacyPool.BroadcastTxFn = func(txs []*ethtypes.Transaction) error { logger.Debug("broadcasting EVM transactions", "tx_count", len(txs)) + fmt.Println(clientCtx) return broadcastEVMTransactions(clientCtx, txConfig, txs) } } @@ -143,6 +147,21 @@ func NewExperimentalEVMMempool(getCtxCallback func(height int64, prove bool) (sd panic("tx pool should contain only legacypool") } + var localTxTracker *locals.TxTracker + + if !legacyConfig.NoLocals { + rejournal := legacyConfig.Rejournal + if rejournal < time.Second { + logger.Debug("Sanitizing invalid txpool journal time", "provided", rejournal, "updated", time.Second) + rejournal = time.Second + } + localTxTracker = locals.New(legacyConfig.Journal, rejournal, blockchain.Config(), txPool) + err := localTxTracker.Start() + if err != nil { + return nil + } + } + // Create Cosmos Mempool from configuration cosmosPoolConfig := config.CosmosPoolConfig if cosmosPoolConfig == nil { @@ -174,18 +193,19 @@ func NewExperimentalEVMMempool(getCtxCallback func(height int64, prove bool) (sd cosmosPool = sdkmempool.NewPriorityMempool(*cosmosPoolConfig) evmMempool := &ExperimentalEVMMempool{ - vmKeeper: vmKeeper, - txPool: txPool, - legacyTxPool: txPool.Subpools[0].(*legacypool.LegacyPool), - cosmosPool: cosmosPool, - logger: logger, - txConfig: txConfig, - blockchain: blockchain, - bondDenom: bondDenom, - evmDenom: evmDenom, - blockGasLimit: config.BlockGasLimit, - minTip: config.MinTip, - anteHandler: config.AnteHandler, + vmKeeper: vmKeeper, + txPool: txPool, + legacyTxPool: txPool.Subpools[0].(*legacypool.LegacyPool), + localTxTracker: localTxTracker, + cosmosPool: cosmosPool, + logger: logger, + txConfig: txConfig, + blockchain: blockchain, + bondDenom: bondDenom, + evmDenom: evmDenom, + blockGasLimit: config.BlockGasLimit, + minTip: config.MinTip, + anteHandler: config.AnteHandler, } vmKeeper.SetEvmMempool(evmMempool) @@ -303,6 +323,10 @@ func (m *ExperimentalEVMMempool) Remove(tx sdk.Tx) error { m.mtx.Lock() defer m.mtx.Unlock() + if m.blockchain.latestCtx.BlockHeight() == 0 { + return nil + } + m.logger.Debug("removing transaction from mempool") msg, err := m.getEVMMessage(tx) @@ -419,6 +443,10 @@ func (m *ExperimentalEVMMempool) Close() error { errs = append(errs, fmt.Errorf("failed to close txpool: %w", err)) } + if err := m.localTxTracker.Stop(); err != nil { + errs = append(errs, fmt.Errorf("failed to close localTxTracker: %w", err)) + } + return errors.Join(errs...) } diff --git a/mempool/txpool/locals/errors.go b/mempool/txpool/locals/errors.go new file mode 100644 index 000000000..fda50bf21 --- /dev/null +++ b/mempool/txpool/locals/errors.go @@ -0,0 +1,46 @@ +// Copyright 2025 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package locals + +import ( + "errors" + + "github.com/ethereum/go-ethereum/core/txpool" + "github.com/ethereum/go-ethereum/core/txpool/legacypool" +) + +// IsTemporaryReject determines whether the given error indicates a temporary +// reason to reject a transaction from being included in the txpool. The result +// may change if the txpool's state changes later. +func IsTemporaryReject(err error) bool { + switch { + case errors.Is(err, legacypool.ErrOutOfOrderTxFromDelegated): + return true + case errors.Is(err, txpool.ErrInflightTxLimitReached): + return true + case errors.Is(err, legacypool.ErrAuthorityReserved): + return true + case errors.Is(err, txpool.ErrUnderpriced): + return true + case errors.Is(err, legacypool.ErrTxPoolOverflow): + return true + case errors.Is(err, legacypool.ErrFutureReplacePending): + return true + default: + return false + } +} diff --git a/mempool/txpool/locals/journal.go b/mempool/txpool/locals/journal.go new file mode 100644 index 000000000..46fd6de34 --- /dev/null +++ b/mempool/txpool/locals/journal.go @@ -0,0 +1,186 @@ +// Copyright 2017 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package locals + +import ( + "errors" + "io" + "io/fs" + "os" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/rlp" +) + +// errNoActiveJournal is returned if a transaction is attempted to be inserted +// into the journal, but no such file is currently open. +var errNoActiveJournal = errors.New("no active journal") + +// devNull is a WriteCloser that just discards anything written into it. Its +// goal is to allow the transaction journal to write into a fake journal when +// loading transactions on startup without printing warnings due to no file +// being read for write. +type devNull struct{} + +func (*devNull) Write(p []byte) (n int, err error) { return len(p), nil } +func (*devNull) Close() error { return nil } + +// journal is a rotating log of transactions with the aim of storing locally +// created transactions to allow non-executed ones to survive node restarts. +type journal struct { + path string // Filesystem path to store the transactions at + writer io.WriteCloser // Output stream to write new transactions into +} + +// newTxJournal creates a new transaction journal to +func newTxJournal(path string) *journal { + return &journal{ + path: path, + } +} + +// load parses a transaction journal dump from disk, loading its contents into +// the specified pool. +func (journal *journal) load(add func([]*types.Transaction) []error) error { + // Open the journal for loading any past transactions + input, err := os.Open(journal.path) + if errors.Is(err, fs.ErrNotExist) { + // Skip the parsing if the journal file doesn't exist at all + return nil + } + if err != nil { + return err + } + defer input.Close() + + // Temporarily discard any journal additions (don't double add on load) + journal.writer = new(devNull) + defer func() { journal.writer = nil }() + + // Inject all transactions from the journal into the pool + stream := rlp.NewStream(input, 0) + total, dropped := 0, 0 + + // Create a method to load a limited batch of transactions and bump the + // appropriate progress counters. Then use this method to load all the + // journaled transactions in small-ish batches. + loadBatch := func(txs types.Transactions) { + for _, err := range add(txs) { + if err != nil { + log.Debug("Failed to add journaled transaction", "err", err) + dropped++ + } + } + } + var ( + failure error + batch types.Transactions + ) + for { + // Parse the next transaction and terminate on error + tx := new(types.Transaction) + if err = stream.Decode(tx); err != nil { + if err != io.EOF { + failure = err + } + if batch.Len() > 0 { + loadBatch(batch) + } + break + } + // New transaction parsed, queue up for later, import if threshold is reached + total++ + + if batch = append(batch, tx); batch.Len() > 1024 { + loadBatch(batch) + batch = batch[:0] + } + } + log.Info("Loaded local transaction journal", "transactions", total, "dropped", dropped) + + return failure +} + +// insert adds the specified transaction to the local disk journal. +func (journal *journal) insert(tx *types.Transaction) error { + if journal.writer == nil { + return errNoActiveJournal + } + if err := rlp.Encode(journal.writer, tx); err != nil { + return err + } + return nil +} + +// rotate regenerates the transaction journal based on the current contents of +// the transaction pool. +func (journal *journal) rotate(all map[common.Address]types.Transactions) error { + // Close the current journal (if any is open) + if journal.writer != nil { + if err := journal.writer.Close(); err != nil { + return err + } + journal.writer = nil + } + // Generate a new journal with the contents of the current pool + replacement, err := os.OpenFile(journal.path+".new", os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644) + if err != nil { + return err + } + journaled := 0 + for _, txs := range all { + for _, tx := range txs { + if err = rlp.Encode(replacement, tx); err != nil { + replacement.Close() + return err + } + } + journaled += len(txs) + } + replacement.Close() + + // Replace the live journal with the newly generated one + if err = os.Rename(journal.path+".new", journal.path); err != nil { + return err + } + sink, err := os.OpenFile(journal.path, os.O_WRONLY|os.O_APPEND, 0644) + if err != nil { + return err + } + journal.writer = sink + + logger := log.Info + if len(all) == 0 { + logger = log.Debug + } + logger("Regenerated local transaction journal", "transactions", journaled, "accounts", len(all)) + + return nil +} + +// close flushes the transaction journal contents to disk and closes the file. +func (journal *journal) close() error { + var err error + + if journal.writer != nil { + err = journal.writer.Close() + journal.writer = nil + } + return err +} diff --git a/mempool/txpool/locals/tx_tracker.go b/mempool/txpool/locals/tx_tracker.go new file mode 100644 index 000000000..0f22b1c42 --- /dev/null +++ b/mempool/txpool/locals/tx_tracker.go @@ -0,0 +1,216 @@ +// Copyright 2023 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Package locals implements tracking for "local" transactions +package locals + +import ( + "github.com/cosmos/evm/mempool/txpool" + "github.com/cosmos/evm/mempool/txpool/legacypool" + "slices" + "sync" + "time" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/metrics" + "github.com/ethereum/go-ethereum/params" +) + +var ( + recheckInterval = time.Minute + localGauge = metrics.GetOrRegisterGauge("txpool/local", nil) +) + +// TxTracker is a struct used to track priority transactions; it will check from +// time to time if the main pool has forgotten about any of the transaction +// it is tracking, and if so, submit it again. +// This is used to track 'locals'. +// This struct does not care about transaction validity, price-bumps or account limits, +// but optimistically accepts transactions. +type TxTracker struct { + all map[common.Hash]*types.Transaction // All tracked transactions + byAddr map[common.Address]*legacypool.SortedMap // Transactions by address + + journal *journal // Journal of local transaction to back up to disk + rejournal time.Duration // How often to rotate journal + pool *txpool.TxPool // The tx pool to interact with + signer types.Signer + + shutdownCh chan struct{} + mu sync.Mutex + wg sync.WaitGroup +} + +// New creates a new TxTracker +func New(journalPath string, journalTime time.Duration, chainConfig *params.ChainConfig, next *txpool.TxPool) *TxTracker { + pool := &TxTracker{ + all: make(map[common.Hash]*types.Transaction), + byAddr: make(map[common.Address]*legacypool.SortedMap), + signer: types.LatestSigner(chainConfig), + shutdownCh: make(chan struct{}), + pool: next, + } + if journalPath != "" { + pool.journal = newTxJournal(journalPath) + pool.rejournal = journalTime + } + return pool +} + +// Track adds a transaction to the tracked set. +// Note: blob-type transactions are ignored. +func (tracker *TxTracker) Track(tx *types.Transaction) { + tracker.TrackAll([]*types.Transaction{tx}) +} + +// TrackAll adds a list of transactions to the tracked set. +// Note: blob-type transactions are ignored. +func (tracker *TxTracker) TrackAll(txs []*types.Transaction) { + tracker.mu.Lock() + defer tracker.mu.Unlock() + + for _, tx := range txs { + if tx.Type() == types.BlobTxType { + continue + } + // If we're already tracking it, it's a no-op + if _, ok := tracker.all[tx.Hash()]; ok { + continue + } + // Theoretically, checking the error here is unnecessary since sender recovery + // is already part of basic validation. However, retrieving the sender address + // from the transaction cache is effectively a no-op if it was previously verified. + // Therefore, the error is still checked just in case. + addr, err := types.Sender(tracker.signer, tx) + if err != nil { + continue + } + tracker.all[tx.Hash()] = tx + if tracker.byAddr[addr] == nil { + tracker.byAddr[addr] = legacypool.NewSortedMap() + } + tracker.byAddr[addr].Put(tx) + + if tracker.journal != nil { + _ = tracker.journal.insert(tx) + } + } + localGauge.Update(int64(len(tracker.all))) +} + +// recheck checks and returns any transactions that needs to be resubmitted. +func (tracker *TxTracker) recheck(journalCheck bool) (resubmits []*types.Transaction, rejournal map[common.Address]types.Transactions) { + tracker.mu.Lock() + defer tracker.mu.Unlock() + + var ( + numStales = 0 + numOk = 0 + ) + for sender, txs := range tracker.byAddr { + // Wipe the stales + stales := txs.Forward(tracker.pool.Nonce(sender)) + for _, tx := range stales { + delete(tracker.all, tx.Hash()) + } + numStales += len(stales) + + // Check the non-stale + for _, tx := range txs.Flatten() { + if tracker.pool.Has(tx.Hash()) { + numOk++ + continue + } + resubmits = append(resubmits, tx) + } + } + + if journalCheck { // rejournal + rejournal = make(map[common.Address]types.Transactions) + for _, tx := range tracker.all { + addr, _ := types.Sender(tracker.signer, tx) + rejournal[addr] = append(rejournal[addr], tx) + } + // Sort them + for _, list := range rejournal { + // cmp(a, b) should return a negative number when a < b, + slices.SortFunc(list, func(a, b *types.Transaction) int { + return int(a.Nonce() - b.Nonce()) + }) + } + } + localGauge.Update(int64(len(tracker.all))) + log.Debug("Tx tracker status", "need-resubmit", len(resubmits), "stale", numStales, "ok", numOk) + return resubmits, rejournal +} + +// Start implements node.Lifecycle interface +// Start is called after all services have been constructed and the networking +// layer was also initialized to spawn any goroutines required by the service. +func (tracker *TxTracker) Start() error { + tracker.wg.Add(1) + go tracker.loop() + return nil +} + +// Stop implements node.Lifecycle interface +// Stop terminates all goroutines belonging to the service, blocking until they +// are all terminated. +func (tracker *TxTracker) Stop() error { + close(tracker.shutdownCh) + tracker.wg.Wait() + return nil +} + +func (tracker *TxTracker) loop() { + defer tracker.wg.Done() + + if tracker.journal != nil { + tracker.journal.load(func(transactions []*types.Transaction) []error { + tracker.TrackAll(transactions) + return nil + }) + defer tracker.journal.close() + } + var ( + lastJournal = time.Now() + timer = time.NewTimer(10 * time.Second) // Do initial check after 10 seconds, do rechecks more seldom. + ) + for { + select { + case <-tracker.shutdownCh: + return + case <-timer.C: + checkJournal := tracker.journal != nil && time.Since(lastJournal) > tracker.rejournal + resubmits, rejournal := tracker.recheck(checkJournal) + if len(resubmits) > 0 { + tracker.pool.Add(resubmits, false) + } + if checkJournal { + // Lock to prevent journal.rotate <-> journal.insert (via TrackAll) conflicts + tracker.mu.Lock() + lastJournal = time.Now() + if err := tracker.journal.rotate(rejournal); err != nil { + log.Warn("Transaction journal rotation failed", "err", err) + } + tracker.mu.Unlock() + } + timer.Reset(recheckInterval) + } + } +} diff --git a/mempool/txpool/locals/tx_tracker_test.go b/mempool/txpool/locals/tx_tracker_test.go new file mode 100644 index 000000000..367fb6b6d --- /dev/null +++ b/mempool/txpool/locals/tx_tracker_test.go @@ -0,0 +1,165 @@ +// Copyright 2025 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package locals + +import ( + "math/big" + "testing" + "time" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/consensus/ethash" + "github.com/ethereum/go-ethereum/core" + "github.com/ethereum/go-ethereum/core/rawdb" + "github.com/ethereum/go-ethereum/core/txpool" + "github.com/ethereum/go-ethereum/core/txpool/legacypool" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/crypto" + "github.com/ethereum/go-ethereum/ethdb" + "github.com/ethereum/go-ethereum/params" +) + +var ( + key, _ = crypto.HexToECDSA("b71c71a67e1177ad4e901695e1b4b9ee17ae16c6668d313eac2f96dbcda3f291") + address = crypto.PubkeyToAddress(key.PublicKey) + funds = big.NewInt(1000000000000000) + gspec = &core.Genesis{ + Config: params.TestChainConfig, + Alloc: types.GenesisAlloc{ + address: {Balance: funds}, + }, + BaseFee: big.NewInt(params.InitialBaseFee), + } + signer = types.LatestSigner(gspec.Config) +) + +type testEnv struct { + chain *core.BlockChain + pool *txpool.TxPool + tracker *TxTracker + genDb ethdb.Database +} + +func newTestEnv(t *testing.T, n int, gasTip uint64, journal string) *testEnv { + genDb, blocks, _ := core.GenerateChainWithGenesis(gspec, ethash.NewFaker(), n, func(i int, gen *core.BlockGen) { + tx, err := types.SignTx(types.NewTransaction(gen.TxNonce(address), common.Address{0x00}, big.NewInt(1000), params.TxGas, gen.BaseFee(), nil), signer, key) + if err != nil { + panic(err) + } + gen.AddTx(tx) + }) + + db := rawdb.NewMemoryDatabase() + chain, _ := core.NewBlockChain(db, gspec, ethash.NewFaker(), nil) + + legacyPool := legacypool.New(legacypool.DefaultConfig, chain) + pool, err := txpool.New(gasTip, chain, []txpool.SubPool{legacyPool}) + if err != nil { + t.Fatalf("Failed to create tx pool: %v", err) + } + if n, err := chain.InsertChain(blocks); err != nil { + t.Fatalf("Failed to process block %d: %v", n, err) + } + if err := pool.Sync(); err != nil { + t.Fatalf("Failed to sync the txpool, %v", err) + } + return &testEnv{ + chain: chain, + pool: pool, + tracker: New(journal, time.Minute, gspec.Config, pool), + genDb: genDb, + } +} + +func (env *testEnv) close() { + env.chain.Stop() +} + +// nolint:unused +func (env *testEnv) setGasTip(gasTip uint64) { + env.pool.SetGasTip(new(big.Int).SetUint64(gasTip)) +} + +// nolint:unused +func (env *testEnv) makeTx(nonce uint64, gasPrice *big.Int) *types.Transaction { + if nonce == 0 { + head := env.chain.CurrentHeader() + state, _ := env.chain.StateAt(head.Root) + nonce = state.GetNonce(address) + } + if gasPrice == nil { + gasPrice = big.NewInt(params.GWei) + } + tx, _ := types.SignTx(types.NewTransaction(nonce, common.Address{0x00}, big.NewInt(1000), params.TxGas, gasPrice, nil), signer, key) + return tx +} + +func (env *testEnv) makeTxs(n int) []*types.Transaction { + head := env.chain.CurrentHeader() + state, _ := env.chain.StateAt(head.Root) + nonce := state.GetNonce(address) + + var txs []*types.Transaction + for i := 0; i < n; i++ { + tx, _ := types.SignTx(types.NewTransaction(nonce+uint64(i), common.Address{0x00}, big.NewInt(1000), params.TxGas, big.NewInt(params.GWei), nil), signer, key) + txs = append(txs, tx) + } + return txs +} + +// nolint:unused +func (env *testEnv) commit() { + head := env.chain.CurrentBlock() + block := env.chain.GetBlock(head.Hash(), head.Number.Uint64()) + blocks, _ := core.GenerateChain(env.chain.Config(), block, ethash.NewFaker(), env.genDb, 1, func(i int, gen *core.BlockGen) { + tx, err := types.SignTx(types.NewTransaction(gen.TxNonce(address), common.Address{0x00}, big.NewInt(1000), params.TxGas, gen.BaseFee(), nil), signer, key) + if err != nil { + panic(err) + } + gen.AddTx(tx) + }) + env.chain.InsertChain(blocks) + if err := env.pool.Sync(); err != nil { + panic(err) + } +} + +func TestResubmit(t *testing.T) { + env := newTestEnv(t, 10, 0, "") + defer env.close() + + txs := env.makeTxs(10) + txsA := txs[:len(txs)/2] + txsB := txs[len(txs)/2:] + env.pool.Add(txsA, true) + pending, queued := env.pool.ContentFrom(address) + if len(pending) != len(txsA) || len(queued) != 0 { + t.Fatalf("Unexpected txpool content: %d, %d", len(pending), len(queued)) + } + env.tracker.TrackAll(txs) + + resubmit, all := env.tracker.recheck(true) + if len(resubmit) != len(txsB) { + t.Fatalf("Unexpected transactions to resubmit, got: %d, want: %d", len(resubmit), len(txsB)) + } + if len(all) == 0 || len(all[address]) == 0 { + t.Fatalf("Unexpected transactions being tracked, got: %d, want: %d", 0, len(txs)) + } + if len(all[address]) != len(txs) { + t.Fatalf("Unexpected transactions being tracked, got: %d, want: %d", len(all[address]), len(txs)) + } +} From a720a8d6f52dc5bded6f87da78ade25363410173 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Fri, 24 Oct 2025 21:14:48 +0600 Subject: [PATCH 02/47] add rejection handling and new tests --- CHANGELOG.md | 1 + config/server_app_options.go | 9 ++ mempool/mempool.go | 12 ++- mempool/track_local_txs_test.go | 4 + mempool/txpool/locals/errors.go | 20 +---- mempool/txpool/locals/errors_test.go | 48 +++++++++++ mempool/txpool/locals/tx_tracker.go | 4 +- mempool/txpool/locals/tx_tracker_test.go | 104 ++++++++++++++++++++++- rpc/backend/call_tx.go | 15 ++++ rpc/backend/sign_tx.go | 13 +++ 10 files changed, 204 insertions(+), 26 deletions(-) create mode 100644 mempool/track_local_txs_test.go create mode 100644 mempool/txpool/locals/errors_test.go diff --git a/CHANGELOG.md b/CHANGELOG.md index 1ac4e2df3..74e0fdb89 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -82,6 +82,7 @@ ### FEATURES +- [\#646](https://github.com/cosmos/evm/pull/646) Add TxTracker support for tracking priority transactions and handling temporary rejections - [\#665](https://github.com/cosmos/evm/pull/665) Add EvmCodec address codec implementation - [\#346](https://github.com/cosmos/evm/pull/346) Add eth_createAccessList method and implementation - [\#337](https://github.com/cosmos/evm/pull/337) Support state overrides in eth_call. diff --git a/config/server_app_options.go b/config/server_app_options.go index c98f1bf2a..46bdd305c 100644 --- a/config/server_app_options.go +++ b/config/server_app_options.go @@ -2,6 +2,7 @@ package config import ( "math" + "os" "path/filepath" "github.com/holiman/uint256" @@ -140,6 +141,14 @@ func GetLegacyPoolConfig(appOpts servertypes.AppOptions, logger log.Logger) *leg legacyConfig.Lifetime = lifetime } + // Set journal path under .evmd data dir and ensure dir exists + homeDir := cast.ToString(appOpts.Get(flags.FlagHome)) + if homeDir != "" { + journalDir := filepath.Join(homeDir, "data", "txpool") + _ = os.MkdirAll(journalDir, 0o755) + legacyConfig.Journal = filepath.Join(journalDir, legacyConfig.Journal) + } + return &legacyConfig } diff --git a/mempool/mempool.go b/mempool/mempool.go index d768e2610..d5ea3729d 100644 --- a/mempool/mempool.go +++ b/mempool/mempool.go @@ -7,8 +7,6 @@ import ( "sync" "time" - "github.com/cosmos/evm/mempool/txpool/locals" - ethtypes "github.com/ethereum/go-ethereum/core/types" "github.com/holiman/uint256" @@ -17,6 +15,7 @@ import ( "github.com/cosmos/evm/mempool/miner" "github.com/cosmos/evm/mempool/txpool" "github.com/cosmos/evm/mempool/txpool/legacypool" + "github.com/cosmos/evm/mempool/txpool/locals" "github.com/cosmos/evm/rpc/stream" evmtypes "github.com/cosmos/evm/x/vm/types" @@ -218,6 +217,15 @@ func NewExperimentalEVMMempool( return evmMempool } +// TrackLocalTxs tracks transactions as local priority via TxTracker. +// No-op if local tracking is not initialized. +func (m *ExperimentalEVMMempool) TrackLocalTxs(txs []*ethtypes.Transaction) { + if m == nil || m.localTxTracker == nil || len(txs) == 0 { + return + } + m.localTxTracker.TrackAll(txs) +} + // GetBlockchain returns the blockchain interface used for chain head event notifications. // This is primarily used to notify the mempool when new blocks are finalized. func (m *ExperimentalEVMMempool) GetBlockchain() *Blockchain { diff --git a/mempool/track_local_txs_test.go b/mempool/track_local_txs_test.go new file mode 100644 index 000000000..74aed4921 --- /dev/null +++ b/mempool/track_local_txs_test.go @@ -0,0 +1,4 @@ +// Intentionally left empty; comprehensive TxTracker behavior is tested in +// package locals where internals are accessible. This file reserved for future +// integration wiring tests if needed. +package mempool diff --git a/mempool/txpool/locals/errors.go b/mempool/txpool/locals/errors.go index fda50bf21..2eca36b8e 100644 --- a/mempool/txpool/locals/errors.go +++ b/mempool/txpool/locals/errors.go @@ -1,26 +1,10 @@ -// Copyright 2025 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - package locals import ( "errors" - "github.com/ethereum/go-ethereum/core/txpool" - "github.com/ethereum/go-ethereum/core/txpool/legacypool" + "github.com/cosmos/evm/mempool/txpool" + "github.com/cosmos/evm/mempool/txpool/legacypool" ) // IsTemporaryReject determines whether the given error indicates a temporary diff --git a/mempool/txpool/locals/errors_test.go b/mempool/txpool/locals/errors_test.go new file mode 100644 index 000000000..a163131b7 --- /dev/null +++ b/mempool/txpool/locals/errors_test.go @@ -0,0 +1,48 @@ +package locals + +import ( + "errors" + "testing" + + "github.com/cosmos/evm/mempool/txpool" + "github.com/cosmos/evm/mempool/txpool/legacypool" +) + +func TestIsTemporaryReject_PositiveCases(t *testing.T) { + cases := []struct { + name string + err error + }{ + {name: "delegated out-of-order nonce", err: legacypool.ErrOutOfOrderTxFromDelegated}, + {name: "inflight tx limit reached", err: txpool.ErrInflightTxLimitReached}, + {name: "authority reserved", err: legacypool.ErrAuthorityReserved}, + {name: "underpriced", err: txpool.ErrUnderpriced}, + {name: "txpool overflow", err: legacypool.ErrTxPoolOverflow}, + {name: "future replace pending", err: legacypool.ErrFutureReplacePending}, + } + for _, tc := range cases { + t.Run(tc.name, func(t *testing.T) { + if !IsTemporaryReject(tc.err) { + t.Fatalf("expected temporary reject error to be detected, got false: %v", tc.err) + } + }) + } +} + +func TestIsTemporaryReject_NegativeCases(t *testing.T) { + cases := []struct { + name string + err error + }{ + {name: "nil", err: nil}, + {name: "unrelated", err: errors.New("some unrelated error")}, + {name: "substring lookalike", err: errors.New("under price threshold")}, + } + for _, tc := range cases { + t.Run(tc.name, func(t *testing.T) { + if IsTemporaryReject(tc.err) { + t.Fatalf("did not expect temporary reject error for: %v", tc.err) + } + }) + } +} diff --git a/mempool/txpool/locals/tx_tracker.go b/mempool/txpool/locals/tx_tracker.go index 0f22b1c42..7b616e167 100644 --- a/mempool/txpool/locals/tx_tracker.go +++ b/mempool/txpool/locals/tx_tracker.go @@ -18,12 +18,12 @@ package locals import ( - "github.com/cosmos/evm/mempool/txpool" - "github.com/cosmos/evm/mempool/txpool/legacypool" "slices" "sync" "time" + "github.com/cosmos/evm/mempool/txpool" + "github.com/cosmos/evm/mempool/txpool/legacypool" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/log" diff --git a/mempool/txpool/locals/tx_tracker_test.go b/mempool/txpool/locals/tx_tracker_test.go index 367fb6b6d..af9ef713b 100644 --- a/mempool/txpool/locals/tx_tracker_test.go +++ b/mempool/txpool/locals/tx_tracker_test.go @@ -21,15 +21,17 @@ import ( "testing" "time" + "github.com/cosmos/evm/mempool/txpool" + "github.com/cosmos/evm/mempool/txpool/legacypool" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/consensus/ethash" "github.com/ethereum/go-ethereum/core" "github.com/ethereum/go-ethereum/core/rawdb" - "github.com/ethereum/go-ethereum/core/txpool" - "github.com/ethereum/go-ethereum/core/txpool/legacypool" "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/core/vm" "github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/ethdb" + "github.com/ethereum/go-ethereum/event" "github.com/ethereum/go-ethereum/params" ) @@ -54,6 +56,22 @@ type testEnv struct { genDb ethdb.Database } +// testChainAdapter adapts geth *core.BlockChain to the local txpool/legacypool BlockChain interfaces. +type testChainAdapter struct{ c *core.BlockChain } + +func (a *testChainAdapter) Config() *params.ChainConfig { return a.c.Config() } +func (a *testChainAdapter) CurrentBlock() *types.Header { return a.c.CurrentHeader() } +func (a *testChainAdapter) GetBlock(hash common.Hash, number uint64) *types.Block { + return a.c.GetBlock(hash, number) +} +func (a *testChainAdapter) StateAt(root common.Hash) (vm.StateDB, error) { + st, err := a.c.StateAt(root) + return st, err +} +func (a *testChainAdapter) SubscribeChainHeadEvent(ch chan<- core.ChainHeadEvent) event.Subscription { + return a.c.SubscribeChainHeadEvent(ch) +} + func newTestEnv(t *testing.T, n int, gasTip uint64, journal string) *testEnv { genDb, blocks, _ := core.GenerateChainWithGenesis(gspec, ethash.NewFaker(), n, func(i int, gen *core.BlockGen) { tx, err := types.SignTx(types.NewTransaction(gen.TxNonce(address), common.Address{0x00}, big.NewInt(1000), params.TxGas, gen.BaseFee(), nil), signer, key) @@ -66,8 +84,9 @@ func newTestEnv(t *testing.T, n int, gasTip uint64, journal string) *testEnv { db := rawdb.NewMemoryDatabase() chain, _ := core.NewBlockChain(db, gspec, ethash.NewFaker(), nil) - legacyPool := legacypool.New(legacypool.DefaultConfig, chain) - pool, err := txpool.New(gasTip, chain, []txpool.SubPool{legacyPool}) + adapter := &testChainAdapter{c: chain} + legacyPool := legacypool.New(legacypool.DefaultConfig, adapter) + pool, err := txpool.New(gasTip, adapter, []txpool.SubPool{legacyPool}) if err != nil { t.Fatalf("Failed to create tx pool: %v", err) } @@ -163,3 +182,80 @@ func TestResubmit(t *testing.T) { t.Fatalf("Unexpected transactions being tracked, got: %d, want: %d", len(all[address]), len(txs)) } } + +func TestTrackAddsAndResubmitsAll(t *testing.T) { + env := newTestEnv(t, 5, 0, "") + defer env.close() + + // Create a contiguous set of txs but do NOT add any to the pool + txs := env.makeTxs(5) + env.tracker.TrackAll(txs) + + // Since none are present in the pool, all should be scheduled for resubmission + resubmit, all := env.tracker.recheck(true) + if len(resubmit) != len(txs) { + t.Fatalf("expected all transactions to be resubmitted, got %d want %d", len(resubmit), len(txs)) + } + if len(all[address]) != len(txs) { + t.Fatalf("expected all transactions tracked, got %d want %d", len(all[address]), len(txs)) + } + + // Now add them to the pool as if resubmitted + env.pool.Add(txs, false) + + resubmit2, _ := env.tracker.recheck(false) + if len(resubmit2) != 0 { + t.Fatalf("expected no resubmissions after promotion, got %d", len(resubmit2)) + } +} + +func TestDropObsoleteOnHigherNonce(t *testing.T) { + env := newTestEnv(t, 5, 0, "") + defer env.close() + + // Make and track 6 txs starting at current nonce + txs := env.makeTxs(6) + env.tracker.TrackAll(txs) + + // Advance the chain/account nonce by 3 (mine 3 blocks each adding a tx from the same account) + for i := 0; i < 3; i++ { + env.commit() + } + + // Recheck should drop the first 3 as stale + resubmit, all := env.tracker.recheck(true) + if len(all[address]) != 3 { + t.Fatalf("expected 3 transactions to remain tracked after nonce advance, got %d", len(all[address])) + } + for _, tx := range resubmit { + // none of the resubmits should have nonce less than current pool nonce + sender, _ := types.Sender(signer, tx) + if tx.Nonce() < env.pool.Nonce(sender) { + t.Fatalf("found stale tx in resubmits: nonce %d < pool nonce %d", tx.Nonce(), env.pool.Nonce(sender)) + } + } +} + +func TestPromoteThenNoRetry(t *testing.T) { + env := newTestEnv(t, 4, 0, "") + defer env.close() + + // Track 4 txs, add 2 to pool. Expect 2 resubmits. + txs := env.makeTxs(4) + txsA := txs[:2] + txsB := txs[2:] + env.pool.Add(txsA, true) + env.tracker.TrackAll(txs) + + resubmit, _ := env.tracker.recheck(false) + if len(resubmit) != len(txsB) { + t.Fatalf("unexpected resubmit count, got %d want %d", len(resubmit), len(txsB)) + } + + // Promote missing ones by adding them; next recheck should yield none + env.pool.Add(resubmit, false) + resubmit2, _ := env.tracker.recheck(false) + if len(resubmit2) != 0 { + t.Fatalf("expected no resubmits after all txs present in pool, got %d", len(resubmit2)) + } +} diff --git a/rpc/backend/call_tx.go b/rpc/backend/call_tx.go index 61dfc0b34..5c65bce3b 100644 --- a/rpc/backend/call_tx.go +++ b/rpc/backend/call_tx.go @@ -17,6 +17,7 @@ import ( "google.golang.org/grpc/status" "github.com/cosmos/evm/mempool" + txlocals "github.com/cosmos/evm/mempool/txpool/locals" rpctypes "github.com/cosmos/evm/rpc/types" evmtypes "github.com/cosmos/evm/x/vm/types" @@ -158,6 +159,14 @@ func (b *Backend) SendRawTransaction(data hexutil.Bytes) (common.Hash, error) { if b.Mempool != nil && strings.Contains(err.Error(), mempool.ErrNonceGap.Error()) { // Transaction was successfully queued due to nonce gap, return success to client b.Logger.Debug("transaction queued due to nonce gap", "hash", txHash.Hex()) + // Track as local for priority and persistence + b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{tx}) + return txHash, nil + } + // Temporary txpool rejections should be locally tracked for resubmission + if b.Mempool != nil && txlocals.IsTemporaryReject(err) { + b.Logger.Debug("temporary rejection, tracking locally", "hash", txHash.Hex(), "err", err.Error()) + b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{tx}) return txHash, nil } if b.Mempool != nil && strings.Contains(err.Error(), mempool.ErrNonceLow.Error()) { @@ -176,6 +185,8 @@ func (b *Backend) SendRawTransaction(data hexutil.Bytes) (common.Hash, error) { } // SendRawTransaction does not return error when committed nonce <= tx.Nonce < pending nonce + // Track as local for persistence until mined + b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{tx}) return txHash, nil } @@ -183,6 +194,10 @@ func (b *Backend) SendRawTransaction(data hexutil.Bytes) (common.Hash, error) { return txHash, fmt.Errorf("failed to broadcast transaction: %w", err) } + // On success, track as local too to persist across restarts until mined + if b.Mempool != nil { + b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{tx}) + } return txHash, nil } diff --git a/rpc/backend/sign_tx.go b/rpc/backend/sign_tx.go index 859c2c3a8..a8bd67437 100644 --- a/rpc/backend/sign_tx.go +++ b/rpc/backend/sign_tx.go @@ -14,6 +14,7 @@ import ( "github.com/ethereum/go-ethereum/signer/core/apitypes" "github.com/cosmos/evm/mempool" + txlocals "github.com/cosmos/evm/mempool/txpool/locals" evmtypes "github.com/cosmos/evm/x/vm/types" errorsmod "cosmossdk.io/errors" @@ -113,6 +114,14 @@ func (b *Backend) SendTransaction(args evmtypes.TransactionArgs) (common.Hash, e if b.Mempool != nil && strings.Contains(err.Error(), mempool.ErrNonceGap.Error()) { // Transaction was successfully queued due to nonce gap, return success to client b.Logger.Debug("transaction queued due to nonce gap", "hash", txHash.Hex()) + // Track as local for priority and persistence + b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{ethTx}) + return txHash, nil + } + // Temporary txpool rejections should be locally tracked for resubmission + if b.Mempool != nil && txlocals.IsTemporaryReject(err) { + b.Logger.Debug("temporary rejection, tracking locally", "hash", txHash.Hex(), "err", err.Error()) + b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{ethTx}) return txHash, nil } b.Logger.Error("failed to broadcast tx", "error", err.Error()) @@ -120,6 +129,10 @@ func (b *Backend) SendTransaction(args evmtypes.TransactionArgs) (common.Hash, e } // Return transaction hash + // On success, track as local too to persist across restarts until mined + if b.Mempool != nil { + b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{ethTx}) + } return txHash, nil } From 8433cc58895386531074f99281358c37bc650fe4 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Mon, 27 Oct 2025 19:07:45 +0600 Subject: [PATCH 03/47] add journal file opening step --- config/server_app_options.go | 3 +++ mempool/txpool/locals/journal.go | 15 +++++++++++++++ mempool/txpool/locals/tx_tracker.go | 9 ++++++++- 3 files changed, 26 insertions(+), 1 deletion(-) diff --git a/config/server_app_options.go b/config/server_app_options.go index 46bdd305c..9858633be 100644 --- a/config/server_app_options.go +++ b/config/server_app_options.go @@ -147,6 +147,9 @@ func GetLegacyPoolConfig(appOpts servertypes.AppOptions, logger log.Logger) *leg journalDir := filepath.Join(homeDir, "data", "txpool") _ = os.MkdirAll(journalDir, 0o755) legacyConfig.Journal = filepath.Join(journalDir, legacyConfig.Journal) + } else { + // Disable local transaction journaling when no home directory is set (e.g. in tests) + legacyConfig.Journal = "" } return &legacyConfig diff --git a/mempool/txpool/locals/journal.go b/mempool/txpool/locals/journal.go index 46fd6de34..74977a905 100644 --- a/mempool/txpool/locals/journal.go +++ b/mempool/txpool/locals/journal.go @@ -174,6 +174,21 @@ func (journal *journal) rotate(all map[common.Address]types.Transactions) error return nil } +// open opens the journal file for writing (in append mode). +// This should be called after load() to ensure new transactions are persisted. +func (journal *journal) open() error { + if journal.writer != nil { + return nil // Already open + } + // Open file for appending, create if it doesn't exist + sink, err := os.OpenFile(journal.path, os.O_WRONLY|os.O_APPEND|os.O_CREATE, 0644) + if err != nil { + return err + } + journal.writer = sink + return nil +} + // close flushes the transaction journal contents to disk and closes the file. func (journal *journal) close() error { var err error diff --git a/mempool/txpool/locals/tx_tracker.go b/mempool/txpool/locals/tx_tracker.go index 7b616e167..fab9747af 100644 --- a/mempool/txpool/locals/tx_tracker.go +++ b/mempool/txpool/locals/tx_tracker.go @@ -107,7 +107,9 @@ func (tracker *TxTracker) TrackAll(txs []*types.Transaction) { tracker.byAddr[addr].Put(tx) if tracker.journal != nil { - _ = tracker.journal.insert(tx) + if err := tracker.journal.insert(tx); err != nil { + log.Warn("Failed to journal local transaction", "hash", tx.Hash(), "err", err) + } } } localGauge.Update(int64(len(tracker.all))) @@ -185,6 +187,11 @@ func (tracker *TxTracker) loop() { tracker.TrackAll(transactions) return nil }) + // Open the journal for writing immediately after load so new transactions + // are persisted right away instead of waiting for first rotation + if err := tracker.journal.open(); err != nil { + log.Warn("Failed to open transaction journal for writing", "err", err) + } defer tracker.journal.close() } var ( From 0ecd5ee36a571b55ba214da824c6ea8afcaea580 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Mon, 27 Oct 2025 21:44:00 +0600 Subject: [PATCH 04/47] add local tx configuration to mempool params --- config/server_app_options.go | 46 +++++++++++++++++++++++++++--------- server/config/config.go | 34 +++++++++++++++++++------- server/config/toml.go | 12 ++++++++++ server/flags/flags.go | 4 ++++ server/start.go | 6 +++++ 5 files changed, 82 insertions(+), 20 deletions(-) diff --git a/config/server_app_options.go b/config/server_app_options.go index 9858633be..eaeb85cb9 100644 --- a/config/server_app_options.go +++ b/config/server_app_options.go @@ -2,9 +2,10 @@ package config import ( "math" - "os" "path/filepath" + "strings" + "github.com/ethereum/go-ethereum/common" "github.com/holiman/uint256" "github.com/spf13/cast" @@ -91,6 +92,21 @@ func GetMinGasPrices(appOpts servertypes.AppOptions, logger log.Logger) sdk.DecC return minGasPrices } +// parseAddresses converts a slice of address strings to common.Address slice +func parseAddresses(addressStrs []string) []common.Address { + addresses := make([]common.Address, 0, len(addressStrs)) + for _, addrStr := range addressStrs { + addrStr = strings.TrimSpace(addrStr) + if addrStr == "" { + continue + } + if common.IsHexAddress(addrStr) { + addresses = append(addresses, common.HexToAddress(addrStr)) + } + } + return addresses +} + // GetMinTip reads the min tip from the app options, set from app.toml // This field is also known as the minimum priority fee func GetMinTip(appOpts servertypes.AppOptions, logger log.Logger) *uint256.Int { @@ -140,16 +156,24 @@ func GetLegacyPoolConfig(appOpts servertypes.AppOptions, logger log.Logger) *leg if lifetime := cast.ToDuration(appOpts.Get(srvflags.EVMMempoolLifetime)); lifetime != 0 { legacyConfig.Lifetime = lifetime } - - // Set journal path under .evmd data dir and ensure dir exists - homeDir := cast.ToString(appOpts.Get(flags.FlagHome)) - if homeDir != "" { - journalDir := filepath.Join(homeDir, "data", "txpool") - _ = os.MkdirAll(journalDir, 0o755) - legacyConfig.Journal = filepath.Join(journalDir, legacyConfig.Journal) - } else { - // Disable local transaction journaling when no home directory is set (e.g. in tests) - legacyConfig.Journal = "" + if localsSlice := cast.ToStringSlice(appOpts.Get(srvflags.EVMMempoolLocals)); len(localsSlice) > 0 { + legacyConfig.Locals = parseAddresses(localsSlice) + } + if noLocals := appOpts.Get(srvflags.EVMMempoolNoLocals); noLocals != nil { + legacyConfig.NoLocals = cast.ToBool(noLocals) + } + if journal := cast.ToString(appOpts.Get(srvflags.EVMMempoolJournal)); journal != "" { + legacyConfig.Journal = journal + if homeDir := cast.ToString(appOpts.Get(flags.FlagHome)); homeDir != "" { + legacyConfig.Journal = filepath.Join(homeDir, "data", "txpool", journal) + } else { + // this is a valid configuration for testing where we don't want to persist local transactions + logger.Warn("no home directory set, disabling local transaction journaling", "journal", journal) + legacyConfig.Journal = "" + } + } + if rejournal := cast.ToDuration(appOpts.Get(srvflags.EVMMempoolRejournal)); rejournal != 0 { + legacyConfig.Rejournal = rejournal } return &legacyConfig diff --git a/server/config/config.go b/server/config/config.go index ddec86698..85f101f8e 100644 --- a/server/config/config.go +++ b/server/config/config.go @@ -5,11 +5,12 @@ import ( "fmt" "net/netip" "path" + "strings" "time" "github.com/spf13/viper" - "github.com/cometbft/cometbft/libs/strings" + cometstrs "github.com/cometbft/cometbft/libs/strings" errorsmod "cosmossdk.io/errors" @@ -171,18 +172,30 @@ type MempoolConfig struct { GlobalQueue uint64 `mapstructure:"global-queue"` // Lifetime is the maximum amount of time non-executable transaction are queued Lifetime time.Duration `mapstructure:"lifetime"` + // Locals is the set of addresses that should be treated by default as local + Locals []string `mapstructure:"locals"` + // NoLocals disables local transaction handling, exempting local accounts from pricing and acceptance + NoLocals bool `mapstructure:"no-locals"` + // Journal is the path to the local transaction journal file + Journal string `mapstructure:"journal"` + // Rejournal is the time interval to regenerate the local transaction journal + Rejournal time.Duration `mapstructure:"rejournal"` } // DefaultMempoolConfig returns the default mempool configuration func DefaultMempoolConfig() MempoolConfig { return MempoolConfig{ - PriceLimit: 1, // Minimum gas price of 1 wei - PriceBump: 10, // 10% price bump to replace transaction - AccountSlots: 16, // 16 executable transaction slots per account - GlobalSlots: 5120, // 4096 + 1024 = 5120 global executable slots - AccountQueue: 64, // 64 non-executable transaction slots per account - GlobalQueue: 1024, // 1024 global non-executable slots - Lifetime: 3 * time.Hour, // 3 hour lifetime for queued transactions + PriceLimit: 1, // Minimum gas price of 1 wei + PriceBump: 10, // 10% price bump to replace transaction + AccountSlots: 16, // 16 executable transaction slots per account + GlobalSlots: 5120, // 4096 + 1024 = 5120 global executable slots + AccountQueue: 64, // 64 non-executable transaction slots per account + GlobalQueue: 1024, // 1024 global non-executable slots + Lifetime: 3 * time.Hour, // 3 hour lifetime for queued transactions + Locals: []string{}, // No local addresses by default + NoLocals: false, // Local transaction handling enabled by default + Journal: "transactions.rlp", // Default journal filename + Rejournal: time.Hour, // Regenerate journal every hour } } @@ -209,6 +222,9 @@ func (c MempoolConfig) Validate() error { if c.Lifetime < 1 { return fmt.Errorf("lifetime must be at least 1 nanosecond, got %s", c.Lifetime) } + if !strings.HasSuffix(c.Journal, ".rlp") { + return fmt.Errorf("journal must end with .rlp, got %s", c.Journal) + } return nil } @@ -285,7 +301,7 @@ func DefaultEVMConfig() *EVMConfig { // Validate returns an error if the tracer type is invalid. func (c EVMConfig) Validate() error { - if c.Tracer != "" && !strings.StringInSlice(c.Tracer, evmTracers) { + if c.Tracer != "" && !cometstrs.StringInSlice(c.Tracer, evmTracers) { return fmt.Errorf("invalid tracer type %s, available types: %v", c.Tracer, evmTracers) } diff --git a/server/config/toml.go b/server/config/toml.go index 60bd3402d..b3f8bc39f 100644 --- a/server/config/toml.go +++ b/server/config/toml.go @@ -52,6 +52,18 @@ global-queue = {{ .EVM.Mempool.GlobalQueue }} # Lifetime is the maximum amount of time non-executable transaction are queued lifetime = "{{ .EVM.Mempool.Lifetime }}" +# Locals is the set of addresses that should be treated by default as local (comma-separated) +locals = [{{range $index, $elmt := .EVM.Mempool.Locals}}{{if $index}}, {{end}}"{{$elmt}}"{{end}}] + +# NoLocals disables local transaction handling, exempting local accounts from pricing and acceptance +no-locals = {{ .EVM.Mempool.NoLocals }} + +# Journal is the path to the local transaction journal file +journal = "{{ .EVM.Mempool.Journal }}" + +# Rejournal is the time interval to regenerate the local transaction journal +rejournal = "{{ .EVM.Mempool.Rejournal }}" + ############################################################################### ### JSON RPC Configuration ### ############################################################################### diff --git a/server/flags/flags.go b/server/flags/flags.go index 9550a7747..5df87d0e1 100644 --- a/server/flags/flags.go +++ b/server/flags/flags.go @@ -79,6 +79,10 @@ const ( EVMMempoolAccountQueue = "evm.mempool.account-queue" EVMMempoolGlobalQueue = "evm.mempool.global-queue" EVMMempoolLifetime = "evm.mempool.lifetime" + EVMMempoolLocals = "evm.mempool.locals" + EVMMempoolNoLocals = "evm.mempool.no-locals" + EVMMempoolJournal = "evm.mempool.journal" + EVMMempoolRejournal = "evm.mempool.rejournal" ) // TLS flags diff --git a/server/start.go b/server/start.go index 725778857..038335e50 100644 --- a/server/start.go +++ b/server/start.go @@ -8,6 +8,7 @@ import ( "os" "path/filepath" "runtime/pprof" + "time" ethmetricsexp "github.com/ethereum/go-ethereum/metrics/exp" "github.com/spf13/cobra" @@ -232,6 +233,11 @@ which accepts a path for the resulting pprof file. cmd.Flags().Uint64(srvflags.EVMMempoolGlobalQueue, cosmosevmserverconfig.DefaultMempoolConfig().GlobalQueue, "the maximum number of non-executable transaction slots for all accounts") cmd.Flags().Duration(srvflags.EVMMempoolLifetime, cosmosevmserverconfig.DefaultMempoolConfig().Lifetime, "the maximum amount of time non-executable transaction are queued") + cmd.Flags().StringSlice(srvflags.EVMMempoolLocals, []string{}, "the set of addresses that should be treated by default as local (comma-separated)") + cmd.Flags().Bool(srvflags.EVMMempoolNoLocals, false, "disables local transaction handling, exempting local accounts from pricing and acceptance") + cmd.Flags().String(srvflags.EVMMempoolJournal, "", "the path to the local transaction journal file") + cmd.Flags().Duration(srvflags.EVMMempoolRejournal, time.Hour, "the time interval to regenerate the local transaction journal") + cmd.Flags().String(srvflags.TLSCertPath, "", "the cert.pem file path for the server TLS configuration") cmd.Flags().String(srvflags.TLSKeyPath, "", "the key.pem file path for the server TLS configuration") From 75efc7df1badb6fe9b6d9fbcb56a5a4001db12e4 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Tue, 4 Nov 2025 15:57:12 -0500 Subject: [PATCH 05/47] use sdk legacy --- mempool/txpool/legacypool/legacypool.go | 33 +++++--------------- mempool/txpool/legacypool/legacypool_test.go | 13 ++++---- 2 files changed, 14 insertions(+), 32 deletions(-) diff --git a/mempool/txpool/legacypool/legacypool.go b/mempool/txpool/legacypool/legacypool.go index 7139a3b9d..4faccebdf 100644 --- a/mempool/txpool/legacypool/legacypool.go +++ b/mempool/txpool/legacypool/legacypool.go @@ -18,7 +18,6 @@ package legacypool import ( - "errors" "maps" "math/big" "slices" @@ -31,6 +30,7 @@ import ( "github.com/ethereum/go-ethereum/common/prque" "github.com/ethereum/go-ethereum/consensus/misc/eip1559" "github.com/ethereum/go-ethereum/core" + "github.com/ethereum/go-ethereum/core/txpool/legacypool" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/vm" "github.com/ethereum/go-ethereum/crypto/kzg4844" @@ -58,25 +58,6 @@ const ( txMaxSize = 4 * txSlotSize // 128KB ) -var ( - // ErrTxPoolOverflow is returned if the transaction pool is full and can't accept - // another remote transaction. - ErrTxPoolOverflow = errors.New("txpool is full") - - // ErrOutOfOrderTxFromDelegated is returned when the transaction with gapped - // nonce received from the accounts with delegation or pending delegation. - ErrOutOfOrderTxFromDelegated = errors.New("gapped-nonce tx from delegated accounts") - - // ErrAuthorityReserved is returned if a transaction has an authorization - // signed by an address which already has in-flight transactions known to the - // pool. - ErrAuthorityReserved = errors.New("authority already reserved") - - // ErrFutureReplacePending is returned if a future transaction replaces a pending - // one. Future transactions should only be able to replace other future transactions. - ErrFutureReplacePending = errors.New("future transaction tries to replace pending") -) - var ( evictionInterval = time.Minute // Time interval to check for evictable transactions statsReportInterval = 8 * time.Second // Time interval to report transaction pool stats @@ -623,7 +604,7 @@ func (pool *LegacyPool) checkDelegationLimit(tx *types.Transaction) error { if pending == nil { // Transaction with gapped nonce is not supported for delegated accounts if pool.pendingNonces.get(from) != tx.Nonce() { - return ErrOutOfOrderTxFromDelegated + return legacypool.ErrOutOfOrderTxFromDelegated } return nil } @@ -654,7 +635,7 @@ func (pool *LegacyPool) validateAuth(tx *types.Transaction) error { count += queue.Len() } if count > 1 { - return ErrAuthorityReserved + return legacypool.ErrAuthorityReserved } // Because there is no exclusive lock held between different subpools // when processing transactions, the SetCode transaction may be accepted @@ -665,7 +646,7 @@ func (pool *LegacyPool) validateAuth(tx *types.Transaction) error { // that attackers cannot easily stack a SetCode transaction when the sender // is reserved by other pools. if pool.reserver.Has(auth) { - return ErrAuthorityReserved + return legacypool.ErrAuthorityReserved } } } @@ -730,7 +711,7 @@ func (pool *LegacyPool) add(tx *types.Transaction) (replaced bool, err error) { // replacements to 25% of the slots if pool.changesSinceReorg > int(pool.config.GlobalSlots/4) { throttleTxMeter.Mark(1) - return false, ErrTxPoolOverflow + return false, legacypool.ErrTxPoolOverflow } // New transaction is better than our worse ones, make room for it. @@ -741,7 +722,7 @@ func (pool *LegacyPool) add(tx *types.Transaction) (replaced bool, err error) { if !success { log.Trace("Discarding overflown transaction", "hash", hash) overflowedTxMeter.Mark(1) - return false, ErrTxPoolOverflow + return false, legacypool.ErrTxPoolOverflow } // If the new transaction is a future transaction it should never churn pending transactions @@ -760,7 +741,7 @@ func (pool *LegacyPool) add(tx *types.Transaction) (replaced bool, err error) { pool.priced.Put(dropTx) } log.Trace("Discarding future transaction replacing pending tx", "hash", hash) - return false, ErrFutureReplacePending + return false, legacypool.ErrFutureReplacePending } } diff --git a/mempool/txpool/legacypool/legacypool_test.go b/mempool/txpool/legacypool/legacypool_test.go index f0ed2dfb7..89ac36263 100644 --- a/mempool/txpool/legacypool/legacypool_test.go +++ b/mempool/txpool/legacypool/legacypool_test.go @@ -33,6 +33,7 @@ import ( "github.com/ethereum/go-ethereum/core" "github.com/ethereum/go-ethereum/core/state" "github.com/ethereum/go-ethereum/core/tracing" + "github.com/ethereum/go-ethereum/core/txpool/legacypool" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/vm" "github.com/ethereum/go-ethereum/crypto" @@ -1696,8 +1697,8 @@ func TestUnderpricing(t *testing.T) { t.Fatalf("failed to add well priced transaction: %v", err) } // Ensure that replacing a pending transaction with a future transaction fails - if err := pool.addRemoteSync(pricedTransaction(5, 100000, big.NewInt(6), keys[1])); !errors.Is(err, ErrFutureReplacePending) { - t.Fatalf("adding future replace transaction error mismatch: have %v, want %v", err, ErrFutureReplacePending) + if err := pool.addRemoteSync(pricedTransaction(5, 100000, big.NewInt(6), keys[1])); !errors.Is(err, legacypool.ErrFutureReplacePending) { + t.Fatalf("adding future replace transaction error mismatch: have %v, want %v", err, legacypool.ErrFutureReplacePending) } pending, queued = pool.Stats() if pending != 4 { @@ -2297,8 +2298,8 @@ func TestSetCodeTransactions(t *testing.T) { statedb.SetCode(aa, []byte{byte(vm.ADDRESS), byte(vm.PUSH0), byte(vm.SSTORE)}) // Send gapped transaction, it should be rejected. - if err := pool.addRemoteSync(pricedTransaction(2, 100000, big.NewInt(1), keyA)); !errors.Is(err, ErrOutOfOrderTxFromDelegated) { - t.Fatalf("%s: error mismatch: want %v, have %v", name, ErrOutOfOrderTxFromDelegated, err) + if err := pool.addRemoteSync(pricedTransaction(2, 100000, big.NewInt(1), keyA)); !errors.Is(err, legacypool.ErrOutOfOrderTxFromDelegated) { + t.Fatalf("%s: error mismatch: want %v, have %v", name, legacypool.ErrOutOfOrderTxFromDelegated, err) } // Send transactions. First is accepted, second is rejected. if err := pool.addRemoteSync(pricedTransaction(0, 100000, big.NewInt(1), keyA)); err != nil { @@ -2377,8 +2378,8 @@ func TestSetCodeTransactions(t *testing.T) { t.Fatalf("%s: failed to add with pending delegation: %v", name, err) } // Delegation rejected since two txs are already in-flight. - if err := pool.addRemoteSync(setCodeTx(0, keyA, []unsignedAuth{{0, keyB}})); !errors.Is(err, ErrAuthorityReserved) { - t.Fatalf("%s: error mismatch: want %v, have %v", name, ErrAuthorityReserved, err) + if err := pool.addRemoteSync(setCodeTx(0, keyA, []unsignedAuth{{0, keyB}})); !errors.Is(err, legacypool.ErrAuthorityReserved) { + t.Fatalf("%s: error mismatch: want %v, have %v", name, legacypool.ErrAuthorityReserved, err) } }, }, From 05cff08cb1743396d227ecb1d470eedf86a9085e Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 5 Nov 2025 10:11:51 -0500 Subject: [PATCH 06/47] consilidate error checking func --- mempool/txpool/locals/errors.go | 37 ++--- mempool/txpool/locals/errors_test.go | 3 +- rpc/backend/call_tx.go | 212 +++++++++++++++++++++------ rpc/backend/sign_tx.go | 123 ---------------- 4 files changed, 187 insertions(+), 188 deletions(-) diff --git a/mempool/txpool/locals/errors.go b/mempool/txpool/locals/errors.go index 2eca36b8e..641cbcfa4 100644 --- a/mempool/txpool/locals/errors.go +++ b/mempool/txpool/locals/errors.go @@ -2,29 +2,32 @@ package locals import ( "errors" + "strings" "github.com/cosmos/evm/mempool/txpool" - "github.com/cosmos/evm/mempool/txpool/legacypool" + "github.com/ethereum/go-ethereum/core/txpool/legacypool" ) -// IsTemporaryReject determines whether the given error indicates a temporary -// reason to reject a transaction from being included in the txpool. The result -// may change if the txpool's state changes later. +var ( + // ErrNonceGap is returned if the tx nonce is higher than the account nonce. + // This is a duplicate of mempool.ErrNonceGap to avoid import cycle. + ErrNonceGap = errors.New("tx nonce is higher than account nonce") +) + +// IsTemporaryReject determines whether the given error indicates a temporary reason to reject a +// transaction from being included in the txpool. The result may change if the txpool's state changes later. +// We use strings.Contains instead of errors.Is because we are passing in rawLog errors. func IsTemporaryReject(err error) bool { switch { - case errors.Is(err, legacypool.ErrOutOfOrderTxFromDelegated): - return true - case errors.Is(err, txpool.ErrInflightTxLimitReached): - return true - case errors.Is(err, legacypool.ErrAuthorityReserved): + case strings.Contains(err.Error(), legacypool.ErrOutOfOrderTxFromDelegated.Error()): + case strings.Contains(err.Error(), txpool.ErrInflightTxLimitReached.Error()): + case strings.Contains(err.Error(), legacypool.ErrAuthorityReserved.Error()): + case strings.Contains(err.Error(), txpool.ErrUnderpriced.Error()): + case strings.Contains(err.Error(), legacypool.ErrTxPoolOverflow.Error()): + case strings.Contains(err.Error(), legacypool.ErrFutureReplacePending.Error()): + case strings.Contains(err.Error(), ErrNonceGap.Error()): return true - case errors.Is(err, txpool.ErrUnderpriced): - return true - case errors.Is(err, legacypool.ErrTxPoolOverflow): - return true - case errors.Is(err, legacypool.ErrFutureReplacePending): - return true - default: - return false } + + return false } diff --git a/mempool/txpool/locals/errors_test.go b/mempool/txpool/locals/errors_test.go index a163131b7..902f703a1 100644 --- a/mempool/txpool/locals/errors_test.go +++ b/mempool/txpool/locals/errors_test.go @@ -5,7 +5,7 @@ import ( "testing" "github.com/cosmos/evm/mempool/txpool" - "github.com/cosmos/evm/mempool/txpool/legacypool" + "github.com/ethereum/go-ethereum/core/txpool/legacypool" ) func TestIsTemporaryReject_PositiveCases(t *testing.T) { @@ -19,6 +19,7 @@ func TestIsTemporaryReject_PositiveCases(t *testing.T) { {name: "underpriced", err: txpool.ErrUnderpriced}, {name: "txpool overflow", err: legacypool.ErrTxPoolOverflow}, {name: "future replace pending", err: legacypool.ErrFutureReplacePending}, + {name: "tx nonce is higher than account nonce", err: ErrNonceGap}, } for _, tc := range cases { t.Run(tc.name, func(t *testing.T) { diff --git a/rpc/backend/call_tx.go b/rpc/backend/call_tx.go index 5c65bce3b..14f41227b 100644 --- a/rpc/backend/call_tx.go +++ b/rpc/backend/call_tx.go @@ -8,6 +8,7 @@ import ( "math/big" "strings" + "github.com/ethereum/go-ethereum/accounts/keystore" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common/hexutil" ethtypes "github.com/ethereum/go-ethereum/core/types" @@ -17,6 +18,7 @@ import ( "google.golang.org/grpc/status" "github.com/cosmos/evm/mempool" + "github.com/cosmos/evm/mempool/txpool/locals" txlocals "github.com/cosmos/evm/mempool/txpool/locals" rpctypes "github.com/cosmos/evm/rpc/types" evmtypes "github.com/cosmos/evm/x/vm/types" @@ -27,77 +29,116 @@ import ( sdk "github.com/cosmos/cosmos-sdk/types" ) -// Resend accepts an existing transaction and a new gas price and limit. It will remove -// the given transaction from the pool and reinsert it with the new gas price and limit. -func (b *Backend) Resend(args evmtypes.TransactionArgs, gasPrice *hexutil.Big, gasLimit *hexutil.Uint64) (common.Hash, error) { - if args.Nonce == nil { - return common.Hash{}, fmt.Errorf("missing transaction nonce in transaction spec") +// SendTransaction sends transaction based on received args using Node's key to sign it +func (b *Backend) SendTransaction(args evmtypes.TransactionArgs) (common.Hash, error) { + // Look up the wallet containing the requested signer + if !b.Cfg.JSONRPC.AllowInsecureUnlock { + b.Logger.Debug("account unlock with HTTP access is forbidden") + return common.Hash{}, fmt.Errorf("account unlock with HTTP access is forbidden") } - args, err := b.SetTxDefaults(args) + _, err := b.ClientCtx.Keyring.KeyByAddress(sdk.AccAddress(args.GetFrom().Bytes())) if err != nil { - return common.Hash{}, err + b.Logger.Error("failed to find key in keyring", "address", args.GetFrom(), "error", err.Error()) + return common.Hash{}, fmt.Errorf("failed to find key in the node's keyring; %s; %s", keystore.ErrNoMatch, err.Error()) } - // The signer used should always be the 'latest' known one because we expect - // signers to be backwards-compatible with old transactions. - cfg := b.ChainConfig() - if cfg == nil { - cfg = evmtypes.DefaultChainConfig(b.EvmChainID.Uint64()).EthereumConfig(nil) + if args.ChainID != nil && (b.EvmChainID).Cmp((*big.Int)(args.ChainID)) != 0 { + return common.Hash{}, fmt.Errorf("chainId does not match node's (have=%v, want=%v)", args.ChainID, (*hexutil.Big)(b.EvmChainID)) } - signer := ethtypes.LatestSigner(cfg) + args, err = b.SetTxDefaults(args) + if err != nil { + return common.Hash{}, err + } - matchTx := args.ToTransaction(ethtypes.LegacyTxType) + bn, err := b.BlockNumber() + if err != nil { + b.Logger.Debug("failed to fetch latest block number", "error", err.Error()) + return common.Hash{}, err + } - // Before replacing the old transaction, ensure the _new_ transaction fee is reasonable. - price := matchTx.GasPrice() - if gasPrice != nil { - price = gasPrice.ToInt() + header, err := b.CurrentHeader() + if err != nil { + return common.Hash{}, err } - gas := matchTx.Gas() - if gasLimit != nil { - gas = uint64(*gasLimit) + + signer := ethtypes.MakeSigner(b.ChainConfig(), new(big.Int).SetUint64(uint64(bn)), header.Time) + + // LegacyTx derives EvmChainID from the signature. To make sure the msg.ValidateBasic makes + // the corresponding EvmChainID validation, we need to sign the transaction before calling it + + // Sign transaction + msg := evmtypes.NewTxFromArgs(&args) + if err := msg.Sign(signer, b.ClientCtx.Keyring); err != nil { + b.Logger.Debug("failed to sign tx", "error", err.Error()) + return common.Hash{}, err } - if err := rpctypes.CheckTxFee(price, gas, b.RPCTxFeeCap()); err != nil { + + if err := msg.ValidateBasic(); err != nil { + b.Logger.Debug("tx failed basic validation", "error", err.Error()) return common.Hash{}, err } - pending, err := b.PendingTransactions() + baseDenom := evmtypes.GetEVMCoinDenom() + + // Assemble transaction from fields + tx, err := msg.BuildTx(b.ClientCtx.TxConfig.NewTxBuilder(), baseDenom) if err != nil { + b.Logger.Error("build cosmos tx failed", "error", err.Error()) return common.Hash{}, err } - for _, tx := range pending { - // FIXME does Resend api possible at all? https://github.com/evmos/ethermint/issues/905 - p, err := evmtypes.UnwrapEthereumMsg(tx, common.Hash{}) - if err != nil { - // not valid ethereum tx - continue - } + // Encode transaction by default Tx encoder + txEncoder := b.ClientCtx.TxConfig.TxEncoder() + txBytes, err := txEncoder(tx) + if err != nil { + b.Logger.Error("failed to encode eth tx using default encoder", "error", err.Error()) + return common.Hash{}, err + } - pTx := p.AsTransaction() + ethTx := msg.AsTransaction() - wantSigHash := signer.Hash(matchTx) - pFrom, err := ethtypes.Sender(signer, pTx) - if err != nil { - continue - } + // check the local node config in case unprotected txs are disabled + if !b.UnprotectedAllowed() && !ethTx.Protected() { + // Ensure only eip155 signed transactions are submitted if EIP155Required is set. + return common.Hash{}, errors.New("only replay-protected (EIP-155) transactions allowed over RPC") + } - if pFrom == *args.From && signer.Hash(pTx) == wantSigHash { - // Match. Re-sign and send the transaction. - if gasPrice != nil && (*big.Int)(gasPrice).Sign() != 0 { - args.GasPrice = gasPrice - } - if gasLimit != nil && *gasLimit != 0 { - args.Gas = gasLimit - } + txHash := ethTx.Hash() - return b.SendTransaction(args) // TODO: this calls SetTxDefaults again, refactor to avoid calling it twice + // Broadcast transaction in sync mode (default) + // NOTE: If error is encountered on the node, the broadcast will not return an error + syncCtx := b.ClientCtx.WithBroadcastMode(flags.BroadcastSync) + rsp, err := syncCtx.BroadcastTx(txBytes) + if rsp != nil && rsp.Code != 0 { + err = errorsmod.ABCIError(rsp.Codespace, rsp.Code, rsp.RawLog) + } + if err != nil { + // Check if this is a nonce gap error that was successfully queued + if b.Mempool != nil && strings.Contains(err.Error(), mempool.ErrNonceGap.Error()) { + // Transaction was successfully queued due to nonce gap, return success to client + b.Logger.Debug("transaction queued due to nonce gap", "hash", txHash.Hex()) + // Track as local for priority and persistence + b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{ethTx}) + return txHash, nil } + // Temporary txpool rejections should be locally tracked for resubmission + if b.Mempool != nil && txlocals.IsTemporaryReject(err) { + b.Logger.Debug("temporary rejection, tracking locally", "hash", txHash.Hex(), "err", err.Error()) + b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{ethTx}) + return txHash, nil + } + b.Logger.Error("failed to broadcast tx", "error", err.Error()) + return txHash, err } - return common.Hash{}, fmt.Errorf("transaction %#x not found", matchTx.Hash()) + // Return transaction hash + // On success, track as local too to persist across restarts until mined + if b.Mempool != nil { + b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{ethTx}) + } + return txHash, nil } // SendRawTransaction send a raw Ethereum transaction. @@ -151,7 +192,11 @@ func (b *Backend) SendRawTransaction(data hexutil.Bytes) (common.Hash, error) { syncCtx := b.ClientCtx.WithBroadcastMode(flags.BroadcastSync) rsp, err := syncCtx.BroadcastTx(txBytes) - if rsp != nil && rsp.Code != 0 { + if b.Mempool != nil && rsp != nil && rsp.Code != 0 { + if shouldSkip := locals.IsTemporaryReject(errors.New(rsp.RawLog)); shouldSkip { + b.Logger.Debug("temporary rejection, tracking locally", "tx_hash", txHash.Hex()) + return txHash, nil + } err = errorsmod.ABCIError(rsp.Codespace, rsp.Code, rsp.RawLog) } if err != nil { @@ -201,6 +246,79 @@ func (b *Backend) SendRawTransaction(data hexutil.Bytes) (common.Hash, error) { return txHash, nil } +// Resend accepts an existing transaction and a new gas price and limit. It will remove +// the given transaction from the pool and reinsert it with the new gas price and limit. +func (b *Backend) Resend(args evmtypes.TransactionArgs, gasPrice *hexutil.Big, gasLimit *hexutil.Uint64) (common.Hash, error) { + if args.Nonce == nil { + return common.Hash{}, fmt.Errorf("missing transaction nonce in transaction spec") + } + + args, err := b.SetTxDefaults(args) + if err != nil { + return common.Hash{}, err + } + + // The signer used should always be the 'latest' known one because we expect + // signers to be backwards-compatible with old transactions. + cfg := b.ChainConfig() + if cfg == nil { + cfg = evmtypes.DefaultChainConfig(b.EvmChainID.Uint64()).EthereumConfig(nil) + } + + signer := ethtypes.LatestSigner(cfg) + + matchTx := args.ToTransaction(ethtypes.LegacyTxType) + + // Before replacing the old transaction, ensure the _new_ transaction fee is reasonable. + price := matchTx.GasPrice() + if gasPrice != nil { + price = gasPrice.ToInt() + } + gas := matchTx.Gas() + if gasLimit != nil { + gas = uint64(*gasLimit) + } + if err := rpctypes.CheckTxFee(price, gas, b.RPCTxFeeCap()); err != nil { + return common.Hash{}, err + } + + pending, err := b.PendingTransactions() + if err != nil { + return common.Hash{}, err + } + + for _, tx := range pending { + // FIXME does Resend api possible at all? https://github.com/evmos/ethermint/issues/905 + p, err := evmtypes.UnwrapEthereumMsg(tx, common.Hash{}) + if err != nil { + // not valid ethereum tx + continue + } + + pTx := p.AsTransaction() + + wantSigHash := signer.Hash(matchTx) + pFrom, err := ethtypes.Sender(signer, pTx) + if err != nil { + continue + } + + if pFrom == *args.From && signer.Hash(pTx) == wantSigHash { + // Match. Re-sign and send the transaction. + if gasPrice != nil && (*big.Int)(gasPrice).Sign() != 0 { + args.GasPrice = gasPrice + } + if gasLimit != nil && *gasLimit != 0 { + args.Gas = gasLimit + } + + return b.SendTransaction(args) // TODO: this calls SetTxDefaults again, refactor to avoid calling it twice + } + } + + return common.Hash{}, fmt.Errorf("transaction %#x not found", matchTx.Hash()) +} + // SetTxDefaults populates tx message with default values in case they are not // provided on the args func (b *Backend) SetTxDefaults(args evmtypes.TransactionArgs) (evmtypes.TransactionArgs, error) { diff --git a/rpc/backend/sign_tx.go b/rpc/backend/sign_tx.go index a8bd67437..19d444507 100644 --- a/rpc/backend/sign_tx.go +++ b/rpc/backend/sign_tx.go @@ -1,141 +1,18 @@ package backend import ( - "errors" "fmt" - "math/big" - "strings" "github.com/ethereum/go-ethereum/accounts/keystore" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common/hexutil" - ethtypes "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/signer/core/apitypes" - "github.com/cosmos/evm/mempool" - txlocals "github.com/cosmos/evm/mempool/txpool/locals" - evmtypes "github.com/cosmos/evm/x/vm/types" - - errorsmod "cosmossdk.io/errors" - - "github.com/cosmos/cosmos-sdk/client/flags" sdk "github.com/cosmos/cosmos-sdk/types" signingtypes "github.com/cosmos/cosmos-sdk/types/tx/signing" ) -// SendTransaction sends transaction based on received args using Node's key to sign it -func (b *Backend) SendTransaction(args evmtypes.TransactionArgs) (common.Hash, error) { - // Look up the wallet containing the requested signer - if !b.Cfg.JSONRPC.AllowInsecureUnlock { - b.Logger.Debug("account unlock with HTTP access is forbidden") - return common.Hash{}, fmt.Errorf("account unlock with HTTP access is forbidden") - } - - _, err := b.ClientCtx.Keyring.KeyByAddress(sdk.AccAddress(args.GetFrom().Bytes())) - if err != nil { - b.Logger.Error("failed to find key in keyring", "address", args.GetFrom(), "error", err.Error()) - return common.Hash{}, fmt.Errorf("failed to find key in the node's keyring; %s; %s", keystore.ErrNoMatch, err.Error()) - } - - if args.ChainID != nil && (b.EvmChainID).Cmp((*big.Int)(args.ChainID)) != 0 { - return common.Hash{}, fmt.Errorf("chainId does not match node's (have=%v, want=%v)", args.ChainID, (*hexutil.Big)(b.EvmChainID)) - } - - args, err = b.SetTxDefaults(args) - if err != nil { - return common.Hash{}, err - } - - bn, err := b.BlockNumber() - if err != nil { - b.Logger.Debug("failed to fetch latest block number", "error", err.Error()) - return common.Hash{}, err - } - - header, err := b.CurrentHeader() - if err != nil { - return common.Hash{}, err - } - - signer := ethtypes.MakeSigner(b.ChainConfig(), new(big.Int).SetUint64(uint64(bn)), header.Time) - - // LegacyTx derives EvmChainID from the signature. To make sure the msg.ValidateBasic makes - // the corresponding EvmChainID validation, we need to sign the transaction before calling it - - // Sign transaction - msg := evmtypes.NewTxFromArgs(&args) - if err := msg.Sign(signer, b.ClientCtx.Keyring); err != nil { - b.Logger.Debug("failed to sign tx", "error", err.Error()) - return common.Hash{}, err - } - - if err := msg.ValidateBasic(); err != nil { - b.Logger.Debug("tx failed basic validation", "error", err.Error()) - return common.Hash{}, err - } - - baseDenom := evmtypes.GetEVMCoinDenom() - - // Assemble transaction from fields - tx, err := msg.BuildTx(b.ClientCtx.TxConfig.NewTxBuilder(), baseDenom) - if err != nil { - b.Logger.Error("build cosmos tx failed", "error", err.Error()) - return common.Hash{}, err - } - - // Encode transaction by default Tx encoder - txEncoder := b.ClientCtx.TxConfig.TxEncoder() - txBytes, err := txEncoder(tx) - if err != nil { - b.Logger.Error("failed to encode eth tx using default encoder", "error", err.Error()) - return common.Hash{}, err - } - - ethTx := msg.AsTransaction() - - // check the local node config in case unprotected txs are disabled - if !b.UnprotectedAllowed() && !ethTx.Protected() { - // Ensure only eip155 signed transactions are submitted if EIP155Required is set. - return common.Hash{}, errors.New("only replay-protected (EIP-155) transactions allowed over RPC") - } - - txHash := ethTx.Hash() - - // Broadcast transaction in sync mode (default) - // NOTE: If error is encountered on the node, the broadcast will not return an error - syncCtx := b.ClientCtx.WithBroadcastMode(flags.BroadcastSync) - rsp, err := syncCtx.BroadcastTx(txBytes) - if rsp != nil && rsp.Code != 0 { - err = errorsmod.ABCIError(rsp.Codespace, rsp.Code, rsp.RawLog) - } - if err != nil { - // Check if this is a nonce gap error that was successfully queued - if b.Mempool != nil && strings.Contains(err.Error(), mempool.ErrNonceGap.Error()) { - // Transaction was successfully queued due to nonce gap, return success to client - b.Logger.Debug("transaction queued due to nonce gap", "hash", txHash.Hex()) - // Track as local for priority and persistence - b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{ethTx}) - return txHash, nil - } - // Temporary txpool rejections should be locally tracked for resubmission - if b.Mempool != nil && txlocals.IsTemporaryReject(err) { - b.Logger.Debug("temporary rejection, tracking locally", "hash", txHash.Hex(), "err", err.Error()) - b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{ethTx}) - return txHash, nil - } - b.Logger.Error("failed to broadcast tx", "error", err.Error()) - return txHash, err - } - - // Return transaction hash - // On success, track as local too to persist across restarts until mined - if b.Mempool != nil { - b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{ethTx}) - } - return txHash, nil -} - // Sign signs the provided data using the private key of address via Geth's signature standard. func (b *Backend) Sign(address common.Address, data hexutil.Bytes) (hexutil.Bytes, error) { from := sdk.AccAddress(address.Bytes()) From bffcec6bfdeade763323425a6362386a8ec263ef Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 5 Nov 2025 10:33:10 -0500 Subject: [PATCH 07/47] check both resp log and err --- rpc/backend/call_tx.go | 32 ++++++++++++++++---------------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/rpc/backend/call_tx.go b/rpc/backend/call_tx.go index 14f41227b..867d17c0d 100644 --- a/rpc/backend/call_tx.go +++ b/rpc/backend/call_tx.go @@ -18,7 +18,6 @@ import ( "google.golang.org/grpc/status" "github.com/cosmos/evm/mempool" - "github.com/cosmos/evm/mempool/txpool/locals" txlocals "github.com/cosmos/evm/mempool/txpool/locals" rpctypes "github.com/cosmos/evm/rpc/types" evmtypes "github.com/cosmos/evm/x/vm/types" @@ -114,18 +113,19 @@ func (b *Backend) SendTransaction(args evmtypes.TransactionArgs) (common.Hash, e if rsp != nil && rsp.Code != 0 { err = errorsmod.ABCIError(rsp.Codespace, rsp.Code, rsp.RawLog) } - if err != nil { - // Check if this is a nonce gap error that was successfully queued - if b.Mempool != nil && strings.Contains(err.Error(), mempool.ErrNonceGap.Error()) { - // Transaction was successfully queued due to nonce gap, return success to client - b.Logger.Debug("transaction queued due to nonce gap", "hash", txHash.Hex()) - // Track as local for priority and persistence + // Check for temporary rejection in response raw log + if b.Mempool != nil && rsp != nil && rsp.Code != 0 { + if txlocals.IsTemporaryReject(errors.New(rsp.RawLog)) { + b.Logger.Debug("temporary rejection in response raw log, tracking locally", "hash", txHash.Hex(), "err", rsp.RawLog) b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{ethTx}) return txHash, nil } - // Temporary txpool rejections should be locally tracked for resubmission + err = errorsmod.ABCIError(rsp.Codespace, rsp.Code, rsp.RawLog) + } + if err != nil { + // Check for temporary rejection in error if b.Mempool != nil && txlocals.IsTemporaryReject(err) { - b.Logger.Debug("temporary rejection, tracking locally", "hash", txHash.Hex(), "err", err.Error()) + b.Logger.Debug("temporary rejection in error, tracking locally", "hash", txHash.Hex(), "err", err.Error()) b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{ethTx}) return txHash, nil } @@ -192,19 +192,19 @@ func (b *Backend) SendRawTransaction(data hexutil.Bytes) (common.Hash, error) { syncCtx := b.ClientCtx.WithBroadcastMode(flags.BroadcastSync) rsp, err := syncCtx.BroadcastTx(txBytes) + // Check for temporary rejection in response raw log if b.Mempool != nil && rsp != nil && rsp.Code != 0 { - if shouldSkip := locals.IsTemporaryReject(errors.New(rsp.RawLog)); shouldSkip { - b.Logger.Debug("temporary rejection, tracking locally", "tx_hash", txHash.Hex()) + if txlocals.IsTemporaryReject(errors.New(rsp.RawLog)) { + b.Logger.Debug("temporary rejection in response raw log, tracking locally", "hash", txHash.Hex(), "err", rsp.RawLog) + b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{tx}) return txHash, nil } err = errorsmod.ABCIError(rsp.Codespace, rsp.Code, rsp.RawLog) } if err != nil { - // Check if this is a nonce gap error that was successfully queued - if b.Mempool != nil && strings.Contains(err.Error(), mempool.ErrNonceGap.Error()) { - // Transaction was successfully queued due to nonce gap, return success to client - b.Logger.Debug("transaction queued due to nonce gap", "hash", txHash.Hex()) - // Track as local for priority and persistence + // Check for temporary rejection in response raw log + if b.Mempool != nil && rsp != nil && rsp.Code != 0 && txlocals.IsTemporaryReject(errors.New(rsp.RawLog)) { + b.Logger.Debug("temporary rejection in response raw log, tracking locally", "hash", txHash.Hex(), "err", rsp.RawLog) b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{tx}) return txHash, nil } From efdbf2d99976dc6b39a7745cafc1276eed05cc80 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 5 Nov 2025 10:43:08 -0500 Subject: [PATCH 08/47] fix string match case --- mempool/txpool/locals/errors.go | 21 ++++++++++++--------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/mempool/txpool/locals/errors.go b/mempool/txpool/locals/errors.go index 641cbcfa4..3274bc2b6 100644 --- a/mempool/txpool/locals/errors.go +++ b/mempool/txpool/locals/errors.go @@ -2,7 +2,6 @@ package locals import ( "errors" - "strings" "github.com/cosmos/evm/mempool/txpool" "github.com/ethereum/go-ethereum/core/txpool/legacypool" @@ -18,14 +17,18 @@ var ( // transaction from being included in the txpool. The result may change if the txpool's state changes later. // We use strings.Contains instead of errors.Is because we are passing in rawLog errors. func IsTemporaryReject(err error) bool { - switch { - case strings.Contains(err.Error(), legacypool.ErrOutOfOrderTxFromDelegated.Error()): - case strings.Contains(err.Error(), txpool.ErrInflightTxLimitReached.Error()): - case strings.Contains(err.Error(), legacypool.ErrAuthorityReserved.Error()): - case strings.Contains(err.Error(), txpool.ErrUnderpriced.Error()): - case strings.Contains(err.Error(), legacypool.ErrTxPoolOverflow.Error()): - case strings.Contains(err.Error(), legacypool.ErrFutureReplacePending.Error()): - case strings.Contains(err.Error(), ErrNonceGap.Error()): + if err == nil { + return false + } + + switch err.Error() { + case legacypool.ErrOutOfOrderTxFromDelegated.Error(), + txpool.ErrInflightTxLimitReached.Error(), + legacypool.ErrAuthorityReserved.Error(), + txpool.ErrUnderpriced.Error(), + legacypool.ErrTxPoolOverflow.Error(), + legacypool.ErrFutureReplacePending.Error(), + ErrNonceGap.Error(): return true } From 7864f9de949b5a189650da4d10e1ed72d6622196 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 5 Nov 2025 10:56:06 -0500 Subject: [PATCH 09/47] remove dup check --- rpc/backend/call_tx.go | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/rpc/backend/call_tx.go b/rpc/backend/call_tx.go index 867d17c0d..a65c6fb9b 100644 --- a/rpc/backend/call_tx.go +++ b/rpc/backend/call_tx.go @@ -203,17 +203,11 @@ func (b *Backend) SendRawTransaction(data hexutil.Bytes) (common.Hash, error) { } if err != nil { // Check for temporary rejection in response raw log - if b.Mempool != nil && rsp != nil && rsp.Code != 0 && txlocals.IsTemporaryReject(errors.New(rsp.RawLog)) { + if b.Mempool != nil && txlocals.IsTemporaryReject(errors.New(rsp.RawLog)) { b.Logger.Debug("temporary rejection in response raw log, tracking locally", "hash", txHash.Hex(), "err", rsp.RawLog) b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{tx}) return txHash, nil } - // Temporary txpool rejections should be locally tracked for resubmission - if b.Mempool != nil && txlocals.IsTemporaryReject(err) { - b.Logger.Debug("temporary rejection, tracking locally", "hash", txHash.Hex(), "err", err.Error()) - b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{tx}) - return txHash, nil - } if b.Mempool != nil && strings.Contains(err.Error(), mempool.ErrNonceLow.Error()) { from, err := ethSigner.Sender(tx) if err != nil { From b138e61be6b0bb9e067936e399c45a60886a1b08 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 5 Nov 2025 11:00:07 -0500 Subject: [PATCH 10/47] expect error for last tx --- tests/systemtests/mempool/test_ordering.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/systemtests/mempool/test_ordering.go b/tests/systemtests/mempool/test_ordering.go index f92dfecf4..7f0550d05 100644 --- a/tests/systemtests/mempool/test_ordering.go +++ b/tests/systemtests/mempool/test_ordering.go @@ -36,7 +36,11 @@ func RunTxsOrdering(t *testing.T, base *suite.BaseTestSuite) { } txInfo, err := s.SendTx(t, nodeId, signer.ID, nonceIdx, s.GasPriceMultiplier(10), big.NewInt(1)) - require.NoError(t, err, "failed to send tx") + if i < 4 { + require.NoError(t, err, "failed to send tx") + } else { + require.Error(t, err, "expected error for nonce gap") + } // nonce order of committed txs: 0,1,2,3,4 expPendingTxs[nonceIdx] = txInfo From ac052304b83f5932ca5273cd26d17d041e3fa004 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 5 Nov 2025 11:04:08 -0500 Subject: [PATCH 11/47] filter by nonce idx --- tests/systemtests/mempool/test_ordering.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/systemtests/mempool/test_ordering.go b/tests/systemtests/mempool/test_ordering.go index 7f0550d05..6b02156b2 100644 --- a/tests/systemtests/mempool/test_ordering.go +++ b/tests/systemtests/mempool/test_ordering.go @@ -36,7 +36,7 @@ func RunTxsOrdering(t *testing.T, base *suite.BaseTestSuite) { } txInfo, err := s.SendTx(t, nodeId, signer.ID, nonceIdx, s.GasPriceMultiplier(10), big.NewInt(1)) - if i < 4 { + if nonceIdx < 4 { require.NoError(t, err, "failed to send tx") } else { require.Error(t, err, "expected error for nonce gap") From 825117a78a243131bd078bfcca98caa06b939f7d Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 5 Nov 2025 11:06:35 -0500 Subject: [PATCH 12/47] rev --- tests/systemtests/mempool/test_ordering.go | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/tests/systemtests/mempool/test_ordering.go b/tests/systemtests/mempool/test_ordering.go index 6b02156b2..f92dfecf4 100644 --- a/tests/systemtests/mempool/test_ordering.go +++ b/tests/systemtests/mempool/test_ordering.go @@ -36,11 +36,7 @@ func RunTxsOrdering(t *testing.T, base *suite.BaseTestSuite) { } txInfo, err := s.SendTx(t, nodeId, signer.ID, nonceIdx, s.GasPriceMultiplier(10), big.NewInt(1)) - if nonceIdx < 4 { - require.NoError(t, err, "failed to send tx") - } else { - require.Error(t, err, "expected error for nonce gap") - } + require.NoError(t, err, "failed to send tx") // nonce order of committed txs: 0,1,2,3,4 expPendingTxs[nonceIdx] = txInfo From ef631ab2d73b6f44c18e0eb8410388dfc6864cfd Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 5 Nov 2025 11:27:40 -0500 Subject: [PATCH 13/47] expect error for all except --- tests/systemtests/mempool/test_ordering.go | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/tests/systemtests/mempool/test_ordering.go b/tests/systemtests/mempool/test_ordering.go index f92dfecf4..d81b994ce 100644 --- a/tests/systemtests/mempool/test_ordering.go +++ b/tests/systemtests/mempool/test_ordering.go @@ -20,7 +20,8 @@ func RunTxsOrdering(t *testing.T, base *suite.BaseTestSuite) { name: "ordering of pending txs %s", actions: []func(*TestSuite, *TestContext){ func(s *TestSuite, ctx *TestContext) { - signer := s.Acc(0) + accIdx := 0 + signer := s.Acc(accIdx) expPendingTxs := make([]*suite.TxInfo, 5) for i := 0; i < 5; i++ { @@ -36,7 +37,11 @@ func RunTxsOrdering(t *testing.T, base *suite.BaseTestSuite) { } txInfo, err := s.SendTx(t, nodeId, signer.ID, nonceIdx, s.GasPriceMultiplier(10), big.NewInt(1)) - require.NoError(t, err, "failed to send tx") + if i == accIdx { + require.NoError(t, err, "failed to send tx") + } else { + require.Error(t, err, "expected error for nonce gap") + } // nonce order of committed txs: 0,1,2,3,4 expPendingTxs[nonceIdx] = txInfo From 240b021f4db0e85718ace711164a53a443ee8bbc Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 5 Nov 2025 11:30:52 -0500 Subject: [PATCH 14/47] use nonce gap --- tests/systemtests/mempool/test_ordering.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/systemtests/mempool/test_ordering.go b/tests/systemtests/mempool/test_ordering.go index d81b994ce..895d9cc93 100644 --- a/tests/systemtests/mempool/test_ordering.go +++ b/tests/systemtests/mempool/test_ordering.go @@ -37,7 +37,7 @@ func RunTxsOrdering(t *testing.T, base *suite.BaseTestSuite) { } txInfo, err := s.SendTx(t, nodeId, signer.ID, nonceIdx, s.GasPriceMultiplier(10), big.NewInt(1)) - if i == accIdx { + if nonceIdx <= uint64(accIdx) { require.NoError(t, err, "failed to send tx") } else { require.Error(t, err, "expected error for nonce gap") From deafb4e9f9005417204071504d91611af6b11c2b Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 5 Nov 2025 12:28:33 -0500 Subject: [PATCH 15/47] rev --- tests/systemtests/mempool/test_ordering.go | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/tests/systemtests/mempool/test_ordering.go b/tests/systemtests/mempool/test_ordering.go index 895d9cc93..f92dfecf4 100644 --- a/tests/systemtests/mempool/test_ordering.go +++ b/tests/systemtests/mempool/test_ordering.go @@ -20,8 +20,7 @@ func RunTxsOrdering(t *testing.T, base *suite.BaseTestSuite) { name: "ordering of pending txs %s", actions: []func(*TestSuite, *TestContext){ func(s *TestSuite, ctx *TestContext) { - accIdx := 0 - signer := s.Acc(accIdx) + signer := s.Acc(0) expPendingTxs := make([]*suite.TxInfo, 5) for i := 0; i < 5; i++ { @@ -37,11 +36,7 @@ func RunTxsOrdering(t *testing.T, base *suite.BaseTestSuite) { } txInfo, err := s.SendTx(t, nodeId, signer.ID, nonceIdx, s.GasPriceMultiplier(10), big.NewInt(1)) - if nonceIdx <= uint64(accIdx) { - require.NoError(t, err, "failed to send tx") - } else { - require.Error(t, err, "expected error for nonce gap") - } + require.NoError(t, err, "failed to send tx") // nonce order of committed txs: 0,1,2,3,4 expPendingTxs[nonceIdx] = txInfo From 6c4a6b2541c972cbb9311fdbb90853d357a6dccd Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 5 Nov 2025 12:50:15 -0500 Subject: [PATCH 16/47] rm nonce gap --- mempool/txpool/locals/errors.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/mempool/txpool/locals/errors.go b/mempool/txpool/locals/errors.go index 3274bc2b6..d201faed5 100644 --- a/mempool/txpool/locals/errors.go +++ b/mempool/txpool/locals/errors.go @@ -27,8 +27,7 @@ func IsTemporaryReject(err error) bool { legacypool.ErrAuthorityReserved.Error(), txpool.ErrUnderpriced.Error(), legacypool.ErrTxPoolOverflow.Error(), - legacypool.ErrFutureReplacePending.Error(), - ErrNonceGap.Error(): + legacypool.ErrFutureReplacePending.Error(): return true } From 26f02d32d9c340647b69a13582fe54bfde0c0be4 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Fri, 7 Nov 2025 10:58:22 -0500 Subject: [PATCH 17/47] add back nonce gap case --- mempool/txpool/locals/errors.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/mempool/txpool/locals/errors.go b/mempool/txpool/locals/errors.go index d201faed5..3274bc2b6 100644 --- a/mempool/txpool/locals/errors.go +++ b/mempool/txpool/locals/errors.go @@ -27,7 +27,8 @@ func IsTemporaryReject(err error) bool { legacypool.ErrAuthorityReserved.Error(), txpool.ErrUnderpriced.Error(), legacypool.ErrTxPoolOverflow.Error(), - legacypool.ErrFutureReplacePending.Error(): + legacypool.ErrFutureReplacePending.Error(), + ErrNonceGap.Error(): return true } From 25efd230bcf3479b48d778d0fa9b7884a1b6ccd9 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Fri, 7 Nov 2025 11:01:13 -0500 Subject: [PATCH 18/47] fix wrong arg --- rpc/backend/call_tx.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rpc/backend/call_tx.go b/rpc/backend/call_tx.go index a65c6fb9b..ecac03780 100644 --- a/rpc/backend/call_tx.go +++ b/rpc/backend/call_tx.go @@ -203,7 +203,7 @@ func (b *Backend) SendRawTransaction(data hexutil.Bytes) (common.Hash, error) { } if err != nil { // Check for temporary rejection in response raw log - if b.Mempool != nil && txlocals.IsTemporaryReject(errors.New(rsp.RawLog)) { + if b.Mempool != nil && txlocals.IsTemporaryReject(err) { b.Logger.Debug("temporary rejection in response raw log, tracking locally", "hash", txHash.Hex(), "err", rsp.RawLog) b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{tx}) return txHash, nil From a68e02d36731f35e8856cea3f6935676b45479b9 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Fri, 7 Nov 2025 11:08:44 -0500 Subject: [PATCH 19/47] add panic --- rpc/backend/call_tx.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/rpc/backend/call_tx.go b/rpc/backend/call_tx.go index ecac03780..c60871233 100644 --- a/rpc/backend/call_tx.go +++ b/rpc/backend/call_tx.go @@ -204,7 +204,7 @@ func (b *Backend) SendRawTransaction(data hexutil.Bytes) (common.Hash, error) { if err != nil { // Check for temporary rejection in response raw log if b.Mempool != nil && txlocals.IsTemporaryReject(err) { - b.Logger.Debug("temporary rejection in response raw log, tracking locally", "hash", txHash.Hex(), "err", rsp.RawLog) + b.Logger.Debug("temporary rejection in error, tracking locally", "hash", txHash.Hex(), "err", err.Error()) b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{tx}) return txHash, nil } @@ -224,11 +224,13 @@ func (b *Backend) SendRawTransaction(data hexutil.Bytes) (common.Hash, error) { } // SendRawTransaction does not return error when committed nonce <= tx.Nonce < pending nonce - // Track as local for persistence until mined + // Track as local for persistence until pending b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{tx}) return txHash, nil } + panic(fmt.Sprintf("------------------>>>> %s, %s, %v", err.Error(), txlocals.IsTemporaryReject(err), b.Mempool != nil)) + b.Logger.Error("failed to broadcast tx", "error", err.Error()) return txHash, fmt.Errorf("failed to broadcast transaction: %w", err) } From 7a3dc863c86b4c74fdebf0494203ef8d58863566 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Fri, 7 Nov 2025 11:20:34 -0500 Subject: [PATCH 20/47] edit return --- rpc/backend/call_tx.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rpc/backend/call_tx.go b/rpc/backend/call_tx.go index c60871233..08132fb5d 100644 --- a/rpc/backend/call_tx.go +++ b/rpc/backend/call_tx.go @@ -229,10 +229,10 @@ func (b *Backend) SendRawTransaction(data hexutil.Bytes) (common.Hash, error) { return txHash, nil } - panic(fmt.Sprintf("------------------>>>> %s, %s, %v", err.Error(), txlocals.IsTemporaryReject(err), b.Mempool != nil)) + x := fmt.Sprintf("------------------>>>> %s, %s, %v", err.Error(), txlocals.IsTemporaryReject(err), b.Mempool != nil) b.Logger.Error("failed to broadcast tx", "error", err.Error()) - return txHash, fmt.Errorf("failed to broadcast transaction: %w", err) + return txHash, fmt.Errorf("failed to broadcast transaction: %s", x) } // On success, track as local too to persist across restarts until mined From 307fb074855c1964014c7438a2b2357999b6b98e Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Fri, 7 Nov 2025 11:37:34 -0500 Subject: [PATCH 21/47] use contains instead of match --- mempool/txpool/locals/errors.go | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/mempool/txpool/locals/errors.go b/mempool/txpool/locals/errors.go index 3274bc2b6..4e7d16209 100644 --- a/mempool/txpool/locals/errors.go +++ b/mempool/txpool/locals/errors.go @@ -2,6 +2,7 @@ package locals import ( "errors" + "strings" "github.com/cosmos/evm/mempool/txpool" "github.com/ethereum/go-ethereum/core/txpool/legacypool" @@ -21,14 +22,14 @@ func IsTemporaryReject(err error) bool { return false } - switch err.Error() { - case legacypool.ErrOutOfOrderTxFromDelegated.Error(), - txpool.ErrInflightTxLimitReached.Error(), - legacypool.ErrAuthorityReserved.Error(), - txpool.ErrUnderpriced.Error(), - legacypool.ErrTxPoolOverflow.Error(), - legacypool.ErrFutureReplacePending.Error(), - ErrNonceGap.Error(): + switch { + case strings.Contains(err.Error(), legacypool.ErrOutOfOrderTxFromDelegated.Error()), + strings.Contains(err.Error(), txpool.ErrInflightTxLimitReached.Error()), + strings.Contains(err.Error(), legacypool.ErrAuthorityReserved.Error()), + strings.Contains(err.Error(), txpool.ErrUnderpriced.Error()), + strings.Contains(err.Error(), legacypool.ErrTxPoolOverflow.Error()), + strings.Contains(err.Error(), legacypool.ErrFutureReplacePending.Error()), + strings.Contains(err.Error(), ErrNonceGap.Error()): return true } From 17cea3d197880530ddf5b9d3b60c9b2d09800517 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Fri, 7 Nov 2025 11:43:39 -0500 Subject: [PATCH 22/47] fix system ordering test --- rpc/backend/call_tx.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/rpc/backend/call_tx.go b/rpc/backend/call_tx.go index 08132fb5d..1c2d24275 100644 --- a/rpc/backend/call_tx.go +++ b/rpc/backend/call_tx.go @@ -229,10 +229,8 @@ func (b *Backend) SendRawTransaction(data hexutil.Bytes) (common.Hash, error) { return txHash, nil } - x := fmt.Sprintf("------------------>>>> %s, %s, %v", err.Error(), txlocals.IsTemporaryReject(err), b.Mempool != nil) - b.Logger.Error("failed to broadcast tx", "error", err.Error()) - return txHash, fmt.Errorf("failed to broadcast transaction: %s", x) + return txHash, fmt.Errorf("failed to broadcast transaction: %w", err) } // On success, track as local too to persist across restarts until mined From f0fdd7716ed54fd4ab50a6e3848603df1d3a4c5f Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 12 Nov 2025 10:19:37 -0500 Subject: [PATCH 23/47] fix orphaned rlp file creation --- server/server_app_options.go | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/server/server_app_options.go b/server/server_app_options.go index def0ed9e7..2afed3710 100644 --- a/server/server_app_options.go +++ b/server/server_app_options.go @@ -163,15 +163,14 @@ func GetLegacyPoolConfig(appOpts servertypes.AppOptions, logger log.Logger) *leg if noLocals := appOpts.Get(srvflags.EVMMempoolNoLocals); noLocals != nil { legacyConfig.NoLocals = cast.ToBool(noLocals) } - if journal := cast.ToString(appOpts.Get(srvflags.EVMMempoolJournal)); journal != "" { - legacyConfig.Journal = journal - if homeDir := cast.ToString(appOpts.Get(flags.FlagHome)); homeDir != "" { - legacyConfig.Journal = filepath.Join(homeDir, "data", "txpool", journal) - } else { - // this is a valid configuration for testing where we don't want to persist local transactions - logger.Warn("no home directory set, disabling local transaction journaling", "journal", journal) - legacyConfig.Journal = "" + if homeDir := cast.ToString(appOpts.Get(flags.FlagHome)); homeDir != "" { + if journal := cast.ToString(appOpts.Get(srvflags.EVMMempoolJournal)); journal != "" { + legacyConfig.Journal = journal } + legacyConfig.Journal = filepath.Join(homeDir, "data", "txpool", legacyConfig.Journal) + } else { + logger.Warn("no home directory set, disabling local transaction journaling") + legacyConfig.Journal = "" } if rejournal := cast.ToDuration(appOpts.Get(srvflags.EVMMempoolRejournal)); rejournal != 0 { legacyConfig.Rejournal = rejournal From ae7f9db7731a08ea5b76d7004b93a440619b8881 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 12 Nov 2025 10:28:04 -0500 Subject: [PATCH 24/47] fix path creation before writing --- mempool/txpool/locals/journal.go | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/mempool/txpool/locals/journal.go b/mempool/txpool/locals/journal.go index 74977a905..4cd1fc53a 100644 --- a/mempool/txpool/locals/journal.go +++ b/mempool/txpool/locals/journal.go @@ -18,9 +18,11 @@ package locals import ( "errors" + "fmt" "io" "io/fs" "os" + "path/filepath" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/types" @@ -138,6 +140,12 @@ func (journal *journal) rotate(all map[common.Address]types.Transactions) error } journal.writer = nil } + // Ensure parent directory exists + if dir := filepath.Dir(journal.path); dir != "." { + if err := os.MkdirAll(dir, 0755); err != nil { + return fmt.Errorf("failed to create journal directory: %w", err) + } + } // Generate a new journal with the contents of the current pool replacement, err := os.OpenFile(journal.path+".new", os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644) if err != nil { @@ -180,6 +188,12 @@ func (journal *journal) open() error { if journal.writer != nil { return nil // Already open } + // Ensure parent directory exists + if dir := filepath.Dir(journal.path); dir != "." { + if err := os.MkdirAll(dir, 0755); err != nil { + return fmt.Errorf("failed to create journal directory: %w", err) + } + } // Open file for appending, create if it doesn't exist sink, err := os.OpenFile(journal.path, os.O_WRONLY|os.O_APPEND|os.O_CREATE, 0644) if err != nil { From fc7c2f33c4089b70f6a95e89172e2848a53af2d8 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 12 Nov 2025 10:40:33 -0500 Subject: [PATCH 25/47] remove print --- mempool/mempool.go | 1 - 1 file changed, 1 deletion(-) diff --git a/mempool/mempool.go b/mempool/mempool.go index d5ea3729d..162d2a549 100644 --- a/mempool/mempool.go +++ b/mempool/mempool.go @@ -133,7 +133,6 @@ func NewExperimentalEVMMempool( // from queued into pending, noting their readiness to be executed. legacyPool.BroadcastTxFn = func(txs []*ethtypes.Transaction) error { logger.Debug("broadcasting EVM transactions", "tx_count", len(txs)) - fmt.Println(clientCtx) return broadcastEVMTransactions(clientCtx, txConfig, txs) } } From c431e8fe67a06f4fe6b9bee55b1acc3d862f79ea Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 12 Nov 2025 13:52:07 -0500 Subject: [PATCH 26/47] dedupe mempool config --- mempool/mempool.go | 6 +- mempool/txpool/legacypool/legacypool.go | 147 ++++++++++--------- mempool/txpool/legacypool/legacypool_test.go | 6 +- mempool/txpool/legacypool/list_test.go | 12 +- server/config/config.go | 78 +--------- server/server_app_options.go | 10 +- server/start.go | 15 +- 7 files changed, 105 insertions(+), 169 deletions(-) diff --git a/mempool/mempool.go b/mempool/mempool.go index 162d2a549..3e50965e2 100644 --- a/mempool/mempool.go +++ b/mempool/mempool.go @@ -73,7 +73,7 @@ type ( // It allows customization of the underlying mempools, verification functions, // and broadcasting functions used by the sdkmempool. type EVMMempoolConfig struct { - LegacyPoolConfig *legacypool.Config + LegacyPoolConfig *legacypool.MempoolConfig CosmosPoolConfig *sdkmempool.PriorityNonceMempoolConfig[math.Int] AnteHandler sdk.AnteHandler BroadCastTxFn func(txs []*ethtypes.Transaction) error @@ -117,9 +117,9 @@ func NewExperimentalEVMMempool( blockchain = NewBlockchain(getCtxCallback, logger, vmKeeper, feeMarketKeeper, config.BlockGasLimit) // Create txPool from configuration - legacyConfig := legacypool.DefaultConfig + legacyConfig := legacypool.DefaultMempoolConfig() if config.LegacyPoolConfig != nil { - legacyConfig = *config.LegacyPoolConfig + legacyConfig = config.LegacyPoolConfig } legacyPool := legacypool.New(legacyConfig, blockchain) diff --git a/mempool/txpool/legacypool/legacypool.go b/mempool/txpool/legacypool/legacypool.go index 4faccebdf..9514e0487 100644 --- a/mempool/txpool/legacypool/legacypool.go +++ b/mempool/txpool/legacypool/legacypool.go @@ -18,10 +18,12 @@ package legacypool import ( + "fmt" "maps" "math/big" "slices" "sort" + "strings" "sync" "sync/atomic" "time" @@ -116,73 +118,76 @@ type BlockChain interface { StateAt(root common.Hash) (vm.StateDB, error) } -// Config are the configuration parameters of the transaction pool. -type Config struct { - Locals []common.Address // Addresses that should be treated by default as local - NoLocals bool // Whether local transaction handling should be disabled - Journal string // Journal of local transactions to survive node restarts - Rejournal time.Duration // Time interval to regenerate the local transaction journal - - PriceLimit uint64 // Minimum gas price to enforce for acceptance into the pool - PriceBump uint64 // Minimum price bump percentage to replace an already existing transaction (nonce) - - AccountSlots uint64 // Number of executable transaction slots guaranteed per account - GlobalSlots uint64 // Maximum number of executable transaction slots for all accounts - AccountQueue uint64 // Maximum number of non-executable transaction slots permitted per account - GlobalQueue uint64 // Maximum number of non-executable transaction slots for all accounts - - Lifetime time.Duration // Maximum amount of time non-executable transaction are queued -} - -// DefaultConfig contains the default configurations for the transaction pool. -var DefaultConfig = Config{ - Journal: "transactions.rlp", - Rejournal: time.Hour, - - PriceLimit: 1, - PriceBump: 10, - - AccountSlots: 16, - GlobalSlots: 4096 + 1024, // urgent + floating queue capacity with 4:1 ratio - AccountQueue: 64, - GlobalQueue: 1024, - - Lifetime: 3 * time.Hour, -} - -// sanitize checks the provided user configurations and changes anything that's -// unreasonable or unworkable. -func (config *Config) sanitize() Config { - conf := *config - if conf.PriceLimit < 1 { - log.Warn("Sanitizing invalid txpool price limit", "provided", conf.PriceLimit, "updated", DefaultConfig.PriceLimit) - conf.PriceLimit = DefaultConfig.PriceLimit - } - if conf.PriceBump < 1 { - log.Warn("Sanitizing invalid txpool price bump", "provided", conf.PriceBump, "updated", DefaultConfig.PriceBump) - conf.PriceBump = DefaultConfig.PriceBump - } - if conf.AccountSlots < 1 { - log.Warn("Sanitizing invalid txpool account slots", "provided", conf.AccountSlots, "updated", DefaultConfig.AccountSlots) - conf.AccountSlots = DefaultConfig.AccountSlots - } - if conf.GlobalSlots < 1 { - log.Warn("Sanitizing invalid txpool global slots", "provided", conf.GlobalSlots, "updated", DefaultConfig.GlobalSlots) - conf.GlobalSlots = DefaultConfig.GlobalSlots - } - if conf.AccountQueue < 1 { - log.Warn("Sanitizing invalid txpool account queue", "provided", conf.AccountQueue, "updated", DefaultConfig.AccountQueue) - conf.AccountQueue = DefaultConfig.AccountQueue +// MempoolConfig defines the configuration for the EVM mempool transaction pool. +type MempoolConfig struct { + // PriceLimit is the minimum gas price to enforce for acceptance into the pool + PriceLimit uint64 `mapstructure:"price-limit"` + // PriceBump is the minimum price bump percentage to replace an already existing transaction (nonce) + PriceBump uint64 `mapstructure:"price-bump"` + // AccountSlots is the number of executable transaction slots guaranteed per account + AccountSlots uint64 `mapstructure:"account-slots"` + // GlobalSlots is the maximum number of executable transaction slots for all accounts + GlobalSlots uint64 `mapstructure:"global-slots"` + // AccountQueue is the maximum number of non-executable transaction slots permitted per account + AccountQueue uint64 `mapstructure:"account-queue"` + // GlobalQueue is the maximum number of non-executable transaction slots for all accounts + GlobalQueue uint64 `mapstructure:"global-queue"` + // Lifetime is the maximum amount of time non-executable transaction are queued + Lifetime time.Duration `mapstructure:"lifetime"` + // Locals is the set of addresses that should be treated by default as local + Locals []string `mapstructure:"locals"` + // NoLocals disables local transaction handling, exempting local accounts from pricing and acceptance + NoLocals bool `mapstructure:"no-locals"` + // Journal is the path to the local transaction journal file + Journal string `mapstructure:"journal"` + // Rejournal is the time interval to regenerate the local transaction journal + Rejournal time.Duration `mapstructure:"rejournal"` +} + +// DefaultMempoolConfig returns the default mempool configuration +func DefaultMempoolConfig() *MempoolConfig { + return &MempoolConfig{ + PriceLimit: 1, // Minimum gas price of 1 wei + PriceBump: 10, // 10% price bump to replace transaction + AccountSlots: 16, // 16 executable transaction slots per account + GlobalSlots: 5120, // 4096 + 1024 = 5120 global executable slots + AccountQueue: 64, // 64 non-executable transaction slots per account + GlobalQueue: 1024, // 1024 global non-executable slots + Lifetime: 3 * time.Hour, // 3 hour lifetime for queued transactions + Locals: []string{}, // No local addresses by default + NoLocals: false, // Local transaction handling enabled by default + Journal: "transactions.rlp", // Default journal filename + Rejournal: time.Hour, // Regenerate journal every hour + } +} + +// Validate returns an error if the mempool configuration is invalid +func (c MempoolConfig) Validate() error { + if c.PriceLimit < 1 { + return fmt.Errorf("price limit must be at least 1, got %d", c.PriceLimit) + } + if c.PriceBump < 1 { + return fmt.Errorf("price bump must be at least 1, got %d", c.PriceBump) + } + if c.AccountSlots < 1 { + return fmt.Errorf("account slots must be at least 1, got %d", c.AccountSlots) + } + if c.GlobalSlots < 1 { + return fmt.Errorf("global slots must be at least 1, got %d", c.GlobalSlots) + } + if c.AccountQueue < 1 { + return fmt.Errorf("account queue must be at least 1, got %d", c.AccountQueue) + } + if c.GlobalQueue < 1 { + return fmt.Errorf("global queue must be at least 1, got %d", c.GlobalQueue) + } + if c.Lifetime < 1 { + return fmt.Errorf("lifetime must be at least 1 nanosecond, got %s", c.Lifetime) + } + if !strings.HasSuffix(c.Journal, ".rlp") { + return fmt.Errorf("journal must end with .rlp, got %s", c.Journal) } - if conf.GlobalQueue < 1 { - log.Warn("Sanitizing invalid txpool global queue", "provided", conf.GlobalQueue, "updated", DefaultConfig.GlobalQueue) - conf.GlobalQueue = DefaultConfig.GlobalQueue - } - if conf.Lifetime < 1 { - log.Warn("Sanitizing invalid txpool lifetime", "provided", conf.Lifetime, "updated", DefaultConfig.Lifetime) - conf.Lifetime = DefaultConfig.Lifetime - } - return conf + return nil } // LegacyPool contains all currently known transactions. Transactions @@ -207,7 +212,7 @@ func (config *Config) sanitize() Config { // will reject new transactions with delegations from that account with standard in-flight // transactions. type LegacyPool struct { - config Config + config MempoolConfig chainconfig *params.ChainConfig chain BlockChain gasTip atomic.Pointer[uint256.Int] @@ -245,13 +250,15 @@ type txpoolResetRequest struct { // New creates a new transaction pool to gather, sort and filter inbound // transactions from the network. -func New(config Config, chain BlockChain) *LegacyPool { +func New(config *MempoolConfig, chain BlockChain) *LegacyPool { // Sanitize the input to ensure no vulnerable gas prices are set - config = (&config).sanitize() + if err := config.Validate(); err != nil { + panic(err) + } // Create the transaction pool with its initial settings pool := &LegacyPool{ - config: config, + config: *config, chain: chain, chainconfig: chain.Config(), signer: types.LatestSigner(chain.Config()), diff --git a/mempool/txpool/legacypool/legacypool_test.go b/mempool/txpool/legacypool/legacypool_test.go index 41d298bae..0333b9129 100644 --- a/mempool/txpool/legacypool/legacypool_test.go +++ b/mempool/txpool/legacypool/legacypool_test.go @@ -48,14 +48,14 @@ import ( var ( // testTxPoolConfig is a transaction pool configuration without stateful disk // sideeffects used during testing. - testTxPoolConfig Config + testTxPoolConfig *MempoolConfig // eip1559Config is a chain config with EIP-1559 enabled at block 0. eip1559Config *params.ChainConfig ) func init() { - testTxPoolConfig = DefaultConfig + testTxPoolConfig = DefaultMempoolConfig() testTxPoolConfig.Journal = "" cpy := *params.TestChainConfig @@ -1488,7 +1488,7 @@ func TestMinGasPriceEnforced(t *testing.T) { statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) blockchain := newTestBlockChain(eip1559Config, 10000000, statedb, new(event.Feed)) - txPoolConfig := DefaultConfig + txPoolConfig := DefaultMempoolConfig() txPoolConfig.NoLocals = true pool := New(txPoolConfig, blockchain) pool.Init(txPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) diff --git a/mempool/txpool/legacypool/list_test.go b/mempool/txpool/legacypool/list_test.go index 8587c66f7..c3a96beaf 100644 --- a/mempool/txpool/legacypool/list_test.go +++ b/mempool/txpool/legacypool/list_test.go @@ -40,7 +40,7 @@ func TestStrictListAdd(t *testing.T) { // Insert the transactions in a random order list := newList(true) for _, v := range rand.Perm(len(txs)) { - list.Add(txs[v], DefaultConfig.PriceBump) + list.Add(txs[v], DefaultMempoolConfig().PriceBump) } // Verify internal state if len(list.txs.items) != len(txs) { @@ -64,7 +64,7 @@ func TestListAddVeryExpensive(t *testing.T) { gaslimit := uint64(i) tx, _ := types.SignTx(types.NewTransaction(uint64(i), common.Address{}, value, gaslimit, gasprice, nil), types.HomesteadSigner{}, key) t.Logf("cost: %x bitlen: %d\n", tx.Cost(), tx.Cost().BitLen()) - list.Add(tx, DefaultConfig.PriceBump) + list.Add(tx, DefaultMempoolConfig().PriceBump) } } @@ -77,13 +77,13 @@ func BenchmarkListAdd(b *testing.B) { txs[i] = transaction(uint64(i), 0, key) } // Insert the transactions in a random order - priceLimit := uint256.NewInt(DefaultConfig.PriceLimit) + priceLimit := uint256.NewInt(DefaultMempoolConfig().PriceLimit) b.ResetTimer() for i := 0; i < b.N; i++ { list := newList(true) for _, v := range rand.Perm(len(txs)) { - list.Add(txs[v], DefaultConfig.PriceBump) - list.Filter(priceLimit, DefaultConfig.PriceBump) + list.Add(txs[v], DefaultMempoolConfig().PriceBump) + list.Filter(priceLimit, DefaultMempoolConfig().PriceBump) } } } @@ -102,7 +102,7 @@ func BenchmarkListCapOneTx(b *testing.B) { list := newList(true) // Insert the transactions in a random order for _, v := range rand.Perm(len(txs)) { - list.Add(txs[v], DefaultConfig.PriceBump) + list.Add(txs[v], DefaultMempoolConfig().PriceBump) } b.StartTimer() list.Cap(list.Len() - 1) diff --git a/server/config/config.go b/server/config/config.go index 85f101f8e..a43000e1d 100644 --- a/server/config/config.go +++ b/server/config/config.go @@ -5,7 +5,6 @@ import ( "fmt" "net/netip" "path" - "strings" "time" "github.com/spf13/viper" @@ -16,6 +15,7 @@ import ( "github.com/cosmos/cosmos-sdk/server/config" errortypes "github.com/cosmos/cosmos-sdk/types/errors" + "github.com/cosmos/evm/mempool/txpool/legacypool" ) const ( @@ -153,79 +153,7 @@ type EVMConfig struct { // GethMetricsAddress is the address the geth metrics server will bind to. Default 127.0.0.1:8100 GethMetricsAddress string `mapstructure:"geth-metrics-address"` // Mempool defines the EVM mempool configuration - Mempool MempoolConfig `mapstructure:"mempool"` -} - -// MempoolConfig defines the configuration for the EVM mempool transaction pool. -type MempoolConfig struct { - // PriceLimit is the minimum gas price to enforce for acceptance into the pool - PriceLimit uint64 `mapstructure:"price-limit"` - // PriceBump is the minimum price bump percentage to replace an already existing transaction (nonce) - PriceBump uint64 `mapstructure:"price-bump"` - // AccountSlots is the number of executable transaction slots guaranteed per account - AccountSlots uint64 `mapstructure:"account-slots"` - // GlobalSlots is the maximum number of executable transaction slots for all accounts - GlobalSlots uint64 `mapstructure:"global-slots"` - // AccountQueue is the maximum number of non-executable transaction slots permitted per account - AccountQueue uint64 `mapstructure:"account-queue"` - // GlobalQueue is the maximum number of non-executable transaction slots for all accounts - GlobalQueue uint64 `mapstructure:"global-queue"` - // Lifetime is the maximum amount of time non-executable transaction are queued - Lifetime time.Duration `mapstructure:"lifetime"` - // Locals is the set of addresses that should be treated by default as local - Locals []string `mapstructure:"locals"` - // NoLocals disables local transaction handling, exempting local accounts from pricing and acceptance - NoLocals bool `mapstructure:"no-locals"` - // Journal is the path to the local transaction journal file - Journal string `mapstructure:"journal"` - // Rejournal is the time interval to regenerate the local transaction journal - Rejournal time.Duration `mapstructure:"rejournal"` -} - -// DefaultMempoolConfig returns the default mempool configuration -func DefaultMempoolConfig() MempoolConfig { - return MempoolConfig{ - PriceLimit: 1, // Minimum gas price of 1 wei - PriceBump: 10, // 10% price bump to replace transaction - AccountSlots: 16, // 16 executable transaction slots per account - GlobalSlots: 5120, // 4096 + 1024 = 5120 global executable slots - AccountQueue: 64, // 64 non-executable transaction slots per account - GlobalQueue: 1024, // 1024 global non-executable slots - Lifetime: 3 * time.Hour, // 3 hour lifetime for queued transactions - Locals: []string{}, // No local addresses by default - NoLocals: false, // Local transaction handling enabled by default - Journal: "transactions.rlp", // Default journal filename - Rejournal: time.Hour, // Regenerate journal every hour - } -} - -// Validate returns an error if the mempool configuration is invalid -func (c MempoolConfig) Validate() error { - if c.PriceLimit < 1 { - return fmt.Errorf("price limit must be at least 1, got %d", c.PriceLimit) - } - if c.PriceBump < 1 { - return fmt.Errorf("price bump must be at least 1, got %d", c.PriceBump) - } - if c.AccountSlots < 1 { - return fmt.Errorf("account slots must be at least 1, got %d", c.AccountSlots) - } - if c.GlobalSlots < 1 { - return fmt.Errorf("global slots must be at least 1, got %d", c.GlobalSlots) - } - if c.AccountQueue < 1 { - return fmt.Errorf("account queue must be at least 1, got %d", c.AccountQueue) - } - if c.GlobalQueue < 1 { - return fmt.Errorf("global queue must be at least 1, got %d", c.GlobalQueue) - } - if c.Lifetime < 1 { - return fmt.Errorf("lifetime must be at least 1 nanosecond, got %s", c.Lifetime) - } - if !strings.HasSuffix(c.Journal, ".rlp") { - return fmt.Errorf("journal must end with .rlp, got %s", c.Journal) - } - return nil + Mempool legacypool.MempoolConfig `mapstructure:"mempool"` } // JSONRPCConfig defines configuration for the EVM RPC server. @@ -295,7 +223,7 @@ func DefaultEVMConfig() *EVMConfig { EnablePreimageRecording: DefaultEnablePreimageRecording, MinTip: DefaultEVMMinTip, GethMetricsAddress: DefaultGethMetricsAddress, - Mempool: DefaultMempoolConfig(), + Mempool: *legacypool.DefaultMempoolConfig(), } } diff --git a/server/server_app_options.go b/server/server_app_options.go index 2afed3710..5b6d58c05 100644 --- a/server/server_app_options.go +++ b/server/server_app_options.go @@ -129,13 +129,13 @@ func GetMinTip(appOpts servertypes.AppOptions, logger log.Logger) *uint256.Int { // GetLegacyPoolConfig reads the legacy pool configuration from appOpts and overrides // default values with values from app.toml if they exist and are non-zero. -func GetLegacyPoolConfig(appOpts servertypes.AppOptions, logger log.Logger) *legacypool.Config { +func GetLegacyPoolConfig(appOpts servertypes.AppOptions, logger log.Logger) *legacypool.MempoolConfig { if appOpts == nil { logger.Error("app options is nil, using default mempool config") - return &legacypool.DefaultConfig + return legacypool.DefaultMempoolConfig() } - legacyConfig := legacypool.DefaultConfig + legacyConfig := legacypool.DefaultMempoolConfig() if priceLimit := cast.ToUint64(appOpts.Get(srvflags.EVMMempoolPriceLimit)); priceLimit != 0 { legacyConfig.PriceLimit = priceLimit } @@ -158,7 +158,7 @@ func GetLegacyPoolConfig(appOpts servertypes.AppOptions, logger log.Logger) *leg legacyConfig.Lifetime = lifetime } if localsSlice := cast.ToStringSlice(appOpts.Get(srvflags.EVMMempoolLocals)); len(localsSlice) > 0 { - legacyConfig.Locals = parseAddresses(localsSlice) + legacyConfig.Locals = localsSlice } if noLocals := appOpts.Get(srvflags.EVMMempoolNoLocals); noLocals != nil { legacyConfig.NoLocals = cast.ToBool(noLocals) @@ -176,7 +176,7 @@ func GetLegacyPoolConfig(appOpts servertypes.AppOptions, logger log.Logger) *leg legacyConfig.Rejournal = rejournal } - return &legacyConfig + return legacyConfig } func GetCosmosPoolMaxTx(appOpts servertypes.AppOptions, logger log.Logger) int { diff --git a/server/start.go b/server/start.go index ce3cb3a89..0a3db8632 100644 --- a/server/start.go +++ b/server/start.go @@ -30,6 +30,7 @@ import ( dbm "github.com/cosmos/cosmos-db" "github.com/cosmos/evm/indexer" evmmempool "github.com/cosmos/evm/mempool" + "github.com/cosmos/evm/mempool/txpool/legacypool" evmmetrics "github.com/cosmos/evm/metrics" ethdebug "github.com/cosmos/evm/rpc/namespaces/ethereum/debug" cosmosevmserverconfig "github.com/cosmos/evm/server/config" @@ -230,13 +231,13 @@ which accepts a path for the resulting pprof file. cmd.Flags().Uint64(srvflags.EVMMinTip, cosmosevmserverconfig.DefaultEVMMinTip, "the minimum priority fee for the mempool") cmd.Flags().String(srvflags.EvmGethMetricsAddress, cosmosevmserverconfig.DefaultGethMetricsAddress, "the address to bind the geth metrics server to") - cmd.Flags().Uint64(srvflags.EVMMempoolPriceLimit, cosmosevmserverconfig.DefaultMempoolConfig().PriceLimit, "the minimum gas price to enforce for acceptance into the pool (in wei)") - cmd.Flags().Uint64(srvflags.EVMMempoolPriceBump, cosmosevmserverconfig.DefaultMempoolConfig().PriceBump, "the minimum price bump percentage to replace an already existing transaction (nonce)") - cmd.Flags().Uint64(srvflags.EVMMempoolAccountSlots, cosmosevmserverconfig.DefaultMempoolConfig().AccountSlots, "the number of executable transaction slots guaranteed per account") - cmd.Flags().Uint64(srvflags.EVMMempoolGlobalSlots, cosmosevmserverconfig.DefaultMempoolConfig().GlobalSlots, "the maximum number of executable transaction slots for all accounts") - cmd.Flags().Uint64(srvflags.EVMMempoolAccountQueue, cosmosevmserverconfig.DefaultMempoolConfig().AccountQueue, "the maximum number of non-executable transaction slots permitted per account") - cmd.Flags().Uint64(srvflags.EVMMempoolGlobalQueue, cosmosevmserverconfig.DefaultMempoolConfig().GlobalQueue, "the maximum number of non-executable transaction slots for all accounts") - cmd.Flags().Duration(srvflags.EVMMempoolLifetime, cosmosevmserverconfig.DefaultMempoolConfig().Lifetime, "the maximum amount of time non-executable transaction are queued") + cmd.Flags().Uint64(srvflags.EVMMempoolPriceLimit, legacypool.DefaultMempoolConfig().PriceLimit, "the minimum gas price to enforce for acceptance into the pool (in wei)") + cmd.Flags().Uint64(srvflags.EVMMempoolPriceBump, legacypool.DefaultMempoolConfig().PriceBump, "the minimum price bump percentage to replace an already existing transaction (nonce)") + cmd.Flags().Uint64(srvflags.EVMMempoolAccountSlots, legacypool.DefaultMempoolConfig().AccountSlots, "the number of executable transaction slots guaranteed per account") + cmd.Flags().Uint64(srvflags.EVMMempoolGlobalSlots, legacypool.DefaultMempoolConfig().GlobalSlots, "the maximum number of executable transaction slots for all accounts") + cmd.Flags().Uint64(srvflags.EVMMempoolAccountQueue, legacypool.DefaultMempoolConfig().AccountQueue, "the maximum number of non-executable transaction slots permitted per account") + cmd.Flags().Uint64(srvflags.EVMMempoolGlobalQueue, legacypool.DefaultMempoolConfig().GlobalQueue, "the maximum number of non-executable transaction slots for all accounts") + cmd.Flags().Duration(srvflags.EVMMempoolLifetime, legacypool.DefaultMempoolConfig().Lifetime, "the maximum amount of time non-executable transaction are queued") cmd.Flags().StringSlice(srvflags.EVMMempoolLocals, []string{}, "the set of addresses that should be treated by default as local (comma-separated)") cmd.Flags().Bool(srvflags.EVMMempoolNoLocals, false, "disables local transaction handling, exempting local accounts from pricing and acceptance") From e885db39b22a0308847c2d715b0cd5aebaf5e83a Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 12 Nov 2025 13:58:09 -0500 Subject: [PATCH 27/47] fix journal name validation --- mempool/txpool/legacypool/legacypool.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mempool/txpool/legacypool/legacypool.go b/mempool/txpool/legacypool/legacypool.go index 9514e0487..56098d7b0 100644 --- a/mempool/txpool/legacypool/legacypool.go +++ b/mempool/txpool/legacypool/legacypool.go @@ -184,7 +184,7 @@ func (c MempoolConfig) Validate() error { if c.Lifetime < 1 { return fmt.Errorf("lifetime must be at least 1 nanosecond, got %s", c.Lifetime) } - if !strings.HasSuffix(c.Journal, ".rlp") { + if c.Journal != "" && !strings.HasSuffix(c.Journal, ".rlp") { return fmt.Errorf("journal must end with .rlp, got %s", c.Journal) } return nil From 11c35d1691f9baf99bd40f60f0e73a6efdb66710 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 12 Nov 2025 14:14:01 -0500 Subject: [PATCH 28/47] fix data race --- mempool/mempool.go | 4 +- mempool/txpool/legacypool/legacypool.go | 38 ++++++++-------- mempool/txpool/legacypool/legacypool2_test.go | 8 ++-- mempool/txpool/legacypool/legacypool_test.go | 43 +++++++++---------- mempool/txpool/legacypool/list_test.go | 12 +++--- mempool/txpool/locals/tx_tracker_test.go | 2 +- mempool/txpool/txpool_test.go | 2 +- server/config/config.go | 4 +- server/server_app_options.go | 8 ++-- server/start.go | 14 +++--- 10 files changed, 66 insertions(+), 69 deletions(-) diff --git a/mempool/mempool.go b/mempool/mempool.go index 3e50965e2..aaae1ce24 100644 --- a/mempool/mempool.go +++ b/mempool/mempool.go @@ -73,7 +73,7 @@ type ( // It allows customization of the underlying mempools, verification functions, // and broadcasting functions used by the sdkmempool. type EVMMempoolConfig struct { - LegacyPoolConfig *legacypool.MempoolConfig + LegacyPoolConfig *legacypool.Config CosmosPoolConfig *sdkmempool.PriorityNonceMempoolConfig[math.Int] AnteHandler sdk.AnteHandler BroadCastTxFn func(txs []*ethtypes.Transaction) error @@ -117,7 +117,7 @@ func NewExperimentalEVMMempool( blockchain = NewBlockchain(getCtxCallback, logger, vmKeeper, feeMarketKeeper, config.BlockGasLimit) // Create txPool from configuration - legacyConfig := legacypool.DefaultMempoolConfig() + legacyConfig := &legacypool.DefaultConfig if config.LegacyPoolConfig != nil { legacyConfig = config.LegacyPoolConfig } diff --git a/mempool/txpool/legacypool/legacypool.go b/mempool/txpool/legacypool/legacypool.go index 56098d7b0..e62cdb142 100644 --- a/mempool/txpool/legacypool/legacypool.go +++ b/mempool/txpool/legacypool/legacypool.go @@ -118,8 +118,8 @@ type BlockChain interface { StateAt(root common.Hash) (vm.StateDB, error) } -// MempoolConfig defines the configuration for the EVM mempool transaction pool. -type MempoolConfig struct { +// Config defines the configuration for the EVM mempool transaction pool. +type Config struct { // PriceLimit is the minimum gas price to enforce for acceptance into the pool PriceLimit uint64 `mapstructure:"price-limit"` // PriceBump is the minimum price bump percentage to replace an already existing transaction (nonce) @@ -144,25 +144,23 @@ type MempoolConfig struct { Rejournal time.Duration `mapstructure:"rejournal"` } -// DefaultMempoolConfig returns the default mempool configuration -func DefaultMempoolConfig() *MempoolConfig { - return &MempoolConfig{ - PriceLimit: 1, // Minimum gas price of 1 wei - PriceBump: 10, // 10% price bump to replace transaction - AccountSlots: 16, // 16 executable transaction slots per account - GlobalSlots: 5120, // 4096 + 1024 = 5120 global executable slots - AccountQueue: 64, // 64 non-executable transaction slots per account - GlobalQueue: 1024, // 1024 global non-executable slots - Lifetime: 3 * time.Hour, // 3 hour lifetime for queued transactions - Locals: []string{}, // No local addresses by default - NoLocals: false, // Local transaction handling enabled by default - Journal: "transactions.rlp", // Default journal filename - Rejournal: time.Hour, // Regenerate journal every hour - } +// DefaultConfig returns the default mempool configuration +var DefaultConfig = Config{ + PriceLimit: 1, // Minimum gas price of 1 wei + PriceBump: 10, // 10% price bump to replace transaction + AccountSlots: 16, // 16 executable transaction slots per account + GlobalSlots: 5120, // 4096 + 1024 = 5120 global executable slots + AccountQueue: 64, // 64 non-executable transaction slots per account + GlobalQueue: 1024, // 1024 global non-executable slots + Lifetime: 3 * time.Hour, // 3 hour lifetime for queued transactions + Locals: []string{}, // No local addresses by default + NoLocals: false, // Local transaction handling enabled by default + Journal: "transactions.rlp", // Default journal filename + Rejournal: time.Hour, // Regenerate journal every hour } // Validate returns an error if the mempool configuration is invalid -func (c MempoolConfig) Validate() error { +func (c Config) Validate() error { if c.PriceLimit < 1 { return fmt.Errorf("price limit must be at least 1, got %d", c.PriceLimit) } @@ -212,7 +210,7 @@ func (c MempoolConfig) Validate() error { // will reject new transactions with delegations from that account with standard in-flight // transactions. type LegacyPool struct { - config MempoolConfig + config Config chainconfig *params.ChainConfig chain BlockChain gasTip atomic.Pointer[uint256.Int] @@ -250,7 +248,7 @@ type txpoolResetRequest struct { // New creates a new transaction pool to gather, sort and filter inbound // transactions from the network. -func New(config *MempoolConfig, chain BlockChain) *LegacyPool { +func New(config *Config, chain BlockChain) *LegacyPool { // Sanitize the input to ensure no vulnerable gas prices are set if err := config.Validate(); err != nil { panic(err) diff --git a/mempool/txpool/legacypool/legacypool2_test.go b/mempool/txpool/legacypool/legacypool2_test.go index deb06aa61..20baf9018 100644 --- a/mempool/txpool/legacypool/legacypool2_test.go +++ b/mempool/txpool/legacypool/legacypool2_test.go @@ -86,7 +86,7 @@ func TestTransactionFutureAttack(t *testing.T) { config := testTxPoolConfig config.GlobalQueue = 100 config.GlobalSlots = 100 - pool := New(config, blockchain) + pool := New(&config, blockchain) pool.Init(config.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -121,7 +121,7 @@ func TestTransactionFuture1559(t *testing.T) { // Create the pool to test the pricing enforcement with statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) blockchain := newTestBlockChain(eip1559Config, 1000000, statedb, new(event.Feed)) - pool := New(testTxPoolConfig, blockchain) + pool := New(&testTxPoolConfig, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -154,7 +154,7 @@ func TestTransactionZAttack(t *testing.T) { // Create the pool to test the pricing enforcement with statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) blockchain := newTestBlockChain(eip1559Config, 1000000, statedb, new(event.Feed)) - pool := New(testTxPoolConfig, blockchain) + pool := New(&testTxPoolConfig, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() // Create a number of test accounts, fund them and make transactions @@ -227,7 +227,7 @@ func BenchmarkFutureAttack(b *testing.B) { config := testTxPoolConfig config.GlobalQueue = 100 config.GlobalSlots = 100 - pool := New(config, blockchain) + pool := New(&config, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() fillPool(b, pool) diff --git a/mempool/txpool/legacypool/legacypool_test.go b/mempool/txpool/legacypool/legacypool_test.go index 0333b9129..56fae66e0 100644 --- a/mempool/txpool/legacypool/legacypool_test.go +++ b/mempool/txpool/legacypool/legacypool_test.go @@ -48,14 +48,14 @@ import ( var ( // testTxPoolConfig is a transaction pool configuration without stateful disk // sideeffects used during testing. - testTxPoolConfig *MempoolConfig + testTxPoolConfig Config // eip1559Config is a chain config with EIP-1559 enabled at block 0. eip1559Config *params.ChainConfig ) func init() { - testTxPoolConfig = DefaultMempoolConfig() + testTxPoolConfig = DefaultConfig testTxPoolConfig.Journal = "" cpy := *params.TestChainConfig @@ -236,7 +236,7 @@ func setupPoolWithConfig(config *params.ChainConfig) (*LegacyPool, *ecdsa.Privat blockchain := newTestBlockChain(config, 10000000, statedb, new(event.Feed)) key, _ := crypto.GenerateKey() - pool := New(testTxPoolConfig, blockchain) + pool := New(&testTxPoolConfig, blockchain) if err := pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()); err != nil { panic(err) } @@ -369,7 +369,7 @@ func TestStateChangeDuringReset(t *testing.T) { tx0 := transaction(0, 100000, key) tx1 := transaction(1, 100000, key) - pool := New(testTxPoolConfig, blockchain) + pool := New(&testTxPoolConfig, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -786,7 +786,7 @@ func TestPostponing(t *testing.T) { statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed)) - pool := New(testTxPoolConfig, blockchain) + pool := New(&testTxPoolConfig, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -996,7 +996,7 @@ func TestQueueGlobalLimiting(t *testing.T) { config.NoLocals = true config.GlobalQueue = config.AccountQueue*3 - 1 // reduce the queue limits to shorten test time (-1 to make it non divisible) - pool := New(config, blockchain) + pool := New(&config, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -1048,7 +1048,7 @@ func TestQueueTimeLimiting(t *testing.T) { config := testTxPoolConfig config.Lifetime = time.Second - pool := New(config, blockchain) + pool := New(&config, blockchain) pool.Init(config.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -1209,7 +1209,7 @@ func TestPendingGlobalLimiting(t *testing.T) { config := testTxPoolConfig config.GlobalSlots = config.AccountSlots * 10 - pool := New(config, blockchain) + pool := New(&config, blockchain) pool.Init(config.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -1308,7 +1308,7 @@ func TestCapClearsFromAll(t *testing.T) { config.AccountQueue = 2 config.GlobalSlots = 8 - pool := New(config, blockchain) + pool := New(&config, blockchain) pool.Init(config.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -1341,7 +1341,7 @@ func TestPendingMinimumAllowance(t *testing.T) { config := testTxPoolConfig config.GlobalSlots = 1 - pool := New(config, blockchain) + pool := New(&config, blockchain) pool.Init(config.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -1385,7 +1385,7 @@ func TestRepricing(t *testing.T) { statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed)) - pool := New(testTxPoolConfig, blockchain) + pool := New(&testTxPoolConfig, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -1488,9 +1488,9 @@ func TestMinGasPriceEnforced(t *testing.T) { statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) blockchain := newTestBlockChain(eip1559Config, 10000000, statedb, new(event.Feed)) - txPoolConfig := DefaultMempoolConfig() + txPoolConfig := DefaultConfig txPoolConfig.NoLocals = true - pool := New(txPoolConfig, blockchain) + pool := New(&txPoolConfig, blockchain) pool.Init(txPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -1639,7 +1639,7 @@ func TestUnderpricing(t *testing.T) { config.GlobalSlots = 2 config.GlobalQueue = 2 - pool := New(config, blockchain) + pool := New(&config, blockchain) pool.Init(config.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -1727,9 +1727,8 @@ func TestStableUnderpricing(t *testing.T) { config := testTxPoolConfig config.GlobalSlots = 128 - config.GlobalQueue = 0 - pool := New(config, blockchain) + pool := New(&config, blockchain) pool.Init(config.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -1932,7 +1931,7 @@ func TestDeduplication(t *testing.T) { statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed)) - pool := New(testTxPoolConfig, blockchain) + pool := New(&testTxPoolConfig, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -1999,7 +1998,7 @@ func TestReplacement(t *testing.T) { statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed)) - pool := New(testTxPoolConfig, blockchain) + pool := New(&testTxPoolConfig, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -2191,7 +2190,7 @@ func TestStatusCheck(t *testing.T) { statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed)) - pool := New(testTxPoolConfig, blockchain) + pool := New(&testTxPoolConfig, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -2264,7 +2263,7 @@ func TestSetCodeTransactions(t *testing.T) { statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) blockchain := newTestBlockChain(params.MergedTestChainConfig, 1000000, statedb, new(event.Feed)) - pool := New(testTxPoolConfig, blockchain) + pool := New(&testTxPoolConfig, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -2562,7 +2561,7 @@ func TestSetCodeTransactionsReorg(t *testing.T) { statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) blockchain := newTestBlockChain(params.MergedTestChainConfig, 1000000, statedb, new(event.Feed)) - pool := New(testTxPoolConfig, blockchain) + pool := New(&testTxPoolConfig, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -2621,7 +2620,7 @@ func TestRemoveTxTruncatePoolRace(t *testing.T) { statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) blockchain := newTestBlockChain(params.MergedTestChainConfig, 1000000, statedb, new(event.Feed)) - pool := New(testTxPoolConfig, blockchain) + pool := New(&testTxPoolConfig, blockchain) err := pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) if err != nil { t.Fatalf("failed to init pool: %v", err) diff --git a/mempool/txpool/legacypool/list_test.go b/mempool/txpool/legacypool/list_test.go index c3a96beaf..8587c66f7 100644 --- a/mempool/txpool/legacypool/list_test.go +++ b/mempool/txpool/legacypool/list_test.go @@ -40,7 +40,7 @@ func TestStrictListAdd(t *testing.T) { // Insert the transactions in a random order list := newList(true) for _, v := range rand.Perm(len(txs)) { - list.Add(txs[v], DefaultMempoolConfig().PriceBump) + list.Add(txs[v], DefaultConfig.PriceBump) } // Verify internal state if len(list.txs.items) != len(txs) { @@ -64,7 +64,7 @@ func TestListAddVeryExpensive(t *testing.T) { gaslimit := uint64(i) tx, _ := types.SignTx(types.NewTransaction(uint64(i), common.Address{}, value, gaslimit, gasprice, nil), types.HomesteadSigner{}, key) t.Logf("cost: %x bitlen: %d\n", tx.Cost(), tx.Cost().BitLen()) - list.Add(tx, DefaultMempoolConfig().PriceBump) + list.Add(tx, DefaultConfig.PriceBump) } } @@ -77,13 +77,13 @@ func BenchmarkListAdd(b *testing.B) { txs[i] = transaction(uint64(i), 0, key) } // Insert the transactions in a random order - priceLimit := uint256.NewInt(DefaultMempoolConfig().PriceLimit) + priceLimit := uint256.NewInt(DefaultConfig.PriceLimit) b.ResetTimer() for i := 0; i < b.N; i++ { list := newList(true) for _, v := range rand.Perm(len(txs)) { - list.Add(txs[v], DefaultMempoolConfig().PriceBump) - list.Filter(priceLimit, DefaultMempoolConfig().PriceBump) + list.Add(txs[v], DefaultConfig.PriceBump) + list.Filter(priceLimit, DefaultConfig.PriceBump) } } } @@ -102,7 +102,7 @@ func BenchmarkListCapOneTx(b *testing.B) { list := newList(true) // Insert the transactions in a random order for _, v := range rand.Perm(len(txs)) { - list.Add(txs[v], DefaultMempoolConfig().PriceBump) + list.Add(txs[v], DefaultConfig.PriceBump) } b.StartTimer() list.Cap(list.Len() - 1) diff --git a/mempool/txpool/locals/tx_tracker_test.go b/mempool/txpool/locals/tx_tracker_test.go index af9ef713b..3553c50cd 100644 --- a/mempool/txpool/locals/tx_tracker_test.go +++ b/mempool/txpool/locals/tx_tracker_test.go @@ -85,7 +85,7 @@ func newTestEnv(t *testing.T, n int, gasTip uint64, journal string) *testEnv { chain, _ := core.NewBlockChain(db, gspec, ethash.NewFaker(), nil) adapter := &testChainAdapter{c: chain} - legacyPool := legacypool.New(legacypool.DefaultConfig, adapter) + legacyPool := legacypool.New(&legacypool.DefaultConfig, adapter) pool, err := txpool.New(gasTip, adapter, []txpool.SubPool{legacyPool}) if err != nil { t.Fatalf("Failed to create tx pool: %v", err) diff --git a/mempool/txpool/txpool_test.go b/mempool/txpool/txpool_test.go index e80805ba6..0d072fd72 100644 --- a/mempool/txpool/txpool_test.go +++ b/mempool/txpool/txpool_test.go @@ -79,7 +79,7 @@ func TestTxPoolCosmosReorg(t *testing.T) { genesisState.On("GetNonce", mock.Anything).Return(uint64(1)) genesisState.On("GetCodeHash", mock.Anything).Return(types.EmptyCodeHash) - legacyPool := legacypool.New(legacypool.DefaultConfig, legacyChain) + legacyPool := legacypool.New(&legacypool.DefaultConfig, legacyChain) // handle txpool subscribing to new head events from the chain. grab the // reference to the chan that it is going to wait on so we can push mock diff --git a/server/config/config.go b/server/config/config.go index a43000e1d..10b221eb0 100644 --- a/server/config/config.go +++ b/server/config/config.go @@ -153,7 +153,7 @@ type EVMConfig struct { // GethMetricsAddress is the address the geth metrics server will bind to. Default 127.0.0.1:8100 GethMetricsAddress string `mapstructure:"geth-metrics-address"` // Mempool defines the EVM mempool configuration - Mempool legacypool.MempoolConfig `mapstructure:"mempool"` + Mempool legacypool.Config `mapstructure:"mempool"` } // JSONRPCConfig defines configuration for the EVM RPC server. @@ -223,7 +223,7 @@ func DefaultEVMConfig() *EVMConfig { EnablePreimageRecording: DefaultEnablePreimageRecording, MinTip: DefaultEVMMinTip, GethMetricsAddress: DefaultGethMetricsAddress, - Mempool: *legacypool.DefaultMempoolConfig(), + Mempool: legacypool.DefaultConfig, } } diff --git a/server/server_app_options.go b/server/server_app_options.go index 5b6d58c05..fc9c0054b 100644 --- a/server/server_app_options.go +++ b/server/server_app_options.go @@ -129,13 +129,13 @@ func GetMinTip(appOpts servertypes.AppOptions, logger log.Logger) *uint256.Int { // GetLegacyPoolConfig reads the legacy pool configuration from appOpts and overrides // default values with values from app.toml if they exist and are non-zero. -func GetLegacyPoolConfig(appOpts servertypes.AppOptions, logger log.Logger) *legacypool.MempoolConfig { +func GetLegacyPoolConfig(appOpts servertypes.AppOptions, logger log.Logger) *legacypool.Config { if appOpts == nil { logger.Error("app options is nil, using default mempool config") - return legacypool.DefaultMempoolConfig() + return &legacypool.DefaultConfig } - legacyConfig := legacypool.DefaultMempoolConfig() + legacyConfig := legacypool.DefaultConfig if priceLimit := cast.ToUint64(appOpts.Get(srvflags.EVMMempoolPriceLimit)); priceLimit != 0 { legacyConfig.PriceLimit = priceLimit } @@ -176,7 +176,7 @@ func GetLegacyPoolConfig(appOpts servertypes.AppOptions, logger log.Logger) *leg legacyConfig.Rejournal = rejournal } - return legacyConfig + return &legacyConfig } func GetCosmosPoolMaxTx(appOpts servertypes.AppOptions, logger log.Logger) int { diff --git a/server/start.go b/server/start.go index 0a3db8632..4473593d8 100644 --- a/server/start.go +++ b/server/start.go @@ -231,13 +231,13 @@ which accepts a path for the resulting pprof file. cmd.Flags().Uint64(srvflags.EVMMinTip, cosmosevmserverconfig.DefaultEVMMinTip, "the minimum priority fee for the mempool") cmd.Flags().String(srvflags.EvmGethMetricsAddress, cosmosevmserverconfig.DefaultGethMetricsAddress, "the address to bind the geth metrics server to") - cmd.Flags().Uint64(srvflags.EVMMempoolPriceLimit, legacypool.DefaultMempoolConfig().PriceLimit, "the minimum gas price to enforce for acceptance into the pool (in wei)") - cmd.Flags().Uint64(srvflags.EVMMempoolPriceBump, legacypool.DefaultMempoolConfig().PriceBump, "the minimum price bump percentage to replace an already existing transaction (nonce)") - cmd.Flags().Uint64(srvflags.EVMMempoolAccountSlots, legacypool.DefaultMempoolConfig().AccountSlots, "the number of executable transaction slots guaranteed per account") - cmd.Flags().Uint64(srvflags.EVMMempoolGlobalSlots, legacypool.DefaultMempoolConfig().GlobalSlots, "the maximum number of executable transaction slots for all accounts") - cmd.Flags().Uint64(srvflags.EVMMempoolAccountQueue, legacypool.DefaultMempoolConfig().AccountQueue, "the maximum number of non-executable transaction slots permitted per account") - cmd.Flags().Uint64(srvflags.EVMMempoolGlobalQueue, legacypool.DefaultMempoolConfig().GlobalQueue, "the maximum number of non-executable transaction slots for all accounts") - cmd.Flags().Duration(srvflags.EVMMempoolLifetime, legacypool.DefaultMempoolConfig().Lifetime, "the maximum amount of time non-executable transaction are queued") + cmd.Flags().Uint64(srvflags.EVMMempoolPriceLimit, legacypool.DefaultConfig.PriceLimit, "the minimum gas price to enforce for acceptance into the pool (in wei)") + cmd.Flags().Uint64(srvflags.EVMMempoolPriceBump, legacypool.DefaultConfig.PriceBump, "the minimum price bump percentage to replace an already existing transaction (nonce)") + cmd.Flags().Uint64(srvflags.EVMMempoolAccountSlots, legacypool.DefaultConfig.AccountSlots, "the number of executable transaction slots guaranteed per account") + cmd.Flags().Uint64(srvflags.EVMMempoolGlobalSlots, legacypool.DefaultConfig.GlobalSlots, "the maximum number of executable transaction slots for all accounts") + cmd.Flags().Uint64(srvflags.EVMMempoolAccountQueue, legacypool.DefaultConfig.AccountQueue, "the maximum number of non-executable transaction slots permitted per account") + cmd.Flags().Uint64(srvflags.EVMMempoolGlobalQueue, legacypool.DefaultConfig.GlobalQueue, "the maximum number of non-executable transaction slots for all accounts") + cmd.Flags().Duration(srvflags.EVMMempoolLifetime, legacypool.DefaultConfig.Lifetime, "the maximum amount of time non-executable transaction are queued") cmd.Flags().StringSlice(srvflags.EVMMempoolLocals, []string{}, "the set of addresses that should be treated by default as local (comma-separated)") cmd.Flags().Bool(srvflags.EVMMempoolNoLocals, false, "disables local transaction handling, exempting local accounts from pricing and acceptance") From 0d1d4cb92a9ad694a0846dd0810c64bdcb393c0c Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 12 Nov 2025 14:21:44 -0500 Subject: [PATCH 29/47] remove parseAddress helper --- server/server_app_options.go | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/server/server_app_options.go b/server/server_app_options.go index fc9c0054b..208154595 100644 --- a/server/server_app_options.go +++ b/server/server_app_options.go @@ -3,9 +3,7 @@ package server import ( "math" "path/filepath" - "strings" - "github.com/ethereum/go-ethereum/common" "github.com/holiman/uint256" "github.com/spf13/cast" @@ -93,21 +91,6 @@ func GetMinGasPrices(appOpts servertypes.AppOptions, logger log.Logger) sdk.DecC return minGasPrices } -// parseAddresses converts a slice of address strings to common.Address slice -func parseAddresses(addressStrs []string) []common.Address { - addresses := make([]common.Address, 0, len(addressStrs)) - for _, addrStr := range addressStrs { - addrStr = strings.TrimSpace(addrStr) - if addrStr == "" { - continue - } - if common.IsHexAddress(addrStr) { - addresses = append(addresses, common.HexToAddress(addrStr)) - } - } - return addresses -} - // GetMinTip reads the min tip from the app options, set from app.toml // This field is also known as the minimum priority fee func GetMinTip(appOpts servertypes.AppOptions, logger log.Logger) *uint256.Int { From 67a175ee8214d7e0887f81d4b21a830f56cf8d89 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 12 Nov 2025 14:28:40 -0500 Subject: [PATCH 30/47] gci lint --- server/config/config.go | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/server/config/config.go b/server/config/config.go index 10b221eb0..5c543334d 100644 --- a/server/config/config.go +++ b/server/config/config.go @@ -7,15 +7,12 @@ import ( "path" "time" - "github.com/spf13/viper" - - cometstrs "github.com/cometbft/cometbft/libs/strings" - errorsmod "cosmossdk.io/errors" - + cometstrs "github.com/cometbft/cometbft/libs/strings" "github.com/cosmos/cosmos-sdk/server/config" errortypes "github.com/cosmos/cosmos-sdk/types/errors" "github.com/cosmos/evm/mempool/txpool/legacypool" + "github.com/spf13/viper" ) const ( From 4311ae4f2bc35a3465f76fcaceb4e26ea2a1d535 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 12 Nov 2025 15:01:53 -0500 Subject: [PATCH 31/47] lint --- rpc/backend/call_tx.go | 24 ++++++++++++------------ server/config/config.go | 10 +++++++--- server/config/opendb.go | 1 - server/config/opendb_rocksdb.go | 5 +++-- 4 files changed, 22 insertions(+), 18 deletions(-) diff --git a/rpc/backend/call_tx.go b/rpc/backend/call_tx.go index 1c2d24275..daea4beb0 100644 --- a/rpc/backend/call_tx.go +++ b/rpc/backend/call_tx.go @@ -124,11 +124,11 @@ func (b *Backend) SendTransaction(args evmtypes.TransactionArgs) (common.Hash, e } if err != nil { // Check for temporary rejection in error - if b.Mempool != nil && txlocals.IsTemporaryReject(err) { - b.Logger.Debug("temporary rejection in error, tracking locally", "hash", txHash.Hex(), "err", err.Error()) - b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{ethTx}) - return txHash, nil - } + // if b.Mempool != nil && txlocals.IsTemporaryReject(err) { + // b.Logger.Debug("temporary rejection in error, tracking locally", "hash", txHash.Hex(), "err", err.Error()) + // b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{ethTx}) + // return txHash, nil + // } b.Logger.Error("failed to broadcast tx", "error", err.Error()) return txHash, err } @@ -202,12 +202,12 @@ func (b *Backend) SendRawTransaction(data hexutil.Bytes) (common.Hash, error) { err = errorsmod.ABCIError(rsp.Codespace, rsp.Code, rsp.RawLog) } if err != nil { - // Check for temporary rejection in response raw log - if b.Mempool != nil && txlocals.IsTemporaryReject(err) { - b.Logger.Debug("temporary rejection in error, tracking locally", "hash", txHash.Hex(), "err", err.Error()) - b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{tx}) - return txHash, nil - } + // Check for temporary rejection in err + // if b.Mempool != nil && txlocals.IsTemporaryReject(err) { + // b.Logger.Debug("temporary rejection in error, tracking locally", "hash", txHash.Hex(), "err", err.Error()) + // b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{tx}) + // return txHash, nil + // } if b.Mempool != nil && strings.Contains(err.Error(), mempool.ErrNonceLow.Error()) { from, err := ethSigner.Sender(tx) if err != nil { @@ -224,7 +224,7 @@ func (b *Backend) SendRawTransaction(data hexutil.Bytes) (common.Hash, error) { } // SendRawTransaction does not return error when committed nonce <= tx.Nonce < pending nonce - // Track as local for persistence until pending + // Track as local until pending or rejected b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{tx}) return txHash, nil } diff --git a/server/config/config.go b/server/config/config.go index 5c543334d..88bb71bbd 100644 --- a/server/config/config.go +++ b/server/config/config.go @@ -7,12 +7,16 @@ import ( "path" "time" - errorsmod "cosmossdk.io/errors" + "github.com/spf13/viper" + cometstrs "github.com/cometbft/cometbft/libs/strings" + + "github.com/cosmos/evm/mempool/txpool/legacypool" + + errorsmod "cosmossdk.io/errors" + "github.com/cosmos/cosmos-sdk/server/config" errortypes "github.com/cosmos/cosmos-sdk/types/errors" - "github.com/cosmos/evm/mempool/txpool/legacypool" - "github.com/spf13/viper" ) const ( diff --git a/server/config/opendb.go b/server/config/opendb.go index af6de8dc3..1d0d99be7 100644 --- a/server/config/opendb.go +++ b/server/config/opendb.go @@ -1,5 +1,4 @@ //go:build !rocksdb -// +build !rocksdb package config diff --git a/server/config/opendb_rocksdb.go b/server/config/opendb_rocksdb.go index f81aa0271..fea6ebb95 100644 --- a/server/config/opendb_rocksdb.go +++ b/server/config/opendb_rocksdb.go @@ -1,5 +1,4 @@ //go:build rocksdb -// +build rocksdb package config @@ -8,9 +7,11 @@ import ( "runtime" "strings" + "github.com/linxGnu/grocksdb" + dbm "github.com/cosmos/cosmos-db" + "github.com/cosmos/cosmos-sdk/server/types" - "github.com/linxGnu/grocksdb" ) // 3G block cache From 82624f091dcc70ac3efdb829959d15dfdb94d4ec Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 12 Nov 2025 15:22:27 -0500 Subject: [PATCH 32/47] mirror geth logic --- mempool/mempool.go | 4 +- mempool/txpool/legacypool/legacypool.go | 55 ++++++++++--------- mempool/txpool/legacypool/legacypool2_test.go | 8 +-- mempool/txpool/legacypool/legacypool_test.go | 36 ++++++------ mempool/txpool/locals/tx_tracker_test.go | 2 +- mempool/txpool/txpool_test.go | 2 +- server/config/config.go | 4 +- 7 files changed, 55 insertions(+), 56 deletions(-) diff --git a/mempool/mempool.go b/mempool/mempool.go index aaae1ce24..162d2a549 100644 --- a/mempool/mempool.go +++ b/mempool/mempool.go @@ -117,9 +117,9 @@ func NewExperimentalEVMMempool( blockchain = NewBlockchain(getCtxCallback, logger, vmKeeper, feeMarketKeeper, config.BlockGasLimit) // Create txPool from configuration - legacyConfig := &legacypool.DefaultConfig + legacyConfig := legacypool.DefaultConfig if config.LegacyPoolConfig != nil { - legacyConfig = config.LegacyPoolConfig + legacyConfig = *config.LegacyPoolConfig } legacyPool := legacypool.New(legacyConfig, blockchain) diff --git a/mempool/txpool/legacypool/legacypool.go b/mempool/txpool/legacypool/legacypool.go index e62cdb142..8c0ef3f7b 100644 --- a/mempool/txpool/legacypool/legacypool.go +++ b/mempool/txpool/legacypool/legacypool.go @@ -18,12 +18,10 @@ package legacypool import ( - "fmt" "maps" "math/big" "slices" "sort" - "strings" "sync" "sync/atomic" "time" @@ -159,33 +157,38 @@ var DefaultConfig = Config{ Rejournal: time.Hour, // Regenerate journal every hour } -// Validate returns an error if the mempool configuration is invalid -func (c Config) Validate() error { - if c.PriceLimit < 1 { - return fmt.Errorf("price limit must be at least 1, got %d", c.PriceLimit) +// Sanitize checks the provided user configurations and changes anything that's unreasonable or unworkable. +func (config *Config) Sanitize() Config { + conf := *config + if conf.PriceLimit < 1 { + log.Warn("Sanitizing invalid txpool price limit", "provided", conf.PriceLimit, "updated", DefaultConfig.PriceLimit) + conf.PriceLimit = DefaultConfig.PriceLimit } - if c.PriceBump < 1 { - return fmt.Errorf("price bump must be at least 1, got %d", c.PriceBump) + if conf.PriceBump < 1 { + log.Warn("Sanitizing invalid txpool price bump", "provided", conf.PriceBump, "updated", DefaultConfig.PriceBump) + conf.PriceBump = DefaultConfig.PriceBump } - if c.AccountSlots < 1 { - return fmt.Errorf("account slots must be at least 1, got %d", c.AccountSlots) + if conf.AccountSlots < 1 { + log.Warn("Sanitizing invalid txpool account slots", "provided", conf.AccountSlots, "updated", DefaultConfig.AccountSlots) + conf.AccountSlots = DefaultConfig.AccountSlots } - if c.GlobalSlots < 1 { - return fmt.Errorf("global slots must be at least 1, got %d", c.GlobalSlots) + if conf.GlobalSlots < 1 { + log.Warn("Sanitizing invalid txpool global slots", "provided", conf.GlobalSlots, "updated", DefaultConfig.GlobalSlots) + conf.GlobalSlots = DefaultConfig.GlobalSlots } - if c.AccountQueue < 1 { - return fmt.Errorf("account queue must be at least 1, got %d", c.AccountQueue) + if conf.AccountQueue < 1 { + log.Warn("Sanitizing invalid txpool account queue", "provided", conf.AccountQueue, "updated", DefaultConfig.AccountQueue) + conf.AccountQueue = DefaultConfig.AccountQueue } - if c.GlobalQueue < 1 { - return fmt.Errorf("global queue must be at least 1, got %d", c.GlobalQueue) + if conf.GlobalQueue < 1 { + log.Warn("Sanitizing invalid txpool global queue", "provided", conf.GlobalQueue, "updated", DefaultConfig.GlobalQueue) + conf.GlobalQueue = DefaultConfig.GlobalQueue } - if c.Lifetime < 1 { - return fmt.Errorf("lifetime must be at least 1 nanosecond, got %s", c.Lifetime) + if conf.Lifetime < 1 { + log.Warn("Sanitizing invalid txpool lifetime", "provided", conf.Lifetime, "updated", DefaultConfig.Lifetime) + conf.Lifetime = DefaultConfig.Lifetime } - if c.Journal != "" && !strings.HasSuffix(c.Journal, ".rlp") { - return fmt.Errorf("journal must end with .rlp, got %s", c.Journal) - } - return nil + return conf } // LegacyPool contains all currently known transactions. Transactions @@ -248,15 +251,13 @@ type txpoolResetRequest struct { // New creates a new transaction pool to gather, sort and filter inbound // transactions from the network. -func New(config *Config, chain BlockChain) *LegacyPool { +func New(config Config, chain BlockChain) *LegacyPool { // Sanitize the input to ensure no vulnerable gas prices are set - if err := config.Validate(); err != nil { - panic(err) - } + config = config.Sanitize() // Create the transaction pool with its initial settings pool := &LegacyPool{ - config: *config, + config: config, chain: chain, chainconfig: chain.Config(), signer: types.LatestSigner(chain.Config()), diff --git a/mempool/txpool/legacypool/legacypool2_test.go b/mempool/txpool/legacypool/legacypool2_test.go index 20baf9018..deb06aa61 100644 --- a/mempool/txpool/legacypool/legacypool2_test.go +++ b/mempool/txpool/legacypool/legacypool2_test.go @@ -86,7 +86,7 @@ func TestTransactionFutureAttack(t *testing.T) { config := testTxPoolConfig config.GlobalQueue = 100 config.GlobalSlots = 100 - pool := New(&config, blockchain) + pool := New(config, blockchain) pool.Init(config.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -121,7 +121,7 @@ func TestTransactionFuture1559(t *testing.T) { // Create the pool to test the pricing enforcement with statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) blockchain := newTestBlockChain(eip1559Config, 1000000, statedb, new(event.Feed)) - pool := New(&testTxPoolConfig, blockchain) + pool := New(testTxPoolConfig, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -154,7 +154,7 @@ func TestTransactionZAttack(t *testing.T) { // Create the pool to test the pricing enforcement with statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) blockchain := newTestBlockChain(eip1559Config, 1000000, statedb, new(event.Feed)) - pool := New(&testTxPoolConfig, blockchain) + pool := New(testTxPoolConfig, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() // Create a number of test accounts, fund them and make transactions @@ -227,7 +227,7 @@ func BenchmarkFutureAttack(b *testing.B) { config := testTxPoolConfig config.GlobalQueue = 100 config.GlobalSlots = 100 - pool := New(&config, blockchain) + pool := New(config, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() fillPool(b, pool) diff --git a/mempool/txpool/legacypool/legacypool_test.go b/mempool/txpool/legacypool/legacypool_test.go index 56fae66e0..58ce66eaa 100644 --- a/mempool/txpool/legacypool/legacypool_test.go +++ b/mempool/txpool/legacypool/legacypool_test.go @@ -236,7 +236,7 @@ func setupPoolWithConfig(config *params.ChainConfig) (*LegacyPool, *ecdsa.Privat blockchain := newTestBlockChain(config, 10000000, statedb, new(event.Feed)) key, _ := crypto.GenerateKey() - pool := New(&testTxPoolConfig, blockchain) + pool := New(testTxPoolConfig, blockchain) if err := pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()); err != nil { panic(err) } @@ -369,7 +369,7 @@ func TestStateChangeDuringReset(t *testing.T) { tx0 := transaction(0, 100000, key) tx1 := transaction(1, 100000, key) - pool := New(&testTxPoolConfig, blockchain) + pool := New(testTxPoolConfig, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -786,7 +786,7 @@ func TestPostponing(t *testing.T) { statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed)) - pool := New(&testTxPoolConfig, blockchain) + pool := New(testTxPoolConfig, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -996,7 +996,7 @@ func TestQueueGlobalLimiting(t *testing.T) { config.NoLocals = true config.GlobalQueue = config.AccountQueue*3 - 1 // reduce the queue limits to shorten test time (-1 to make it non divisible) - pool := New(&config, blockchain) + pool := New(config, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -1048,7 +1048,7 @@ func TestQueueTimeLimiting(t *testing.T) { config := testTxPoolConfig config.Lifetime = time.Second - pool := New(&config, blockchain) + pool := New(config, blockchain) pool.Init(config.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -1209,7 +1209,7 @@ func TestPendingGlobalLimiting(t *testing.T) { config := testTxPoolConfig config.GlobalSlots = config.AccountSlots * 10 - pool := New(&config, blockchain) + pool := New(config, blockchain) pool.Init(config.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -1308,7 +1308,7 @@ func TestCapClearsFromAll(t *testing.T) { config.AccountQueue = 2 config.GlobalSlots = 8 - pool := New(&config, blockchain) + pool := New(config, blockchain) pool.Init(config.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -1341,7 +1341,7 @@ func TestPendingMinimumAllowance(t *testing.T) { config := testTxPoolConfig config.GlobalSlots = 1 - pool := New(&config, blockchain) + pool := New(config, blockchain) pool.Init(config.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -1385,7 +1385,7 @@ func TestRepricing(t *testing.T) { statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed)) - pool := New(&testTxPoolConfig, blockchain) + pool := New(testTxPoolConfig, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -1490,7 +1490,7 @@ func TestMinGasPriceEnforced(t *testing.T) { txPoolConfig := DefaultConfig txPoolConfig.NoLocals = true - pool := New(&txPoolConfig, blockchain) + pool := New(txPoolConfig, blockchain) pool.Init(txPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -1639,7 +1639,7 @@ func TestUnderpricing(t *testing.T) { config.GlobalSlots = 2 config.GlobalQueue = 2 - pool := New(&config, blockchain) + pool := New(config, blockchain) pool.Init(config.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -1728,7 +1728,7 @@ func TestStableUnderpricing(t *testing.T) { config := testTxPoolConfig config.GlobalSlots = 128 - pool := New(&config, blockchain) + pool := New(config, blockchain) pool.Init(config.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -1931,7 +1931,7 @@ func TestDeduplication(t *testing.T) { statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed)) - pool := New(&testTxPoolConfig, blockchain) + pool := New(testTxPoolConfig, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -1998,7 +1998,7 @@ func TestReplacement(t *testing.T) { statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed)) - pool := New(&testTxPoolConfig, blockchain) + pool := New(testTxPoolConfig, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -2190,7 +2190,7 @@ func TestStatusCheck(t *testing.T) { statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed)) - pool := New(&testTxPoolConfig, blockchain) + pool := New(testTxPoolConfig, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -2263,7 +2263,7 @@ func TestSetCodeTransactions(t *testing.T) { statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) blockchain := newTestBlockChain(params.MergedTestChainConfig, 1000000, statedb, new(event.Feed)) - pool := New(&testTxPoolConfig, blockchain) + pool := New(testTxPoolConfig, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -2561,7 +2561,7 @@ func TestSetCodeTransactionsReorg(t *testing.T) { statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) blockchain := newTestBlockChain(params.MergedTestChainConfig, 1000000, statedb, new(event.Feed)) - pool := New(&testTxPoolConfig, blockchain) + pool := New(testTxPoolConfig, blockchain) pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) defer pool.Close() @@ -2620,7 +2620,7 @@ func TestRemoveTxTruncatePoolRace(t *testing.T) { statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) blockchain := newTestBlockChain(params.MergedTestChainConfig, 1000000, statedb, new(event.Feed)) - pool := New(&testTxPoolConfig, blockchain) + pool := New(testTxPoolConfig, blockchain) err := pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), newReserver()) if err != nil { t.Fatalf("failed to init pool: %v", err) diff --git a/mempool/txpool/locals/tx_tracker_test.go b/mempool/txpool/locals/tx_tracker_test.go index 3553c50cd..af9ef713b 100644 --- a/mempool/txpool/locals/tx_tracker_test.go +++ b/mempool/txpool/locals/tx_tracker_test.go @@ -85,7 +85,7 @@ func newTestEnv(t *testing.T, n int, gasTip uint64, journal string) *testEnv { chain, _ := core.NewBlockChain(db, gspec, ethash.NewFaker(), nil) adapter := &testChainAdapter{c: chain} - legacyPool := legacypool.New(&legacypool.DefaultConfig, adapter) + legacyPool := legacypool.New(legacypool.DefaultConfig, adapter) pool, err := txpool.New(gasTip, adapter, []txpool.SubPool{legacyPool}) if err != nil { t.Fatalf("Failed to create tx pool: %v", err) diff --git a/mempool/txpool/txpool_test.go b/mempool/txpool/txpool_test.go index 0d072fd72..e80805ba6 100644 --- a/mempool/txpool/txpool_test.go +++ b/mempool/txpool/txpool_test.go @@ -79,7 +79,7 @@ func TestTxPoolCosmosReorg(t *testing.T) { genesisState.On("GetNonce", mock.Anything).Return(uint64(1)) genesisState.On("GetCodeHash", mock.Anything).Return(types.EmptyCodeHash) - legacyPool := legacypool.New(&legacypool.DefaultConfig, legacyChain) + legacyPool := legacypool.New(legacypool.DefaultConfig, legacyChain) // handle txpool subscribing to new head events from the chain. grab the // reference to the chan that it is going to wait on so we can push mock diff --git a/server/config/config.go b/server/config/config.go index 88bb71bbd..4bccdf841 100644 --- a/server/config/config.go +++ b/server/config/config.go @@ -238,9 +238,7 @@ func (c EVMConfig) Validate() error { return fmt.Errorf("invalid geth metrics address %q: %w", c.GethMetricsAddress, err) } - if err := c.Mempool.Validate(); err != nil { - return fmt.Errorf("invalid mempool config: %w", err) - } + c.Mempool = c.Mempool.Sanitize() return nil } From be63fbb52151c46638c63e873ee88748e575ba08 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 12 Nov 2025 15:25:37 -0500 Subject: [PATCH 33/47] add back sanitized value to match geth --- mempool/track_local_txs_test.go | 4 ---- mempool/txpool/legacypool/legacypool_test.go | 1 + 2 files changed, 1 insertion(+), 4 deletions(-) delete mode 100644 mempool/track_local_txs_test.go diff --git a/mempool/track_local_txs_test.go b/mempool/track_local_txs_test.go deleted file mode 100644 index 74aed4921..000000000 --- a/mempool/track_local_txs_test.go +++ /dev/null @@ -1,4 +0,0 @@ -// Intentionally left empty; comprehensive TxTracker behavior is tested in -// package locals where internals are accessible. This file reserved for future -// integration wiring tests if needed. -package mempool diff --git a/mempool/txpool/legacypool/legacypool_test.go b/mempool/txpool/legacypool/legacypool_test.go index 58ce66eaa..41d298bae 100644 --- a/mempool/txpool/legacypool/legacypool_test.go +++ b/mempool/txpool/legacypool/legacypool_test.go @@ -1727,6 +1727,7 @@ func TestStableUnderpricing(t *testing.T) { config := testTxPoolConfig config.GlobalSlots = 128 + config.GlobalQueue = 0 pool := New(config, blockchain) pool.Init(config.PriceLimit, blockchain.CurrentBlock(), newReserver()) From 706e9b9c47dccc578a229f9f99c8bb54f4c65ea8 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 12 Nov 2025 16:20:50 -0500 Subject: [PATCH 34/47] uncomment temp err check --- rpc/backend/call_tx.go | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/rpc/backend/call_tx.go b/rpc/backend/call_tx.go index daea4beb0..03e63b6b5 100644 --- a/rpc/backend/call_tx.go +++ b/rpc/backend/call_tx.go @@ -124,11 +124,11 @@ func (b *Backend) SendTransaction(args evmtypes.TransactionArgs) (common.Hash, e } if err != nil { // Check for temporary rejection in error - // if b.Mempool != nil && txlocals.IsTemporaryReject(err) { - // b.Logger.Debug("temporary rejection in error, tracking locally", "hash", txHash.Hex(), "err", err.Error()) - // b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{ethTx}) - // return txHash, nil - // } + if b.Mempool != nil && txlocals.IsTemporaryReject(err) { + b.Logger.Debug("temporary rejection in error, tracking locally", "hash", txHash.Hex(), "err", err.Error()) + b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{ethTx}) + return txHash, nil + } b.Logger.Error("failed to broadcast tx", "error", err.Error()) return txHash, err } @@ -203,11 +203,11 @@ func (b *Backend) SendRawTransaction(data hexutil.Bytes) (common.Hash, error) { } if err != nil { // Check for temporary rejection in err - // if b.Mempool != nil && txlocals.IsTemporaryReject(err) { - // b.Logger.Debug("temporary rejection in error, tracking locally", "hash", txHash.Hex(), "err", err.Error()) - // b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{tx}) - // return txHash, nil - // } + if b.Mempool != nil && txlocals.IsTemporaryReject(err) { + b.Logger.Debug("temporary rejection in error, tracking locally", "hash", txHash.Hex(), "err", err.Error()) + b.Mempool.TrackLocalTxs([]*ethtypes.Transaction{tx}) + return txHash, nil + } if b.Mempool != nil && strings.Contains(err.Error(), mempool.ErrNonceLow.Error()) { from, err := ethSigner.Sender(tx) if err != nil { From f03440ab655f19656f275630dbb212614c93d388 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 12 Nov 2025 16:39:40 -0500 Subject: [PATCH 35/47] account for temp rejections in system tests --- tests/systemtests/mempool/test_replacement.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/systemtests/mempool/test_replacement.go b/tests/systemtests/mempool/test_replacement.go index f705e5079..24d5f3137 100644 --- a/tests/systemtests/mempool/test_replacement.go +++ b/tests/systemtests/mempool/test_replacement.go @@ -420,8 +420,7 @@ func RunMixedTxsReplacementLegacyAndDynamicFee(t *testing.T, base *suite.BaseTes require.NoError(t, err, "failed to send eth legacy tx") _, err = s.SendEthDynamicFeeTx(t, s.Node(0), signer.ID, 1, s.GasPriceMultiplier(20), big.NewInt(1)) - require.Error(t, err) - require.Contains(t, err.Error(), "replacement transaction underpriced") + require.NoError(t, err, "temporary underpriced tx rejection is tracked locally") ctx.SetExpQueuedTxs(tx1) }, From a406eb178ed2626468dde2993fa926a99b47aed3 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Wed, 12 Nov 2025 16:45:58 -0500 Subject: [PATCH 36/47] account for temp rejections in system tests in legacy case --- tests/systemtests/mempool/test_replacement.go | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/systemtests/mempool/test_replacement.go b/tests/systemtests/mempool/test_replacement.go index 24d5f3137..8a64ecf50 100644 --- a/tests/systemtests/mempool/test_replacement.go +++ b/tests/systemtests/mempool/test_replacement.go @@ -420,7 +420,7 @@ func RunMixedTxsReplacementLegacyAndDynamicFee(t *testing.T, base *suite.BaseTes require.NoError(t, err, "failed to send eth legacy tx") _, err = s.SendEthDynamicFeeTx(t, s.Node(0), signer.ID, 1, s.GasPriceMultiplier(20), big.NewInt(1)) - require.NoError(t, err, "temporary underpriced tx rejection is tracked locally") + require.NoError(t, err, "temporary underpriced tx rejection should be tracked locally and not error") ctx.SetExpQueuedTxs(tx1) }, @@ -470,8 +470,7 @@ func RunMixedTxsReplacementLegacyAndDynamicFee(t *testing.T, base *suite.BaseTes require.NoError(t, err) _, err = s.SendEthLegacyTx(t, s.Node(0), signer.ID, 1, s.GasPriceMultiplier(10)) - require.Error(t, err, "failed to send eth legacy tx") - require.Contains(t, err.Error(), "replacement transaction underpriced") + require.NoError(t, err, "temporary underpriced tx rejection should be tracked locally and not error") // Legacy tx cannot replace dynamic fee tx. ctx.SetExpQueuedTxs(tx1) From 6d48eecbec685858c081a19ae469ef470f4224ea Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Thu, 13 Nov 2025 09:57:17 -0500 Subject: [PATCH 37/47] move rotate to within recheck lock --- mempool/mempool.go | 8 +++++ mempool/txpool/locals/journal.go | 50 ++++++++++++----------------- mempool/txpool/locals/tx_tracker.go | 41 ++++++++++++----------- 3 files changed, 51 insertions(+), 48 deletions(-) diff --git a/mempool/mempool.go b/mempool/mempool.go index 162d2a549..1d44817db 100644 --- a/mempool/mempool.go +++ b/mempool/mempool.go @@ -4,6 +4,8 @@ import ( "context" "errors" "fmt" + "os" + "path/filepath" "sync" "time" @@ -157,6 +159,12 @@ func NewExperimentalEVMMempool( logger.Debug("Sanitizing invalid txpool journal time", "provided", rejournal, "updated", time.Second) rejournal = time.Second } + // Ensure journal directory exists before starting the tracker + if dir := filepath.Dir(legacyConfig.Journal); dir != "." { + if err := os.MkdirAll(dir, 0755); err != nil { + logger.Error("failed to create journal directory", "error", err) + } + } localTxTracker = locals.New(legacyConfig.Journal, rejournal, blockchain.Config(), txPool) err := localTxTracker.Start() if err != nil { diff --git a/mempool/txpool/locals/journal.go b/mempool/txpool/locals/journal.go index 4cd1fc53a..950f47538 100644 --- a/mempool/txpool/locals/journal.go +++ b/mempool/txpool/locals/journal.go @@ -18,11 +18,9 @@ package locals import ( "errors" - "fmt" "io" "io/fs" "os" - "path/filepath" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/types" @@ -119,6 +117,26 @@ func (journal *journal) load(add func([]*types.Transaction) []error) error { return failure } +// setupWriter opens the journal file for writing in append mode +func (journal *journal) setupWriter() error { + if journal.writer != nil { + if err := journal.writer.Close(); err != nil { + return err + } + journal.writer = nil + } + + // Re-open the journal file for appending + // Use O_APPEND to ensure we always write to the end of the file + sink, err := os.OpenFile(journal.path, os.O_WRONLY|os.O_CREATE|os.O_APPEND, 0644) + if err != nil { + return err + } + journal.writer = sink + + return nil +} + // insert adds the specified transaction to the local disk journal. func (journal *journal) insert(tx *types.Transaction) error { if journal.writer == nil { @@ -140,12 +158,6 @@ func (journal *journal) rotate(all map[common.Address]types.Transactions) error } journal.writer = nil } - // Ensure parent directory exists - if dir := filepath.Dir(journal.path); dir != "." { - if err := os.MkdirAll(dir, 0755); err != nil { - return fmt.Errorf("failed to create journal directory: %w", err) - } - } // Generate a new journal with the contents of the current pool replacement, err := os.OpenFile(journal.path+".new", os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644) if err != nil { @@ -182,31 +194,9 @@ func (journal *journal) rotate(all map[common.Address]types.Transactions) error return nil } -// open opens the journal file for writing (in append mode). -// This should be called after load() to ensure new transactions are persisted. -func (journal *journal) open() error { - if journal.writer != nil { - return nil // Already open - } - // Ensure parent directory exists - if dir := filepath.Dir(journal.path); dir != "." { - if err := os.MkdirAll(dir, 0755); err != nil { - return fmt.Errorf("failed to create journal directory: %w", err) - } - } - // Open file for appending, create if it doesn't exist - sink, err := os.OpenFile(journal.path, os.O_WRONLY|os.O_APPEND|os.O_CREATE, 0644) - if err != nil { - return err - } - journal.writer = sink - return nil -} - // close flushes the transaction journal contents to disk and closes the file. func (journal *journal) close() error { var err error - if journal.writer != nil { err = journal.writer.Close() journal.writer = nil diff --git a/mempool/txpool/locals/tx_tracker.go b/mempool/txpool/locals/tx_tracker.go index fab9747af..89c89a0f3 100644 --- a/mempool/txpool/locals/tx_tracker.go +++ b/mempool/txpool/locals/tx_tracker.go @@ -116,13 +116,14 @@ func (tracker *TxTracker) TrackAll(txs []*types.Transaction) { } // recheck checks and returns any transactions that needs to be resubmitted. -func (tracker *TxTracker) recheck(journalCheck bool) (resubmits []*types.Transaction, rejournal map[common.Address]types.Transactions) { +func (tracker *TxTracker) recheck(journalCheck bool) []*types.Transaction { tracker.mu.Lock() defer tracker.mu.Unlock() var ( numStales = 0 numOk = 0 + resubmits []*types.Transaction ) for sender, txs := range tracker.byAddr { // Wipe the stales @@ -143,7 +144,7 @@ func (tracker *TxTracker) recheck(journalCheck bool) (resubmits []*types.Transac } if journalCheck { // rejournal - rejournal = make(map[common.Address]types.Transactions) + rejournal := make(map[common.Address]types.Transactions) for _, tx := range tracker.all { addr, _ := types.Sender(tracker.signer, tx) rejournal[addr] = append(rejournal[addr], tx) @@ -155,10 +156,18 @@ func (tracker *TxTracker) recheck(journalCheck bool) (resubmits []*types.Transac return int(a.Nonce() - b.Nonce()) }) } + // Rejournal the tracker while holding the lock. No new transactions will + // be added to the old journal during this period, preventing any potential + // transaction loss. + if tracker.journal != nil { + if err := tracker.journal.rotate(rejournal); err != nil { + log.Warn("Transaction journal rotation failed", "err", err) + } + } } localGauge.Update(int64(len(tracker.all))) log.Debug("Tx tracker status", "need-resubmit", len(resubmits), "stale", numStales, "ok", numOk) - return resubmits, rejournal + return resubmits } // Start implements node.Lifecycle interface @@ -187,10 +196,11 @@ func (tracker *TxTracker) loop() { tracker.TrackAll(transactions) return nil }) - // Open the journal for writing immediately after load so new transactions - // are persisted right away instead of waiting for first rotation - if err := tracker.journal.open(); err != nil { - log.Warn("Failed to open transaction journal for writing", "err", err) + + // Setup the writer for the upcoming transactions + if err := tracker.journal.setupWriter(); err != nil { + log.Error("Failed to setup the journal writer", "err", err) + return } defer tracker.journal.close() } @@ -203,20 +213,15 @@ func (tracker *TxTracker) loop() { case <-tracker.shutdownCh: return case <-timer.C: - checkJournal := tracker.journal != nil && time.Since(lastJournal) > tracker.rejournal - resubmits, rejournal := tracker.recheck(checkJournal) + var rejournal bool + if tracker.journal != nil && time.Since(lastJournal) > tracker.rejournal { + rejournal, lastJournal = true, time.Now() + log.Debug("Rejournal the transaction tracker") + } + resubmits := tracker.recheck(rejournal) if len(resubmits) > 0 { tracker.pool.Add(resubmits, false) } - if checkJournal { - // Lock to prevent journal.rotate <-> journal.insert (via TrackAll) conflicts - tracker.mu.Lock() - lastJournal = time.Now() - if err := tracker.journal.rotate(rejournal); err != nil { - log.Warn("Transaction journal rotation failed", "err", err) - } - tracker.mu.Unlock() - } timer.Reset(recheckInterval) } } From 83b68da7dca31d58561a202e16a99daf187bdf7f Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Thu, 13 Nov 2025 10:06:56 -0500 Subject: [PATCH 38/47] handle gh error checking complaint --- mempool/txpool/locals/journal.go | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/mempool/txpool/locals/journal.go b/mempool/txpool/locals/journal.go index 950f47538..ffd7257f1 100644 --- a/mempool/txpool/locals/journal.go +++ b/mempool/txpool/locals/journal.go @@ -167,13 +167,17 @@ func (journal *journal) rotate(all map[common.Address]types.Transactions) error for _, txs := range all { for _, tx := range txs { if err = rlp.Encode(replacement, tx); err != nil { - replacement.Close() + if closeErr := replacement.Close(); closeErr != nil { + return errors.Join(err, closeErr) + } return err } } journaled += len(txs) } - replacement.Close() + if err := replacement.Close(); err != nil { + return err + } // Replace the live journal with the newly generated one if err = os.Rename(journal.path+".new", journal.path); err != nil { From 421c5157f931dfcf16108a2f32f49b59a1ca7f2d Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Thu, 13 Nov 2025 10:20:14 -0500 Subject: [PATCH 39/47] remove redundant sanitization for rejournal time --- mempool/mempool.go | 8 +------- mempool/txpool/legacypool/legacypool.go | 9 +++++++++ server/server_app_options.go | 1 + 3 files changed, 11 insertions(+), 7 deletions(-) diff --git a/mempool/mempool.go b/mempool/mempool.go index 1d44817db..3c079e915 100644 --- a/mempool/mempool.go +++ b/mempool/mempool.go @@ -7,7 +7,6 @@ import ( "os" "path/filepath" "sync" - "time" ethtypes "github.com/ethereum/go-ethereum/core/types" "github.com/holiman/uint256" @@ -154,18 +153,13 @@ func NewExperimentalEVMMempool( var localTxTracker *locals.TxTracker if !legacyConfig.NoLocals { - rejournal := legacyConfig.Rejournal - if rejournal < time.Second { - logger.Debug("Sanitizing invalid txpool journal time", "provided", rejournal, "updated", time.Second) - rejournal = time.Second - } // Ensure journal directory exists before starting the tracker if dir := filepath.Dir(legacyConfig.Journal); dir != "." { if err := os.MkdirAll(dir, 0755); err != nil { logger.Error("failed to create journal directory", "error", err) } } - localTxTracker = locals.New(legacyConfig.Journal, rejournal, blockchain.Config(), txPool) + localTxTracker = locals.New(legacyConfig.Journal, legacyConfig.Rejournal, blockchain.Config(), txPool) err := localTxTracker.Start() if err != nil { return nil diff --git a/mempool/txpool/legacypool/legacypool.go b/mempool/txpool/legacypool/legacypool.go index 8c0ef3f7b..77230a5b6 100644 --- a/mempool/txpool/legacypool/legacypool.go +++ b/mempool/txpool/legacypool/legacypool.go @@ -22,6 +22,7 @@ import ( "math/big" "slices" "sort" + "strings" "sync" "sync/atomic" "time" @@ -188,6 +189,14 @@ func (config *Config) Sanitize() Config { log.Warn("Sanitizing invalid txpool lifetime", "provided", conf.Lifetime, "updated", DefaultConfig.Lifetime) conf.Lifetime = DefaultConfig.Lifetime } + if conf.Journal != "" && strings.HasSuffix(conf.Journal, ".rlp") { + log.Warn("Sanitizing invalid txpool journal", "provided", conf.Journal, "updated", DefaultConfig.Journal) + conf.Journal = DefaultConfig.Journal + } + if conf.Rejournal < time.Second { + log.Warn("Sanitizing invalid txpool rejournal time", "provided", conf.Rejournal, "updated", time.Second) + conf.Rejournal = time.Second + } return conf } diff --git a/server/server_app_options.go b/server/server_app_options.go index 208154595..08f7515ca 100644 --- a/server/server_app_options.go +++ b/server/server_app_options.go @@ -159,6 +159,7 @@ func GetLegacyPoolConfig(appOpts servertypes.AppOptions, logger log.Logger) *leg legacyConfig.Rejournal = rejournal } + legacyConfig = legacyConfig.Sanitize() return &legacyConfig } From 4d8e357218cb38b48bf76b6cd3222049ceda73e4 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Thu, 13 Nov 2025 10:23:08 -0500 Subject: [PATCH 40/47] negate sanitize check --- mempool/txpool/legacypool/legacypool.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mempool/txpool/legacypool/legacypool.go b/mempool/txpool/legacypool/legacypool.go index 77230a5b6..65893330c 100644 --- a/mempool/txpool/legacypool/legacypool.go +++ b/mempool/txpool/legacypool/legacypool.go @@ -189,7 +189,7 @@ func (config *Config) Sanitize() Config { log.Warn("Sanitizing invalid txpool lifetime", "provided", conf.Lifetime, "updated", DefaultConfig.Lifetime) conf.Lifetime = DefaultConfig.Lifetime } - if conf.Journal != "" && strings.HasSuffix(conf.Journal, ".rlp") { + if conf.Journal != "" && !strings.HasSuffix(conf.Journal, ".rlp") { log.Warn("Sanitizing invalid txpool journal", "provided", conf.Journal, "updated", DefaultConfig.Journal) conf.Journal = DefaultConfig.Journal } From e3b2ce2e4513bdc137b7007734a0244cffeea487 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Thu, 13 Nov 2025 10:30:43 -0500 Subject: [PATCH 41/47] update tests to only use resubmit returns --- mempool/txpool/locals/tx_tracker_test.go | 24 ++++++------------------ 1 file changed, 6 insertions(+), 18 deletions(-) diff --git a/mempool/txpool/locals/tx_tracker_test.go b/mempool/txpool/locals/tx_tracker_test.go index af9ef713b..c8f0b4108 100644 --- a/mempool/txpool/locals/tx_tracker_test.go +++ b/mempool/txpool/locals/tx_tracker_test.go @@ -171,16 +171,10 @@ func TestResubmit(t *testing.T) { } env.tracker.TrackAll(txs) - resubmit, all := env.tracker.recheck(true) + resubmit := env.tracker.recheck(true) if len(resubmit) != len(txsB) { t.Fatalf("Unexpected transactions to resubmit, got: %d, want: %d", len(resubmit), len(txsB)) } - if len(all) == 0 || len(all[address]) == 0 { - t.Fatalf("Unexpected transactions being tracked, got: %d, want: %d", 0, len(txs)) - } - if len(all[address]) != len(txs) { - t.Fatalf("Unexpected transactions being tracked, got: %d, want: %d", len(all[address]), len(txs)) - } } func TestTrackAddsAndResubmitsAll(t *testing.T) { @@ -192,18 +186,15 @@ func TestTrackAddsAndResubmitsAll(t *testing.T) { env.tracker.TrackAll(txs) // Since none are present in the pool, all should be scheduled for resubmission - resubmit, all := env.tracker.recheck(true) + resubmit := env.tracker.recheck(true) if len(resubmit) != len(txs) { t.Fatalf("expected all transactions to be resubmitted, got %d want %d", len(resubmit), len(txs)) } - if len(all[address]) != len(txs) { - t.Fatalf("expected all transactions tracked, got %d want %d", len(all[address]), len(txs)) - } // Now add them to the pool as if resubmitted env.pool.Add(txs, false) - resubmit2, _ := env.tracker.recheck(false) + resubmit2 := env.tracker.recheck(false) if len(resubmit2) != 0 { t.Fatalf("expected no resubmissions after promotion, got %d", len(resubmit2)) } @@ -223,10 +214,7 @@ func TestDropObsoleteOnHigherNonce(t *testing.T) { } // Recheck should drop the first 3 as stale - resubmit, all := env.tracker.recheck(true) - if len(all[address]) != 3 { - t.Fatalf("expected 3 transactions to remain tracked after nonce advance, got %d", len(all[address])) - } + resubmit := env.tracker.recheck(true) for _, tx := range resubmit { // none of the resubmits should have nonce less than current pool nonce sender, _ := types.Sender(signer, tx) @@ -247,14 +235,14 @@ func TestPromoteThenNoRetry(t *testing.T) { env.pool.Add(txsA, true) env.tracker.TrackAll(txs) - resubmit, _ := env.tracker.recheck(false) + resubmit := env.tracker.recheck(false) if len(resubmit) != len(txsB) { t.Fatalf("unexpected resubmit count, got %d want %d", len(resubmit), len(txsB)) } // Promote missing ones by adding them; next recheck should yield none env.pool.Add(resubmit, false) - resubmit2, _ := env.tracker.recheck(false) + resubmit2 := env.tracker.recheck(false) if len(resubmit2) != 0 { t.Fatalf("expected no resubmits after all txs present in pool, got %d", len(resubmit2)) } From 47d4174f58cae80780cf425a2e37ed447ac8d6ce Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Thu, 13 Nov 2025 10:40:15 -0500 Subject: [PATCH 42/47] gofumpt --- mempool/mempool.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mempool/mempool.go b/mempool/mempool.go index 3c079e915..80b370683 100644 --- a/mempool/mempool.go +++ b/mempool/mempool.go @@ -155,7 +155,7 @@ func NewExperimentalEVMMempool( if !legacyConfig.NoLocals { // Ensure journal directory exists before starting the tracker if dir := filepath.Dir(legacyConfig.Journal); dir != "." { - if err := os.MkdirAll(dir, 0755); err != nil { + if err := os.MkdirAll(dir, 0o755); err != nil { logger.Error("failed to create journal directory", "error", err) } } From fc35a1b6c93e2d28a31e7ad72dc6a61f965c6b85 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Thu, 13 Nov 2025 16:27:27 -0500 Subject: [PATCH 43/47] add tests --- tests/integration/mempool/test_tx_tracker.go | 777 +++++++++++++++++++ tests/systemtests/mempool/test_tx_tracker.go | 478 ++++++++++++ 2 files changed, 1255 insertions(+) create mode 100644 tests/integration/mempool/test_tx_tracker.go create mode 100644 tests/systemtests/mempool/test_tx_tracker.go diff --git a/tests/integration/mempool/test_tx_tracker.go b/tests/integration/mempool/test_tx_tracker.go new file mode 100644 index 000000000..7f09eb6c8 --- /dev/null +++ b/tests/integration/mempool/test_tx_tracker.go @@ -0,0 +1,777 @@ +package mempool + +import ( + "math/big" + "os" + "path/filepath" + "time" + + ethtypes "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/metrics" + + evmmempool "github.com/cosmos/evm/mempool" + "github.com/cosmos/evm/mempool/txpool" + "github.com/cosmos/evm/mempool/txpool/locals" + evmtypes "github.com/cosmos/evm/x/vm/types" + + sdk "github.com/cosmos/cosmos-sdk/types" +) + +// TestTxTrackerLifecycle tests the TxTracker lifecycle (start/stop). +func (s *IntegrationTestSuite) TestTxTrackerLifecycle() { + testCases := []struct { + name string + setupFunc func() (*locals.TxTracker, string) + verifyFunc func(*locals.TxTracker) + cleanUp func(string) + }{ + { + name: "TxTracker starts and stops cleanly", + setupFunc: func() (*locals.TxTracker, string) { + evmMempool := s.network.App.GetMempool() + evmmp, ok := evmMempool.(*evmmempool.ExperimentalEVMMempool) + s.Require().True(ok, "mempool should be ExperimentalEVMMempool") + + tmpDir := filepath.Join(os.TempDir(), "tx_tracker_test") + journalPath := filepath.Join(tmpDir, "test_journal.rlp") + + tracker := locals.New( + journalPath, + time.Minute, + evmtypes.GetEthChainConfig(), + evmmp.GetTxPool(), + ) + return tracker, tmpDir + }, + verifyFunc: func(tracker *locals.TxTracker) { + err := tracker.Start() + s.Require().NoError(err, "tracker should start without error") + err = tracker.Stop() + s.Require().NoError(err, "tracker should stop without error") + }, + cleanUp: func(tmpDir string) { + os.RemoveAll(tmpDir) + }, + }, + { + name: "TxTracker without journal works starts and stops cleanly", + setupFunc: func() (*locals.TxTracker, string) { + evmMempool := s.network.App.GetMempool() + evmmp, ok := evmMempool.(*evmmempool.ExperimentalEVMMempool) + s.Require().True(ok) + + tracker := locals.New( + "", // local journal disabled + time.Minute, + evmtypes.GetEthChainConfig(), + evmmp.GetTxPool(), + ) + return tracker, "" + }, + verifyFunc: func(tracker *locals.TxTracker) { + err := tracker.Start() + s.Require().NoError(err) + err = tracker.Stop() + s.Require().NoError(err) + }, + cleanUp: func(tmpDir string) {}, + }, + } + + for _, tc := range testCases { + s.Run(tc.name, func() { + tracker, tmpDir := tc.setupFunc() + tc.verifyFunc(tracker) + if tc.cleanUp != nil { + tc.cleanUp(tmpDir) + } + }) + } +} + +// TestTxTrackerTrackLocalTxs tests tracking transactions through the TxTracker. +func (s *IntegrationTestSuite) TestTxTrackerTrackLocalTxs() { + testCases := []struct { + name string + setupTxs func() []*ethtypes.Transaction + verifyFunc func([]*ethtypes.Transaction) + }{ + { + name: "track single EVM transaction", + setupTxs: func() []*ethtypes.Transaction { + key := s.keyring.GetKey(0) + tx := s.createEVMValueTransferTx(key, 0, big.NewInt(1000000000)) + + ethMsg, ok := tx.GetMsgs()[0].(*evmtypes.MsgEthereumTx) + s.Require().True(ok, "should be EVM transaction") + + return []*ethtypes.Transaction{ethMsg.AsTransaction()} + }, + verifyFunc: func(txs []*ethtypes.Transaction) { + evmMempool := s.network.App.GetMempool() + evmmp, ok := evmMempool.(*evmmempool.ExperimentalEVMMempool) + s.Require().True(ok) + + evmmp.TrackLocalTxs(txs) + guage := metrics.GetOrRegisterGauge("txpool/local", nil) + s.Require().Equal(int64(len(txs)), guage.Snapshot().Value(), "should have tracked one transaction") + }, + }, + { + name: "track multiple EVM transactions", + setupTxs: func() []*ethtypes.Transaction { + key := s.keyring.GetKey(0) + var ethTxs []*ethtypes.Transaction + + for i := 0; i < 3; i++ { + tx := s.createEVMValueTransferTx(key, i, big.NewInt(1000000000)) + ethMsg, ok := tx.GetMsgs()[0].(*evmtypes.MsgEthereumTx) + s.Require().True(ok) + ethTxs = append(ethTxs, ethMsg.AsTransaction()) + } + + return ethTxs + }, + verifyFunc: func(txs []*ethtypes.Transaction) { + evmMempool := s.network.App.GetMempool() + evmmp, ok := evmMempool.(*evmmempool.ExperimentalEVMMempool) + s.Require().True(ok) + + evmmp.TrackLocalTxs(txs) + guage := metrics.GetOrRegisterGauge("txpool/local", nil) + s.Require().Equal(int64(len(txs)), guage.Snapshot().Value(), "should have tracked three transactions") + }, + }, + { + name: "track transactions from multiple accounts", + setupTxs: func() []*ethtypes.Transaction { + var ethTxs []*ethtypes.Transaction + + for i := 0; i < 3; i++ { + key := s.keyring.GetKey(i) + tx := s.createEVMValueTransferTx(key, 0, big.NewInt(1000000000)) + ethMsg, ok := tx.GetMsgs()[0].(*evmtypes.MsgEthereumTx) + s.Require().True(ok) + ethTxs = append(ethTxs, ethMsg.AsTransaction()) + } + + return ethTxs + }, + verifyFunc: func(txs []*ethtypes.Transaction) { + evmMempool := s.network.App.GetMempool() + evmmp, ok := evmMempool.(*evmmempool.ExperimentalEVMMempool) + s.Require().True(ok) + + evmmp.TrackLocalTxs(txs) + guage := metrics.GetOrRegisterGauge("txpool/local", nil) + s.Require().Equal(int64(len(txs)), guage.Snapshot().Value(), "should have tracked three transactions from different accounts") + }, + }, + } + + for _, tc := range testCases { + s.Run(tc.name, func() { + s.TearDownTest() + s.SetupTest() + + txs := tc.setupTxs() + tc.verifyFunc(txs) + }) + } +} + +// TestTxTrackerResubmission tests that TxTracker resubmits missing transactions. +func (s *IntegrationTestSuite) TestTxTrackerResubmission() { + testCases := []struct { + name string + setupTxs func() ([]*ethtypes.Transaction, []sdk.Tx) + insertTxs func([]sdk.Tx) + verifyFunc func([]*ethtypes.Transaction) + }{ + { + name: "resubmits transactions not in mempool", + setupTxs: func() ([]*ethtypes.Transaction, []sdk.Tx) { + key := s.keyring.GetKey(0) + var ethTxs []*ethtypes.Transaction + var sdkTxs []sdk.Tx + + for i := 0; i < 3; i++ { + tx := s.createEVMValueTransferTx(key, i, big.NewInt(1000000000)) + ethMsg, ok := tx.GetMsgs()[0].(*evmtypes.MsgEthereumTx) + s.Require().True(ok) + ethTxs = append(ethTxs, ethMsg.AsTransaction()) + sdkTxs = append(sdkTxs, tx) + } + + return ethTxs, sdkTxs + }, + insertTxs: func(sdkTxs []sdk.Tx) { + mpool := s.network.App.GetMempool() + err := mpool.Insert(s.network.GetContext(), sdkTxs[0]) + s.Require().NoError(err) + }, + verifyFunc: func(ethTxs []*ethtypes.Transaction) { + evmMempool := s.network.App.GetMempool() + evmmp, ok := evmMempool.(*evmmempool.ExperimentalEVMMempool) + s.Require().True(ok) + + evmmp.TrackLocalTxs(ethTxs) + txPool := evmmp.GetTxPool() + s.Require().True(txPool.Has(ethTxs[0].Hash()), "first transaction should be in pool") + + guage := metrics.GetOrRegisterGauge("txpool/local", nil) + s.Require().Equal(int64(len(ethTxs)), guage.Snapshot().Value(), "all transactions should be tracked") + // it is not practical to wait for recheck and test the tracker state + }, + }, + { + name: "does not resubmit transactions already in pool", + setupTxs: func() ([]*ethtypes.Transaction, []sdk.Tx) { + key := s.keyring.GetKey(0) + var ethTxs []*ethtypes.Transaction + var sdkTxs []sdk.Tx + + for i := 0; i < 2; i++ { + tx := s.createEVMValueTransferTx(key, i, big.NewInt(1000000000)) + ethMsg, ok := tx.GetMsgs()[0].(*evmtypes.MsgEthereumTx) + s.Require().True(ok) + ethTxs = append(ethTxs, ethMsg.AsTransaction()) + sdkTxs = append(sdkTxs, tx) + } + + return ethTxs, sdkTxs + }, + insertTxs: func(sdkTxs []sdk.Tx) { + mpool := s.network.App.GetMempool() + for _, tx := range sdkTxs { + err := mpool.Insert(s.network.GetContext(), tx) + s.Require().NoError(err) + } + }, + verifyFunc: func(ethTxs []*ethtypes.Transaction) { + evmMempool := s.network.App.GetMempool() + evmmp, ok := evmMempool.(*evmmempool.ExperimentalEVMMempool) + s.Require().True(ok) + + evmmp.TrackLocalTxs(ethTxs) + + txPool := evmmp.GetTxPool() + for _, ethTx := range ethTxs { + s.Require().True(txPool.Has(ethTx.Hash()), "transaction should be in pool") + } + + guage := metrics.GetOrRegisterGauge("txpool/local", nil) + s.Require().Equal(int64(len(ethTxs)), guage.Snapshot().Value(), "transactions should still be in the tracker") + // it is not practical to wait for recheck and test the tracker state + }, + }, + } + + for _, tc := range testCases { + s.Run(tc.name, func() { + s.TearDownTest() + s.SetupTest() + + ethTxs, sdkTxs := tc.setupTxs() + tc.insertTxs(sdkTxs) + tc.verifyFunc(ethTxs) + }) + } +} + +// TestTxTrackerWithBlockProgression tests TxTracker behavior as blocks progress. +func (s *IntegrationTestSuite) TestTxTrackerWithBlockProgression() { + testCases := []struct { + name string + setupTxs func() ([]*ethtypes.Transaction, []sdk.Tx) + verifyFunc func([]*ethtypes.Transaction, []sdk.Tx) + }{ + { + name: "drops stale transactions after block progression", + setupTxs: func() ([]*ethtypes.Transaction, []sdk.Tx) { + key := s.keyring.GetKey(0) + var ethTxs []*ethtypes.Transaction + var sdkTxs []sdk.Tx + + for i := 0; i < 5; i++ { + tx := s.createEVMValueTransferTx(key, i, big.NewInt(1000000000)) + ethMsg, ok := tx.GetMsgs()[0].(*evmtypes.MsgEthereumTx) + s.Require().True(ok) + ethTxs = append(ethTxs, ethMsg.AsTransaction()) + sdkTxs = append(sdkTxs, tx) + } + + return ethTxs, sdkTxs + }, + verifyFunc: func(ethTxs []*ethtypes.Transaction, sdkTxs []sdk.Tx) { + evmMempool := s.network.App.GetMempool() + evmmp, ok := evmMempool.(*evmmempool.ExperimentalEVMMempool) + s.Require().True(ok) + + evmmp.TrackLocalTxs(ethTxs) + + mpool := s.network.App.GetMempool() + for i := 0; i < 2; i++ { + err := mpool.Insert(s.network.GetContext(), sdkTxs[i]) + s.Require().NoError(err) + } + + for i := 0; i < 2; i++ { + err := s.network.NextBlock() + s.Require().NoError(err) + } + + s.notifyNewBlockToMempool() + + // After block progression, the first 2 transactions should be considered stale + // (their nonces are now below the account's current nonce) + // The tracker should drop these stale transactions on next recheck + // We can verify by checking the mempool state + s.Require().Equal(5, len(ethTxs), "started with 5 tracked transactions") + }, + }, + { + name: "maintains tracked transactions across block progression", + setupTxs: func() ([]*ethtypes.Transaction, []sdk.Tx) { + key := s.keyring.GetKey(0) + var ethTxs []*ethtypes.Transaction + var sdkTxs []sdk.Tx + + for i := 0; i < 3; i++ { + tx := s.createEVMValueTransferTx(key, i, big.NewInt(1000000000)) + ethMsg, ok := tx.GetMsgs()[0].(*evmtypes.MsgEthereumTx) + s.Require().True(ok) + ethTxs = append(ethTxs, ethMsg.AsTransaction()) + sdkTxs = append(sdkTxs, tx) + } + + return ethTxs, sdkTxs + }, + verifyFunc: func(ethTxs []*ethtypes.Transaction, sdkTxs []sdk.Tx) { + evmMempool := s.network.App.GetMempool() + evmmp, ok := evmMempool.(*evmmempool.ExperimentalEVMMempool) + s.Require().True(ok) + + mpool := s.network.App.GetMempool() + for _, tx := range sdkTxs { + err := mpool.Insert(s.network.GetContext(), tx) + s.Require().NoError(err) + } + + evmmp.TrackLocalTxs(ethTxs) + + err := s.network.NextBlock() + s.Require().NoError(err) + + s.notifyNewBlockToMempool() + + txPool := evmmp.GetTxPool() + + // At least the first transaction should still be accessible + // (others might be in queued state depending on pool state) + s.Require().NotNil(txPool, "txPool should exist") + s.Require().Equal(3, len(ethTxs), "all transactions should still be tracked") + }, + }, + } + + for _, tc := range testCases { + s.Run(tc.name, func() { + // Reset test setup to ensure clean state + s.TearDownTest() + s.SetupTest() + + ethTxs, sdkTxs := tc.setupTxs() + tc.verifyFunc(ethTxs, sdkTxs) + }) + } +} + +// TestTxTrackerWithJournal tests TxTracker journal persistence and recovery. +func (s *IntegrationTestSuite) TestTxTrackerWithJournal() { + testCases := []struct { + name string + setupFunc func() (string, []*ethtypes.Transaction) + verifyFunc func(string, []*ethtypes.Transaction) + cleanUp func(string) + }{ + { + name: "persists tracked transactions to journal", + setupFunc: func() (string, []*ethtypes.Transaction) { + tmpDir := filepath.Join(os.TempDir(), "tx_tracker_journal_test") + err := os.MkdirAll(tmpDir, 0o755) + s.Require().NoError(err) + + journalPath := filepath.Join(tmpDir, "test_transactions.rlp") + + evmMempool := s.network.App.GetMempool() + evmmp, ok := evmMempool.(*evmmempool.ExperimentalEVMMempool) + s.Require().True(ok) + + tracker := locals.New( + journalPath, + time.Second, + evmtypes.GetEthChainConfig(), + evmmp.GetTxPool(), + ) + + err = tracker.Start() + s.Require().NoError(err) + + key := s.keyring.GetKey(0) + var ethTxs []*ethtypes.Transaction + + for i := 0; i < 3; i++ { + tx := s.createEVMValueTransferTx(key, i, big.NewInt(1000000000)) + ethMsg, ok := tx.GetMsgs()[0].(*evmtypes.MsgEthereumTx) + s.Require().True(ok) + ethTxs = append(ethTxs, ethMsg.AsTransaction()) + } + + tracker.TrackAll(ethTxs) + + time.Sleep(200 * time.Millisecond) + + err = tracker.Stop() + s.Require().NoError(err) + + return tmpDir, ethTxs + }, + verifyFunc: func(tmpDir string, ethTxs []*ethtypes.Transaction) { + journalPath := filepath.Join(tmpDir, "test_transactions.rlp") + + _, err := os.Stat(journalPath) + s.Require().NoError(err, "journal file should exist") + + evmMempool := s.network.App.GetMempool() + evmmp, ok := evmMempool.(*evmmempool.ExperimentalEVMMempool) + s.Require().True(ok) + + tracker := locals.New( + journalPath, + time.Minute, + evmtypes.GetEthChainConfig(), + evmmp.GetTxPool(), + ) + + err = tracker.Start() + s.Require().NoError(err) + + time.Sleep(200 * time.Millisecond) + + err = tracker.Stop() + s.Require().NoError(err) + + // The tracker should have loaded transactions from the journal + // We can't directly verify the internal state, but we verified the journal exists + s.Require().Equal(3, len(ethTxs), "tracked 3 transactions") + }, + cleanUp: func(tmpDir string) { + os.RemoveAll(tmpDir) + }, + }, + } + + for _, tc := range testCases { + s.Run(tc.name, func() { + tmpDir, ethTxs := tc.setupFunc() + tc.verifyFunc(tmpDir, ethTxs) + if tc.cleanUp != nil { + tc.cleanUp(tmpDir) + } + }) + } +} + +// TestTxTrackerIntegrationWithMempool tests TxTracker integration with the mempool. +func (s *IntegrationTestSuite) TestTxTrackerIntegrationWithMempool() { + testCases := []struct { + name string + setupFunc func() ([]*ethtypes.Transaction, []sdk.Tx) + verifyFunc func([]*ethtypes.Transaction, []sdk.Tx) + }{ + { + name: "tracked transactions are prioritized in mempool", + setupFunc: func() ([]*ethtypes.Transaction, []sdk.Tx) { + key := s.keyring.GetKey(0) + var ethTxs []*ethtypes.Transaction + var sdkTxs []sdk.Tx + + gasPrices := []*big.Int{ + big.NewInt(1000000000), + big.NewInt(2000000000), + big.NewInt(3000000000), + } + + for i, gasPrice := range gasPrices { + tx := s.createEVMValueTransferTx(key, i, gasPrice) + ethMsg, ok := tx.GetMsgs()[0].(*evmtypes.MsgEthereumTx) + s.Require().True(ok) + ethTxs = append(ethTxs, ethMsg.AsTransaction()) + sdkTxs = append(sdkTxs, tx) + } + + return ethTxs, sdkTxs + }, + verifyFunc: func(ethTxs []*ethtypes.Transaction, sdkTxs []sdk.Tx) { + evmMempool := s.network.App.GetMempool() + evmmp, ok := evmMempool.(*evmmempool.ExperimentalEVMMempool) + s.Require().True(ok) + + mpool := s.network.App.GetMempool() + for _, tx := range sdkTxs { + err := mpool.Insert(s.network.GetContext(), tx) + s.Require().NoError(err) + } + + evmmp.TrackLocalTxs(ethTxs) + + txPool := evmmp.GetTxPool() + for _, ethTx := range ethTxs { + s.Require().True(txPool.Has(ethTx.Hash()), "tracked transaction should be in pool") + } + }, + }, + { + name: "TxTracker works with mempool transaction removal", + setupFunc: func() ([]*ethtypes.Transaction, []sdk.Tx) { + key := s.keyring.GetKey(0) + var ethTxs []*ethtypes.Transaction + var sdkTxs []sdk.Tx + + for i := 0; i < 2; i++ { + tx := s.createEVMValueTransferTx(key, i, big.NewInt(1000000000)) + ethMsg, ok := tx.GetMsgs()[0].(*evmtypes.MsgEthereumTx) + s.Require().True(ok) + ethTxs = append(ethTxs, ethMsg.AsTransaction()) + sdkTxs = append(sdkTxs, tx) + } + + return ethTxs, sdkTxs + }, + verifyFunc: func(ethTxs []*ethtypes.Transaction, sdkTxs []sdk.Tx) { + evmMempool := s.network.App.GetMempool() + evmmp, ok := evmMempool.(*evmmempool.ExperimentalEVMMempool) + s.Require().True(ok) + + mpool := s.network.App.GetMempool() + for _, tx := range sdkTxs { + err := mpool.Insert(s.network.GetContext(), tx) + s.Require().NoError(err) + } + + evmmp.TrackLocalTxs(ethTxs) + + txPool := evmmp.GetTxPool() + for _, ethTx := range ethTxs { + s.Require().True(txPool.Has(ethTx.Hash()), "transaction should be in pool") + } + + // Even if transactions are removed from pool, tracker should maintain them + // and attempt to resubmit them on the next recheck + s.Require().Equal(2, len(ethTxs), "tracker should maintain 2 transactions") + }, + }, + } + + for _, tc := range testCases { + s.Run(tc.name, func() { + // Reset test setup to ensure clean state + s.TearDownTest() + s.SetupTest() + + ethTxs, sdkTxs := tc.setupFunc() + tc.verifyFunc(ethTxs, sdkTxs) + }) + } +} + +// TestTxTrackerPoolInteraction tests TxTracker interaction with the TxPool. +func (s *IntegrationTestSuite) TestTxTrackerPoolInteraction() { + testCases := []struct { + name string + setupFunc func() ([]*ethtypes.Transaction, []sdk.Tx, *txpool.TxPool) + verifyFunc func([]*ethtypes.Transaction, *txpool.TxPool) + }{ + { + name: "verifies transaction presence in pool", + setupFunc: func() ([]*ethtypes.Transaction, []sdk.Tx, *txpool.TxPool) { + key := s.keyring.GetKey(0) + var ethTxs []*ethtypes.Transaction + var sdkTxs []sdk.Tx + + for i := 0; i < 2; i++ { + tx := s.createEVMValueTransferTx(key, i, big.NewInt(1000000000)) + ethMsg, ok := tx.GetMsgs()[0].(*evmtypes.MsgEthereumTx) + s.Require().True(ok) + ethTxs = append(ethTxs, ethMsg.AsTransaction()) + sdkTxs = append(sdkTxs, tx) + } + + evmMempool := s.network.App.GetMempool() + evmmp, ok := evmMempool.(*evmmempool.ExperimentalEVMMempool) + s.Require().True(ok) + + txPool := evmmp.GetTxPool() + return ethTxs, sdkTxs, txPool + }, + verifyFunc: func(ethTxs []*ethtypes.Transaction, txPool *txpool.TxPool) { + mpool := s.network.App.GetMempool() + for _, tx := range ethTxs { + ethTxMsg := &evmtypes.MsgEthereumTx{} + ethTxMsg.FromEthereumTx(tx) + + txBuilder := s.network.App.GetTxConfig().NewTxBuilder() + err := txBuilder.SetMsgs(ethTxMsg) + s.Require().NoError(err) + + sdkTx := txBuilder.GetTx() + err = mpool.Insert(s.network.GetContext(), sdkTx) + s.Require().NoError(err) + } + + for _, ethTx := range ethTxs { + s.Require().True(txPool.Has(ethTx.Hash()), "transaction should be present in pool") + } + + pending, queued := txPool.Stats() + s.Require().True(pending > 0, "pool should have pending transactions") + _ = queued // may be 0, just checking pending is sufficient + }, + }, + { + name: "checks account nonce from pool", + setupFunc: func() ([]*ethtypes.Transaction, []sdk.Tx, *txpool.TxPool) { + key := s.keyring.GetKey(0) + var ethTxs []*ethtypes.Transaction + var sdkTxs []sdk.Tx + + tx := s.createEVMValueTransferTx(key, 0, big.NewInt(1000000000)) + ethMsg, ok := tx.GetMsgs()[0].(*evmtypes.MsgEthereumTx) + s.Require().True(ok) + ethTxs = append(ethTxs, ethMsg.AsTransaction()) + sdkTxs = append(sdkTxs, tx) + + evmMempool := s.network.App.GetMempool() + evmmp, ok := evmMempool.(*evmmempool.ExperimentalEVMMempool) + s.Require().True(ok) + + txPool := evmmp.GetTxPool() + return ethTxs, sdkTxs, txPool + }, + verifyFunc: func(ethTxs []*ethtypes.Transaction, txPool *txpool.TxPool) { + mpool := s.network.App.GetMempool() + ethTxMsg := &evmtypes.MsgEthereumTx{} + ethTxMsg.FromEthereumTx(ethTxs[0]) + + txBuilder := s.network.App.GetTxConfig().NewTxBuilder() + err := txBuilder.SetMsgs(ethTxMsg) + s.Require().NoError(err) + + sdkTx := txBuilder.GetTx() + err = mpool.Insert(s.network.GetContext(), sdkTx) + s.Require().NoError(err) + + signer := ethtypes.LatestSigner(evmtypes.GetEthChainConfig()) + sender, err := ethtypes.Sender(signer, ethTxs[0]) + s.Require().NoError(err) + + nonce := txPool.Nonce(sender) + s.Require().Equal(uint64(0), nonce, "pool nonce should match account nonce") + }, + }, + } + + for _, tc := range testCases { + s.Run(tc.name, func() { + s.TearDownTest() + s.SetupTest() + + ethTxs, _, txPool := tc.setupFunc() + tc.verifyFunc(ethTxs, txPool) + }) + } +} + +// TestTxTrackerReplacement tests transaction replacement scenarios with TxTracker. +func (s *IntegrationTestSuite) TestTxTrackerReplacement() { + testCases := []struct { + name string + setupFunc func() ([]*ethtypes.Transaction, []sdk.Tx) + verifyFunc func([]*ethtypes.Transaction, []sdk.Tx) + }{ + { + name: "tracks replacement transaction with higher gas price", + setupFunc: func() ([]*ethtypes.Transaction, []sdk.Tx) { + key := s.keyring.GetKey(0) + var ethTxs []*ethtypes.Transaction + var sdkTxs []sdk.Tx + + tx1 := s.createEVMValueTransferTx(key, 0, big.NewInt(1000000000)) + ethMsg1, ok := tx1.GetMsgs()[0].(*evmtypes.MsgEthereumTx) + s.Require().True(ok) + ethTxs = append(ethTxs, ethMsg1.AsTransaction()) + sdkTxs = append(sdkTxs, tx1) + + tx2 := s.createEVMValueTransferTx(key, 0, big.NewInt(2000000000)) + ethMsg2, ok := tx2.GetMsgs()[0].(*evmtypes.MsgEthereumTx) + s.Require().True(ok) + ethTxs = append(ethTxs, ethMsg2.AsTransaction()) + sdkTxs = append(sdkTxs, tx2) + + return ethTxs, sdkTxs + }, + verifyFunc: func(ethTxs []*ethtypes.Transaction, sdkTxs []sdk.Tx) { + evmMempool := s.network.App.GetMempool() + evmmp, ok := evmMempool.(*evmmempool.ExperimentalEVMMempool) + s.Require().True(ok) + + mpool := s.network.App.GetMempool() + + err := mpool.Insert(s.network.GetContext(), sdkTxs[0]) + s.Require().NoError(err) + + evmmp.TrackLocalTxs([]*ethtypes.Transaction{ethTxs[0]}) + + err = mpool.Insert(s.network.GetContext(), sdkTxs[1]) + s.Require().NoError(err) + + evmmp.TrackLocalTxs([]*ethtypes.Transaction{ethTxs[1]}) + + txPool := evmmp.GetTxPool() + s.Require().True(txPool.Has(ethTxs[1].Hash()), "replacement transaction should be in pool") + + // The original might or might not be in the pool depending on replacement logic + // But both should be tracked + s.Require().Equal(2, len(ethTxs), "both transactions should be tracked") + }, + }, + } + + for _, tc := range testCases { + s.Run(tc.name, func() { + s.TearDownTest() + s.SetupTest() + + ethTxs, sdkTxs := tc.setupFunc() + tc.verifyFunc(ethTxs, sdkTxs) + }) + } +} + +// TestTxTrackerNilChecks tests that TxTracker handles nil cases gracefully. +func (s *IntegrationTestSuite) TestTxTrackerNilChecks() { + s.Run("TrackLocalTxs with nil tracker", func() { + evmMempool := s.network.App.GetMempool() + evmmp, ok := evmMempool.(*evmmempool.ExperimentalEVMMempool) + s.Require().True(ok) + + s.Require().NotPanics(func() { + evmmp.TrackLocalTxs([]*ethtypes.Transaction{}) + }) + + s.Require().NotPanics(func() { + evmmp.TrackLocalTxs(nil) + }) + }) +} diff --git a/tests/systemtests/mempool/test_tx_tracker.go b/tests/systemtests/mempool/test_tx_tracker.go new file mode 100644 index 000000000..e526e55c8 --- /dev/null +++ b/tests/systemtests/mempool/test_tx_tracker.go @@ -0,0 +1,478 @@ +//go:build system_test + +package mempool + +import ( + "context" + "fmt" + "testing" + + "github.com/ethereum/go-ethereum/common" + ethtypes "github.com/ethereum/go-ethereum/core/types" + "github.com/holiman/uint256" + "github.com/stretchr/testify/require" + + "github.com/cosmos/evm/tests/systemtests/suite" +) + +// RunTxTrackerPersistence tests that the TxTracker persists local transactions +// and resubmits them after node restart. +func RunTxTrackerPersistence(t *testing.T, base *suite.BaseTestSuite) { + testCases := []struct { + name string + actions []func(*TestSuite, *TestContext) + }{ + { + name: "tracks and persists local txs %s", + actions: []func(*TestSuite, *TestContext){ + func(s *TestSuite, ctx *TestContext) { + // Submit transaction to node0 + tx1, err := s.SendTx(t, s.Node(0), "acc0", 0, s.GasPriceMultiplier(10), nil) + require.NoError(t, err, "failed to send tx1") + + tx2, err := s.SendTx(t, s.Node(0), "acc0", 1, s.GasPriceMultiplier(10), nil) + require.NoError(t, err, "failed to send tx2") + + // Expect both transactions to be in pending + ctx.SetExpPendingTxs(tx1, tx2) + }, + }, + }, + } + + testOptions := []*suite.TestOptions{ + { + Description: "EVM Legacy Tx", + TxType: suite.TxTypeEVM, + }, + } + + s := NewTestSuite(base) + s.SetupTest(t) + + for _, to := range testOptions { + s.SetOptions(to) + for _, tc := range testCases { + testName := fmt.Sprintf(tc.name, to.Description) + t.Run(testName, func(t *testing.T) { + ctx := NewTestContext() + + s.BeforeEachCase(t, ctx) + for _, action := range tc.actions { + action(s, ctx) + } + s.AfterEachCase(t, ctx) + }) + } + } +} + +// RunTxTrackerResubmission tests that the TxTracker resubmits transactions +// that are missing from the mempool. +func RunTxTrackerResubmission(t *testing.T, base *suite.BaseTestSuite) { + testCases := []struct { + name string + actions []func(*TestSuite, *TestContext) + }{ + { + name: "resubmits missing local txs %s", + actions: []func(*TestSuite, *TestContext){ + func(s *TestSuite, ctx *TestContext) { + // Submit transactions to node0 + tx1, err := s.SendTx(t, s.Node(0), "acc0", 0, s.GasPriceMultiplier(10), nil) + require.NoError(t, err, "failed to send tx1") + + tx2, err := s.SendTx(t, s.Node(0), "acc0", 1, s.GasPriceMultiplier(10), nil) + require.NoError(t, err, "failed to send tx2") + + tx3, err := s.SendTx(t, s.Node(0), "acc0", 2, s.GasPriceMultiplier(10), nil) + require.NoError(t, err, "failed to send tx3") + + // All transactions should be in pending + ctx.SetExpPendingTxs(tx1, tx2, tx3) + }, + }, + }, + } + + testOptions := []*suite.TestOptions{ + { + Description: "EVM Legacy Tx", + TxType: suite.TxTypeEVM, + }, + { + Description: "EVM Dynamic Fee Tx", + TxType: suite.TxTypeEVM, + }, + } + + s := NewTestSuite(base) + s.SetupTest(t) + + for _, to := range testOptions { + s.SetOptions(to) + for _, tc := range testCases { + testName := fmt.Sprintf(tc.name, to.Description) + t.Run(testName, func(t *testing.T) { + ctx := NewTestContext() + + s.BeforeEachCase(t, ctx) + for _, action := range tc.actions { + action(s, ctx) + } + s.AfterEachCase(t, ctx) + }) + } + } +} + +// RunTxTrackerMultipleAccounts tests TxTracker with transactions from multiple accounts. +func RunTxTrackerMultipleAccounts(t *testing.T, base *suite.BaseTestSuite) { + testCases := []struct { + name string + actions []func(*TestSuite, *TestContext) + }{ + { + name: "tracks txs from multiple accounts %s", + actions: []func(*TestSuite, *TestContext){ + func(s *TestSuite, ctx *TestContext) { + // Submit transactions from different accounts + tx1, err := s.SendTx(t, s.Node(0), "acc0", 0, s.GasPriceMultiplier(10), nil) + require.NoError(t, err, "failed to send tx1 from acc0") + + tx2, err := s.SendTx(t, s.Node(0), "acc1", 0, s.GasPriceMultiplier(10), nil) + require.NoError(t, err, "failed to send tx2 from acc1") + + tx3, err := s.SendTx(t, s.Node(0), "acc2", 0, s.GasPriceMultiplier(10), nil) + require.NoError(t, err, "failed to send tx3 from acc2") + + // All transactions should be in pending + ctx.SetExpPendingTxs(tx1, tx2, tx3) + }, + }, + }, + { + name: "tracks and orders txs from multiple accounts by gas price %s", + actions: []func(*TestSuite, *TestContext){ + func(s *TestSuite, ctx *TestContext) { + // Submit transactions with different gas prices + tx1, err := s.SendTx(t, s.Node(0), "acc0", 0, s.GasPriceMultiplier(5), nil) + require.NoError(t, err, "failed to send tx1 with 5x gas") + + tx2, err := s.SendTx(t, s.Node(0), "acc1", 0, s.GasPriceMultiplier(15), nil) + require.NoError(t, err, "failed to send tx2 with 15x gas") + + tx3, err := s.SendTx(t, s.Node(0), "acc2", 0, s.GasPriceMultiplier(10), nil) + require.NoError(t, err, "failed to send tx3 with 10x gas") + + // All transactions should be in pending + // The order might be affected by gas price, but all should be present + ctx.SetExpPendingTxs(tx1, tx2, tx3) + }, + }, + }, + } + + testOptions := []*suite.TestOptions{ + { + Description: "EVM Legacy Tx", + TxType: suite.TxTypeEVM, + }, + } + + s := NewTestSuite(base) + s.SetupTest(t) + + for _, to := range testOptions { + s.SetOptions(to) + for _, tc := range testCases { + testName := fmt.Sprintf(tc.name, to.Description) + t.Run(testName, func(t *testing.T) { + ctx := NewTestContext() + + s.BeforeEachCase(t, ctx) + for _, action := range tc.actions { + action(s, ctx) + } + s.AfterEachCase(t, ctx) + }) + } + } +} + +// RunTxTrackerNonceGaps tests that TxTracker handles nonce gaps correctly. +func RunTxTrackerNonceGaps(t *testing.T, base *suite.BaseTestSuite) { + testCases := []struct { + name string + actions []func(*TestSuite, *TestContext) + }{ + { + name: "handles nonce gaps correctly %s", + actions: []func(*TestSuite, *TestContext){ + func(s *TestSuite, ctx *TestContext) { + // Submit transactions with a nonce gap (0, 2, 4) + tx1, err := s.SendTx(t, s.Node(0), "acc0", 0, s.GasPriceMultiplier(10), nil) + require.NoError(t, err, "failed to send tx1 with nonce 0") + + tx2, err := s.SendTx(t, s.Node(0), "acc0", 2, s.GasPriceMultiplier(10), nil) + require.NoError(t, err, "failed to send tx2 with nonce 2") + + tx3, err := s.SendTx(t, s.Node(0), "acc0", 4, s.GasPriceMultiplier(10), nil) + require.NoError(t, err, "failed to send tx3 with nonce 4") + + // Only tx1 should be pending, tx2 and tx3 should be queued + ctx.SetExpPendingTxs(tx1) + ctx.SetExpQueuedTxs(tx2, tx3) + }, + func(s *TestSuite, ctx *TestContext) { + // Fill the gap by submitting tx with nonce 1 + tx4, err := s.SendTx(t, s.Node(0), "acc0", 1, s.GasPriceMultiplier(10), nil) + require.NoError(t, err, "failed to send tx4 with nonce 1") + + // Now tx1, tx4, tx2 should be pending; tx3 still queued + ctx.SetExpPendingTxs(tx4) + ctx.PromoteExpTxs(1) // Promote tx2 from queued to pending + }, + func(s *TestSuite, ctx *TestContext) { + // Fill the remaining gap by submitting tx with nonce 3 + tx5, err := s.SendTx(t, s.Node(0), "acc0", 3, s.GasPriceMultiplier(10), nil) + require.NoError(t, err, "failed to send tx5 with nonce 3") + + // Now all transactions should be pending + ctx.SetExpPendingTxs(tx5) + ctx.PromoteExpTxs(1) // Promote tx3 from queued to pending + }, + }, + }, + } + + testOptions := []*suite.TestOptions{ + { + Description: "EVM Legacy Tx", + TxType: suite.TxTypeEVM, + }, + } + + s := NewTestSuite(base) + s.SetupTest(t) + + for _, to := range testOptions { + s.SetOptions(to) + for _, tc := range testCases { + testName := fmt.Sprintf(tc.name, to.Description) + t.Run(testName, func(t *testing.T) { + ctx := NewTestContext() + + s.BeforeEachCase(t, ctx) + for _, action := range tc.actions { + action(s, ctx) + s.AfterEachAction(t, ctx) + } + s.AfterEachCase(t, ctx) + }) + } + } +} + +// RunTxTrackerWithReplacement tests that TxTracker handles transaction replacements correctly. +func RunTxTrackerWithReplacement(t *testing.T, base *suite.BaseTestSuite) { + testCases := []struct { + name string + actions []func(*TestSuite, *TestContext) + }{ + { + name: "tracks replacement transaction %s", + actions: []func(*TestSuite, *TestContext){ + func(s *TestSuite, ctx *TestContext) { + // Submit initial transaction + tx1, err := s.SendTx(t, s.Node(0), "acc0", 0, s.GasPriceMultiplier(10), nil) + require.NoError(t, err, "failed to send tx1") + + ctx.SetExpPendingTxs(tx1) + }, + func(s *TestSuite, ctx *TestContext) { + // Submit replacement transaction with higher gas price + tx2, err := s.SendTx(t, s.Node(0), "acc0", 0, s.GasPriceMultiplier(15), nil) + require.NoError(t, err, "failed to send replacement tx2") + + // The replacement transaction should replace tx1 + ctx.SetExpPendingTxs(tx2) + }, + }, + }, + } + + testOptions := []*suite.TestOptions{ + { + Description: "EVM Legacy Tx", + TxType: suite.TxTypeEVM, + }, + } + + s := NewTestSuite(base) + s.SetupTest(t) + + for _, to := range testOptions { + s.SetOptions(to) + for _, tc := range testCases { + testName := fmt.Sprintf(tc.name, to.Description) + t.Run(testName, func(t *testing.T) { + ctx := NewTestContext() + + s.BeforeEachCase(t, ctx) + for _, action := range tc.actions { + action(s, ctx) + s.AfterEachAction(t, ctx) + } + s.AfterEachCase(t, ctx) + }) + } + } +} + +// RunTxTrackerNonceGapEIP7702 tests that nonce gap transactions are tracked locally +// (temporary rejection) rather than being outright rejected when using EIP-7702 delegation. +func RunTxTrackerNonceGapEIP7702(t *testing.T, base *suite.BaseTestSuite) { + var currentNonce uint64 + testCases := []struct { + name string + actions []func(*TestSuite, *TestContext) + }{ + { + name: "tracks nonce gap transaction locally with EIP-7702 delegation %s", + actions: []func(*TestSuite, *TestContext){ + func(s *TestSuite, ctx *TestContext) { + nodeID := s.Node(0) + accID := "acc0" + account := s.EthAccount(accID) + require.NotNil(t, account, "account %s not found", accID) + + // Get current nonce + var err error + currentNonce, err = s.NonceAt(nodeID, accID) + require.NoError(t, err, "failed to get current nonce") + + // Create and send an EIP-7702 transaction with delegation + // Using a dummy delegate address for testing + delegateAddr := common.HexToAddress("0x1111111111111111111111111111111111111111") + + authorization := ethtypes.SetCodeAuthorization{ + ChainID: *uint256.MustFromBig(s.EthClient.ChainID), + Address: delegateAddr, + Nonce: currentNonce, + } + + signedAuth, err := ethtypes.SignSetCode(account.PrivKey, authorization) + require.NoError(t, err, "failed to sign set code authorization") + + // Send EIP-7702 transaction + tx1Hash, err := sendSetCodeTx(s, nodeID, accID, signedAuth) + require.NoError(t, err, "failed to send EIP-7702 tx") + + tx1 := suite.NewTxInfo(nodeID, tx1Hash.Hex(), suite.TxTypeEVM) + ctx.SetExpPendingTxs(tx1) + }, + func(s *TestSuite, ctx *TestContext) { + nodeID := s.Node(0) + accID := "acc0" + + // Get current nonce (should be incremented after first tx) + var err error + currentNonce, err = s.NonceAt(nodeID, accID) + require.NoError(t, err, "failed to get current nonce") + + // Submit a transaction with a gapped nonce (skip currentNonce, use currentNonce+1) + // This should be tracked locally (temporary rejection) rather than outright rejected + _, err = s.SendEthLegacyTx(t, nodeID, accID, 2, s.GasPriceMultiplier(10)) + require.NoError(t, err, "temporary nonce gap rejection should be tracked locally and not error") + + // The gapped transaction should NOT be in pending or queued since it was rejected + // but it should be tracked locally for potential resubmission + // We verify this by checking that no error was returned + }, + func(s *TestSuite, ctx *TestContext) { + nodeID := s.Node(0) + accID := "acc0" + + // Now fill the gap by sending the missing transaction + tx3, err := s.SendEthLegacyTx(t, nodeID, accID, 1, s.GasPriceMultiplier(10)) + require.NoError(t, err, "failed to send gap-filling tx") + + // After filling the gap, the gap-filling transaction should be pending + ctx.SetExpPendingTxs(tx3) + }, + }, + }, + } + + testOptions := []*suite.TestOptions{ + { + Description: "EVM Legacy Tx", + TxType: suite.TxTypeEVM, + }, + } + + s := NewTestSuite(base) + s.SetupTest(t) + + for _, to := range testOptions { + s.SetOptions(to) + for _, tc := range testCases { + testName := fmt.Sprintf(tc.name, to.Description) + t.Run(testName, func(t *testing.T) { + ctx := NewTestContext() + + s.BeforeEachCase(t, ctx) + for _, action := range tc.actions { + action(s, ctx) + s.AfterEachAction(t, ctx) + } + s.AfterEachCase(t, ctx) + }) + } + } +} + +// sendSetCodeTx is a helper function to send an EIP-7702 SetCode transaction. +func sendSetCodeTx(s *TestSuite, nodeID, accID string, signedAuths ...ethtypes.SetCodeAuthorization) (common.Hash, error) { + ctx := context.Background() + ethCli := s.EthClient.Clients[nodeID] + acc := s.EthAccount(accID) + if acc == nil { + return common.Hash{}, fmt.Errorf("account %s not found", accID) + } + key := acc.PrivKey + + chainID, err := ethCli.ChainID(ctx) + if err != nil { + return common.Hash{}, fmt.Errorf("failed to get evm chain id: %w", err) + } + + fromAddr := acc.Address + nonce, err := ethCli.PendingNonceAt(ctx, fromAddr) + if err != nil { + return common.Hash{}, fmt.Errorf("failed to fetch pending nonce: %w", err) + } + + txdata := ðtypes.SetCodeTx{ + ChainID: uint256.MustFromBig(chainID), + Nonce: nonce, + GasTipCap: uint256.NewInt(1_000_000), + GasFeeCap: uint256.NewInt(1_000_000_000), + Gas: 100_000, + To: common.Address{}, + Value: uint256.NewInt(0), + Data: []byte{}, + AccessList: ethtypes.AccessList{}, + AuthList: signedAuths, + } + + signer := ethtypes.LatestSignerForChainID(chainID) + signedTx := ethtypes.MustSignNewTx(key, signer, txdata) + + if err := ethCli.SendTransaction(ctx, signedTx); err != nil { + return common.Hash{}, fmt.Errorf("failed to send transaction: %w", err) + } + + return signedTx.Hash(), nil +} From bc9b918c0d015c6851cb49652246064a3b65862e Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Thu, 13 Nov 2025 16:38:24 -0500 Subject: [PATCH 44/47] add runners --- tests/systemtests/main_test.go | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/tests/systemtests/main_test.go b/tests/systemtests/main_test.go index 50ab0c319..a6c103347 100644 --- a/tests/systemtests/main_test.go +++ b/tests/systemtests/main_test.go @@ -45,6 +45,30 @@ func TestMempoolTxRebroadcasting(t *testing.T) { suite.RunWithSharedSuite(t, mempool.RunTxRebroadcasting) } +func TestMempoolTxTrackerPersistence(t *testing.T) { + suite.RunWithSharedSuite(t, mempool.RunTxTrackerPersistence) +} + +func TestMempoolTxTrackerResubmission(t *testing.T) { + suite.RunWithSharedSuite(t, mempool.RunTxTrackerResubmission) +} + +func TestMempoolTxTrackerMultipleAccounts(t *testing.T) { + suite.RunWithSharedSuite(t, mempool.RunTxTrackerMultipleAccounts) +} + +func TestMempoolTxTrackerNonceGaps(t *testing.T) { + suite.RunWithSharedSuite(t, mempool.RunTxTrackerNonceGaps) +} + +func TestMempoolTxTrackerWithReplacement(t *testing.T) { + suite.RunWithSharedSuite(t, mempool.RunTxTrackerWithReplacement) +} + +func TestMempoolTxTrackerNonceGapEIP7702(t *testing.T) { + suite.RunWithSharedSuite(t, mempool.RunTxTrackerNonceGapEIP7702) +} + func TestMinimumGasPricesZero(t *testing.T) { suite.RunWithSharedSuite(t, mempool.RunMinimumGasPricesZero) } From 534f36986b49b43d40a19c9dfa7f7784cbdf429c Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Thu, 13 Nov 2025 16:52:47 -0500 Subject: [PATCH 45/47] fix replacement --- tests/systemtests/mempool/test_tx_tracker.go | 24 +++++++++++++------- 1 file changed, 16 insertions(+), 8 deletions(-) diff --git a/tests/systemtests/mempool/test_tx_tracker.go b/tests/systemtests/mempool/test_tx_tracker.go index e526e55c8..6954dbf26 100644 --- a/tests/systemtests/mempool/test_tx_tracker.go +++ b/tests/systemtests/mempool/test_tx_tracker.go @@ -284,19 +284,27 @@ func RunTxTrackerWithReplacement(t *testing.T, base *suite.BaseTestSuite) { name: "tracks replacement transaction %s", actions: []func(*TestSuite, *TestContext){ func(s *TestSuite, ctx *TestContext) { - // Submit initial transaction - tx1, err := s.SendTx(t, s.Node(0), "acc0", 0, s.GasPriceMultiplier(10), nil) + // Submit initial transaction with a future nonce to keep it queued + _, err := s.SendTx(t, s.Node(0), "acc0", 1, s.GasPriceMultiplier(10), nil) require.NoError(t, err, "failed to send tx1") - ctx.SetExpPendingTxs(tx1) + // Submit replacement transaction with higher gas price (same nonce) + // Both transactions are sent in the same action to avoid race conditions + // where the first tx gets committed before the replacement arrives + tx2, err := s.SendTx(t, s.Node(0), "acc0", 1, s.GasPriceMultiplier(15), nil) + require.NoError(t, err, "failed to send replacement tx2") + + // The replacement transaction should replace tx1 in the queued state + ctx.SetExpQueuedTxs(tx2) }, func(s *TestSuite, ctx *TestContext) { - // Submit replacement transaction with higher gas price - tx2, err := s.SendTx(t, s.Node(0), "acc0", 0, s.GasPriceMultiplier(15), nil) - require.NoError(t, err, "failed to send replacement tx2") + // Submit transaction with nonce 0 to promote the queued replacement + tx0, err := s.SendTx(t, s.Node(0), "acc0", 0, s.GasPriceMultiplier(10), nil) + require.NoError(t, err, "failed to send tx0") - // The replacement transaction should replace tx1 - ctx.SetExpPendingTxs(tx2) + // tx0 should be pending, and tx2 should be promoted from queued to pending + ctx.SetExpPendingTxs(tx0) + ctx.PromoteExpTxs(1) }, }, }, From e2df42eb6d38561a50a02fa2686b2ee7af9eeed7 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Fri, 14 Nov 2025 09:59:12 -0500 Subject: [PATCH 46/47] fix typo --- tests/integration/mempool/test_tx_tracker.go | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/integration/mempool/test_tx_tracker.go b/tests/integration/mempool/test_tx_tracker.go index 7f09eb6c8..a75041279 100644 --- a/tests/integration/mempool/test_tx_tracker.go +++ b/tests/integration/mempool/test_tx_tracker.go @@ -113,8 +113,8 @@ func (s *IntegrationTestSuite) TestTxTrackerTrackLocalTxs() { s.Require().True(ok) evmmp.TrackLocalTxs(txs) - guage := metrics.GetOrRegisterGauge("txpool/local", nil) - s.Require().Equal(int64(len(txs)), guage.Snapshot().Value(), "should have tracked one transaction") + gauge := metrics.GetOrRegisterGauge("txpool/local", nil) + s.Require().Equal(int64(len(txs)), gauge.Snapshot().Value(), "should have tracked one transaction") }, }, { @@ -138,8 +138,8 @@ func (s *IntegrationTestSuite) TestTxTrackerTrackLocalTxs() { s.Require().True(ok) evmmp.TrackLocalTxs(txs) - guage := metrics.GetOrRegisterGauge("txpool/local", nil) - s.Require().Equal(int64(len(txs)), guage.Snapshot().Value(), "should have tracked three transactions") + gauge := metrics.GetOrRegisterGauge("txpool/local", nil) + s.Require().Equal(int64(len(txs)), gauge.Snapshot().Value(), "should have tracked three transactions") }, }, { @@ -163,8 +163,8 @@ func (s *IntegrationTestSuite) TestTxTrackerTrackLocalTxs() { s.Require().True(ok) evmmp.TrackLocalTxs(txs) - guage := metrics.GetOrRegisterGauge("txpool/local", nil) - s.Require().Equal(int64(len(txs)), guage.Snapshot().Value(), "should have tracked three transactions from different accounts") + gauge := metrics.GetOrRegisterGauge("txpool/local", nil) + s.Require().Equal(int64(len(txs)), gauge.Snapshot().Value(), "should have tracked three transactions from different accounts") }, }, } @@ -219,8 +219,8 @@ func (s *IntegrationTestSuite) TestTxTrackerResubmission() { txPool := evmmp.GetTxPool() s.Require().True(txPool.Has(ethTxs[0].Hash()), "first transaction should be in pool") - guage := metrics.GetOrRegisterGauge("txpool/local", nil) - s.Require().Equal(int64(len(ethTxs)), guage.Snapshot().Value(), "all transactions should be tracked") + gauge := metrics.GetOrRegisterGauge("txpool/local", nil) + s.Require().Equal(int64(len(ethTxs)), gauge.Snapshot().Value(), "all transactions should be tracked") // it is not practical to wait for recheck and test the tracker state }, }, @@ -260,8 +260,8 @@ func (s *IntegrationTestSuite) TestTxTrackerResubmission() { s.Require().True(txPool.Has(ethTx.Hash()), "transaction should be in pool") } - guage := metrics.GetOrRegisterGauge("txpool/local", nil) - s.Require().Equal(int64(len(ethTxs)), guage.Snapshot().Value(), "transactions should still be in the tracker") + gauge := metrics.GetOrRegisterGauge("txpool/local", nil) + s.Require().Equal(int64(len(ethTxs)), gauge.Snapshot().Value(), "transactions should still be in the tracker") // it is not practical to wait for recheck and test the tracker state }, }, From 9a2b4c68aa07d24a48d42efa51b1791cdfe39dd0 Mon Sep 17 00:00:00 2001 From: Abdul Malek Date: Fri, 14 Nov 2025 10:35:10 -0500 Subject: [PATCH 47/47] add mutex --- mempool/txpool/locals/journal.go | 22 +++++++++++++++++++++- 1 file changed, 21 insertions(+), 1 deletion(-) diff --git a/mempool/txpool/locals/journal.go b/mempool/txpool/locals/journal.go index ffd7257f1..dcff933a1 100644 --- a/mempool/txpool/locals/journal.go +++ b/mempool/txpool/locals/journal.go @@ -21,6 +21,7 @@ import ( "io" "io/fs" "os" + "sync" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/types" @@ -46,6 +47,7 @@ func (*devNull) Close() error { return nil } type journal struct { path string // Filesystem path to store the transactions at writer io.WriteCloser // Output stream to write new transactions into + mu sync.Mutex // Mutex to protect concurrent access to writer } // newTxJournal creates a new transaction journal to @@ -70,8 +72,14 @@ func (journal *journal) load(add func([]*types.Transaction) []error) error { defer input.Close() // Temporarily discard any journal additions (don't double add on load) + journal.mu.Lock() journal.writer = new(devNull) - defer func() { journal.writer = nil }() + journal.mu.Unlock() + defer func() { + journal.mu.Lock() + journal.writer = nil + journal.mu.Unlock() + }() // Inject all transactions from the journal into the pool stream := rlp.NewStream(input, 0) @@ -119,6 +127,9 @@ func (journal *journal) load(add func([]*types.Transaction) []error) error { // setupWriter opens the journal file for writing in append mode func (journal *journal) setupWriter() error { + journal.mu.Lock() + defer journal.mu.Unlock() + if journal.writer != nil { if err := journal.writer.Close(); err != nil { return err @@ -139,6 +150,9 @@ func (journal *journal) setupWriter() error { // insert adds the specified transaction to the local disk journal. func (journal *journal) insert(tx *types.Transaction) error { + journal.mu.Lock() + defer journal.mu.Unlock() + if journal.writer == nil { return errNoActiveJournal } @@ -151,6 +165,9 @@ func (journal *journal) insert(tx *types.Transaction) error { // rotate regenerates the transaction journal based on the current contents of // the transaction pool. func (journal *journal) rotate(all map[common.Address]types.Transactions) error { + journal.mu.Lock() + defer journal.mu.Unlock() + // Close the current journal (if any is open) if journal.writer != nil { if err := journal.writer.Close(); err != nil { @@ -200,6 +217,9 @@ func (journal *journal) rotate(all map[common.Address]types.Transactions) error // close flushes the transaction journal contents to disk and closes the file. func (journal *journal) close() error { + journal.mu.Lock() + defer journal.mu.Unlock() + var err error if journal.writer != nil { err = journal.writer.Close()