From 0ef327bbee79c01a69ba59258acc6ce3a48bc288 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?P=C3=A9ter=20Szil=C3=A1gyi?= Date: Fri, 1 Jul 2016 18:59:55 +0300 Subject: core, eth, internal, miner: optimize txpool for quick ops --- core/tx_list.go | 331 +++++++++++++++++++++++++++ core/tx_list_test.go | 58 +++++ core/tx_pool.go | 503 ++++++++++++++++++----------------------- core/tx_pool_test.go | 226 ++++++++++-------- core/types/transaction.go | 27 +-- core/types/transaction_test.go | 7 +- 6 files changed, 751 insertions(+), 401 deletions(-) create mode 100644 core/tx_list.go create mode 100644 core/tx_list_test.go (limited to 'core') diff --git a/core/tx_list.go b/core/tx_list.go new file mode 100644 index 000000000..e30fee38f --- /dev/null +++ b/core/tx_list.go @@ -0,0 +1,331 @@ +// Copyright 2016 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 core + +import ( + "container/heap" + "math" + "math/big" + "sort" + + "github.com/ethereum/go-ethereum/core/types" +) + +// nonceHeap is a heap.Interface implementation over 64bit unsigned integers for +// retrieving sorted transactions from the possibly gapped future queue. +type nonceHeap []uint64 + +func (h nonceHeap) Len() int { return len(h) } +func (h nonceHeap) Less(i, j int) bool { return h[i] < h[j] } +func (h nonceHeap) Swap(i, j int) { h[i], h[j] = h[j], h[i] } + +func (h *nonceHeap) Push(x interface{}) { + *h = append(*h, x.(uint64)) +} + +func (h *nonceHeap) Pop() interface{} { + old := *h + n := len(old) + x := old[n-1] + *h = old[0 : n-1] + return x +} + +// txList is a "list" of transactions belonging to an account, sorted by account +// nonce. The same type can be used both for storing contiguous transactions for +// the executable/pending queue; and for storing gapped transactions for the non- +// executable/future queue, with minor behavoiral changes. +type txList struct { + strict bool // Whether nonces are strictly continuous or not + items map[uint64]*types.Transaction // Hash map storing the transaction data + cache types.Transactions // cache of the transactions already sorted + + first uint64 // Nonce of the lowest stored transaction (strict mode) + last uint64 // Nonce of the highest stored transaction (strict mode) + index *nonceHeap // Heap of nonces of all teh stored transactions (non-strict mode) + + costcap *big.Int // Price of the highest costing transaction (reset only if exceeds balance) +} + +// newTxList create a new transaction list for maintaining nonce-indexable fast, +// gapped, sortable transaction lists. +func newTxList(strict bool) *txList { + return &txList{ + strict: strict, + items: make(map[uint64]*types.Transaction), + first: math.MaxUint64, + index: &nonceHeap{}, + costcap: new(big.Int), + } +} + +// Add tries to inserts a new transaction into the list, returning whether the +// transaction was acceped, and if yes, any previous transaction it replaced. +// +// In case of strict lists (contiguous nonces) the nonce boundaries are updated +// appropriately with the new transaction. Otherwise (gapped nonces) the heap of +// nonces is expanded with the new transaction. +func (l *txList) Add(tx *types.Transaction) (bool, *types.Transaction) { + // If an existing transaction is better, discard new one + nonce := tx.Nonce() + + old, ok := l.items[nonce] + if ok && old.GasPrice().Cmp(tx.GasPrice()) >= 0 { + return false, nil + } + // Otherwise insert the transaction and replace any previous one + l.items[nonce] = tx + if cost := tx.Cost(); l.costcap.Cmp(cost) < 0 { + l.costcap = cost + } + if l.strict { + // In strict mode, maintain the nonce sequence boundaries + if nonce < l.first { + l.first = nonce + } + if nonce > l.last { + l.last = nonce + } + } else { + // In gapped mode, maintain the nonce heap + heap.Push(l.index, nonce) + } + l.cache = nil + + return true, old +} + +// Forward removes all transactions from the list with a nonce lower than the +// provided threshold. Every removed transaction is returned for any post-removal +// maintenance. +func (l *txList) Forward(threshold uint64) types.Transactions { + var removed types.Transactions + + if l.strict { + // In strict mode, push the lowest nonce forward to the threshold + for l.first < threshold { + if tx, ok := l.items[l.first]; ok { + removed = append(removed, tx) + } + delete(l.items, l.first) + l.first++ + } + if l.first > l.last { + l.last = l.first + } + } else { + // In gapped mode, pop off heap items until the threshold is reached + for l.index.Len() > 0 && (*l.index)[0] < threshold { + nonce := heap.Pop(l.index).(uint64) + removed = append(removed, l.items[nonce]) + delete(l.items, nonce) + } + } + l.cache = nil + + return removed +} + +// Filter removes all transactions from the list with a cost higher than the +// provided threshold. Every removed transaction is returned for any post-removal +// maintenance. Strict-mode invalidated transactions are also returned. +// +// This method uses the cached costcap to quickly decide if there's even a point +// in calculating all the costs or if the balance covers all. If the threshold is +// loewr than the costcap, the costcap will be reset to a new high after removing +// expensive the too transactions. +func (l *txList) Filter(threshold *big.Int) (types.Transactions, types.Transactions) { + // If all transactions are blow the threshold, short circuit + if l.costcap.Cmp(threshold) <= 0 { + return nil, nil + } + l.costcap = new(big.Int).Set(threshold) // Lower the cap to the threshold + + // Gather all the transactions needing deletion + var removed types.Transactions + for _, tx := range l.items { + if cost := tx.Cost(); cost.Cmp(threshold) > 0 { + removed = append(removed, tx) + delete(l.items, tx.Nonce()) + } + } + // Readjust the nonce boundaries/indexes and gather invalidate tranactions + var invalids types.Transactions + if l.strict { + // In strict mode iterate find the first gap and invalidate everything after it + for i := l.first; i <= l.last; i++ { + if _, ok := l.items[i]; !ok { + // Gap found, invalidate all subsequent transactions + for j := i + 1; j <= l.last; j++ { + if tx, ok := l.items[j]; ok { + invalids = append(invalids, tx) + delete(l.items, j) + } + } + // Reduce the highest transaction nonce and return + l.last = i - 1 + break + } + } + } else { + // In gapped mode no transactions are invalid, but the heap is ruined + l.index = &nonceHeap{} + for nonce, _ := range l.items { + *l.index = append(*l.index, nonce) + } + heap.Init(l.index) + } + l.cache = nil + + return removed, invalids +} + +// Cap places a hard limit on the number of items, returning all transactions +// exceeding tht limit. +func (l *txList) Cap(threshold int) types.Transactions { + // Short circuit if the number of items is under the limit + if len(l.items) < threshold { + return nil + } + // Otherwise gather and drop the highest nonce'd transactions + var drops types.Transactions + + if l.strict { + // In strict mode, just gather top down from last to first + for len(l.items) > threshold { + if tx, ok := l.items[l.last]; ok { + drops = append(drops, tx) + delete(l.items, l.last) + l.last-- + } + } + } else { + // In gapped mode it's expensive: we need to sort and drop like that + sort.Sort(*l.index) + for size := len(l.items); size > threshold; size-- { + drops = append(drops, l.items[(*l.index)[size-1]]) + delete(l.items, (*l.index)[size-1]) + *l.index = (*l.index)[:size-1] + } + heap.Init(l.index) + } + l.cache = nil + + return drops +} + +// Remove deletes a transaction from the maintained list, returning whether the +// transaction was found, and also returning any transaction invalidated due to +// the deletion (strict mode only). +func (l *txList) Remove(tx *types.Transaction) (bool, types.Transactions) { + nonce := tx.Nonce() + if _, ok := l.items[nonce]; ok { + // Remove the item and invalidate the sorted cache + delete(l.items, nonce) + l.cache = nil + + // Remove all invalidated transactions (strict mode only!) + invalids := make(types.Transactions, 0, l.last-nonce) + if l.strict { + for i := nonce + 1; i <= l.last; i++ { + invalids = append(invalids, l.items[i]) + delete(l.items, i) + } + l.last = nonce - 1 + } else { + // In gapped mode, remove the nonce from the index but honour the heap + for i := 0; i < l.index.Len(); i++ { + if (*l.index)[i] == nonce { + heap.Remove(l.index, i) + break + } + } + } + // Figure out the new highest nonce + return true, invalids + } + return false, nil +} + +// Ready retrieves a sequentially increasing list of transactions starting at the +// provided nonce that is ready for processing. The returned transactions will be +// removed from the list. +// +// Note, all transactions with nonces lower that start will also be returned to +// prevent getting into and invalid state. This is not something that should ever +// happen but better to be self correcting than failing! +func (l *txList) Ready(start uint64) types.Transactions { + var txs types.Transactions + if l.strict { + // In strict mode make sure we have valid transaction, return all contiguous + if l.first > start { + return nil + } + for { + if tx, ok := l.items[l.first]; ok { + txs = append(txs, tx) + delete(l.items, l.first) + l.first++ + continue + } + break + } + } else { + // In gapped mode, check the heap start and return all contiguous + if l.index.Len() == 0 || (*l.index)[0] > start { + return nil + } + next := (*l.index)[0] + for l.index.Len() > 0 && (*l.index)[0] == next { + txs = append(txs, l.items[next]) + delete(l.items, next) + heap.Pop(l.index) + next++ + } + } + l.cache = nil + + return txs +} + +// Len returns the length of the transaction list. +func (l *txList) Len() int { + return len(l.items) +} + +// Empty returns whether the list of transactions is empty or not. +func (l *txList) Empty() bool { + return len(l.items) == 0 +} + +// Flatten creates a nonce-sorted slice of transactions based on the loosely +// sorted internal representation. The result of the sorting is cached in case +// it's requested again before any modifications are made to the contents. +func (l *txList) Flatten() types.Transactions { + // If the sorting was not cached yet, create and cache it + if l.cache == nil { + l.cache = make(types.Transactions, 0, len(l.items)) + for _, tx := range l.items { + l.cache = append(l.cache, tx) + } + sort.Sort(types.TxByNonce(l.cache)) + } + // Copy the cache to prevent accidental modifications + txs := make(types.Transactions, len(l.cache)) + copy(txs, l.cache) + return txs +} diff --git a/core/tx_list_test.go b/core/tx_list_test.go new file mode 100644 index 000000000..ea83ca479 --- /dev/null +++ b/core/tx_list_test.go @@ -0,0 +1,58 @@ +// Copyright 2016 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 core + +import ( + "math/big" + "math/rand" + "testing" + + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/crypto" +) + +// Tests that transactions can be added to strict lists and list contents and +// nonce boundaries are correctly maintained. +func TestStrictTxListAdd(t *testing.T) { + // Generate a list of transactions to insert + key, _ := crypto.GenerateKey() + + txs := make(types.Transactions, 1024) + for i := 0; i < len(txs); i++ { + txs[i] = transaction(uint64(i), new(big.Int), key) + } + // Insert the transactions in a random order + list := newTxList(true) + for _, v := range rand.Perm(len(txs)) { + list.Add(txs[v]) + } + // Verify internal state + if list.first != 0 { + t.Errorf("lowest nonce mismatch: have %d, want %d", list.first, 0) + } + if int(list.last) != len(txs)-1 { + t.Errorf("highest nonce mismatch: have %d, want %d", list.last, len(txs)-1) + } + if len(list.items) != len(txs) { + t.Errorf("transaction count mismatch: have %d, want %d", len(list.items), len(txs)) + } + for i, tx := range txs { + if list.items[tx.Nonce()] != tx { + t.Errorf("item %d: transaction mismatch: have %v, want %v", i, list.items[tx.Nonce()], tx) + } + } +} diff --git a/core/tx_pool.go b/core/tx_pool.go index ec3d5c16b..c4dcceba0 100644 --- a/core/tx_pool.go +++ b/core/tx_pool.go @@ -20,7 +20,6 @@ import ( "errors" "fmt" "math/big" - "sort" "sync" "time" @@ -51,11 +50,6 @@ const ( type stateFn func() (*state.StateDB, error) -// TxList is a "list" of transactions belonging to an account, sorted by account -// nonce. To allow gaps and avoid constant copying, the list is represented as a -// hash map. -type TxList map[uint64]*types.Transaction - // 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. @@ -74,8 +68,8 @@ type TxPool struct { localTx *txSet mu sync.RWMutex - pending map[common.Address]TxList // All currently processable transactions - queue map[common.Address]TxList // Queued but non-processable transactions + pending map[common.Address]*txList // All currently processable transactions + queue map[common.Address]*txList // Queued but non-processable transactions all map[common.Hash]*types.Transaction // All transactions to allow lookups wg sync.WaitGroup // for shutdown sync @@ -86,8 +80,8 @@ type TxPool struct { func NewTxPool(config *ChainConfig, eventMux *event.TypeMux, currentStateFn stateFn, gasLimitFn func() *big.Int) *TxPool { pool := &TxPool{ config: config, - pending: make(map[common.Address]TxList), - queue: make(map[common.Address]TxList), + pending: make(map[common.Address]*txList), + queue: make(map[common.Address]*txList), all: make(map[common.Hash]*types.Transaction), eventMux: eventMux, currentState: currentStateFn, @@ -125,7 +119,7 @@ func (pool *TxPool) eventLoop() { pool.minGasPrice = ev.Price pool.mu.Unlock() case RemovedTransactionEvent: - pool.AddTransactions(ev.Txs) + pool.AddBatch(ev.Txs) } } } @@ -133,12 +127,12 @@ func (pool *TxPool) eventLoop() { func (pool *TxPool) resetState() { currentState, err := pool.currentState() if err != nil { - glog.V(logger.Info).Infoln("failed to get current state: %v", err) + glog.V(logger.Error).Infof("Failed to get current state: %v", err) return } managedState := state.ManageState(currentState) if err != nil { - glog.V(logger.Info).Infoln("failed to get managed state: %v", err) + glog.V(logger.Error).Infof("Failed to get managed state: %v", err) return } pool.pendingState = managedState @@ -147,22 +141,15 @@ func (pool *TxPool) resetState() { // any transactions that have been included in the block or // have been invalidated because of another transaction (e.g. // higher gas price) - pool.validatePool() - - // Loop over the pending transactions and base the nonce of the new - // pending transaction set. - for addr, txs := range pool.pending { - // Set the nonce. Transaction nonce can never be lower - // than the state nonce; validatePool took care of that. - for nonce, _ := range txs { - if pool.pendingState.GetNonce(addr) <= nonce { - pool.pendingState.SetNonce(addr, nonce+1) - } - } + pool.demoteUnexecutables() + + // Update all accounts to the latest known pending nonce + for addr, list := range pool.pending { + pool.pendingState.SetNonce(addr, list.last+1) } // Check the queue and move transactions over to the pending if possible // or remove those that have become invalid - pool.checkQueue() + pool.promoteExecutables() } func (pool *TxPool) Stop() { @@ -178,46 +165,58 @@ func (pool *TxPool) State() *state.ManagedState { return pool.pendingState } +// Stats retrieves the current pool stats, namely the number of pending and the +// number of queued (non-executable) transactions. func (pool *TxPool) Stats() (pending int, queued int) { pool.mu.RLock() defer pool.mu.RUnlock() - for _, txs := range pool.pending { - pending += len(txs) + for _, list := range pool.pending { + pending += list.Len() } - for _, txs := range pool.queue { - queued += len(txs) + for _, list := range pool.queue { + queued += list.Len() } return } // 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]TxList, map[common.Address]TxList) { +// 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.RLock() defer pool.mu.RUnlock() - // Retrieve all the pending transactions and sort by account and by nonce - pending := make(map[common.Address]TxList) - for addr, txs := range pool.pending { - copy := make(TxList) - for nonce, tx := range txs { - copy[nonce] = tx - } - pending[addr] = copy - } - // Retrieve all the queued transactions and sort by account and by nonce - queued := make(map[common.Address]TxList) - for addr, txs := range pool.queue { - copy := make(TxList) - for nonce, tx := range txs { - copy[nonce] = tx - } - queued[addr] = copy + pending := make(map[common.Address]types.Transactions) + for addr, list := range pool.pending { + pending[addr] = list.Flatten() + } + queued := make(map[common.Address]types.Transactions) + for addr, list := range pool.queue { + queued[addr] = list.Flatten() } return pending, queued } +// Pending retrieves all currently processable transactions, groupped 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) Pending() map[common.Address]types.Transactions { + pool.mu.Lock() + defer pool.mu.Unlock() + + // check queue first + pool.promoteExecutables() + + // invalidate any txs + pool.demoteUnexecutables() + + pending := make(map[common.Address]types.Transactions) + for addr, list := range pool.pending { + pending[addr] = list.Flatten() + } + return pending +} + // SetLocal marks a transaction as local, skipping gas price // check against local miner minimum in the future func (pool *TxPool) SetLocal(tx *types.Transaction) { @@ -283,340 +282,268 @@ func (pool *TxPool) validateTx(tx *types.Transaction) error { return nil } -// validate and queue transactions. -func (self *TxPool) add(tx *types.Transaction) error { +// add validates a transaction and inserts it into the non-executable queue for +// later pending promotion and execution. +func (pool *TxPool) add(tx *types.Transaction) error { + // If the transaction is alreayd known, discard it hash := tx.Hash() - - if self.all[hash] != nil { - return fmt.Errorf("Known transaction (%x)", hash[:4]) + if pool.all[hash] != nil { + return fmt.Errorf("Known transaction: %x", hash[:4]) } - err := self.validateTx(tx) - if err != nil { + // Otherwise ensure basic validation passes nd queue it up + if err := pool.validateTx(tx); err != nil { return err } - self.queueTx(hash, tx) + pool.enqueueTx(hash, tx) + // Print a log message if low enough level is set if glog.V(logger.Debug) { - var toname string + rcpt := "[NEW_CONTRACT]" if to := tx.To(); to != nil { - toname = common.Bytes2Hex(to[:4]) - } else { - toname = "[NEW_CONTRACT]" + rcpt = common.Bytes2Hex(to[:4]) } - // we can ignore the error here because From is - // verified in ValidateTransaction. - f, _ := tx.From() - from := common.Bytes2Hex(f[:4]) - glog.Infof("(t) %x => %s (%v) %x\n", from, toname, tx.Value, hash) + from, _ := tx.From() // from already verified during tx validation + glog.Infof("(t) 0x%x => %s (%v) %x\n", from[:4], rcpt, tx.Value, hash) } - return nil } -// queueTx will queue an unknown transaction. -func (self *TxPool) queueTx(hash common.Hash, tx *types.Transaction) { - addr, _ := tx.From() // already validated - if self.queue[addr] == nil { - self.queue[addr] = make(TxList) +// enqueueTx inserts a new transction 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) { + // Try to insert the transaction into the future queue + from, _ := tx.From() // already validated + if pool.queue[from] == nil { + pool.queue[from] = newTxList(false) } - // If the nonce is already used, discard the lower priced transaction - nonce := tx.Nonce() - - if old, ok := self.queue[addr][nonce]; ok { - if old.GasPrice().Cmp(tx.GasPrice()) >= 0 { - return // Old was better, discard this - } - delete(self.all, old.Hash()) // New is better, drop and overwrite old one + inserted, old := pool.queue[from].Add(tx) + if !inserted { + return // An older transaction was better, discard this + } + // Discard any previous transaction and mark this + if old != nil { + delete(pool.all, old.Hash()) } - self.queue[addr][nonce] = tx - self.all[hash] = tx + pool.all[hash] = tx } -// addTx will moves a transaction from the non-executable queue to the pending -// (processable) list of transactions. -func (pool *TxPool) addTx(addr common.Address, tx *types.Transaction) { +// promoteTx adds a transaction to the pending (processable) list of transactions. +// +// Note, this method assumes the pool lock is held! +func (pool *TxPool) promoteTx(addr common.Address, hash common.Hash, tx *types.Transaction) { // Init delayed since tx pool could have been started before any state sync if pool.pendingState == nil { pool.resetState() } - // If the nonce is already used, discard the lower priced transaction - hash, nonce := tx.Hash(), tx.Nonce() - - if old, ok := pool.pending[addr][nonce]; ok { - oldHash := old.Hash() + // Try to insert the transaction into the pending queue + if pool.pending[addr] == nil { + pool.pending[addr] = newTxList(true) + } + list := pool.pending[addr] - switch { - case oldHash == hash: // Nothing changed, noop - return - case old.GasPrice().Cmp(tx.GasPrice()) >= 0: // Old was better, discard this - delete(pool.all, hash) - return - default: // New is better, discard old - delete(pool.all, oldHash) - } + inserted, old := list.Add(tx) + if !inserted { + // An older transaction was better, discard this + delete(pool.all, hash) + return } - // The transaction is being kept, insert it into the tx pool - if _, ok := pool.pending[addr]; !ok { - pool.pending[addr] = make(TxList) + // Otherwise discard any previous transaction and mark this + if old != nil { + delete(pool.all, old.Hash()) } - pool.pending[addr][nonce] = tx - pool.all[hash] = tx - - // Increment the nonce on the pending state. This can only happen if - // the nonce is +1 to the previous one. - pool.pendingState.SetNonce(addr, nonce+1) + pool.all[hash] = tx // Failsafe to work around direct pending inserts (tests) - // Notify the subscribers. This event is posted in a goroutine - // because it's possible that somewhere during the post "Remove transaction" - // gets called which will then wait for the global tx pool lock and deadlock. + // Set the potentially new pending nonce and notify any subsystems of the new tx + pool.pendingState.SetNonce(addr, list.last+1) go pool.eventMux.Post(TxPreEvent{tx}) } // Add queues a single transaction in the pool if it is valid. -func (self *TxPool) Add(tx *types.Transaction) error { - self.mu.Lock() - defer self.mu.Unlock() +func (pool *TxPool) Add(tx *types.Transaction) error { + pool.mu.Lock() + defer pool.mu.Unlock() - if err := self.add(tx); err != nil { + if err := pool.add(tx); err != nil { return err } - self.checkQueue() + pool.promoteExecutables() + return nil } -// AddTransactions attempts to queue all valid transactions in txs. -func (self *TxPool) AddTransactions(txs []*types.Transaction) { - self.mu.Lock() - defer self.mu.Unlock() +// AddBatch attempts to queue a batch of transactions. +func (pool *TxPool) AddBatch(txs []*types.Transaction) { + pool.mu.Lock() + defer pool.mu.Unlock() for _, tx := range txs { - if err := self.add(tx); err != nil { + if err := pool.add(tx); err != nil { glog.V(logger.Debug).Infoln("tx error:", err) - } else { - h := tx.Hash() - glog.V(logger.Debug).Infof("tx %x\n", h[:4]) } } - - // check and validate the queue - self.checkQueue() + pool.promoteExecutables() } -// GetTransaction returns a transaction if it is contained in the pool +// Get returns a transaction if it is contained in the pool // and nil otherwise. -func (tp *TxPool) GetTransaction(hash common.Hash) *types.Transaction { - tp.mu.RLock() - defer tp.mu.RUnlock() - - return tp.all[hash] -} - -// GetTransactions returns all currently processable transactions. -// The returned slice may be modified by the caller. -func (self *TxPool) GetTransactions() types.Transactions { - self.mu.Lock() - defer self.mu.Unlock() - - // check queue first - self.checkQueue() - - // invalidate any txs - self.validatePool() +func (pool *TxPool) Get(hash common.Hash) *types.Transaction { + pool.mu.RLock() + defer pool.mu.RUnlock() - count := 0 - for _, txs := range self.pending { - count += len(txs) - } - pending := make(types.Transactions, 0, count) - for _, txs := range self.pending { - for _, tx := range txs { - pending = append(pending, tx) - } - } - return pending + return pool.all[hash] } -// RemoveTransactions removes all given transactions from the pool. -func (self *TxPool) RemoveTransactions(txs types.Transactions) { - self.mu.Lock() - defer self.mu.Unlock() +// Remove removes the transaction with the given hash from the pool. +func (pool *TxPool) Remove(hash common.Hash) { + pool.mu.Lock() + defer pool.mu.Unlock() - for _, tx := range txs { - self.removeTx(tx.Hash()) - } + pool.removeTx(hash) } -// RemoveTx removes the transaction with the given hash from the pool. -func (pool *TxPool) RemoveTx(hash common.Hash) { +// RemoveBatch removes all given transactions from the pool. +func (pool *TxPool) RemoveBatch(txs types.Transactions) { pool.mu.Lock() defer pool.mu.Unlock() - pool.removeTx(hash) + for _, tx := range txs { + pool.removeTx(tx.Hash()) + } } +// removeTx iterates removes a single transaction from the queue, moving all +// subsequent transactions back to the future queue. func (pool *TxPool) removeTx(hash common.Hash) { // Fetch the transaction we wish to delete tx, ok := pool.all[hash] if !ok { return } - addr, _ := tx.From() + addr, _ := tx.From() // already validated during insertion - // Remove it from all internal lists + // Remove it from the list of known transactions delete(pool.all, hash) - delete(pool.pending[addr], tx.Nonce()) - if len(pool.pending[addr]) == 0 { - delete(pool.pending, addr) + // 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 transactions are left, remove the list and reset the nonce + if pending.Empty() { + delete(pool.pending, addr) + pool.pendingState.SetNonce(addr, tx.Nonce()) + } else { + // Otherwise update the nonce and postpone any invalidated transactions + pool.pendingState.SetNonce(addr, pending.last) + for _, tx := range invalids { + pool.enqueueTx(tx.Hash(), tx) + } + } + } } - delete(pool.queue[addr], tx.Nonce()) - if len(pool.queue[addr]) == 0 { - delete(pool.queue, addr) + // Transaction is in the future queue + if future := pool.queue[addr]; future != nil { + future.Remove(tx) + if future.Empty() { + delete(pool.queue, addr) + } } } -// checkQueue moves transactions that have become processable from the pool's -// queue to the set of pending transactions. -func (pool *TxPool) checkQueue() { +// 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() { // Init delayed since tx pool could have been started before any state sync if pool.pendingState == nil { pool.resetState() } - - var promote txQueue - for address, txs := range pool.queue { - currentState, err := pool.currentState() - if err != nil { - glog.Errorf("could not get current state: %v", err) - return + // Retrieve the current state to allow nonce and balance checking + state, err := pool.currentState() + if err != nil { + glog.Errorf("Could not get current state: %v", err) + return + } + // Iterate over all accounts and promote any executable transactions + for addr, list := range pool.queue { + // Drop all transactions that are deemed too old (low nonce) + for _, tx := range list.Forward(state.GetNonce(addr)) { + if glog.V(logger.Core) { + glog.Infof("Removed old queued transaction: %v", tx) + } + delete(pool.all, tx.Hash()) } - balance := currentState.GetBalance(address) - - var ( - guessedNonce = pool.pendingState.GetNonce(address) // nonce currently kept by the tx pool (pending state) - trueNonce = currentState.GetNonce(address) // nonce known by the last state - ) - promote = promote[:0] - for nonce, tx := range txs { - // Drop processed or out of fund transactions - if nonce < trueNonce || balance.Cmp(tx.Cost()) < 0 { - if glog.V(logger.Core) { - glog.Infof("removed tx (%v) from pool queue: low tx nonce or out of funds\n", tx) - } - delete(txs, nonce) - delete(pool.all, tx.Hash()) - - continue + // Drop all transactions that are too costly (low balance) + drops, _ := list.Filter(state.GetBalance(addr)) + for _, tx := range drops { + if glog.V(logger.Core) { + glog.Infof("Removed unpayable queued transaction: %v", tx) } - // Collect the remaining transactions for the next pass. - promote = append(promote, txQueueEntry{address, tx}) + delete(pool.all, tx.Hash()) } - // Find the next consecutive nonce range starting at the current account nonce, - // pushing the guessed nonce forward if we add consecutive transactions. - sort.Sort(promote) - for i, entry := range promote { - // If we reached a gap in the nonces, enforce transaction limit and stop - if entry.Nonce() > guessedNonce { - if len(promote)-i > maxQueued { - if glog.V(logger.Debug) { - glog.Infof("Queued tx limit exceeded for %s. Tx %s removed\n", common.PP(address[:]), common.PP(entry.Hash().Bytes())) - } - for _, drop := range promote[i+maxQueued:] { - delete(txs, drop.Nonce()) - delete(pool.all, drop.Hash()) - } - } - break + // Gather all executable transactions and promote them + for _, tx := range list.Ready(pool.pendingState.GetNonce(addr)) { + if glog.V(logger.Core) { + glog.Infof("Promoting queued transaction: %v", tx) } - // Otherwise promote the transaction and move the guess nonce if needed - pool.addTx(address, entry.Transaction) - delete(txs, entry.Nonce()) - - if entry.Nonce() == guessedNonce { - guessedNonce++ + pool.promoteTx(addr, tx.Hash(), tx) + } + // Drop all transactions over the allowed limit + for _, tx := range list.Cap(maxQueued) { + if glog.V(logger.Core) { + glog.Infof("Removed cap-exceeding queued transaction: %v", tx) } + delete(pool.all, tx.Hash()) } // Delete the entire queue entry if it became empty. - if len(txs) == 0 { - delete(pool.queue, address) + if list.Empty() { + delete(pool.queue, addr) } } } -// validatePool removes invalid and processed transactions from the main pool. -// If a transaction is removed for being invalid (e.g. out of funds), all sub- -// sequent (Still valid) transactions are moved back into the future queue. This -// is important to prevent a drained account from DOSing the network with non -// executable transactions. -func (pool *TxPool) validatePool() { +// 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. +func (pool *TxPool) demoteUnexecutables() { + // Retrieve the current state to allow nonce and balance checking state, err := pool.currentState() if err != nil { glog.V(logger.Info).Infoln("failed to get current state: %v", err) return } - balanceCache := make(map[common.Address]*big.Int) - - // Clean up the pending pool, accumulating invalid nonces - gaps := make(map[common.Address]uint64) + // Iterate over all accounts and demote any non-executable transactions + for addr, list := range pool.pending { + nonce := state.GetNonce(addr) - for addr, txs := range pool.pending { - for nonce, tx := range txs { - // Perform light nonce and balance validation - balance := balanceCache[addr] - if balance == nil { - balance = state.GetBalance(addr) - balanceCache[addr] = balance + // Drop all transactions that are deemed too old (low nonce) + for _, tx := range list.Forward(nonce) { + if glog.V(logger.Core) { + glog.Infof("Removed old pending transaction: %v", tx) } - if past := state.GetNonce(addr) > nonce; past || balance.Cmp(tx.Cost()) < 0 { - // Remove an already past it invalidated transaction - if glog.V(logger.Core) { - glog.Infof("removed tx (%v) from pool: low tx nonce or out of funds\n", tx) - } - delete(pool.pending[addr], nonce) - if len(pool.pending[addr]) == 0 { - delete(pool.pending, addr) - } - delete(pool.all, tx.Hash()) - - // Track the smallest invalid nonce to postpone subsequent transactions - if !past { - if prev, ok := gaps[addr]; !ok || nonce < prev { - gaps[addr] = nonce - } - } + delete(pool.all, tx.Hash()) + } + // Drop all transactions that are too costly (low balance), and queue any invalids back for later + drops, invalids := list.Filter(state.GetBalance(addr)) + for _, tx := range drops { + if glog.V(logger.Core) { + glog.Infof("Removed unpayable pending transaction: %v", tx) } + delete(pool.all, tx.Hash()) } - } - // Move all transactions after a gap back to the future queue - if len(gaps) > 0 { - for addr, txs := range pool.pending { - for nonce, tx := range txs { - if gap, ok := gaps[addr]; ok && nonce >= gap { - if glog.V(logger.Core) { - glog.Infof("postponed tx (%v) due to introduced gap\n", tx) - } - delete(pool.pending[addr], nonce) - if len(pool.pending[addr]) == 0 { - delete(pool.pending, addr) - } - pool.queueTx(tx.Hash(), tx) - } + for _, tx := range invalids { + if glog.V(logger.Core) { + glog.Infof("Demoting pending transaction: %v", tx) } + pool.enqueueTx(tx.Hash(), tx) + } + // Delete the entire queue entry if it became empty. + if list.Empty() { + delete(pool.pending, addr) } } } -type txQueue []txQueueEntry - -type txQueueEntry struct { - addr common.Address - *types.Transaction -} - -func (q txQueue) Len() int { return len(q) } -func (q txQueue) Swap(i, j int) { q[i], q[j] = q[j], q[i] } -func (q txQueue) Less(i, j int) bool { return q[i].Nonce() < q[j].Nonce() } - // txSet represents a set of transaction hashes in which entries // are automatically dropped after txSetDuration time type txSet struct { diff --git a/core/tx_pool_test.go b/core/tx_pool_test.go index 8aa69233d..ec54d8c0e 100644 --- a/core/tx_pool_test.go +++ b/core/tx_pool_test.go @@ -91,9 +91,9 @@ func TestTransactionQueue(t *testing.T) { from, _ := tx.From() currentState, _ := pool.currentState() currentState.AddBalance(from, big.NewInt(1000)) - pool.queueTx(tx.Hash(), tx) + pool.enqueueTx(tx.Hash(), tx) - pool.checkQueue() + pool.promoteExecutables() if len(pool.pending) != 1 { t.Error("expected valid txs to be 1 is", len(pool.pending)) } @@ -101,14 +101,14 @@ func TestTransactionQueue(t *testing.T) { tx = transaction(1, big.NewInt(100), key) from, _ = tx.From() currentState.SetNonce(from, 2) - pool.queueTx(tx.Hash(), tx) - pool.checkQueue() - if _, ok := pool.pending[from][tx.Nonce()]; ok { + pool.enqueueTx(tx.Hash(), tx) + pool.promoteExecutables() + if _, ok := pool.pending[from].items[tx.Nonce()]; ok { t.Error("expected transaction to be in tx pool") } - if len(pool.queue[from]) > 0 { - t.Error("expected transaction queue to be empty. is", len(pool.queue[from])) + if len(pool.queue) > 0 { + t.Error("expected transaction queue to be empty. is", len(pool.queue)) } pool, key = setupTxPool() @@ -118,17 +118,17 @@ func TestTransactionQueue(t *testing.T) { from, _ = tx1.From() currentState, _ = pool.currentState() currentState.AddBalance(from, big.NewInt(1000)) - pool.queueTx(tx1.Hash(), tx1) - pool.queueTx(tx2.Hash(), tx2) - pool.queueTx(tx3.Hash(), tx3) + pool.enqueueTx(tx1.Hash(), tx1) + pool.enqueueTx(tx2.Hash(), tx2) + pool.enqueueTx(tx3.Hash(), tx3) - pool.checkQueue() + pool.promoteExecutables() if len(pool.pending) != 1 { t.Error("expected tx pool to be 1, got", len(pool.pending)) } - if len(pool.queue[from]) != 2 { - t.Error("expected len(queue) == 2, got", len(pool.queue[from])) + if pool.queue[from].Len() != 2 { + t.Error("expected len(queue) == 2, got", pool.queue[from].Len()) } } @@ -138,24 +138,21 @@ func TestRemoveTx(t *testing.T) { from, _ := tx.From() currentState, _ := pool.currentState() currentState.AddBalance(from, big.NewInt(1)) - pool.queueTx(tx.Hash(), tx) - pool.addTx(from, tx) + + pool.enqueueTx(tx.Hash(), tx) + pool.promoteTx(from, tx.Hash(), tx) if len(pool.queue) != 1 { t.Error("expected queue to be 1, got", len(pool.queue)) } - if len(pool.pending) != 1 { - t.Error("expected txs to be 1, got", len(pool.pending)) + t.Error("expected pending to be 1, got", len(pool.pending)) } - - pool.RemoveTx(tx.Hash()) - + pool.Remove(tx.Hash()) if len(pool.queue) > 0 { t.Error("expected queue to be 0, got", len(pool.queue)) } - if len(pool.pending) > 0 { - t.Error("expected txs to be 0, got", len(pool.pending)) + t.Error("expected pending to be 0, got", len(pool.pending)) } } @@ -188,7 +185,7 @@ func TestTransactionChainFork(t *testing.T) { if err := pool.add(tx); err != nil { t.Error("didn't expect error", err) } - pool.RemoveTransactions([]*types.Transaction{tx}) + pool.RemoveBatch([]*types.Transaction{tx}) // reset the pool's internal state resetState() @@ -221,22 +218,22 @@ func TestTransactionDoubleNonce(t *testing.T) { if err := pool.add(tx2); err != nil { t.Error("didn't expect error", err) } - pool.checkQueue() - if len(pool.pending[addr]) != 1 { - t.Error("expected 1 pending transactions, got", len(pool.pending)) + pool.promoteExecutables() + if pool.pending[addr].Len() != 1 { + t.Error("expected 1 pending transactions, got", pool.pending[addr].Len()) } - if tx := pool.pending[addr][0]; tx.Hash() != tx2.Hash() { + if tx := pool.pending[addr].items[0]; tx.Hash() != tx2.Hash() { t.Errorf("transaction mismatch: have %x, want %x", tx.Hash(), tx2.Hash()) } // Add the thid transaction and ensure it's not saved (smaller price) if err := pool.add(tx3); err != nil { t.Error("didn't expect error", err) } - pool.checkQueue() - if len(pool.pending[addr]) != 1 { - t.Error("expected 1 pending transactions, got", len(pool.pending)) + pool.promoteExecutables() + if pool.pending[addr].Len() != 1 { + t.Error("expected 1 pending transactions, got", pool.pending[addr].Len()) } - if tx := pool.pending[addr][0]; tx.Hash() != tx2.Hash() { + if tx := pool.pending[addr].items[0]; tx.Hash() != tx2.Hash() { t.Errorf("transaction mismatch: have %x, want %x", tx.Hash(), tx2.Hash()) } // Ensure the total transaction count is correct @@ -254,11 +251,11 @@ func TestMissingNonce(t *testing.T) { if err := pool.add(tx); err != nil { t.Error("didn't expect error", err) } - if len(pool.pending[addr]) != 0 { - t.Error("expected 0 pending transactions, got", len(pool.pending[addr])) + if len(pool.pending) != 0 { + t.Error("expected 0 pending transactions, got", len(pool.pending)) } - if len(pool.queue[addr]) != 1 { - t.Error("expected 1 queued transaction, got", len(pool.queue[addr])) + if pool.queue[addr].Len() != 1 { + t.Error("expected 1 queued transaction, got", pool.queue[addr].Len()) } if len(pool.all) != 1 { t.Error("expected 1 total transactions, got", len(pool.all)) @@ -293,8 +290,8 @@ func TestRemovedTxEvent(t *testing.T) { currentState.AddBalance(from, big.NewInt(1000000000000)) pool.eventMux.Post(RemovedTransactionEvent{types.Transactions{tx}}) pool.eventMux.Post(ChainHeadEvent{nil}) - if len(pool.pending[from]) != 1 { - t.Error("expected 1 pending tx, got", len(pool.pending[from])) + if pool.pending[from].Len() != 1 { + t.Error("expected 1 pending tx, got", pool.pending[from].Len()) } if len(pool.all) != 1 { t.Error("expected 1 total transactions, got", len(pool.all)) @@ -318,27 +315,27 @@ func TestTransactionDropping(t *testing.T) { tx10 = transaction(10, big.NewInt(100), key) tx11 = transaction(11, big.NewInt(200), key) ) - pool.addTx(account, tx0) - pool.addTx(account, tx1) - pool.queueTx(tx10.Hash(), tx10) - pool.queueTx(tx11.Hash(), tx11) + pool.promoteTx(account, tx0.Hash(), tx0) + pool.promoteTx(account, tx1.Hash(), tx1) + pool.enqueueTx(tx10.Hash(), tx10) + pool.enqueueTx(tx11.Hash(), tx11) // Check that pre and post validations leave the pool as is - if len(pool.pending[account]) != 2 { - t.Errorf("pending transaction mismatch: have %d, want %d", len(pool.pending[account]), 2) + if pool.pending[account].Len() != 2 { + t.Errorf("pending transaction mismatch: have %d, want %d", pool.pending[account].Len(), 2) } - if len(pool.queue[account]) != 2 { - t.Errorf("queued transaction mismatch: have %d, want %d", len(pool.queue[account]), 2) + if pool.queue[account].Len() != 2 { + t.Errorf("queued transaction mismatch: have %d, want %d", pool.queue[account].Len(), 2) } if len(pool.all) != 4 { t.Errorf("total transaction mismatch: have %d, want %d", len(pool.all), 4) } pool.resetState() - if len(pool.pending[account]) != 2 { - t.Errorf("pending transaction mismatch: have %d, want %d", len(pool.pending[account]), 2) + if pool.pending[account].Len() != 2 { + t.Errorf("pending transaction mismatch: have %d, want %d", pool.pending[account].Len(), 2) } - if len(pool.queue[account]) != 2 { - t.Errorf("queued transaction mismatch: have %d, want %d", len(pool.queue[account]), 2) + if pool.queue[account].Len() != 2 { + t.Errorf("queued transaction mismatch: have %d, want %d", pool.queue[account].Len(), 2) } if len(pool.all) != 4 { t.Errorf("total transaction mismatch: have %d, want %d", len(pool.all), 4) @@ -347,16 +344,16 @@ func TestTransactionDropping(t *testing.T) { state.AddBalance(account, big.NewInt(-750)) pool.resetState() - if _, ok := pool.pending[account][tx0.Nonce()]; !ok { + if _, ok := pool.pending[account].items[tx0.Nonce()]; !ok { t.Errorf("funded pending transaction missing: %v", tx0) } - if _, ok := pool.pending[account][tx1.Nonce()]; ok { + if _, ok := pool.pending[account].items[tx1.Nonce()]; ok { t.Errorf("out-of-fund pending transaction present: %v", tx1) } - if _, ok := pool.queue[account][tx10.Nonce()]; !ok { + if _, ok := pool.queue[account].items[tx10.Nonce()]; !ok { t.Errorf("funded queued transaction missing: %v", tx10) } - if _, ok := pool.queue[account][tx11.Nonce()]; ok { + if _, ok := pool.queue[account].items[tx11.Nonce()]; ok { t.Errorf("out-of-fund queued transaction present: %v", tx11) } if len(pool.all) != 2 { @@ -384,25 +381,25 @@ func TestTransactionPostponing(t *testing.T) { } else { tx = transaction(uint64(i), big.NewInt(500), key) } - pool.addTx(account, tx) + pool.promoteTx(account, tx.Hash(), tx) txns = append(txns, tx) } // Check that pre and post validations leave the pool as is - if len(pool.pending[account]) != len(txns) { - t.Errorf("pending transaction mismatch: have %d, want %d", len(pool.pending[account]), len(txns)) + if pool.pending[account].Len() != len(txns) { + t.Errorf("pending transaction mismatch: have %d, want %d", pool.pending[account].Len(), len(txns)) } - if len(pool.queue[account]) != 0 { - t.Errorf("queued transaction mismatch: have %d, want %d", len(pool.queue[account]), 0) + if len(pool.queue) != 0 { + t.Errorf("queued transaction mismatch: have %d, want %d", pool.queue[account].Len(), 0) } if len(pool.all) != len(txns) { t.Errorf("total transaction mismatch: have %d, want %d", len(pool.all), len(txns)) } pool.resetState() - if len(pool.pending[account]) != len(txns) { - t.Errorf("pending transaction mismatch: have %d, want %d", len(pool.pending[account]), len(txns)) + if pool.pending[account].Len() != len(txns) { + t.Errorf("pending transaction mismatch: have %d, want %d", pool.pending[account].Len(), len(txns)) } - if len(pool.queue[account]) != 0 { - t.Errorf("queued transaction mismatch: have %d, want %d", len(pool.queue[account]), 0) + if len(pool.queue) != 0 { + t.Errorf("queued transaction mismatch: have %d, want %d", pool.queue[account].Len(), 0) } if len(pool.all) != len(txns) { t.Errorf("total transaction mismatch: have %d, want %d", len(pool.all), len(txns)) @@ -411,25 +408,25 @@ func TestTransactionPostponing(t *testing.T) { state.AddBalance(account, big.NewInt(-750)) pool.resetState() - if _, ok := pool.pending[account][txns[0].Nonce()]; !ok { + if _, ok := pool.pending[account].items[txns[0].Nonce()]; !ok { t.Errorf("tx %d: valid and funded transaction missing from pending pool: %v", 0, txns[0]) } - if _, ok := pool.queue[account][txns[0].Nonce()]; ok { + if _, ok := pool.queue[account].items[txns[0].Nonce()]; ok { t.Errorf("tx %d: valid and funded transaction present in future queue: %v", 0, txns[0]) } for i, tx := range txns[1:] { if i%2 == 1 { - if _, ok := pool.pending[account][tx.Nonce()]; ok { + if _, ok := pool.pending[account].items[tx.Nonce()]; ok { t.Errorf("tx %d: valid but future transaction present in pending pool: %v", i+1, tx) } - if _, ok := pool.queue[account][tx.Nonce()]; !ok { + if _, ok := pool.queue[account].items[tx.Nonce()]; !ok { t.Errorf("tx %d: valid but future transaction missing from future queue: %v", i+1, tx) } } else { - if _, ok := pool.pending[account][tx.Nonce()]; ok { + if _, ok := pool.pending[account].items[tx.Nonce()]; ok { t.Errorf("tx %d: out-of-fund transaction present in pending pool: %v", i+1, tx) } - if _, ok := pool.queue[account][tx.Nonce()]; ok { + if _, ok := pool.queue[account].items[tx.Nonce()]; ok { t.Errorf("tx %d: out-of-fund transaction present in future queue: %v", i+1, tx) } } @@ -458,12 +455,12 @@ func TestTransactionQueueLimiting(t *testing.T) { t.Errorf("tx %d: pending pool size mismatch: have %d, want %d", i, len(pool.pending), 0) } if i <= maxQueued { - if len(pool.queue[account]) != int(i) { - t.Errorf("tx %d: queue size mismatch: have %d, want %d", i, len(pool.queue[account]), i) + if pool.queue[account].Len() != int(i) { + t.Errorf("tx %d: queue size mismatch: have %d, want %d", i, pool.queue[account].Len(), i) } } else { - if len(pool.queue[account]) != maxQueued { - t.Errorf("tx %d: queue limit mismatch: have %d, want %d", i, len(pool.queue[account]), maxQueued) + if pool.queue[account].Len() != maxQueued { + t.Errorf("tx %d: queue limit mismatch: have %d, want %d", i, pool.queue[account].Len(), maxQueued) } } } @@ -488,11 +485,11 @@ func TestTransactionPendingLimiting(t *testing.T) { if err := pool.Add(transaction(i, big.NewInt(100000), key)); err != nil { t.Fatalf("tx %d: failed to add transaction: %v", i, err) } - if len(pool.pending[account]) != int(i)+1 { - t.Errorf("tx %d: pending pool size mismatch: have %d, want %d", i, len(pool.pending[account]), i+1) + if pool.pending[account].Len() != int(i)+1 { + t.Errorf("tx %d: pending pool size mismatch: have %d, want %d", i, pool.pending[account].Len(), i+1) } - if len(pool.queue[account]) != 0 { - t.Errorf("tx %d: queue size mismatch: have %d, want %d", i, len(pool.queue[account]), 0) + if len(pool.queue) != 0 { + t.Errorf("tx %d: queue size mismatch: have %d, want %d", i, pool.queue[account].Len(), 0) } } if len(pool.all) != maxQueued+5 { @@ -517,7 +514,7 @@ func testTransactionLimitingEquivalency(t *testing.T, origin uint64) { t.Fatalf("tx %d: failed to add transaction: %v", i, err) } } - // Add a batch of transactions to a pool in one bit batch + // Add a batch of transactions to a pool in one big batch pool2, key2 := setupTxPool() account2, _ := transaction(0, big.NewInt(0), key2).From() state2, _ := pool2.currentState() @@ -527,14 +524,14 @@ func testTransactionLimitingEquivalency(t *testing.T, origin uint64) { for i := uint64(0); i < maxQueued+5; i++ { txns = append(txns, transaction(origin+i, big.NewInt(100000), key2)) } - pool2.AddTransactions(txns) + pool2.AddBatch(txns) // Ensure the batch optimization honors the same pool mechanics if len(pool1.pending) != len(pool2.pending) { t.Errorf("pending transaction count mismatch: one-by-one algo: %d, batch algo: %d", len(pool1.pending), len(pool2.pending)) } - if len(pool1.queue[account1]) != len(pool2.queue[account2]) { - t.Errorf("queued transaction count mismatch: one-by-one algo: %d, batch algo: %d", len(pool1.queue[account1]), len(pool2.queue[account2])) + if len(pool1.queue) != len(pool2.queue) { + t.Errorf("queued transaction count mismatch: one-by-one algo: %d, batch algo: %d", len(pool1.queue), len(pool2.queue)) } if len(pool1.all) != len(pool2.all) { t.Errorf("total transaction count mismatch: one-by-one algo %d, batch algo %d", len(pool1.all), len(pool2.all)) @@ -543,11 +540,11 @@ func testTransactionLimitingEquivalency(t *testing.T, origin uint64) { // Benchmarks the speed of validating the contents of the pending queue of the // transaction pool. -func BenchmarkValidatePool100(b *testing.B) { benchmarkValidatePool(b, 100) } -func BenchmarkValidatePool1000(b *testing.B) { benchmarkValidatePool(b, 1000) } -func BenchmarkValidatePool10000(b *testing.B) { benchmarkValidatePool(b, 10000) } +func BenchmarkPendingDemotion100(b *testing.B) { benchmarkPendingDemotion(b, 100) } +func BenchmarkPendingDemotion1000(b *testing.B) { benchmarkPendingDemotion(b, 1000) } +func BenchmarkPendingDemotion10000(b *testing.B) { benchmarkPendingDemotion(b, 10000) } -func benchmarkValidatePool(b *testing.B, size int) { +func benchmarkPendingDemotion(b *testing.B, size int) { // Add a batch of transactions to a pool one by one pool, key := setupTxPool() account, _ := transaction(0, big.NewInt(0), key).From() @@ -556,22 +553,22 @@ func benchmarkValidatePool(b *testing.B, size int) { for i := 0; i < size; i++ { tx := transaction(uint64(i), big.NewInt(100000), key) - pool.addTx(account, tx) + pool.promoteTx(account, tx.Hash(), tx) } // Benchmark the speed of pool validation b.ResetTimer() for i := 0; i < b.N; i++ { - pool.validatePool() + pool.demoteUnexecutables() } } // Benchmarks the speed of scheduling the contents of the future queue of the // transaction pool. -func BenchmarkCheckQueue100(b *testing.B) { benchmarkCheckQueue(b, 100) } -func BenchmarkCheckQueue1000(b *testing.B) { benchmarkCheckQueue(b, 1000) } -func BenchmarkCheckQueue10000(b *testing.B) { benchmarkCheckQueue(b, 10000) } +func BenchmarkFuturePromotion100(b *testing.B) { benchmarkFuturePromotion(b, 100) } +func BenchmarkFuturePromotion1000(b *testing.B) { benchmarkFuturePromotion(b, 1000) } +func BenchmarkFuturePromotion10000(b *testing.B) { benchmarkFuturePromotion(b, 10000) } -func benchmarkCheckQueue(b *testing.B, size int) { +func benchmarkFuturePromotion(b *testing.B, size int) { // Add a batch of transactions to a pool one by one pool, key := setupTxPool() account, _ := transaction(0, big.NewInt(0), key).From() @@ -580,11 +577,56 @@ func benchmarkCheckQueue(b *testing.B, size int) { for i := 0; i < size; i++ { tx := transaction(uint64(1+i), big.NewInt(100000), key) - pool.queueTx(tx.Hash(), tx) + pool.enqueueTx(tx.Hash(), tx) } // Benchmark the speed of pool validation b.ResetTimer() for i := 0; i < b.N; i++ { - pool.checkQueue() + pool.promoteExecutables() + } +} + +// Benchmarks the speed of iterative transaction insertion. +func BenchmarkPoolInsert(b *testing.B) { + // Generate a batch of transactions to enqueue into the pool + pool, key := setupTxPool() + account, _ := transaction(0, big.NewInt(0), key).From() + state, _ := pool.currentState() + state.AddBalance(account, big.NewInt(1000000)) + + txs := make(types.Transactions, b.N) + for i := 0; i < b.N; i++ { + txs[i] = transaction(uint64(i), big.NewInt(100000), key) + } + // Benchmark importing the transactions into the queue + b.ResetTimer() + for _, tx := range txs { + pool.Add(tx) + } +} + +// Benchmarks the speed of batched transaction insertion. +func BenchmarkPoolBatchInsert100(b *testing.B) { benchmarkPoolBatchInsert(b, 100) } +func BenchmarkPoolBatchInsert1000(b *testing.B) { benchmarkPoolBatchInsert(b, 1000) } +func BenchmarkPoolBatchInsert10000(b *testing.B) { benchmarkPoolBatchInsert(b, 10000) } + +func benchmarkPoolBatchInsert(b *testing.B, size int) { + // Generate a batch of transactions to enqueue into the pool + pool, key := setupTxPool() + account, _ := transaction(0, big.NewInt(0), key).From() + state, _ := pool.currentState() + state.AddBalance(account, big.NewInt(1000000)) + + batches := make([]types.Transactions, b.N) + for i := 0; i < b.N; i++ { + batches[i] = make(types.Transactions, size) + for j := 0; j < size; j++ { + batches[i][j] = transaction(uint64(size*i+j), big.NewInt(100000), key) + } + } + // Benchmark importing the transactions into the queue + b.ResetTimer() + for _, batch := range batches { + pool.AddBatch(batch) } } diff --git a/core/types/transaction.go b/core/types/transaction.go index d369d7772..af48e4d07 100644 --- a/core/types/transaction.go +++ b/core/types/transaction.go @@ -24,7 +24,6 @@ import ( "fmt" "io" "math/big" - "sort" "sync/atomic" "github.com/ethereum/go-ethereum/common" @@ -439,37 +438,29 @@ func (s *TxByPrice) Pop() interface{} { // sender accounts and sorts them by nonce. After the account nonce ordering is // satisfied, the results are merged back together by price, always comparing only // the head transaction from each account. This is done via a heap to keep it fast. -func SortByPriceAndNonce(txs []*Transaction) { - // Separate the transactions by account and sort by nonce - byNonce := make(map[common.Address][]*Transaction) - for _, tx := range txs { - acc, _ := tx.From() // we only sort valid txs so this cannot fail - byNonce[acc] = append(byNonce[acc], tx) - } - for _, accTxs := range byNonce { - sort.Sort(TxByNonce(accTxs)) - } +func SortByPriceAndNonce(txs map[common.Address]Transactions) Transactions { // Initialize a price based heap with the head transactions - byPrice := make(TxByPrice, 0, len(byNonce)) - for acc, accTxs := range byNonce { + byPrice := make(TxByPrice, 0, len(txs)) + for acc, accTxs := range txs { byPrice = append(byPrice, accTxs[0]) - byNonce[acc] = accTxs[1:] + txs[acc] = accTxs[1:] } heap.Init(&byPrice) // Merge by replacing the best with the next from the same account - txs = txs[:0] + var sorted Transactions for len(byPrice) > 0 { // Retrieve the next best transaction by price best := heap.Pop(&byPrice).(*Transaction) // Push in its place the next transaction from the same account acc, _ := best.From() // we only sort valid txs so this cannot fail - if accTxs, ok := byNonce[acc]; ok && len(accTxs) > 0 { + if accTxs, ok := txs[acc]; ok && len(accTxs) > 0 { heap.Push(&byPrice, accTxs[0]) - byNonce[acc] = accTxs[1:] + txs[acc] = accTxs[1:] } // Accumulate the best priced transaction - txs = append(txs, best) + sorted = append(sorted, best) } + return sorted } diff --git a/core/types/transaction_test.go b/core/types/transaction_test.go index 62420e71f..c6e6e3790 100644 --- a/core/types/transaction_test.go +++ b/core/types/transaction_test.go @@ -128,15 +128,16 @@ func TestTransactionPriceNonceSort(t *testing.T) { keys[i], _ = crypto.GenerateKey() } // Generate a batch of transactions with overlapping values, but shifted nonces - txs := []*Transaction{} + groups := map[common.Address]Transactions{} for start, key := range keys { + addr := crypto.PubkeyToAddress(key.PublicKey) for i := 0; i < 25; i++ { tx, _ := NewTransaction(uint64(start+i), common.Address{}, big.NewInt(100), big.NewInt(100), big.NewInt(int64(start+i)), nil).SignECDSA(key) - txs = append(txs, tx) + groups[addr] = append(groups[addr], tx) } } // Sort the transactions and cross check the nonce ordering - SortByPriceAndNonce(txs) + txs := SortByPriceAndNonce(groups) for i, txi := range txs { fromi, _ := txi.From() -- cgit v1.2.3