diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go
index 692970477..de4e58a3b 100644
--- a/cmd/utils/flags.go
+++ b/cmd/utils/flags.go
@@ -41,7 +41,7 @@ import (
"github.com/ethereum/go-ethereum/common/hexutil"
"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"
@@ -334,18 +334,18 @@ var (
TxPoolJournalFlag = &cli.StringFlag{
Name: "txpool.journal",
Usage: "Disk journal for local transaction to survive node restarts",
- Value: txpool.DefaultConfig.Journal,
+ Value: ethconfig.Defaults.TxPool.Journal,
Category: flags.TxPoolCategory,
}
TxPoolRejournalFlag = &cli.DurationFlag{
Name: "txpool.rejournal",
Usage: "Time interval to regenerate the local transaction journal",
- Value: txpool.DefaultConfig.Rejournal,
+ Value: ethconfig.Defaults.TxPool.Rejournal,
Category: flags.TxPoolCategory,
}
TxPoolPriceLimitFlag = &cli.Uint64Flag{
Name: "txpool.pricelimit",
- Usage: "Minimum gas price limit to enforce for acceptance into the pool",
+ Usage: "Minimum gas price tip to enforce for acceptance into the pool",
Value: ethconfig.Defaults.TxPool.PriceLimit,
Category: flags.TxPoolCategory,
}
@@ -385,7 +385,6 @@ var (
Value: ethconfig.Defaults.TxPool.Lifetime,
Category: flags.TxPoolCategory,
}
-
// Performance tuning settings
CacheFlag = &cli.IntFlag{
Name: "cache",
@@ -1500,7 +1499,7 @@ func setGPO(ctx *cli.Context, cfg *gasprice.Config, light bool) {
}
}
-func setTxPool(ctx *cli.Context, cfg *txpool.Config) {
+func setTxPool(ctx *cli.Context, cfg *legacypool.Config) {
if ctx.IsSet(TxPoolLocalsFlag.Name) {
locals := strings.Split(ctx.String(TxPoolLocalsFlag.Name), ",")
for _, account := range locals {
diff --git a/core/txpool/errors.go b/core/txpool/errors.go
new file mode 100644
index 000000000..b8c1c914a
--- /dev/null
+++ b/core/txpool/errors.go
@@ -0,0 +1,53 @@
+// Copyright 2014 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 txpool
+
+import "errors"
+
+var (
+ // ErrAlreadyKnown is returned if the transactions is already contained
+ // within the pool.
+ ErrAlreadyKnown = errors.New("already known")
+
+ // ErrInvalidSender is returned if the transaction contains an invalid signature.
+ ErrInvalidSender = errors.New("invalid sender")
+
+ // ErrUnderpriced is returned if a transaction's gas price is below the minimum
+ // configured for the transaction pool.
+ ErrUnderpriced = errors.New("transaction underpriced")
+
+ // ErrReplaceUnderpriced is returned if a transaction is attempted to be replaced
+ // with a different one without the required price bump.
+ ErrReplaceUnderpriced = errors.New("replacement transaction underpriced")
+
+ // ErrGasLimit is returned if a transaction's requested gas limit exceeds the
+ // maximum allowance of the current block.
+ ErrGasLimit = errors.New("exceeds block gas limit")
+
+ // ErrNegativeValue is a sanity error to ensure no one is able to specify a
+ // transaction with a negative value.
+ ErrNegativeValue = errors.New("negative value")
+
+ // ErrOversizedData is returned if the input data of a transaction is greater
+ // than some meaningful limit a user might use. This is not a consensus error
+ // making the transaction invalid, rather a DOS protection.
+ ErrOversizedData = errors.New("oversized data")
+
+ // ErrFutureReplacePending is returned if a future transaction replaces a pending
+ // transaction. Future transactions should only be able to replace other future transactions.
+ ErrFutureReplacePending = errors.New("future transaction tries to replace pending")
+)
diff --git a/core/txpool/journal.go b/core/txpool/legacypool/journal.go
similarity index 99%
rename from core/txpool/journal.go
rename to core/txpool/legacypool/journal.go
index 1b330b0c3..f04ab8fc1 100644
--- a/core/txpool/journal.go
+++ b/core/txpool/legacypool/journal.go
@@ -14,7 +14,7 @@
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see .
-package txpool
+package legacypool
import (
"errors"
diff --git a/core/txpool/legacypool/legacypool.go b/core/txpool/legacypool/legacypool.go
new file mode 100644
index 000000000..3525747c7
--- /dev/null
+++ b/core/txpool/legacypool/legacypool.go
@@ -0,0 +1,1847 @@
+// Copyright 2014 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 legacypool implements the normal EVM execution transaction pool.
+package legacypool
+
+import (
+ "errors"
+ "math"
+ "math/big"
+ "sort"
+ "sync"
+ "sync/atomic"
+ "time"
+
+ "github.com/ethereum/go-ethereum/common"
+ "github.com/ethereum/go-ethereum/common/prque"
+ "github.com/ethereum/go-ethereum/consensus/misc"
+ "github.com/ethereum/go-ethereum/core"
+ "github.com/ethereum/go-ethereum/core/state"
+ "github.com/ethereum/go-ethereum/core/txpool"
+ "github.com/ethereum/go-ethereum/core/types"
+ "github.com/ethereum/go-ethereum/event"
+ "github.com/ethereum/go-ethereum/log"
+ "github.com/ethereum/go-ethereum/metrics"
+ "github.com/ethereum/go-ethereum/params"
+)
+
+const (
+ // txSlotSize is used to calculate how many data slots a single transaction
+ // takes up based on its size. The slots are used as DoS protection, ensuring
+ // that validating a new transaction remains a constant operation (in reality
+ // O(maxslots), where max slots are 4 currently).
+ txSlotSize = 32 * 1024
+
+ // txMaxSize is the maximum size a single transaction can have. This field has
+ // non-trivial consequences: larger transactions are significantly harder and
+ // more expensive to propagate; larger transactions also take more resources
+ // to validate whether they fit into the pool or not.
+ txMaxSize = 4 * txSlotSize // 128KB
+)
+
+var (
+ // ErrAlreadyKnown is returned if the transactions is already contained
+ // within the pool.
+ ErrAlreadyKnown = errors.New("already known")
+
+ // ErrTxPoolOverflow is returned if the transaction pool is full and can't accept
+ // another remote transaction.
+ ErrTxPoolOverflow = errors.New("txpool is full")
+)
+
+var (
+ evictionInterval = time.Minute // Time interval to check for evictable transactions
+ statsReportInterval = 8 * time.Second // Time interval to report transaction pool stats
+)
+
+var (
+ // Metrics for the pending pool
+ pendingDiscardMeter = metrics.NewRegisteredMeter("txpool/pending/discard", nil)
+ pendingReplaceMeter = metrics.NewRegisteredMeter("txpool/pending/replace", nil)
+ pendingRateLimitMeter = metrics.NewRegisteredMeter("txpool/pending/ratelimit", nil) // Dropped due to rate limiting
+ pendingNofundsMeter = metrics.NewRegisteredMeter("txpool/pending/nofunds", nil) // Dropped due to out-of-funds
+
+ // Metrics for the queued pool
+ queuedDiscardMeter = metrics.NewRegisteredMeter("txpool/queued/discard", nil)
+ queuedReplaceMeter = metrics.NewRegisteredMeter("txpool/queued/replace", nil)
+ queuedRateLimitMeter = metrics.NewRegisteredMeter("txpool/queued/ratelimit", nil) // Dropped due to rate limiting
+ queuedNofundsMeter = metrics.NewRegisteredMeter("txpool/queued/nofunds", nil) // Dropped due to out-of-funds
+ queuedEvictionMeter = metrics.NewRegisteredMeter("txpool/queued/eviction", nil) // Dropped due to lifetime
+
+ // General tx metrics
+ knownTxMeter = metrics.NewRegisteredMeter("txpool/known", nil)
+ validTxMeter = metrics.NewRegisteredMeter("txpool/valid", nil)
+ invalidTxMeter = metrics.NewRegisteredMeter("txpool/invalid", nil)
+ underpricedTxMeter = metrics.NewRegisteredMeter("txpool/underpriced", nil)
+ overflowedTxMeter = metrics.NewRegisteredMeter("txpool/overflowed", nil)
+
+ // throttleTxMeter counts how many transactions are rejected due to too-many-changes between
+ // txpool reorgs.
+ throttleTxMeter = metrics.NewRegisteredMeter("txpool/throttle", nil)
+ // reorgDurationTimer measures how long time a txpool reorg takes.
+ reorgDurationTimer = metrics.NewRegisteredTimer("txpool/reorgtime", nil)
+ // dropBetweenReorgHistogram counts how many drops we experience between two reorg runs. It is expected
+ // that this number is pretty low, since txpool reorgs happen very frequently.
+ dropBetweenReorgHistogram = metrics.NewRegisteredHistogram("txpool/dropbetweenreorg", nil, metrics.NewExpDecaySample(1028, 0.015))
+
+ pendingGauge = metrics.NewRegisteredGauge("txpool/pending", nil)
+ queuedGauge = metrics.NewRegisteredGauge("txpool/queued", nil)
+ localGauge = metrics.NewRegisteredGauge("txpool/local", nil)
+ slotsGauge = metrics.NewRegisteredGauge("txpool/slots", nil)
+
+ reheapTimer = metrics.NewRegisteredTimer("txpool/reheap", nil)
+)
+
+// BlockChain defines the minimal set of methods needed to back a tx pool with
+// a chain. Exists to allow mocking the live chain out of tests.
+type BlockChain interface {
+ // Config retrieves the chain's fork configuration.
+ Config() *params.ChainConfig
+
+ // CurrentBlock returns the current head of the chain.
+ CurrentBlock() *types.Header
+
+ // GetBlock retrieves a specific block, used during pool resets.
+ GetBlock(hash common.Hash, number uint64) *types.Block
+
+ // StateAt returns a state database for a given root hash (generally the head).
+ StateAt(root common.Hash) (*state.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.Rejournal < time.Second {
+ log.Warn("Sanitizing invalid txpool journal time", "provided", conf.Rejournal, "updated", time.Second)
+ conf.Rejournal = time.Second
+ }
+ 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
+ }
+ 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
+}
+
+// LegacyPool contains all currently known transactions. Transactions
+// enter the pool when they are received from the network or submitted
+// locally. They exit the pool when they are included in the blockchain.
+//
+// The pool separates processable transactions (which can be applied to the
+// current state) and future transactions. Transactions move between those
+// two states over time as they are received and processed.
+type LegacyPool struct {
+ config Config
+ chainconfig *params.ChainConfig
+ chain BlockChain
+ gasTip atomic.Pointer[big.Int]
+ txFeed event.Feed
+ scope event.SubscriptionScope
+ signer types.Signer
+ mu sync.RWMutex
+
+ currentHead atomic.Pointer[types.Header] // Current head of the blockchain
+ currentState *state.StateDB // Current state in the blockchain head
+ pendingNonces *noncer // Pending state tracking virtual nonces
+
+ locals *accountSet // Set of local transaction to exempt from eviction rules
+ journal *journal // Journal of local transaction to back up to disk
+
+ pending map[common.Address]*list // All currently processable transactions
+ queue map[common.Address]*list // Queued but non-processable transactions
+ beats map[common.Address]time.Time // Last heartbeat from each known account
+ all *lookup // All transactions to allow lookups
+ priced *pricedList // All transactions sorted by price
+
+ reqResetCh chan *txpoolResetRequest
+ reqPromoteCh chan *accountSet
+ queueTxEventCh chan *types.Transaction
+ reorgDoneCh chan chan struct{}
+ reorgShutdownCh chan struct{} // requests shutdown of scheduleReorgLoop
+ wg sync.WaitGroup // tracks loop, scheduleReorgLoop
+ initDoneCh chan struct{} // is closed once the pool is initialized (for tests)
+
+ changesSinceReorg int // A counter for how many drops we've performed in-between reorg.
+}
+
+type txpoolResetRequest struct {
+ oldHead, newHead *types.Header
+}
+
+// New creates a new transaction pool to gather, sort and filter inbound
+// transactions from the network.
+func New(config Config, chain BlockChain) *LegacyPool {
+ // Sanitize the input to ensure no vulnerable gas prices are set
+ config = (&config).sanitize()
+
+ // Create the transaction pool with its initial settings
+ pool := &LegacyPool{
+ config: config,
+ chain: chain,
+ chainconfig: chain.Config(),
+ signer: types.LatestSigner(chain.Config()),
+ pending: make(map[common.Address]*list),
+ queue: make(map[common.Address]*list),
+ beats: make(map[common.Address]time.Time),
+ all: newLookup(),
+ reqResetCh: make(chan *txpoolResetRequest),
+ reqPromoteCh: make(chan *accountSet),
+ queueTxEventCh: make(chan *types.Transaction),
+ reorgDoneCh: make(chan chan struct{}),
+ reorgShutdownCh: make(chan struct{}),
+ initDoneCh: make(chan struct{}),
+ }
+ pool.locals = newAccountSet(pool.signer)
+ for _, addr := range config.Locals {
+ log.Info("Setting new local account", "address", addr)
+ pool.locals.add(addr)
+ }
+ pool.priced = newPricedList(pool.all)
+
+ if !config.NoLocals && config.Journal != "" {
+ pool.journal = newTxJournal(config.Journal)
+ }
+ return pool
+}
+
+// Filter returns whether the given transaction can be consumed by the legacy
+// pool, specifically, whether it is a Legacy, AccessList or Dynamic transaction.
+func (pool *LegacyPool) Filter(tx *types.Transaction) bool {
+ switch tx.Type() {
+ case types.LegacyTxType, types.AccessListTxType, types.DynamicFeeTxType:
+ return true
+ default:
+ return false
+ }
+}
+
+// Init sets the gas price needed to keep a transaction in the pool and the chain
+// head to allow balance / nonce checks. The transaction journal will be loaded
+// from disk and filtered based on the provided starting settings. The internal
+// goroutines will be spun up and the pool deemed operational afterwards.
+func (pool *LegacyPool) Init(gasTip *big.Int, head *types.Header) error {
+ // Set the basic pool parameters
+ pool.gasTip.Store(gasTip)
+ pool.reset(nil, head)
+
+ // Start the reorg loop early, so it can handle requests generated during
+ // journal loading.
+ pool.wg.Add(1)
+ go pool.scheduleReorgLoop()
+
+ // If local transactions and journaling is enabled, load from disk
+ if pool.journal != nil {
+ if err := pool.journal.load(pool.addLocals); err != nil {
+ log.Warn("Failed to load transaction journal", "err", err)
+ }
+ if err := pool.journal.rotate(pool.local()); err != nil {
+ log.Warn("Failed to rotate transaction journal", "err", err)
+ }
+ }
+ pool.wg.Add(1)
+ go pool.loop()
+ return nil
+}
+
+// loop is the transaction pool's main event loop, waiting for and reacting to
+// outside blockchain events as well as for various reporting and transaction
+// eviction events.
+func (pool *LegacyPool) loop() {
+ defer pool.wg.Done()
+
+ var (
+ prevPending, prevQueued, prevStales int
+
+ // Start the stats reporting and transaction eviction tickers
+ report = time.NewTicker(statsReportInterval)
+ evict = time.NewTicker(evictionInterval)
+ journal = time.NewTicker(pool.config.Rejournal)
+ )
+ defer report.Stop()
+ defer evict.Stop()
+ defer journal.Stop()
+
+ // Notify tests that the init phase is done
+ close(pool.initDoneCh)
+ for {
+ select {
+ // Handle pool shutdown
+ case <-pool.reorgShutdownCh:
+ return
+
+ // Handle stats reporting ticks
+ case <-report.C:
+ pool.mu.RLock()
+ pending, queued := pool.stats()
+ pool.mu.RUnlock()
+ stales := int(pool.priced.stales.Load())
+
+ if pending != prevPending || queued != prevQueued || stales != prevStales {
+ log.Debug("Transaction pool status report", "executable", pending, "queued", queued, "stales", stales)
+ prevPending, prevQueued, prevStales = pending, queued, stales
+ }
+
+ // Handle inactive account transaction eviction
+ case <-evict.C:
+ pool.mu.Lock()
+ for addr := range pool.queue {
+ // Skip local transactions from the eviction mechanism
+ if pool.locals.contains(addr) {
+ continue
+ }
+ // Any non-locals old enough should be removed
+ if time.Since(pool.beats[addr]) > pool.config.Lifetime {
+ list := pool.queue[addr].Flatten()
+ for _, tx := range list {
+ pool.removeTx(tx.Hash(), true)
+ }
+ queuedEvictionMeter.Mark(int64(len(list)))
+ }
+ }
+ pool.mu.Unlock()
+
+ // Handle local transaction journal rotation
+ case <-journal.C:
+ if pool.journal != nil {
+ pool.mu.Lock()
+ if err := pool.journal.rotate(pool.local()); err != nil {
+ log.Warn("Failed to rotate local tx journal", "err", err)
+ }
+ pool.mu.Unlock()
+ }
+ }
+ }
+}
+
+// Close terminates the transaction pool.
+func (pool *LegacyPool) Close() error {
+ // Unsubscribe all subscriptions registered from txpool
+ pool.scope.Close()
+
+ // Terminate the pool reorger and return
+ close(pool.reorgShutdownCh)
+ pool.wg.Wait()
+
+ if pool.journal != nil {
+ pool.journal.close()
+ }
+ log.Info("Transaction pool stopped")
+ return nil
+}
+
+// Reset implements txpool.SubPool, allowing the legacy pool's internal state to be
+// kept in sync with the main transacion pool's internal state.
+func (pool *LegacyPool) Reset(oldHead, newHead *types.Header) {
+ wait := pool.requestReset(oldHead, newHead)
+ <-wait
+}
+
+// SubscribeTransactions registers a subscription of NewTxsEvent and
+// starts sending event to the given channel.
+func (pool *LegacyPool) SubscribeTransactions(ch chan<- core.NewTxsEvent) event.Subscription {
+ return pool.scope.Track(pool.txFeed.Subscribe(ch))
+}
+
+// SetGasTip updates the minimum gas tip required by the transaction pool for a
+// new transaction, and drops all transactions below this threshold.
+func (pool *LegacyPool) SetGasTip(tip *big.Int) {
+ pool.mu.Lock()
+ defer pool.mu.Unlock()
+
+ old := pool.gasTip.Load()
+ pool.gasTip.Store(new(big.Int).Set(tip))
+
+ // If the min miner fee increased, remove transactions below the new threshold
+ if tip.Cmp(old) > 0 {
+ // pool.priced is sorted by GasFeeCap, so we have to iterate through pool.all instead
+ drop := pool.all.RemotesBelowTip(tip)
+ for _, tx := range drop {
+ pool.removeTx(tx.Hash(), false)
+ }
+ pool.priced.Removed(len(drop))
+ }
+ log.Info("LEgacy pool tip threshold updated", "tip", tip)
+}
+
+// Nonce returns the next nonce of an account, with all transactions executable
+// by the pool already applied on top.
+func (pool *LegacyPool) Nonce(addr common.Address) uint64 {
+ pool.mu.RLock()
+ defer pool.mu.RUnlock()
+
+ return pool.pendingNonces.get(addr)
+}
+
+// Stats retrieves the current pool stats, namely the number of pending and the
+// number of queued (non-executable) transactions.
+func (pool *LegacyPool) Stats() (int, int) {
+ pool.mu.RLock()
+ defer pool.mu.RUnlock()
+
+ return pool.stats()
+}
+
+// stats retrieves the current pool stats, namely the number of pending and the
+// number of queued (non-executable) transactions.
+func (pool *LegacyPool) stats() (int, int) {
+ pending := 0
+ for _, list := range pool.pending {
+ pending += list.Len()
+ }
+ queued := 0
+ for _, list := range pool.queue {
+ queued += list.Len()
+ }
+ return pending, queued
+}
+
+// Content retrieves the data content of the transaction pool, returning all the
+// pending as well as queued transactions, grouped by account and sorted by nonce.
+func (pool *LegacyPool) Content() (map[common.Address][]*types.Transaction, map[common.Address][]*types.Transaction) {
+ pool.mu.Lock()
+ defer pool.mu.Unlock()
+
+ pending := make(map[common.Address][]*types.Transaction, len(pool.pending))
+ for addr, list := range pool.pending {
+ pending[addr] = list.Flatten()
+ }
+ queued := make(map[common.Address][]*types.Transaction, len(pool.queue))
+ for addr, list := range pool.queue {
+ queued[addr] = list.Flatten()
+ }
+ return pending, queued
+}
+
+// ContentFrom retrieves the data content of the transaction pool, returning the
+// pending as well as queued transactions of this address, grouped by nonce.
+func (pool *LegacyPool) ContentFrom(addr common.Address) ([]*types.Transaction, []*types.Transaction) {
+ pool.mu.RLock()
+ defer pool.mu.RUnlock()
+
+ var pending []*types.Transaction
+ if list, ok := pool.pending[addr]; ok {
+ pending = list.Flatten()
+ }
+ var queued []*types.Transaction
+ if list, ok := pool.queue[addr]; ok {
+ queued = list.Flatten()
+ }
+ return pending, queued
+}
+
+// Pending retrieves all currently processable transactions, grouped by origin
+// account and sorted by nonce. The returned transaction set is a copy and can be
+// freely modified by calling code.
+//
+// The enforceTips parameter can be used to do an extra filtering on the pending
+// transactions and only return those whose **effective** tip is large enough in
+// the next pending execution environment.
+func (pool *LegacyPool) Pending(enforceTips bool) map[common.Address][]*types.Transaction {
+ pool.mu.Lock()
+ defer pool.mu.Unlock()
+
+ pending := make(map[common.Address][]*types.Transaction, len(pool.pending))
+ for addr, list := range pool.pending {
+ txs := list.Flatten()
+
+ // If the miner requests tip enforcement, cap the lists now
+ if enforceTips && !pool.locals.contains(addr) {
+ for i, tx := range txs {
+ if tx.EffectiveGasTipIntCmp(pool.gasTip.Load(), pool.priced.urgent.baseFee) < 0 {
+ txs = txs[:i]
+ break
+ }
+ }
+ }
+ if len(txs) > 0 {
+ pending[addr] = txs
+ }
+ }
+ return pending
+}
+
+// Locals retrieves the accounts currently considered local by the pool.
+func (pool *LegacyPool) Locals() []common.Address {
+ pool.mu.Lock()
+ defer pool.mu.Unlock()
+
+ return pool.locals.flatten()
+}
+
+// local retrieves all currently known local transactions, grouped by origin
+// account and sorted by nonce. The returned transaction set is a copy and can be
+// freely modified by calling code.
+func (pool *LegacyPool) local() map[common.Address]types.Transactions {
+ txs := make(map[common.Address]types.Transactions)
+ for addr := range pool.locals.accounts {
+ if pending := pool.pending[addr]; pending != nil {
+ txs[addr] = append(txs[addr], pending.Flatten()...)
+ }
+ if queued := pool.queue[addr]; queued != nil {
+ txs[addr] = append(txs[addr], queued.Flatten()...)
+ }
+ }
+ return txs
+}
+
+// validateTxBasics checks whether a transaction is valid according to the consensus
+// rules, but does not check state-dependent validation such as sufficient balance.
+// This check is meant as an early check which only needs to be performed once,
+// and does not require the pool mutex to be held.
+func (pool *LegacyPool) validateTxBasics(tx *types.Transaction, local bool) error {
+ opts := &txpool.ValidationOptions{
+ Config: pool.chainconfig,
+ Accept: 0 |
+ 1< pool.config.GlobalSlots+pool.config.GlobalQueue {
+ // If the new transaction is underpriced, don't accept it
+ if !isLocal && pool.priced.Underpriced(tx) {
+ log.Trace("Discarding underpriced transaction", "hash", hash, "gasTipCap", tx.GasTipCap(), "gasFeeCap", tx.GasFeeCap())
+ underpricedTxMeter.Mark(1)
+ return false, txpool.ErrUnderpriced
+ }
+
+ // We're about to replace a transaction. The reorg does a more thorough
+ // analysis of what to remove and how, but it runs async. We don't want to
+ // do too many replacements between reorg-runs, so we cap the number of
+ // replacements to 25% of the slots
+ if pool.changesSinceReorg > int(pool.config.GlobalSlots/4) {
+ throttleTxMeter.Mark(1)
+ return false, ErrTxPoolOverflow
+ }
+
+ // New transaction is better than our worse ones, make room for it.
+ // If it's a local transaction, forcibly discard all available transactions.
+ // Otherwise if we can't make enough room for new one, abort the operation.
+ drop, success := pool.priced.Discard(pool.all.Slots()-int(pool.config.GlobalSlots+pool.config.GlobalQueue)+numSlots(tx), isLocal)
+
+ // Special case, we still can't make the room for the new remote one.
+ if !isLocal && !success {
+ log.Trace("Discarding overflown transaction", "hash", hash)
+ overflowedTxMeter.Mark(1)
+ return false, ErrTxPoolOverflow
+ }
+
+ // If the new transaction is a future transaction it should never churn pending transactions
+ if !isLocal && pool.isGapped(from, tx) {
+ var replacesPending bool
+ for _, dropTx := range drop {
+ dropSender, _ := types.Sender(pool.signer, dropTx)
+ if list := pool.pending[dropSender]; list != nil && list.Contains(dropTx.Nonce()) {
+ replacesPending = true
+ break
+ }
+ }
+ // Add all transactions back to the priced queue
+ if replacesPending {
+ for _, dropTx := range drop {
+ pool.priced.Put(dropTx, false)
+ }
+ log.Trace("Discarding future transaction replacing pending tx", "hash", hash)
+ return false, txpool.ErrFutureReplacePending
+ }
+ }
+
+ // Kick out the underpriced remote transactions.
+ for _, tx := range drop {
+ log.Trace("Discarding freshly underpriced transaction", "hash", tx.Hash(), "gasTipCap", tx.GasTipCap(), "gasFeeCap", tx.GasFeeCap())
+ underpricedTxMeter.Mark(1)
+ dropped := pool.removeTx(tx.Hash(), false)
+ pool.changesSinceReorg += dropped
+ }
+ }
+
+ // Try to replace an existing transaction in the pending pool
+ if list := pool.pending[from]; list != nil && list.Contains(tx.Nonce()) {
+ // Nonce already pending, check if required price bump is met
+ inserted, old := list.Add(tx, pool.config.PriceBump)
+ if !inserted {
+ pendingDiscardMeter.Mark(1)
+ return false, txpool.ErrReplaceUnderpriced
+ }
+ // New transaction is better, replace old one
+ if old != nil {
+ pool.all.Remove(old.Hash())
+ pool.priced.Removed(1)
+ pendingReplaceMeter.Mark(1)
+ }
+ pool.all.Add(tx, isLocal)
+ pool.priced.Put(tx, isLocal)
+ pool.journalTx(from, tx)
+ pool.queueTxEvent(tx)
+ log.Trace("Pooled new executable transaction", "hash", hash, "from", from, "to", tx.To())
+
+ // Successful promotion, bump the heartbeat
+ pool.beats[from] = time.Now()
+ return old != nil, nil
+ }
+ // New transaction isn't replacing a pending one, push into queue
+ replaced, err = pool.enqueueTx(hash, tx, isLocal, true)
+ if err != nil {
+ return false, err
+ }
+ // Mark local addresses and journal local transactions
+ if local && !pool.locals.contains(from) {
+ log.Info("Setting new local account", "address", from)
+ pool.locals.add(from)
+ pool.priced.Removed(pool.all.RemoteToLocals(pool.locals)) // Migrate the remotes if it's marked as local first time.
+ }
+ if isLocal {
+ localGauge.Inc(1)
+ }
+ pool.journalTx(from, tx)
+
+ log.Trace("Pooled new future transaction", "hash", hash, "from", from, "to", tx.To())
+ return replaced, nil
+}
+
+// isGapped reports whether the given transaction is immediately executable.
+func (pool *LegacyPool) isGapped(from common.Address, tx *types.Transaction) bool {
+ // Short circuit if transaction falls within the scope of the pending list
+ // or matches the next pending nonce which can be promoted as an executable
+ // transaction afterwards. Note, the tx staleness is already checked in
+ // 'validateTx' function previously.
+ next := pool.pendingNonces.get(from)
+ if tx.Nonce() <= next {
+ return false
+ }
+ // The transaction has a nonce gap with pending list, it's only considered
+ // as executable if transactions in queue can fill up the nonce gap.
+ queue, ok := pool.queue[from]
+ if !ok {
+ return true
+ }
+ for nonce := next; nonce < tx.Nonce(); nonce++ {
+ if !queue.Contains(nonce) {
+ return true // txs in queue can't fill up the nonce gap
+ }
+ }
+ return false
+}
+
+// enqueueTx inserts a new transaction into the non-executable transaction queue.
+//
+// Note, this method assumes the pool lock is held!
+func (pool *LegacyPool) enqueueTx(hash common.Hash, tx *types.Transaction, local bool, addAll bool) (bool, error) {
+ // Try to insert the transaction into the future queue
+ from, _ := types.Sender(pool.signer, tx) // already validated
+ if pool.queue[from] == nil {
+ pool.queue[from] = newList(false)
+ }
+ inserted, old := pool.queue[from].Add(tx, pool.config.PriceBump)
+ if !inserted {
+ // An older transaction was better, discard this
+ queuedDiscardMeter.Mark(1)
+ return false, txpool.ErrReplaceUnderpriced
+ }
+ // Discard any previous transaction and mark this
+ if old != nil {
+ pool.all.Remove(old.Hash())
+ pool.priced.Removed(1)
+ queuedReplaceMeter.Mark(1)
+ } else {
+ // Nothing was replaced, bump the queued counter
+ queuedGauge.Inc(1)
+ }
+ // If the transaction isn't in lookup set but it's expected to be there,
+ // show the error log.
+ if pool.all.Get(hash) == nil && !addAll {
+ log.Error("Missing transaction in lookup set, please report the issue", "hash", hash)
+ }
+ if addAll {
+ pool.all.Add(tx, local)
+ pool.priced.Put(tx, local)
+ }
+ // If we never record the heartbeat, do it right now.
+ if _, exist := pool.beats[from]; !exist {
+ pool.beats[from] = time.Now()
+ }
+ return old != nil, nil
+}
+
+// journalTx adds the specified transaction to the local disk journal if it is
+// deemed to have been sent from a local account.
+func (pool *LegacyPool) journalTx(from common.Address, tx *types.Transaction) {
+ // Only journal if it's enabled and the transaction is local
+ if pool.journal == nil || !pool.locals.contains(from) {
+ return
+ }
+ if err := pool.journal.insert(tx); err != nil {
+ log.Warn("Failed to journal local transaction", "err", err)
+ }
+}
+
+// promoteTx adds a transaction to the pending (processable) list of transactions
+// and returns whether it was inserted or an older was better.
+//
+// Note, this method assumes the pool lock is held!
+func (pool *LegacyPool) promoteTx(addr common.Address, hash common.Hash, tx *types.Transaction) bool {
+ // Try to insert the transaction into the pending queue
+ if pool.pending[addr] == nil {
+ pool.pending[addr] = newList(true)
+ }
+ list := pool.pending[addr]
+
+ inserted, old := list.Add(tx, pool.config.PriceBump)
+ if !inserted {
+ // An older transaction was better, discard this
+ pool.all.Remove(hash)
+ pool.priced.Removed(1)
+ pendingDiscardMeter.Mark(1)
+ return false
+ }
+ // Otherwise discard any previous transaction and mark this
+ if old != nil {
+ pool.all.Remove(old.Hash())
+ pool.priced.Removed(1)
+ pendingReplaceMeter.Mark(1)
+ } else {
+ // Nothing was replaced, bump the pending counter
+ pendingGauge.Inc(1)
+ }
+ // Set the potentially new pending nonce and notify any subsystems of the new tx
+ pool.pendingNonces.set(addr, tx.Nonce()+1)
+
+ // Successful promotion, bump the heartbeat
+ pool.beats[addr] = time.Now()
+ return true
+}
+
+// Add enqueues a batch of transactions into the pool if they are valid. Depending
+// on the local flag, full pricing contraints will or will not be applied.
+//
+// If sync is set, the method will block until all internal maintenance related
+// to the add is finished. Only use this during tests for determinism!
+func (pool *LegacyPool) Add(txs []*txpool.Transaction, local bool, sync bool) []error {
+ unwrapped := make([]*types.Transaction, len(txs))
+ for i, tx := range txs {
+ unwrapped[i] = tx.Tx
+ }
+ return pool.addTxs(unwrapped, local, sync)
+}
+
+// addLocals enqueues a batch of transactions into the pool if they are valid, marking the
+// senders as a local ones, ensuring they go around the local pricing constraints.
+//
+// This method is used to add transactions from the RPC API and performs synchronous pool
+// reorganization and event propagation.
+func (pool *LegacyPool) addLocals(txs []*types.Transaction) []error {
+ return pool.addTxs(txs, !pool.config.NoLocals, true)
+}
+
+// addLocal enqueues a single local transaction into the pool if it is valid. This is
+// a convenience wrapper around addLocals.
+func (pool *LegacyPool) addLocal(tx *types.Transaction) error {
+ errs := pool.addLocals([]*types.Transaction{tx})
+ return errs[0]
+}
+
+// addRemotes enqueues a batch of transactions into the pool if they are valid. If the
+// senders are not among the locally tracked ones, full pricing constraints will apply.
+//
+// This method is used to add transactions from the p2p network and does not wait for pool
+// reorganization and internal event propagation.
+func (pool *LegacyPool) addRemotes(txs []*types.Transaction) []error {
+ return pool.addTxs(txs, false, false)
+}
+
+// addRemote enqueues a single transaction into the pool if it is valid. This is a convenience
+// wrapper around addRemotes.
+func (pool *LegacyPool) addRemote(tx *types.Transaction) error {
+ errs := pool.addRemotes([]*types.Transaction{tx})
+ return errs[0]
+}
+
+// addRemotesSync is like addRemotes, but waits for pool reorganization. Tests use this method.
+func (pool *LegacyPool) addRemotesSync(txs []*types.Transaction) []error {
+ return pool.addTxs(txs, false, true)
+}
+
+// This is like addRemotes with a single transaction, but waits for pool reorganization. Tests use this method.
+func (pool *LegacyPool) addRemoteSync(tx *types.Transaction) error {
+ return pool.addTxs([]*types.Transaction{tx}, false, true)[0]
+}
+
+// addTxs attempts to queue a batch of transactions if they are valid.
+func (pool *LegacyPool) addTxs(txs []*types.Transaction, local, sync bool) []error {
+ // Filter out known ones without obtaining the pool lock or recovering signatures
+ var (
+ errs = make([]error, len(txs))
+ news = make([]*types.Transaction, 0, len(txs))
+ )
+ for i, tx := range txs {
+ // If the transaction is known, pre-set the error slot
+ if pool.all.Get(tx.Hash()) != nil {
+ errs[i] = ErrAlreadyKnown
+ knownTxMeter.Mark(1)
+ continue
+ }
+ // Exclude transactions with basic errors, e.g invalid signatures and
+ // insufficient intrinsic gas as soon as possible and cache senders
+ // in transactions before obtaining lock
+ if err := pool.validateTxBasics(tx, local); err != nil {
+ errs[i] = err
+ invalidTxMeter.Mark(1)
+ continue
+ }
+ // Accumulate all unknown transactions for deeper processing
+ news = append(news, tx)
+ }
+ if len(news) == 0 {
+ return errs
+ }
+
+ // Process all the new transaction and merge any errors into the original slice
+ pool.mu.Lock()
+ newErrs, dirtyAddrs := pool.addTxsLocked(news, local)
+ pool.mu.Unlock()
+
+ var nilSlot = 0
+ for _, err := range newErrs {
+ for errs[nilSlot] != nil {
+ nilSlot++
+ }
+ errs[nilSlot] = err
+ nilSlot++
+ }
+ // Reorg the pool internals if needed and return
+ done := pool.requestPromoteExecutables(dirtyAddrs)
+ if sync {
+ <-done
+ }
+ return errs
+}
+
+// addTxsLocked attempts to queue a batch of transactions if they are valid.
+// The transaction pool lock must be held.
+func (pool *LegacyPool) addTxsLocked(txs []*types.Transaction, local bool) ([]error, *accountSet) {
+ dirty := newAccountSet(pool.signer)
+ errs := make([]error, len(txs))
+ for i, tx := range txs {
+ replaced, err := pool.add(tx, local)
+ errs[i] = err
+ if err == nil && !replaced {
+ dirty.addTx(tx)
+ }
+ }
+ validTxMeter.Mark(int64(len(dirty.accounts)))
+ return errs, dirty
+}
+
+// Status returns the status (unknown/pending/queued) of a batch of transactions
+// identified by their hashes.
+func (pool *LegacyPool) Status(hash common.Hash) txpool.TxStatus {
+ tx := pool.get(hash)
+ if tx == nil {
+ return txpool.TxStatusUnknown
+ }
+ from, _ := types.Sender(pool.signer, tx) // already validated
+
+ pool.mu.RLock()
+ defer pool.mu.RUnlock()
+
+ if txList := pool.pending[from]; txList != nil && txList.txs.items[tx.Nonce()] != nil {
+ return txpool.TxStatusPending
+ } else if txList := pool.queue[from]; txList != nil && txList.txs.items[tx.Nonce()] != nil {
+ return txpool.TxStatusQueued
+ }
+ return txpool.TxStatusUnknown
+}
+
+// Get returns a transaction if it is contained in the pool and nil otherwise.
+func (pool *LegacyPool) Get(hash common.Hash) *txpool.Transaction {
+ tx := pool.get(hash)
+ if tx == nil {
+ return nil
+ }
+ return &txpool.Transaction{Tx: tx}
+}
+
+// get returns a transaction if it is contained in the pool and nil otherwise.
+func (pool *LegacyPool) get(hash common.Hash) *types.Transaction {
+ return pool.all.Get(hash)
+}
+
+// Has returns an indicator whether txpool has a transaction cached with the
+// given hash.
+func (pool *LegacyPool) Has(hash common.Hash) bool {
+ return pool.all.Get(hash) != nil
+}
+
+// removeTx removes a single transaction from the queue, moving all subsequent
+// transactions back to the future queue.
+// Returns the number of transactions removed from the pending queue.
+func (pool *LegacyPool) removeTx(hash common.Hash, outofbound bool) int {
+ // Fetch the transaction we wish to delete
+ tx := pool.all.Get(hash)
+ if tx == nil {
+ return 0
+ }
+ addr, _ := types.Sender(pool.signer, tx) // already validated during insertion
+
+ // Remove it from the list of known transactions
+ pool.all.Remove(hash)
+ if outofbound {
+ pool.priced.Removed(1)
+ }
+ if pool.locals.contains(addr) {
+ localGauge.Dec(1)
+ }
+ // Remove the transaction from the pending lists and reset the account nonce
+ if pending := pool.pending[addr]; pending != nil {
+ if removed, invalids := pending.Remove(tx); removed {
+ // If no more pending transactions are left, remove the list
+ if pending.Empty() {
+ delete(pool.pending, addr)
+ }
+ // Postpone any invalidated transactions
+ for _, tx := range invalids {
+ // Internal shuffle shouldn't touch the lookup set.
+ pool.enqueueTx(tx.Hash(), tx, false, false)
+ }
+ // Update the account nonce if needed
+ pool.pendingNonces.setIfLower(addr, tx.Nonce())
+ // Reduce the pending counter
+ pendingGauge.Dec(int64(1 + len(invalids)))
+ return 1 + len(invalids)
+ }
+ }
+ // Transaction is in the future queue
+ if future := pool.queue[addr]; future != nil {
+ if removed, _ := future.Remove(tx); removed {
+ // Reduce the queued counter
+ queuedGauge.Dec(1)
+ }
+ if future.Empty() {
+ delete(pool.queue, addr)
+ delete(pool.beats, addr)
+ }
+ }
+ return 0
+}
+
+// requestReset requests a pool reset to the new head block.
+// The returned channel is closed when the reset has occurred.
+func (pool *LegacyPool) requestReset(oldHead *types.Header, newHead *types.Header) chan struct{} {
+ select {
+ case pool.reqResetCh <- &txpoolResetRequest{oldHead, newHead}:
+ return <-pool.reorgDoneCh
+ case <-pool.reorgShutdownCh:
+ return pool.reorgShutdownCh
+ }
+}
+
+// requestPromoteExecutables requests transaction promotion checks for the given addresses.
+// The returned channel is closed when the promotion checks have occurred.
+func (pool *LegacyPool) requestPromoteExecutables(set *accountSet) chan struct{} {
+ select {
+ case pool.reqPromoteCh <- set:
+ return <-pool.reorgDoneCh
+ case <-pool.reorgShutdownCh:
+ return pool.reorgShutdownCh
+ }
+}
+
+// queueTxEvent enqueues a transaction event to be sent in the next reorg run.
+func (pool *LegacyPool) queueTxEvent(tx *types.Transaction) {
+ select {
+ case pool.queueTxEventCh <- tx:
+ case <-pool.reorgShutdownCh:
+ }
+}
+
+// scheduleReorgLoop schedules runs of reset and promoteExecutables. Code above should not
+// call those methods directly, but request them being run using requestReset and
+// requestPromoteExecutables instead.
+func (pool *LegacyPool) scheduleReorgLoop() {
+ defer pool.wg.Done()
+
+ var (
+ curDone chan struct{} // non-nil while runReorg is active
+ nextDone = make(chan struct{})
+ launchNextRun bool
+ reset *txpoolResetRequest
+ dirtyAccounts *accountSet
+ queuedEvents = make(map[common.Address]*sortedMap)
+ )
+ for {
+ // Launch next background reorg if needed
+ if curDone == nil && launchNextRun {
+ // Run the background reorg and announcements
+ go pool.runReorg(nextDone, reset, dirtyAccounts, queuedEvents)
+
+ // Prepare everything for the next round of reorg
+ curDone, nextDone = nextDone, make(chan struct{})
+ launchNextRun = false
+
+ reset, dirtyAccounts = nil, nil
+ queuedEvents = make(map[common.Address]*sortedMap)
+ }
+
+ select {
+ case req := <-pool.reqResetCh:
+ // Reset request: update head if request is already pending.
+ if reset == nil {
+ reset = req
+ } else {
+ reset.newHead = req.newHead
+ }
+ launchNextRun = true
+ pool.reorgDoneCh <- nextDone
+
+ case req := <-pool.reqPromoteCh:
+ // Promote request: update address set if request is already pending.
+ if dirtyAccounts == nil {
+ dirtyAccounts = req
+ } else {
+ dirtyAccounts.merge(req)
+ }
+ launchNextRun = true
+ pool.reorgDoneCh <- nextDone
+
+ case tx := <-pool.queueTxEventCh:
+ // Queue up the event, but don't schedule a reorg. It's up to the caller to
+ // request one later if they want the events sent.
+ addr, _ := types.Sender(pool.signer, tx)
+ if _, ok := queuedEvents[addr]; !ok {
+ queuedEvents[addr] = newSortedMap()
+ }
+ queuedEvents[addr].Put(tx)
+
+ case <-curDone:
+ curDone = nil
+
+ case <-pool.reorgShutdownCh:
+ // Wait for current run to finish.
+ if curDone != nil {
+ <-curDone
+ }
+ close(nextDone)
+ return
+ }
+ }
+}
+
+// runReorg runs reset and promoteExecutables on behalf of scheduleReorgLoop.
+func (pool *LegacyPool) runReorg(done chan struct{}, reset *txpoolResetRequest, dirtyAccounts *accountSet, events map[common.Address]*sortedMap) {
+ defer func(t0 time.Time) {
+ reorgDurationTimer.Update(time.Since(t0))
+ }(time.Now())
+ defer close(done)
+
+ var promoteAddrs []common.Address
+ if dirtyAccounts != nil && reset == nil {
+ // Only dirty accounts need to be promoted, unless we're resetting.
+ // For resets, all addresses in the tx queue will be promoted and
+ // the flatten operation can be avoided.
+ promoteAddrs = dirtyAccounts.flatten()
+ }
+ pool.mu.Lock()
+ if reset != nil {
+ // Reset from the old head to the new, rescheduling any reorged transactions
+ pool.reset(reset.oldHead, reset.newHead)
+
+ // Nonces were reset, discard any events that became stale
+ for addr := range events {
+ events[addr].Forward(pool.pendingNonces.get(addr))
+ if events[addr].Len() == 0 {
+ delete(events, addr)
+ }
+ }
+ // Reset needs promote for all addresses
+ promoteAddrs = make([]common.Address, 0, len(pool.queue))
+ for addr := range pool.queue {
+ promoteAddrs = append(promoteAddrs, addr)
+ }
+ }
+ // Check for pending transactions for every account that sent new ones
+ promoted := pool.promoteExecutables(promoteAddrs)
+
+ // If a new block appeared, validate the pool of pending transactions. This will
+ // remove any transaction that has been included in the block or was invalidated
+ // because of another transaction (e.g. higher gas price).
+ if reset != nil {
+ pool.demoteUnexecutables()
+ if reset.newHead != nil && pool.chainconfig.IsLondon(new(big.Int).Add(reset.newHead.Number, big.NewInt(1))) {
+ pendingBaseFee := misc.CalcBaseFee(pool.chainconfig, reset.newHead)
+ pool.priced.SetBaseFee(pendingBaseFee)
+ }
+ // Update all accounts to the latest known pending nonce
+ nonces := make(map[common.Address]uint64, len(pool.pending))
+ for addr, list := range pool.pending {
+ highestPending := list.LastElement()
+ nonces[addr] = highestPending.Nonce() + 1
+ }
+ pool.pendingNonces.setAll(nonces)
+ }
+ // Ensure pool.queue and pool.pending sizes stay within the configured limits.
+ pool.truncatePending()
+ pool.truncateQueue()
+
+ dropBetweenReorgHistogram.Update(int64(pool.changesSinceReorg))
+ pool.changesSinceReorg = 0 // Reset change counter
+ pool.mu.Unlock()
+
+ // Notify subsystems for newly added transactions
+ for _, tx := range promoted {
+ addr, _ := types.Sender(pool.signer, tx)
+ if _, ok := events[addr]; !ok {
+ events[addr] = newSortedMap()
+ }
+ events[addr].Put(tx)
+ }
+ if len(events) > 0 {
+ var txs []*types.Transaction
+ for _, set := range events {
+ txs = append(txs, set.Flatten()...)
+ }
+ pool.txFeed.Send(core.NewTxsEvent{Txs: txs})
+ }
+}
+
+// reset retrieves the current state of the blockchain and ensures the content
+// of the transaction pool is valid with regard to the chain state.
+func (pool *LegacyPool) reset(oldHead, newHead *types.Header) {
+ // If we're reorging an old state, reinject all dropped transactions
+ var reinject types.Transactions
+
+ if oldHead != nil && oldHead.Hash() != newHead.ParentHash {
+ // If the reorg is too deep, avoid doing it (will happen during fast sync)
+ oldNum := oldHead.Number.Uint64()
+ newNum := newHead.Number.Uint64()
+
+ if depth := uint64(math.Abs(float64(oldNum) - float64(newNum))); depth > 64 {
+ log.Debug("Skipping deep transaction reorg", "depth", depth)
+ } else {
+ // Reorg seems shallow enough to pull in all transactions into memory
+ var discarded, included types.Transactions
+ var (
+ rem = pool.chain.GetBlock(oldHead.Hash(), oldHead.Number.Uint64())
+ add = pool.chain.GetBlock(newHead.Hash(), newHead.Number.Uint64())
+ )
+ if rem == nil {
+ // This can happen if a setHead is performed, where we simply discard the old
+ // head from the chain.
+ // If that is the case, we don't have the lost transactions anymore, and
+ // there's nothing to add
+ if newNum >= oldNum {
+ // If we reorged to a same or higher number, then it's not a case of setHead
+ log.Warn("Transaction pool reset with missing oldhead",
+ "old", oldHead.Hash(), "oldnum", oldNum, "new", newHead.Hash(), "newnum", newNum)
+ return
+ }
+ // If the reorg ended up on a lower number, it's indicative of setHead being the cause
+ log.Debug("Skipping transaction reset caused by setHead",
+ "old", oldHead.Hash(), "oldnum", oldNum, "new", newHead.Hash(), "newnum", newNum)
+ // We still need to update the current state s.th. the lost transactions can be readded by the user
+ } else {
+ for rem.NumberU64() > add.NumberU64() {
+ discarded = append(discarded, rem.Transactions()...)
+ if rem = pool.chain.GetBlock(rem.ParentHash(), rem.NumberU64()-1); rem == nil {
+ log.Error("Unrooted old chain seen by tx pool", "block", oldHead.Number, "hash", oldHead.Hash())
+ return
+ }
+ }
+ for add.NumberU64() > rem.NumberU64() {
+ included = append(included, add.Transactions()...)
+ if add = pool.chain.GetBlock(add.ParentHash(), add.NumberU64()-1); add == nil {
+ log.Error("Unrooted new chain seen by tx pool", "block", newHead.Number, "hash", newHead.Hash())
+ return
+ }
+ }
+ for rem.Hash() != add.Hash() {
+ discarded = append(discarded, rem.Transactions()...)
+ if rem = pool.chain.GetBlock(rem.ParentHash(), rem.NumberU64()-1); rem == nil {
+ log.Error("Unrooted old chain seen by tx pool", "block", oldHead.Number, "hash", oldHead.Hash())
+ return
+ }
+ included = append(included, add.Transactions()...)
+ if add = pool.chain.GetBlock(add.ParentHash(), add.NumberU64()-1); add == nil {
+ log.Error("Unrooted new chain seen by tx pool", "block", newHead.Number, "hash", newHead.Hash())
+ return
+ }
+ }
+ reinject = types.TxDifference(discarded, included)
+ }
+ }
+ }
+ // Initialize the internal state to the current head
+ if newHead == nil {
+ newHead = pool.chain.CurrentBlock() // Special case during testing
+ }
+ statedb, err := pool.chain.StateAt(newHead.Root)
+ if err != nil {
+ log.Error("Failed to reset txpool state", "err", err)
+ return
+ }
+ pool.currentHead.Store(newHead)
+ pool.currentState = statedb
+ pool.pendingNonces = newNoncer(statedb)
+
+ // Inject any transactions discarded due to reorgs
+ log.Debug("Reinjecting stale transactions", "count", len(reinject))
+ core.SenderCacher.Recover(pool.signer, reinject)
+ pool.addTxsLocked(reinject, false)
+}
+
+// promoteExecutables moves transactions that have become processable from the
+// future queue to the set of pending transactions. During this process, all
+// invalidated transactions (low nonce, low balance) are deleted.
+func (pool *LegacyPool) promoteExecutables(accounts []common.Address) []*types.Transaction {
+ // Track the promoted transactions to broadcast them at once
+ var promoted []*types.Transaction
+
+ // Iterate over all accounts and promote any executable transactions
+ gasLimit := pool.currentHead.Load().GasLimit
+ for _, addr := range accounts {
+ list := pool.queue[addr]
+ if list == nil {
+ continue // Just in case someone calls with a non existing account
+ }
+ // Drop all transactions that are deemed too old (low nonce)
+ forwards := list.Forward(pool.currentState.GetNonce(addr))
+ for _, tx := range forwards {
+ hash := tx.Hash()
+ pool.all.Remove(hash)
+ }
+ log.Trace("Removed old queued transactions", "count", len(forwards))
+ // Drop all transactions that are too costly (low balance or out of gas)
+ drops, _ := list.Filter(pool.currentState.GetBalance(addr), gasLimit)
+ for _, tx := range drops {
+ hash := tx.Hash()
+ pool.all.Remove(hash)
+ }
+ log.Trace("Removed unpayable queued transactions", "count", len(drops))
+ queuedNofundsMeter.Mark(int64(len(drops)))
+
+ // Gather all executable transactions and promote them
+ readies := list.Ready(pool.pendingNonces.get(addr))
+ for _, tx := range readies {
+ hash := tx.Hash()
+ if pool.promoteTx(addr, hash, tx) {
+ promoted = append(promoted, tx)
+ }
+ }
+ log.Trace("Promoted queued transactions", "count", len(promoted))
+ queuedGauge.Dec(int64(len(readies)))
+
+ // Drop all transactions over the allowed limit
+ var caps types.Transactions
+ if !pool.locals.contains(addr) {
+ caps = list.Cap(int(pool.config.AccountQueue))
+ for _, tx := range caps {
+ hash := tx.Hash()
+ pool.all.Remove(hash)
+ log.Trace("Removed cap-exceeding queued transaction", "hash", hash)
+ }
+ queuedRateLimitMeter.Mark(int64(len(caps)))
+ }
+ // Mark all the items dropped as removed
+ pool.priced.Removed(len(forwards) + len(drops) + len(caps))
+ queuedGauge.Dec(int64(len(forwards) + len(drops) + len(caps)))
+ if pool.locals.contains(addr) {
+ localGauge.Dec(int64(len(forwards) + len(drops) + len(caps)))
+ }
+ // Delete the entire queue entry if it became empty.
+ if list.Empty() {
+ delete(pool.queue, addr)
+ delete(pool.beats, addr)
+ }
+ }
+ return promoted
+}
+
+// truncatePending removes transactions from the pending queue if the pool is above the
+// pending limit. The algorithm tries to reduce transaction counts by an approximately
+// equal number for all for accounts with many pending transactions.
+func (pool *LegacyPool) truncatePending() {
+ pending := uint64(0)
+ for _, list := range pool.pending {
+ pending += uint64(list.Len())
+ }
+ if pending <= pool.config.GlobalSlots {
+ return
+ }
+
+ pendingBeforeCap := pending
+ // Assemble a spam order to penalize large transactors first
+ spammers := prque.New[int64, common.Address](nil)
+ for addr, list := range pool.pending {
+ // Only evict transactions from high rollers
+ if !pool.locals.contains(addr) && uint64(list.Len()) > pool.config.AccountSlots {
+ spammers.Push(addr, int64(list.Len()))
+ }
+ }
+ // Gradually drop transactions from offenders
+ offenders := []common.Address{}
+ for pending > pool.config.GlobalSlots && !spammers.Empty() {
+ // Retrieve the next offender if not local address
+ offender, _ := spammers.Pop()
+ offenders = append(offenders, offender)
+
+ // Equalize balances until all the same or below threshold
+ if len(offenders) > 1 {
+ // Calculate the equalization threshold for all current offenders
+ threshold := pool.pending[offender].Len()
+
+ // Iteratively reduce all offenders until below limit or threshold reached
+ for pending > pool.config.GlobalSlots && pool.pending[offenders[len(offenders)-2]].Len() > threshold {
+ for i := 0; i < len(offenders)-1; i++ {
+ list := pool.pending[offenders[i]]
+
+ caps := list.Cap(list.Len() - 1)
+ for _, tx := range caps {
+ // Drop the transaction from the global pools too
+ hash := tx.Hash()
+ pool.all.Remove(hash)
+
+ // Update the account nonce to the dropped transaction
+ pool.pendingNonces.setIfLower(offenders[i], tx.Nonce())
+ log.Trace("Removed fairness-exceeding pending transaction", "hash", hash)
+ }
+ pool.priced.Removed(len(caps))
+ pendingGauge.Dec(int64(len(caps)))
+ if pool.locals.contains(offenders[i]) {
+ localGauge.Dec(int64(len(caps)))
+ }
+ pending--
+ }
+ }
+ }
+ }
+
+ // If still above threshold, reduce to limit or min allowance
+ if pending > pool.config.GlobalSlots && len(offenders) > 0 {
+ for pending > pool.config.GlobalSlots && uint64(pool.pending[offenders[len(offenders)-1]].Len()) > pool.config.AccountSlots {
+ for _, addr := range offenders {
+ list := pool.pending[addr]
+
+ caps := list.Cap(list.Len() - 1)
+ for _, tx := range caps {
+ // Drop the transaction from the global pools too
+ hash := tx.Hash()
+ pool.all.Remove(hash)
+
+ // Update the account nonce to the dropped transaction
+ pool.pendingNonces.setIfLower(addr, tx.Nonce())
+ log.Trace("Removed fairness-exceeding pending transaction", "hash", hash)
+ }
+ pool.priced.Removed(len(caps))
+ pendingGauge.Dec(int64(len(caps)))
+ if pool.locals.contains(addr) {
+ localGauge.Dec(int64(len(caps)))
+ }
+ pending--
+ }
+ }
+ }
+ pendingRateLimitMeter.Mark(int64(pendingBeforeCap - pending))
+}
+
+// truncateQueue drops the oldest transactions in the queue if the pool is above the global queue limit.
+func (pool *LegacyPool) truncateQueue() {
+ queued := uint64(0)
+ for _, list := range pool.queue {
+ queued += uint64(list.Len())
+ }
+ if queued <= pool.config.GlobalQueue {
+ return
+ }
+
+ // Sort all accounts with queued transactions by heartbeat
+ addresses := make(addressesByHeartbeat, 0, len(pool.queue))
+ for addr := range pool.queue {
+ if !pool.locals.contains(addr) { // don't drop locals
+ addresses = append(addresses, addressByHeartbeat{addr, pool.beats[addr]})
+ }
+ }
+ sort.Sort(sort.Reverse(addresses))
+
+ // Drop transactions until the total is below the limit or only locals remain
+ for drop := queued - pool.config.GlobalQueue; drop > 0 && len(addresses) > 0; {
+ addr := addresses[len(addresses)-1]
+ list := pool.queue[addr.address]
+
+ addresses = addresses[:len(addresses)-1]
+
+ // Drop all transactions if they are less than the overflow
+ if size := uint64(list.Len()); size <= drop {
+ for _, tx := range list.Flatten() {
+ pool.removeTx(tx.Hash(), true)
+ }
+ drop -= size
+ queuedRateLimitMeter.Mark(int64(size))
+ continue
+ }
+ // Otherwise drop only last few transactions
+ txs := list.Flatten()
+ for i := len(txs) - 1; i >= 0 && drop > 0; i-- {
+ pool.removeTx(txs[i].Hash(), true)
+ drop--
+ queuedRateLimitMeter.Mark(1)
+ }
+ }
+}
+
+// demoteUnexecutables removes invalid and processed transactions from the pools
+// executable/pending queue and any subsequent transactions that become unexecutable
+// are moved back into the future queue.
+//
+// Note: transactions are not marked as removed in the priced list because re-heaping
+// is always explicitly triggered by SetBaseFee and it would be unnecessary and wasteful
+// to trigger a re-heap is this function
+func (pool *LegacyPool) demoteUnexecutables() {
+ // Iterate over all accounts and demote any non-executable transactions
+ gasLimit := pool.currentHead.Load().GasLimit
+ for addr, list := range pool.pending {
+ nonce := pool.currentState.GetNonce(addr)
+
+ // Drop all transactions that are deemed too old (low nonce)
+ olds := list.Forward(nonce)
+ for _, tx := range olds {
+ hash := tx.Hash()
+ pool.all.Remove(hash)
+ log.Trace("Removed old pending transaction", "hash", hash)
+ }
+ // Drop all transactions that are too costly (low balance or out of gas), and queue any invalids back for later
+ drops, invalids := list.Filter(pool.currentState.GetBalance(addr), gasLimit)
+ for _, tx := range drops {
+ hash := tx.Hash()
+ log.Trace("Removed unpayable pending transaction", "hash", hash)
+ pool.all.Remove(hash)
+ }
+ pendingNofundsMeter.Mark(int64(len(drops)))
+
+ for _, tx := range invalids {
+ hash := tx.Hash()
+ log.Trace("Demoting pending transaction", "hash", hash)
+
+ // Internal shuffle shouldn't touch the lookup set.
+ pool.enqueueTx(hash, tx, false, false)
+ }
+ pendingGauge.Dec(int64(len(olds) + len(drops) + len(invalids)))
+ if pool.locals.contains(addr) {
+ localGauge.Dec(int64(len(olds) + len(drops) + len(invalids)))
+ }
+ // If there's a gap in front, alert (should never happen) and postpone all transactions
+ if list.Len() > 0 && list.txs.Get(nonce) == nil {
+ gapped := list.Cap(0)
+ for _, tx := range gapped {
+ hash := tx.Hash()
+ log.Error("Demoting invalidated transaction", "hash", hash)
+
+ // Internal shuffle shouldn't touch the lookup set.
+ pool.enqueueTx(hash, tx, false, false)
+ }
+ pendingGauge.Dec(int64(len(gapped)))
+ }
+ // Delete the entire pending entry if it became empty.
+ if list.Empty() {
+ delete(pool.pending, addr)
+ }
+ }
+}
+
+// addressByHeartbeat is an account address tagged with its last activity timestamp.
+type addressByHeartbeat struct {
+ address common.Address
+ heartbeat time.Time
+}
+
+type addressesByHeartbeat []addressByHeartbeat
+
+func (a addressesByHeartbeat) Len() int { return len(a) }
+func (a addressesByHeartbeat) Less(i, j int) bool { return a[i].heartbeat.Before(a[j].heartbeat) }
+func (a addressesByHeartbeat) Swap(i, j int) { a[i], a[j] = a[j], a[i] }
+
+// accountSet is simply a set of addresses to check for existence, and a signer
+// capable of deriving addresses from transactions.
+type accountSet struct {
+ accounts map[common.Address]struct{}
+ signer types.Signer
+ cache *[]common.Address
+}
+
+// newAccountSet creates a new address set with an associated signer for sender
+// derivations.
+func newAccountSet(signer types.Signer, addrs ...common.Address) *accountSet {
+ as := &accountSet{
+ accounts: make(map[common.Address]struct{}, len(addrs)),
+ signer: signer,
+ }
+ for _, addr := range addrs {
+ as.add(addr)
+ }
+ return as
+}
+
+// contains checks if a given address is contained within the set.
+func (as *accountSet) contains(addr common.Address) bool {
+ _, exist := as.accounts[addr]
+ return exist
+}
+
+// containsTx checks if the sender of a given tx is within the set. If the sender
+// cannot be derived, this method returns false.
+func (as *accountSet) containsTx(tx *types.Transaction) bool {
+ if addr, err := types.Sender(as.signer, tx); err == nil {
+ return as.contains(addr)
+ }
+ return false
+}
+
+// add inserts a new address into the set to track.
+func (as *accountSet) add(addr common.Address) {
+ as.accounts[addr] = struct{}{}
+ as.cache = nil
+}
+
+// addTx adds the sender of tx into the set.
+func (as *accountSet) addTx(tx *types.Transaction) {
+ if addr, err := types.Sender(as.signer, tx); err == nil {
+ as.add(addr)
+ }
+}
+
+// flatten returns the list of addresses within this set, also caching it for later
+// reuse. The returned slice should not be changed!
+func (as *accountSet) flatten() []common.Address {
+ if as.cache == nil {
+ accounts := make([]common.Address, 0, len(as.accounts))
+ for account := range as.accounts {
+ accounts = append(accounts, account)
+ }
+ as.cache = &accounts
+ }
+ return *as.cache
+}
+
+// merge adds all addresses from the 'other' set into 'as'.
+func (as *accountSet) merge(other *accountSet) {
+ for addr := range other.accounts {
+ as.accounts[addr] = struct{}{}
+ }
+ as.cache = nil
+}
+
+// lookup is used internally by LegacyPool to track transactions while allowing
+// lookup without mutex contention.
+//
+// Note, although this type is properly protected against concurrent access, it
+// is **not** a type that should ever be mutated or even exposed outside of the
+// transaction pool, since its internal state is tightly coupled with the pools
+// internal mechanisms. The sole purpose of the type is to permit out-of-bound
+// peeking into the pool in LegacyPool.Get without having to acquire the widely scoped
+// LegacyPool.mu mutex.
+//
+// This lookup set combines the notion of "local transactions", which is useful
+// to build upper-level structure.
+type lookup struct {
+ slots int
+ lock sync.RWMutex
+ locals map[common.Hash]*types.Transaction
+ remotes map[common.Hash]*types.Transaction
+}
+
+// newLookup returns a new lookup structure.
+func newLookup() *lookup {
+ return &lookup{
+ locals: make(map[common.Hash]*types.Transaction),
+ remotes: make(map[common.Hash]*types.Transaction),
+ }
+}
+
+// Range calls f on each key and value present in the map. The callback passed
+// should return the indicator whether the iteration needs to be continued.
+// Callers need to specify which set (or both) to be iterated.
+func (t *lookup) Range(f func(hash common.Hash, tx *types.Transaction, local bool) bool, local bool, remote bool) {
+ t.lock.RLock()
+ defer t.lock.RUnlock()
+
+ if local {
+ for key, value := range t.locals {
+ if !f(key, value, true) {
+ return
+ }
+ }
+ }
+ if remote {
+ for key, value := range t.remotes {
+ if !f(key, value, false) {
+ return
+ }
+ }
+ }
+}
+
+// Get returns a transaction if it exists in the lookup, or nil if not found.
+func (t *lookup) Get(hash common.Hash) *types.Transaction {
+ t.lock.RLock()
+ defer t.lock.RUnlock()
+
+ if tx := t.locals[hash]; tx != nil {
+ return tx
+ }
+ return t.remotes[hash]
+}
+
+// GetLocal returns a transaction if it exists in the lookup, or nil if not found.
+func (t *lookup) GetLocal(hash common.Hash) *types.Transaction {
+ t.lock.RLock()
+ defer t.lock.RUnlock()
+
+ return t.locals[hash]
+}
+
+// GetRemote returns a transaction if it exists in the lookup, or nil if not found.
+func (t *lookup) GetRemote(hash common.Hash) *types.Transaction {
+ t.lock.RLock()
+ defer t.lock.RUnlock()
+
+ return t.remotes[hash]
+}
+
+// Count returns the current number of transactions in the lookup.
+func (t *lookup) Count() int {
+ t.lock.RLock()
+ defer t.lock.RUnlock()
+
+ return len(t.locals) + len(t.remotes)
+}
+
+// LocalCount returns the current number of local transactions in the lookup.
+func (t *lookup) LocalCount() int {
+ t.lock.RLock()
+ defer t.lock.RUnlock()
+
+ return len(t.locals)
+}
+
+// RemoteCount returns the current number of remote transactions in the lookup.
+func (t *lookup) RemoteCount() int {
+ t.lock.RLock()
+ defer t.lock.RUnlock()
+
+ return len(t.remotes)
+}
+
+// Slots returns the current number of slots used in the lookup.
+func (t *lookup) Slots() int {
+ t.lock.RLock()
+ defer t.lock.RUnlock()
+
+ return t.slots
+}
+
+// Add adds a transaction to the lookup.
+func (t *lookup) Add(tx *types.Transaction, local bool) {
+ t.lock.Lock()
+ defer t.lock.Unlock()
+
+ t.slots += numSlots(tx)
+ slotsGauge.Update(int64(t.slots))
+
+ if local {
+ t.locals[tx.Hash()] = tx
+ } else {
+ t.remotes[tx.Hash()] = tx
+ }
+}
+
+// Remove removes a transaction from the lookup.
+func (t *lookup) Remove(hash common.Hash) {
+ t.lock.Lock()
+ defer t.lock.Unlock()
+
+ tx, ok := t.locals[hash]
+ if !ok {
+ tx, ok = t.remotes[hash]
+ }
+ if !ok {
+ log.Error("No transaction found to be deleted", "hash", hash)
+ return
+ }
+ t.slots -= numSlots(tx)
+ slotsGauge.Update(int64(t.slots))
+
+ delete(t.locals, hash)
+ delete(t.remotes, hash)
+}
+
+// RemoteToLocals migrates the transactions belongs to the given locals to locals
+// set. The assumption is held the locals set is thread-safe to be used.
+func (t *lookup) RemoteToLocals(locals *accountSet) int {
+ t.lock.Lock()
+ defer t.lock.Unlock()
+
+ var migrated int
+ for hash, tx := range t.remotes {
+ if locals.containsTx(tx) {
+ t.locals[hash] = tx
+ delete(t.remotes, hash)
+ migrated += 1
+ }
+ }
+ return migrated
+}
+
+// RemotesBelowTip finds all remote transactions below the given tip threshold.
+func (t *lookup) RemotesBelowTip(threshold *big.Int) types.Transactions {
+ found := make(types.Transactions, 0, 128)
+ t.Range(func(hash common.Hash, tx *types.Transaction, local bool) bool {
+ if tx.GasTipCapIntCmp(threshold) < 0 {
+ found = append(found, tx)
+ }
+ return true
+ }, false, true) // Only iterate remotes
+ return found
+}
+
+// numSlots calculates the number of slots needed for a single transaction.
+func numSlots(tx *types.Transaction) int {
+ return int((tx.Size() + txSlotSize - 1) / txSlotSize)
+}
diff --git a/core/txpool/txpool2_test.go b/core/txpool/legacypool/legacypool2_test.go
similarity index 84%
rename from core/txpool/txpool2_test.go
rename to core/txpool/legacypool/legacypool2_test.go
index b67905056..5de34588a 100644
--- a/core/txpool/txpool2_test.go
+++ b/core/txpool/legacypool/legacypool2_test.go
@@ -13,7 +13,7 @@
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see .
-package txpool
+package legacypool
import (
"crypto/ecdsa"
@@ -33,7 +33,7 @@ func pricedValuedTransaction(nonce uint64, value int64, gaslimit uint64, gaspric
return tx
}
-func count(t *testing.T, pool *TxPool) (pending int, queued int) {
+func count(t *testing.T, pool *LegacyPool) (pending int, queued int) {
t.Helper()
pending, queued = pool.stats()
if err := validatePoolInternals(pool); err != nil {
@@ -42,7 +42,7 @@ func count(t *testing.T, pool *TxPool) (pending int, queued int) {
return pending, queued
}
-func fillPool(t testing.TB, pool *TxPool) {
+func fillPool(t testing.TB, pool *LegacyPool) {
t.Helper()
// Create a number of test accounts, fund them and make transactions
executableTxs := types.Transactions{}
@@ -56,8 +56,8 @@ func fillPool(t testing.TB, pool *TxPool) {
}
}
// Import the batch and verify that limits have been enforced
- pool.AddRemotesSync(executableTxs)
- pool.AddRemotesSync(nonExecutableTxs)
+ pool.addRemotesSync(executableTxs)
+ pool.addRemotesSync(nonExecutableTxs)
pending, queued := pool.Stats()
slots := pool.all.Slots()
// sanity-check that the test prerequisites are ok (pending full)
@@ -79,12 +79,13 @@ func TestTransactionFutureAttack(t *testing.T) {
// Create the pool to test the limit enforcement with
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabase(rawdb.NewMemoryDatabase()), nil)
- blockchain := newTestBlockChain(1000000, statedb, new(event.Feed))
+ blockchain := newTestBlockChain(eip1559Config, 1000000, statedb, new(event.Feed))
config := testTxPoolConfig
config.GlobalQueue = 100
config.GlobalSlots = 100
- pool := New(config, eip1559Config, blockchain)
- defer pool.Stop()
+ pool := New(config, blockchain)
+ pool.Init(new(big.Int).SetUint64(config.PriceLimit), blockchain.CurrentBlock())
+ defer pool.Close()
fillPool(t, pool)
pending, _ := pool.Stats()
// Now, future transaction attack starts, let's add a bunch of expensive non-executables, and see if the pending-count drops
@@ -96,7 +97,7 @@ func TestTransactionFutureAttack(t *testing.T) {
futureTxs = append(futureTxs, pricedTransaction(1000+uint64(j), 100000, big.NewInt(500), key))
}
for i := 0; i < 5; i++ {
- pool.AddRemotesSync(futureTxs)
+ pool.addRemotesSync(futureTxs)
newPending, newQueued := count(t, pool)
t.Logf("pending: %d queued: %d, all: %d\n", newPending, newQueued, pool.all.Slots())
}
@@ -115,9 +116,10 @@ func TestTransactionFuture1559(t *testing.T) {
t.Parallel()
// Create the pool to test the pricing enforcement with
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabase(rawdb.NewMemoryDatabase()), nil)
- blockchain := newTestBlockChain(1000000, statedb, new(event.Feed))
- pool := New(testTxPoolConfig, eip1559Config, blockchain)
- defer pool.Stop()
+ blockchain := newTestBlockChain(eip1559Config, 1000000, statedb, new(event.Feed))
+ pool := New(testTxPoolConfig, blockchain)
+ pool.Init(new(big.Int).SetUint64(testTxPoolConfig.PriceLimit), blockchain.CurrentBlock())
+ defer pool.Close()
// Create a number of test accounts, fund them and make transactions
fillPool(t, pool)
@@ -131,7 +133,7 @@ func TestTransactionFuture1559(t *testing.T) {
for j := 0; j < int(pool.config.GlobalSlots+pool.config.GlobalQueue); j++ {
futureTxs = append(futureTxs, dynamicFeeTx(1000+uint64(j), 100000, big.NewInt(200), big.NewInt(101), key))
}
- pool.AddRemotesSync(futureTxs)
+ pool.addRemotesSync(futureTxs)
}
newPending, _ := pool.Stats()
// Pending should not have been touched
@@ -147,9 +149,10 @@ func TestTransactionZAttack(t *testing.T) {
t.Parallel()
// Create the pool to test the pricing enforcement with
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabase(rawdb.NewMemoryDatabase()), nil)
- blockchain := newTestBlockChain(1000000, statedb, new(event.Feed))
- pool := New(testTxPoolConfig, eip1559Config, blockchain)
- defer pool.Stop()
+ blockchain := newTestBlockChain(eip1559Config, 1000000, statedb, new(event.Feed))
+ pool := New(testTxPoolConfig, blockchain)
+ pool.Init(new(big.Int).SetUint64(testTxPoolConfig.PriceLimit), blockchain.CurrentBlock())
+ defer pool.Close()
// Create a number of test accounts, fund them and make transactions
fillPool(t, pool)
@@ -181,7 +184,7 @@ func TestTransactionZAttack(t *testing.T) {
key, _ := crypto.GenerateKey()
pool.currentState.AddBalance(crypto.PubkeyToAddress(key.PublicKey), big.NewInt(100000000000))
futureTxs = append(futureTxs, pricedTransaction(1000+uint64(j), 21000, big.NewInt(500), key))
- pool.AddRemotesSync(futureTxs)
+ pool.addRemotesSync(futureTxs)
}
overDraftTxs := types.Transactions{}
@@ -192,11 +195,11 @@ func TestTransactionZAttack(t *testing.T) {
overDraftTxs = append(overDraftTxs, pricedValuedTransaction(uint64(j), 600000000000, 21000, big.NewInt(500), key))
}
}
- pool.AddRemotesSync(overDraftTxs)
- pool.AddRemotesSync(overDraftTxs)
- pool.AddRemotesSync(overDraftTxs)
- pool.AddRemotesSync(overDraftTxs)
- pool.AddRemotesSync(overDraftTxs)
+ pool.addRemotesSync(overDraftTxs)
+ pool.addRemotesSync(overDraftTxs)
+ pool.addRemotesSync(overDraftTxs)
+ pool.addRemotesSync(overDraftTxs)
+ pool.addRemotesSync(overDraftTxs)
newPending, newQueued := count(t, pool)
newIvPending := countInvalidPending()
@@ -214,12 +217,13 @@ func TestTransactionZAttack(t *testing.T) {
func BenchmarkFutureAttack(b *testing.B) {
// Create the pool to test the limit enforcement with
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabase(rawdb.NewMemoryDatabase()), nil)
- blockchain := newTestBlockChain(1000000, statedb, new(event.Feed))
+ blockchain := newTestBlockChain(eip1559Config, 1000000, statedb, new(event.Feed))
config := testTxPoolConfig
config.GlobalQueue = 100
config.GlobalSlots = 100
- pool := New(config, eip1559Config, blockchain)
- defer pool.Stop()
+ pool := New(config, blockchain)
+ pool.Init(new(big.Int).SetUint64(testTxPoolConfig.PriceLimit), blockchain.CurrentBlock())
+ defer pool.Close()
fillPool(b, pool)
key, _ := crypto.GenerateKey()
@@ -231,6 +235,6 @@ func BenchmarkFutureAttack(b *testing.B) {
}
b.ResetTimer()
for i := 0; i < 5; i++ {
- pool.AddRemotesSync(futureTxs)
+ pool.addRemotesSync(futureTxs)
}
}
diff --git a/core/txpool/txpool_test.go b/core/txpool/legacypool/legacypool_test.go
similarity index 87%
rename from core/txpool/txpool_test.go
rename to core/txpool/legacypool/legacypool_test.go
index 319e25bea..e3302487b 100644
--- a/core/txpool/txpool_test.go
+++ b/core/txpool/legacypool/legacypool_test.go
@@ -14,7 +14,7 @@
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see .
-package txpool
+package legacypool
import (
"crypto/ecdsa"
@@ -32,6 +32,7 @@ import (
"github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/rawdb"
"github.com/ethereum/go-ethereum/core/state"
+ "github.com/ethereum/go-ethereum/core/txpool"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/event"
@@ -59,17 +60,22 @@ func init() {
}
type testBlockChain struct {
+ config *params.ChainConfig
gasLimit atomic.Uint64
statedb *state.StateDB
chainHeadFeed *event.Feed
}
-func newTestBlockChain(gasLimit uint64, statedb *state.StateDB, chainHeadFeed *event.Feed) *testBlockChain {
- bc := testBlockChain{statedb: statedb, chainHeadFeed: new(event.Feed)}
+func newTestBlockChain(config *params.ChainConfig, gasLimit uint64, statedb *state.StateDB, chainHeadFeed *event.Feed) *testBlockChain {
+ bc := testBlockChain{config: config, statedb: statedb, chainHeadFeed: new(event.Feed)}
bc.gasLimit.Store(gasLimit)
return &bc
}
+func (bc *testBlockChain) Config() *params.ChainConfig {
+ return bc.config
+}
+
func (bc *testBlockChain) CurrentBlock() *types.Header {
return &types.Header{
Number: new(big.Int),
@@ -121,24 +127,26 @@ func dynamicFeeTx(nonce uint64, gaslimit uint64, gasFee *big.Int, tip *big.Int,
return tx
}
-func setupPool() (*TxPool, *ecdsa.PrivateKey) {
+func setupPool() (*LegacyPool, *ecdsa.PrivateKey) {
return setupPoolWithConfig(params.TestChainConfig)
}
-func setupPoolWithConfig(config *params.ChainConfig) (*TxPool, *ecdsa.PrivateKey) {
+func setupPoolWithConfig(config *params.ChainConfig) (*LegacyPool, *ecdsa.PrivateKey) {
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabase(rawdb.NewMemoryDatabase()), nil)
- blockchain := newTestBlockChain(10000000, statedb, new(event.Feed))
+ blockchain := newTestBlockChain(config, 10000000, statedb, new(event.Feed))
key, _ := crypto.GenerateKey()
- pool := New(testTxPoolConfig, config, blockchain)
-
+ pool := New(testTxPoolConfig, blockchain)
+ if err := pool.Init(new(big.Int).SetUint64(testTxPoolConfig.PriceLimit), blockchain.CurrentBlock()); err != nil {
+ panic(err)
+ }
// wait for the pool to initialize
<-pool.initDoneCh
return pool, key
}
// validatePoolInternals checks various consistency invariants within the pool.
-func validatePoolInternals(pool *TxPool) error {
+func validatePoolInternals(pool *LegacyPool) error {
pool.mu.RLock()
defer pool.mu.RUnlock()
@@ -242,20 +250,21 @@ func TestStateChangeDuringReset(t *testing.T) {
// setup pool with 2 transaction in it
statedb.SetBalance(address, new(big.Int).SetUint64(params.Ether))
- blockchain := &testChain{newTestBlockChain(1000000000, statedb, new(event.Feed)), address, &trigger}
+ blockchain := &testChain{newTestBlockChain(params.TestChainConfig, 1000000000, statedb, new(event.Feed)), address, &trigger}
tx0 := transaction(0, 100000, key)
tx1 := transaction(1, 100000, key)
- pool := New(testTxPoolConfig, params.TestChainConfig, blockchain)
- defer pool.Stop()
+ pool := New(testTxPoolConfig, blockchain)
+ pool.Init(new(big.Int).SetUint64(testTxPoolConfig.PriceLimit), blockchain.CurrentBlock())
+ defer pool.Close()
nonce := pool.Nonce(address)
if nonce != 0 {
t.Fatalf("Invalid nonce, want 0, got %d", nonce)
}
- pool.AddRemotesSync([]*types.Transaction{tx0, tx1})
+ pool.addRemotesSync([]*types.Transaction{tx0, tx1})
nonce = pool.Nonce(address)
if nonce != 2 {
@@ -272,13 +281,13 @@ func TestStateChangeDuringReset(t *testing.T) {
}
}
-func testAddBalance(pool *TxPool, addr common.Address, amount *big.Int) {
+func testAddBalance(pool *LegacyPool, addr common.Address, amount *big.Int) {
pool.mu.Lock()
pool.currentState.AddBalance(addr, amount)
pool.mu.Unlock()
}
-func testSetNonce(pool *TxPool, addr common.Address, nonce uint64) {
+func testSetNonce(pool *LegacyPool, addr common.Address, nonce uint64) {
pool.mu.Lock()
pool.currentState.SetNonce(addr, nonce)
pool.mu.Unlock()
@@ -288,36 +297,36 @@ func TestInvalidTransactions(t *testing.T) {
t.Parallel()
pool, key := setupPool()
- defer pool.Stop()
+ defer pool.Close()
tx := transaction(0, 100, key)
from, _ := deriveSender(tx)
// Intrinsic gas too low
testAddBalance(pool, from, big.NewInt(1))
- if err, want := pool.AddRemote(tx), core.ErrIntrinsicGas; !errors.Is(err, want) {
+ if err, want := pool.addRemote(tx), core.ErrIntrinsicGas; !errors.Is(err, want) {
t.Errorf("want %v have %v", want, err)
}
// Insufficient funds
tx = transaction(0, 100000, key)
- if err, want := pool.AddRemote(tx), core.ErrInsufficientFunds; !errors.Is(err, want) {
+ if err, want := pool.addRemote(tx), core.ErrInsufficientFunds; !errors.Is(err, want) {
t.Errorf("want %v have %v", want, err)
}
testSetNonce(pool, from, 1)
testAddBalance(pool, from, big.NewInt(0xffffffffffffff))
tx = transaction(0, 100000, key)
- if err, want := pool.AddRemote(tx), core.ErrNonceTooLow; !errors.Is(err, want) {
+ if err, want := pool.addRemote(tx), core.ErrNonceTooLow; !errors.Is(err, want) {
t.Errorf("want %v have %v", want, err)
}
tx = transaction(1, 100000, key)
pool.gasTip.Store(big.NewInt(1000))
- if err, want := pool.AddRemote(tx), ErrUnderpriced; !errors.Is(err, want) {
+ if err, want := pool.addRemote(tx), txpool.ErrUnderpriced; !errors.Is(err, want) {
t.Errorf("want %v have %v", want, err)
}
- if err := pool.AddLocal(tx); err != nil {
+ if err := pool.addLocal(tx); err != nil {
t.Error("expected", nil, "got", err)
}
}
@@ -326,7 +335,7 @@ func TestQueue(t *testing.T) {
t.Parallel()
pool, key := setupPool()
- defer pool.Stop()
+ defer pool.Close()
tx := transaction(0, 100, key)
from, _ := deriveSender(tx)
@@ -357,7 +366,7 @@ func TestQueue2(t *testing.T) {
t.Parallel()
pool, key := setupPool()
- defer pool.Stop()
+ defer pool.Close()
tx1 := transaction(0, 100, key)
tx2 := transaction(10, 100, key)
@@ -383,13 +392,13 @@ func TestNegativeValue(t *testing.T) {
t.Parallel()
pool, key := setupPool()
- defer pool.Stop()
+ defer pool.Close()
tx, _ := types.SignTx(types.NewTransaction(0, common.Address{}, big.NewInt(-1), 100, big.NewInt(1), nil), types.HomesteadSigner{}, key)
from, _ := deriveSender(tx)
testAddBalance(pool, from, big.NewInt(1))
- if err := pool.AddRemote(tx); err != ErrNegativeValue {
- t.Error("expected", ErrNegativeValue, "got", err)
+ if err := pool.addRemote(tx); err != txpool.ErrNegativeValue {
+ t.Error("expected", txpool.ErrNegativeValue, "got", err)
}
}
@@ -397,11 +406,11 @@ func TestTipAboveFeeCap(t *testing.T) {
t.Parallel()
pool, key := setupPoolWithConfig(eip1559Config)
- defer pool.Stop()
+ defer pool.Close()
tx := dynamicFeeTx(0, 100, big.NewInt(1), big.NewInt(2), key)
- if err := pool.AddRemote(tx); err != core.ErrTipAboveFeeCap {
+ if err := pool.addRemote(tx); err != core.ErrTipAboveFeeCap {
t.Error("expected", core.ErrTipAboveFeeCap, "got", err)
}
}
@@ -410,18 +419,18 @@ func TestVeryHighValues(t *testing.T) {
t.Parallel()
pool, key := setupPoolWithConfig(eip1559Config)
- defer pool.Stop()
+ defer pool.Close()
veryBigNumber := big.NewInt(1)
veryBigNumber.Lsh(veryBigNumber, 300)
tx := dynamicFeeTx(0, 100, big.NewInt(1), veryBigNumber, key)
- if err := pool.AddRemote(tx); err != core.ErrTipVeryHigh {
+ if err := pool.addRemote(tx); err != core.ErrTipVeryHigh {
t.Error("expected", core.ErrTipVeryHigh, "got", err)
}
tx2 := dynamicFeeTx(0, 100, veryBigNumber, big.NewInt(1), key)
- if err := pool.AddRemote(tx2); err != core.ErrFeeCapVeryHigh {
+ if err := pool.addRemote(tx2); err != core.ErrFeeCapVeryHigh {
t.Error("expected", core.ErrFeeCapVeryHigh, "got", err)
}
}
@@ -430,14 +439,14 @@ func TestChainFork(t *testing.T) {
t.Parallel()
pool, key := setupPool()
- defer pool.Stop()
+ defer pool.Close()
addr := crypto.PubkeyToAddress(key.PublicKey)
resetState := func() {
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabase(rawdb.NewMemoryDatabase()), nil)
statedb.AddBalance(addr, big.NewInt(100000000000000))
- pool.chain = newTestBlockChain(1000000, statedb, new(event.Feed))
+ pool.chain = newTestBlockChain(pool.chainconfig, 1000000, statedb, new(event.Feed))
<-pool.requestReset(nil, nil)
}
resetState()
@@ -459,14 +468,14 @@ func TestDoubleNonce(t *testing.T) {
t.Parallel()
pool, key := setupPool()
- defer pool.Stop()
+ defer pool.Close()
addr := crypto.PubkeyToAddress(key.PublicKey)
resetState := func() {
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabase(rawdb.NewMemoryDatabase()), nil)
statedb.AddBalance(addr, big.NewInt(100000000000000))
- pool.chain = newTestBlockChain(1000000, statedb, new(event.Feed))
+ pool.chain = newTestBlockChain(pool.chainconfig, 1000000, statedb, new(event.Feed))
<-pool.requestReset(nil, nil)
}
resetState()
@@ -510,7 +519,7 @@ func TestMissingNonce(t *testing.T) {
t.Parallel()
pool, key := setupPool()
- defer pool.Stop()
+ defer pool.Close()
addr := crypto.PubkeyToAddress(key.PublicKey)
testAddBalance(pool, addr, big.NewInt(100000000000000))
@@ -534,7 +543,7 @@ func TestNonceRecovery(t *testing.T) {
const n = 10
pool, key := setupPool()
- defer pool.Stop()
+ defer pool.Close()
addr := crypto.PubkeyToAddress(key.PublicKey)
testSetNonce(pool, addr, n)
@@ -542,7 +551,7 @@ func TestNonceRecovery(t *testing.T) {
<-pool.requestReset(nil, nil)
tx := transaction(n, 100000, key)
- if err := pool.AddRemote(tx); err != nil {
+ if err := pool.addRemote(tx); err != nil {
t.Error(err)
}
// simulate some weird re-order of transactions and missing nonce(s)
@@ -560,7 +569,7 @@ func TestDropping(t *testing.T) {
// Create a test account and fund it
pool, key := setupPool()
- defer pool.Stop()
+ defer pool.Close()
account := crypto.PubkeyToAddress(key.PublicKey)
testAddBalance(pool, account, big.NewInt(1000))
@@ -664,10 +673,11 @@ func TestPostponing(t *testing.T) {
// Create the pool to test the postponing with
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabase(rawdb.NewMemoryDatabase()), nil)
- blockchain := newTestBlockChain(1000000, statedb, new(event.Feed))
+ blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed))
- pool := New(testTxPoolConfig, params.TestChainConfig, blockchain)
- defer pool.Stop()
+ pool := New(testTxPoolConfig, blockchain)
+ pool.Init(new(big.Int).SetUint64(testTxPoolConfig.PriceLimit), blockchain.CurrentBlock())
+ defer pool.Close()
// Create two test accounts to produce different gap profiles with
keys := make([]*ecdsa.PrivateKey, 2)
@@ -692,7 +702,7 @@ func TestPostponing(t *testing.T) {
txs = append(txs, tx)
}
}
- for i, err := range pool.AddRemotesSync(txs) {
+ for i, err := range pool.addRemotesSync(txs) {
if err != nil {
t.Fatalf("tx %d: failed to add transactions: %v", i, err)
}
@@ -777,7 +787,7 @@ func TestGapFilling(t *testing.T) {
// Create a test account and fund it
pool, key := setupPool()
- defer pool.Stop()
+ defer pool.Close()
account := crypto.PubkeyToAddress(key.PublicKey)
testAddBalance(pool, account, big.NewInt(1000000))
@@ -788,7 +798,7 @@ func TestGapFilling(t *testing.T) {
defer sub.Unsubscribe()
// Create a pending and a queued transaction with a nonce-gap in between
- pool.AddRemotesSync([]*types.Transaction{
+ pool.addRemotesSync([]*types.Transaction{
transaction(0, 100000, key),
transaction(2, 100000, key),
})
@@ -831,7 +841,7 @@ func TestQueueAccountLimiting(t *testing.T) {
// Create a test account and fund it
pool, key := setupPool()
- defer pool.Stop()
+ defer pool.Close()
account := crypto.PubkeyToAddress(key.PublicKey)
testAddBalance(pool, account, big.NewInt(1000000))
@@ -876,14 +886,15 @@ func testQueueGlobalLimiting(t *testing.T, nolocals bool) {
// Create the pool to test the limit enforcement with
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabase(rawdb.NewMemoryDatabase()), nil)
- blockchain := newTestBlockChain(1000000, statedb, new(event.Feed))
+ blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed))
config := testTxPoolConfig
config.NoLocals = nolocals
config.GlobalQueue = config.AccountQueue*3 - 1 // reduce the queue limits to shorten test time (-1 to make it non divisible)
- pool := New(config, params.TestChainConfig, blockchain)
- defer pool.Stop()
+ pool := New(config, blockchain)
+ pool.Init(new(big.Int).SetUint64(testTxPoolConfig.PriceLimit), blockchain.CurrentBlock())
+ defer pool.Close()
// Create a number of test accounts and fund them (last one will be the local)
keys := make([]*ecdsa.PrivateKey, 5)
@@ -905,7 +916,7 @@ func testQueueGlobalLimiting(t *testing.T, nolocals bool) {
nonces[addr]++
}
// Import the batch and verify that limits have been enforced
- pool.AddRemotesSync(txs)
+ pool.addRemotesSync(txs)
queued := 0
for addr, list := range pool.queue {
@@ -922,7 +933,7 @@ func testQueueGlobalLimiting(t *testing.T, nolocals bool) {
for i := uint64(0); i < 3*config.GlobalQueue; i++ {
txs = append(txs, transaction(i+1, 100000, local))
}
- pool.AddLocals(txs)
+ pool.addLocals(txs)
// If locals are disabled, the previous eviction algorithm should apply here too
if nolocals {
@@ -968,14 +979,15 @@ func testQueueTimeLimiting(t *testing.T, nolocals bool) {
// Create the pool to test the non-expiration enforcement
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabase(rawdb.NewMemoryDatabase()), nil)
- blockchain := newTestBlockChain(1000000, statedb, new(event.Feed))
+ blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed))
config := testTxPoolConfig
config.Lifetime = time.Second
config.NoLocals = nolocals
- pool := New(config, params.TestChainConfig, blockchain)
- defer pool.Stop()
+ pool := New(config, blockchain)
+ pool.Init(new(big.Int).SetUint64(config.PriceLimit), blockchain.CurrentBlock())
+ defer pool.Close()
// Create two test accounts to ensure remotes expire but locals do not
local, _ := crypto.GenerateKey()
@@ -985,10 +997,10 @@ func testQueueTimeLimiting(t *testing.T, nolocals bool) {
testAddBalance(pool, crypto.PubkeyToAddress(remote.PublicKey), big.NewInt(1000000000))
// Add the two transactions and ensure they both are queued up
- if err := pool.AddLocal(pricedTransaction(1, 100000, big.NewInt(1), local)); err != nil {
+ if err := pool.addLocal(pricedTransaction(1, 100000, big.NewInt(1), local)); err != nil {
t.Fatalf("failed to add local transaction: %v", err)
}
- if err := pool.AddRemote(pricedTransaction(1, 100000, big.NewInt(1), remote)); err != nil {
+ if err := pool.addRemote(pricedTransaction(1, 100000, big.NewInt(1), remote)); err != nil {
t.Fatalf("failed to add remote transaction: %v", err)
}
pending, queued := pool.Stats()
@@ -1055,7 +1067,7 @@ func testQueueTimeLimiting(t *testing.T, nolocals bool) {
}
// Queue gapped transactions
- if err := pool.AddLocal(pricedTransaction(4, 100000, big.NewInt(1), local)); err != nil {
+ if err := pool.addLocal(pricedTransaction(4, 100000, big.NewInt(1), local)); err != nil {
t.Fatalf("failed to add remote transaction: %v", err)
}
if err := pool.addRemoteSync(pricedTransaction(4, 100000, big.NewInt(1), remote)); err != nil {
@@ -1064,7 +1076,7 @@ func testQueueTimeLimiting(t *testing.T, nolocals bool) {
time.Sleep(5 * evictionInterval) // A half lifetime pass
// Queue executable transactions, the life cycle should be restarted.
- if err := pool.AddLocal(pricedTransaction(2, 100000, big.NewInt(1), local)); err != nil {
+ if err := pool.addLocal(pricedTransaction(2, 100000, big.NewInt(1), local)); err != nil {
t.Fatalf("failed to add remote transaction: %v", err)
}
if err := pool.addRemoteSync(pricedTransaction(2, 100000, big.NewInt(1), remote)); err != nil {
@@ -1112,7 +1124,7 @@ func TestPendingLimiting(t *testing.T) {
// Create a test account and fund it
pool, key := setupPool()
- defer pool.Stop()
+ defer pool.Close()
account := crypto.PubkeyToAddress(key.PublicKey)
testAddBalance(pool, account, big.NewInt(1000000000000))
@@ -1153,13 +1165,14 @@ func TestPendingGlobalLimiting(t *testing.T) {
// Create the pool to test the limit enforcement with
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabase(rawdb.NewMemoryDatabase()), nil)
- blockchain := newTestBlockChain(1000000, statedb, new(event.Feed))
+ blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed))
config := testTxPoolConfig
config.GlobalSlots = config.AccountSlots * 10
- pool := New(config, params.TestChainConfig, blockchain)
- defer pool.Stop()
+ pool := New(config, blockchain)
+ pool.Init(new(big.Int).SetUint64(config.PriceLimit), blockchain.CurrentBlock())
+ defer pool.Close()
// Create a number of test accounts and fund them
keys := make([]*ecdsa.PrivateKey, 5)
@@ -1179,7 +1192,7 @@ func TestPendingGlobalLimiting(t *testing.T) {
}
}
// Import the batch and verify that limits have been enforced
- pool.AddRemotesSync(txs)
+ pool.addRemotesSync(txs)
pending := 0
for _, list := range pool.pending {
@@ -1201,7 +1214,7 @@ func TestAllowedTxSize(t *testing.T) {
// Create a test account and fund it
pool, key := setupPool()
- defer pool.Stop()
+ defer pool.Close()
account := crypto.PubkeyToAddress(key.PublicKey)
testAddBalance(pool, account, big.NewInt(1000000000))
@@ -1254,15 +1267,16 @@ func TestCapClearsFromAll(t *testing.T) {
// Create the pool to test the limit enforcement with
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabase(rawdb.NewMemoryDatabase()), nil)
- blockchain := newTestBlockChain(1000000, statedb, new(event.Feed))
+ blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed))
config := testTxPoolConfig
config.AccountSlots = 2
config.AccountQueue = 2
config.GlobalSlots = 8
- pool := New(config, params.TestChainConfig, blockchain)
- defer pool.Stop()
+ pool := New(config, blockchain)
+ pool.Init(new(big.Int).SetUint64(config.PriceLimit), blockchain.CurrentBlock())
+ defer pool.Close()
// Create a number of test accounts and fund them
key, _ := crypto.GenerateKey()
@@ -1274,7 +1288,7 @@ func TestCapClearsFromAll(t *testing.T) {
txs = append(txs, transaction(uint64(j), 100000, key))
}
// Import the batch and verify that limits have been enforced
- pool.AddRemotes(txs)
+ pool.addRemotes(txs)
if err := validatePoolInternals(pool); err != nil {
t.Fatalf("pool internal state corrupted: %v", err)
}
@@ -1288,13 +1302,14 @@ func TestPendingMinimumAllowance(t *testing.T) {
// Create the pool to test the limit enforcement with
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabase(rawdb.NewMemoryDatabase()), nil)
- blockchain := newTestBlockChain(1000000, statedb, new(event.Feed))
+ blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed))
config := testTxPoolConfig
config.GlobalSlots = 1
- pool := New(config, params.TestChainConfig, blockchain)
- defer pool.Stop()
+ pool := New(config, blockchain)
+ pool.Init(new(big.Int).SetUint64(config.PriceLimit), blockchain.CurrentBlock())
+ defer pool.Close()
// Create a number of test accounts and fund them
keys := make([]*ecdsa.PrivateKey, 5)
@@ -1314,7 +1329,7 @@ func TestPendingMinimumAllowance(t *testing.T) {
}
}
// Import the batch and verify that limits have been enforced
- pool.AddRemotesSync(txs)
+ pool.addRemotesSync(txs)
for addr, list := range pool.pending {
if list.Len() != int(config.AccountSlots) {
@@ -1336,10 +1351,11 @@ func TestRepricing(t *testing.T) {
// Create the pool to test the pricing enforcement with
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabase(rawdb.NewMemoryDatabase()), nil)
- blockchain := newTestBlockChain(1000000, statedb, new(event.Feed))
+ blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed))
- pool := New(testTxPoolConfig, params.TestChainConfig, blockchain)
- defer pool.Stop()
+ pool := New(testTxPoolConfig, blockchain)
+ pool.Init(new(big.Int).SetUint64(testTxPoolConfig.PriceLimit), blockchain.CurrentBlock())
+ defer pool.Close()
// Keep track of transaction events to ensure all executables get announced
events := make(chan core.NewTxsEvent, 32)
@@ -1370,8 +1386,8 @@ func TestRepricing(t *testing.T) {
ltx := pricedTransaction(0, 100000, big.NewInt(1), keys[3])
// Import the batch and that both pending and queued transactions match up
- pool.AddRemotesSync(txs)
- pool.AddLocal(ltx)
+ pool.addRemotesSync(txs)
+ pool.addLocal(ltx)
pending, queued := pool.Stats()
if pending != 7 {
@@ -1403,14 +1419,14 @@ func TestRepricing(t *testing.T) {
t.Fatalf("pool internal state corrupted: %v", err)
}
// Check that we can't add the old transactions back
- if err := pool.AddRemote(pricedTransaction(1, 100000, big.NewInt(1), keys[0])); !errors.Is(err, ErrUnderpriced) {
- t.Fatalf("adding underpriced pending transaction error mismatch: have %v, want %v", err, ErrUnderpriced)
+ if err := pool.addRemote(pricedTransaction(1, 100000, big.NewInt(1), keys[0])); !errors.Is(err, txpool.ErrUnderpriced) {
+ t.Fatalf("adding underpriced pending transaction error mismatch: have %v, want %v", err, txpool.ErrUnderpriced)
}
- if err := pool.AddRemote(pricedTransaction(0, 100000, big.NewInt(1), keys[1])); !errors.Is(err, ErrUnderpriced) {
- t.Fatalf("adding underpriced pending transaction error mismatch: have %v, want %v", err, ErrUnderpriced)
+ if err := pool.addRemote(pricedTransaction(0, 100000, big.NewInt(1), keys[1])); !errors.Is(err, txpool.ErrUnderpriced) {
+ t.Fatalf("adding underpriced pending transaction error mismatch: have %v, want %v", err, txpool.ErrUnderpriced)
}
- if err := pool.AddRemote(pricedTransaction(2, 100000, big.NewInt(1), keys[2])); !errors.Is(err, ErrUnderpriced) {
- t.Fatalf("adding underpriced queued transaction error mismatch: have %v, want %v", err, ErrUnderpriced)
+ if err := pool.addRemote(pricedTransaction(2, 100000, big.NewInt(1), keys[2])); !errors.Is(err, txpool.ErrUnderpriced) {
+ t.Fatalf("adding underpriced queued transaction error mismatch: have %v, want %v", err, txpool.ErrUnderpriced)
}
if err := validateEvents(events, 0); err != nil {
t.Fatalf("post-reprice event firing failed: %v", err)
@@ -1420,7 +1436,7 @@ func TestRepricing(t *testing.T) {
}
// However we can add local underpriced transactions
tx := pricedTransaction(1, 100000, big.NewInt(1), keys[3])
- if err := pool.AddLocal(tx); err != nil {
+ if err := pool.addLocal(tx); err != nil {
t.Fatalf("failed to add underpriced local transaction: %v", err)
}
if pending, _ = pool.Stats(); pending != 3 {
@@ -1433,13 +1449,13 @@ func TestRepricing(t *testing.T) {
t.Fatalf("pool internal state corrupted: %v", err)
}
// And we can fill gaps with properly priced transactions
- if err := pool.AddRemote(pricedTransaction(1, 100000, big.NewInt(2), keys[0])); err != nil {
+ if err := pool.addRemote(pricedTransaction(1, 100000, big.NewInt(2), keys[0])); err != nil {
t.Fatalf("failed to add pending transaction: %v", err)
}
- if err := pool.AddRemote(pricedTransaction(0, 100000, big.NewInt(2), keys[1])); err != nil {
+ if err := pool.addRemote(pricedTransaction(0, 100000, big.NewInt(2), keys[1])); err != nil {
t.Fatalf("failed to add pending transaction: %v", err)
}
- if err := pool.AddRemote(pricedTransaction(2, 100000, big.NewInt(2), keys[2])); err != nil {
+ if err := pool.addRemote(pricedTransaction(2, 100000, big.NewInt(2), keys[2])); err != nil {
t.Fatalf("failed to add queued transaction: %v", err)
}
if err := validateEvents(events, 5); err != nil {
@@ -1460,7 +1476,7 @@ func TestRepricingDynamicFee(t *testing.T) {
// Create the pool to test the pricing enforcement with
pool, _ := setupPoolWithConfig(eip1559Config)
- defer pool.Stop()
+ defer pool.Close()
// Keep track of transaction events to ensure all executables get announced
events := make(chan core.NewTxsEvent, 32)
@@ -1491,8 +1507,8 @@ func TestRepricingDynamicFee(t *testing.T) {
ltx := dynamicFeeTx(0, 100000, big.NewInt(2), big.NewInt(1), keys[3])
// Import the batch and that both pending and queued transactions match up
- pool.AddRemotesSync(txs)
- pool.AddLocal(ltx)
+ pool.addRemotesSync(txs)
+ pool.addLocal(ltx)
pending, queued := pool.Stats()
if pending != 7 {
@@ -1525,16 +1541,16 @@ func TestRepricingDynamicFee(t *testing.T) {
}
// Check that we can't add the old transactions back
tx := pricedTransaction(1, 100000, big.NewInt(1), keys[0])
- if err := pool.AddRemote(tx); !errors.Is(err, ErrUnderpriced) {
- t.Fatalf("adding underpriced pending transaction error mismatch: have %v, want %v", err, ErrUnderpriced)
+ if err := pool.addRemote(tx); !errors.Is(err, txpool.ErrUnderpriced) {
+ t.Fatalf("adding underpriced pending transaction error mismatch: have %v, want %v", err, txpool.ErrUnderpriced)
}
tx = dynamicFeeTx(0, 100000, big.NewInt(2), big.NewInt(1), keys[1])
- if err := pool.AddRemote(tx); !errors.Is(err, ErrUnderpriced) {
- t.Fatalf("adding underpriced pending transaction error mismatch: have %v, want %v", err, ErrUnderpriced)
+ if err := pool.addRemote(tx); !errors.Is(err, txpool.ErrUnderpriced) {
+ t.Fatalf("adding underpriced pending transaction error mismatch: have %v, want %v", err, txpool.ErrUnderpriced)
}
tx = dynamicFeeTx(2, 100000, big.NewInt(1), big.NewInt(1), keys[2])
- if err := pool.AddRemote(tx); !errors.Is(err, ErrUnderpriced) {
- t.Fatalf("adding underpriced queued transaction error mismatch: have %v, want %v", err, ErrUnderpriced)
+ if err := pool.addRemote(tx); !errors.Is(err, txpool.ErrUnderpriced) {
+ t.Fatalf("adding underpriced queued transaction error mismatch: have %v, want %v", err, txpool.ErrUnderpriced)
}
if err := validateEvents(events, 0); err != nil {
t.Fatalf("post-reprice event firing failed: %v", err)
@@ -1544,7 +1560,7 @@ func TestRepricingDynamicFee(t *testing.T) {
}
// However we can add local underpriced transactions
tx = dynamicFeeTx(1, 100000, big.NewInt(1), big.NewInt(1), keys[3])
- if err := pool.AddLocal(tx); err != nil {
+ if err := pool.addLocal(tx); err != nil {
t.Fatalf("failed to add underpriced local transaction: %v", err)
}
if pending, _ = pool.Stats(); pending != 3 {
@@ -1558,15 +1574,15 @@ func TestRepricingDynamicFee(t *testing.T) {
}
// And we can fill gaps with properly priced transactions
tx = pricedTransaction(1, 100000, big.NewInt(2), keys[0])
- if err := pool.AddRemote(tx); err != nil {
+ if err := pool.addRemote(tx); err != nil {
t.Fatalf("failed to add pending transaction: %v", err)
}
tx = dynamicFeeTx(0, 100000, big.NewInt(3), big.NewInt(2), keys[1])
- if err := pool.AddRemote(tx); err != nil {
+ if err := pool.addRemote(tx); err != nil {
t.Fatalf("failed to add pending transaction: %v", err)
}
tx = dynamicFeeTx(2, 100000, big.NewInt(2), big.NewInt(2), keys[2])
- if err := pool.AddRemote(tx); err != nil {
+ if err := pool.addRemote(tx); err != nil {
t.Fatalf("failed to add queued transaction: %v", err)
}
if err := validateEvents(events, 5); err != nil {
@@ -1584,10 +1600,11 @@ func TestRepricingKeepsLocals(t *testing.T) {
// Create the pool to test the pricing enforcement with
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabase(rawdb.NewMemoryDatabase()), nil)
- blockchain := newTestBlockChain(1000000, statedb, new(event.Feed))
+ blockchain := newTestBlockChain(eip1559Config, 1000000, statedb, new(event.Feed))
- pool := New(testTxPoolConfig, eip1559Config, blockchain)
- defer pool.Stop()
+ pool := New(testTxPoolConfig, blockchain)
+ pool.Init(new(big.Int).SetUint64(testTxPoolConfig.PriceLimit), blockchain.CurrentBlock())
+ defer pool.Close()
// Create a number of test accounts and fund them
keys := make([]*ecdsa.PrivateKey, 3)
@@ -1599,23 +1616,23 @@ func TestRepricingKeepsLocals(t *testing.T) {
for i := uint64(0); i < 500; i++ {
// Add pending transaction.
pendingTx := pricedTransaction(i, 100000, big.NewInt(int64(i)), keys[2])
- if err := pool.AddLocal(pendingTx); err != nil {
+ if err := pool.addLocal(pendingTx); err != nil {
t.Fatal(err)
}
// Add queued transaction.
queuedTx := pricedTransaction(i+501, 100000, big.NewInt(int64(i)), keys[2])
- if err := pool.AddLocal(queuedTx); err != nil {
+ if err := pool.addLocal(queuedTx); err != nil {
t.Fatal(err)
}
// Add pending dynamic fee transaction.
pendingTx = dynamicFeeTx(i, 100000, big.NewInt(int64(i)+1), big.NewInt(int64(i)), keys[1])
- if err := pool.AddLocal(pendingTx); err != nil {
+ if err := pool.addLocal(pendingTx); err != nil {
t.Fatal(err)
}
// Add queued dynamic fee transaction.
queuedTx = dynamicFeeTx(i+501, 100000, big.NewInt(int64(i)+1), big.NewInt(int64(i)), keys[1])
- if err := pool.AddLocal(queuedTx); err != nil {
+ if err := pool.addLocal(queuedTx); err != nil {
t.Fatal(err)
}
}
@@ -1657,14 +1674,15 @@ func TestUnderpricing(t *testing.T) {
// Create the pool to test the pricing enforcement with
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabase(rawdb.NewMemoryDatabase()), nil)
- blockchain := newTestBlockChain(1000000, statedb, new(event.Feed))
+ blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed))
config := testTxPoolConfig
config.GlobalSlots = 2
config.GlobalQueue = 2
- pool := New(config, params.TestChainConfig, blockchain)
- defer pool.Stop()
+ pool := New(config, blockchain)
+ pool.Init(new(big.Int).SetUint64(config.PriceLimit), blockchain.CurrentBlock())
+ defer pool.Close()
// Keep track of transaction events to ensure all executables get announced
events := make(chan core.NewTxsEvent, 32)
@@ -1688,8 +1706,8 @@ func TestUnderpricing(t *testing.T) {
ltx := pricedTransaction(0, 100000, big.NewInt(1), keys[2])
// Import the batch and that both pending and queued transactions match up
- pool.AddRemotes(txs)
- pool.AddLocal(ltx)
+ pool.addRemotes(txs)
+ pool.addLocal(ltx)
pending, queued := pool.Stats()
if pending != 3 {
@@ -1705,26 +1723,26 @@ func TestUnderpricing(t *testing.T) {
t.Fatalf("pool internal state corrupted: %v", err)
}
// Ensure that adding an underpriced transaction on block limit fails
- if err := pool.AddRemote(pricedTransaction(0, 100000, big.NewInt(1), keys[1])); !errors.Is(err, ErrUnderpriced) {
- t.Fatalf("adding underpriced pending transaction error mismatch: have %v, want %v", err, ErrUnderpriced)
+ if err := pool.addRemote(pricedTransaction(0, 100000, big.NewInt(1), keys[1])); !errors.Is(err, txpool.ErrUnderpriced) {
+ t.Fatalf("adding underpriced pending transaction error mismatch: have %v, want %v", err, txpool.ErrUnderpriced)
}
// Replace a future transaction with a future transaction
- if err := pool.AddRemote(pricedTransaction(1, 100000, big.NewInt(2), keys[1])); err != nil { // +K1:1 => -K1:1 => Pend K0:0, K0:1, K2:0; Que K1:1
+ if err := pool.addRemote(pricedTransaction(1, 100000, big.NewInt(2), keys[1])); err != nil { // +K1:1 => -K1:1 => Pend K0:0, K0:1, K2:0; Que K1:1
t.Fatalf("failed to add well priced transaction: %v", err)
}
// Ensure that adding high priced transactions drops cheap ones, but not own
- if err := pool.AddRemote(pricedTransaction(0, 100000, big.NewInt(3), keys[1])); err != nil { // +K1:0 => -K1:1 => Pend K0:0, K0:1, K1:0, K2:0; Que -
+ if err := pool.addRemote(pricedTransaction(0, 100000, big.NewInt(3), keys[1])); err != nil { // +K1:0 => -K1:1 => Pend K0:0, K0:1, K1:0, K2:0; Que -
t.Fatalf("failed to add well priced transaction: %v", err)
}
- if err := pool.AddRemote(pricedTransaction(2, 100000, big.NewInt(4), keys[1])); err != nil { // +K1:2 => -K0:0 => Pend K1:0, K2:0; Que K0:1 K1:2
+ if err := pool.addRemote(pricedTransaction(2, 100000, big.NewInt(4), keys[1])); err != nil { // +K1:2 => -K0:0 => Pend K1:0, K2:0; Que K0:1 K1:2
t.Fatalf("failed to add well priced transaction: %v", err)
}
- if err := pool.AddRemote(pricedTransaction(3, 100000, big.NewInt(5), keys[1])); err != nil { // +K1:3 => -K0:1 => Pend K1:0, K2:0; Que K1:2 K1:3
+ if err := pool.addRemote(pricedTransaction(3, 100000, big.NewInt(5), keys[1])); err != nil { // +K1:3 => -K0:1 => Pend K1:0, K2:0; Que K1:2 K1:3
t.Fatalf("failed to add well priced transaction: %v", err)
}
// Ensure that replacing a pending transaction with a future transaction fails
- if err := pool.AddRemote(pricedTransaction(5, 100000, big.NewInt(6), keys[1])); err != ErrFutureReplacePending {
- t.Fatalf("adding future replace transaction error mismatch: have %v, want %v", err, ErrFutureReplacePending)
+ if err := pool.addRemote(pricedTransaction(5, 100000, big.NewInt(6), keys[1])); err != txpool.ErrFutureReplacePending {
+ t.Fatalf("adding future replace transaction error mismatch: have %v, want %v", err, txpool.ErrFutureReplacePending)
}
pending, queued = pool.Stats()
if pending != 2 {
@@ -1741,11 +1759,11 @@ func TestUnderpricing(t *testing.T) {
}
// Ensure that adding local transactions can push out even higher priced ones
ltx = pricedTransaction(1, 100000, big.NewInt(0), keys[2])
- if err := pool.AddLocal(ltx); err != nil {
+ if err := pool.addLocal(ltx); err != nil {
t.Fatalf("failed to append underpriced local transaction: %v", err)
}
ltx = pricedTransaction(0, 100000, big.NewInt(0), keys[3])
- if err := pool.AddLocal(ltx); err != nil {
+ if err := pool.addLocal(ltx); err != nil {
t.Fatalf("failed to add new underpriced local transaction: %v", err)
}
pending, queued = pool.Stats()
@@ -1771,14 +1789,15 @@ func TestStableUnderpricing(t *testing.T) {
// Create the pool to test the pricing enforcement with
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabase(rawdb.NewMemoryDatabase()), nil)
- blockchain := newTestBlockChain(1000000, statedb, new(event.Feed))
+ blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed))
config := testTxPoolConfig
config.GlobalSlots = 128
config.GlobalQueue = 0
- pool := New(config, params.TestChainConfig, blockchain)
- defer pool.Stop()
+ pool := New(config, blockchain)
+ pool.Init(new(big.Int).SetUint64(config.PriceLimit), blockchain.CurrentBlock())
+ defer pool.Close()
// Keep track of transaction events to ensure all executables get announced
events := make(chan core.NewTxsEvent, 32)
@@ -1796,7 +1815,7 @@ func TestStableUnderpricing(t *testing.T) {
for i := uint64(0); i < config.GlobalSlots; i++ {
txs = append(txs, pricedTransaction(i, 100000, big.NewInt(1), keys[0]))
}
- pool.AddRemotesSync(txs)
+ pool.addRemotesSync(txs)
pending, queued := pool.Stats()
if pending != int(config.GlobalSlots) {
@@ -1839,7 +1858,7 @@ func TestUnderpricingDynamicFee(t *testing.T) {
t.Parallel()
pool, _ := setupPoolWithConfig(eip1559Config)
- defer pool.Stop()
+ defer pool.Close()
pool.config.GlobalSlots = 2
pool.config.GlobalQueue = 2
@@ -1866,8 +1885,8 @@ func TestUnderpricingDynamicFee(t *testing.T) {
ltx := dynamicFeeTx(0, 100000, big.NewInt(2), big.NewInt(1), keys[2])
// Import the batch and that both pending and queued transactions match up
- pool.AddRemotes(txs) // Pend K0:0, K0:1; Que K1:1
- pool.AddLocal(ltx) // +K2:0 => Pend K0:0, K0:1, K2:0; Que K1:1
+ pool.addRemotes(txs) // Pend K0:0, K0:1; Que K1:1
+ pool.addLocal(ltx) // +K2:0 => Pend K0:0, K0:1, K2:0; Que K1:1
pending, queued := pool.Stats()
if pending != 3 {
@@ -1885,22 +1904,22 @@ func TestUnderpricingDynamicFee(t *testing.T) {
// Ensure that adding an underpriced transaction fails
tx := dynamicFeeTx(0, 100000, big.NewInt(2), big.NewInt(1), keys[1])
- if err := pool.AddRemote(tx); !errors.Is(err, ErrUnderpriced) { // Pend K0:0, K0:1, K2:0; Que K1:1
- t.Fatalf("adding underpriced pending transaction error mismatch: have %v, want %v", err, ErrUnderpriced)
+ if err := pool.addRemote(tx); !errors.Is(err, txpool.ErrUnderpriced) { // Pend K0:0, K0:1, K2:0; Que K1:1
+ t.Fatalf("adding underpriced pending transaction error mismatch: have %v, want %v", err, txpool.ErrUnderpriced)
}
// Ensure that adding high priced transactions drops cheap ones, but not own
tx = pricedTransaction(0, 100000, big.NewInt(2), keys[1])
- if err := pool.AddRemote(tx); err != nil { // +K1:0, -K1:1 => Pend K0:0, K0:1, K1:0, K2:0; Que -
+ if err := pool.addRemote(tx); err != nil { // +K1:0, -K1:1 => Pend K0:0, K0:1, K1:0, K2:0; Que -
t.Fatalf("failed to add well priced transaction: %v", err)
}
tx = pricedTransaction(1, 100000, big.NewInt(3), keys[1])
- if err := pool.AddRemote(tx); err != nil { // +K1:2, -K0:1 => Pend K0:0 K1:0, K2:0; Que K1:2
+ if err := pool.addRemote(tx); err != nil { // +K1:2, -K0:1 => Pend K0:0 K1:0, K2:0; Que K1:2
t.Fatalf("failed to add well priced transaction: %v", err)
}
tx = dynamicFeeTx(2, 100000, big.NewInt(4), big.NewInt(1), keys[1])
- if err := pool.AddRemote(tx); err != nil { // +K1:3, -K1:0 => Pend K0:0 K2:0; Que K1:2 K1:3
+ if err := pool.addRemote(tx); err != nil { // +K1:3, -K1:0 => Pend K0:0 K2:0; Que K1:2 K1:3
t.Fatalf("failed to add well priced transaction: %v", err)
}
pending, queued = pool.Stats()
@@ -1918,11 +1937,11 @@ func TestUnderpricingDynamicFee(t *testing.T) {
}
// Ensure that adding local transactions can push out even higher priced ones
ltx = dynamicFeeTx(1, 100000, big.NewInt(0), big.NewInt(0), keys[2])
- if err := pool.AddLocal(ltx); err != nil {
+ if err := pool.addLocal(ltx); err != nil {
t.Fatalf("failed to append underpriced local transaction: %v", err)
}
ltx = dynamicFeeTx(0, 100000, big.NewInt(0), big.NewInt(0), keys[3])
- if err := pool.AddLocal(ltx); err != nil {
+ if err := pool.addLocal(ltx); err != nil {
t.Fatalf("failed to add new underpriced local transaction: %v", err)
}
pending, queued = pool.Stats()
@@ -1946,7 +1965,7 @@ func TestDualHeapEviction(t *testing.T) {
t.Parallel()
pool, _ := setupPoolWithConfig(eip1559Config)
- defer pool.Stop()
+ defer pool.Close()
pool.config.GlobalSlots = 10
pool.config.GlobalQueue = 10
@@ -1975,7 +1994,7 @@ func TestDualHeapEviction(t *testing.T) {
tx = dynamicFeeTx(0, 100000, big.NewInt(int64(baseFee+200+i)), big.NewInt(1), key)
highCap = tx
}
- pool.AddRemotesSync([]*types.Transaction{tx})
+ pool.addRemotesSync([]*types.Transaction{tx})
}
pending, queued := pool.Stats()
if pending+queued != 20 {
@@ -2003,10 +2022,11 @@ func TestDeduplication(t *testing.T) {
// Create the pool to test the pricing enforcement with
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabase(rawdb.NewMemoryDatabase()), nil)
- blockchain := newTestBlockChain(1000000, statedb, new(event.Feed))
+ blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed))
- pool := New(testTxPoolConfig, params.TestChainConfig, blockchain)
- defer pool.Stop()
+ pool := New(testTxPoolConfig, blockchain)
+ pool.Init(new(big.Int).SetUint64(testTxPoolConfig.PriceLimit), blockchain.CurrentBlock())
+ defer pool.Close()
// Create a test account to add transactions with
key, _ := crypto.GenerateKey()
@@ -2021,7 +2041,7 @@ func TestDeduplication(t *testing.T) {
for i := 0; i < len(txs); i += 2 {
firsts = append(firsts, txs[i])
}
- errs := pool.AddRemotesSync(firsts)
+ errs := pool.addRemotesSync(firsts)
if len(errs) != len(firsts) {
t.Fatalf("first add mismatching result count: have %d, want %d", len(errs), len(firsts))
}
@@ -2038,7 +2058,7 @@ func TestDeduplication(t *testing.T) {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, len(txs)/2-1)
}
// Try to add all of them now and ensure previous ones error out as knowns
- errs = pool.AddRemotesSync(txs)
+ errs = pool.addRemotesSync(txs)
if len(errs) != len(txs) {
t.Fatalf("all add mismatching result count: have %d, want %d", len(errs), len(txs))
}
@@ -2069,10 +2089,11 @@ func TestReplacement(t *testing.T) {
// Create the pool to test the pricing enforcement with
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabase(rawdb.NewMemoryDatabase()), nil)
- blockchain := newTestBlockChain(1000000, statedb, new(event.Feed))
+ blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed))
- pool := New(testTxPoolConfig, params.TestChainConfig, blockchain)
- defer pool.Stop()
+ pool := New(testTxPoolConfig, blockchain)
+ pool.Init(new(big.Int).SetUint64(testTxPoolConfig.PriceLimit), blockchain.CurrentBlock())
+ defer pool.Close()
// Keep track of transaction events to ensure all executables get announced
events := make(chan core.NewTxsEvent, 32)
@@ -2090,10 +2111,10 @@ func TestReplacement(t *testing.T) {
if err := pool.addRemoteSync(pricedTransaction(0, 100000, big.NewInt(1), key)); err != nil {
t.Fatalf("failed to add original cheap pending transaction: %v", err)
}
- if err := pool.AddRemote(pricedTransaction(0, 100001, big.NewInt(1), key)); err != ErrReplaceUnderpriced {
- t.Fatalf("original cheap pending transaction replacement error mismatch: have %v, want %v", err, ErrReplaceUnderpriced)
+ if err := pool.addRemote(pricedTransaction(0, 100001, big.NewInt(1), key)); err != txpool.ErrReplaceUnderpriced {
+ t.Fatalf("original cheap pending transaction replacement error mismatch: have %v, want %v", err, txpool.ErrReplaceUnderpriced)
}
- if err := pool.AddRemote(pricedTransaction(0, 100000, big.NewInt(2), key)); err != nil {
+ if err := pool.addRemote(pricedTransaction(0, 100000, big.NewInt(2), key)); err != nil {
t.Fatalf("failed to replace original cheap pending transaction: %v", err)
}
if err := validateEvents(events, 2); err != nil {
@@ -2103,10 +2124,10 @@ func TestReplacement(t *testing.T) {
if err := pool.addRemoteSync(pricedTransaction(0, 100000, big.NewInt(price), key)); err != nil {
t.Fatalf("failed to add original proper pending transaction: %v", err)
}
- if err := pool.AddRemote(pricedTransaction(0, 100001, big.NewInt(threshold-1), key)); err != ErrReplaceUnderpriced {
- t.Fatalf("original proper pending transaction replacement error mismatch: have %v, want %v", err, ErrReplaceUnderpriced)
+ if err := pool.addRemote(pricedTransaction(0, 100001, big.NewInt(threshold-1), key)); err != txpool.ErrReplaceUnderpriced {
+ t.Fatalf("original proper pending transaction replacement error mismatch: have %v, want %v", err, txpool.ErrReplaceUnderpriced)
}
- if err := pool.AddRemote(pricedTransaction(0, 100000, big.NewInt(threshold), key)); err != nil {
+ if err := pool.addRemote(pricedTransaction(0, 100000, big.NewInt(threshold), key)); err != nil {
t.Fatalf("failed to replace original proper pending transaction: %v", err)
}
if err := validateEvents(events, 2); err != nil {
@@ -2114,23 +2135,23 @@ func TestReplacement(t *testing.T) {
}
// Add queued transactions, ensuring the minimum price bump is enforced for replacement (for ultra low prices too)
- if err := pool.AddRemote(pricedTransaction(2, 100000, big.NewInt(1), key)); err != nil {
+ if err := pool.addRemote(pricedTransaction(2, 100000, big.NewInt(1), key)); err != nil {
t.Fatalf("failed to add original cheap queued transaction: %v", err)
}
- if err := pool.AddRemote(pricedTransaction(2, 100001, big.NewInt(1), key)); err != ErrReplaceUnderpriced {
- t.Fatalf("original cheap queued transaction replacement error mismatch: have %v, want %v", err, ErrReplaceUnderpriced)
+ if err := pool.addRemote(pricedTransaction(2, 100001, big.NewInt(1), key)); err != txpool.ErrReplaceUnderpriced {
+ t.Fatalf("original cheap queued transaction replacement error mismatch: have %v, want %v", err, txpool.ErrReplaceUnderpriced)
}
- if err := pool.AddRemote(pricedTransaction(2, 100000, big.NewInt(2), key)); err != nil {
+ if err := pool.addRemote(pricedTransaction(2, 100000, big.NewInt(2), key)); err != nil {
t.Fatalf("failed to replace original cheap queued transaction: %v", err)
}
- if err := pool.AddRemote(pricedTransaction(2, 100000, big.NewInt(price), key)); err != nil {
+ if err := pool.addRemote(pricedTransaction(2, 100000, big.NewInt(price), key)); err != nil {
t.Fatalf("failed to add original proper queued transaction: %v", err)
}
- if err := pool.AddRemote(pricedTransaction(2, 100001, big.NewInt(threshold-1), key)); err != ErrReplaceUnderpriced {
- t.Fatalf("original proper queued transaction replacement error mismatch: have %v, want %v", err, ErrReplaceUnderpriced)
+ if err := pool.addRemote(pricedTransaction(2, 100001, big.NewInt(threshold-1), key)); err != txpool.ErrReplaceUnderpriced {
+ t.Fatalf("original proper queued transaction replacement error mismatch: have %v, want %v", err, txpool.ErrReplaceUnderpriced)
}
- if err := pool.AddRemote(pricedTransaction(2, 100000, big.NewInt(threshold), key)); err != nil {
+ if err := pool.addRemote(pricedTransaction(2, 100000, big.NewInt(threshold), key)); err != nil {
t.Fatalf("failed to replace original proper queued transaction: %v", err)
}
@@ -2149,7 +2170,7 @@ func TestReplacementDynamicFee(t *testing.T) {
// Create the pool to test the pricing enforcement with
pool, key := setupPoolWithConfig(eip1559Config)
- defer pool.Stop()
+ defer pool.Close()
testAddBalance(pool, crypto.PubkeyToAddress(key.PublicKey), big.NewInt(1000000000))
// Keep track of transaction events to ensure all executables get announced
@@ -2191,12 +2212,12 @@ func TestReplacementDynamicFee(t *testing.T) {
}
// 2. Don't bump tip or feecap => discard
tx = dynamicFeeTx(nonce, 100001, big.NewInt(2), big.NewInt(1), key)
- if err := pool.AddRemote(tx); err != ErrReplaceUnderpriced {
- t.Fatalf("original cheap %s transaction replacement error mismatch: have %v, want %v", stage, err, ErrReplaceUnderpriced)
+ if err := pool.addRemote(tx); err != txpool.ErrReplaceUnderpriced {
+ t.Fatalf("original cheap %s transaction replacement error mismatch: have %v, want %v", stage, err, txpool.ErrReplaceUnderpriced)
}
// 3. Bump both more than min => accept
tx = dynamicFeeTx(nonce, 100000, big.NewInt(3), big.NewInt(2), key)
- if err := pool.AddRemote(tx); err != nil {
+ if err := pool.addRemote(tx); err != nil {
t.Fatalf("failed to replace original cheap %s transaction: %v", stage, err)
}
// 4. Check events match expected (2 new executable txs during pending, 0 during queue)
@@ -2214,27 +2235,27 @@ func TestReplacementDynamicFee(t *testing.T) {
}
// 6. Bump tip max allowed so it's still underpriced => discard
tx = dynamicFeeTx(nonce, 100000, big.NewInt(gasFeeCap), big.NewInt(tipThreshold-1), key)
- if err := pool.AddRemote(tx); err != ErrReplaceUnderpriced {
- t.Fatalf("original proper %s transaction replacement error mismatch: have %v, want %v", stage, err, ErrReplaceUnderpriced)
+ if err := pool.addRemote(tx); err != txpool.ErrReplaceUnderpriced {
+ t.Fatalf("original proper %s transaction replacement error mismatch: have %v, want %v", stage, err, txpool.ErrReplaceUnderpriced)
}
// 7. Bump fee cap max allowed so it's still underpriced => discard
tx = dynamicFeeTx(nonce, 100000, big.NewInt(feeCapThreshold-1), big.NewInt(gasTipCap), key)
- if err := pool.AddRemote(tx); err != ErrReplaceUnderpriced {
- t.Fatalf("original proper %s transaction replacement error mismatch: have %v, want %v", stage, err, ErrReplaceUnderpriced)
+ if err := pool.addRemote(tx); err != txpool.ErrReplaceUnderpriced {
+ t.Fatalf("original proper %s transaction replacement error mismatch: have %v, want %v", stage, err, txpool.ErrReplaceUnderpriced)
}
// 8. Bump tip min for acceptance => accept
tx = dynamicFeeTx(nonce, 100000, big.NewInt(gasFeeCap), big.NewInt(tipThreshold), key)
- if err := pool.AddRemote(tx); err != ErrReplaceUnderpriced {
- t.Fatalf("original proper %s transaction replacement error mismatch: have %v, want %v", stage, err, ErrReplaceUnderpriced)
+ if err := pool.addRemote(tx); err != txpool.ErrReplaceUnderpriced {
+ t.Fatalf("original proper %s transaction replacement error mismatch: have %v, want %v", stage, err, txpool.ErrReplaceUnderpriced)
}
// 9. Bump fee cap min for acceptance => accept
tx = dynamicFeeTx(nonce, 100000, big.NewInt(feeCapThreshold), big.NewInt(gasTipCap), key)
- if err := pool.AddRemote(tx); err != ErrReplaceUnderpriced {
- t.Fatalf("original proper %s transaction replacement error mismatch: have %v, want %v", stage, err, ErrReplaceUnderpriced)
+ if err := pool.addRemote(tx); err != txpool.ErrReplaceUnderpriced {
+ t.Fatalf("original proper %s transaction replacement error mismatch: have %v, want %v", stage, err, txpool.ErrReplaceUnderpriced)
}
// 10. Check events match expected (3 new executable txs during pending, 0 during queue)
tx = dynamicFeeTx(nonce, 100000, big.NewInt(feeCapThreshold), big.NewInt(tipThreshold), key)
- if err := pool.AddRemote(tx); err != nil {
+ if err := pool.addRemote(tx); err != nil {
t.Fatalf("failed to replace original cheap %s transaction: %v", stage, err)
}
// 11. Check events match expected (3 new executable txs during pending, 0 during queue)
@@ -2274,14 +2295,15 @@ func testJournaling(t *testing.T, nolocals bool) {
// Create the original pool to inject transaction into the journal
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabase(rawdb.NewMemoryDatabase()), nil)
- blockchain := newTestBlockChain(1000000, statedb, new(event.Feed))
+ blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed))
config := testTxPoolConfig
config.NoLocals = nolocals
config.Journal = journal
config.Rejournal = time.Second
- pool := New(config, params.TestChainConfig, blockchain)
+ pool := New(config, blockchain)
+ pool.Init(new(big.Int).SetUint64(config.PriceLimit), blockchain.CurrentBlock())
// Create two test accounts to ensure remotes expire but locals do not
local, _ := crypto.GenerateKey()
@@ -2291,13 +2313,13 @@ func testJournaling(t *testing.T, nolocals bool) {
testAddBalance(pool, crypto.PubkeyToAddress(remote.PublicKey), big.NewInt(1000000000))
// Add three local and a remote transactions and ensure they are queued up
- if err := pool.AddLocal(pricedTransaction(0, 100000, big.NewInt(1), local)); err != nil {
+ if err := pool.addLocal(pricedTransaction(0, 100000, big.NewInt(1), local)); err != nil {
t.Fatalf("failed to add local transaction: %v", err)
}
- if err := pool.AddLocal(pricedTransaction(1, 100000, big.NewInt(1), local)); err != nil {
+ if err := pool.addLocal(pricedTransaction(1, 100000, big.NewInt(1), local)); err != nil {
t.Fatalf("failed to add local transaction: %v", err)
}
- if err := pool.AddLocal(pricedTransaction(2, 100000, big.NewInt(1), local)); err != nil {
+ if err := pool.addLocal(pricedTransaction(2, 100000, big.NewInt(1), local)); err != nil {
t.Fatalf("failed to add local transaction: %v", err)
}
if err := pool.addRemoteSync(pricedTransaction(0, 100000, big.NewInt(1), remote)); err != nil {
@@ -2314,11 +2336,12 @@ func testJournaling(t *testing.T, nolocals bool) {
t.Fatalf("pool internal state corrupted: %v", err)
}
// Terminate the old pool, bump the local nonce, create a new pool and ensure relevant transaction survive
- pool.Stop()
+ pool.Close()
statedb.SetNonce(crypto.PubkeyToAddress(local.PublicKey), 1)
- blockchain = newTestBlockChain(1000000, statedb, new(event.Feed))
+ blockchain = newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed))
- pool = New(config, params.TestChainConfig, blockchain)
+ pool = New(config, blockchain)
+ pool.Init(new(big.Int).SetUint64(config.PriceLimit), blockchain.CurrentBlock())
pending, queued = pool.Stats()
if queued != 0 {
@@ -2340,11 +2363,12 @@ func testJournaling(t *testing.T, nolocals bool) {
statedb.SetNonce(crypto.PubkeyToAddress(local.PublicKey), 2)
<-pool.requestReset(nil, nil)
time.Sleep(2 * config.Rejournal)
- pool.Stop()
+ pool.Close()
statedb.SetNonce(crypto.PubkeyToAddress(local.PublicKey), 1)
- blockchain = newTestBlockChain(1000000, statedb, new(event.Feed))
- pool = New(config, params.TestChainConfig, blockchain)
+ blockchain = newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed))
+ pool = New(config, blockchain)
+ pool.Init(new(big.Int).SetUint64(config.PriceLimit), blockchain.CurrentBlock())
pending, queued = pool.Stats()
if pending != 0 {
@@ -2362,7 +2386,7 @@ func testJournaling(t *testing.T, nolocals bool) {
if err := validatePoolInternals(pool); err != nil {
t.Fatalf("pool internal state corrupted: %v", err)
}
- pool.Stop()
+ pool.Close()
}
// TestStatusCheck tests that the pool can correctly retrieve the
@@ -2372,10 +2396,11 @@ func TestStatusCheck(t *testing.T) {
// Create the pool to test the status retrievals with
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabase(rawdb.NewMemoryDatabase()), nil)
- blockchain := newTestBlockChain(1000000, statedb, new(event.Feed))
+ blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed))
- pool := New(testTxPoolConfig, params.TestChainConfig, blockchain)
- defer pool.Stop()
+ pool := New(testTxPoolConfig, blockchain)
+ pool.Init(new(big.Int).SetUint64(testTxPoolConfig.PriceLimit), blockchain.CurrentBlock())
+ defer pool.Close()
// Create the test accounts to check various transaction statuses with
keys := make([]*ecdsa.PrivateKey, 3)
@@ -2392,7 +2417,7 @@ func TestStatusCheck(t *testing.T) {
txs = append(txs, pricedTransaction(2, 100000, big.NewInt(1), keys[2])) // Queued only
// Import the transaction and ensure they are correctly added
- pool.AddRemotesSync(txs)
+ pool.addRemotesSync(txs)
pending, queued := pool.Stats()
if pending != 2 {
@@ -2410,13 +2435,11 @@ func TestStatusCheck(t *testing.T) {
hashes[i] = tx.Hash()
}
hashes = append(hashes, common.Hash{})
+ expect := []txpool.TxStatus{txpool.TxStatusPending, txpool.TxStatusPending, txpool.TxStatusQueued, txpool.TxStatusQueued, txpool.TxStatusUnknown}
- statuses := pool.Status(hashes)
- expect := []TxStatus{TxStatusPending, TxStatusPending, TxStatusQueued, TxStatusQueued, TxStatusUnknown}
-
- for i := 0; i < len(statuses); i++ {
- if statuses[i] != expect[i] {
- t.Errorf("transaction %d: status mismatch: have %v, want %v", i, statuses[i], expect[i])
+ for i := 0; i < len(hashes); i++ {
+ if status := pool.Status(hashes[i]); status != expect[i] {
+ t.Errorf("transaction %d: status mismatch: have %v, want %v", i, status, expect[i])
}
}
}
@@ -2448,7 +2471,7 @@ func BenchmarkPendingDemotion10000(b *testing.B) { benchmarkPendingDemotion(b, 1
func benchmarkPendingDemotion(b *testing.B, size int) {
// Add a batch of transactions to a pool one by one
pool, key := setupPool()
- defer pool.Stop()
+ defer pool.Close()
account := crypto.PubkeyToAddress(key.PublicKey)
testAddBalance(pool, account, big.NewInt(1000000))
@@ -2473,7 +2496,7 @@ func BenchmarkFuturePromotion10000(b *testing.B) { benchmarkFuturePromotion(b, 1
func benchmarkFuturePromotion(b *testing.B, size int) {
// Add a batch of transactions to a pool one by one
pool, key := setupPool()
- defer pool.Stop()
+ defer pool.Close()
account := crypto.PubkeyToAddress(key.PublicKey)
testAddBalance(pool, account, big.NewInt(1000000))
@@ -2501,7 +2524,7 @@ func BenchmarkBatchLocalInsert10000(b *testing.B) { benchmarkBatchInsert(b, 1000
func benchmarkBatchInsert(b *testing.B, size int, local bool) {
// Generate a batch of transactions to enqueue into the pool
pool, key := setupPool()
- defer pool.Stop()
+ defer pool.Close()
account := crypto.PubkeyToAddress(key.PublicKey)
testAddBalance(pool, account, big.NewInt(1000000000000000000))
@@ -2517,9 +2540,9 @@ func benchmarkBatchInsert(b *testing.B, size int, local bool) {
b.ResetTimer()
for _, batch := range batches {
if local {
- pool.AddLocals(batch)
+ pool.addLocals(batch)
} else {
- pool.AddRemotes(batch)
+ pool.addRemotes(batch)
}
}
}
@@ -2547,15 +2570,15 @@ func BenchmarkInsertRemoteWithAllLocals(b *testing.B) {
pool, _ := setupPool()
testAddBalance(pool, account, big.NewInt(100000000))
for _, local := range locals {
- pool.AddLocal(local)
+ pool.addLocal(local)
}
b.StartTimer()
// Assign a high enough balance for testing
testAddBalance(pool, remoteAddr, big.NewInt(100000000))
for i := 0; i < len(remotes); i++ {
- pool.AddRemotes([]*types.Transaction{remotes[i]})
+ pool.addRemotes([]*types.Transaction{remotes[i]})
}
- pool.Stop()
+ pool.Close()
}
}
@@ -2563,7 +2586,7 @@ func BenchmarkInsertRemoteWithAllLocals(b *testing.B) {
func BenchmarkMultiAccountBatchInsert(b *testing.B) {
// Generate a batch of transactions to enqueue into the pool
pool, _ := setupPool()
- defer pool.Stop()
+ defer pool.Close()
b.ReportAllocs()
batches := make(types.Transactions, b.N)
for i := 0; i < b.N; i++ {
@@ -2576,6 +2599,6 @@ func BenchmarkMultiAccountBatchInsert(b *testing.B) {
// Benchmark importing the transactions into the queue
b.ResetTimer()
for _, tx := range batches {
- pool.AddRemotesSync([]*types.Transaction{tx})
+ pool.addRemotesSync([]*types.Transaction{tx})
}
}
diff --git a/core/txpool/list.go b/core/txpool/legacypool/list.go
similarity index 99%
rename from core/txpool/list.go
rename to core/txpool/legacypool/list.go
index fae7c2fca..d3b6768f8 100644
--- a/core/txpool/list.go
+++ b/core/txpool/legacypool/list.go
@@ -14,7 +14,7 @@
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see .
-package txpool
+package legacypool
import (
"container/heap"
diff --git a/core/txpool/list_test.go b/core/txpool/legacypool/list_test.go
similarity index 99%
rename from core/txpool/list_test.go
rename to core/txpool/legacypool/list_test.go
index 4e1a5d5e8..b5cd34b23 100644
--- a/core/txpool/list_test.go
+++ b/core/txpool/legacypool/list_test.go
@@ -14,7 +14,7 @@
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see .
-package txpool
+package legacypool
import (
"math/big"
diff --git a/core/txpool/noncer.go b/core/txpool/legacypool/noncer.go
similarity index 99%
rename from core/txpool/noncer.go
rename to core/txpool/legacypool/noncer.go
index ba7fbedad..2c65dd2ca 100644
--- a/core/txpool/noncer.go
+++ b/core/txpool/legacypool/noncer.go
@@ -14,7 +14,7 @@
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see .
-package txpool
+package legacypool
import (
"sync"
diff --git a/core/txpool/subpool.go b/core/txpool/subpool.go
new file mode 100644
index 000000000..835f0de97
--- /dev/null
+++ b/core/txpool/subpool.go
@@ -0,0 +1,111 @@
+// 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 txpool
+
+import (
+ "math/big"
+
+ "github.com/ethereum/go-ethereum/common"
+ "github.com/ethereum/go-ethereum/core"
+ "github.com/ethereum/go-ethereum/core/types"
+ "github.com/ethereum/go-ethereum/crypto/kzg4844"
+ "github.com/ethereum/go-ethereum/event"
+)
+
+// Transaction is a helper struct to group together a canonical transaction with
+// satellite data items that are needed by the pool but are not part of the chain.
+type Transaction struct {
+ Tx *types.Transaction // Canonical transaction
+
+ BlobTxBlobs []kzg4844.Blob // Blobs needed by the blob pool
+ BlobTxCommits []kzg4844.Commitment // Commitments needed by the blob pool
+ BlobTxProofs []kzg4844.Proof // Proofs needed by the blob pool
+}
+
+// SubPool represents a specialized transaction pool that lives on its own (e.g.
+// blob pool). Since independent of how many specialized pools we have, they do
+// need to be updated in lockstep and assemble into one coherent view for block
+// production, this interface defines the common methods that allow the primary
+// transaction pool to manage the subpools.
+type SubPool interface {
+ // Filter is a selector used to decide whether a transaction whould be added
+ // to this particular subpool.
+ Filter(tx *types.Transaction) bool
+
+ // Init sets the base parameters of the subpool, allowing it to load any saved
+ // transactions from disk and also permitting internal maintenance routines to
+ // start up.
+ //
+ // These should not be passed as a constructor argument - nor should the pools
+ // start by themselves - in order to keep multiple subpools in lockstep with
+ // one another.
+ Init(gasTip *big.Int, head *types.Header) error
+
+ // Close terminates any background processing threads and releases any held
+ // resources.
+ Close() error
+
+ // Reset retrieves the current state of the blockchain and ensures the content
+ // of the transaction pool is valid with regard to the chain state.
+ Reset(oldHead, newHead *types.Header)
+
+ // SetGasTip updates the minimum price required by the subpool for a new
+ // transaction, and drops all transactions below this threshold.
+ SetGasTip(tip *big.Int)
+
+ // Has returns an indicator whether subpool has a transaction cached with the
+ // given hash.
+ Has(hash common.Hash) bool
+
+ // Get returns a transaction if it is contained in the pool, or nil otherwise.
+ Get(hash common.Hash) *Transaction
+
+ // Add enqueues a batch of transactions into the pool if they are valid. Due
+ // to the large transaction churn, add may postpone fully integrating the tx
+ // to a later point to batch multiple ones together.
+ Add(txs []*Transaction, local bool, sync bool) []error
+
+ // Pending retrieves all currently processable transactions, grouped by origin
+ // account and sorted by nonce.
+ Pending(enforceTips bool) map[common.Address][]*types.Transaction
+
+ // SubscribeTransactions subscribes to new transaction events.
+ SubscribeTransactions(ch chan<- core.NewTxsEvent) event.Subscription
+
+ // Nonce returns the next nonce of an account, with all transactions executable
+ // by the pool already applied on top.
+ Nonce(addr common.Address) uint64
+
+ // Stats retrieves the current pool stats, namely the number of pending and the
+ // number of queued (non-executable) transactions.
+ Stats() (int, int)
+
+ // Content retrieves the data content of the transaction pool, returning all the
+ // pending as well as queued transactions, grouped by account and sorted by nonce.
+ Content() (map[common.Address][]*types.Transaction, map[common.Address][]*types.Transaction)
+
+ // ContentFrom retrieves the data content of the transaction pool, returning the
+ // pending as well as queued transactions of this address, grouped by nonce.
+ ContentFrom(addr common.Address) ([]*types.Transaction, []*types.Transaction)
+
+ // Locals retrieves the accounts currently considered local by the pool.
+ Locals() []common.Address
+
+ // Status returns the known status (unknown/pending/queued) of a transaction
+ // identified by their hashes.
+ Status(hash common.Hash) TxStatus
+}
diff --git a/core/txpool/txpool.go b/core/txpool/txpool.go
index 139572540..f2c94563b 100644
--- a/core/txpool/txpool.go
+++ b/core/txpool/txpool.go
@@ -1,4 +1,4 @@
-// Copyright 2014 The go-ethereum Authors
+// 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
@@ -18,121 +18,12 @@ package txpool
import (
"errors"
- "math"
"math/big"
- "sort"
- "sync"
- "sync/atomic"
- "time"
"github.com/ethereum/go-ethereum/common"
- "github.com/ethereum/go-ethereum/common/prque"
- "github.com/ethereum/go-ethereum/consensus/misc"
"github.com/ethereum/go-ethereum/core"
- "github.com/ethereum/go-ethereum/core/state"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/event"
- "github.com/ethereum/go-ethereum/log"
- "github.com/ethereum/go-ethereum/metrics"
- "github.com/ethereum/go-ethereum/params"
-)
-
-const (
- // chainHeadChanSize is the size of channel listening to ChainHeadEvent.
- chainHeadChanSize = 10
-
- // txSlotSize is used to calculate how many data slots a single transaction
- // takes up based on its size. The slots are used as DoS protection, ensuring
- // that validating a new transaction remains a constant operation (in reality
- // O(maxslots), where max slots are 4 currently).
- txSlotSize = 32 * 1024
-
- // txMaxSize is the maximum size a single transaction can have. This field has
- // non-trivial consequences: larger transactions are significantly harder and
- // more expensive to propagate; larger transactions also take more resources
- // to validate whether they fit into the pool or not.
- txMaxSize = 4 * txSlotSize // 128KB
-)
-
-var (
- // ErrAlreadyKnown is returned if the transactions is already contained
- // within the pool.
- ErrAlreadyKnown = errors.New("already known")
-
- // ErrInvalidSender is returned if the transaction contains an invalid signature.
- ErrInvalidSender = errors.New("invalid sender")
-
- // ErrUnderpriced is returned if a transaction's gas price is below the minimum
- // configured for the transaction pool.
- ErrUnderpriced = errors.New("transaction underpriced")
-
- // ErrTxPoolOverflow is returned if the transaction pool is full and can't accept
- // another remote transaction.
- ErrTxPoolOverflow = errors.New("txpool is full")
-
- // ErrReplaceUnderpriced is returned if a transaction is attempted to be replaced
- // with a different one without the required price bump.
- ErrReplaceUnderpriced = errors.New("replacement transaction underpriced")
-
- // ErrGasLimit is returned if a transaction's requested gas limit exceeds the
- // maximum allowance of the current block.
- ErrGasLimit = errors.New("exceeds block gas limit")
-
- // ErrNegativeValue is a sanity error to ensure no one is able to specify a
- // transaction with a negative value.
- ErrNegativeValue = errors.New("negative value")
-
- // ErrOversizedData is returned if the input data of a transaction is greater
- // than some meaningful limit a user might use. This is not a consensus error
- // making the transaction invalid, rather a DOS protection.
- ErrOversizedData = errors.New("oversized data")
-
- // ErrFutureReplacePending is returned if a future transaction replaces a pending
- // transaction. 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
-)
-
-var (
- // Metrics for the pending pool
- pendingDiscardMeter = metrics.NewRegisteredMeter("txpool/pending/discard", nil)
- pendingReplaceMeter = metrics.NewRegisteredMeter("txpool/pending/replace", nil)
- pendingRateLimitMeter = metrics.NewRegisteredMeter("txpool/pending/ratelimit", nil) // Dropped due to rate limiting
- pendingNofundsMeter = metrics.NewRegisteredMeter("txpool/pending/nofunds", nil) // Dropped due to out-of-funds
-
- // Metrics for the queued pool
- queuedDiscardMeter = metrics.NewRegisteredMeter("txpool/queued/discard", nil)
- queuedReplaceMeter = metrics.NewRegisteredMeter("txpool/queued/replace", nil)
- queuedRateLimitMeter = metrics.NewRegisteredMeter("txpool/queued/ratelimit", nil) // Dropped due to rate limiting
- queuedNofundsMeter = metrics.NewRegisteredMeter("txpool/queued/nofunds", nil) // Dropped due to out-of-funds
- queuedEvictionMeter = metrics.NewRegisteredMeter("txpool/queued/eviction", nil) // Dropped due to lifetime
-
- // General tx metrics
- knownTxMeter = metrics.NewRegisteredMeter("txpool/known", nil)
- validTxMeter = metrics.NewRegisteredMeter("txpool/valid", nil)
- invalidTxMeter = metrics.NewRegisteredMeter("txpool/invalid", nil)
- underpricedTxMeter = metrics.NewRegisteredMeter("txpool/underpriced", nil)
- overflowedTxMeter = metrics.NewRegisteredMeter("txpool/overflowed", nil)
-
- // throttleTxMeter counts how many transactions are rejected due to too-many-changes between
- // txpool reorgs.
- throttleTxMeter = metrics.NewRegisteredMeter("txpool/throttle", nil)
- // reorgDurationTimer measures how long time a txpool reorg takes.
- reorgDurationTimer = metrics.NewRegisteredTimer("txpool/reorgtime", nil)
- // dropBetweenReorgHistogram counts how many drops we experience between two reorg runs. It is expected
- // that this number is pretty low, since txpool reorgs happen very frequently.
- dropBetweenReorgHistogram = metrics.NewRegisteredHistogram("txpool/dropbetweenreorg", nil, metrics.NewExpDecaySample(1028, 0.015))
-
- pendingGauge = metrics.NewRegisteredGauge("txpool/pending", nil)
- queuedGauge = metrics.NewRegisteredGauge("txpool/queued", nil)
- localGauge = metrics.NewRegisteredGauge("txpool/local", nil)
- slotsGauge = metrics.NewRegisteredGauge("txpool/slots", nil)
-
- reheapTimer = metrics.NewRegisteredTimer("txpool/reheap", nil)
)
// TxStatus is the current status of a transaction as seen by the pool.
@@ -145,1705 +36,309 @@ const (
TxStatusIncluded
)
-// blockChain provides the state of blockchain and current gas limit to do
-// some pre checks in tx pool and event subscribers.
-type blockChain interface {
+// BlockChain defines the minimal set of methods needed to back a tx pool with
+// a chain. Exists to allow mocking the live chain out of tests.
+type BlockChain interface {
+ // CurrentBlock returns the current head of the chain.
CurrentBlock() *types.Header
- GetBlock(hash common.Hash, number uint64) *types.Block
- StateAt(root common.Hash) (*state.StateDB, error)
+ // SubscribeChainHeadEvent subscribes to new blocks being added to the chain.
SubscribeChainHeadEvent(ch chan<- core.ChainHeadEvent) event.Subscription
}
-// 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.Rejournal < time.Second {
- log.Warn("Sanitizing invalid txpool journal time", "provided", conf.Rejournal, "updated", time.Second)
- conf.Rejournal = time.Second
- }
- 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
- }
- 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
-}
-
-// TxPool contains all currently known transactions. Transactions
-// enter the pool when they are received from the network or submitted
-// locally. They exit the pool when they are included in the blockchain.
-//
-// The pool separates processable transactions (which can be applied to the
-// current state) and future transactions. Transactions move between those
-// two states over time as they are received and processed.
+// TxPool is an aggregator for various transaction specific pools, collectively
+// tracking all the transactions deemed interesting by the node. Transactions
+// enter the pool when they are received from the network or submitted locally.
+// They exit the pool when they are included in the blockchain or evicted due to
+// resource constraints.
type TxPool struct {
- config Config
- chainconfig *params.ChainConfig
- chain blockChain
- gasTip atomic.Pointer[big.Int]
- txFeed event.Feed
- scope event.SubscriptionScope
- signer types.Signer
- mu sync.RWMutex
-
- currentHead atomic.Pointer[types.Header] // Current head of the blockchain
- currentState *state.StateDB // Current state in the blockchain head
- pendingNonces *noncer // Pending state tracking virtual nonces
-
- locals *accountSet // Set of local transaction to exempt from eviction rules
- journal *journal // Journal of local transaction to back up to disk
-
- pending map[common.Address]*list // All currently processable transactions
- queue map[common.Address]*list // Queued but non-processable transactions
- beats map[common.Address]time.Time // Last heartbeat from each known account
- all *lookup // All transactions to allow lookups
- priced *pricedList // All transactions sorted by price
-
- chainHeadCh chan core.ChainHeadEvent
- chainHeadSub event.Subscription
- reqResetCh chan *txpoolResetRequest
- reqPromoteCh chan *accountSet
- queueTxEventCh chan *types.Transaction
- reorgDoneCh chan chan struct{}
- reorgShutdownCh chan struct{} // requests shutdown of scheduleReorgLoop
- wg sync.WaitGroup // tracks loop, scheduleReorgLoop
- initDoneCh chan struct{} // is closed once the pool is initialized (for tests)
-
- changesSinceReorg int // A counter for how many drops we've performed in-between reorg.
-}
-
-type txpoolResetRequest struct {
- oldHead, newHead *types.Header
+ subpools []SubPool // List of subpools for specialized transaction handling
+ subs event.SubscriptionScope // Subscription scope to unscubscribe all on shutdown
+ quit chan chan error // Quit channel to tear down the head updater
}
// New creates a new transaction pool to gather, sort and filter inbound
// transactions from the network.
-func New(config Config, chainconfig *params.ChainConfig, chain blockChain) *TxPool {
- // Sanitize the input to ensure no vulnerable gas prices are set
- config = (&config).sanitize()
+func New(gasTip *big.Int, chain BlockChain, subpools []SubPool) (*TxPool, error) {
+ // Retrieve the current head so that all subpools and this main coordinator
+ // pool will have the same starting state, even if the chain moves forward
+ // during initialization.
+ head := chain.CurrentBlock()
- // Create the transaction pool with its initial settings
pool := &TxPool{
- config: config,
- chainconfig: chainconfig,
- chain: chain,
- signer: types.LatestSigner(chainconfig),
- pending: make(map[common.Address]*list),
- queue: make(map[common.Address]*list),
- beats: make(map[common.Address]time.Time),
- all: newLookup(),
- chainHeadCh: make(chan core.ChainHeadEvent, chainHeadChanSize),
- reqResetCh: make(chan *txpoolResetRequest),
- reqPromoteCh: make(chan *accountSet),
- queueTxEventCh: make(chan *types.Transaction),
- reorgDoneCh: make(chan chan struct{}),
- reorgShutdownCh: make(chan struct{}),
- initDoneCh: make(chan struct{}),
+ subpools: subpools,
+ quit: make(chan chan error),
}
- pool.gasTip.Store(new(big.Int).SetUint64(config.PriceLimit))
- pool.locals = newAccountSet(pool.signer)
- for _, addr := range config.Locals {
- log.Info("Setting new local account", "address", addr)
- pool.locals.add(addr)
- }
- pool.priced = newPricedList(pool.all)
- pool.reset(nil, chain.CurrentBlock())
-
- // Start the reorg loop early so it can handle requests generated during journal loading.
- pool.wg.Add(1)
- go pool.scheduleReorgLoop()
-
- // If local transactions and journaling is enabled, load from disk
- if !config.NoLocals && config.Journal != "" {
- pool.journal = newTxJournal(config.Journal)
-
- if err := pool.journal.load(pool.AddLocals); err != nil {
- log.Warn("Failed to load transaction journal", "err", err)
- }
- if err := pool.journal.rotate(pool.local()); err != nil {
- log.Warn("Failed to rotate transaction journal", "err", err)
+ for i, subpool := range subpools {
+ if err := subpool.Init(gasTip, head); err != nil {
+ for j := i - 1; j >= 0; j-- {
+ subpools[j].Close()
+ }
+ return nil, err
}
}
+ go pool.loop(head, chain)
+ return pool, nil
+}
- // Subscribe events from blockchain and start the main event loop.
- pool.chainHeadSub = pool.chain.SubscribeChainHeadEvent(pool.chainHeadCh)
- pool.wg.Add(1)
- go pool.loop()
+// Close terminates the transaction pool and all its subpools.
+func (p *TxPool) Close() error {
+ var errs []error
- return pool
+ // Terminate the reset loop and wait for it to finish
+ errc := make(chan error)
+ p.quit <- errc
+ errs = append(errs, <-errc)
+
+ // Terminate each subpool
+ for _, subpool := range p.subpools {
+ errs = append(errs, subpool.Close())
+ }
+ return errors.Join(errs...)
}
// loop is the transaction pool's main event loop, waiting for and reacting to
// outside blockchain events as well as for various reporting and transaction
// eviction events.
-func (pool *TxPool) loop() {
- defer pool.wg.Done()
-
+func (p *TxPool) loop(head *types.Header, chain BlockChain) {
+ // Subscribe to chain head events to trigger subpool resets
var (
- prevPending, prevQueued, prevStales int
- // Start the stats reporting and transaction eviction tickers
- report = time.NewTicker(statsReportInterval)
- evict = time.NewTicker(evictionInterval)
- journal = time.NewTicker(pool.config.Rejournal)
- // Track the previous head headers for transaction reorgs
- head = pool.chain.CurrentBlock()
+ newHeadCh = make(chan core.ChainHeadEvent)
+ newHeadSub = chain.SubscribeChainHeadEvent(newHeadCh)
)
- defer report.Stop()
- defer evict.Stop()
- defer journal.Stop()
+ defer newHeadSub.Unsubscribe()
- // Notify tests that the init phase is done
- close(pool.initDoneCh)
- for {
- select {
- // Handle ChainHeadEvent
- case ev := <-pool.chainHeadCh:
- if ev.Block != nil {
- pool.requestReset(head, ev.Block.Header())
- head = ev.Block.Header()
- }
-
- // System shutdown.
- case <-pool.chainHeadSub.Err():
- close(pool.reorgShutdownCh)
- return
-
- // Handle stats reporting ticks
- case <-report.C:
- pool.mu.RLock()
- pending, queued := pool.stats()
- pool.mu.RUnlock()
- stales := int(pool.priced.stales.Load())
-
- if pending != prevPending || queued != prevQueued || stales != prevStales {
- log.Debug("Transaction pool status report", "executable", pending, "queued", queued, "stales", stales)
- prevPending, prevQueued, prevStales = pending, queued, stales
- }
-
- // Handle inactive account transaction eviction
- case <-evict.C:
- pool.mu.Lock()
- for addr := range pool.queue {
- // Skip local transactions from the eviction mechanism
- if pool.locals.contains(addr) {
- continue
- }
- // Any non-locals old enough should be removed
- if time.Since(pool.beats[addr]) > pool.config.Lifetime {
- list := pool.queue[addr].Flatten()
- for _, tx := range list {
- pool.removeTx(tx.Hash(), true)
+ // Track the previous and current head to feed to an idle reset
+ var (
+ oldHead = head
+ newHead = oldHead
+ )
+ // Consume chain head events and start resets when none is running
+ var (
+ resetBusy = make(chan struct{}, 1) // Allow 1 reset to run concurrently
+ resetDone = make(chan *types.Header)
+ )
+ var errc chan error
+ for errc == nil {
+ // Something interesting might have happened, run a reset if there is
+ // one needed but none is running. The resetter will run on its own
+ // goroutine to allow chain head events to be consumed contiguously.
+ if newHead != oldHead {
+ // Try to inject a busy marker and start a reset if successful
+ select {
+ case resetBusy <- struct{}{}:
+ // Busy marker injected, start a new subpool reset
+ go func(oldHead, newHead *types.Header) {
+ for _, subpool := range p.subpools {
+ subpool.Reset(oldHead, newHead)
}
- queuedEvictionMeter.Mark(int64(len(list)))
- }
- }
- pool.mu.Unlock()
+ resetDone <- newHead
+ }(oldHead, newHead)
- // Handle local transaction journal rotation
- case <-journal.C:
- if pool.journal != nil {
- pool.mu.Lock()
- if err := pool.journal.rotate(pool.local()); err != nil {
- log.Warn("Failed to rotate local tx journal", "err", err)
- }
- pool.mu.Unlock()
+ default:
+ // Reset already running, wait until it finishes
}
}
+ // Wait for the next chain head event or a previous reset finish
+ select {
+ case event := <-newHeadCh:
+ // Chain moved forward, store the head for later consumption
+ newHead = event.Block.Header()
+
+ case head := <-resetDone:
+ // Previous reset finished, update the old head and allow a new reset
+ oldHead = head
+ <-resetBusy
+
+ case errc = <-p.quit:
+ // Termination requested, break out on the next loop round
+ }
}
-}
-
-// Stop terminates the transaction pool.
-func (pool *TxPool) Stop() {
- // Unsubscribe all subscriptions registered from txpool
- pool.scope.Close()
-
- // Unsubscribe subscriptions registered from blockchain
- pool.chainHeadSub.Unsubscribe()
- pool.wg.Wait()
-
- if pool.journal != nil {
- pool.journal.close()
- }
- log.Info("Transaction pool stopped")
-}
-
-// SubscribeNewTxsEvent registers a subscription of NewTxsEvent and
-// starts sending event to the given channel.
-func (pool *TxPool) SubscribeNewTxsEvent(ch chan<- core.NewTxsEvent) event.Subscription {
- return pool.scope.Track(pool.txFeed.Subscribe(ch))
+ // Notify the closer of termination (no error possible for now)
+ errc <- nil
}
// SetGasTip updates the minimum gas tip required by the transaction pool for a
// new transaction, and drops all transactions below this threshold.
-func (pool *TxPool) SetGasTip(tip *big.Int) {
- pool.mu.Lock()
- defer pool.mu.Unlock()
+func (p *TxPool) SetGasTip(tip *big.Int) {
+ for _, subpool := range p.subpools {
+ subpool.SetGasTip(tip)
+ }
+}
- old := pool.gasTip.Load()
- pool.gasTip.Store(new(big.Int).Set(tip))
-
- // If the min miner fee increased, remove transactions below the new threshold
- if tip.Cmp(old) > 0 {
- // pool.priced is sorted by GasFeeCap, so we have to iterate through pool.all instead
- drop := pool.all.RemotesBelowTip(tip)
- for _, tx := range drop {
- pool.removeTx(tx.Hash(), false)
+// Has returns an indicator whether the pool has a transaction cached with the
+// given hash.
+func (p *TxPool) Has(hash common.Hash) bool {
+ for _, subpool := range p.subpools {
+ if subpool.Has(hash) {
+ return true
}
- pool.priced.Removed(len(drop))
}
- log.Info("Transaction pool tip threshold updated", "tip", tip)
+ return false
}
-// Nonce returns the next nonce of an account, with all transactions executable
-// by the pool already applied on top.
-func (pool *TxPool) Nonce(addr common.Address) uint64 {
- pool.mu.RLock()
- defer pool.mu.RUnlock()
-
- return pool.pendingNonces.get(addr)
+// Get returns a transaction if it is contained in the pool, or nil otherwise.
+func (p *TxPool) Get(hash common.Hash) *Transaction {
+ for _, subpool := range p.subpools {
+ if tx := subpool.Get(hash); tx != nil {
+ return tx
+ }
+ }
+ return nil
}
-// Stats retrieves the current pool stats, namely the number of pending and the
-// number of queued (non-executable) transactions.
-func (pool *TxPool) Stats() (int, int) {
- pool.mu.RLock()
- defer pool.mu.RUnlock()
+// Add enqueues a batch of transactions into the pool if they are valid. Due
+// to the large transaction churn, add may postpone fully integrating the tx
+// to a later point to batch multiple ones together.
+func (p *TxPool) Add(txs []*Transaction, local bool, sync bool) []error {
+ // Split the input transactions between the subpools. It shouldn't really
+ // happen that we receive merged batches, but better graceful than strange
+ // errors.
+ //
+ // We also need to track how the transactions were split across the subpools,
+ // so we can piece back the returned errors into the original order.
+ txsets := make([][]*Transaction, len(p.subpools))
+ splits := make([]int, len(txs))
- return pool.stats()
-}
+ for i, tx := range txs {
+ // Mark this transaction belonging to no-subpool
+ splits[i] = -1
-// stats retrieves the current pool stats, namely the number of pending and the
-// number of queued (non-executable) transactions.
-func (pool *TxPool) stats() (int, int) {
- pending := 0
- for _, list := range pool.pending {
- pending += list.Len()
+ // Try to find a subpool that accepts the transaction
+ for j, subpool := range p.subpools {
+ if subpool.Filter(tx.Tx) {
+ txsets[j] = append(txsets[j], tx)
+ splits[i] = j
+ break
+ }
+ }
}
- queued := 0
- for _, list := range pool.queue {
- queued += list.Len()
+ // Add the transactions split apart to the individual subpools and piece
+ // back the errors into the original sort order.
+ errsets := make([][]error, len(p.subpools))
+ for i := 0; i < len(p.subpools); i++ {
+ errsets[i] = p.subpools[i].Add(txsets[i], local, sync)
}
- return pending, queued
-}
-
-// Content retrieves the data content of the transaction pool, returning all the
-// pending as well as queued transactions, grouped by account and sorted by nonce.
-func (pool *TxPool) Content() (map[common.Address]types.Transactions, map[common.Address]types.Transactions) {
- pool.mu.Lock()
- defer pool.mu.Unlock()
-
- pending := make(map[common.Address]types.Transactions, len(pool.pending))
- for addr, list := range pool.pending {
- pending[addr] = list.Flatten()
+ errs := make([]error, len(txs))
+ for i, split := range splits {
+ // If the transaction was rejected by all subpools, mark it unsupported
+ if split == -1 {
+ errs[i] = core.ErrTxTypeNotSupported
+ continue
+ }
+ // Find which subpool handled it and pull in the corresponding error
+ errs[i] = errsets[split][0]
+ errsets[split] = errsets[split][1:]
}
- queued := make(map[common.Address]types.Transactions, len(pool.queue))
- for addr, list := range pool.queue {
- queued[addr] = list.Flatten()
- }
- return pending, queued
-}
-
-// ContentFrom retrieves the data content of the transaction pool, returning the
-// pending as well as queued transactions of this address, grouped by nonce.
-func (pool *TxPool) ContentFrom(addr common.Address) (types.Transactions, types.Transactions) {
- pool.mu.RLock()
- defer pool.mu.RUnlock()
-
- var pending types.Transactions
- if list, ok := pool.pending[addr]; ok {
- pending = list.Flatten()
- }
- var queued types.Transactions
- if list, ok := pool.queue[addr]; ok {
- queued = list.Flatten()
- }
- return pending, queued
+ return errs
}
// Pending retrieves all currently processable transactions, grouped by origin
-// account and sorted by nonce. The returned transaction set is a copy and can be
-// freely modified by calling code.
-//
-// The enforceTips parameter can be used to do an extra filtering on the pending
-// transactions and only return those whose **effective** tip is large enough in
-// the next pending execution environment.
-func (pool *TxPool) Pending(enforceTips bool) map[common.Address]types.Transactions {
- pool.mu.Lock()
- defer pool.mu.Unlock()
-
- pending := make(map[common.Address]types.Transactions, len(pool.pending))
- for addr, list := range pool.pending {
- txs := list.Flatten()
-
- // If the miner requests tip enforcement, cap the lists now
- if enforceTips && !pool.locals.contains(addr) {
- for i, tx := range txs {
- if tx.EffectiveGasTipIntCmp(pool.gasTip.Load(), pool.priced.urgent.baseFee) < 0 {
- txs = txs[:i]
- break
- }
- }
- }
- if len(txs) > 0 {
- pending[addr] = txs
- }
- }
- return pending
-}
-
-// Locals retrieves the accounts currently considered local by the pool.
-func (pool *TxPool) Locals() []common.Address {
- pool.mu.Lock()
- defer pool.mu.Unlock()
-
- return pool.locals.flatten()
-}
-
-// local retrieves all currently known local transactions, grouped by origin
-// account and sorted by nonce. The returned transaction set is a copy and can be
-// freely modified by calling code.
-func (pool *TxPool) local() map[common.Address]types.Transactions {
- txs := make(map[common.Address]types.Transactions)
- for addr := range pool.locals.accounts {
- if pending := pool.pending[addr]; pending != nil {
- txs[addr] = append(txs[addr], pending.Flatten()...)
- }
- if queued := pool.queue[addr]; queued != nil {
- txs[addr] = append(txs[addr], queued.Flatten()...)
+// account and sorted by nonce.
+func (p *TxPool) Pending(enforceTips bool) map[common.Address][]*types.Transaction {
+ txs := make(map[common.Address][]*types.Transaction)
+ for _, subpool := range p.subpools {
+ for addr, set := range subpool.Pending(enforceTips) {
+ txs[addr] = set
}
}
return txs
}
-// validateTxBasics checks whether a transaction is valid according to the consensus
-// rules, but does not check state-dependent validation such as sufficient balance.
-// This check is meant as an early check which only needs to be performed once,
-// and does not require the pool mutex to be held.
-func (pool *TxPool) validateTxBasics(tx *types.Transaction, local bool) error {
- opts := &ValidationOptions{
- Config: pool.chainconfig,
- Accept: 0 |
- 1< pool.config.GlobalSlots+pool.config.GlobalQueue {
- // If the new transaction is underpriced, don't accept it
- if !isLocal && pool.priced.Underpriced(tx) {
- log.Trace("Discarding underpriced transaction", "hash", hash, "gasTipCap", tx.GasTipCap(), "gasFeeCap", tx.GasFeeCap())
- underpricedTxMeter.Mark(1)
- return false, ErrUnderpriced
- }
-
- // We're about to replace a transaction. The reorg does a more thorough
- // analysis of what to remove and how, but it runs async. We don't want to
- // do too many replacements between reorg-runs, so we cap the number of
- // replacements to 25% of the slots
- if pool.changesSinceReorg > int(pool.config.GlobalSlots/4) {
- throttleTxMeter.Mark(1)
- return false, ErrTxPoolOverflow
- }
-
- // New transaction is better than our worse ones, make room for it.
- // If it's a local transaction, forcibly discard all available transactions.
- // Otherwise if we can't make enough room for new one, abort the operation.
- drop, success := pool.priced.Discard(pool.all.Slots()-int(pool.config.GlobalSlots+pool.config.GlobalQueue)+numSlots(tx), isLocal)
-
- // Special case, we still can't make the room for the new remote one.
- if !isLocal && !success {
- log.Trace("Discarding overflown transaction", "hash", hash)
- overflowedTxMeter.Mark(1)
- return false, ErrTxPoolOverflow
- }
-
- // If the new transaction is a future transaction it should never churn pending transactions
- if !isLocal && pool.isGapped(from, tx) {
- var replacesPending bool
- for _, dropTx := range drop {
- dropSender, _ := types.Sender(pool.signer, dropTx)
- if list := pool.pending[dropSender]; list != nil && list.Contains(dropTx.Nonce()) {
- replacesPending = true
- break
- }
- }
- // Add all transactions back to the priced queue
- if replacesPending {
- for _, dropTx := range drop {
- pool.priced.Put(dropTx, false)
- }
- log.Trace("Discarding future transaction replacing pending tx", "hash", hash)
- return false, ErrFutureReplacePending
- }
- }
-
- // Kick out the underpriced remote transactions.
- for _, tx := range drop {
- log.Trace("Discarding freshly underpriced transaction", "hash", tx.Hash(), "gasTipCap", tx.GasTipCap(), "gasFeeCap", tx.GasFeeCap())
- underpricedTxMeter.Mark(1)
- dropped := pool.removeTx(tx.Hash(), false)
- pool.changesSinceReorg += dropped
+// Nonce returns the next nonce of an account, with all transactions executable
+// by the pool already applied on top.
+func (p *TxPool) Nonce(addr common.Address) uint64 {
+ // Since (for now) accounts are unique to subpools, only one pool will have
+ // (at max) a non-state nonce. To avoid stateful lookups, just return the
+ // highest nonce for now.
+ var nonce uint64
+ for _, subpool := range p.subpools {
+ if next := subpool.Nonce(addr); nonce < next {
+ nonce = next
}
}
-
- // Try to replace an existing transaction in the pending pool
- if list := pool.pending[from]; list != nil && list.Contains(tx.Nonce()) {
- // Nonce already pending, check if required price bump is met
- inserted, old := list.Add(tx, pool.config.PriceBump)
- if !inserted {
- pendingDiscardMeter.Mark(1)
- return false, ErrReplaceUnderpriced
- }
- // New transaction is better, replace old one
- if old != nil {
- pool.all.Remove(old.Hash())
- pool.priced.Removed(1)
- pendingReplaceMeter.Mark(1)
- }
- pool.all.Add(tx, isLocal)
- pool.priced.Put(tx, isLocal)
- pool.journalTx(from, tx)
- pool.queueTxEvent(tx)
- log.Trace("Pooled new executable transaction", "hash", hash, "from", from, "to", tx.To())
-
- // Successful promotion, bump the heartbeat
- pool.beats[from] = time.Now()
- return old != nil, nil
- }
- // New transaction isn't replacing a pending one, push into queue
- replaced, err = pool.enqueueTx(hash, tx, isLocal, true)
- if err != nil {
- return false, err
- }
- // Mark local addresses and journal local transactions
- if local && !pool.locals.contains(from) {
- log.Info("Setting new local account", "address", from)
- pool.locals.add(from)
- pool.priced.Removed(pool.all.RemoteToLocals(pool.locals)) // Migrate the remotes if it's marked as local first time.
- }
- if isLocal {
- localGauge.Inc(1)
- }
- pool.journalTx(from, tx)
-
- log.Trace("Pooled new future transaction", "hash", hash, "from", from, "to", tx.To())
- return replaced, nil
+ return nonce
}
-// isGapped reports whether the given transaction is immediately executable.
-func (pool *TxPool) isGapped(from common.Address, tx *types.Transaction) bool {
- // Short circuit if transaction falls within the scope of the pending list
- // or matches the next pending nonce which can be promoted as an executable
- // transaction afterwards. Note, the tx staleness is already checked in
- // 'validateTx' function previously.
- next := pool.pendingNonces.get(from)
- if tx.Nonce() <= next {
- return false
+// Stats retrieves the current pool stats, namely the number of pending and the
+// number of queued (non-executable) transactions.
+func (p *TxPool) Stats() (int, int) {
+ var runnable, blocked int
+ for _, subpool := range p.subpools {
+ run, block := subpool.Stats()
+
+ runnable += run
+ blocked += block
}
- // The transaction has a nonce gap with pending list, it's only considered
- // as executable if transactions in queue can fill up the nonce gap.
- queue, ok := pool.queue[from]
- if !ok {
- return true
- }
- for nonce := next; nonce < tx.Nonce(); nonce++ {
- if !queue.Contains(nonce) {
- return true // txs in queue can't fill up the nonce gap
- }
- }
- return false
+ return runnable, blocked
}
-// enqueueTx inserts a new transaction into the non-executable transaction queue.
-//
-// Note, this method assumes the pool lock is held!
-func (pool *TxPool) enqueueTx(hash common.Hash, tx *types.Transaction, local bool, addAll bool) (bool, error) {
- // Try to insert the transaction into the future queue
- from, _ := types.Sender(pool.signer, tx) // already validated
- if pool.queue[from] == nil {
- pool.queue[from] = newList(false)
- }
- inserted, old := pool.queue[from].Add(tx, pool.config.PriceBump)
- if !inserted {
- // An older transaction was better, discard this
- queuedDiscardMeter.Mark(1)
- return false, ErrReplaceUnderpriced
- }
- // Discard any previous transaction and mark this
- if old != nil {
- pool.all.Remove(old.Hash())
- pool.priced.Removed(1)
- queuedReplaceMeter.Mark(1)
- } else {
- // Nothing was replaced, bump the queued counter
- queuedGauge.Inc(1)
- }
- // If the transaction isn't in lookup set but it's expected to be there,
- // show the error log.
- if pool.all.Get(hash) == nil && !addAll {
- log.Error("Missing transaction in lookup set, please report the issue", "hash", hash)
- }
- if addAll {
- pool.all.Add(tx, local)
- pool.priced.Put(tx, local)
- }
- // If we never record the heartbeat, do it right now.
- if _, exist := pool.beats[from]; !exist {
- pool.beats[from] = time.Now()
- }
- return old != nil, nil
-}
-
-// journalTx adds the specified transaction to the local disk journal if it is
-// deemed to have been sent from a local account.
-func (pool *TxPool) journalTx(from common.Address, tx *types.Transaction) {
- // Only journal if it's enabled and the transaction is local
- if pool.journal == nil || !pool.locals.contains(from) {
- return
- }
- if err := pool.journal.insert(tx); err != nil {
- log.Warn("Failed to journal local transaction", "err", err)
- }
-}
-
-// promoteTx adds a transaction to the pending (processable) list of transactions
-// and returns whether it was inserted or an older was better.
-//
-// Note, this method assumes the pool lock is held!
-func (pool *TxPool) promoteTx(addr common.Address, hash common.Hash, tx *types.Transaction) bool {
- // Try to insert the transaction into the pending queue
- if pool.pending[addr] == nil {
- pool.pending[addr] = newList(true)
- }
- list := pool.pending[addr]
-
- inserted, old := list.Add(tx, pool.config.PriceBump)
- if !inserted {
- // An older transaction was better, discard this
- pool.all.Remove(hash)
- pool.priced.Removed(1)
- pendingDiscardMeter.Mark(1)
- return false
- }
- // Otherwise discard any previous transaction and mark this
- if old != nil {
- pool.all.Remove(old.Hash())
- pool.priced.Removed(1)
- pendingReplaceMeter.Mark(1)
- } else {
- // Nothing was replaced, bump the pending counter
- pendingGauge.Inc(1)
- }
- // Set the potentially new pending nonce and notify any subsystems of the new tx
- pool.pendingNonces.set(addr, tx.Nonce()+1)
-
- // Successful promotion, bump the heartbeat
- pool.beats[addr] = time.Now()
- return true
-}
-
-// AddLocals enqueues a batch of transactions into the pool if they are valid, marking the
-// senders as a local ones, ensuring they go around the local pricing constraints.
-//
-// This method is used to add transactions from the RPC API and performs synchronous pool
-// reorganization and event propagation.
-func (pool *TxPool) AddLocals(txs []*types.Transaction) []error {
- return pool.addTxs(txs, !pool.config.NoLocals, true)
-}
-
-// AddLocal enqueues a single local transaction into the pool if it is valid. This is
-// a convenience wrapper around AddLocals.
-func (pool *TxPool) AddLocal(tx *types.Transaction) error {
- errs := pool.AddLocals([]*types.Transaction{tx})
- return errs[0]
-}
-
-// AddRemotes enqueues a batch of transactions into the pool if they are valid. If the
-// senders are not among the locally tracked ones, full pricing constraints will apply.
-//
-// This method is used to add transactions from the p2p network and does not wait for pool
-// reorganization and internal event propagation.
-func (pool *TxPool) AddRemotes(txs []*types.Transaction) []error {
- return pool.addTxs(txs, false, false)
-}
-
-// AddRemotesSync is like AddRemotes, but waits for pool reorganization. Tests use this method.
-func (pool *TxPool) AddRemotesSync(txs []*types.Transaction) []error {
- return pool.addTxs(txs, false, true)
-}
-
-// This is like AddRemotes with a single transaction, but waits for pool reorganization. Tests use this method.
-func (pool *TxPool) addRemoteSync(tx *types.Transaction) error {
- errs := pool.AddRemotesSync([]*types.Transaction{tx})
- return errs[0]
-}
-
-// AddRemote enqueues a single transaction into the pool if it is valid. This is a convenience
-// wrapper around AddRemotes.
-//
-// Deprecated: use AddRemotes
-func (pool *TxPool) AddRemote(tx *types.Transaction) error {
- errs := pool.AddRemotes([]*types.Transaction{tx})
- return errs[0]
-}
-
-// addTxs attempts to queue a batch of transactions if they are valid.
-func (pool *TxPool) addTxs(txs []*types.Transaction, local, sync bool) []error {
- // Filter out known ones without obtaining the pool lock or recovering signatures
+// Content retrieves the data content of the transaction pool, returning all the
+// pending as well as queued transactions, grouped by account and sorted by nonce.
+func (p *TxPool) Content() (map[common.Address][]*types.Transaction, map[common.Address][]*types.Transaction) {
var (
- errs = make([]error, len(txs))
- news = make([]*types.Transaction, 0, len(txs))
+ runnable = make(map[common.Address][]*types.Transaction)
+ blocked = make(map[common.Address][]*types.Transaction)
)
- for i, tx := range txs {
- // If the transaction is known, pre-set the error slot
- if pool.all.Get(tx.Hash()) != nil {
- errs[i] = ErrAlreadyKnown
- knownTxMeter.Mark(1)
- continue
- }
- // Exclude transactions with basic errors, e.g invalid signatures and
- // insufficient intrinsic gas as soon as possible and cache senders
- // in transactions before obtaining lock
- if err := pool.validateTxBasics(tx, local); err != nil {
- errs[i] = err
- invalidTxMeter.Mark(1)
- continue
- }
- // Accumulate all unknown transactions for deeper processing
- news = append(news, tx)
- }
- if len(news) == 0 {
- return errs
- }
+ for _, subpool := range p.subpools {
+ run, block := subpool.Content()
- // Process all the new transaction and merge any errors into the original slice
- pool.mu.Lock()
- newErrs, dirtyAddrs := pool.addTxsLocked(news, local)
- pool.mu.Unlock()
-
- var nilSlot = 0
- for _, err := range newErrs {
- for errs[nilSlot] != nil {
- nilSlot++
+ for addr, txs := range run {
+ runnable[addr] = txs
+ }
+ for addr, txs := range block {
+ blocked[addr] = txs
}
- errs[nilSlot] = err
- nilSlot++
}
- // Reorg the pool internals if needed and return
- done := pool.requestPromoteExecutables(dirtyAddrs)
- if sync {
- <-done
- }
- return errs
+ return runnable, blocked
}
-// addTxsLocked attempts to queue a batch of transactions if they are valid.
-// The transaction pool lock must be held.
-func (pool *TxPool) addTxsLocked(txs []*types.Transaction, local bool) ([]error, *accountSet) {
- dirty := newAccountSet(pool.signer)
- errs := make([]error, len(txs))
- for i, tx := range txs {
- replaced, err := pool.add(tx, local)
- errs[i] = err
- if err == nil && !replaced {
- dirty.addTx(tx)
+// ContentFrom retrieves the data content of the transaction pool, returning the
+// pending as well as queued transactions of this address, grouped by nonce.
+func (p *TxPool) ContentFrom(addr common.Address) ([]*types.Transaction, []*types.Transaction) {
+ for _, subpool := range p.subpools {
+ run, block := subpool.ContentFrom(addr)
+ if len(run) != 0 || len(block) != 0 {
+ return run, block
}
}
- validTxMeter.Mark(int64(len(dirty.accounts)))
- return errs, dirty
+ return []*types.Transaction{}, []*types.Transaction{}
}
-// Status returns the status (unknown/pending/queued) of a batch of transactions
+// Locals retrieves the accounts currently considered local by the pool.
+func (p *TxPool) Locals() []common.Address {
+ // Retrieve the locals from each subpool and deduplicate them
+ locals := make(map[common.Address]struct{})
+ for _, subpool := range p.subpools {
+ for _, local := range subpool.Locals() {
+ locals[local] = struct{}{}
+ }
+ }
+ // Flatten and return the deduplicated local set
+ flat := make([]common.Address, 0, len(locals))
+ for local := range locals {
+ flat = append(flat, local)
+ }
+ return flat
+}
+
+// Status returns the known status (unknown/pending/queued) of a transaction
// identified by their hashes.
-func (pool *TxPool) Status(hashes []common.Hash) []TxStatus {
- status := make([]TxStatus, len(hashes))
- for i, hash := range hashes {
- tx := pool.Get(hash)
- if tx == nil {
- continue
- }
- from, _ := types.Sender(pool.signer, tx) // already validated
- pool.mu.RLock()
- if txList := pool.pending[from]; txList != nil && txList.txs.items[tx.Nonce()] != nil {
- status[i] = TxStatusPending
- } else if txList := pool.queue[from]; txList != nil && txList.txs.items[tx.Nonce()] != nil {
- status[i] = TxStatusQueued
- }
- // implicit else: the tx may have been included into a block between
- // checking pool.Get and obtaining the lock. In that case, TxStatusUnknown is correct
- pool.mu.RUnlock()
- }
- return status
-}
-
-// Get returns a transaction if it is contained in the pool and nil otherwise.
-func (pool *TxPool) Get(hash common.Hash) *types.Transaction {
- return pool.all.Get(hash)
-}
-
-// Has returns an indicator whether txpool has a transaction cached with the
-// given hash.
-func (pool *TxPool) Has(hash common.Hash) bool {
- return pool.all.Get(hash) != nil
-}
-
-// removeTx removes a single transaction from the queue, moving all subsequent
-// transactions back to the future queue.
-// Returns the number of transactions removed from the pending queue.
-func (pool *TxPool) removeTx(hash common.Hash, outofbound bool) int {
- // Fetch the transaction we wish to delete
- tx := pool.all.Get(hash)
- if tx == nil {
- return 0
- }
- addr, _ := types.Sender(pool.signer, tx) // already validated during insertion
-
- // Remove it from the list of known transactions
- pool.all.Remove(hash)
- if outofbound {
- pool.priced.Removed(1)
- }
- if pool.locals.contains(addr) {
- localGauge.Dec(1)
- }
- // Remove the transaction from the pending lists and reset the account nonce
- if pending := pool.pending[addr]; pending != nil {
- if removed, invalids := pending.Remove(tx); removed {
- // If no more pending transactions are left, remove the list
- if pending.Empty() {
- delete(pool.pending, addr)
- }
- // Postpone any invalidated transactions
- for _, tx := range invalids {
- // Internal shuffle shouldn't touch the lookup set.
- pool.enqueueTx(tx.Hash(), tx, false, false)
- }
- // Update the account nonce if needed
- pool.pendingNonces.setIfLower(addr, tx.Nonce())
- // Reduce the pending counter
- pendingGauge.Dec(int64(1 + len(invalids)))
- return 1 + len(invalids)
+func (p *TxPool) Status(hash common.Hash) TxStatus {
+ for _, subpool := range p.subpools {
+ if status := subpool.Status(hash); status != TxStatusUnknown {
+ return status
}
}
- // Transaction is in the future queue
- if future := pool.queue[addr]; future != nil {
- if removed, _ := future.Remove(tx); removed {
- // Reduce the queued counter
- queuedGauge.Dec(1)
- }
- if future.Empty() {
- delete(pool.queue, addr)
- delete(pool.beats, addr)
- }
- }
- return 0
-}
-
-// requestReset requests a pool reset to the new head block.
-// The returned channel is closed when the reset has occurred.
-func (pool *TxPool) requestReset(oldHead *types.Header, newHead *types.Header) chan struct{} {
- select {
- case pool.reqResetCh <- &txpoolResetRequest{oldHead, newHead}:
- return <-pool.reorgDoneCh
- case <-pool.reorgShutdownCh:
- return pool.reorgShutdownCh
- }
-}
-
-// requestPromoteExecutables requests transaction promotion checks for the given addresses.
-// The returned channel is closed when the promotion checks have occurred.
-func (pool *TxPool) requestPromoteExecutables(set *accountSet) chan struct{} {
- select {
- case pool.reqPromoteCh <- set:
- return <-pool.reorgDoneCh
- case <-pool.reorgShutdownCh:
- return pool.reorgShutdownCh
- }
-}
-
-// queueTxEvent enqueues a transaction event to be sent in the next reorg run.
-func (pool *TxPool) queueTxEvent(tx *types.Transaction) {
- select {
- case pool.queueTxEventCh <- tx:
- case <-pool.reorgShutdownCh:
- }
-}
-
-// scheduleReorgLoop schedules runs of reset and promoteExecutables. Code above should not
-// call those methods directly, but request them being run using requestReset and
-// requestPromoteExecutables instead.
-func (pool *TxPool) scheduleReorgLoop() {
- defer pool.wg.Done()
-
- var (
- curDone chan struct{} // non-nil while runReorg is active
- nextDone = make(chan struct{})
- launchNextRun bool
- reset *txpoolResetRequest
- dirtyAccounts *accountSet
- queuedEvents = make(map[common.Address]*sortedMap)
- )
- for {
- // Launch next background reorg if needed
- if curDone == nil && launchNextRun {
- // Run the background reorg and announcements
- go pool.runReorg(nextDone, reset, dirtyAccounts, queuedEvents)
-
- // Prepare everything for the next round of reorg
- curDone, nextDone = nextDone, make(chan struct{})
- launchNextRun = false
-
- reset, dirtyAccounts = nil, nil
- queuedEvents = make(map[common.Address]*sortedMap)
- }
-
- select {
- case req := <-pool.reqResetCh:
- // Reset request: update head if request is already pending.
- if reset == nil {
- reset = req
- } else {
- reset.newHead = req.newHead
- }
- launchNextRun = true
- pool.reorgDoneCh <- nextDone
-
- case req := <-pool.reqPromoteCh:
- // Promote request: update address set if request is already pending.
- if dirtyAccounts == nil {
- dirtyAccounts = req
- } else {
- dirtyAccounts.merge(req)
- }
- launchNextRun = true
- pool.reorgDoneCh <- nextDone
-
- case tx := <-pool.queueTxEventCh:
- // Queue up the event, but don't schedule a reorg. It's up to the caller to
- // request one later if they want the events sent.
- addr, _ := types.Sender(pool.signer, tx)
- if _, ok := queuedEvents[addr]; !ok {
- queuedEvents[addr] = newSortedMap()
- }
- queuedEvents[addr].Put(tx)
-
- case <-curDone:
- curDone = nil
-
- case <-pool.reorgShutdownCh:
- // Wait for current run to finish.
- if curDone != nil {
- <-curDone
- }
- close(nextDone)
- return
- }
- }
-}
-
-// runReorg runs reset and promoteExecutables on behalf of scheduleReorgLoop.
-func (pool *TxPool) runReorg(done chan struct{}, reset *txpoolResetRequest, dirtyAccounts *accountSet, events map[common.Address]*sortedMap) {
- defer func(t0 time.Time) {
- reorgDurationTimer.Update(time.Since(t0))
- }(time.Now())
- defer close(done)
-
- var promoteAddrs []common.Address
- if dirtyAccounts != nil && reset == nil {
- // Only dirty accounts need to be promoted, unless we're resetting.
- // For resets, all addresses in the tx queue will be promoted and
- // the flatten operation can be avoided.
- promoteAddrs = dirtyAccounts.flatten()
- }
- pool.mu.Lock()
- if reset != nil {
- // Reset from the old head to the new, rescheduling any reorged transactions
- pool.reset(reset.oldHead, reset.newHead)
-
- // Nonces were reset, discard any events that became stale
- for addr := range events {
- events[addr].Forward(pool.pendingNonces.get(addr))
- if events[addr].Len() == 0 {
- delete(events, addr)
- }
- }
- // Reset needs promote for all addresses
- promoteAddrs = make([]common.Address, 0, len(pool.queue))
- for addr := range pool.queue {
- promoteAddrs = append(promoteAddrs, addr)
- }
- }
- // Check for pending transactions for every account that sent new ones
- promoted := pool.promoteExecutables(promoteAddrs)
-
- // If a new block appeared, validate the pool of pending transactions. This will
- // remove any transaction that has been included in the block or was invalidated
- // because of another transaction (e.g. higher gas price).
- if reset != nil {
- pool.demoteUnexecutables()
- if reset.newHead != nil && pool.chainconfig.IsLondon(new(big.Int).Add(reset.newHead.Number, big.NewInt(1))) {
- pendingBaseFee := misc.CalcBaseFee(pool.chainconfig, reset.newHead)
- pool.priced.SetBaseFee(pendingBaseFee)
- }
- // Update all accounts to the latest known pending nonce
- nonces := make(map[common.Address]uint64, len(pool.pending))
- for addr, list := range pool.pending {
- highestPending := list.LastElement()
- nonces[addr] = highestPending.Nonce() + 1
- }
- pool.pendingNonces.setAll(nonces)
- }
- // Ensure pool.queue and pool.pending sizes stay within the configured limits.
- pool.truncatePending()
- pool.truncateQueue()
-
- dropBetweenReorgHistogram.Update(int64(pool.changesSinceReorg))
- pool.changesSinceReorg = 0 // Reset change counter
- pool.mu.Unlock()
-
- // Notify subsystems for newly added transactions
- for _, tx := range promoted {
- addr, _ := types.Sender(pool.signer, tx)
- if _, ok := events[addr]; !ok {
- events[addr] = newSortedMap()
- }
- events[addr].Put(tx)
- }
- if len(events) > 0 {
- var txs []*types.Transaction
- for _, set := range events {
- txs = append(txs, set.Flatten()...)
- }
- pool.txFeed.Send(core.NewTxsEvent{Txs: txs})
- }
-}
-
-// reset retrieves the current state of the blockchain and ensures the content
-// of the transaction pool is valid with regard to the chain state.
-func (pool *TxPool) reset(oldHead, newHead *types.Header) {
- // If we're reorging an old state, reinject all dropped transactions
- var reinject types.Transactions
-
- if oldHead != nil && oldHead.Hash() != newHead.ParentHash {
- // If the reorg is too deep, avoid doing it (will happen during fast sync)
- oldNum := oldHead.Number.Uint64()
- newNum := newHead.Number.Uint64()
-
- if depth := uint64(math.Abs(float64(oldNum) - float64(newNum))); depth > 64 {
- log.Debug("Skipping deep transaction reorg", "depth", depth)
- } else {
- // Reorg seems shallow enough to pull in all transactions into memory
- var discarded, included types.Transactions
- var (
- rem = pool.chain.GetBlock(oldHead.Hash(), oldHead.Number.Uint64())
- add = pool.chain.GetBlock(newHead.Hash(), newHead.Number.Uint64())
- )
- if rem == nil {
- // This can happen if a setHead is performed, where we simply discard the old
- // head from the chain.
- // If that is the case, we don't have the lost transactions anymore, and
- // there's nothing to add
- if newNum >= oldNum {
- // If we reorged to a same or higher number, then it's not a case of setHead
- log.Warn("Transaction pool reset with missing oldhead",
- "old", oldHead.Hash(), "oldnum", oldNum, "new", newHead.Hash(), "newnum", newNum)
- return
- }
- // If the reorg ended up on a lower number, it's indicative of setHead being the cause
- log.Debug("Skipping transaction reset caused by setHead",
- "old", oldHead.Hash(), "oldnum", oldNum, "new", newHead.Hash(), "newnum", newNum)
- // We still need to update the current state s.th. the lost transactions can be readded by the user
- } else {
- for rem.NumberU64() > add.NumberU64() {
- discarded = append(discarded, rem.Transactions()...)
- if rem = pool.chain.GetBlock(rem.ParentHash(), rem.NumberU64()-1); rem == nil {
- log.Error("Unrooted old chain seen by tx pool", "block", oldHead.Number, "hash", oldHead.Hash())
- return
- }
- }
- for add.NumberU64() > rem.NumberU64() {
- included = append(included, add.Transactions()...)
- if add = pool.chain.GetBlock(add.ParentHash(), add.NumberU64()-1); add == nil {
- log.Error("Unrooted new chain seen by tx pool", "block", newHead.Number, "hash", newHead.Hash())
- return
- }
- }
- for rem.Hash() != add.Hash() {
- discarded = append(discarded, rem.Transactions()...)
- if rem = pool.chain.GetBlock(rem.ParentHash(), rem.NumberU64()-1); rem == nil {
- log.Error("Unrooted old chain seen by tx pool", "block", oldHead.Number, "hash", oldHead.Hash())
- return
- }
- included = append(included, add.Transactions()...)
- if add = pool.chain.GetBlock(add.ParentHash(), add.NumberU64()-1); add == nil {
- log.Error("Unrooted new chain seen by tx pool", "block", newHead.Number, "hash", newHead.Hash())
- return
- }
- }
- reinject = types.TxDifference(discarded, included)
- }
- }
- }
- // Initialize the internal state to the current head
- if newHead == nil {
- newHead = pool.chain.CurrentBlock() // Special case during testing
- }
- statedb, err := pool.chain.StateAt(newHead.Root)
- if err != nil {
- log.Error("Failed to reset txpool state", "err", err)
- return
- }
- pool.currentHead.Store(newHead)
- pool.currentState = statedb
- pool.pendingNonces = newNoncer(statedb)
-
- // Inject any transactions discarded due to reorgs
- log.Debug("Reinjecting stale transactions", "count", len(reinject))
- core.SenderCacher.Recover(pool.signer, reinject)
- pool.addTxsLocked(reinject, false)
-}
-
-// promoteExecutables moves transactions that have become processable from the
-// future queue to the set of pending transactions. During this process, all
-// invalidated transactions (low nonce, low balance) are deleted.
-func (pool *TxPool) promoteExecutables(accounts []common.Address) []*types.Transaction {
- // Track the promoted transactions to broadcast them at once
- var promoted []*types.Transaction
-
- // Iterate over all accounts and promote any executable transactions
- gasLimit := pool.currentHead.Load().GasLimit
- for _, addr := range accounts {
- list := pool.queue[addr]
- if list == nil {
- continue // Just in case someone calls with a non existing account
- }
- // Drop all transactions that are deemed too old (low nonce)
- forwards := list.Forward(pool.currentState.GetNonce(addr))
- for _, tx := range forwards {
- hash := tx.Hash()
- pool.all.Remove(hash)
- }
- log.Trace("Removed old queued transactions", "count", len(forwards))
- // Drop all transactions that are too costly (low balance or out of gas)
- drops, _ := list.Filter(pool.currentState.GetBalance(addr), gasLimit)
- for _, tx := range drops {
- hash := tx.Hash()
- pool.all.Remove(hash)
- }
- log.Trace("Removed unpayable queued transactions", "count", len(drops))
- queuedNofundsMeter.Mark(int64(len(drops)))
-
- // Gather all executable transactions and promote them
- readies := list.Ready(pool.pendingNonces.get(addr))
- for _, tx := range readies {
- hash := tx.Hash()
- if pool.promoteTx(addr, hash, tx) {
- promoted = append(promoted, tx)
- }
- }
- log.Trace("Promoted queued transactions", "count", len(promoted))
- queuedGauge.Dec(int64(len(readies)))
-
- // Drop all transactions over the allowed limit
- var caps types.Transactions
- if !pool.locals.contains(addr) {
- caps = list.Cap(int(pool.config.AccountQueue))
- for _, tx := range caps {
- hash := tx.Hash()
- pool.all.Remove(hash)
- log.Trace("Removed cap-exceeding queued transaction", "hash", hash)
- }
- queuedRateLimitMeter.Mark(int64(len(caps)))
- }
- // Mark all the items dropped as removed
- pool.priced.Removed(len(forwards) + len(drops) + len(caps))
- queuedGauge.Dec(int64(len(forwards) + len(drops) + len(caps)))
- if pool.locals.contains(addr) {
- localGauge.Dec(int64(len(forwards) + len(drops) + len(caps)))
- }
- // Delete the entire queue entry if it became empty.
- if list.Empty() {
- delete(pool.queue, addr)
- delete(pool.beats, addr)
- }
- }
- return promoted
-}
-
-// truncatePending removes transactions from the pending queue if the pool is above the
-// pending limit. The algorithm tries to reduce transaction counts by an approximately
-// equal number for all for accounts with many pending transactions.
-func (pool *TxPool) truncatePending() {
- pending := uint64(0)
- for _, list := range pool.pending {
- pending += uint64(list.Len())
- }
- if pending <= pool.config.GlobalSlots {
- return
- }
-
- pendingBeforeCap := pending
- // Assemble a spam order to penalize large transactors first
- spammers := prque.New[int64, common.Address](nil)
- for addr, list := range pool.pending {
- // Only evict transactions from high rollers
- if !pool.locals.contains(addr) && uint64(list.Len()) > pool.config.AccountSlots {
- spammers.Push(addr, int64(list.Len()))
- }
- }
- // Gradually drop transactions from offenders
- offenders := []common.Address{}
- for pending > pool.config.GlobalSlots && !spammers.Empty() {
- // Retrieve the next offender if not local address
- offender, _ := spammers.Pop()
- offenders = append(offenders, offender)
-
- // Equalize balances until all the same or below threshold
- if len(offenders) > 1 {
- // Calculate the equalization threshold for all current offenders
- threshold := pool.pending[offender].Len()
-
- // Iteratively reduce all offenders until below limit or threshold reached
- for pending > pool.config.GlobalSlots && pool.pending[offenders[len(offenders)-2]].Len() > threshold {
- for i := 0; i < len(offenders)-1; i++ {
- list := pool.pending[offenders[i]]
-
- caps := list.Cap(list.Len() - 1)
- for _, tx := range caps {
- // Drop the transaction from the global pools too
- hash := tx.Hash()
- pool.all.Remove(hash)
-
- // Update the account nonce to the dropped transaction
- pool.pendingNonces.setIfLower(offenders[i], tx.Nonce())
- log.Trace("Removed fairness-exceeding pending transaction", "hash", hash)
- }
- pool.priced.Removed(len(caps))
- pendingGauge.Dec(int64(len(caps)))
- if pool.locals.contains(offenders[i]) {
- localGauge.Dec(int64(len(caps)))
- }
- pending--
- }
- }
- }
- }
-
- // If still above threshold, reduce to limit or min allowance
- if pending > pool.config.GlobalSlots && len(offenders) > 0 {
- for pending > pool.config.GlobalSlots && uint64(pool.pending[offenders[len(offenders)-1]].Len()) > pool.config.AccountSlots {
- for _, addr := range offenders {
- list := pool.pending[addr]
-
- caps := list.Cap(list.Len() - 1)
- for _, tx := range caps {
- // Drop the transaction from the global pools too
- hash := tx.Hash()
- pool.all.Remove(hash)
-
- // Update the account nonce to the dropped transaction
- pool.pendingNonces.setIfLower(addr, tx.Nonce())
- log.Trace("Removed fairness-exceeding pending transaction", "hash", hash)
- }
- pool.priced.Removed(len(caps))
- pendingGauge.Dec(int64(len(caps)))
- if pool.locals.contains(addr) {
- localGauge.Dec(int64(len(caps)))
- }
- pending--
- }
- }
- }
- pendingRateLimitMeter.Mark(int64(pendingBeforeCap - pending))
-}
-
-// truncateQueue drops the oldest transactions in the queue if the pool is above the global queue limit.
-func (pool *TxPool) truncateQueue() {
- queued := uint64(0)
- for _, list := range pool.queue {
- queued += uint64(list.Len())
- }
- if queued <= pool.config.GlobalQueue {
- return
- }
-
- // Sort all accounts with queued transactions by heartbeat
- addresses := make(addressesByHeartbeat, 0, len(pool.queue))
- for addr := range pool.queue {
- if !pool.locals.contains(addr) { // don't drop locals
- addresses = append(addresses, addressByHeartbeat{addr, pool.beats[addr]})
- }
- }
- sort.Sort(sort.Reverse(addresses))
-
- // Drop transactions until the total is below the limit or only locals remain
- for drop := queued - pool.config.GlobalQueue; drop > 0 && len(addresses) > 0; {
- addr := addresses[len(addresses)-1]
- list := pool.queue[addr.address]
-
- addresses = addresses[:len(addresses)-1]
-
- // Drop all transactions if they are less than the overflow
- if size := uint64(list.Len()); size <= drop {
- for _, tx := range list.Flatten() {
- pool.removeTx(tx.Hash(), true)
- }
- drop -= size
- queuedRateLimitMeter.Mark(int64(size))
- continue
- }
- // Otherwise drop only last few transactions
- txs := list.Flatten()
- for i := len(txs) - 1; i >= 0 && drop > 0; i-- {
- pool.removeTx(txs[i].Hash(), true)
- drop--
- queuedRateLimitMeter.Mark(1)
- }
- }
-}
-
-// demoteUnexecutables removes invalid and processed transactions from the pools
-// executable/pending queue and any subsequent transactions that become unexecutable
-// are moved back into the future queue.
-//
-// Note: transactions are not marked as removed in the priced list because re-heaping
-// is always explicitly triggered by SetBaseFee and it would be unnecessary and wasteful
-// to trigger a re-heap is this function
-func (pool *TxPool) demoteUnexecutables() {
- // Iterate over all accounts and demote any non-executable transactions
- gasLimit := pool.currentHead.Load().GasLimit
- for addr, list := range pool.pending {
- nonce := pool.currentState.GetNonce(addr)
-
- // Drop all transactions that are deemed too old (low nonce)
- olds := list.Forward(nonce)
- for _, tx := range olds {
- hash := tx.Hash()
- pool.all.Remove(hash)
- log.Trace("Removed old pending transaction", "hash", hash)
- }
- // Drop all transactions that are too costly (low balance or out of gas), and queue any invalids back for later
- drops, invalids := list.Filter(pool.currentState.GetBalance(addr), gasLimit)
- for _, tx := range drops {
- hash := tx.Hash()
- log.Trace("Removed unpayable pending transaction", "hash", hash)
- pool.all.Remove(hash)
- }
- pendingNofundsMeter.Mark(int64(len(drops)))
-
- for _, tx := range invalids {
- hash := tx.Hash()
- log.Trace("Demoting pending transaction", "hash", hash)
-
- // Internal shuffle shouldn't touch the lookup set.
- pool.enqueueTx(hash, tx, false, false)
- }
- pendingGauge.Dec(int64(len(olds) + len(drops) + len(invalids)))
- if pool.locals.contains(addr) {
- localGauge.Dec(int64(len(olds) + len(drops) + len(invalids)))
- }
- // If there's a gap in front, alert (should never happen) and postpone all transactions
- if list.Len() > 0 && list.txs.Get(nonce) == nil {
- gapped := list.Cap(0)
- for _, tx := range gapped {
- hash := tx.Hash()
- log.Error("Demoting invalidated transaction", "hash", hash)
-
- // Internal shuffle shouldn't touch the lookup set.
- pool.enqueueTx(hash, tx, false, false)
- }
- pendingGauge.Dec(int64(len(gapped)))
- }
- // Delete the entire pending entry if it became empty.
- if list.Empty() {
- delete(pool.pending, addr)
- }
- }
-}
-
-// addressByHeartbeat is an account address tagged with its last activity timestamp.
-type addressByHeartbeat struct {
- address common.Address
- heartbeat time.Time
-}
-
-type addressesByHeartbeat []addressByHeartbeat
-
-func (a addressesByHeartbeat) Len() int { return len(a) }
-func (a addressesByHeartbeat) Less(i, j int) bool { return a[i].heartbeat.Before(a[j].heartbeat) }
-func (a addressesByHeartbeat) Swap(i, j int) { a[i], a[j] = a[j], a[i] }
-
-// accountSet is simply a set of addresses to check for existence, and a signer
-// capable of deriving addresses from transactions.
-type accountSet struct {
- accounts map[common.Address]struct{}
- signer types.Signer
- cache *[]common.Address
-}
-
-// newAccountSet creates a new address set with an associated signer for sender
-// derivations.
-func newAccountSet(signer types.Signer, addrs ...common.Address) *accountSet {
- as := &accountSet{
- accounts: make(map[common.Address]struct{}, len(addrs)),
- signer: signer,
- }
- for _, addr := range addrs {
- as.add(addr)
- }
- return as
-}
-
-// contains checks if a given address is contained within the set.
-func (as *accountSet) contains(addr common.Address) bool {
- _, exist := as.accounts[addr]
- return exist
-}
-
-// containsTx checks if the sender of a given tx is within the set. If the sender
-// cannot be derived, this method returns false.
-func (as *accountSet) containsTx(tx *types.Transaction) bool {
- if addr, err := types.Sender(as.signer, tx); err == nil {
- return as.contains(addr)
- }
- return false
-}
-
-// add inserts a new address into the set to track.
-func (as *accountSet) add(addr common.Address) {
- as.accounts[addr] = struct{}{}
- as.cache = nil
-}
-
-// addTx adds the sender of tx into the set.
-func (as *accountSet) addTx(tx *types.Transaction) {
- if addr, err := types.Sender(as.signer, tx); err == nil {
- as.add(addr)
- }
-}
-
-// flatten returns the list of addresses within this set, also caching it for later
-// reuse. The returned slice should not be changed!
-func (as *accountSet) flatten() []common.Address {
- if as.cache == nil {
- accounts := make([]common.Address, 0, len(as.accounts))
- for account := range as.accounts {
- accounts = append(accounts, account)
- }
- as.cache = &accounts
- }
- return *as.cache
-}
-
-// merge adds all addresses from the 'other' set into 'as'.
-func (as *accountSet) merge(other *accountSet) {
- for addr := range other.accounts {
- as.accounts[addr] = struct{}{}
- }
- as.cache = nil
-}
-
-// lookup is used internally by TxPool to track transactions while allowing
-// lookup without mutex contention.
-//
-// Note, although this type is properly protected against concurrent access, it
-// is **not** a type that should ever be mutated or even exposed outside of the
-// transaction pool, since its internal state is tightly coupled with the pools
-// internal mechanisms. The sole purpose of the type is to permit out-of-bound
-// peeking into the pool in TxPool.Get without having to acquire the widely scoped
-// TxPool.mu mutex.
-//
-// This lookup set combines the notion of "local transactions", which is useful
-// to build upper-level structure.
-type lookup struct {
- slots int
- lock sync.RWMutex
- locals map[common.Hash]*types.Transaction
- remotes map[common.Hash]*types.Transaction
-}
-
-// newLookup returns a new lookup structure.
-func newLookup() *lookup {
- return &lookup{
- locals: make(map[common.Hash]*types.Transaction),
- remotes: make(map[common.Hash]*types.Transaction),
- }
-}
-
-// Range calls f on each key and value present in the map. The callback passed
-// should return the indicator whether the iteration needs to be continued.
-// Callers need to specify which set (or both) to be iterated.
-func (t *lookup) Range(f func(hash common.Hash, tx *types.Transaction, local bool) bool, local bool, remote bool) {
- t.lock.RLock()
- defer t.lock.RUnlock()
-
- if local {
- for key, value := range t.locals {
- if !f(key, value, true) {
- return
- }
- }
- }
- if remote {
- for key, value := range t.remotes {
- if !f(key, value, false) {
- return
- }
- }
- }
-}
-
-// Get returns a transaction if it exists in the lookup, or nil if not found.
-func (t *lookup) Get(hash common.Hash) *types.Transaction {
- t.lock.RLock()
- defer t.lock.RUnlock()
-
- if tx := t.locals[hash]; tx != nil {
- return tx
- }
- return t.remotes[hash]
-}
-
-// GetLocal returns a transaction if it exists in the lookup, or nil if not found.
-func (t *lookup) GetLocal(hash common.Hash) *types.Transaction {
- t.lock.RLock()
- defer t.lock.RUnlock()
-
- return t.locals[hash]
-}
-
-// GetRemote returns a transaction if it exists in the lookup, or nil if not found.
-func (t *lookup) GetRemote(hash common.Hash) *types.Transaction {
- t.lock.RLock()
- defer t.lock.RUnlock()
-
- return t.remotes[hash]
-}
-
-// Count returns the current number of transactions in the lookup.
-func (t *lookup) Count() int {
- t.lock.RLock()
- defer t.lock.RUnlock()
-
- return len(t.locals) + len(t.remotes)
-}
-
-// LocalCount returns the current number of local transactions in the lookup.
-func (t *lookup) LocalCount() int {
- t.lock.RLock()
- defer t.lock.RUnlock()
-
- return len(t.locals)
-}
-
-// RemoteCount returns the current number of remote transactions in the lookup.
-func (t *lookup) RemoteCount() int {
- t.lock.RLock()
- defer t.lock.RUnlock()
-
- return len(t.remotes)
-}
-
-// Slots returns the current number of slots used in the lookup.
-func (t *lookup) Slots() int {
- t.lock.RLock()
- defer t.lock.RUnlock()
-
- return t.slots
-}
-
-// Add adds a transaction to the lookup.
-func (t *lookup) Add(tx *types.Transaction, local bool) {
- t.lock.Lock()
- defer t.lock.Unlock()
-
- t.slots += numSlots(tx)
- slotsGauge.Update(int64(t.slots))
-
- if local {
- t.locals[tx.Hash()] = tx
- } else {
- t.remotes[tx.Hash()] = tx
- }
-}
-
-// Remove removes a transaction from the lookup.
-func (t *lookup) Remove(hash common.Hash) {
- t.lock.Lock()
- defer t.lock.Unlock()
-
- tx, ok := t.locals[hash]
- if !ok {
- tx, ok = t.remotes[hash]
- }
- if !ok {
- log.Error("No transaction found to be deleted", "hash", hash)
- return
- }
- t.slots -= numSlots(tx)
- slotsGauge.Update(int64(t.slots))
-
- delete(t.locals, hash)
- delete(t.remotes, hash)
-}
-
-// RemoteToLocals migrates the transactions belongs to the given locals to locals
-// set. The assumption is held the locals set is thread-safe to be used.
-func (t *lookup) RemoteToLocals(locals *accountSet) int {
- t.lock.Lock()
- defer t.lock.Unlock()
-
- var migrated int
- for hash, tx := range t.remotes {
- if locals.containsTx(tx) {
- t.locals[hash] = tx
- delete(t.remotes, hash)
- migrated += 1
- }
- }
- return migrated
-}
-
-// RemotesBelowTip finds all remote transactions below the given tip threshold.
-func (t *lookup) RemotesBelowTip(threshold *big.Int) types.Transactions {
- found := make(types.Transactions, 0, 128)
- t.Range(func(hash common.Hash, tx *types.Transaction, local bool) bool {
- if tx.GasTipCapIntCmp(threshold) < 0 {
- found = append(found, tx)
- }
- return true
- }, false, true) // Only iterate remotes
- return found
-}
-
-// numSlots calculates the number of slots needed for a single transaction.
-func numSlots(tx *types.Transaction) int {
- return int((tx.Size() + txSlotSize - 1) / txSlotSize)
+ return TxStatusUnknown
}
diff --git a/core/types/transaction.go b/core/types/transaction.go
index b7cb36b60..579fc00d1 100644
--- a/core/types/transaction.go
+++ b/core/types/transaction.go
@@ -555,10 +555,10 @@ func (s *TxByPriceAndTime) Pop() interface{} {
// transactions in a profit-maximizing sorted order, while supporting removing
// entire batches of transactions for non-executable accounts.
type TransactionsByPriceAndNonce struct {
- txs map[common.Address]Transactions // Per account nonce-sorted list of transactions
- heads TxByPriceAndTime // Next transaction for each unique account (price heap)
- signer Signer // Signer for the set of transactions
- baseFee *big.Int // Current base fee
+ txs map[common.Address][]*Transaction // Per account nonce-sorted list of transactions
+ heads TxByPriceAndTime // Next transaction for each unique account (price heap)
+ signer Signer // Signer for the set of transactions
+ baseFee *big.Int // Current base fee
}
// NewTransactionsByPriceAndNonce creates a transaction set that can retrieve
@@ -566,7 +566,7 @@ type TransactionsByPriceAndNonce struct {
//
// Note, the input map is reowned so the caller should not interact any more with
// if after providing it to the constructor.
-func NewTransactionsByPriceAndNonce(signer Signer, txs map[common.Address]Transactions, baseFee *big.Int) *TransactionsByPriceAndNonce {
+func NewTransactionsByPriceAndNonce(signer Signer, txs map[common.Address][]*Transaction, baseFee *big.Int) *TransactionsByPriceAndNonce {
// Initialize a price and received time based heap with the head transactions
heads := make(TxByPriceAndTime, 0, len(txs))
for from, accTxs := range txs {
diff --git a/core/types/transaction_signing.go b/core/types/transaction_signing.go
index 59dd2e76e..863344082 100644
--- a/core/types/transaction_signing.go
+++ b/core/types/transaction_signing.go
@@ -57,9 +57,9 @@ func MakeSigner(config *params.ChainConfig, blockNumber *big.Int, blockTime uint
}
// LatestSigner returns the 'most permissive' Signer available for the given chain
-// configuration. Specifically, this enables support of EIP-155 replay protection and
-// EIP-2930 access list transactions when their respective forks are scheduled to occur at
-// any block number in the chain config.
+// configuration. Specifically, this enables support of all types of transacrions
+// when their respective forks are scheduled to occur at any block number (or time)
+// in the chain config.
//
// Use this in transaction-handling code where the current block number is unknown. If you
// have the current block number available, use MakeSigner instead.
diff --git a/core/types/transaction_test.go b/core/types/transaction_test.go
index f6442c5a3..dbe77fa60 100644
--- a/core/types/transaction_test.go
+++ b/core/types/transaction_test.go
@@ -281,7 +281,7 @@ func testTransactionPriceNonceSort(t *testing.T, baseFee *big.Int) {
signer := LatestSignerForChainID(common.Big1)
// Generate a batch of transactions with overlapping values, but shifted nonces
- groups := map[common.Address]Transactions{}
+ groups := map[common.Address][]*Transaction{}
expectedCount := 0
for start, key := range keys {
addr := crypto.PubkeyToAddress(key.PublicKey)
@@ -368,7 +368,7 @@ func TestTransactionTimeSort(t *testing.T) {
signer := HomesteadSigner{}
// Generate a batch of transactions with overlapping prices, but different creation times
- groups := map[common.Address]Transactions{}
+ groups := map[common.Address][]*Transaction{}
for start, key := range keys {
addr := crypto.PubkeyToAddress(key.PublicKey)
diff --git a/eth/api_backend.go b/eth/api_backend.go
index 927dcdef1..02d1946e8 100644
--- a/eth/api_backend.go
+++ b/eth/api_backend.go
@@ -294,7 +294,7 @@ func (b *EthAPIBackend) SubscribeLogsEvent(ch chan<- []*types.Log) event.Subscri
}
func (b *EthAPIBackend) SendTx(ctx context.Context, signedTx *types.Transaction) error {
- return b.eth.txPool.AddLocal(signedTx)
+ return b.eth.txPool.Add([]*txpool.Transaction{&txpool.Transaction{Tx: signedTx}}, true, false)[0]
}
func (b *EthAPIBackend) GetPoolTransactions() (types.Transactions, error) {
@@ -307,7 +307,10 @@ func (b *EthAPIBackend) GetPoolTransactions() (types.Transactions, error) {
}
func (b *EthAPIBackend) GetPoolTransaction(hash common.Hash) *types.Transaction {
- return b.eth.txPool.Get(hash)
+ if tx := b.eth.txPool.Get(hash); tx != nil {
+ return tx.Tx
+ }
+ return nil
}
func (b *EthAPIBackend) GetTransaction(ctx context.Context, txHash common.Hash) (*types.Transaction, common.Hash, uint64, uint64, error) {
@@ -319,24 +322,24 @@ func (b *EthAPIBackend) GetPoolNonce(ctx context.Context, addr common.Address) (
return b.eth.txPool.Nonce(addr), nil
}
-func (b *EthAPIBackend) Stats() (pending int, queued int) {
+func (b *EthAPIBackend) Stats() (runnable int, blocked int) {
return b.eth.txPool.Stats()
}
-func (b *EthAPIBackend) TxPoolContent() (map[common.Address]types.Transactions, map[common.Address]types.Transactions) {
- return b.eth.TxPool().Content()
+func (b *EthAPIBackend) TxPoolContent() (map[common.Address][]*types.Transaction, map[common.Address][]*types.Transaction) {
+ return b.eth.txPool.Content()
}
-func (b *EthAPIBackend) TxPoolContentFrom(addr common.Address) (types.Transactions, types.Transactions) {
- return b.eth.TxPool().ContentFrom(addr)
+func (b *EthAPIBackend) TxPoolContentFrom(addr common.Address) ([]*types.Transaction, []*types.Transaction) {
+ return b.eth.txPool.ContentFrom(addr)
}
func (b *EthAPIBackend) TxPool() *txpool.TxPool {
- return b.eth.TxPool()
+ return b.eth.txPool
}
func (b *EthAPIBackend) SubscribeNewTxsEvent(ch chan<- core.NewTxsEvent) event.Subscription {
- return b.eth.TxPool().SubscribeNewTxsEvent(ch)
+ return b.eth.txPool.SubscribeNewTxsEvent(ch)
}
func (b *EthAPIBackend) SyncProgress() ethereum.SyncProgress {
diff --git a/eth/backend.go b/eth/backend.go
index b3338918b..8d6977205 100644
--- a/eth/backend.go
+++ b/eth/backend.go
@@ -35,6 +35,7 @@ import (
"github.com/ethereum/go-ethereum/core/rawdb"
"github.com/ethereum/go-ethereum/core/state/pruner"
"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/eth/downloader"
@@ -66,7 +67,8 @@ type Ethereum struct {
config *ethconfig.Config
// Handlers
- txPool *txpool.TxPool
+ txPool *txpool.TxPool
+
blockchain *core.BlockChain
handler *handler
ethDialCandidates enode.Iterator
@@ -206,8 +208,12 @@ func New(stack *node.Node, config *ethconfig.Config) (*Ethereum, error) {
if config.TxPool.Journal != "" {
config.TxPool.Journal = stack.ResolvePath(config.TxPool.Journal)
}
- eth.txPool = txpool.New(config.TxPool, eth.blockchain.Config(), eth.blockchain)
+ legacyPool := legacypool.New(config.TxPool, eth.blockchain)
+ eth.txPool, err = txpool.New(new(big.Int).SetUint64(config.TxPool.PriceLimit), eth.blockchain, []txpool.SubPool{legacyPool})
+ if err != nil {
+ return nil, err
+ }
// Permit the downloader to use the trie cache allowance during fast sync
cacheLimit := cacheConfig.TrieCleanLimit + cacheConfig.TrieDirtyLimit + cacheConfig.SnapshotLimit
if eth.handler, err = newHandler(&handlerConfig{
@@ -512,7 +518,7 @@ func (s *Ethereum) Stop() error {
// Then stop everything else.
s.bloomIndexer.Close()
close(s.closeBloomHandler)
- s.txPool.Stop()
+ s.txPool.Close()
s.miner.Close()
s.blockchain.Stop()
s.engine.Close()
diff --git a/eth/catalyst/api_test.go b/eth/catalyst/api_test.go
index 5bab7ba18..254a510ab 100644
--- a/eth/catalyst/api_test.go
+++ b/eth/catalyst/api_test.go
@@ -35,6 +35,7 @@ import (
beaconConsensus "github.com/ethereum/go-ethereum/consensus/beacon"
"github.com/ethereum/go-ethereum/consensus/ethash"
"github.com/ethereum/go-ethereum/core"
+ "github.com/ethereum/go-ethereum/core/txpool"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/eth"
@@ -106,7 +107,7 @@ func TestEth2AssembleBlock(t *testing.T) {
if err != nil {
t.Fatalf("error signing transaction, err=%v", err)
}
- ethservice.TxPool().AddLocal(tx)
+ ethservice.TxPool().Add([]*txpool.Transaction{{Tx: tx}}, true, false)
blockParams := engine.PayloadAttributes{
Timestamp: blocks[9].Time() + 5,
}
@@ -142,7 +143,12 @@ func TestEth2AssembleBlockWithAnotherBlocksTxs(t *testing.T) {
api := NewConsensusAPI(ethservice)
// Put the 10th block's tx in the pool and produce a new block
- api.eth.TxPool().AddRemotesSync(blocks[9].Transactions())
+ txs := blocks[9].Transactions()
+ wrapped := make([]*txpool.Transaction, len(txs))
+ for i, tx := range txs {
+ wrapped[i] = &txpool.Transaction{Tx: tx}
+ }
+ api.eth.TxPool().Add(wrapped, false, true)
blockParams := engine.PayloadAttributes{
Timestamp: blocks[8].Time() + 5,
}
@@ -181,7 +187,12 @@ func TestEth2PrepareAndGetPayload(t *testing.T) {
api := NewConsensusAPI(ethservice)
// Put the 10th block's tx in the pool and produce a new block
- ethservice.TxPool().AddLocals(blocks[9].Transactions())
+ txs := blocks[9].Transactions()
+ wrapped := make([]*txpool.Transaction, len(txs))
+ for i, tx := range txs {
+ wrapped[i] = &txpool.Transaction{Tx: tx}
+ }
+ ethservice.TxPool().Add(wrapped, true, false)
blockParams := engine.PayloadAttributes{
Timestamp: blocks[8].Time() + 5,
}
@@ -303,7 +314,7 @@ func TestEth2NewBlock(t *testing.T) {
statedb, _ := ethservice.BlockChain().StateAt(parent.Root())
nonce := statedb.GetNonce(testAddr)
tx, _ := types.SignTx(types.NewContractCreation(nonce, new(big.Int), 1000000, big.NewInt(2*params.InitialBaseFee), logCode), types.LatestSigner(ethservice.BlockChain().Config()), testKey)
- ethservice.TxPool().AddLocal(tx)
+ ethservice.TxPool().Add([]*txpool.Transaction{{Tx: tx}}, true, false)
execData, err := assembleWithTransactions(api, parent.Hash(), &engine.PayloadAttributes{
Timestamp: parent.Time() + 5,
@@ -472,7 +483,7 @@ func TestFullAPI(t *testing.T) {
statedb, _ := ethservice.BlockChain().StateAt(parent.Root)
nonce := statedb.GetNonce(testAddr)
tx, _ := types.SignTx(types.NewContractCreation(nonce, new(big.Int), 1000000, big.NewInt(2*params.InitialBaseFee), logCode), types.LatestSigner(ethservice.BlockChain().Config()), testKey)
- ethservice.TxPool().AddLocal(tx)
+ ethservice.TxPool().Add([]*txpool.Transaction{{Tx: tx}}, true, false)
}
setupBlocks(t, ethservice, 10, parent, callback, nil)
@@ -598,7 +609,7 @@ func TestNewPayloadOnInvalidChain(t *testing.T) {
GasPrice: big.NewInt(2 * params.InitialBaseFee),
Data: logCode,
})
- ethservice.TxPool().AddRemotesSync([]*types.Transaction{tx})
+ ethservice.TxPool().Add([]*txpool.Transaction{{Tx: tx}}, false, true)
var (
params = engine.PayloadAttributes{
Timestamp: parent.Time + 1,
@@ -1272,7 +1283,7 @@ func setupBodies(t *testing.T) (*node.Node, *eth.Ethereum, []*types.Block) {
statedb, _ := ethservice.BlockChain().StateAt(parent.Root)
nonce := statedb.GetNonce(testAddr)
tx, _ := types.SignTx(types.NewContractCreation(nonce, new(big.Int), 1000000, big.NewInt(2*params.InitialBaseFee), logCode), types.LatestSigner(ethservice.BlockChain().Config()), testKey)
- ethservice.TxPool().AddLocal(tx)
+ ethservice.TxPool().Add([]*txpool.Transaction{{Tx: tx}}, true, false)
}
withdrawals := make([][]*types.Withdrawal, 10)
diff --git a/eth/ethconfig/config.go b/eth/ethconfig/config.go
index a98d9ee4a..432ec8ffb 100644
--- a/eth/ethconfig/config.go
+++ b/eth/ethconfig/config.go
@@ -27,7 +27,7 @@ import (
"github.com/ethereum/go-ethereum/consensus/clique"
"github.com/ethereum/go-ethereum/consensus/ethash"
"github.com/ethereum/go-ethereum/core"
- "github.com/ethereum/go-ethereum/core/txpool"
+ "github.com/ethereum/go-ethereum/core/txpool/legacypool"
"github.com/ethereum/go-ethereum/eth/downloader"
"github.com/ethereum/go-ethereum/eth/gasprice"
"github.com/ethereum/go-ethereum/ethdb"
@@ -71,7 +71,7 @@ var Defaults = Config{
SnapshotCache: 102,
FilterLogCacheSize: 32,
Miner: miner.DefaultConfig,
- TxPool: txpool.DefaultConfig,
+ TxPool: legacypool.DefaultConfig,
RPCGasCap: 50000000,
RPCEVMTimeout: 5 * time.Second,
GPO: FullNodeGPO,
@@ -139,7 +139,7 @@ type Config struct {
Miner miner.Config
// Transaction pool options
- TxPool txpool.Config
+ TxPool legacypool.Config
// Gas Price Oracle options
GPO gasprice.Config
diff --git a/eth/ethconfig/gen_config.go b/eth/ethconfig/gen_config.go
index 0e2f03774..5e1d75e3b 100644
--- a/eth/ethconfig/gen_config.go
+++ b/eth/ethconfig/gen_config.go
@@ -7,7 +7,7 @@ import (
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core"
- "github.com/ethereum/go-ethereum/core/txpool"
+ "github.com/ethereum/go-ethereum/core/txpool/legacypool"
"github.com/ethereum/go-ethereum/eth/downloader"
"github.com/ethereum/go-ethereum/eth/gasprice"
"github.com/ethereum/go-ethereum/miner"
@@ -47,7 +47,7 @@ func (c Config) MarshalTOML() (interface{}, error) {
Preimages bool
FilterLogCacheSize int
Miner miner.Config
- TxPool txpool.Config
+ TxPool legacypool.Config
GPO gasprice.Config
EnablePreimageRecording bool
DocRoot string `toml:"-"`
@@ -133,7 +133,7 @@ func (c *Config) UnmarshalTOML(unmarshal func(interface{}) error) error {
Preimages *bool
FilterLogCacheSize *int
Miner *miner.Config
- TxPool *txpool.Config
+ TxPool *legacypool.Config
GPO *gasprice.Config
EnablePreimageRecording *bool
DocRoot *string `toml:"-"`
diff --git a/eth/fetcher/tx_fetcher.go b/eth/fetcher/tx_fetcher.go
index 39727e007..638727340 100644
--- a/eth/fetcher/tx_fetcher.go
+++ b/eth/fetcher/tx_fetcher.go
@@ -169,9 +169,9 @@ type TxFetcher struct {
alternates map[common.Hash]map[string]struct{} // In-flight transaction alternate origins if retrieval fails
// Callbacks
- hasTx func(common.Hash) bool // Retrieves a tx from the local txpool
- addTxs func([]*types.Transaction) []error // Insert a batch of transactions into local txpool
- fetchTxs func(string, []common.Hash) error // Retrieves a set of txs from a remote peer
+ hasTx func(common.Hash) bool // Retrieves a tx from the local txpool
+ addTxs func([]*txpool.Transaction) []error // Insert a batch of transactions into local txpool
+ fetchTxs func(string, []common.Hash) error // Retrieves a set of txs from a remote peer
step chan struct{} // Notification channel when the fetcher loop iterates
clock mclock.Clock // Time wrapper to simulate in tests
@@ -180,14 +180,14 @@ type TxFetcher struct {
// NewTxFetcher creates a transaction fetcher to retrieve transaction
// based on hash announcements.
-func NewTxFetcher(hasTx func(common.Hash) bool, addTxs func([]*types.Transaction) []error, fetchTxs func(string, []common.Hash) error) *TxFetcher {
+func NewTxFetcher(hasTx func(common.Hash) bool, addTxs func([]*txpool.Transaction) []error, fetchTxs func(string, []common.Hash) error) *TxFetcher {
return NewTxFetcherForTests(hasTx, addTxs, fetchTxs, mclock.System{}, nil)
}
// NewTxFetcherForTests is a testing method to mock out the realtime clock with
// a simulated version and the internal randomness with a deterministic one.
func NewTxFetcherForTests(
- hasTx func(common.Hash) bool, addTxs func([]*types.Transaction) []error, fetchTxs func(string, []common.Hash) error,
+ hasTx func(common.Hash) bool, addTxs func([]*txpool.Transaction) []error, fetchTxs func(string, []common.Hash) error,
clock mclock.Clock, rand *mrand.Rand) *TxFetcher {
return &TxFetcher{
notify: make(chan *txAnnounce),
@@ -294,7 +294,12 @@ func (f *TxFetcher) Enqueue(peer string, txs []*types.Transaction, direct bool)
otherreject int64
)
batch := txs[i:end]
- for j, err := range f.addTxs(batch) {
+
+ wrapped := make([]*txpool.Transaction, len(batch))
+ for j, tx := range batch {
+ wrapped[j] = &txpool.Transaction{Tx: tx}
+ }
+ for j, err := range f.addTxs(wrapped) {
// Track the transaction hash if the price is too low for us.
// Avoid re-request this transaction when we receive another
// announcement.
diff --git a/eth/fetcher/tx_fetcher_test.go b/eth/fetcher/tx_fetcher_test.go
index 1715def99..c5805d6ef 100644
--- a/eth/fetcher/tx_fetcher_test.go
+++ b/eth/fetcher/tx_fetcher_test.go
@@ -378,7 +378,7 @@ func TestTransactionFetcherCleanup(t *testing.T) {
init: func() *TxFetcher {
return NewTxFetcher(
func(common.Hash) bool { return false },
- func(txs []*types.Transaction) []error {
+ func(txs []*txpool.Transaction) []error {
return make([]error, len(txs))
},
func(string, []common.Hash) error { return nil },
@@ -417,7 +417,7 @@ func TestTransactionFetcherCleanupEmpty(t *testing.T) {
init: func() *TxFetcher {
return NewTxFetcher(
func(common.Hash) bool { return false },
- func(txs []*types.Transaction) []error {
+ func(txs []*txpool.Transaction) []error {
return make([]error, len(txs))
},
func(string, []common.Hash) error { return nil },
@@ -455,7 +455,7 @@ func TestTransactionFetcherMissingRescheduling(t *testing.T) {
init: func() *TxFetcher {
return NewTxFetcher(
func(common.Hash) bool { return false },
- func(txs []*types.Transaction) []error {
+ func(txs []*txpool.Transaction) []error {
return make([]error, len(txs))
},
func(string, []common.Hash) error { return nil },
@@ -501,7 +501,7 @@ func TestTransactionFetcherMissingCleanup(t *testing.T) {
init: func() *TxFetcher {
return NewTxFetcher(
func(common.Hash) bool { return false },
- func(txs []*types.Transaction) []error {
+ func(txs []*txpool.Transaction) []error {
return make([]error, len(txs))
},
func(string, []common.Hash) error { return nil },
@@ -539,7 +539,7 @@ func TestTransactionFetcherBroadcasts(t *testing.T) {
init: func() *TxFetcher {
return NewTxFetcher(
func(common.Hash) bool { return false },
- func(txs []*types.Transaction) []error {
+ func(txs []*txpool.Transaction) []error {
return make([]error, len(txs))
},
func(string, []common.Hash) error { return nil },
@@ -644,7 +644,7 @@ func TestTransactionFetcherTimeoutRescheduling(t *testing.T) {
init: func() *TxFetcher {
return NewTxFetcher(
func(common.Hash) bool { return false },
- func(txs []*types.Transaction) []error {
+ func(txs []*txpool.Transaction) []error {
return make([]error, len(txs))
},
func(string, []common.Hash) error { return nil },
@@ -865,7 +865,7 @@ func TestTransactionFetcherUnderpricedDedup(t *testing.T) {
init: func() *TxFetcher {
return NewTxFetcher(
func(common.Hash) bool { return false },
- func(txs []*types.Transaction) []error {
+ func(txs []*txpool.Transaction) []error {
errs := make([]error, len(txs))
for i := 0; i < len(errs); i++ {
if i%2 == 0 {
@@ -938,7 +938,7 @@ func TestTransactionFetcherUnderpricedDoSProtection(t *testing.T) {
init: func() *TxFetcher {
return NewTxFetcher(
func(common.Hash) bool { return false },
- func(txs []*types.Transaction) []error {
+ func(txs []*txpool.Transaction) []error {
errs := make([]error, len(txs))
for i := 0; i < len(errs); i++ {
errs[i] = txpool.ErrUnderpriced
@@ -964,7 +964,7 @@ func TestTransactionFetcherOutOfBoundDeliveries(t *testing.T) {
init: func() *TxFetcher {
return NewTxFetcher(
func(common.Hash) bool { return false },
- func(txs []*types.Transaction) []error {
+ func(txs []*txpool.Transaction) []error {
return make([]error, len(txs))
},
func(string, []common.Hash) error { return nil },
@@ -1017,7 +1017,7 @@ func TestTransactionFetcherDrop(t *testing.T) {
init: func() *TxFetcher {
return NewTxFetcher(
func(common.Hash) bool { return false },
- func(txs []*types.Transaction) []error {
+ func(txs []*txpool.Transaction) []error {
return make([]error, len(txs))
},
func(string, []common.Hash) error { return nil },
@@ -1083,7 +1083,7 @@ func TestTransactionFetcherDropRescheduling(t *testing.T) {
init: func() *TxFetcher {
return NewTxFetcher(
func(common.Hash) bool { return false },
- func(txs []*types.Transaction) []error {
+ func(txs []*txpool.Transaction) []error {
return make([]error, len(txs))
},
func(string, []common.Hash) error { return nil },
@@ -1128,7 +1128,7 @@ func TestTransactionFetcherFuzzCrash01(t *testing.T) {
init: func() *TxFetcher {
return NewTxFetcher(
func(common.Hash) bool { return false },
- func(txs []*types.Transaction) []error {
+ func(txs []*txpool.Transaction) []error {
return make([]error, len(txs))
},
func(string, []common.Hash) error { return nil },
@@ -1155,7 +1155,7 @@ func TestTransactionFetcherFuzzCrash02(t *testing.T) {
init: func() *TxFetcher {
return NewTxFetcher(
func(common.Hash) bool { return false },
- func(txs []*types.Transaction) []error {
+ func(txs []*txpool.Transaction) []error {
return make([]error, len(txs))
},
func(string, []common.Hash) error { return nil },
@@ -1184,7 +1184,7 @@ func TestTransactionFetcherFuzzCrash03(t *testing.T) {
init: func() *TxFetcher {
return NewTxFetcher(
func(common.Hash) bool { return false },
- func(txs []*types.Transaction) []error {
+ func(txs []*txpool.Transaction) []error {
return make([]error, len(txs))
},
func(string, []common.Hash) error { return nil },
@@ -1217,7 +1217,7 @@ func TestTransactionFetcherFuzzCrash04(t *testing.T) {
init: func() *TxFetcher {
return NewTxFetcher(
func(common.Hash) bool { return false },
- func(txs []*types.Transaction) []error {
+ func(txs []*txpool.Transaction) []error {
return make([]error, len(txs))
},
func(string, []common.Hash) error {
diff --git a/eth/handler.go b/eth/handler.go
index f0b043166..46113fbc2 100644
--- a/eth/handler.go
+++ b/eth/handler.go
@@ -29,6 +29,7 @@ import (
"github.com/ethereum/go-ethereum/consensus/beacon"
"github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/forkid"
+ "github.com/ethereum/go-ethereum/core/txpool"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/eth/downloader"
"github.com/ethereum/go-ethereum/eth/fetcher"
@@ -59,14 +60,14 @@ type txPool interface {
// Get retrieves the transaction from local txpool with given
// tx hash.
- Get(hash common.Hash) *types.Transaction
+ Get(hash common.Hash) *txpool.Transaction
- // AddRemotes should add the given transactions to the pool.
- AddRemotes([]*types.Transaction) []error
+ // Add should add the given transactions to the pool.
+ Add(txs []*txpool.Transaction, local bool, sync bool) []error
// Pending should return pending transactions.
// The slice should be modifiable by the caller.
- Pending(enforceTips bool) map[common.Address]types.Transactions
+ Pending(enforceTips bool) map[common.Address][]*types.Transaction
// SubscribeNewTxsEvent should return an event subscription of
// NewTxsEvent and send events to the given channel.
@@ -274,7 +275,10 @@ func newHandler(config *handlerConfig) (*handler, error) {
}
return p.RequestTxs(hashes)
}
- h.txFetcher = fetcher.NewTxFetcher(h.txpool.Has, h.txpool.AddRemotes, fetchTx)
+ addTxs := func(txs []*txpool.Transaction) []error {
+ return h.txpool.Add(txs, false, false)
+ }
+ h.txFetcher = fetcher.NewTxFetcher(h.txpool.Has, addTxs, fetchTx)
h.chainSync = newChainSyncer(h)
return h, nil
}
diff --git a/eth/handler_eth_test.go b/eth/handler_eth_test.go
index fd7df2340..a9ce83e2e 100644
--- a/eth/handler_eth_test.go
+++ b/eth/handler_eth_test.go
@@ -28,6 +28,7 @@ import (
"github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/forkid"
"github.com/ethereum/go-ethereum/core/rawdb"
+ "github.com/ethereum/go-ethereum/core/txpool"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/core/vm"
"github.com/ethereum/go-ethereum/eth/downloader"
@@ -307,15 +308,15 @@ func testSendTransactions(t *testing.T, protocol uint) {
handler := newTestHandler()
defer handler.close()
- insert := make([]*types.Transaction, 100)
+ insert := make([]*txpool.Transaction, 100)
for nonce := range insert {
tx := types.NewTransaction(uint64(nonce), common.Address{}, big.NewInt(0), 100000, big.NewInt(0), make([]byte, 10240))
tx, _ = types.SignTx(tx, types.HomesteadSigner{}, testKey)
- insert[nonce] = tx
+ insert[nonce] = &txpool.Transaction{Tx: tx}
}
- go handler.txpool.AddRemotes(insert) // Need goroutine to not block on feed
- time.Sleep(250 * time.Millisecond) // Wait until tx events get out of the system (can't use events, tx broadcaster races with peer join)
+ go handler.txpool.Add(insert, false, false) // Need goroutine to not block on feed
+ time.Sleep(250 * time.Millisecond) // Wait until tx events get out of the system (can't use events, tx broadcaster races with peer join)
// Create a source handler to send messages through and a sink peer to receive them
p2pSrc, p2pSink := p2p.MsgPipe()
@@ -375,8 +376,8 @@ func testSendTransactions(t *testing.T, protocol uint) {
}
}
for _, tx := range insert {
- if _, ok := seen[tx.Hash()]; !ok {
- t.Errorf("missing transaction: %x", tx.Hash())
+ if _, ok := seen[tx.Tx.Hash()]; !ok {
+ t.Errorf("missing transaction: %x", tx.Tx.Hash())
}
}
}
@@ -433,14 +434,14 @@ func testTransactionPropagation(t *testing.T, protocol uint) {
defer sub.Unsubscribe()
}
// Fill the source pool with transactions and wait for them at the sinks
- txs := make([]*types.Transaction, 1024)
+ txs := make([]*txpool.Transaction, 1024)
for nonce := range txs {
tx := types.NewTransaction(uint64(nonce), common.Address{}, big.NewInt(0), 100000, big.NewInt(0), nil)
tx, _ = types.SignTx(tx, types.HomesteadSigner{}, testKey)
- txs[nonce] = tx
+ txs[nonce] = &txpool.Transaction{Tx: tx}
}
- source.txpool.AddRemotes(txs)
+ source.txpool.Add(txs, false, false)
// Iterate through all the sinks and ensure they all got the transactions
for i := range sinks {
diff --git a/eth/handler_test.go b/eth/handler_test.go
index 8939e53a9..2f776e874 100644
--- a/eth/handler_test.go
+++ b/eth/handler_test.go
@@ -26,6 +26,7 @@ import (
"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/types"
"github.com/ethereum/go-ethereum/core/vm"
"github.com/ethereum/go-ethereum/crypto"
@@ -71,32 +72,40 @@ func (p *testTxPool) Has(hash common.Hash) bool {
// Get retrieves the transaction from local txpool with given
// tx hash.
-func (p *testTxPool) Get(hash common.Hash) *types.Transaction {
+func (p *testTxPool) Get(hash common.Hash) *txpool.Transaction {
p.lock.Lock()
defer p.lock.Unlock()
- return p.pool[hash]
+ if tx := p.pool[hash]; tx != nil {
+ return &txpool.Transaction{Tx: tx}
+ }
+ return nil
}
-// AddRemotes appends a batch of transactions to the pool, and notifies any
+// Add appends a batch of transactions to the pool, and notifies any
// listeners if the addition channel is non nil
-func (p *testTxPool) AddRemotes(txs []*types.Transaction) []error {
+func (p *testTxPool) Add(txs []*txpool.Transaction, local bool, sync bool) []error {
+ unwrapped := make([]*types.Transaction, len(txs))
+ for i, tx := range txs {
+ unwrapped[i] = tx.Tx
+ }
p.lock.Lock()
defer p.lock.Unlock()
- for _, tx := range txs {
+ for _, tx := range unwrapped {
p.pool[tx.Hash()] = tx
}
- p.txFeed.Send(core.NewTxsEvent{Txs: txs})
- return make([]error, len(txs))
+
+ p.txFeed.Send(core.NewTxsEvent{Txs: unwrapped})
+ return make([]error, len(unwrapped))
}
// Pending returns all the transactions known to the pool
-func (p *testTxPool) Pending(enforceTips bool) map[common.Address]types.Transactions {
+func (p *testTxPool) Pending(enforceTips bool) map[common.Address][]*types.Transaction {
p.lock.RLock()
defer p.lock.RUnlock()
- batches := make(map[common.Address]types.Transactions)
+ batches := make(map[common.Address][]*types.Transaction)
for _, tx := range p.pool {
from, _ := types.Sender(types.HomesteadSigner{}, tx)
batches[from] = append(batches[from], tx)
diff --git a/eth/protocols/eth/broadcast.go b/eth/protocols/eth/broadcast.go
index 3045303f2..c431aa400 100644
--- a/eth/protocols/eth/broadcast.go
+++ b/eth/protocols/eth/broadcast.go
@@ -81,8 +81,8 @@ func (p *Peer) broadcastTransactions() {
)
for i := 0; i < len(queue) && size < maxTxPacketSize; i++ {
if tx := p.txpool.Get(queue[i]); tx != nil {
- txs = append(txs, tx)
- size += common.StorageSize(tx.Size())
+ txs = append(txs, tx.Tx)
+ size += common.StorageSize(tx.Tx.Size())
}
hashesCount++
}
@@ -151,8 +151,8 @@ func (p *Peer) announceTransactions() {
for count = 0; count < len(queue) && size < maxTxPacketSize; count++ {
if tx := p.txpool.Get(queue[count]); tx != nil {
pending = append(pending, queue[count])
- pendingTypes = append(pendingTypes, tx.Type())
- pendingSizes = append(pendingSizes, uint32(tx.Size()))
+ pendingTypes = append(pendingTypes, tx.Tx.Type())
+ pendingSizes = append(pendingSizes, uint32(tx.Tx.Size()))
size += common.HashLength
}
}
diff --git a/eth/protocols/eth/handler.go b/eth/protocols/eth/handler.go
index 313686427..7f51d4f5c 100644
--- a/eth/protocols/eth/handler.go
+++ b/eth/protocols/eth/handler.go
@@ -23,7 +23,7 @@ import (
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core"
- "github.com/ethereum/go-ethereum/core/types"
+ "github.com/ethereum/go-ethereum/core/txpool"
"github.com/ethereum/go-ethereum/metrics"
"github.com/ethereum/go-ethereum/p2p"
"github.com/ethereum/go-ethereum/p2p/enode"
@@ -90,7 +90,7 @@ type Backend interface {
// TxPool defines the methods needed by the protocol handler to serve transactions.
type TxPool interface {
// Get retrieves the transaction from the local txpool with the given hash.
- Get(hash common.Hash) *types.Transaction
+ Get(hash common.Hash) *txpool.Transaction
}
// MakeProtocols constructs the P2P protocol definitions for `eth`.
diff --git a/eth/protocols/eth/handler_test.go b/eth/protocols/eth/handler_test.go
index 21782103c..3f1dc9fe7 100644
--- a/eth/protocols/eth/handler_test.go
+++ b/eth/protocols/eth/handler_test.go
@@ -30,6 +30,7 @@ import (
"github.com/ethereum/go-ethereum/core/rawdb"
"github.com/ethereum/go-ethereum/core/state"
"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"
@@ -113,19 +114,22 @@ func newTestBackendWithGenerator(blocks int, shanghai bool, generator func(int,
for _, block := range bs {
chain.StateCache().TrieDB().Commit(block.Root(), false)
}
- txconfig := txpool.DefaultConfig
+ txconfig := legacypool.DefaultConfig
txconfig.Journal = "" // Don't litter the disk with test journals
+ pool := legacypool.New(txconfig, chain)
+ txpool, _ := txpool.New(new(big.Int).SetUint64(txconfig.PriceLimit), chain, []txpool.SubPool{pool})
+
return &testBackend{
db: db,
chain: chain,
- txpool: txpool.New(txconfig, params.TestChainConfig, chain),
+ txpool: txpool,
}
}
// close tears down the transaction pool and chain behind the mock backend.
func (b *testBackend) close() {
- b.txpool.Stop()
+ b.txpool.Close()
b.chain.Stop()
}
diff --git a/eth/protocols/eth/handlers.go b/eth/protocols/eth/handlers.go
index 74e514b86..f9fbf72b7 100644
--- a/eth/protocols/eth/handlers.go
+++ b/eth/protocols/eth/handlers.go
@@ -498,7 +498,7 @@ func answerGetPooledTransactions(backend Backend, query GetPooledTransactionsPac
continue
}
// If known, encode and queue for response packet
- if encoded, err := rlp.EncodeToBytes(tx); err != nil {
+ if encoded, err := rlp.EncodeToBytes(tx.Tx); err != nil {
log.Error("Failed to encode transaction", "err", err)
} else {
hashes = append(hashes, hash)
diff --git a/event/multisub.go b/event/multisub.go
new file mode 100644
index 000000000..5c8d2df48
--- /dev/null
+++ b/event/multisub.go
@@ -0,0 +1,50 @@
+// 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 event
+
+// JoinSubscriptions joins multiple subscriptions to be able to track them as
+// one entity and collectively cancel them of consume any errors from them.
+func JoinSubscriptions(subs ...Subscription) Subscription {
+ return NewSubscription(func(unsubbed <-chan struct{}) error {
+ // Unsubscribe all subscriptions before returning
+ defer func() {
+ for _, sub := range subs {
+ sub.Unsubscribe()
+ }
+ }()
+ // Wait for an error on any of the subscriptions and propagate up
+ errc := make(chan error, len(subs))
+ for i := range subs {
+ go func(sub Subscription) {
+ select {
+ case err := <-sub.Err():
+ if err != nil {
+ errc <- err
+ }
+ case <-unsubbed:
+ }
+ }(subs[i])
+ }
+
+ select {
+ case err := <-errc:
+ return err
+ case <-unsubbed:
+ return nil
+ }
+ })
+}
diff --git a/event/multisub_test.go b/event/multisub_test.go
new file mode 100644
index 000000000..c92bcfae9
--- /dev/null
+++ b/event/multisub_test.go
@@ -0,0 +1,175 @@
+// 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 event
+
+import (
+ "testing"
+ "time"
+)
+
+func TestMultisub(t *testing.T) {
+ // Create a double subscription and ensure events propagate through
+ var (
+ feed1 Feed
+ feed2 Feed
+ )
+ sink1 := make(chan int, 1)
+ sink2 := make(chan int, 1)
+
+ sub1 := feed1.Subscribe(sink1)
+ sub2 := feed2.Subscribe(sink2)
+
+ sub := JoinSubscriptions(sub1, sub2)
+
+ feed1.Send(1)
+ select {
+ case n := <-sink1:
+ if n != 1 {
+ t.Errorf("sink 1 delivery mismatch: have %d, want %d", n, 1)
+ }
+ default:
+ t.Error("sink 1 missing delivery")
+ }
+
+ feed2.Send(2)
+ select {
+ case n := <-sink2:
+ if n != 2 {
+ t.Errorf("sink 2 delivery mismatch: have %d, want %d", n, 2)
+ }
+ default:
+ t.Error("sink 2 missing delivery")
+ }
+ // Unsubscribe and ensure no more events are delivered
+ sub.Unsubscribe()
+ select {
+ case <-sub.Err():
+ case <-time.After(50 * time.Millisecond):
+ t.Error("multisub didn't propagate closure")
+ }
+
+ feed1.Send(11)
+ select {
+ case n := <-sink1:
+ t.Errorf("sink 1 unexpected delivery: %d", n)
+ default:
+ }
+
+ feed2.Send(22)
+ select {
+ case n := <-sink2:
+ t.Errorf("sink 2 unexpected delivery: %d", n)
+ default:
+ }
+}
+
+func TestMutisubPartialUnsubscribe(t *testing.T) {
+ // Create a double subscription but terminate one half, ensuring no error
+ // is propagated yet up to the outer subscription
+ var (
+ feed1 Feed
+ feed2 Feed
+ )
+ sink1 := make(chan int, 1)
+ sink2 := make(chan int, 1)
+
+ sub1 := feed1.Subscribe(sink1)
+ sub2 := feed2.Subscribe(sink2)
+
+ sub := JoinSubscriptions(sub1, sub2)
+
+ sub1.Unsubscribe()
+ select {
+ case <-sub.Err():
+ t.Error("multisub propagated closure")
+ case <-time.After(50 * time.Millisecond):
+ }
+ // Ensure that events cross only the second feed
+ feed1.Send(1)
+ select {
+ case n := <-sink1:
+ t.Errorf("sink 1 unexpected delivery: %d", n)
+ default:
+ }
+
+ feed2.Send(2)
+ select {
+ case n := <-sink2:
+ if n != 2 {
+ t.Errorf("sink 2 delivery mismatch: have %d, want %d", n, 2)
+ }
+ default:
+ t.Error("sink 2 missing delivery")
+ }
+ // Unsubscribe and ensure no more events are delivered
+ sub.Unsubscribe()
+ select {
+ case <-sub.Err():
+ case <-time.After(50 * time.Millisecond):
+ t.Error("multisub didn't propagate closure")
+ }
+
+ feed1.Send(11)
+ select {
+ case n := <-sink1:
+ t.Errorf("sink 1 unexpected delivery: %d", n)
+ default:
+ }
+
+ feed2.Send(22)
+ select {
+ case n := <-sink2:
+ t.Errorf("sink 2 unexpected delivery: %d", n)
+ default:
+ }
+}
+
+func TestMultisubFullUnsubscribe(t *testing.T) {
+ // Create a double subscription and terminate the multi sub, ensuring an
+ // error is propagated up.
+ var (
+ feed1 Feed
+ feed2 Feed
+ )
+ sink1 := make(chan int, 1)
+ sink2 := make(chan int, 1)
+
+ sub1 := feed1.Subscribe(sink1)
+ sub2 := feed2.Subscribe(sink2)
+
+ sub := JoinSubscriptions(sub1, sub2)
+ sub.Unsubscribe()
+ select {
+ case <-sub.Err():
+ case <-time.After(50 * time.Millisecond):
+ t.Error("multisub didn't propagate closure")
+ }
+ // Ensure no more events are delivered
+ feed1.Send(1)
+ select {
+ case n := <-sink1:
+ t.Errorf("sink 1 unexpected delivery: %d", n)
+ default:
+ }
+
+ feed2.Send(2)
+ select {
+ case n := <-sink2:
+ t.Errorf("sink 2 unexpected delivery: %d", n)
+ default:
+ }
+}
diff --git a/internal/ethapi/api_test.go b/internal/ethapi/api_test.go
index f80f11147..7c503dc39 100644
--- a/internal/ethapi/api_test.go
+++ b/internal/ethapi/api_test.go
@@ -366,10 +366,10 @@ func (b testBackend) GetPoolNonce(ctx context.Context, addr common.Address) (uin
panic("implement me")
}
func (b testBackend) Stats() (pending int, queued int) { panic("implement me") }
-func (b testBackend) TxPoolContent() (map[common.Address]types.Transactions, map[common.Address]types.Transactions) {
+func (b testBackend) TxPoolContent() (map[common.Address][]*types.Transaction, map[common.Address][]*types.Transaction) {
panic("implement me")
}
-func (b testBackend) TxPoolContentFrom(addr common.Address) (types.Transactions, types.Transactions) {
+func (b testBackend) TxPoolContentFrom(addr common.Address) ([]*types.Transaction, []*types.Transaction) {
panic("implement me")
}
func (b testBackend) SubscribeNewTxsEvent(events chan<- core.NewTxsEvent) event.Subscription {
diff --git a/internal/ethapi/backend.go b/internal/ethapi/backend.go
index 918b3b630..458fb811e 100644
--- a/internal/ethapi/backend.go
+++ b/internal/ethapi/backend.go
@@ -80,8 +80,8 @@ type Backend interface {
GetPoolTransaction(txHash common.Hash) *types.Transaction
GetPoolNonce(ctx context.Context, addr common.Address) (uint64, error)
Stats() (pending int, queued int)
- TxPoolContent() (map[common.Address]types.Transactions, map[common.Address]types.Transactions)
- TxPoolContentFrom(addr common.Address) (types.Transactions, types.Transactions)
+ TxPoolContent() (map[common.Address][]*types.Transaction, map[common.Address][]*types.Transaction)
+ TxPoolContentFrom(addr common.Address) ([]*types.Transaction, []*types.Transaction)
SubscribeNewTxsEvent(chan<- core.NewTxsEvent) event.Subscription
ChainConfig() *params.ChainConfig
diff --git a/internal/ethapi/transaction_args_test.go b/internal/ethapi/transaction_args_test.go
index 0868f8762..9161d5e68 100644
--- a/internal/ethapi/transaction_args_test.go
+++ b/internal/ethapi/transaction_args_test.go
@@ -325,10 +325,10 @@ func (b *backendMock) GetPoolNonce(ctx context.Context, addr common.Address) (ui
return 0, nil
}
func (b *backendMock) Stats() (pending int, queued int) { return 0, 0 }
-func (b *backendMock) TxPoolContent() (map[common.Address]types.Transactions, map[common.Address]types.Transactions) {
+func (b *backendMock) TxPoolContent() (map[common.Address][]*types.Transaction, map[common.Address][]*types.Transaction) {
return nil, nil
}
-func (b *backendMock) TxPoolContentFrom(addr common.Address) (types.Transactions, types.Transactions) {
+func (b *backendMock) TxPoolContentFrom(addr common.Address) ([]*types.Transaction, []*types.Transaction) {
return nil, nil
}
func (b *backendMock) SubscribeNewTxsEvent(chan<- core.NewTxsEvent) event.Subscription { return nil }
diff --git a/internal/flags/categories.go b/internal/flags/categories.go
index c2db6c6c1..ac7193187 100644
--- a/internal/flags/categories.go
+++ b/internal/flags/categories.go
@@ -23,7 +23,8 @@ const (
LightCategory = "LIGHT CLIENT"
DevCategory = "DEVELOPER CHAIN"
EthashCategory = "ETHASH"
- TxPoolCategory = "TRANSACTION POOL"
+ TxPoolCategory = "TRANSACTION POOL (EVM)"
+ BlobPoolCategory = "TRANSACTION POOL (BLOB)"
PerfCategory = "PERFORMANCE TUNING"
AccountCategory = "ACCOUNT"
APICategory = "API AND CONSOLE"
diff --git a/les/api_backend.go b/les/api_backend.go
index 97665757b..dee3e3bfd 100644
--- a/les/api_backend.go
+++ b/les/api_backend.go
@@ -224,11 +224,11 @@ func (b *LesApiBackend) Stats() (pending int, queued int) {
return b.eth.txPool.Stats(), 0
}
-func (b *LesApiBackend) TxPoolContent() (map[common.Address]types.Transactions, map[common.Address]types.Transactions) {
+func (b *LesApiBackend) TxPoolContent() (map[common.Address][]*types.Transaction, map[common.Address][]*types.Transaction) {
return b.eth.txPool.Content()
}
-func (b *LesApiBackend) TxPoolContentFrom(addr common.Address) (types.Transactions, types.Transactions) {
+func (b *LesApiBackend) TxPoolContentFrom(addr common.Address) ([]*types.Transaction, []*types.Transaction) {
return b.eth.txPool.ContentFrom(addr)
}
diff --git a/les/server_requests.go b/les/server_requests.go
index 033b11d79..29f8f63ce 100644
--- a/les/server_requests.go
+++ b/les/server_requests.go
@@ -518,12 +518,7 @@ func handleSendTx(msg Decoder) (serveRequestFn, uint64, uint64, error) {
hash := tx.Hash()
stats[i] = txStatus(backend, hash)
if stats[i].Status == txpool.TxStatusUnknown {
- addFn := backend.TxPool().AddRemotes
- // Add txs synchronously for testing purpose
- if backend.AddTxsSync() {
- addFn = backend.TxPool().AddRemotesSync
- }
- if errs := addFn([]*types.Transaction{tx}); errs[0] != nil {
+ if errs := backend.TxPool().Add([]*txpool.Transaction{{Tx: tx}}, false, backend.AddTxsSync()); errs[0] != nil {
stats[i].Error = errs[0].Error()
continue
}
@@ -556,7 +551,7 @@ func handleGetTxStatus(msg Decoder) (serveRequestFn, uint64, uint64, error) {
func txStatus(b serverBackend, hash common.Hash) light.TxStatus {
var stat light.TxStatus
// Looking the transaction in txpool first.
- stat.Status = b.TxPool().Status([]common.Hash{hash})[0]
+ stat.Status = b.TxPool().Status(hash)
// If the transaction is unknown to the pool, try looking it up locally.
if stat.Status == txpool.TxStatusUnknown {
diff --git a/les/test_helper.go b/les/test_helper.go
index 44a454eae..057903557 100644
--- a/les/test_helper.go
+++ b/les/test_helper.go
@@ -37,6 +37,7 @@ import (
"github.com/ethereum/go-ethereum/core/forkid"
"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/eth/ethconfig"
@@ -232,9 +233,11 @@ func newTestServerHandler(blocks int, indexers []*core.ChainIndexer, db ethdb.Da
simulation := backends.NewSimulatedBackendWithDatabase(db, gspec.Alloc, 100000000)
prepare(blocks, simulation)
- txpoolConfig := txpool.DefaultConfig
+ txpoolConfig := legacypool.DefaultConfig
txpoolConfig.Journal = ""
- txpool := txpool.New(txpoolConfig, gspec.Config, simulation.Blockchain())
+
+ pool := legacypool.New(txpoolConfig, simulation.Blockchain())
+ txpool, _ := txpool.New(new(big.Int).SetUint64(txpoolConfig.PriceLimit), simulation.Blockchain(), []txpool.SubPool{pool})
server := &LesServer{
lesCommons: lesCommons{
diff --git a/light/txpool.go b/light/txpool.go
index 8d2a189c0..b792d70b1 100644
--- a/light/txpool.go
+++ b/light/txpool.go
@@ -494,29 +494,29 @@ func (pool *TxPool) GetTransactions() (txs types.Transactions, err error) {
// Content retrieves the data content of the transaction pool, returning all the
// pending as well as queued transactions, grouped by account and nonce.
-func (pool *TxPool) Content() (map[common.Address]types.Transactions, map[common.Address]types.Transactions) {
+func (pool *TxPool) Content() (map[common.Address][]*types.Transaction, map[common.Address][]*types.Transaction) {
pool.mu.RLock()
defer pool.mu.RUnlock()
// Retrieve all the pending transactions and sort by account and by nonce
- pending := make(map[common.Address]types.Transactions)
+ pending := make(map[common.Address][]*types.Transaction)
for _, tx := range pool.pending {
account, _ := types.Sender(pool.signer, tx)
pending[account] = append(pending[account], tx)
}
// There are no queued transactions in a light pool, just return an empty map
- queued := make(map[common.Address]types.Transactions)
+ queued := make(map[common.Address][]*types.Transaction)
return pending, queued
}
// ContentFrom retrieves the data content of the transaction pool, returning the
// pending as well as queued transactions of this address, grouped by nonce.
-func (pool *TxPool) ContentFrom(addr common.Address) (types.Transactions, types.Transactions) {
+func (pool *TxPool) ContentFrom(addr common.Address) ([]*types.Transaction, []*types.Transaction) {
pool.mu.RLock()
defer pool.mu.RUnlock()
// Retrieve the pending transactions and sort by nonce
- var pending types.Transactions
+ var pending []*types.Transaction
for _, tx := range pool.pending {
account, _ := types.Sender(pool.signer, tx)
if account != addr {
@@ -525,7 +525,7 @@ func (pool *TxPool) ContentFrom(addr common.Address) (types.Transactions, types.
pending = append(pending, tx)
}
// There are no queued transactions in a light pool, just return an empty map
- return pending, types.Transactions{}
+ return pending, []*types.Transaction{}
}
// RemoveTransactions removes all given transactions from the pool.
diff --git a/miner/miner_test.go b/miner/miner_test.go
index 67d038d68..3f84446ff 100644
--- a/miner/miner_test.go
+++ b/miner/miner_test.go
@@ -29,10 +29,12 @@ import (
"github.com/ethereum/go-ethereum/core/rawdb"
"github.com/ethereum/go-ethereum/core/state"
"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/eth/downloader"
"github.com/ethereum/go-ethereum/event"
+ "github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/trie"
)
@@ -61,11 +63,16 @@ func (m *mockBackend) StateAtBlock(block *types.Block, reexec uint64, base *stat
}
type testBlockChain struct {
+ config *params.ChainConfig
statedb *state.StateDB
gasLimit uint64
chainHeadFeed *event.Feed
}
+func (bc *testBlockChain) Config() *params.ChainConfig {
+ return bc.config
+}
+
func (bc *testBlockChain) CurrentBlock() *types.Header {
return &types.Header{
Number: new(big.Int),
@@ -265,10 +272,12 @@ func createMiner(t *testing.T) (*Miner, *event.TypeMux, func(skipMiner bool)) {
t.Fatalf("can't create new chain %v", err)
}
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabase(chainDB), nil)
- blockchain := &testBlockChain{statedb, 10000000, new(event.Feed)}
+ blockchain := &testBlockChain{chainConfig, statedb, 10000000, new(event.Feed)}
- pool := txpool.New(testTxPoolConfig, chainConfig, blockchain)
- backend := NewMockBackend(bc, pool)
+ pool := legacypool.New(testTxPoolConfig, blockchain)
+ txpool, _ := txpool.New(new(big.Int).SetUint64(testTxPoolConfig.PriceLimit), blockchain, []txpool.SubPool{pool})
+
+ backend := NewMockBackend(bc, txpool)
// Create event Mux
mux := new(event.TypeMux)
// Create Miner
@@ -276,7 +285,7 @@ func createMiner(t *testing.T) (*Miner, *event.TypeMux, func(skipMiner bool)) {
cleanup := func(skipMiner bool) {
bc.Stop()
engine.Close()
- pool.Stop()
+ txpool.Close()
if !skipMiner {
miner.Close()
}
diff --git a/miner/stress/clique/main.go b/miner/stress/clique/main.go
index 4742dc438..53ff2450c 100644
--- a/miner/stress/clique/main.go
+++ b/miner/stress/clique/main.go
@@ -31,6 +31,7 @@ import (
"github.com/ethereum/go-ethereum/common/fdlimit"
"github.com/ethereum/go-ethereum/core"
"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/eth"
@@ -132,7 +133,7 @@ func main() {
if err != nil {
panic(err)
}
- if err := backend.TxPool().AddLocal(tx); err != nil {
+ if err := backend.TxPool().Add([]*txpool.Transaction{{Tx: tx}}, true, false); err != nil {
panic(err)
}
nonces[index]++
@@ -206,7 +207,7 @@ func makeSealer(genesis *core.Genesis) (*node.Node, *eth.Ethereum, error) {
SyncMode: downloader.FullSync,
DatabaseCache: 256,
DatabaseHandles: 256,
- TxPool: txpool.DefaultConfig,
+ TxPool: legacypool.DefaultConfig,
GPO: ethconfig.Defaults.GPO,
Miner: miner.Config{
GasCeil: genesis.GasLimit * 11 / 10,
diff --git a/miner/worker.go b/miner/worker.go
index d61f8dcad..fb30142aa 100644
--- a/miner/worker.go
+++ b/miner/worker.go
@@ -533,7 +533,7 @@ func (w *worker) mainLoop() {
if gp := w.current.gasPool; gp != nil && gp.Gas() < params.TxGas {
continue
}
- txs := make(map[common.Address]types.Transactions, len(ev.Txs))
+ txs := make(map[common.Address][]*types.Transaction, len(ev.Txs))
for _, tx := range ev.Txs {
acc, _ := types.Sender(w.current.signer, tx)
txs[acc] = append(txs[acc], tx)
@@ -904,7 +904,13 @@ func (w *worker) fillTransactions(interrupt *atomic.Int32, env *environment) err
// Split the pending transactions into locals and remotes
// Fill the block with all available pending transactions.
pending := w.eth.TxPool().Pending(true)
- localTxs, remoteTxs := make(map[common.Address]types.Transactions), pending
+ /*blobtxs := w.eth.BlobPool().Pending(
+ uint256.MustFromBig(env.header.BaseFee),
+ uint256.MustFromBig(misc.CalcBlobFee(*env.header.ExcessDataGas)),
+ )
+ log.Trace("Side-effect log, much wow", "blobs", len(blobtxs))*/
+
+ localTxs, remoteTxs := make(map[common.Address][]*types.Transaction), pending
for _, account := range w.eth.TxPool().Locals() {
if txs := remoteTxs[account]; len(txs) > 0 {
delete(remoteTxs, account)
diff --git a/miner/worker_test.go b/miner/worker_test.go
index d58382e1e..80557d99b 100644
--- a/miner/worker_test.go
+++ b/miner/worker_test.go
@@ -30,6 +30,7 @@ import (
"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"
@@ -49,7 +50,7 @@ const (
var (
// Test chain configurations
- testTxPoolConfig txpool.Config
+ testTxPoolConfig legacypool.Config
ethashChainConfig *params.ChainConfig
cliqueChainConfig *params.ChainConfig
@@ -62,7 +63,7 @@ var (
testUserAddress = crypto.PubkeyToAddress(testUserKey.PublicKey)
// Test transactions
- pendingTxs []*types.Transaction
+ pendingTxs []*txpool.Transaction
newTxs []*types.Transaction
testConfig = &Config{
@@ -72,7 +73,7 @@ var (
)
func init() {
- testTxPoolConfig = txpool.DefaultConfig
+ testTxPoolConfig = legacypool.DefaultConfig
testTxPoolConfig.Journal = ""
ethashChainConfig = new(params.ChainConfig)
*ethashChainConfig = *params.TestChainConfig
@@ -92,7 +93,7 @@ func init() {
Gas: params.TxGas,
GasPrice: big.NewInt(params.InitialBaseFee),
})
- pendingTxs = append(pendingTxs, tx1)
+ pendingTxs = append(pendingTxs, &txpool.Transaction{Tx: tx1})
tx2 := types.MustSignNewTx(testBankKey, signer, &types.LegacyTx{
Nonce: 1,
@@ -132,10 +133,13 @@ func newTestWorkerBackend(t *testing.T, chainConfig *params.ChainConfig, engine
if err != nil {
t.Fatalf("core.NewBlockChain failed: %v", err)
}
+ pool := legacypool.New(testTxPoolConfig, chain)
+ txpool, _ := txpool.New(new(big.Int).SetUint64(testTxPoolConfig.PriceLimit), chain, []txpool.SubPool{pool})
+
return &testWorkerBackend{
db: db,
chain: chain,
- txPool: txpool.New(testTxPoolConfig, chainConfig, chain),
+ txPool: txpool,
genesis: gspec,
}
}
@@ -156,7 +160,7 @@ func (b *testWorkerBackend) newRandomTx(creation bool) *types.Transaction {
func newTestWorker(t *testing.T, chainConfig *params.ChainConfig, engine consensus.Engine, db ethdb.Database, blocks int) (*worker, *testWorkerBackend) {
backend := newTestWorkerBackend(t, chainConfig, engine, db, blocks)
- backend.txPool.AddLocals(pendingTxs)
+ backend.txPool.Add(pendingTxs, true, false)
w := newWorker(testConfig, chainConfig, engine, backend, new(event.TypeMux), nil, false)
w.setEtherbase(testBankAddress)
return w, backend
@@ -190,8 +194,8 @@ func TestGenerateAndImportBlock(t *testing.T) {
w.start()
for i := 0; i < 5; i++ {
- b.txPool.AddLocal(b.newRandomTx(true))
- b.txPool.AddLocal(b.newRandomTx(false))
+ b.txPool.Add([]*txpool.Transaction{{Tx: b.newRandomTx(true)}}, true, false)
+ b.txPool.Add([]*txpool.Transaction{{Tx: b.newRandomTx(false)}}, true, false)
select {
case ev := <-sub.Chan():
diff --git a/params/protocol_params.go b/params/protocol_params.go
index c662e4a3f..9a0b8115b 100644
--- a/params/protocol_params.go
+++ b/params/protocol_params.go
@@ -160,6 +160,8 @@ const (
RefundQuotient uint64 = 2
RefundQuotientEIP3529 uint64 = 5
+ BlobTxBytesPerFieldElement = 32 // Size in bytes of a field element
+ BlobTxFieldElementsPerBlob = 4096 // Number of field elements stored in a single data blob
BlobTxHashVersion = 0x01 // Version byte of the commitment hash
BlobTxMaxDataGasPerBlock = 1 << 19 // Maximum consumable data gas for data blobs per block
BlobTxTargetDataGasPerBlock = 1 << 18 // Target consumable data gas for data blobs per block (for 1559-like pricing)
diff --git a/tests/fuzzers/les/les-fuzzer.go b/tests/fuzzers/les/les-fuzzer.go
index c203c87f8..b4599025e 100644
--- a/tests/fuzzers/les/les-fuzzer.go
+++ b/tests/fuzzers/les/les-fuzzer.go
@@ -27,6 +27,7 @@ import (
"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"
@@ -128,6 +129,9 @@ type fuzzer struct {
}
func newFuzzer(input []byte) *fuzzer {
+ pool := legacypool.New(legacypool.DefaultConfig, chain)
+ txpool, _ := txpool.New(new(big.Int).SetUint64(legacypool.DefaultConfig.PriceLimit), chain, []txpool.SubPool{pool})
+
return &fuzzer{
chain: chain,
chainLen: testChainLen,
@@ -138,7 +142,7 @@ func newFuzzer(input []byte) *fuzzer {
chtKeys: chtKeys,
bloomKeys: bloomKeys,
nonce: uint64(len(txHashes)),
- pool: txpool.New(txpool.DefaultConfig, params.TestChainConfig, chain),
+ pool: txpool,
input: bytes.NewReader(input),
}
}
diff --git a/tests/fuzzers/txfetcher/txfetcher_fuzzer.go b/tests/fuzzers/txfetcher/txfetcher_fuzzer.go
index d1d6fdc66..56b6b1e64 100644
--- a/tests/fuzzers/txfetcher/txfetcher_fuzzer.go
+++ b/tests/fuzzers/txfetcher/txfetcher_fuzzer.go
@@ -25,6 +25,7 @@ import (
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/mclock"
+ "github.com/ethereum/go-ethereum/core/txpool"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/eth/fetcher"
)
@@ -79,7 +80,7 @@ func Fuzz(input []byte) int {
f := fetcher.NewTxFetcherForTests(
func(common.Hash) bool { return false },
- func(txs []*types.Transaction) []error {
+ func(txs []*txpool.Transaction) []error {
return make([]error, len(txs))
},
func(string, []common.Hash) error { return nil },