core/txpool/legacypool: get the tests in order

pull/30559/head
Martin Holst Swende 2 months ago
parent 86912e033c
commit 4e77d9921b
No known key found for this signature in database
GPG Key ID: 683B438C05A5DDF0
  1. 22
      core/txpool/legacypool/legacypool.go
  2. 545
      core/txpool/legacypool/legacypool_test.go
  3. 8
      core/txpool/legacypool/list.go

@ -397,7 +397,7 @@ func (pool *LegacyPool) SetGasTip(tip *big.Int) {
// If the min miner fee increased, remove transactions below the new threshold
if newTip.Cmp(old) > 0 {
// pool.priced is sorted by GasFeeCap, so we have to iterate through pool.all instead
drop := pool.all.RemotesBelowTip(tip)
drop := pool.all.TxsBelowTip(tip)
for _, tx := range drop {
pool.removeTx(tx.Hash(), false, true)
}
@ -1685,14 +1685,6 @@ func (t *lookup) Get(hash common.Hash) *types.Transaction {
return t.remotes[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()
@ -1701,14 +1693,6 @@ func (t *lookup) Count() int {
return len(t.remotes)
}
// 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()
@ -1744,8 +1728,8 @@ func (t *lookup) Remove(hash common.Hash) {
delete(t.remotes, hash)
}
// RemotesBelowTip finds all remote transactions below the given tip threshold.
func (t *lookup) RemotesBelowTip(threshold *big.Int) types.Transactions {
// TxsBelowTip finds all remote transactions below the given tip threshold.
func (t *lookup) TxsBelowTip(threshold *big.Int) types.Transactions {
found := make(types.Transactions, 0, 128)
t.Range(func(hash common.Hash, tx *types.Transaction) bool {
if tx.GasTipCapIntCmp(threshold) < 0 {

@ -23,7 +23,6 @@ import (
"fmt"
"math/big"
"math/rand"
"os"
"sync"
"sync/atomic"
"testing"
@ -183,7 +182,7 @@ func validatePoolInternals(pool *LegacyPool) error {
return fmt.Errorf("total transaction count %d != %d pending + %d queued", total, pending, queued)
}
pool.priced.Reheap()
priced, remote := pool.priced.urgent.Len()+pool.priced.floating.Len(), pool.all.RemoteCount()
priced, remote := pool.priced.urgent.Len()+pool.priced.floating.Len(), pool.all.Count()
if priced != remote {
return fmt.Errorf("total priced transaction count %d != %d", priced, remote)
}
@ -350,9 +349,6 @@ func TestInvalidTransactions(t *testing.T) {
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 {
t.Error("expected", nil, "got", err)
}
}
func TestQueue(t *testing.T) {
@ -366,7 +362,7 @@ func TestQueue(t *testing.T) {
testAddBalance(pool, from, big.NewInt(1000))
<-pool.requestReset(nil, nil)
pool.enqueueTx(tx.Hash(), tx, false, true)
pool.enqueueTx(tx.Hash(), tx, true)
<-pool.requestPromoteExecutables(newAccountSet(pool.signer, from))
if len(pool.pending) != 1 {
t.Error("expected valid txs to be 1 is", len(pool.pending))
@ -375,7 +371,7 @@ func TestQueue(t *testing.T) {
tx = transaction(1, 100, key)
from, _ = deriveSender(tx)
testSetNonce(pool, from, 2)
pool.enqueueTx(tx.Hash(), tx, false, true)
pool.enqueueTx(tx.Hash(), tx, true)
<-pool.requestPromoteExecutables(newAccountSet(pool.signer, from))
if _, ok := pool.pending[from].txs.items[tx.Nonce()]; ok {
@ -399,9 +395,9 @@ func TestQueue2(t *testing.T) {
testAddBalance(pool, from, big.NewInt(1000))
pool.reset(nil, nil)
pool.enqueueTx(tx1.Hash(), tx1, false, true)
pool.enqueueTx(tx2.Hash(), tx2, false, true)
pool.enqueueTx(tx3.Hash(), tx3, false, true)
pool.enqueueTx(tx1.Hash(), tx1, true)
pool.enqueueTx(tx2.Hash(), tx2, true)
pool.enqueueTx(tx3.Hash(), tx3, true)
pool.promoteExecutables([]common.Address{from})
if len(pool.pending) != 1 {
@ -476,14 +472,14 @@ func TestChainFork(t *testing.T) {
resetState()
tx := transaction(0, 100000, key)
if _, err := pool.add(tx, false); err != nil {
if _, err := pool.add(tx); err != nil {
t.Error("didn't expect error", err)
}
pool.removeTx(tx.Hash(), true, true)
// reset the pool's internal state
resetState()
if _, err := pool.add(tx, false); err != nil {
if _, err := pool.add(tx); err != nil {
t.Error("didn't expect error", err)
}
}
@ -510,10 +506,10 @@ func TestDoubleNonce(t *testing.T) {
tx3, _ := types.SignTx(types.NewTransaction(0, common.Address{}, big.NewInt(100), 1000000, big.NewInt(1), nil), signer, key)
// Add the first two transaction, ensure higher priced stays only
if replace, err := pool.add(tx1, false); err != nil || replace {
if replace, err := pool.add(tx1); err != nil || replace {
t.Errorf("first transaction insert failed (%v) or reported replacement (%v)", err, replace)
}
if replace, err := pool.add(tx2, false); err != nil || !replace {
if replace, err := pool.add(tx2); err != nil || !replace {
t.Errorf("second transaction insert failed (%v) or not reported replacement (%v)", err, replace)
}
<-pool.requestPromoteExecutables(newAccountSet(signer, addr))
@ -525,7 +521,7 @@ func TestDoubleNonce(t *testing.T) {
}
// Add the third transaction and ensure it's not saved (smaller price)
pool.add(tx3, false)
pool.add(tx3)
<-pool.requestPromoteExecutables(newAccountSet(signer, addr))
if pool.pending[addr].Len() != 1 {
t.Error("expected 1 pending transactions, got", pool.pending[addr].Len())
@ -548,7 +544,7 @@ func TestMissingNonce(t *testing.T) {
addr := crypto.PubkeyToAddress(key.PublicKey)
testAddBalance(pool, addr, big.NewInt(100000000000000))
tx := transaction(1, 100000, key)
if _, err := pool.add(tx, false); err != nil {
if _, err := pool.add(tx); err != nil {
t.Error("didn't expect error", err)
}
if len(pool.pending) != 0 {
@ -607,21 +603,21 @@ func TestDropping(t *testing.T) {
tx11 = transaction(11, 200, key)
tx12 = transaction(12, 300, key)
)
pool.all.Add(tx0, false)
pool.priced.Put(tx0, false)
pool.all.Add(tx0)
pool.priced.Put(tx0)
pool.promoteTx(account, tx0.Hash(), tx0)
pool.all.Add(tx1, false)
pool.priced.Put(tx1, false)
pool.all.Add(tx1)
pool.priced.Put(tx1)
pool.promoteTx(account, tx1.Hash(), tx1)
pool.all.Add(tx2, false)
pool.priced.Put(tx2, false)
pool.all.Add(tx2)
pool.priced.Put(tx2)
pool.promoteTx(account, tx2.Hash(), tx2)
pool.enqueueTx(tx10.Hash(), tx10, false, true)
pool.enqueueTx(tx11.Hash(), tx11, false, true)
pool.enqueueTx(tx12.Hash(), tx12, false, true)
pool.enqueueTx(tx10.Hash(), tx10, true)
pool.enqueueTx(tx11.Hash(), tx11, true)
pool.enqueueTx(tx12.Hash(), tx12, true)
// Check that pre and post validations leave the pool as is
if pool.pending[account].Len() != 3 {
@ -899,13 +895,6 @@ func TestQueueAccountLimiting(t *testing.T) {
// This logic should not hold for local transactions, unless the local tracking
// mechanism is disabled.
func TestQueueGlobalLimiting(t *testing.T) {
testQueueGlobalLimiting(t, false)
}
func TestQueueGlobalLimitingNoLocals(t *testing.T) {
testQueueGlobalLimiting(t, true)
}
func testQueueGlobalLimiting(t *testing.T, nolocals bool) {
t.Parallel()
// Create the pool to test the limit enforcement with
@ -913,7 +902,7 @@ func testQueueGlobalLimiting(t *testing.T, nolocals bool) {
blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed))
config := testTxPoolConfig
config.NoLocals = nolocals
config.NoLocals = true
config.GlobalQueue = config.AccountQueue*3 - 1 // reduce the queue limits to shorten test time (-1 to make it non divisible)
pool := New(config, blockchain)
@ -926,7 +915,6 @@ func testQueueGlobalLimiting(t *testing.T, nolocals bool) {
keys[i], _ = crypto.GenerateKey()
testAddBalance(pool, crypto.PubkeyToAddress(keys[i].PublicKey), big.NewInt(1000000))
}
local := keys[len(keys)-1]
// Generate and queue a batch of transactions
nonces := make(map[common.Address]uint64)
@ -952,51 +940,12 @@ func testQueueGlobalLimiting(t *testing.T, nolocals bool) {
if queued > int(config.GlobalQueue) {
t.Fatalf("total transactions overflow allowance: %d > %d", queued, config.GlobalQueue)
}
// Generate a batch of transactions from the local account and import them
txs = txs[:0]
for i := uint64(0); i < 3*config.GlobalQueue; i++ {
txs = append(txs, transaction(i+1, 100000, local))
}
pool.addLocals(txs)
// If locals are disabled, the previous eviction algorithm should apply here too
if nolocals {
queued := 0
for addr, list := range pool.queue {
if list.Len() > int(config.AccountQueue) {
t.Errorf("addr %x: queued accounts overflown allowance: %d > %d", addr, list.Len(), config.AccountQueue)
}
queued += list.Len()
}
if queued > int(config.GlobalQueue) {
t.Fatalf("total transactions overflow allowance: %d > %d", queued, config.GlobalQueue)
}
} else {
// Local exemptions are enabled, make sure the local account owned the queue
if len(pool.queue) != 1 {
t.Errorf("multiple accounts in queue: have %v, want %v", len(pool.queue), 1)
}
// Also ensure no local transactions are ever dropped, even if above global limits
if queued := pool.queue[crypto.PubkeyToAddress(local.PublicKey)].Len(); uint64(queued) != 3*config.GlobalQueue {
t.Fatalf("local account queued transaction count mismatch: have %v, want %v", queued, 3*config.GlobalQueue)
}
}
}
// Tests that if an account remains idle for a prolonged amount of time, any
// non-executable transactions queued up are dropped to prevent wasting resources
// on shuffling them around.
//
// This logic should not hold for local transactions, unless the local tracking
// mechanism is disabled.
func TestQueueTimeLimiting(t *testing.T) {
testQueueTimeLimiting(t, false)
}
func TestQueueTimeLimitingNoLocals(t *testing.T) {
testQueueTimeLimiting(t, true)
}
func testQueueTimeLimiting(t *testing.T, nolocals bool) {
// Reduce the eviction interval to a testable amount
defer func(old time.Duration) { evictionInterval = old }(evictionInterval)
evictionInterval = time.Millisecond * 100
@ -1007,23 +956,17 @@ func testQueueTimeLimiting(t *testing.T, nolocals bool) {
config := testTxPoolConfig
config.Lifetime = time.Second
config.NoLocals = nolocals
pool := New(config, blockchain)
pool.Init(config.PriceLimit, blockchain.CurrentBlock(), makeAddressReserver())
defer pool.Close()
// Create two test accounts to ensure remotes expire but locals do not
local, _ := crypto.GenerateKey()
// Create a test account to ensure remotes expire
remote, _ := crypto.GenerateKey()
testAddBalance(pool, crypto.PubkeyToAddress(local.PublicKey), big.NewInt(1000000000))
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 {
t.Fatalf("failed to add local transaction: %v", err)
}
// Add the transaction and ensure it is queued up
if err := pool.addRemote(pricedTransaction(1, 100000, big.NewInt(1), remote)); err != nil {
t.Fatalf("failed to add remote transaction: %v", err)
}
@ -1031,7 +974,7 @@ func testQueueTimeLimiting(t *testing.T, nolocals bool) {
if pending != 0 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 0)
}
if queued != 2 {
if queued != 1 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 2)
}
if err := validatePoolInternals(pool); err != nil {
@ -1046,7 +989,7 @@ func testQueueTimeLimiting(t *testing.T, nolocals bool) {
if pending != 0 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 0)
}
if queued != 2 {
if queued != 1 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 2)
}
if err := validatePoolInternals(pool); err != nil {
@ -1060,14 +1003,8 @@ func testQueueTimeLimiting(t *testing.T, nolocals bool) {
if pending != 0 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 0)
}
if nolocals {
if queued != 0 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 0)
}
} else {
if queued != 1 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 1)
}
if queued != 0 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 0)
}
if err := validatePoolInternals(pool); err != nil {
t.Fatalf("pool internal state corrupted: %v", err)
@ -1075,7 +1012,6 @@ func testQueueTimeLimiting(t *testing.T, nolocals bool) {
// remove current transactions and increase nonce to prepare for a reset and cleanup
statedb.SetNonce(crypto.PubkeyToAddress(remote.PublicKey), 2)
statedb.SetNonce(crypto.PubkeyToAddress(local.PublicKey), 2)
<-pool.requestReset(nil, nil)
// make sure queue, pending are cleared
@ -1091,18 +1027,12 @@ func testQueueTimeLimiting(t *testing.T, nolocals bool) {
}
// Queue gapped transactions
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 {
t.Fatalf("failed to add remote transaction: %v", err)
}
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 {
t.Fatalf("failed to add remote transaction: %v", err)
}
if err := pool.addRemoteSync(pricedTransaction(2, 100000, big.NewInt(1), remote)); err != nil {
t.Fatalf("failed to add remote transaction: %v", err)
}
@ -1110,11 +1040,11 @@ func testQueueTimeLimiting(t *testing.T, nolocals bool) {
// All gapped transactions shouldn't be kicked out
pending, queued = pool.Stats()
if pending != 2 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 2)
if pending != 1 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 1)
}
if queued != 2 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 2)
if queued != 1 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 1)
}
if err := validatePoolInternals(pool); err != nil {
t.Fatalf("pool internal state corrupted: %v", err)
@ -1123,17 +1053,11 @@ func testQueueTimeLimiting(t *testing.T, nolocals bool) {
// The whole life time pass after last promotion, kick out stale transactions
time.Sleep(2 * config.Lifetime)
pending, queued = pool.Stats()
if pending != 2 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 2)
if pending != 1 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 1)
}
if nolocals {
if queued != 0 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 0)
}
} else {
if queued != 1 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 1)
}
if queued != 0 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 0)
}
if err := validatePoolInternals(pool); err != nil {
t.Fatalf("pool internal state corrupted: %v", err)
@ -1368,8 +1292,6 @@ func TestPendingMinimumAllowance(t *testing.T) {
// Tests that setting the transaction pool gas price to a higher value correctly
// discards everything cheaper than that and moves any gapped transactions back
// from the pending pool to the queue.
//
// Note, local transactions are never allowed to be dropped.
func TestRepricing(t *testing.T) {
t.Parallel()
@ -1387,7 +1309,7 @@ func TestRepricing(t *testing.T) {
defer sub.Unsubscribe()
// Create a number of test accounts and fund them
keys := make([]*ecdsa.PrivateKey, 4)
keys := make([]*ecdsa.PrivateKey, 3)
for i := 0; i < len(keys); i++ {
keys[i], _ = crypto.GenerateKey()
testAddBalance(pool, crypto.PubkeyToAddress(keys[i].PublicKey), big.NewInt(1000000))
@ -1407,20 +1329,17 @@ func TestRepricing(t *testing.T) {
txs = append(txs, pricedTransaction(2, 100000, big.NewInt(1), keys[2]))
txs = append(txs, pricedTransaction(3, 100000, big.NewInt(2), keys[2]))
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)
pending, queued := pool.Stats()
if pending != 7 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 7)
if pending != 6 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 6)
}
if queued != 3 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 3)
}
if err := validateEvents(events, 7); err != nil {
if err := validateEvents(events, 6); err != nil {
t.Fatalf("original event firing failed: %v", err)
}
if err := validatePoolInternals(pool); err != nil {
@ -1430,8 +1349,8 @@ func TestRepricing(t *testing.T) {
pool.SetGasTip(big.NewInt(2))
pending, queued = pool.Stats()
if pending != 2 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 2)
if pending != 1 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 1)
}
if queued != 5 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 5)
@ -1458,21 +1377,7 @@ func TestRepricing(t *testing.T) {
if err := validatePoolInternals(pool); err != nil {
t.Fatalf("pool internal state corrupted: %v", err)
}
// However we can add local underpriced transactions
tx := pricedTransaction(1, 100000, big.NewInt(1), keys[3])
if err := pool.addLocal(tx); err != nil {
t.Fatalf("failed to add underpriced local transaction: %v", err)
}
if pending, _ = pool.Stats(); pending != 3 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 3)
}
if err := validateEvents(events, 1); err != nil {
t.Fatalf("post-reprice local event firing failed: %v", err)
}
if err := validatePoolInternals(pool); err != nil {
t.Fatalf("pool internal state corrupted: %v", err)
}
// And we can fill gaps with properly priced transactions
// we can fill gaps with properly priced transactions
if err := pool.addRemote(pricedTransaction(1, 100000, big.NewInt(2), keys[0])); err != nil {
t.Fatalf("failed to add pending transaction: %v", err)
}
@ -1509,29 +1414,16 @@ func TestMinGasPriceEnforced(t *testing.T) {
tx := pricedTransaction(0, 100000, big.NewInt(2), key)
pool.SetGasTip(big.NewInt(tx.GasPrice().Int64() + 1))
if err := pool.addLocal(tx); !errors.Is(err, txpool.ErrUnderpriced) {
t.Fatalf("Min tip not enforced")
}
if err := pool.Add([]*types.Transaction{tx}, true, false)[0]; !errors.Is(err, txpool.ErrUnderpriced) {
if err := pool.Add([]*types.Transaction{tx}, true)[0]; !errors.Is(err, txpool.ErrUnderpriced) {
t.Fatalf("Min tip not enforced")
}
tx = dynamicFeeTx(0, 100000, big.NewInt(3), big.NewInt(2), key)
pool.SetGasTip(big.NewInt(tx.GasTipCap().Int64() + 1))
if err := pool.addLocal(tx); !errors.Is(err, txpool.ErrUnderpriced) {
if err := pool.Add([]*types.Transaction{tx}, true)[0]; !errors.Is(err, txpool.ErrUnderpriced) {
t.Fatalf("Min tip not enforced")
}
if err := pool.Add([]*types.Transaction{tx}, true, false)[0]; !errors.Is(err, txpool.ErrUnderpriced) {
t.Fatalf("Min tip not enforced")
}
// Make sure the tx is accepted if locals are enabled
pool.config.NoLocals = false
if err := pool.Add([]*types.Transaction{tx}, true, false)[0]; err != nil {
t.Fatalf("Min tip enforced with locals enabled, error: %v", err)
}
}
// Tests that setting the transaction pool gas price to a higher value correctly
@ -1572,20 +1464,17 @@ func TestRepricingDynamicFee(t *testing.T) {
txs = append(txs, dynamicFeeTx(2, 100000, big.NewInt(1), big.NewInt(1), keys[2]))
txs = append(txs, dynamicFeeTx(3, 100000, big.NewInt(2), big.NewInt(2), keys[2]))
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)
pending, queued := pool.Stats()
if pending != 7 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 7)
if pending != 6 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 6)
}
if queued != 3 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 3)
}
if err := validateEvents(events, 7); err != nil {
if err := validateEvents(events, 6); err != nil {
t.Fatalf("original event firing failed: %v", err)
}
if err := validatePoolInternals(pool); err != nil {
@ -1595,8 +1484,8 @@ func TestRepricingDynamicFee(t *testing.T) {
pool.SetGasTip(big.NewInt(2))
pending, queued = pool.Stats()
if pending != 2 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 2)
if pending != 1 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 1)
}
if queued != 5 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 5)
@ -1626,20 +1515,7 @@ func TestRepricingDynamicFee(t *testing.T) {
if err := validatePoolInternals(pool); err != nil {
t.Fatalf("pool internal state corrupted: %v", err)
}
// 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 {
t.Fatalf("failed to add underpriced local transaction: %v", err)
}
if pending, _ = pool.Stats(); pending != 3 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 3)
}
if err := validateEvents(events, 1); err != nil {
t.Fatalf("post-reprice local event firing failed: %v", err)
}
if err := validatePoolInternals(pool); err != nil {
t.Fatalf("pool internal state corrupted: %v", err)
}
// 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 {
@ -1661,77 +1537,6 @@ func TestRepricingDynamicFee(t *testing.T) {
}
}
// Tests that setting the transaction pool gas price to a higher value does not
// remove local transactions (legacy & dynamic fee).
func TestRepricingKeepsLocals(t *testing.T) {
t.Parallel()
// Create the pool to test the pricing enforcement with
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting())
blockchain := newTestBlockChain(eip1559Config, 1000000, statedb, new(event.Feed))
pool := New(testTxPoolConfig, blockchain)
pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), makeAddressReserver())
defer pool.Close()
// Create a number of test accounts and fund them
keys := make([]*ecdsa.PrivateKey, 3)
for i := 0; i < len(keys); i++ {
keys[i], _ = crypto.GenerateKey()
testAddBalance(pool, crypto.PubkeyToAddress(keys[i].PublicKey), big.NewInt(100000*1000000))
}
// Create transaction (both pending and queued) with a linearly growing gasprice
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 {
t.Fatal(err)
}
// Add queued transaction.
queuedTx := pricedTransaction(i+501, 100000, big.NewInt(int64(i)), keys[2])
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 {
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 {
t.Fatal(err)
}
}
pending, queued := pool.Stats()
expPending, expQueued := 1000, 1000
validate := func() {
pending, queued = pool.Stats()
if pending != expPending {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, expPending)
}
if queued != expQueued {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, expQueued)
}
if err := validatePoolInternals(pool); err != nil {
t.Fatalf("pool internal state corrupted: %v", err)
}
}
validate()
// Reprice the pool and check that nothing is dropped
pool.SetGasTip(big.NewInt(2))
validate()
pool.SetGasTip(big.NewInt(2))
pool.SetGasTip(big.NewInt(4))
pool.SetGasTip(big.NewInt(8))
pool.SetGasTip(big.NewInt(100))
validate()
}
// Tests that when the pool reaches its global transaction limit, underpriced
// transactions are gradually shifted out for more expensive ones and any gapped
// pending transactions are moved into the queue.
@ -1761,21 +1566,18 @@ func TestUnderpricing(t *testing.T) {
keys := make([]*ecdsa.PrivateKey, 5)
for i := 0; i < len(keys); i++ {
keys[i], _ = crypto.GenerateKey()
testAddBalance(pool, crypto.PubkeyToAddress(keys[i].PublicKey), big.NewInt(1000000))
testAddBalance(pool, crypto.PubkeyToAddress(keys[i].PublicKey), big.NewInt(10000000))
}
// Generate and queue a batch of transactions, both pending and queued
txs := types.Transactions{}
txs = append(txs, pricedTransaction(0, 100000, big.NewInt(1), keys[0]))
txs = append(txs, pricedTransaction(1, 100000, big.NewInt(2), keys[0]))
txs = append(txs, pricedTransaction(1, 100000, big.NewInt(1), keys[1]))
ltx := pricedTransaction(0, 100000, big.NewInt(1), keys[2])
txs = append(txs, pricedTransaction(0, 100000, big.NewInt(1), keys[0])) // pending
txs = append(txs, pricedTransaction(1, 100000, big.NewInt(2), keys[0])) // pending
txs = append(txs, pricedTransaction(0, 100000, big.NewInt(1), keys[2])) // pending
txs = append(txs, pricedTransaction(1, 100000, big.NewInt(1), keys[1])) // queued
// Import the batch and that both pending and queued transactions match up
pool.addRemotes(txs)
pool.addLocal(ltx)
pool.addRemotesSync(txs)
pending, queued := pool.Stats()
if pending != 3 {
@ -1805,48 +1607,26 @@ func TestUnderpricing(t *testing.T) {
if err := pool.addRemoteSync(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.addRemoteSync(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 != txpool.ErrFutureReplacePending {
if err := pool.addRemoteSync(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 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 2)
if pending != 4 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 4)
}
if queued != 2 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 2)
if queued != 0 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 0)
}
if err := validateEvents(events, 2); err != nil {
if err := validateEvents(events, 4); err != nil {
t.Fatalf("additional event firing failed: %v", err)
}
if err := validatePoolInternals(pool); err != nil {
t.Fatalf("pool internal state corrupted: %v", err)
}
// 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 {
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 {
t.Fatalf("failed to add new underpriced local transaction: %v", err)
}
pending, queued = pool.Stats()
if pending != 3 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 3)
}
if queued != 1 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 1)
}
if err := validateEvents(events, 2); err != nil {
t.Fatalf("local event firing failed: %v", err)
}
if err := validatePoolInternals(pool); err != nil {
t.Fatalf("pool internal state corrupted: %v", err)
}
}
// Tests that more expensive transactions push out cheap ones from the pool, but
@ -1920,8 +1700,6 @@ func TestStableUnderpricing(t *testing.T) {
// Tests that when the pool reaches its global transaction limit, underpriced
// transactions (legacy & dynamic fee) are gradually shifted out for more
// expensive ones and any gapped pending transactions are moved into the queue.
//
// Note, local transactions are never allowed to be dropped.
func TestUnderpricingDynamicFee(t *testing.T) {
t.Parallel()
@ -1946,15 +1724,13 @@ func TestUnderpricingDynamicFee(t *testing.T) {
// Generate and queue a batch of transactions, both pending and queued
txs := types.Transactions{}
txs = append(txs, dynamicFeeTx(0, 100000, big.NewInt(3), big.NewInt(2), keys[0]))
txs = append(txs, pricedTransaction(1, 100000, big.NewInt(2), keys[0]))
txs = append(txs, dynamicFeeTx(1, 100000, big.NewInt(2), big.NewInt(1), keys[1]))
txs = append(txs, dynamicFeeTx(0, 100000, big.NewInt(3), big.NewInt(2), keys[0])) // pending
txs = append(txs, pricedTransaction(1, 100000, big.NewInt(2), keys[0])) // pending
txs = append(txs, dynamicFeeTx(1, 100000, big.NewInt(2), big.NewInt(1), keys[1])) // queued
txs = append(txs, dynamicFeeTx(0, 100000, big.NewInt(2), big.NewInt(1), keys[2])) // pending
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
// Import the batch and check that both pending and queued transactions match up
pool.addRemotesSync(txs) // Pend K0:0, K0:1; Que K1:1
pending, queued := pool.Stats()
if pending != 3 {
@ -1972,13 +1748,13 @@ 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, txpool.ErrUnderpriced) { // Pend K0:0, K0:1, K2:0; Que K1:1
if err := pool.addRemoteSync(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.addRemoteSync(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)
}
@ -1991,40 +1767,18 @@ func TestUnderpricingDynamicFee(t *testing.T) {
t.Fatalf("failed to add well priced transaction: %v", err)
}
pending, queued = pool.Stats()
if pending != 2 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 2)
if pending != 4 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 4)
}
if queued != 2 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 2)
if queued != 0 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 0)
}
if err := validateEvents(events, 2); err != nil {
if err := validateEvents(events, 3); err != nil {
t.Fatalf("additional event firing failed: %v", err)
}
if err := validatePoolInternals(pool); err != nil {
t.Fatalf("pool internal state corrupted: %v", err)
}
// 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 {
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 {
t.Fatalf("failed to add new underpriced local transaction: %v", err)
}
pending, queued = pool.Stats()
if pending != 3 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 3)
}
if queued != 1 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 1)
}
if err := validateEvents(events, 2); err != nil {
t.Fatalf("local event firing failed: %v", err)
}
if err := validatePoolInternals(pool); err != nil {
t.Fatalf("pool internal state corrupted: %v", err)
}
}
// Tests whether highest fee cap transaction is retained after a batch of high effective
@ -2044,7 +1798,7 @@ func TestDualHeapEviction(t *testing.T) {
)
check := func(tx *types.Transaction, name string) {
if pool.all.GetRemote(tx.Hash()) == nil {
if pool.all.Get(tx.Hash()) == nil {
t.Fatalf("highest %s transaction evicted from the pool", name)
}
}
@ -2341,122 +2095,6 @@ func TestReplacementDynamicFee(t *testing.T) {
}
}
// Tests that local transactions are journaled to disk, but remote transactions
// get discarded between restarts.
func TestJournaling(t *testing.T) { testJournaling(t, false) }
func TestJournalingNoLocals(t *testing.T) { testJournaling(t, true) }
func testJournaling(t *testing.T, nolocals bool) {
t.Parallel()
// Create a temporary file for the journal
file, err := os.CreateTemp("", "")
if err != nil {
t.Fatalf("failed to create temporary journal: %v", err)
}
journal := file.Name()
defer os.Remove(journal)
// Clean up the temporary file, we only need the path for now
file.Close()
os.Remove(journal)
// Create the original pool to inject transaction into the journal
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting())
blockchain := newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed))
config := testTxPoolConfig
config.NoLocals = nolocals
config.Journal = journal
config.Rejournal = time.Second
pool := New(config, blockchain)
pool.Init(config.PriceLimit, blockchain.CurrentBlock(), makeAddressReserver())
// Create two test accounts to ensure remotes expire but locals do not
local, _ := crypto.GenerateKey()
remote, _ := crypto.GenerateKey()
testAddBalance(pool, crypto.PubkeyToAddress(local.PublicKey), big.NewInt(1000000000))
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 {
t.Fatalf("failed to add local transaction: %v", err)
}
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 {
t.Fatalf("failed to add local transaction: %v", err)
}
if err := pool.addRemoteSync(pricedTransaction(0, 100000, big.NewInt(1), remote)); err != nil {
t.Fatalf("failed to add remote transaction: %v", err)
}
pending, queued := pool.Stats()
if pending != 4 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 4)
}
if queued != 0 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 0)
}
if err := validatePoolInternals(pool); err != nil {
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.Close()
statedb.SetNonce(crypto.PubkeyToAddress(local.PublicKey), 1)
blockchain = newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed))
pool = New(config, blockchain)
pool.Init(config.PriceLimit, blockchain.CurrentBlock(), makeAddressReserver())
pending, queued = pool.Stats()
if queued != 0 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 0)
}
if nolocals {
if pending != 0 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 0)
}
} else {
if pending != 2 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 2)
}
}
if err := validatePoolInternals(pool); err != nil {
t.Fatalf("pool internal state corrupted: %v", err)
}
// Bump the nonce temporarily and ensure the newly invalidated transaction is removed
statedb.SetNonce(crypto.PubkeyToAddress(local.PublicKey), 2)
<-pool.requestReset(nil, nil)
time.Sleep(2 * config.Rejournal)
pool.Close()
statedb.SetNonce(crypto.PubkeyToAddress(local.PublicKey), 1)
blockchain = newTestBlockChain(params.TestChainConfig, 1000000, statedb, new(event.Feed))
pool = New(config, blockchain)
pool.Init(config.PriceLimit, blockchain.CurrentBlock(), makeAddressReserver())
pending, queued = pool.Stats()
if pending != 0 {
t.Fatalf("pending transactions mismatched: have %d, want %d", pending, 0)
}
if nolocals {
if queued != 0 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 0)
}
} else {
if queued != 1 {
t.Fatalf("queued transactions mismatched: have %d, want %d", queued, 1)
}
}
if err := validatePoolInternals(pool); err != nil {
t.Fatalf("pool internal state corrupted: %v", err)
}
pool.Close()
}
// TestStatusCheck tests that the pool can correctly retrieve the
// pending status of individual transactions.
func TestStatusCheck(t *testing.T) {
@ -2571,7 +2209,7 @@ func benchmarkFuturePromotion(b *testing.B, size int) {
for i := 0; i < size; i++ {
tx := transaction(uint64(1+i), 100000, key)
pool.enqueueTx(tx.Hash(), tx, false, true)
pool.enqueueTx(tx.Hash(), tx, true)
}
// Benchmark the speed of pool validation
b.ResetTimer()
@ -2581,15 +2219,11 @@ func benchmarkFuturePromotion(b *testing.B, size int) {
}
// Benchmarks the speed of batched transaction insertion.
func BenchmarkBatchInsert100(b *testing.B) { benchmarkBatchInsert(b, 100, false) }
func BenchmarkBatchInsert1000(b *testing.B) { benchmarkBatchInsert(b, 1000, false) }
func BenchmarkBatchInsert10000(b *testing.B) { benchmarkBatchInsert(b, 10000, false) }
func BenchmarkBatchInsert100(b *testing.B) { benchmarkBatchInsert(b, 100) }
func BenchmarkBatchInsert1000(b *testing.B) { benchmarkBatchInsert(b, 1000) }
func BenchmarkBatchInsert10000(b *testing.B) { benchmarkBatchInsert(b, 10000) }
func BenchmarkBatchLocalInsert100(b *testing.B) { benchmarkBatchInsert(b, 100, true) }
func BenchmarkBatchLocalInsert1000(b *testing.B) { benchmarkBatchInsert(b, 1000, true) }
func BenchmarkBatchLocalInsert10000(b *testing.B) { benchmarkBatchInsert(b, 10000, true) }
func benchmarkBatchInsert(b *testing.B, size int, local bool) {
func benchmarkBatchInsert(b *testing.B, size int) {
// Generate a batch of transactions to enqueue into the pool
pool, key := setupPool()
defer pool.Close()
@ -2607,11 +2241,7 @@ func benchmarkBatchInsert(b *testing.B, size int, local bool) {
// Benchmark importing the transactions into the queue
b.ResetTimer()
for _, batch := range batches {
if local {
pool.addLocals(batch)
} else {
pool.addRemotes(batch)
}
pool.addRemotes(batch)
}
}
@ -2623,10 +2253,6 @@ func BenchmarkInsertRemoteWithAllLocals(b *testing.B) {
remoteKey, _ := crypto.GenerateKey()
remoteAddr := crypto.PubkeyToAddress(remoteKey.PublicKey)
locals := make([]*types.Transaction, 4096+1024) // Occupy all slots
for i := 0; i < len(locals); i++ {
locals[i] = transaction(uint64(i), 100000, key)
}
remotes := make([]*types.Transaction, 1000)
for i := 0; i < len(remotes); i++ {
remotes[i] = pricedTransaction(uint64(i), 100000, big.NewInt(2), remoteKey) // Higher gasprice
@ -2637,9 +2263,6 @@ func BenchmarkInsertRemoteWithAllLocals(b *testing.B) {
b.StopTimer()
pool, _ := setupPool()
testAddBalance(pool, account, big.NewInt(100000000))
for _, local := range locals {
pool.addLocal(local)
}
b.StartTimer()
// Assign a high enough balance for testing
testAddBalance(pool, remoteAddr, big.NewInt(100000000))

@ -590,7 +590,7 @@ func (l *pricedList) underpricedFor(h *priceHeap, tx *types.Transaction) bool {
// Discard stale price points if found at the heap start
for len(h.list) > 0 {
head := h.list[0]
if l.all.GetRemote(head.Hash()) == nil { // Removed or migrated
if l.all.Get(head.Hash()) == nil { // Removed or migrated
l.stales.Add(-1)
heap.Pop(h)
continue
@ -615,7 +615,7 @@ func (l *pricedList) Discard(slots int) (types.Transactions, bool) {
if len(l.urgent.list)*floatingRatio > len(l.floating.list)*urgentRatio {
// Discard stale transactions if found during cleanup
tx := heap.Pop(&l.urgent).(*types.Transaction)
if l.all.GetRemote(tx.Hash()) == nil { // Removed or migrated
if l.all.Get(tx.Hash()) == nil { // Removed or migrated
l.stales.Add(-1)
continue
}
@ -628,7 +628,7 @@ func (l *pricedList) Discard(slots int) (types.Transactions, bool) {
}
// Discard stale transactions if found during cleanup
tx := heap.Pop(&l.floating).(*types.Transaction)
if l.all.GetRemote(tx.Hash()) == nil { // Removed or migrated
if l.all.Get(tx.Hash()) == nil { // Removed or migrated
l.stales.Add(-1)
continue
}
@ -653,7 +653,7 @@ func (l *pricedList) Reheap() {
defer l.reheapMu.Unlock()
start := time.Now()
l.stales.Store(0)
l.urgent.list = make([]*types.Transaction, 0, l.all.RemoteCount())
l.urgent.list = make([]*types.Transaction, 0, l.all.Count())
l.all.Range(func(hash common.Hash, tx *types.Transaction) bool {
l.urgent.list = append(l.urgent.list, tx)
return true

Loading…
Cancel
Save