* mempool: consoldate implementations * update chagelog * fix test * Apply suggestions from code review Co-authored-by: M. J. Fromberger <michael.j.fromberger@gmail.com> * cleanup locking comments * context twiddle * migrate away from deprecated ioutil APIs (#7175) Co-authored-by: Callum Waters <cmwaters19@gmail.com> Co-authored-by: M. J. Fromberger <fromberger@interchain.io> Co-authored-by: M. J. Fromberger <michael.j.fromberger@gmail.com> Co-authored-by: Callum Waters <cmwaters19@gmail.com> Co-authored-by: M. J. Fromberger <fromberger@interchain.io>pull/7187/head
@ -1,143 +1,869 @@ | |||
package mempool | |||
import ( | |||
"bytes" | |||
"context" | |||
"errors" | |||
"fmt" | |||
"math" | |||
"reflect" | |||
"sync/atomic" | |||
"time" | |||
abci "github.com/tendermint/tendermint/abci/types" | |||
"github.com/tendermint/tendermint/internal/p2p" | |||
"github.com/tendermint/tendermint/config" | |||
"github.com/tendermint/tendermint/internal/libs/clist" | |||
tmsync "github.com/tendermint/tendermint/internal/libs/sync" | |||
"github.com/tendermint/tendermint/internal/proxy" | |||
"github.com/tendermint/tendermint/libs/log" | |||
tmmath "github.com/tendermint/tendermint/libs/math" | |||
"github.com/tendermint/tendermint/types" | |||
) | |||
const ( | |||
MempoolChannel = p2p.ChannelID(0x30) | |||
var _ Mempool = (*TxMempool)(nil) | |||
// PeerCatchupSleepIntervalMS defines how much time to sleep if a peer is behind | |||
PeerCatchupSleepIntervalMS = 100 | |||
// TxMempoolOption sets an optional parameter on the TxMempool. | |||
type TxMempoolOption func(*TxMempool) | |||
// UnknownPeerID is the peer ID to use when running CheckTx when there is | |||
// no peer (e.g. RPC) | |||
UnknownPeerID uint16 = 0 | |||
// TxMempool defines a prioritized mempool data structure used by the v1 mempool | |||
// reactor. It keeps a thread-safe priority queue of transactions that is used | |||
// when a block proposer constructs a block and a thread-safe linked-list that | |||
// is used to gossip transactions to peers in a FIFO manner. | |||
type TxMempool struct { | |||
logger log.Logger | |||
metrics *Metrics | |||
config *config.MempoolConfig | |||
proxyAppConn proxy.AppConnMempool | |||
MaxActiveIDs = math.MaxUint16 | |||
) | |||
// txsAvailable fires once for each height when the mempool is not empty | |||
txsAvailable chan struct{} | |||
notifiedTxsAvailable bool | |||
// Mempool defines the mempool interface. | |||
// | |||
// Updates to the mempool need to be synchronized with committing a block so | |||
// applications can reset their transient state on Commit. | |||
type Mempool interface { | |||
// CheckTx executes a new transaction against the application to determine | |||
// its validity and whether it should be added to the mempool. | |||
CheckTx(ctx context.Context, tx types.Tx, callback func(*abci.Response), txInfo TxInfo) error | |||
// RemoveTxByKey removes a transaction, identified by its key, | |||
// from the mempool. | |||
RemoveTxByKey(txKey types.TxKey) error | |||
// ReapMaxBytesMaxGas reaps transactions from the mempool up to maxBytes | |||
// bytes total with the condition that the total gasWanted must be less than | |||
// maxGas. | |||
// | |||
// If both maxes are negative, there is no cap on the size of all returned | |||
// transactions (~ all available transactions). | |||
ReapMaxBytesMaxGas(maxBytes, maxGas int64) types.Txs | |||
// height defines the last block height process during Update() | |||
height int64 | |||
// ReapMaxTxs reaps up to max transactions from the mempool. If max is | |||
// negative, there is no cap on the size of all returned transactions | |||
// (~ all available transactions). | |||
ReapMaxTxs(max int) types.Txs | |||
// sizeBytes defines the total size of the mempool (sum of all tx bytes) | |||
sizeBytes int64 | |||
// Lock locks the mempool. The consensus must be able to hold lock to safely | |||
// update. | |||
Lock() | |||
// cache defines a fixed-size cache of already seen transactions as this | |||
// reduces pressure on the proxyApp. | |||
cache TxCache | |||
// Unlock unlocks the mempool. | |||
Unlock() | |||
// txStore defines the main storage of valid transactions. Indexes are built | |||
// on top of this store. | |||
txStore *TxStore | |||
// Update informs the mempool that the given txs were committed and can be | |||
// discarded. | |||
// | |||
// NOTE: | |||
// 1. This should be called *after* block is committed by consensus. | |||
// 2. Lock/Unlock must be managed by the caller. | |||
Update( | |||
blockHeight int64, | |||
blockTxs types.Txs, | |||
deliverTxResponses []*abci.ResponseDeliverTx, | |||
newPreFn PreCheckFunc, | |||
newPostFn PostCheckFunc, | |||
) error | |||
// FlushAppConn flushes the mempool connection to ensure async callback calls | |||
// are done, e.g. from CheckTx. | |||
// gossipIndex defines the gossiping index of valid transactions via a | |||
// thread-safe linked-list. We also use the gossip index as a cursor for | |||
// rechecking transactions already in the mempool. | |||
gossipIndex *clist.CList | |||
// recheckCursor and recheckEnd are used as cursors based on the gossip index | |||
// to recheck transactions that are already in the mempool. Iteration is not | |||
// thread-safe and transaction may be mutated in serial order. | |||
// | |||
// NOTE: | |||
// 1. Lock/Unlock must be managed by caller. | |||
FlushAppConn() error | |||
// XXX/TODO: It might be somewhat of a codesmell to use the gossip index for | |||
// iterator and cursor management when rechecking transactions. If the gossip | |||
// index changes or is removed in a future refactor, this will have to be | |||
// refactored. Instead, we should consider just keeping a slice of a snapshot | |||
// of the mempool's current transactions during Update and an integer cursor | |||
// into that slice. This, however, requires additional O(n) space complexity. | |||
recheckCursor *clist.CElement // next expected response | |||
recheckEnd *clist.CElement // re-checking stops here | |||
// Flush removes all transactions from the mempool and caches. | |||
Flush() | |||
// priorityIndex defines the priority index of valid transactions via a | |||
// thread-safe priority queue. | |||
priorityIndex *TxPriorityQueue | |||
// TxsAvailable returns a channel which fires once for every height, and only | |||
// when transactions are available in the mempool. | |||
// | |||
// NOTE: | |||
// 1. The returned channel may be nil if EnableTxsAvailable was not called. | |||
TxsAvailable() <-chan struct{} | |||
// heightIndex defines a height-based, in ascending order, transaction index. | |||
// i.e. older transactions are first. | |||
heightIndex *WrappedTxList | |||
// timestampIndex defines a timestamp-based, in ascending order, transaction | |||
// index. i.e. older transactions are first. | |||
timestampIndex *WrappedTxList | |||
// A read/write lock is used to safe guard updates, insertions and deletions | |||
// from the mempool. A read-lock is implicitly acquired when executing CheckTx, | |||
// however, a caller must explicitly grab a write-lock via Lock when updating | |||
// the mempool via Update(). | |||
mtx tmsync.RWMutex | |||
preCheck PreCheckFunc | |||
postCheck PostCheckFunc | |||
} | |||
func NewTxMempool( | |||
logger log.Logger, | |||
cfg *config.MempoolConfig, | |||
proxyAppConn proxy.AppConnMempool, | |||
height int64, | |||
options ...TxMempoolOption, | |||
) *TxMempool { | |||
txmp := &TxMempool{ | |||
logger: logger, | |||
config: cfg, | |||
proxyAppConn: proxyAppConn, | |||
height: height, | |||
cache: NopTxCache{}, | |||
metrics: NopMetrics(), | |||
txStore: NewTxStore(), | |||
gossipIndex: clist.New(), | |||
priorityIndex: NewTxPriorityQueue(), | |||
heightIndex: NewWrappedTxList(func(wtx1, wtx2 *WrappedTx) bool { | |||
return wtx1.height >= wtx2.height | |||
}), | |||
timestampIndex: NewWrappedTxList(func(wtx1, wtx2 *WrappedTx) bool { | |||
return wtx1.timestamp.After(wtx2.timestamp) || wtx1.timestamp.Equal(wtx2.timestamp) | |||
}), | |||
} | |||
if cfg.CacheSize > 0 { | |||
txmp.cache = NewLRUTxCache(cfg.CacheSize) | |||
} | |||
proxyAppConn.SetResponseCallback(txmp.defaultTxCallback) | |||
for _, opt := range options { | |||
opt(txmp) | |||
} | |||
return txmp | |||
} | |||
// WithPreCheck sets a filter for the mempool to reject a transaction if f(tx) | |||
// returns an error. This is executed before CheckTx. It only applies to the | |||
// first created block. After that, Update() overwrites the existing value. | |||
func WithPreCheck(f PreCheckFunc) TxMempoolOption { | |||
return func(txmp *TxMempool) { txmp.preCheck = f } | |||
} | |||
// WithPostCheck sets a filter for the mempool to reject a transaction if | |||
// f(tx, resp) returns an error. This is executed after CheckTx. It only applies | |||
// to the first created block. After that, Update overwrites the existing value. | |||
func WithPostCheck(f PostCheckFunc) TxMempoolOption { | |||
return func(txmp *TxMempool) { txmp.postCheck = f } | |||
} | |||
// WithMetrics sets the mempool's metrics collector. | |||
func WithMetrics(metrics *Metrics) TxMempoolOption { | |||
return func(txmp *TxMempool) { txmp.metrics = metrics } | |||
} | |||
// Lock obtains a write-lock on the mempool. A caller must be sure to explicitly | |||
// release the lock when finished. | |||
func (txmp *TxMempool) Lock() { | |||
txmp.mtx.Lock() | |||
} | |||
// EnableTxsAvailable initializes the TxsAvailable channel, ensuring it will | |||
// trigger once every height when transactions are available. | |||
EnableTxsAvailable() | |||
// Unlock releases a write-lock on the mempool. | |||
func (txmp *TxMempool) Unlock() { | |||
txmp.mtx.Unlock() | |||
} | |||
// Size returns the number of valid transactions in the mempool. It is | |||
// thread-safe. | |||
func (txmp *TxMempool) Size() int { | |||
return txmp.txStore.Size() | |||
} | |||
// SizeBytes return the total sum in bytes of all the valid transactions in the | |||
// mempool. It is thread-safe. | |||
func (txmp *TxMempool) SizeBytes() int64 { | |||
return atomic.LoadInt64(&txmp.sizeBytes) | |||
} | |||
// FlushAppConn executes FlushSync on the mempool's proxyAppConn. | |||
// | |||
// NOTE: The caller must obtain a write-lock prior to execution. | |||
func (txmp *TxMempool) FlushAppConn() error { | |||
return txmp.proxyAppConn.FlushSync(context.Background()) | |||
} | |||
// Size returns the number of transactions in the mempool. | |||
Size() int | |||
// WaitForNextTx returns a blocking channel that will be closed when the next | |||
// valid transaction is available to gossip. It is thread-safe. | |||
func (txmp *TxMempool) WaitForNextTx() <-chan struct{} { | |||
return txmp.gossipIndex.WaitChan() | |||
} | |||
// SizeBytes returns the total size of all txs in the mempool. | |||
SizeBytes() int64 | |||
// NextGossipTx returns the next valid transaction to gossip. A caller must wait | |||
// for WaitForNextTx to signal a transaction is available to gossip first. It is | |||
// thread-safe. | |||
func (txmp *TxMempool) NextGossipTx() *clist.CElement { | |||
return txmp.gossipIndex.Front() | |||
} | |||
// PreCheckFunc is an optional filter executed before CheckTx and rejects | |||
// transaction if false is returned. An example would be to ensure that a | |||
// transaction doesn't exceeded the block size. | |||
type PreCheckFunc func(types.Tx) error | |||
// EnableTxsAvailable enables the mempool to trigger events when transactions | |||
// are available on a block by block basis. | |||
func (txmp *TxMempool) EnableTxsAvailable() { | |||
txmp.mtx.Lock() | |||
defer txmp.mtx.Unlock() | |||
// PostCheckFunc is an optional filter executed after CheckTx and rejects | |||
// transaction if false is returned. An example would be to ensure a | |||
// transaction doesn't require more gas than available for the block. | |||
type PostCheckFunc func(types.Tx, *abci.ResponseCheckTx) error | |||
txmp.txsAvailable = make(chan struct{}, 1) | |||
} | |||
// PreCheckMaxBytes checks that the size of the transaction is smaller or equal | |||
// to the expected maxBytes. | |||
func PreCheckMaxBytes(maxBytes int64) PreCheckFunc { | |||
return func(tx types.Tx) error { | |||
txSize := types.ComputeProtoSizeForTxs([]types.Tx{tx}) | |||
// TxsAvailable returns a channel which fires once for every height, and only | |||
// when transactions are available in the mempool. It is thread-safe. | |||
func (txmp *TxMempool) TxsAvailable() <-chan struct{} { | |||
return txmp.txsAvailable | |||
} | |||
if txSize > maxBytes { | |||
return fmt.Errorf("tx size is too big: %d, max: %d", txSize, maxBytes) | |||
// CheckTx executes the ABCI CheckTx method for a given transaction. It acquires | |||
// a read-lock attempts to execute the application's CheckTx ABCI method via | |||
// CheckTxAsync. We return an error if any of the following happen: | |||
// | |||
// - The CheckTxAsync execution fails. | |||
// - The transaction already exists in the cache and we've already received the | |||
// transaction from the peer. Otherwise, if it solely exists in the cache, we | |||
// return nil. | |||
// - The transaction size exceeds the maximum transaction size as defined by the | |||
// configuration provided to the mempool. | |||
// - The transaction fails Pre-Check (if it is defined). | |||
// - The proxyAppConn fails, e.g. the buffer is full. | |||
// | |||
// If the mempool is full, we still execute CheckTx and attempt to find a lower | |||
// priority transaction to evict. If such a transaction exists, we remove the | |||
// lower priority transaction and add the new one with higher priority. | |||
// | |||
// NOTE: | |||
// - The applications' CheckTx implementation may panic. | |||
// - The caller is not to explicitly require any locks for executing CheckTx. | |||
func (txmp *TxMempool) CheckTx( | |||
ctx context.Context, | |||
tx types.Tx, | |||
cb func(*abci.Response), | |||
txInfo TxInfo, | |||
) error { | |||
if ctx == nil { | |||
ctx = context.TODO() | |||
} | |||
txmp.mtx.RLock() | |||
defer txmp.mtx.RUnlock() | |||
if txSize := len(tx); txSize > txmp.config.MaxTxBytes { | |||
return types.ErrTxTooLarge{ | |||
Max: txmp.config.MaxTxBytes, | |||
Actual: txSize, | |||
} | |||
} | |||
if txmp.preCheck != nil { | |||
if err := txmp.preCheck(tx); err != nil { | |||
return types.ErrPreCheck{Reason: err} | |||
} | |||
} | |||
if err := txmp.proxyAppConn.Error(); err != nil { | |||
return err | |||
} | |||
txHash := tx.Key() | |||
// We add the transaction to the mempool's cache and if the transaction already | |||
// exists, i.e. false is returned, then we check if we've seen this transaction | |||
// from the same sender and error if we have. Otherwise, we return nil. | |||
if !txmp.cache.Push(tx) { | |||
wtx, ok := txmp.txStore.GetOrSetPeerByTxHash(txHash, txInfo.SenderID) | |||
if wtx != nil && ok { | |||
// We already have the transaction stored and the we've already seen this | |||
// transaction from txInfo.SenderID. | |||
return types.ErrTxInCache | |||
} | |||
txmp.logger.Debug("tx exists already in cache", "tx_hash", tx.Hash()) | |||
return nil | |||
} | |||
} | |||
// PostCheckMaxGas checks that the wanted gas is smaller or equal to the passed | |||
// maxGas. Returns nil if maxGas is -1. | |||
func PostCheckMaxGas(maxGas int64) PostCheckFunc { | |||
return func(tx types.Tx, res *abci.ResponseCheckTx) error { | |||
if maxGas == -1 { | |||
return nil | |||
reqRes, err := txmp.proxyAppConn.CheckTxAsync(ctx, abci.RequestCheckTx{Tx: tx}) | |||
if err != nil { | |||
txmp.cache.Remove(tx) | |||
return err | |||
} | |||
reqRes.SetCallback(func(res *abci.Response) { | |||
if txmp.recheckCursor != nil { | |||
panic("recheck cursor is non-nil in CheckTx callback") | |||
} | |||
if res.GasWanted < 0 { | |||
return fmt.Errorf("gas wanted %d is negative", | |||
res.GasWanted) | |||
wtx := &WrappedTx{ | |||
tx: tx, | |||
hash: txHash, | |||
timestamp: time.Now().UTC(), | |||
height: txmp.height, | |||
} | |||
if res.GasWanted > maxGas { | |||
return fmt.Errorf("gas wanted %d is greater than max gas %d", | |||
res.GasWanted, maxGas) | |||
txmp.initTxCallback(wtx, res, txInfo) | |||
if cb != nil { | |||
cb(res) | |||
} | |||
}) | |||
return nil | |||
} | |||
func (txmp *TxMempool) RemoveTxByKey(txKey types.TxKey) error { | |||
txmp.Lock() | |||
defer txmp.Unlock() | |||
// remove the committed transaction from the transaction store and indexes | |||
if wtx := txmp.txStore.GetTxByHash(txKey); wtx != nil { | |||
txmp.removeTx(wtx, false) | |||
return nil | |||
} | |||
return errors.New("transaction not found") | |||
} | |||
// Flush empties the mempool. It acquires a read-lock, fetches all the | |||
// transactions currently in the transaction store and removes each transaction | |||
// from the store and all indexes and finally resets the cache. | |||
// | |||
// NOTE: | |||
// - Flushing the mempool may leave the mempool in an inconsistent state. | |||
func (txmp *TxMempool) Flush() { | |||
txmp.mtx.RLock() | |||
defer txmp.mtx.RUnlock() | |||
txmp.heightIndex.Reset() | |||
txmp.timestampIndex.Reset() | |||
for _, wtx := range txmp.txStore.GetAllTxs() { | |||
txmp.removeTx(wtx, false) | |||
} | |||
atomic.SwapInt64(&txmp.sizeBytes, 0) | |||
txmp.cache.Reset() | |||
} | |||
// ReapMaxBytesMaxGas returns a list of transactions within the provided size | |||
// and gas constraints. Transaction are retrieved in priority order. | |||
// | |||
// NOTE: | |||
// - Transactions returned are not removed from the mempool transaction | |||
// store or indexes. | |||
func (txmp *TxMempool) ReapMaxBytesMaxGas(maxBytes, maxGas int64) types.Txs { | |||
txmp.mtx.RLock() | |||
defer txmp.mtx.RUnlock() | |||
var ( | |||
totalGas int64 | |||
totalSize int64 | |||
) | |||
// wTxs contains a list of *WrappedTx retrieved from the priority queue that | |||
// need to be re-enqueued prior to returning. | |||
wTxs := make([]*WrappedTx, 0, txmp.priorityIndex.NumTxs()) | |||
defer func() { | |||
for _, wtx := range wTxs { | |||
txmp.priorityIndex.PushTx(wtx) | |||
} | |||
}() | |||
txs := make([]types.Tx, 0, txmp.priorityIndex.NumTxs()) | |||
for txmp.priorityIndex.NumTxs() > 0 { | |||
wtx := txmp.priorityIndex.PopTx() | |||
txs = append(txs, wtx.tx) | |||
wTxs = append(wTxs, wtx) | |||
size := types.ComputeProtoSizeForTxs([]types.Tx{wtx.tx}) | |||
// Ensure we have capacity for the transaction with respect to the | |||
// transaction size. | |||
if maxBytes > -1 && totalSize+size > maxBytes { | |||
return txs[:len(txs)-1] | |||
} | |||
totalSize += size | |||
// ensure we have capacity for the transaction with respect to total gas | |||
gas := totalGas + wtx.gasWanted | |||
if maxGas > -1 && gas > maxGas { | |||
return txs[:len(txs)-1] | |||
} | |||
totalGas = gas | |||
} | |||
return txs | |||
} | |||
// ReapMaxTxs returns a list of transactions within the provided number of | |||
// transactions bound. Transaction are retrieved in priority order. | |||
// | |||
// NOTE: | |||
// - Transactions returned are not removed from the mempool transaction | |||
// store or indexes. | |||
func (txmp *TxMempool) ReapMaxTxs(max int) types.Txs { | |||
txmp.mtx.RLock() | |||
defer txmp.mtx.RUnlock() | |||
numTxs := txmp.priorityIndex.NumTxs() | |||
if max < 0 { | |||
max = numTxs | |||
} | |||
cap := tmmath.MinInt(numTxs, max) | |||
// wTxs contains a list of *WrappedTx retrieved from the priority queue that | |||
// need to be re-enqueued prior to returning. | |||
wTxs := make([]*WrappedTx, 0, cap) | |||
txs := make([]types.Tx, 0, cap) | |||
for txmp.priorityIndex.NumTxs() > 0 && len(txs) < max { | |||
wtx := txmp.priorityIndex.PopTx() | |||
txs = append(txs, wtx.tx) | |||
wTxs = append(wTxs, wtx) | |||
} | |||
for _, wtx := range wTxs { | |||
txmp.priorityIndex.PushTx(wtx) | |||
} | |||
return txs | |||
} | |||
// Update iterates over all the transactions provided by the block producer, | |||
// removes them from the cache (if applicable), and removes | |||
// the transactions from the main transaction store and associated indexes. | |||
// If there are transactions remaining in the mempool, we initiate a | |||
// re-CheckTx for them (if applicable), otherwise, we notify the caller more | |||
// transactions are available. | |||
// | |||
// NOTE: | |||
// - The caller must explicitly acquire a write-lock. | |||
func (txmp *TxMempool) Update( | |||
blockHeight int64, | |||
blockTxs types.Txs, | |||
deliverTxResponses []*abci.ResponseDeliverTx, | |||
newPreFn PreCheckFunc, | |||
newPostFn PostCheckFunc, | |||
) error { | |||
txmp.height = blockHeight | |||
txmp.notifiedTxsAvailable = false | |||
if newPreFn != nil { | |||
txmp.preCheck = newPreFn | |||
} | |||
if newPostFn != nil { | |||
txmp.postCheck = newPostFn | |||
} | |||
for i, tx := range blockTxs { | |||
if deliverTxResponses[i].Code == abci.CodeTypeOK { | |||
// add the valid committed transaction to the cache (if missing) | |||
_ = txmp.cache.Push(tx) | |||
} else if !txmp.config.KeepInvalidTxsInCache { | |||
// allow invalid transactions to be re-submitted | |||
txmp.cache.Remove(tx) | |||
} | |||
// remove the committed transaction from the transaction store and indexes | |||
if wtx := txmp.txStore.GetTxByHash(tx.Key()); wtx != nil { | |||
txmp.removeTx(wtx, false) | |||
} | |||
} | |||
txmp.purgeExpiredTxs(blockHeight) | |||
// If there any uncommitted transactions left in the mempool, we either | |||
// initiate re-CheckTx per remaining transaction or notify that remaining | |||
// transactions are left. | |||
if txmp.Size() > 0 { | |||
if txmp.config.Recheck { | |||
txmp.logger.Debug( | |||
"executing re-CheckTx for all remaining transactions", | |||
"num_txs", txmp.Size(), | |||
"height", blockHeight, | |||
) | |||
txmp.updateReCheckTxs() | |||
} else { | |||
txmp.notifyTxsAvailable() | |||
} | |||
} | |||
txmp.metrics.Size.Set(float64(txmp.Size())) | |||
return nil | |||
} | |||
// initTxCallback is the callback invoked for a new unique transaction after CheckTx | |||
// has been executed by the ABCI application for the first time on that transaction. | |||
// CheckTx can be called again for the same transaction later when re-checking; | |||
// however, this callback will not be called. | |||
// | |||
// initTxCallback runs after the ABCI application executes CheckTx. | |||
// It runs the postCheck hook if one is defined on the mempool. | |||
// If the CheckTx response response code is not OK, or if the postCheck hook | |||
// reports an error, the transaction is rejected. Otherwise, we attempt to insert | |||
// the transaction into the mempool. | |||
// | |||
// When inserting a transaction, we first check if there is sufficient capacity. | |||
// If there is, the transaction is added to the txStore and all indexes. | |||
// Otherwise, if the mempool is full, we attempt to find a lower priority transaction | |||
// to evict in place of the new incoming transaction. If no such transaction exists, | |||
// the new incoming transaction is rejected. | |||
// | |||
// NOTE: | |||
// - An explicit lock is NOT required. | |||
func (txmp *TxMempool) initTxCallback(wtx *WrappedTx, res *abci.Response, txInfo TxInfo) { | |||
checkTxRes, ok := res.Value.(*abci.Response_CheckTx) | |||
if !ok { | |||
return | |||
} | |||
var err error | |||
if txmp.postCheck != nil { | |||
err = txmp.postCheck(wtx.tx, checkTxRes.CheckTx) | |||
} | |||
if err != nil || checkTxRes.CheckTx.Code != abci.CodeTypeOK { | |||
// ignore bad transactions | |||
txmp.logger.Info( | |||
"rejected bad transaction", | |||
"priority", wtx.priority, | |||
"tx", fmt.Sprintf("%X", wtx.tx.Hash()), | |||
"peer_id", txInfo.SenderNodeID, | |||
"code", checkTxRes.CheckTx.Code, | |||
"post_check_err", err, | |||
) | |||
txmp.metrics.FailedTxs.Add(1) | |||
if !txmp.config.KeepInvalidTxsInCache { | |||
txmp.cache.Remove(wtx.tx) | |||
} | |||
if err != nil { | |||
checkTxRes.CheckTx.MempoolError = err.Error() | |||
} | |||
return | |||
} | |||
sender := checkTxRes.CheckTx.Sender | |||
priority := checkTxRes.CheckTx.Priority | |||
if len(sender) > 0 { | |||
if wtx := txmp.txStore.GetTxBySender(sender); wtx != nil { | |||
txmp.logger.Error( | |||
"rejected incoming good transaction; tx already exists for sender", | |||
"tx", fmt.Sprintf("%X", wtx.tx.Hash()), | |||
"sender", sender, | |||
) | |||
txmp.metrics.RejectedTxs.Add(1) | |||
return | |||
} | |||
} | |||
if err := txmp.canAddTx(wtx); err != nil { | |||
evictTxs := txmp.priorityIndex.GetEvictableTxs( | |||
priority, | |||
int64(wtx.Size()), | |||
txmp.SizeBytes(), | |||
txmp.config.MaxTxsBytes, | |||
) | |||
if len(evictTxs) == 0 { | |||
// No room for the new incoming transaction so we just remove it from | |||
// the cache. | |||
txmp.cache.Remove(wtx.tx) | |||
txmp.logger.Error( | |||
"rejected incoming good transaction; mempool full", | |||
"tx", fmt.Sprintf("%X", wtx.tx.Hash()), | |||
"err", err.Error(), | |||
) | |||
txmp.metrics.RejectedTxs.Add(1) | |||
return | |||
} | |||
// evict an existing transaction(s) | |||
// | |||
// NOTE: | |||
// - The transaction, toEvict, can be removed while a concurrent | |||
// reCheckTx callback is being executed for the same transaction. | |||
for _, toEvict := range evictTxs { | |||
txmp.removeTx(toEvict, true) | |||
txmp.logger.Debug( | |||
"evicted existing good transaction; mempool full", | |||
"old_tx", fmt.Sprintf("%X", toEvict.tx.Hash()), | |||
"old_priority", toEvict.priority, | |||
"new_tx", fmt.Sprintf("%X", wtx.tx.Hash()), | |||
"new_priority", wtx.priority, | |||
) | |||
txmp.metrics.EvictedTxs.Add(1) | |||
} | |||
} | |||
wtx.gasWanted = checkTxRes.CheckTx.GasWanted | |||
wtx.priority = priority | |||
wtx.sender = sender | |||
wtx.peers = map[uint16]struct{}{ | |||
txInfo.SenderID: {}, | |||
} | |||
txmp.metrics.TxSizeBytes.Observe(float64(wtx.Size())) | |||
txmp.metrics.Size.Set(float64(txmp.Size())) | |||
txmp.insertTx(wtx) | |||
txmp.logger.Debug( | |||
"inserted good transaction", | |||
"priority", wtx.priority, | |||
"tx", fmt.Sprintf("%X", wtx.tx.Hash()), | |||
"height", txmp.height, | |||
"num_txs", txmp.Size(), | |||
) | |||
txmp.notifyTxsAvailable() | |||
} | |||
// defaultTxCallback is the CheckTx application callback used when a transaction | |||
// is being re-checked (if re-checking is enabled). The caller must hold a mempool | |||
// write-lock (via Lock()) and when executing Update(), if the mempool is non-empty | |||
// and Recheck is enabled, then all remaining transactions will be rechecked via | |||
// CheckTxAsync. The order transactions are rechecked must be the same as the | |||
// order in which this callback is called. | |||
func (txmp *TxMempool) defaultTxCallback(req *abci.Request, res *abci.Response) { | |||
if txmp.recheckCursor == nil { | |||
return | |||
} | |||
txmp.metrics.RecheckTimes.Add(1) | |||
checkTxRes, ok := res.Value.(*abci.Response_CheckTx) | |||
if !ok { | |||
txmp.logger.Error("received incorrect type in mempool callback", | |||
"expected", reflect.TypeOf(&abci.Response_CheckTx{}).Name(), | |||
"got", reflect.TypeOf(res.Value).Name(), | |||
) | |||
return | |||
} | |||
tx := req.GetCheckTx().Tx | |||
wtx := txmp.recheckCursor.Value.(*WrappedTx) | |||
// Search through the remaining list of tx to recheck for a transaction that matches | |||
// the one we received from the ABCI application. | |||
for { | |||
if bytes.Equal(tx, wtx.tx) { | |||
// We've found a tx in the recheck list that matches the tx that we | |||
// received from the ABCI application. | |||
// Break, and use this transaction for further checks. | |||
break | |||
} | |||
txmp.logger.Error( | |||
"re-CheckTx transaction mismatch", | |||
"got", wtx.tx.Hash(), | |||
"expected", types.Tx(tx).Key(), | |||
) | |||
if txmp.recheckCursor == txmp.recheckEnd { | |||
// we reached the end of the recheckTx list without finding a tx | |||
// matching the one we received from the ABCI application. | |||
// Return without processing any tx. | |||
txmp.recheckCursor = nil | |||
return | |||
} | |||
txmp.recheckCursor = txmp.recheckCursor.Next() | |||
wtx = txmp.recheckCursor.Value.(*WrappedTx) | |||
} | |||
// Only evaluate transactions that have not been removed. This can happen | |||
// if an existing transaction is evicted during CheckTx and while this | |||
// callback is being executed for the same evicted transaction. | |||
if !txmp.txStore.IsTxRemoved(wtx.hash) { | |||
var err error | |||
if txmp.postCheck != nil { | |||
err = txmp.postCheck(tx, checkTxRes.CheckTx) | |||
} | |||
if checkTxRes.CheckTx.Code == abci.CodeTypeOK && err == nil { | |||
wtx.priority = checkTxRes.CheckTx.Priority | |||
} else { | |||
txmp.logger.Debug( | |||
"existing transaction no longer valid; failed re-CheckTx callback", | |||
"priority", wtx.priority, | |||
"tx", fmt.Sprintf("%X", wtx.tx.Hash()), | |||
"err", err, | |||
"code", checkTxRes.CheckTx.Code, | |||
) | |||
if wtx.gossipEl != txmp.recheckCursor { | |||
panic("corrupted reCheckTx cursor") | |||
} | |||
txmp.removeTx(wtx, !txmp.config.KeepInvalidTxsInCache) | |||
} | |||
} | |||
// move reCheckTx cursor to next element | |||
if txmp.recheckCursor == txmp.recheckEnd { | |||
txmp.recheckCursor = nil | |||
} else { | |||
txmp.recheckCursor = txmp.recheckCursor.Next() | |||
} | |||
if txmp.recheckCursor == nil { | |||
txmp.logger.Debug("finished rechecking transactions") | |||
if txmp.Size() > 0 { | |||
txmp.notifyTxsAvailable() | |||
} | |||
} | |||
txmp.metrics.Size.Set(float64(txmp.Size())) | |||
} | |||
// updateReCheckTxs updates the recheck cursors using the gossipIndex. For | |||
// each transaction, it executes CheckTxAsync. The global callback defined on | |||
// the proxyAppConn will be executed for each transaction after CheckTx is | |||
// executed. | |||
// | |||
// NOTE: | |||
// - The caller must have a write-lock when executing updateReCheckTxs. | |||
func (txmp *TxMempool) updateReCheckTxs() { | |||
if txmp.Size() == 0 { | |||
panic("attempted to update re-CheckTx txs when mempool is empty") | |||
} | |||
txmp.recheckCursor = txmp.gossipIndex.Front() | |||
txmp.recheckEnd = txmp.gossipIndex.Back() | |||
ctx := context.Background() | |||
for e := txmp.gossipIndex.Front(); e != nil; e = e.Next() { | |||
wtx := e.Value.(*WrappedTx) | |||
// Only execute CheckTx if the transaction is not marked as removed which | |||
// could happen if the transaction was evicted. | |||
if !txmp.txStore.IsTxRemoved(wtx.hash) { | |||
_, err := txmp.proxyAppConn.CheckTxAsync(ctx, abci.RequestCheckTx{ | |||
Tx: wtx.tx, | |||
Type: abci.CheckTxType_Recheck, | |||
}) | |||
if err != nil { | |||
// no need in retrying since the tx will be rechecked after the next block | |||
txmp.logger.Error("failed to execute CheckTx during rechecking", "err", err) | |||
} | |||
} | |||
} | |||
if _, err := txmp.proxyAppConn.FlushAsync(ctx); err != nil { | |||
txmp.logger.Error("failed to flush transactions during rechecking", "err", err) | |||
} | |||
} | |||
// canAddTx returns an error if we cannot insert the provided *WrappedTx into | |||
// the mempool due to mempool configured constraints. If it returns nil, | |||
// the transaction can be inserted into the mempool. | |||
func (txmp *TxMempool) canAddTx(wtx *WrappedTx) error { | |||
var ( | |||
numTxs = txmp.Size() | |||
sizeBytes = txmp.SizeBytes() | |||
) | |||
if numTxs >= txmp.config.Size || int64(wtx.Size())+sizeBytes > txmp.config.MaxTxsBytes { | |||
return types.ErrMempoolIsFull{ | |||
NumTxs: numTxs, | |||
MaxTxs: txmp.config.Size, | |||
TxsBytes: sizeBytes, | |||
MaxTxsBytes: txmp.config.MaxTxsBytes, | |||
} | |||
} | |||
return nil | |||
} | |||
func (txmp *TxMempool) insertTx(wtx *WrappedTx) { | |||
txmp.txStore.SetTx(wtx) | |||
txmp.priorityIndex.PushTx(wtx) | |||
txmp.heightIndex.Insert(wtx) | |||
txmp.timestampIndex.Insert(wtx) | |||
// Insert the transaction into the gossip index and mark the reference to the | |||
// linked-list element, which will be needed at a later point when the | |||
// transaction is removed. | |||
gossipEl := txmp.gossipIndex.PushBack(wtx) | |||
wtx.gossipEl = gossipEl | |||
atomic.AddInt64(&txmp.sizeBytes, int64(wtx.Size())) | |||
} | |||
func (txmp *TxMempool) removeTx(wtx *WrappedTx, removeFromCache bool) { | |||
if txmp.txStore.IsTxRemoved(wtx.hash) { | |||
return | |||
} | |||
txmp.txStore.RemoveTx(wtx) | |||
txmp.priorityIndex.RemoveTx(wtx) | |||
txmp.heightIndex.Remove(wtx) | |||
txmp.timestampIndex.Remove(wtx) | |||
// Remove the transaction from the gossip index and cleanup the linked-list | |||
// element so it can be garbage collected. | |||
txmp.gossipIndex.Remove(wtx.gossipEl) | |||
wtx.gossipEl.DetachPrev() | |||
atomic.AddInt64(&txmp.sizeBytes, int64(-wtx.Size())) | |||
if removeFromCache { | |||
txmp.cache.Remove(wtx.tx) | |||
} | |||
} | |||
// purgeExpiredTxs removes all transactions that have exceeded their respective | |||
// height- and/or time-based TTLs from their respective indexes. Every expired | |||
// transaction will be removed from the mempool, but preserved in the cache. | |||
// | |||
// NOTE: purgeExpiredTxs must only be called during TxMempool#Update in which | |||
// the caller has a write-lock on the mempool and so we can safely iterate over | |||
// the height and time based indexes. | |||
func (txmp *TxMempool) purgeExpiredTxs(blockHeight int64) { | |||
now := time.Now() | |||
expiredTxs := make(map[types.TxKey]*WrappedTx) | |||
if txmp.config.TTLNumBlocks > 0 { | |||
purgeIdx := -1 | |||
for i, wtx := range txmp.heightIndex.txs { | |||
if (blockHeight - wtx.height) > txmp.config.TTLNumBlocks { | |||
expiredTxs[wtx.tx.Key()] = wtx | |||
purgeIdx = i | |||
} else { | |||
// since the index is sorted, we know no other txs can be be purged | |||
break | |||
} | |||
} | |||
if purgeIdx >= 0 { | |||
txmp.heightIndex.txs = txmp.heightIndex.txs[purgeIdx+1:] | |||
} | |||
} | |||
if txmp.config.TTLDuration > 0 { | |||
purgeIdx := -1 | |||
for i, wtx := range txmp.timestampIndex.txs { | |||
if now.Sub(wtx.timestamp) > txmp.config.TTLDuration { | |||
expiredTxs[wtx.tx.Key()] = wtx | |||
purgeIdx = i | |||
} else { | |||
// since the index is sorted, we know no other txs can be be purged | |||
break | |||
} | |||
} | |||
if purgeIdx >= 0 { | |||
txmp.timestampIndex.txs = txmp.timestampIndex.txs[purgeIdx+1:] | |||
} | |||
} | |||
for _, wtx := range expiredTxs { | |||
txmp.removeTx(wtx, false) | |||
} | |||
} | |||
func (txmp *TxMempool) notifyTxsAvailable() { | |||
if txmp.Size() == 0 { | |||
panic("attempt to notify txs available but mempool is empty!") | |||
} | |||
if txmp.txsAvailable != nil && !txmp.notifiedTxsAvailable { | |||
// channel cap is 1, so this will send once | |||
txmp.notifiedTxsAvailable = true | |||
select { | |||
case txmp.txsAvailable <- struct{}{}: | |||
default: | |||
} | |||
} | |||
} |
@ -0,0 +1,143 @@ | |||
package mempool | |||
import ( | |||
"context" | |||
"fmt" | |||
"math" | |||
abci "github.com/tendermint/tendermint/abci/types" | |||
"github.com/tendermint/tendermint/internal/p2p" | |||
"github.com/tendermint/tendermint/types" | |||
) | |||
const ( | |||
MempoolChannel = p2p.ChannelID(0x30) | |||
// PeerCatchupSleepIntervalMS defines how much time to sleep if a peer is behind | |||
PeerCatchupSleepIntervalMS = 100 | |||
// UnknownPeerID is the peer ID to use when running CheckTx when there is | |||
// no peer (e.g. RPC) | |||
UnknownPeerID uint16 = 0 | |||
MaxActiveIDs = math.MaxUint16 | |||
) | |||
// Mempool defines the mempool interface. | |||
// | |||
// Updates to the mempool need to be synchronized with committing a block so | |||
// applications can reset their transient state on Commit. | |||
type Mempool interface { | |||
// CheckTx executes a new transaction against the application to determine | |||
// its validity and whether it should be added to the mempool. | |||
CheckTx(ctx context.Context, tx types.Tx, callback func(*abci.Response), txInfo TxInfo) error | |||
// RemoveTxByKey removes a transaction, identified by its key, | |||
// from the mempool. | |||
RemoveTxByKey(txKey types.TxKey) error | |||
// ReapMaxBytesMaxGas reaps transactions from the mempool up to maxBytes | |||
// bytes total with the condition that the total gasWanted must be less than | |||
// maxGas. | |||
// | |||
// If both maxes are negative, there is no cap on the size of all returned | |||
// transactions (~ all available transactions). | |||
ReapMaxBytesMaxGas(maxBytes, maxGas int64) types.Txs | |||
// ReapMaxTxs reaps up to max transactions from the mempool. If max is | |||
// negative, there is no cap on the size of all returned transactions | |||
// (~ all available transactions). | |||
ReapMaxTxs(max int) types.Txs | |||
// Lock locks the mempool. The consensus must be able to hold lock to safely | |||
// update. | |||
Lock() | |||
// Unlock unlocks the mempool. | |||
Unlock() | |||
// Update informs the mempool that the given txs were committed and can be | |||
// discarded. | |||
// | |||
// NOTE: | |||
// 1. This should be called *after* block is committed by consensus. | |||
// 2. Lock/Unlock must be managed by the caller. | |||
Update( | |||
blockHeight int64, | |||
blockTxs types.Txs, | |||
deliverTxResponses []*abci.ResponseDeliverTx, | |||
newPreFn PreCheckFunc, | |||
newPostFn PostCheckFunc, | |||
) error | |||
// FlushAppConn flushes the mempool connection to ensure async callback calls | |||
// are done, e.g. from CheckTx. | |||
// | |||
// NOTE: | |||
// 1. Lock/Unlock must be managed by caller. | |||
FlushAppConn() error | |||
// Flush removes all transactions from the mempool and caches. | |||
Flush() | |||
// TxsAvailable returns a channel which fires once for every height, and only | |||
// when transactions are available in the mempool. | |||
// | |||
// NOTE: | |||
// 1. The returned channel may be nil if EnableTxsAvailable was not called. | |||
TxsAvailable() <-chan struct{} | |||
// EnableTxsAvailable initializes the TxsAvailable channel, ensuring it will | |||
// trigger once every height when transactions are available. | |||
EnableTxsAvailable() | |||
// Size returns the number of transactions in the mempool. | |||
Size() int | |||
// SizeBytes returns the total size of all txs in the mempool. | |||
SizeBytes() int64 | |||
} | |||
// PreCheckFunc is an optional filter executed before CheckTx and rejects | |||
// transaction if false is returned. An example would be to ensure that a | |||
// transaction doesn't exceeded the block size. | |||
type PreCheckFunc func(types.Tx) error | |||
// PostCheckFunc is an optional filter executed after CheckTx and rejects | |||
// transaction if false is returned. An example would be to ensure a | |||
// transaction doesn't require more gas than available for the block. | |||
type PostCheckFunc func(types.Tx, *abci.ResponseCheckTx) error | |||
// PreCheckMaxBytes checks that the size of the transaction is smaller or equal | |||
// to the expected maxBytes. | |||
func PreCheckMaxBytes(maxBytes int64) PreCheckFunc { | |||
return func(tx types.Tx) error { | |||
txSize := types.ComputeProtoSizeForTxs([]types.Tx{tx}) | |||
if txSize > maxBytes { | |||
return fmt.Errorf("tx size is too big: %d, max: %d", txSize, maxBytes) | |||
} | |||
return nil | |||
} | |||
} | |||
// PostCheckMaxGas checks that the wanted gas is smaller or equal to the passed | |||
// maxGas. Returns nil if maxGas is -1. | |||
func PostCheckMaxGas(maxGas int64) PostCheckFunc { | |||
return func(tx types.Tx, res *abci.ResponseCheckTx) error { | |||
if maxGas == -1 { | |||
return nil | |||
} | |||
if res.GasWanted < 0 { | |||
return fmt.Errorf("gas wanted %d is negative", | |||
res.GasWanted) | |||
} | |||
if res.GasWanted > maxGas { | |||
return fmt.Errorf("gas wanted %d is greater than max gas %d", | |||
res.GasWanted, maxGas) | |||
} | |||
return nil | |||
} | |||
} |
@ -1,107 +0,0 @@ | |||
package v0 | |||
import ( | |||
"context" | |||
"encoding/binary" | |||
"sync/atomic" | |||
"testing" | |||
"github.com/stretchr/testify/require" | |||
abciclient "github.com/tendermint/tendermint/abci/client" | |||
"github.com/tendermint/tendermint/abci/example/kvstore" | |||
"github.com/tendermint/tendermint/internal/mempool" | |||
) | |||
func BenchmarkReap(b *testing.B) { | |||
app := kvstore.NewApplication() | |||
cc := abciclient.NewLocalCreator(app) | |||
mp, cleanup, err := newMempoolWithApp(cc) | |||
require.NoError(b, err) | |||
defer cleanup() | |||
mp.config.Size = 100000 | |||
size := 10000 | |||
for i := 0; i < size; i++ { | |||
tx := make([]byte, 8) | |||
binary.BigEndian.PutUint64(tx, uint64(i)) | |||
if err := mp.CheckTx(context.Background(), tx, nil, mempool.TxInfo{}); err != nil { | |||
b.Fatal(err) | |||
} | |||
} | |||
b.ResetTimer() | |||
for i := 0; i < b.N; i++ { | |||
mp.ReapMaxBytesMaxGas(100000000, 10000000) | |||
} | |||
} | |||
func BenchmarkCheckTx(b *testing.B) { | |||
app := kvstore.NewApplication() | |||
cc := abciclient.NewLocalCreator(app) | |||
mp, cleanup, err := newMempoolWithApp(cc) | |||
require.NoError(b, err) | |||
defer cleanup() | |||
mp.config.Size = 1000000 | |||
b.ResetTimer() | |||
for i := 0; i < b.N; i++ { | |||
b.StopTimer() | |||
tx := make([]byte, 8) | |||
binary.BigEndian.PutUint64(tx, uint64(i)) | |||
b.StartTimer() | |||
if err := mp.CheckTx(context.Background(), tx, nil, mempool.TxInfo{}); err != nil { | |||
b.Fatal(err) | |||
} | |||
} | |||
} | |||
func BenchmarkParallelCheckTx(b *testing.B) { | |||
app := kvstore.NewApplication() | |||
cc := abciclient.NewLocalCreator(app) | |||
mp, cleanup, err := newMempoolWithApp(cc) | |||
require.NoError(b, err) | |||
defer cleanup() | |||
mp.config.Size = 100000000 | |||
var txcnt uint64 | |||
next := func() uint64 { | |||
return atomic.AddUint64(&txcnt, 1) - 1 | |||
} | |||
b.ResetTimer() | |||
b.RunParallel(func(pb *testing.PB) { | |||
for pb.Next() { | |||
tx := make([]byte, 8) | |||
binary.BigEndian.PutUint64(tx, next()) | |||
if err := mp.CheckTx(context.Background(), tx, nil, mempool.TxInfo{}); err != nil { | |||
b.Fatal(err) | |||
} | |||
} | |||
}) | |||
} | |||
func BenchmarkCheckDuplicateTx(b *testing.B) { | |||
app := kvstore.NewApplication() | |||
cc := abciclient.NewLocalCreator(app) | |||
mp, cleanup, err := newMempoolWithApp(cc) | |||
require.NoError(b, err) | |||
defer cleanup() | |||
mp.config.Size = 1000000 | |||
for i := 0; i < b.N; i++ { | |||
tx := make([]byte, 8) | |||
binary.BigEndian.PutUint64(tx, uint64(i)) | |||
if err := mp.CheckTx(context.Background(), tx, nil, mempool.TxInfo{}); err != nil { | |||
b.Fatal(err) | |||
} | |||
if err := mp.CheckTx(context.Background(), tx, nil, mempool.TxInfo{}); err == nil { | |||
b.Fatal("tx should be duplicate") | |||
} | |||
} | |||
} |
@ -1,83 +0,0 @@ | |||
package v0 | |||
import ( | |||
"context" | |||
"crypto/sha256" | |||
"testing" | |||
"github.com/stretchr/testify/require" | |||
abciclient "github.com/tendermint/tendermint/abci/client" | |||
"github.com/tendermint/tendermint/abci/example/kvstore" | |||
abci "github.com/tendermint/tendermint/abci/types" | |||
"github.com/tendermint/tendermint/internal/mempool" | |||
"github.com/tendermint/tendermint/types" | |||
) | |||
func TestCacheAfterUpdate(t *testing.T) { | |||
app := kvstore.NewApplication() | |||
cc := abciclient.NewLocalCreator(app) | |||
mp, cleanup, err := newMempoolWithApp(cc) | |||
require.NoError(t, err) | |||
defer cleanup() | |||
// reAddIndices & txsInCache can have elements > numTxsToCreate | |||
// also assumes max index is 255 for convenience | |||
// txs in cache also checks order of elements | |||
tests := []struct { | |||
numTxsToCreate int | |||
updateIndices []int | |||
reAddIndices []int | |||
txsInCache []int | |||
}{ | |||
{1, []int{}, []int{1}, []int{1, 0}}, // adding new txs works | |||
{2, []int{1}, []int{}, []int{1, 0}}, // update doesn't remove tx from cache | |||
{2, []int{2}, []int{}, []int{2, 1, 0}}, // update adds new tx to cache | |||
{2, []int{1}, []int{1}, []int{1, 0}}, // re-adding after update doesn't make dupe | |||
} | |||
for tcIndex, tc := range tests { | |||
for i := 0; i < tc.numTxsToCreate; i++ { | |||
tx := types.Tx{byte(i)} | |||
err := mp.CheckTx(context.Background(), tx, nil, mempool.TxInfo{}) | |||
require.NoError(t, err) | |||
} | |||
updateTxs := []types.Tx{} | |||
for _, v := range tc.updateIndices { | |||
tx := types.Tx{byte(v)} | |||
updateTxs = append(updateTxs, tx) | |||
} | |||
err := mp.Update(int64(tcIndex), updateTxs, abciResponses(len(updateTxs), abci.CodeTypeOK), nil, nil) | |||
require.NoError(t, err) | |||
for _, v := range tc.reAddIndices { | |||
tx := types.Tx{byte(v)} | |||
_ = mp.CheckTx(context.Background(), tx, nil, mempool.TxInfo{}) | |||
} | |||
cache := mp.cache.(*mempool.LRUTxCache) | |||
node := cache.GetList().Front() | |||
counter := 0 | |||
for node != nil { | |||
require.NotEqual(t, len(tc.txsInCache), counter, | |||
"cache larger than expected on testcase %d", tcIndex) | |||
nodeVal := node.Value.(types.TxKey) | |||
expectedBz := sha256.Sum256([]byte{byte(tc.txsInCache[len(tc.txsInCache)-counter-1])}) | |||
// Reference for reading the errors: | |||
// >>> sha256('\x00').hexdigest() | |||
// '6e340b9cffb37a989ca544e6bb780a2c78901d3fb33738768511a30617afa01d' | |||
// >>> sha256('\x01').hexdigest() | |||
// '4bf5122f344554c53bde2ebb8cd2b7e3d1600ad631c385a5d7cce23c7785459a' | |||
// >>> sha256('\x02').hexdigest() | |||
// 'dbc1b4c900ffe48d575b5da5c638040125f65db0fe3e24494b76ea986457d986' | |||
require.EqualValues(t, expectedBz, nodeVal, "Equality failed on index %d, tc %d", counter, tcIndex) | |||
counter++ | |||
node = node.Next() | |||
} | |||
require.Equal(t, len(tc.txsInCache), counter, | |||
"cache smaller than expected on testcase %d", tcIndex) | |||
mp.Flush() | |||
} | |||
} |
@ -1,698 +0,0 @@ | |||
package v0 | |||
import ( | |||
"bytes" | |||
"context" | |||
"errors" | |||
"sync" | |||
"sync/atomic" | |||
abci "github.com/tendermint/tendermint/abci/types" | |||
"github.com/tendermint/tendermint/config" | |||
"github.com/tendermint/tendermint/internal/libs/clist" | |||
tmsync "github.com/tendermint/tendermint/internal/libs/sync" | |||
"github.com/tendermint/tendermint/internal/mempool" | |||
"github.com/tendermint/tendermint/internal/proxy" | |||
"github.com/tendermint/tendermint/libs/log" | |||
tmmath "github.com/tendermint/tendermint/libs/math" | |||
"github.com/tendermint/tendermint/types" | |||
) | |||
// CListMempool is an ordered in-memory pool for transactions before they are | |||
// proposed in a consensus round. Transaction validity is checked using the | |||
// CheckTx abci message before the transaction is added to the pool. The | |||
// mempool uses a concurrent list structure for storing transactions that can | |||
// be efficiently accessed by multiple concurrent readers. | |||
type CListMempool struct { | |||
// Atomic integers | |||
height int64 // the last block Update()'d to | |||
txsBytes int64 // total size of mempool, in bytes | |||
// notify listeners (ie. consensus) when txs are available | |||
notifiedTxsAvailable bool | |||
txsAvailable chan struct{} // fires once for each height, when the mempool is not empty | |||
config *config.MempoolConfig | |||
// Exclusive mutex for Update method to prevent concurrent execution of | |||
// CheckTx or ReapMaxBytesMaxGas(ReapMaxTxs) methods. | |||
updateMtx tmsync.RWMutex | |||
preCheck mempool.PreCheckFunc | |||
postCheck mempool.PostCheckFunc | |||
txs *clist.CList // concurrent linked-list of good txs | |||
proxyAppConn proxy.AppConnMempool | |||
// Track whether we're rechecking txs. | |||
// These are not protected by a mutex and are expected to be mutated in | |||
// serial (ie. by abci responses which are called in serial). | |||
recheckCursor *clist.CElement // next expected response | |||
recheckEnd *clist.CElement // re-checking stops here | |||
// Map for quick access to txs to record sender in CheckTx. | |||
// txsMap: txKey -> CElement | |||
txsMap sync.Map | |||
// Keep a cache of already-seen txs. | |||
// This reduces the pressure on the proxyApp. | |||
cache mempool.TxCache | |||
logger log.Logger | |||
metrics *mempool.Metrics | |||
} | |||
var _ mempool.Mempool = &CListMempool{} | |||
// CListMempoolOption sets an optional parameter on the mempool. | |||
type CListMempoolOption func(*CListMempool) | |||
// NewCListMempool returns a new mempool with the given configuration and | |||
// connection to an application. | |||
func NewCListMempool( | |||
cfg *config.MempoolConfig, | |||
proxyAppConn proxy.AppConnMempool, | |||
height int64, | |||
options ...CListMempoolOption, | |||
) *CListMempool { | |||
mp := &CListMempool{ | |||
config: cfg, | |||
proxyAppConn: proxyAppConn, | |||
txs: clist.New(), | |||
height: height, | |||
recheckCursor: nil, | |||
recheckEnd: nil, | |||
logger: log.NewNopLogger(), | |||
metrics: mempool.NopMetrics(), | |||
} | |||
if cfg.CacheSize > 0 { | |||
mp.cache = mempool.NewLRUTxCache(cfg.CacheSize) | |||
} else { | |||
mp.cache = mempool.NopTxCache{} | |||
} | |||
proxyAppConn.SetResponseCallback(mp.globalCb) | |||
for _, option := range options { | |||
option(mp) | |||
} | |||
return mp | |||
} | |||
// NOTE: not thread safe - should only be called once, on startup | |||
func (mem *CListMempool) EnableTxsAvailable() { | |||
mem.txsAvailable = make(chan struct{}, 1) | |||
} | |||
// SetLogger sets the Logger. | |||
func (mem *CListMempool) SetLogger(l log.Logger) { | |||
mem.logger = l | |||
} | |||
// WithPreCheck sets a filter for the mempool to reject a tx if f(tx) returns | |||
// false. This is ran before CheckTx. Only applies to the first created block. | |||
// After that, Update overwrites the existing value. | |||
func WithPreCheck(f mempool.PreCheckFunc) CListMempoolOption { | |||
return func(mem *CListMempool) { mem.preCheck = f } | |||
} | |||
// WithPostCheck sets a filter for the mempool to reject a tx if f(tx) returns | |||
// false. This is ran after CheckTx. Only applies to the first created block. | |||
// After that, Update overwrites the existing value. | |||
func WithPostCheck(f mempool.PostCheckFunc) CListMempoolOption { | |||
return func(mem *CListMempool) { mem.postCheck = f } | |||
} | |||
// WithMetrics sets the metrics. | |||
func WithMetrics(metrics *mempool.Metrics) CListMempoolOption { | |||
return func(mem *CListMempool) { mem.metrics = metrics } | |||
} | |||
// Safe for concurrent use by multiple goroutines. | |||
func (mem *CListMempool) Lock() { | |||
mem.updateMtx.Lock() | |||
} | |||
// Safe for concurrent use by multiple goroutines. | |||
func (mem *CListMempool) Unlock() { | |||
mem.updateMtx.Unlock() | |||
} | |||
// Safe for concurrent use by multiple goroutines. | |||
func (mem *CListMempool) Size() int { | |||
return mem.txs.Len() | |||
} | |||
// Safe for concurrent use by multiple goroutines. | |||
func (mem *CListMempool) SizeBytes() int64 { | |||
return atomic.LoadInt64(&mem.txsBytes) | |||
} | |||
// Lock() must be help by the caller during execution. | |||
func (mem *CListMempool) FlushAppConn() error { | |||
return mem.proxyAppConn.FlushSync(context.Background()) | |||
} | |||
// XXX: Unsafe! Calling Flush may leave mempool in inconsistent state. | |||
func (mem *CListMempool) Flush() { | |||
mem.updateMtx.RLock() | |||
defer mem.updateMtx.RUnlock() | |||
_ = atomic.SwapInt64(&mem.txsBytes, 0) | |||
mem.cache.Reset() | |||
for e := mem.txs.Front(); e != nil; e = e.Next() { | |||
mem.txs.Remove(e) | |||
e.DetachPrev() | |||
} | |||
mem.txsMap.Range(func(key, _ interface{}) bool { | |||
mem.txsMap.Delete(key) | |||
return true | |||
}) | |||
} | |||
// TxsFront returns the first transaction in the ordered list for peer | |||
// goroutines to call .NextWait() on. | |||
// FIXME: leaking implementation details! | |||
// | |||
// Safe for concurrent use by multiple goroutines. | |||
func (mem *CListMempool) TxsFront() *clist.CElement { | |||
return mem.txs.Front() | |||
} | |||
// TxsWaitChan returns a channel to wait on transactions. It will be closed | |||
// once the mempool is not empty (ie. the internal `mem.txs` has at least one | |||
// element) | |||
// | |||
// Safe for concurrent use by multiple goroutines. | |||
func (mem *CListMempool) TxsWaitChan() <-chan struct{} { | |||
return mem.txs.WaitChan() | |||
} | |||
// It blocks if we're waiting on Update() or Reap(). | |||
// cb: A callback from the CheckTx command. | |||
// It gets called from another goroutine. | |||
// CONTRACT: Either cb will get called, or err returned. | |||
// | |||
// Safe for concurrent use by multiple goroutines. | |||
func (mem *CListMempool) CheckTx( | |||
ctx context.Context, | |||
tx types.Tx, | |||
cb func(*abci.Response), | |||
txInfo mempool.TxInfo, | |||
) error { | |||
mem.updateMtx.RLock() | |||
// use defer to unlock mutex because application (*local client*) might panic | |||
defer mem.updateMtx.RUnlock() | |||
txSize := len(tx) | |||
if err := mem.isFull(txSize); err != nil { | |||
return err | |||
} | |||
if txSize > mem.config.MaxTxBytes { | |||
return types.ErrTxTooLarge{ | |||
Max: mem.config.MaxTxBytes, | |||
Actual: txSize, | |||
} | |||
} | |||
if mem.preCheck != nil { | |||
if err := mem.preCheck(tx); err != nil { | |||
return types.ErrPreCheck{ | |||
Reason: err, | |||
} | |||
} | |||
} | |||
// NOTE: proxyAppConn may error if tx buffer is full | |||
if err := mem.proxyAppConn.Error(); err != nil { | |||
return err | |||
} | |||
if !mem.cache.Push(tx) { // if the transaction already exists in the cache | |||
// Record a new sender for a tx we've already seen. | |||
// Note it's possible a tx is still in the cache but no longer in the mempool | |||
// (eg. after committing a block, txs are removed from mempool but not cache), | |||
// so we only record the sender for txs still in the mempool. | |||
if e, ok := mem.txsMap.Load(tx.Key()); ok { | |||
memTx := e.(*clist.CElement).Value.(*mempoolTx) | |||
_, loaded := memTx.senders.LoadOrStore(txInfo.SenderID, true) | |||
// TODO: consider punishing peer for dups, | |||
// its non-trivial since invalid txs can become valid, | |||
// but they can spam the same tx with little cost to them atm. | |||
if loaded { | |||
return types.ErrTxInCache | |||
} | |||
} | |||
mem.logger.Debug("tx exists already in cache", "tx_hash", tx.Hash()) | |||
return nil | |||
} | |||
if ctx == nil { | |||
ctx = context.Background() | |||
} | |||
reqRes, err := mem.proxyAppConn.CheckTxAsync(ctx, abci.RequestCheckTx{Tx: tx}) | |||
if err != nil { | |||
mem.cache.Remove(tx) | |||
return err | |||
} | |||
reqRes.SetCallback(mem.reqResCb(tx, txInfo.SenderID, txInfo.SenderNodeID, cb)) | |||
return nil | |||
} | |||
// Global callback that will be called after every ABCI response. | |||
// Having a single global callback avoids needing to set a callback for each request. | |||
// However, processing the checkTx response requires the peerID (so we can track which txs we heard from who), | |||
// and peerID is not included in the ABCI request, so we have to set request-specific callbacks that | |||
// include this information. If we're not in the midst of a recheck, this function will just return, | |||
// so the request specific callback can do the work. | |||
// | |||
// When rechecking, we don't need the peerID, so the recheck callback happens | |||
// here. | |||
func (mem *CListMempool) globalCb(req *abci.Request, res *abci.Response) { | |||
if mem.recheckCursor == nil { | |||
return | |||
} | |||
mem.metrics.RecheckTimes.Add(1) | |||
mem.resCbRecheck(req, res) | |||
// update metrics | |||
mem.metrics.Size.Set(float64(mem.Size())) | |||
} | |||
// Request specific callback that should be set on individual reqRes objects | |||
// to incorporate local information when processing the response. | |||
// This allows us to track the peer that sent us this tx, so we can avoid sending it back to them. | |||
// NOTE: alternatively, we could include this information in the ABCI request itself. | |||
// | |||
// External callers of CheckTx, like the RPC, can also pass an externalCb through here that is called | |||
// when all other response processing is complete. | |||
// | |||
// Used in CheckTx to record PeerID who sent us the tx. | |||
func (mem *CListMempool) reqResCb( | |||
tx []byte, | |||
peerID uint16, | |||
peerP2PID types.NodeID, | |||
externalCb func(*abci.Response), | |||
) func(res *abci.Response) { | |||
return func(res *abci.Response) { | |||
if mem.recheckCursor != nil { | |||
// this should never happen | |||
panic("recheck cursor is not nil in reqResCb") | |||
} | |||
mem.resCbFirstTime(tx, peerID, peerP2PID, res) | |||
// update metrics | |||
mem.metrics.Size.Set(float64(mem.Size())) | |||
// passed in by the caller of CheckTx, eg. the RPC | |||
if externalCb != nil { | |||
externalCb(res) | |||
} | |||
} | |||
} | |||
// Called from: | |||
// - resCbFirstTime (lock not held) if tx is valid | |||
func (mem *CListMempool) addTx(memTx *mempoolTx) { | |||
e := mem.txs.PushBack(memTx) | |||
mem.txsMap.Store(memTx.tx.Key(), e) | |||
atomic.AddInt64(&mem.txsBytes, int64(len(memTx.tx))) | |||
mem.metrics.TxSizeBytes.Observe(float64(len(memTx.tx))) | |||
} | |||
// Called from: | |||
// - Update (lock held) if tx was committed | |||
// - resCbRecheck (lock not held) if tx was invalidated | |||
func (mem *CListMempool) removeTx(tx types.Tx, elem *clist.CElement, removeFromCache bool) { | |||
mem.txs.Remove(elem) | |||
elem.DetachPrev() | |||
mem.txsMap.Delete(tx.Key()) | |||
atomic.AddInt64(&mem.txsBytes, int64(-len(tx))) | |||
if removeFromCache { | |||
mem.cache.Remove(tx) | |||
} | |||
} | |||
// RemoveTxByKey removes a transaction from the mempool by its TxKey index. | |||
func (mem *CListMempool) RemoveTxByKey(txKey types.TxKey) error { | |||
if e, ok := mem.txsMap.Load(txKey); ok { | |||
memTx := e.(*clist.CElement).Value.(*mempoolTx) | |||
if memTx != nil { | |||
mem.removeTx(memTx.tx, e.(*clist.CElement), false) | |||
return nil | |||
} | |||
return errors.New("transaction not found") | |||
} | |||
return errors.New("invalid transaction found") | |||
} | |||
func (mem *CListMempool) isFull(txSize int) error { | |||
var ( | |||
memSize = mem.Size() | |||
txsBytes = mem.SizeBytes() | |||
) | |||
if memSize >= mem.config.Size || int64(txSize)+txsBytes > mem.config.MaxTxsBytes { | |||
return types.ErrMempoolIsFull{ | |||
NumTxs: memSize, | |||
MaxTxs: mem.config.Size, | |||
TxsBytes: txsBytes, | |||
MaxTxsBytes: mem.config.MaxTxsBytes, | |||
} | |||
} | |||
return nil | |||
} | |||
// callback, which is called after the app checked the tx for the first time. | |||
// | |||
// The case where the app checks the tx for the second and subsequent times is | |||
// handled by the resCbRecheck callback. | |||
func (mem *CListMempool) resCbFirstTime( | |||
tx []byte, | |||
peerID uint16, | |||
peerP2PID types.NodeID, | |||
res *abci.Response, | |||
) { | |||
switch r := res.Value.(type) { | |||
case *abci.Response_CheckTx: | |||
var postCheckErr error | |||
if mem.postCheck != nil { | |||
postCheckErr = mem.postCheck(tx, r.CheckTx) | |||
} | |||
if (r.CheckTx.Code == abci.CodeTypeOK) && postCheckErr == nil { | |||
// Check mempool isn't full again to reduce the chance of exceeding the | |||
// limits. | |||
if err := mem.isFull(len(tx)); err != nil { | |||
// remove from cache (mempool might have a space later) | |||
mem.cache.Remove(tx) | |||
mem.logger.Error(err.Error()) | |||
return | |||
} | |||
memTx := &mempoolTx{ | |||
height: mem.height, | |||
gasWanted: r.CheckTx.GasWanted, | |||
tx: tx, | |||
} | |||
memTx.senders.Store(peerID, true) | |||
mem.addTx(memTx) | |||
mem.logger.Debug( | |||
"added good transaction", | |||
"tx", types.Tx(tx).Hash(), | |||
"res", r, | |||
"height", memTx.height, | |||
"total", mem.Size(), | |||
) | |||
mem.notifyTxsAvailable() | |||
} else { | |||
// ignore bad transaction | |||
mem.logger.Debug( | |||
"rejected bad transaction", | |||
"tx", types.Tx(tx).Hash(), | |||
"peerID", peerP2PID, | |||
"res", r, | |||
"err", postCheckErr, | |||
) | |||
mem.metrics.FailedTxs.Add(1) | |||
if !mem.config.KeepInvalidTxsInCache { | |||
// remove from cache (it might be good later) | |||
mem.cache.Remove(tx) | |||
} | |||
} | |||
default: | |||
// ignore other messages | |||
} | |||
} | |||
// callback, which is called after the app rechecked the tx. | |||
// | |||
// The case where the app checks the tx for the first time is handled by the | |||
// resCbFirstTime callback. | |||
func (mem *CListMempool) resCbRecheck(req *abci.Request, res *abci.Response) { | |||
switch r := res.Value.(type) { | |||
case *abci.Response_CheckTx: | |||
tx := req.GetCheckTx().Tx | |||
memTx := mem.recheckCursor.Value.(*mempoolTx) | |||
// Search through the remaining list of tx to recheck for a transaction that matches | |||
// the one we received from the ABCI application. | |||
for { | |||
if bytes.Equal(tx, memTx.tx) { | |||
// We've found a tx in the recheck list that matches the tx that we | |||
// received from the ABCI application. | |||
// Break, and use this transaction for further checks. | |||
break | |||
} | |||
mem.logger.Error( | |||
"re-CheckTx transaction mismatch", | |||
"got", types.Tx(tx), | |||
"expected", memTx.tx, | |||
) | |||
if mem.recheckCursor == mem.recheckEnd { | |||
// we reached the end of the recheckTx list without finding a tx | |||
// matching the one we received from the ABCI application. | |||
// Return without processing any tx. | |||
mem.recheckCursor = nil | |||
return | |||
} | |||
mem.recheckCursor = mem.recheckCursor.Next() | |||
memTx = mem.recheckCursor.Value.(*mempoolTx) | |||
} | |||
var postCheckErr error | |||
if mem.postCheck != nil { | |||
postCheckErr = mem.postCheck(tx, r.CheckTx) | |||
} | |||
if (r.CheckTx.Code == abci.CodeTypeOK) && postCheckErr == nil { | |||
// Good, nothing to do. | |||
} else { | |||
// Tx became invalidated due to newly committed block. | |||
mem.logger.Debug("tx is no longer valid", "tx", types.Tx(tx).Hash(), "res", r, "err", postCheckErr) | |||
// NOTE: we remove tx from the cache because it might be good later | |||
mem.removeTx(tx, mem.recheckCursor, !mem.config.KeepInvalidTxsInCache) | |||
} | |||
if mem.recheckCursor == mem.recheckEnd { | |||
mem.recheckCursor = nil | |||
} else { | |||
mem.recheckCursor = mem.recheckCursor.Next() | |||
} | |||
if mem.recheckCursor == nil { | |||
// Done! | |||
mem.logger.Debug("done rechecking txs") | |||
// incase the recheck removed all txs | |||
if mem.Size() > 0 { | |||
mem.notifyTxsAvailable() | |||
} | |||
} | |||
default: | |||
// ignore other messages | |||
} | |||
} | |||
// Safe for concurrent use by multiple goroutines. | |||
func (mem *CListMempool) TxsAvailable() <-chan struct{} { | |||
return mem.txsAvailable | |||
} | |||
func (mem *CListMempool) notifyTxsAvailable() { | |||
if mem.Size() == 0 { | |||
panic("notified txs available but mempool is empty!") | |||
} | |||
if mem.txsAvailable != nil && !mem.notifiedTxsAvailable { | |||
// channel cap is 1, so this will send once | |||
mem.notifiedTxsAvailable = true | |||
select { | |||
case mem.txsAvailable <- struct{}{}: | |||
default: | |||
} | |||
} | |||
} | |||
// Safe for concurrent use by multiple goroutines. | |||
func (mem *CListMempool) ReapMaxBytesMaxGas(maxBytes, maxGas int64) types.Txs { | |||
mem.updateMtx.RLock() | |||
defer mem.updateMtx.RUnlock() | |||
var ( | |||
totalGas int64 | |||
runningSize int64 | |||
) | |||
// TODO: we will get a performance boost if we have a good estimate of avg | |||
// size per tx, and set the initial capacity based off of that. | |||
// txs := make([]types.Tx, 0, tmmath.MinInt(mem.txs.Len(), max/mem.avgTxSize)) | |||
txs := make([]types.Tx, 0, mem.txs.Len()) | |||
for e := mem.txs.Front(); e != nil; e = e.Next() { | |||
memTx := e.Value.(*mempoolTx) | |||
txs = append(txs, memTx.tx) | |||
dataSize := types.ComputeProtoSizeForTxs([]types.Tx{memTx.tx}) | |||
// Check total size requirement | |||
if maxBytes > -1 && runningSize+dataSize > maxBytes { | |||
return txs[:len(txs)-1] | |||
} | |||
runningSize += dataSize | |||
// Check total gas requirement. | |||
// If maxGas is negative, skip this check. | |||
// Since newTotalGas < masGas, which | |||
// must be non-negative, it follows that this won't overflow. | |||
newTotalGas := totalGas + memTx.gasWanted | |||
if maxGas > -1 && newTotalGas > maxGas { | |||
return txs[:len(txs)-1] | |||
} | |||
totalGas = newTotalGas | |||
} | |||
return txs | |||
} | |||
// Safe for concurrent use by multiple goroutines. | |||
func (mem *CListMempool) ReapMaxTxs(max int) types.Txs { | |||
mem.updateMtx.RLock() | |||
defer mem.updateMtx.RUnlock() | |||
if max < 0 { | |||
max = mem.txs.Len() | |||
} | |||
txs := make([]types.Tx, 0, tmmath.MinInt(mem.txs.Len(), max)) | |||
for e := mem.txs.Front(); e != nil && len(txs) <= max; e = e.Next() { | |||
memTx := e.Value.(*mempoolTx) | |||
txs = append(txs, memTx.tx) | |||
} | |||
return txs | |||
} | |||
// Lock() must be help by the caller during execution. | |||
func (mem *CListMempool) Update( | |||
height int64, | |||
txs types.Txs, | |||
deliverTxResponses []*abci.ResponseDeliverTx, | |||
preCheck mempool.PreCheckFunc, | |||
postCheck mempool.PostCheckFunc, | |||
) error { | |||
// Set height | |||
mem.height = height | |||
mem.notifiedTxsAvailable = false | |||
if preCheck != nil { | |||
mem.preCheck = preCheck | |||
} | |||
if postCheck != nil { | |||
mem.postCheck = postCheck | |||
} | |||
for i, tx := range txs { | |||
if deliverTxResponses[i].Code == abci.CodeTypeOK { | |||
// Add valid committed tx to the cache (if missing). | |||
_ = mem.cache.Push(tx) | |||
} else if !mem.config.KeepInvalidTxsInCache { | |||
// Allow invalid transactions to be resubmitted. | |||
mem.cache.Remove(tx) | |||
} | |||
// Remove committed tx from the mempool. | |||
// | |||
// Note an evil proposer can drop valid txs! | |||
// Mempool before: | |||
// 100 -> 101 -> 102 | |||
// Block, proposed by an evil proposer: | |||
// 101 -> 102 | |||
// Mempool after: | |||
// 100 | |||
// https://github.com/tendermint/tendermint/issues/3322. | |||
if e, ok := mem.txsMap.Load(tx.Key()); ok { | |||
mem.removeTx(tx, e.(*clist.CElement), false) | |||
} | |||
} | |||
// Either recheck non-committed txs to see if they became invalid | |||
// or just notify there're some txs left. | |||
if mem.Size() > 0 { | |||
if mem.config.Recheck { | |||
mem.logger.Debug("recheck txs", "numtxs", mem.Size(), "height", height) | |||
mem.recheckTxs() | |||
// At this point, mem.txs are being rechecked. | |||
// mem.recheckCursor re-scans mem.txs and possibly removes some txs. | |||
// Before mem.Reap(), we should wait for mem.recheckCursor to be nil. | |||
} else { | |||
mem.notifyTxsAvailable() | |||
} | |||
} | |||
// Update metrics | |||
mem.metrics.Size.Set(float64(mem.Size())) | |||
return nil | |||
} | |||
func (mem *CListMempool) recheckTxs() { | |||
if mem.Size() == 0 { | |||
panic("recheckTxs is called, but the mempool is empty") | |||
} | |||
mem.recheckCursor = mem.txs.Front() | |||
mem.recheckEnd = mem.txs.Back() | |||
ctx := context.Background() | |||
// Push txs to proxyAppConn | |||
// NOTE: globalCb may be called concurrently. | |||
for e := mem.txs.Front(); e != nil; e = e.Next() { | |||
memTx := e.Value.(*mempoolTx) | |||
_, err := mem.proxyAppConn.CheckTxAsync(ctx, abci.RequestCheckTx{ | |||
Tx: memTx.tx, | |||
Type: abci.CheckTxType_Recheck, | |||
}) | |||
if err != nil { | |||
// No need in retrying since memTx will be rechecked after next block. | |||
mem.logger.Error("Can't check tx", "err", err) | |||
} | |||
} | |||
_, err := mem.proxyAppConn.FlushAsync(ctx) | |||
if err != nil { | |||
mem.logger.Error("Can't flush txs", "err", err) | |||
} | |||
} | |||
//-------------------------------------------------------------------------------- | |||
// mempoolTx is a transaction that successfully ran | |||
type mempoolTx struct { | |||
height int64 // height that this tx had been validated in | |||
gasWanted int64 // amount of gas this tx states it will require | |||
tx types.Tx // | |||
// ids of peers who've sent us this tx (as a map for quick lookups). | |||
// senders: PeerID -> bool | |||
senders sync.Map | |||
} | |||
// Height returns the height for this transaction | |||
func (memTx *mempoolTx) Height() int64 { | |||
return atomic.LoadInt64(&memTx.height) | |||
} |
@ -1,691 +0,0 @@ | |||
package v0 | |||
import ( | |||
"context" | |||
"crypto/rand" | |||
"encoding/binary" | |||
"fmt" | |||
mrand "math/rand" | |||
"os" | |||
"testing" | |||
"time" | |||
"github.com/gogo/protobuf/proto" | |||
gogotypes "github.com/gogo/protobuf/types" | |||
"github.com/stretchr/testify/assert" | |||
"github.com/stretchr/testify/mock" | |||
"github.com/stretchr/testify/require" | |||
abciclient "github.com/tendermint/tendermint/abci/client" | |||
abciclimocks "github.com/tendermint/tendermint/abci/client/mocks" | |||
"github.com/tendermint/tendermint/abci/example/kvstore" | |||
abciserver "github.com/tendermint/tendermint/abci/server" | |||
abci "github.com/tendermint/tendermint/abci/types" | |||
"github.com/tendermint/tendermint/config" | |||
"github.com/tendermint/tendermint/internal/mempool" | |||
"github.com/tendermint/tendermint/libs/log" | |||
tmrand "github.com/tendermint/tendermint/libs/rand" | |||
"github.com/tendermint/tendermint/libs/service" | |||
"github.com/tendermint/tendermint/types" | |||
) | |||
// A cleanupFunc cleans up any config / test files created for a particular | |||
// test. | |||
type cleanupFunc func() | |||
func newMempoolWithApp(cc abciclient.Creator) (*CListMempool, cleanupFunc, error) { | |||
conf, err := config.ResetTestRoot("mempool_test") | |||
if err != nil { | |||
return nil, func() {}, err | |||
} | |||
mp, cu := newMempoolWithAppAndConfig(cc, conf) | |||
return mp, cu, nil | |||
} | |||
func newMempoolWithAppAndConfig(cc abciclient.Creator, cfg *config.Config) (*CListMempool, cleanupFunc) { | |||
appConnMem, _ := cc() | |||
appConnMem.SetLogger(log.TestingLogger().With("module", "abci-client", "connection", "mempool")) | |||
err := appConnMem.Start() | |||
if err != nil { | |||
panic(err) | |||
} | |||
mp := NewCListMempool(cfg.Mempool, appConnMem, 0) | |||
mp.SetLogger(log.TestingLogger()) | |||
return mp, func() { os.RemoveAll(cfg.RootDir) } | |||
} | |||
func ensureNoFire(t *testing.T, ch <-chan struct{}, timeoutMS int) { | |||
timer := time.NewTimer(time.Duration(timeoutMS) * time.Millisecond) | |||
select { | |||
case <-ch: | |||
t.Fatal("Expected not to fire") | |||
case <-timer.C: | |||
} | |||
} | |||
func ensureFire(t *testing.T, ch <-chan struct{}, timeoutMS int) { | |||
timer := time.NewTimer(time.Duration(timeoutMS) * time.Millisecond) | |||
select { | |||
case <-ch: | |||
case <-timer.C: | |||
t.Fatal("Expected to fire") | |||
} | |||
} | |||
func checkTxs(t *testing.T, mp mempool.Mempool, count int, peerID uint16) types.Txs { | |||
txs := make(types.Txs, count) | |||
txInfo := mempool.TxInfo{SenderID: peerID} | |||
for i := 0; i < count; i++ { | |||
txBytes := make([]byte, 20) | |||
txs[i] = txBytes | |||
_, err := rand.Read(txBytes) | |||
if err != nil { | |||
t.Error(err) | |||
} | |||
if err := mp.CheckTx(context.Background(), txBytes, nil, txInfo); err != nil { | |||
// Skip invalid txs. | |||
// TestMempoolFilters will fail otherwise. It asserts a number of txs | |||
// returned. | |||
if types.IsPreCheckError(err) { | |||
continue | |||
} | |||
t.Fatalf("CheckTx failed: %v while checking #%d tx", err, i) | |||
} | |||
} | |||
return txs | |||
} | |||
func TestReapMaxBytesMaxGas(t *testing.T) { | |||
app := kvstore.NewApplication() | |||
cc := abciclient.NewLocalCreator(app) | |||
mp, cleanup, err := newMempoolWithApp(cc) | |||
require.NoError(t, err) | |||
defer cleanup() | |||
// Ensure gas calculation behaves as expected | |||
checkTxs(t, mp, 1, mempool.UnknownPeerID) | |||
tx0 := mp.TxsFront().Value.(*mempoolTx) | |||
// assert that kv store has gas wanted = 1. | |||
require.Equal(t, app.CheckTx(abci.RequestCheckTx{Tx: tx0.tx}).GasWanted, int64(1), "KVStore had a gas value neq to 1") | |||
require.Equal(t, tx0.gasWanted, int64(1), "transactions gas was set incorrectly") | |||
// ensure each tx is 20 bytes long | |||
require.Equal(t, len(tx0.tx), 20, "Tx is longer than 20 bytes") | |||
mp.Flush() | |||
// each table driven test creates numTxsToCreate txs with checkTx, and at the end clears all remaining txs. | |||
// each tx has 20 bytes | |||
tests := []struct { | |||
numTxsToCreate int | |||
maxBytes int64 | |||
maxGas int64 | |||
expectedNumTxs int | |||
}{ | |||
{20, -1, -1, 20}, | |||
{20, -1, 0, 0}, | |||
{20, -1, 10, 10}, | |||
{20, -1, 30, 20}, | |||
{20, 0, -1, 0}, | |||
{20, 0, 10, 0}, | |||
{20, 10, 10, 0}, | |||
{20, 24, 10, 1}, | |||
{20, 240, 5, 5}, | |||
{20, 240, -1, 10}, | |||
{20, 240, 10, 10}, | |||
{20, 240, 15, 10}, | |||
{20, 20000, -1, 20}, | |||
{20, 20000, 5, 5}, | |||
{20, 20000, 30, 20}, | |||
} | |||
for tcIndex, tt := range tests { | |||
checkTxs(t, mp, tt.numTxsToCreate, mempool.UnknownPeerID) | |||
got := mp.ReapMaxBytesMaxGas(tt.maxBytes, tt.maxGas) | |||
assert.Equal(t, tt.expectedNumTxs, len(got), "Got %d txs, expected %d, tc #%d", | |||
len(got), tt.expectedNumTxs, tcIndex) | |||
mp.Flush() | |||
} | |||
} | |||
func TestMempoolFilters(t *testing.T) { | |||
app := kvstore.NewApplication() | |||
cc := abciclient.NewLocalCreator(app) | |||
mp, cleanup, err := newMempoolWithApp(cc) | |||
require.NoError(t, err) | |||
defer cleanup() | |||
emptyTxArr := []types.Tx{[]byte{}} | |||
nopPreFilter := func(tx types.Tx) error { return nil } | |||
nopPostFilter := func(tx types.Tx, res *abci.ResponseCheckTx) error { return nil } | |||
// each table driven test creates numTxsToCreate txs with checkTx, and at the end clears all remaining txs. | |||
// each tx has 20 bytes | |||
tests := []struct { | |||
numTxsToCreate int | |||
preFilter mempool.PreCheckFunc | |||
postFilter mempool.PostCheckFunc | |||
expectedNumTxs int | |||
}{ | |||
{10, nopPreFilter, nopPostFilter, 10}, | |||
{10, mempool.PreCheckMaxBytes(10), nopPostFilter, 0}, | |||
{10, mempool.PreCheckMaxBytes(22), nopPostFilter, 10}, | |||
{10, nopPreFilter, mempool.PostCheckMaxGas(-1), 10}, | |||
{10, nopPreFilter, mempool.PostCheckMaxGas(0), 0}, | |||
{10, nopPreFilter, mempool.PostCheckMaxGas(1), 10}, | |||
{10, nopPreFilter, mempool.PostCheckMaxGas(3000), 10}, | |||
{10, mempool.PreCheckMaxBytes(10), mempool.PostCheckMaxGas(20), 0}, | |||
{10, mempool.PreCheckMaxBytes(30), mempool.PostCheckMaxGas(20), 10}, | |||
{10, mempool.PreCheckMaxBytes(22), mempool.PostCheckMaxGas(1), 10}, | |||
{10, mempool.PreCheckMaxBytes(22), mempool.PostCheckMaxGas(0), 0}, | |||
} | |||
for tcIndex, tt := range tests { | |||
err := mp.Update(1, emptyTxArr, abciResponses(len(emptyTxArr), abci.CodeTypeOK), tt.preFilter, tt.postFilter) | |||
require.NoError(t, err) | |||
checkTxs(t, mp, tt.numTxsToCreate, mempool.UnknownPeerID) | |||
require.Equal(t, tt.expectedNumTxs, mp.Size(), "mempool had the incorrect size, on test case %d", tcIndex) | |||
mp.Flush() | |||
} | |||
} | |||
func TestMempoolUpdate(t *testing.T) { | |||
app := kvstore.NewApplication() | |||
cc := abciclient.NewLocalCreator(app) | |||
mp, cleanup, err := newMempoolWithApp(cc) | |||
require.NoError(t, err) | |||
defer cleanup() | |||
// 1. Adds valid txs to the cache | |||
{ | |||
err := mp.Update(1, []types.Tx{[]byte{0x01}}, abciResponses(1, abci.CodeTypeOK), nil, nil) | |||
require.NoError(t, err) | |||
err = mp.CheckTx(context.Background(), []byte{0x01}, nil, mempool.TxInfo{}) | |||
require.NoError(t, err) | |||
} | |||
// 2. Removes valid txs from the mempool | |||
{ | |||
err := mp.CheckTx(context.Background(), []byte{0x02}, nil, mempool.TxInfo{}) | |||
require.NoError(t, err) | |||
err = mp.Update(1, []types.Tx{[]byte{0x02}}, abciResponses(1, abci.CodeTypeOK), nil, nil) | |||
require.NoError(t, err) | |||
assert.Zero(t, mp.Size()) | |||
} | |||
// 3. Removes invalid transactions from the cache and the mempool (if present) | |||
{ | |||
err := mp.CheckTx(context.Background(), []byte{0x03}, nil, mempool.TxInfo{}) | |||
require.NoError(t, err) | |||
err = mp.Update(1, []types.Tx{[]byte{0x03}}, abciResponses(1, 1), nil, nil) | |||
require.NoError(t, err) | |||
assert.Zero(t, mp.Size()) | |||
err = mp.CheckTx(context.Background(), []byte{0x03}, nil, mempool.TxInfo{}) | |||
require.NoError(t, err) | |||
} | |||
} | |||
func TestMempoolUpdateDoesNotPanicWhenApplicationMissedTx(t *testing.T) { | |||
var callback abciclient.Callback | |||
mockClient := new(abciclimocks.Client) | |||
mockClient.On("Start").Return(nil) | |||
mockClient.On("SetLogger", mock.Anything) | |||
mockClient.On("Error").Return(nil).Times(4) | |||
mockClient.On("FlushAsync", mock.Anything).Return(abciclient.NewReqRes(abci.ToRequestFlush()), nil) | |||
mockClient.On("SetResponseCallback", mock.MatchedBy(func(cb abciclient.Callback) bool { callback = cb; return true })) | |||
cc := func() (abciclient.Client, error) { | |||
return mockClient, nil | |||
} | |||
mp, cleanup, err := newMempoolWithApp(cc) | |||
require.NoError(t, err) | |||
defer cleanup() | |||
// Add 4 transactions to the mempool by calling the mempool's `CheckTx` on each of them. | |||
txs := []types.Tx{[]byte{0x01}, []byte{0x02}, []byte{0x03}, []byte{0x04}} | |||
for _, tx := range txs { | |||
reqRes := abciclient.NewReqRes(abci.ToRequestCheckTx(abci.RequestCheckTx{Tx: tx})) | |||
reqRes.Response = abci.ToResponseCheckTx(abci.ResponseCheckTx{Code: abci.CodeTypeOK}) | |||
// SetDone allows the ReqRes to process its callback synchronously. | |||
// This simulates the Response being ready for the client immediately. | |||
reqRes.SetDone() | |||
mockClient.On("CheckTxAsync", mock.Anything, mock.Anything).Return(reqRes, nil) | |||
err := mp.CheckTx(context.Background(), tx, nil, mempool.TxInfo{}) | |||
require.NoError(t, err) | |||
} | |||
// Calling update to remove the first transaction from the mempool. | |||
// This call also triggers the mempool to recheck its remaining transactions. | |||
err = mp.Update(0, []types.Tx{txs[0]}, abciResponses(1, abci.CodeTypeOK), nil, nil) | |||
require.Nil(t, err) | |||
// The mempool has now sent its requests off to the client to be rechecked | |||
// and is waiting for the corresponding callbacks to be called. | |||
// We now call the mempool-supplied callback on the first and third transaction. | |||
// This simulates the client dropping the second request. | |||
// Previous versions of this code panicked when the ABCI application missed | |||
// a recheck-tx request. | |||
resp := abci.ResponseCheckTx{Code: abci.CodeTypeOK} | |||
req := abci.RequestCheckTx{Tx: txs[1]} | |||
callback(abci.ToRequestCheckTx(req), abci.ToResponseCheckTx(resp)) | |||
req = abci.RequestCheckTx{Tx: txs[3]} | |||
callback(abci.ToRequestCheckTx(req), abci.ToResponseCheckTx(resp)) | |||
mockClient.AssertExpectations(t) | |||
} | |||
func TestMempool_KeepInvalidTxsInCache(t *testing.T) { | |||
app := kvstore.NewApplication() | |||
cc := abciclient.NewLocalCreator(app) | |||
wcfg := config.DefaultConfig() | |||
wcfg.Mempool.KeepInvalidTxsInCache = true | |||
mp, cleanup := newMempoolWithAppAndConfig(cc, wcfg) | |||
defer cleanup() | |||
// 1. An invalid transaction must remain in the cache after Update | |||
{ | |||
a := make([]byte, 8) | |||
binary.BigEndian.PutUint64(a, 0) | |||
b := make([]byte, 8) | |||
binary.BigEndian.PutUint64(b, 1) | |||
err := mp.CheckTx(context.Background(), b, nil, mempool.TxInfo{}) | |||
require.NoError(t, err) | |||
// simulate new block | |||
_ = app.DeliverTx(abci.RequestDeliverTx{Tx: a}) | |||
_ = app.DeliverTx(abci.RequestDeliverTx{Tx: b}) | |||
err = mp.Update(1, []types.Tx{a, b}, | |||
[]*abci.ResponseDeliverTx{{Code: abci.CodeTypeOK}, {Code: 2}}, nil, nil) | |||
require.NoError(t, err) | |||
// a must be added to the cache | |||
err = mp.CheckTx(context.Background(), a, nil, mempool.TxInfo{}) | |||
require.NoError(t, err) | |||
// b must remain in the cache | |||
err = mp.CheckTx(context.Background(), b, nil, mempool.TxInfo{}) | |||
require.NoError(t, err) | |||
} | |||
// 2. An invalid transaction must remain in the cache | |||
{ | |||
a := make([]byte, 8) | |||
binary.BigEndian.PutUint64(a, 0) | |||
// remove a from the cache to test (2) | |||
mp.cache.Remove(a) | |||
err := mp.CheckTx(context.Background(), a, nil, mempool.TxInfo{}) | |||
require.NoError(t, err) | |||
} | |||
} | |||
func TestTxsAvailable(t *testing.T) { | |||
app := kvstore.NewApplication() | |||
cc := abciclient.NewLocalCreator(app) | |||
mp, cleanup, err := newMempoolWithApp(cc) | |||
require.NoError(t, err) | |||
defer cleanup() | |||
mp.EnableTxsAvailable() | |||
timeoutMS := 500 | |||
// with no txs, it shouldnt fire | |||
ensureNoFire(t, mp.TxsAvailable(), timeoutMS) | |||
// send a bunch of txs, it should only fire once | |||
txs := checkTxs(t, mp, 100, mempool.UnknownPeerID) | |||
ensureFire(t, mp.TxsAvailable(), timeoutMS) | |||
ensureNoFire(t, mp.TxsAvailable(), timeoutMS) | |||
// call update with half the txs. | |||
// it should fire once now for the new height | |||
// since there are still txs left | |||
committedTxs, txs := txs[:50], txs[50:] | |||
if err := mp.Update(1, committedTxs, abciResponses(len(committedTxs), abci.CodeTypeOK), nil, nil); err != nil { | |||
t.Error(err) | |||
} | |||
ensureFire(t, mp.TxsAvailable(), timeoutMS) | |||
ensureNoFire(t, mp.TxsAvailable(), timeoutMS) | |||
// send a bunch more txs. we already fired for this height so it shouldnt fire again | |||
moreTxs := checkTxs(t, mp, 50, mempool.UnknownPeerID) | |||
ensureNoFire(t, mp.TxsAvailable(), timeoutMS) | |||
// now call update with all the txs. it should not fire as there are no txs left | |||
committedTxs = append(txs, moreTxs...) //nolint: gocritic | |||
if err := mp.Update(2, committedTxs, abciResponses(len(committedTxs), abci.CodeTypeOK), nil, nil); err != nil { | |||
t.Error(err) | |||
} | |||
ensureNoFire(t, mp.TxsAvailable(), timeoutMS) | |||
// send a bunch more txs, it should only fire once | |||
checkTxs(t, mp, 100, mempool.UnknownPeerID) | |||
ensureFire(t, mp.TxsAvailable(), timeoutMS) | |||
ensureNoFire(t, mp.TxsAvailable(), timeoutMS) | |||
} | |||
func TestSerialReap(t *testing.T) { | |||
app := kvstore.NewApplication() | |||
cc := abciclient.NewLocalCreator(app) | |||
mp, cleanup, err := newMempoolWithApp(cc) | |||
require.NoError(t, err) | |||
defer cleanup() | |||
appConnCon, _ := cc() | |||
appConnCon.SetLogger(log.TestingLogger().With("module", "abci-client", "connection", "consensus")) | |||
err = appConnCon.Start() | |||
require.Nil(t, err) | |||
cacheMap := make(map[string]struct{}) | |||
deliverTxsRange := func(start, end int) { | |||
// Deliver some txs. | |||
for i := start; i < end; i++ { | |||
// This will succeed | |||
txBytes := make([]byte, 8) | |||
binary.BigEndian.PutUint64(txBytes, uint64(i)) | |||
err := mp.CheckTx(context.Background(), txBytes, nil, mempool.TxInfo{}) | |||
_, cached := cacheMap[string(txBytes)] | |||
if cached { | |||
require.NotNil(t, err, "expected error for cached tx") | |||
} else { | |||
require.Nil(t, err, "expected no err for uncached tx") | |||
} | |||
cacheMap[string(txBytes)] = struct{}{} | |||
// Duplicates are cached and should return error | |||
err = mp.CheckTx(context.Background(), txBytes, nil, mempool.TxInfo{}) | |||
require.NotNil(t, err, "Expected error after CheckTx on duplicated tx") | |||
} | |||
} | |||
reapCheck := func(exp int) { | |||
txs := mp.ReapMaxBytesMaxGas(-1, -1) | |||
require.Equal(t, len(txs), exp, fmt.Sprintf("Expected to reap %v txs but got %v", exp, len(txs))) | |||
} | |||
updateRange := func(start, end int) { | |||
txs := make([]types.Tx, 0) | |||
for i := start; i < end; i++ { | |||
txBytes := make([]byte, 8) | |||
binary.BigEndian.PutUint64(txBytes, uint64(i)) | |||
txs = append(txs, txBytes) | |||
} | |||
if err := mp.Update(0, txs, abciResponses(len(txs), abci.CodeTypeOK), nil, nil); err != nil { | |||
t.Error(err) | |||
} | |||
} | |||
commitRange := func(start, end int) { | |||
ctx := context.Background() | |||
// Deliver some txs. | |||
for i := start; i < end; i++ { | |||
txBytes := make([]byte, 8) | |||
binary.BigEndian.PutUint64(txBytes, uint64(i)) | |||
res, err := appConnCon.DeliverTxSync(ctx, abci.RequestDeliverTx{Tx: txBytes}) | |||
if err != nil { | |||
t.Errorf("client error committing tx: %v", err) | |||
} | |||
if res.IsErr() { | |||
t.Errorf("error committing tx. Code:%v result:%X log:%v", | |||
res.Code, res.Data, res.Log) | |||
} | |||
} | |||
res, err := appConnCon.CommitSync(ctx) | |||
if err != nil { | |||
t.Errorf("client error committing: %v", err) | |||
} | |||
if len(res.Data) != 8 { | |||
t.Errorf("error committing. Hash:%X", res.Data) | |||
} | |||
} | |||
//---------------------------------------- | |||
// Deliver some txs. | |||
deliverTxsRange(0, 100) | |||
// Reap the txs. | |||
reapCheck(100) | |||
// Reap again. We should get the same amount | |||
reapCheck(100) | |||
// Deliver 0 to 999, we should reap 900 new txs | |||
// because 100 were already counted. | |||
deliverTxsRange(0, 1000) | |||
// Reap the txs. | |||
reapCheck(1000) | |||
// Reap again. We should get the same amount | |||
reapCheck(1000) | |||
// Commit from the conensus AppConn | |||
commitRange(0, 500) | |||
updateRange(0, 500) | |||
// We should have 500 left. | |||
reapCheck(500) | |||
// Deliver 100 invalid txs and 100 valid txs | |||
deliverTxsRange(900, 1100) | |||
// We should have 600 now. | |||
reapCheck(600) | |||
} | |||
func TestMempool_CheckTxChecksTxSize(t *testing.T) { | |||
app := kvstore.NewApplication() | |||
cc := abciclient.NewLocalCreator(app) | |||
mempl, cleanup, err := newMempoolWithApp(cc) | |||
require.NoError(t, err) | |||
defer cleanup() | |||
maxTxSize := mempl.config.MaxTxBytes | |||
testCases := []struct { | |||
len int | |||
err bool | |||
}{ | |||
// check small txs. no error | |||
0: {10, false}, | |||
1: {1000, false}, | |||
2: {1000000, false}, | |||
// check around maxTxSize | |||
3: {maxTxSize - 1, false}, | |||
4: {maxTxSize, false}, | |||
5: {maxTxSize + 1, true}, | |||
} | |||
for i, testCase := range testCases { | |||
caseString := fmt.Sprintf("case %d, len %d", i, testCase.len) | |||
tx := tmrand.Bytes(testCase.len) | |||
err := mempl.CheckTx(context.Background(), tx, nil, mempool.TxInfo{}) | |||
bv := gogotypes.BytesValue{Value: tx} | |||
bz, err2 := bv.Marshal() | |||
require.NoError(t, err2) | |||
require.Equal(t, len(bz), proto.Size(&bv), caseString) | |||
if !testCase.err { | |||
require.NoError(t, err, caseString) | |||
} else { | |||
require.Equal(t, err, types.ErrTxTooLarge{ | |||
Max: maxTxSize, | |||
Actual: testCase.len, | |||
}, caseString) | |||
} | |||
} | |||
} | |||
func TestMempoolTxsBytes(t *testing.T) { | |||
app := kvstore.NewApplication() | |||
cc := abciclient.NewLocalCreator(app) | |||
cfg, err := config.ResetTestRoot("mempool_test") | |||
require.NoError(t, err) | |||
cfg.Mempool.MaxTxsBytes = 10 | |||
mp, cleanup := newMempoolWithAppAndConfig(cc, cfg) | |||
defer cleanup() | |||
// 1. zero by default | |||
assert.EqualValues(t, 0, mp.SizeBytes()) | |||
// 2. len(tx) after CheckTx | |||
err = mp.CheckTx(context.Background(), []byte{0x01}, nil, mempool.TxInfo{}) | |||
require.NoError(t, err) | |||
assert.EqualValues(t, 1, mp.SizeBytes()) | |||
// 3. zero again after tx is removed by Update | |||
err = mp.Update(1, []types.Tx{[]byte{0x01}}, abciResponses(1, abci.CodeTypeOK), nil, nil) | |||
require.NoError(t, err) | |||
assert.EqualValues(t, 0, mp.SizeBytes()) | |||
// 4. zero after Flush | |||
err = mp.CheckTx(context.Background(), []byte{0x02, 0x03}, nil, mempool.TxInfo{}) | |||
require.NoError(t, err) | |||
assert.EqualValues(t, 2, mp.SizeBytes()) | |||
mp.Flush() | |||
assert.EqualValues(t, 0, mp.SizeBytes()) | |||
// 5. ErrMempoolIsFull is returned when/if MaxTxsBytes limit is reached. | |||
err = mp.CheckTx( | |||
context.Background(), | |||
[]byte{0x04, 0x04, 0x04, 0x04, 0x04, 0x04, 0x04, 0x04, 0x04, 0x04}, | |||
nil, | |||
mempool.TxInfo{}, | |||
) | |||
require.NoError(t, err) | |||
err = mp.CheckTx(context.Background(), []byte{0x05}, nil, mempool.TxInfo{}) | |||
if assert.Error(t, err) { | |||
assert.IsType(t, types.ErrMempoolIsFull{}, err) | |||
} | |||
// 6. zero after tx is rechecked and removed due to not being valid anymore | |||
app2 := kvstore.NewApplication() | |||
cc = abciclient.NewLocalCreator(app2) | |||
mp, cleanup, err = newMempoolWithApp(cc) | |||
require.NoError(t, err) | |||
defer cleanup() | |||
txBytes := make([]byte, 8) | |||
binary.BigEndian.PutUint64(txBytes, uint64(0)) | |||
err = mp.CheckTx(context.Background(), txBytes, nil, mempool.TxInfo{}) | |||
require.NoError(t, err) | |||
assert.EqualValues(t, 8, mp.SizeBytes()) | |||
appConnCon, _ := cc() | |||
appConnCon.SetLogger(log.TestingLogger().With("module", "abci-client", "connection", "consensus")) | |||
err = appConnCon.Start() | |||
require.Nil(t, err) | |||
t.Cleanup(func() { | |||
if err := appConnCon.Stop(); err != nil { | |||
t.Error(err) | |||
} | |||
}) | |||
ctx := context.Background() | |||
res, err := appConnCon.DeliverTxSync(ctx, abci.RequestDeliverTx{Tx: txBytes}) | |||
require.NoError(t, err) | |||
require.EqualValues(t, 0, res.Code) | |||
res2, err := appConnCon.CommitSync(ctx) | |||
require.NoError(t, err) | |||
require.NotEmpty(t, res2.Data) | |||
// Pretend like we committed nothing so txBytes gets rechecked and removed. | |||
err = mp.Update(1, []types.Tx{}, abciResponses(0, abci.CodeTypeOK), nil, nil) | |||
require.NoError(t, err) | |||
assert.EqualValues(t, 8, mp.SizeBytes()) | |||
// 7. Test RemoveTxByKey function | |||
err = mp.CheckTx(context.Background(), []byte{0x06}, nil, mempool.TxInfo{}) | |||
require.NoError(t, err) | |||
assert.EqualValues(t, 9, mp.SizeBytes()) | |||
assert.Error(t, mp.RemoveTxByKey(types.Tx([]byte{0x07}).Key())) | |||
assert.EqualValues(t, 9, mp.SizeBytes()) | |||
assert.NoError(t, mp.RemoveTxByKey(types.Tx([]byte{0x06}).Key())) | |||
assert.EqualValues(t, 8, mp.SizeBytes()) | |||
} | |||
// This will non-deterministically catch some concurrency failures like | |||
// https://github.com/tendermint/tendermint/issues/3509 | |||
// TODO: all of the tests should probably also run using the remote proxy app | |||
// since otherwise we're not actually testing the concurrency of the mempool here! | |||
func TestMempoolRemoteAppConcurrency(t *testing.T) { | |||
sockPath := fmt.Sprintf("unix:///tmp/echo_%v.sock", tmrand.Str(6)) | |||
app := kvstore.NewApplication() | |||
cc, server := newRemoteApp(t, sockPath, app) | |||
t.Cleanup(func() { | |||
if err := server.Stop(); err != nil { | |||
t.Error(err) | |||
} | |||
}) | |||
cfg, err := config.ResetTestRoot("mempool_test") | |||
require.NoError(t, err) | |||
mp, cleanup := newMempoolWithAppAndConfig(cc, cfg) | |||
defer cleanup() | |||
// generate small number of txs | |||
nTxs := 10 | |||
txLen := 200 | |||
txs := make([]types.Tx, nTxs) | |||
for i := 0; i < nTxs; i++ { | |||
txs[i] = tmrand.Bytes(txLen) | |||
} | |||
// simulate a group of peers sending them over and over | |||
N := cfg.Mempool.Size | |||
maxPeers := 5 | |||
for i := 0; i < N; i++ { | |||
peerID := mrand.Intn(maxPeers) | |||
txNum := mrand.Intn(nTxs) | |||
tx := txs[txNum] | |||
// this will err with ErrTxInCache many times ... | |||
mp.CheckTx(context.Background(), tx, nil, mempool.TxInfo{SenderID: uint16(peerID)}) //nolint: errcheck // will error | |||
} | |||
err = mp.FlushAppConn() | |||
require.NoError(t, err) | |||
} | |||
// caller must close server | |||
func newRemoteApp( | |||
t *testing.T, | |||
addr string, | |||
app abci.Application, | |||
) ( | |||
clientCreator abciclient.Creator, | |||
server service.Service, | |||
) { | |||
clientCreator = abciclient.NewRemoteCreator(addr, "socket", true) | |||
// Start server | |||
server = abciserver.NewSocketServer(addr, app) | |||
server.SetLogger(log.TestingLogger().With("module", "abci-server")) | |||
if err := server.Start(); err != nil { | |||
t.Fatalf("Error starting socket server: %v", err.Error()) | |||
} | |||
return clientCreator, server | |||
} | |||
func abciResponses(n int, code uint32) []*abci.ResponseDeliverTx { | |||
responses := make([]*abci.ResponseDeliverTx, 0, n) | |||
for i := 0; i < n; i++ { | |||
responses = append(responses, &abci.ResponseDeliverTx{Code: code}) | |||
} | |||
return responses | |||
} |
@ -1,23 +0,0 @@ | |||
// The mempool pushes new txs onto the proxyAppConn. | |||
// It gets a stream of (req, res) tuples from the proxy. | |||
// The mempool stores good txs in a concurrent linked-list. | |||
// Multiple concurrent go-routines can traverse this linked-list | |||
// safely by calling .NextWait() on each element. | |||
// So we have several go-routines: | |||
// 1. Consensus calling Update() and ReapMaxBytesMaxGas() synchronously | |||
// 2. Many mempool reactor's peer routines calling CheckTx() | |||
// 3. Many mempool reactor's peer routines traversing the txs linked list | |||
// To manage these goroutines, there are three methods of locking. | |||
// 1. Mutations to the linked-list is protected by an internal mtx (CList is goroutine-safe) | |||
// 2. Mutations to the linked-list elements are atomic | |||
// 3. CheckTx() and/or ReapMaxBytesMaxGas() calls can be paused upon Update(), protected by .updateMtx | |||
// Garbage collection of old elements from mempool.txs is handlde via the | |||
// DetachPrev() call, which makes old elements not reachable by peer | |||
// broadcastTxRoutine(). | |||
// TODO: Better handle abci client errors. (make it automatically handle connection errors) | |||
package v0 |
@ -1,392 +0,0 @@ | |||
package v0 | |||
import ( | |||
"context" | |||
"errors" | |||
"fmt" | |||
"runtime/debug" | |||
"sync" | |||
"time" | |||
"github.com/tendermint/tendermint/config" | |||
"github.com/tendermint/tendermint/internal/libs/clist" | |||
tmsync "github.com/tendermint/tendermint/internal/libs/sync" | |||
"github.com/tendermint/tendermint/internal/mempool" | |||
"github.com/tendermint/tendermint/internal/p2p" | |||
"github.com/tendermint/tendermint/libs/log" | |||
"github.com/tendermint/tendermint/libs/service" | |||
protomem "github.com/tendermint/tendermint/proto/tendermint/mempool" | |||
"github.com/tendermint/tendermint/types" | |||
) | |||
var ( | |||
_ service.Service = (*Reactor)(nil) | |||
_ p2p.Wrapper = (*protomem.Message)(nil) | |||
) | |||
// PeerManager defines the interface contract required for getting necessary | |||
// peer information. This should eventually be replaced with a message-oriented | |||
// approach utilizing the p2p stack. | |||
type PeerManager interface { | |||
GetHeight(types.NodeID) int64 | |||
} | |||
// Reactor implements a service that contains mempool of txs that are broadcasted | |||
// amongst peers. It maintains a map from peer ID to counter, to prevent gossiping | |||
// txs to the peers you received it from. | |||
type Reactor struct { | |||
service.BaseService | |||
cfg *config.MempoolConfig | |||
mempool *CListMempool | |||
ids *mempool.IDs | |||
// XXX: Currently, this is the only way to get information about a peer. Ideally, | |||
// we rely on message-oriented communication to get necessary peer data. | |||
// ref: https://github.com/tendermint/tendermint/issues/5670 | |||
peerMgr PeerManager | |||
mempoolCh *p2p.Channel | |||
peerUpdates *p2p.PeerUpdates | |||
closeCh chan struct{} | |||
// peerWG is used to coordinate graceful termination of all peer broadcasting | |||
// goroutines. | |||
peerWG sync.WaitGroup | |||
mtx tmsync.Mutex | |||
peerRoutines map[types.NodeID]*tmsync.Closer | |||
} | |||
// NewReactor returns a reference to a new reactor. | |||
func NewReactor( | |||
logger log.Logger, | |||
cfg *config.MempoolConfig, | |||
peerMgr PeerManager, | |||
mp *CListMempool, | |||
mempoolCh *p2p.Channel, | |||
peerUpdates *p2p.PeerUpdates, | |||
) *Reactor { | |||
r := &Reactor{ | |||
cfg: cfg, | |||
peerMgr: peerMgr, | |||
mempool: mp, | |||
ids: mempool.NewMempoolIDs(), | |||
mempoolCh: mempoolCh, | |||
peerUpdates: peerUpdates, | |||
closeCh: make(chan struct{}), | |||
peerRoutines: make(map[types.NodeID]*tmsync.Closer), | |||
} | |||
r.BaseService = *service.NewBaseService(logger, "Mempool", r) | |||
return r | |||
} | |||
// GetChannelDescriptor produces an instance of a descriptor for this | |||
// package's required channels. | |||
func GetChannelDescriptor(cfg *config.MempoolConfig) *p2p.ChannelDescriptor { | |||
largestTx := make([]byte, cfg.MaxTxBytes) | |||
batchMsg := protomem.Message{ | |||
Sum: &protomem.Message_Txs{ | |||
Txs: &protomem.Txs{Txs: [][]byte{largestTx}}, | |||
}, | |||
} | |||
return &p2p.ChannelDescriptor{ | |||
ID: mempool.MempoolChannel, | |||
MessageType: new(protomem.Message), | |||
Priority: 5, | |||
RecvMessageCapacity: batchMsg.Size(), | |||
RecvBufferCapacity: 128, | |||
} | |||
} | |||
// OnStart starts separate go routines for each p2p Channel and listens for | |||
// envelopes on each. In addition, it also listens for peer updates and handles | |||
// messages on that p2p channel accordingly. The caller must be sure to execute | |||
// OnStop to ensure the outbound p2p Channels are closed. | |||
func (r *Reactor) OnStart() error { | |||
if !r.cfg.Broadcast { | |||
r.Logger.Info("tx broadcasting is disabled") | |||
} | |||
go r.processMempoolCh() | |||
go r.processPeerUpdates() | |||
return nil | |||
} | |||
// OnStop stops the reactor by signaling to all spawned goroutines to exit and | |||
// blocking until they all exit. | |||
func (r *Reactor) OnStop() { | |||
r.mtx.Lock() | |||
for _, c := range r.peerRoutines { | |||
c.Close() | |||
} | |||
r.mtx.Unlock() | |||
// wait for all spawned peer tx broadcasting goroutines to gracefully exit | |||
r.peerWG.Wait() | |||
// Close closeCh to signal to all spawned goroutines to gracefully exit. All | |||
// p2p Channels should execute Close(). | |||
close(r.closeCh) | |||
// Wait for all p2p Channels to be closed before returning. This ensures we | |||
// can easily reason about synchronization of all p2p Channels and ensure no | |||
// panics will occur. | |||
<-r.mempoolCh.Done() | |||
<-r.peerUpdates.Done() | |||
} | |||
// handleMempoolMessage handles envelopes sent from peers on the MempoolChannel. | |||
// For every tx in the message, we execute CheckTx. It returns an error if an | |||
// empty set of txs are sent in an envelope or if we receive an unexpected | |||
// message type. | |||
func (r *Reactor) handleMempoolMessage(envelope p2p.Envelope) error { | |||
logger := r.Logger.With("peer", envelope.From) | |||
switch msg := envelope.Message.(type) { | |||
case *protomem.Txs: | |||
protoTxs := msg.GetTxs() | |||
if len(protoTxs) == 0 { | |||
return errors.New("empty txs received from peer") | |||
} | |||
txInfo := mempool.TxInfo{SenderID: r.ids.GetForPeer(envelope.From)} | |||
if len(envelope.From) != 0 { | |||
txInfo.SenderNodeID = envelope.From | |||
} | |||
for _, tx := range protoTxs { | |||
if err := r.mempool.CheckTx(context.Background(), types.Tx(tx), nil, txInfo); err != nil { | |||
logger.Error("checktx failed for tx", "tx", fmt.Sprintf("%X", types.Tx(tx).Hash()), "err", err) | |||
} | |||
} | |||
default: | |||
return fmt.Errorf("received unknown message: %T", msg) | |||
} | |||
return nil | |||
} | |||
// handleMessage handles an Envelope sent from a peer on a specific p2p Channel. | |||
// It will handle errors and any possible panics gracefully. A caller can handle | |||
// any error returned by sending a PeerError on the respective channel. | |||
func (r *Reactor) handleMessage(chID p2p.ChannelID, envelope p2p.Envelope) (err error) { | |||
defer func() { | |||
if e := recover(); e != nil { | |||
err = fmt.Errorf("panic in processing message: %v", e) | |||
r.Logger.Error( | |||
"recovering from processing message panic", | |||
"err", err, | |||
"stack", string(debug.Stack()), | |||
) | |||
} | |||
}() | |||
r.Logger.Debug("received message", "peer", envelope.From) | |||
switch chID { | |||
case mempool.MempoolChannel: | |||
err = r.handleMempoolMessage(envelope) | |||
default: | |||
err = fmt.Errorf("unknown channel ID (%d) for envelope (%v)", chID, envelope) | |||
} | |||
return err | |||
} | |||
// processMempoolCh implements a blocking event loop where we listen for p2p | |||
// Envelope messages from the mempoolCh. | |||
func (r *Reactor) processMempoolCh() { | |||
defer r.mempoolCh.Close() | |||
for { | |||
select { | |||
case envelope := <-r.mempoolCh.In: | |||
if err := r.handleMessage(r.mempoolCh.ID, envelope); err != nil { | |||
r.Logger.Error("failed to process message", "ch_id", r.mempoolCh.ID, "envelope", envelope, "err", err) | |||
r.mempoolCh.Error <- p2p.PeerError{ | |||
NodeID: envelope.From, | |||
Err: err, | |||
} | |||
} | |||
case <-r.closeCh: | |||
r.Logger.Debug("stopped listening on mempool channel; closing...") | |||
return | |||
} | |||
} | |||
} | |||
// processPeerUpdate processes a PeerUpdate. For added peers, PeerStatusUp, we | |||
// check if the reactor is running and if we've already started a tx broadcasting | |||
// goroutine or not. If not, we start one for the newly added peer. For down or | |||
// removed peers, we remove the peer from the mempool peer ID set and signal to | |||
// stop the tx broadcasting goroutine. | |||
func (r *Reactor) processPeerUpdate(peerUpdate p2p.PeerUpdate) { | |||
r.Logger.Debug("received peer update", "peer", peerUpdate.NodeID, "status", peerUpdate.Status) | |||
r.mtx.Lock() | |||
defer r.mtx.Unlock() | |||
switch peerUpdate.Status { | |||
case p2p.PeerStatusUp: | |||
// Do not allow starting new tx broadcast loops after reactor shutdown | |||
// has been initiated. This can happen after we've manually closed all | |||
// peer broadcast loops and closed r.closeCh, but the router still sends | |||
// in-flight peer updates. | |||
if !r.IsRunning() { | |||
return | |||
} | |||
if r.cfg.Broadcast { | |||
// Check if we've already started a goroutine for this peer, if not we create | |||
// a new done channel so we can explicitly close the goroutine if the peer | |||
// is later removed, we increment the waitgroup so the reactor can stop | |||
// safely, and finally start the goroutine to broadcast txs to that peer. | |||
_, ok := r.peerRoutines[peerUpdate.NodeID] | |||
if !ok { | |||
closer := tmsync.NewCloser() | |||
r.peerRoutines[peerUpdate.NodeID] = closer | |||
r.peerWG.Add(1) | |||
r.ids.ReserveForPeer(peerUpdate.NodeID) | |||
// start a broadcast routine ensuring all txs are forwarded to the peer | |||
go r.broadcastTxRoutine(peerUpdate.NodeID, closer) | |||
} | |||
} | |||
case p2p.PeerStatusDown: | |||
r.ids.Reclaim(peerUpdate.NodeID) | |||
// Check if we've started a tx broadcasting goroutine for this peer. | |||
// If we have, we signal to terminate the goroutine via the channel's closure. | |||
// This will internally decrement the peer waitgroup and remove the peer | |||
// from the map of peer tx broadcasting goroutines. | |||
closer, ok := r.peerRoutines[peerUpdate.NodeID] | |||
if ok { | |||
closer.Close() | |||
} | |||
} | |||
} | |||
// processPeerUpdates initiates a blocking process where we listen for and handle | |||
// PeerUpdate messages. When the reactor is stopped, we will catch the signal and | |||
// close the p2p PeerUpdatesCh gracefully. | |||
func (r *Reactor) processPeerUpdates() { | |||
defer r.peerUpdates.Close() | |||
for { | |||
select { | |||
case peerUpdate := <-r.peerUpdates.Updates(): | |||
r.processPeerUpdate(peerUpdate) | |||
case <-r.closeCh: | |||
r.Logger.Debug("stopped listening on peer updates channel; closing...") | |||
return | |||
} | |||
} | |||
} | |||
func (r *Reactor) broadcastTxRoutine(peerID types.NodeID, closer *tmsync.Closer) { | |||
peerMempoolID := r.ids.GetForPeer(peerID) | |||
var next *clist.CElement | |||
// remove the peer ID from the map of routines and mark the waitgroup as done | |||
defer func() { | |||
r.mtx.Lock() | |||
delete(r.peerRoutines, peerID) | |||
r.mtx.Unlock() | |||
r.peerWG.Done() | |||
if e := recover(); e != nil { | |||
r.Logger.Error( | |||
"recovering from broadcasting mempool loop", | |||
"err", e, | |||
"stack", string(debug.Stack()), | |||
) | |||
} | |||
}() | |||
for { | |||
if !r.IsRunning() { | |||
return | |||
} | |||
// This happens because the CElement we were looking at got garbage | |||
// collected (removed). That is, .NextWait() returned nil. Go ahead and | |||
// start from the beginning. | |||
if next == nil { | |||
select { | |||
case <-r.mempool.TxsWaitChan(): // wait until a tx is available | |||
if next = r.mempool.TxsFront(); next == nil { | |||
continue | |||
} | |||
case <-closer.Done(): | |||
// The peer is marked for removal via a PeerUpdate as the doneCh was | |||
// explicitly closed to signal we should exit. | |||
return | |||
case <-r.closeCh: | |||
// The reactor has signaled that we are stopped and thus we should | |||
// implicitly exit this peer's goroutine. | |||
return | |||
} | |||
} | |||
memTx := next.Value.(*mempoolTx) | |||
if r.peerMgr != nil { | |||
height := r.peerMgr.GetHeight(peerID) | |||
if height > 0 && height < memTx.Height()-1 { | |||
// allow for a lag of one block | |||
time.Sleep(mempool.PeerCatchupSleepIntervalMS * time.Millisecond) | |||
continue | |||
} | |||
} | |||
// NOTE: Transaction batching was disabled due to: | |||
// https://github.com/tendermint/tendermint/issues/5796 | |||
if _, ok := memTx.senders.Load(peerMempoolID); !ok { | |||
// Send the mempool tx to the corresponding peer. Note, the peer may be | |||
// behind and thus would not be able to process the mempool tx correctly. | |||
r.mempoolCh.Out <- p2p.Envelope{ | |||
To: peerID, | |||
Message: &protomem.Txs{ | |||
Txs: [][]byte{memTx.tx}, | |||
}, | |||
} | |||
r.Logger.Debug( | |||
"gossiped tx to peer", | |||
"tx", fmt.Sprintf("%X", memTx.tx.Hash()), | |||
"peer", peerID, | |||
) | |||
} | |||
select { | |||
case <-next.NextWaitChan(): | |||
// see the start of the for loop for nil check | |||
next = next.Next() | |||
case <-closer.Done(): | |||
// The peer is marked for removal via a PeerUpdate as the doneCh was | |||
// explicitly closed to signal we should exit. | |||
return | |||
case <-r.closeCh: | |||
// The reactor has signaled that we are stopped and thus we should | |||
// implicitly exit this peer's goroutine. | |||
return | |||
} | |||
} | |||
} |
@ -1,393 +0,0 @@ | |||
package v0 | |||
import ( | |||
"context" | |||
"sync" | |||
"testing" | |||
"time" | |||
"github.com/stretchr/testify/require" | |||
abciclient "github.com/tendermint/tendermint/abci/client" | |||
"github.com/tendermint/tendermint/abci/example/kvstore" | |||
abci "github.com/tendermint/tendermint/abci/types" | |||
"github.com/tendermint/tendermint/config" | |||
"github.com/tendermint/tendermint/internal/mempool" | |||
"github.com/tendermint/tendermint/internal/p2p" | |||
"github.com/tendermint/tendermint/internal/p2p/p2ptest" | |||
"github.com/tendermint/tendermint/libs/log" | |||
tmrand "github.com/tendermint/tendermint/libs/rand" | |||
protomem "github.com/tendermint/tendermint/proto/tendermint/mempool" | |||
"github.com/tendermint/tendermint/types" | |||
) | |||
type reactorTestSuite struct { | |||
network *p2ptest.Network | |||
logger log.Logger | |||
reactors map[types.NodeID]*Reactor | |||
mempoolChnnels map[types.NodeID]*p2p.Channel | |||
mempools map[types.NodeID]*CListMempool | |||
kvstores map[types.NodeID]*kvstore.Application | |||
peerChans map[types.NodeID]chan p2p.PeerUpdate | |||
peerUpdates map[types.NodeID]*p2p.PeerUpdates | |||
nodes []types.NodeID | |||
} | |||
func setup(t *testing.T, config *config.MempoolConfig, numNodes int, chBuf uint) *reactorTestSuite { | |||
t.Helper() | |||
rts := &reactorTestSuite{ | |||
logger: log.TestingLogger().With("testCase", t.Name()), | |||
network: p2ptest.MakeNetwork(t, p2ptest.NetworkOptions{NumNodes: numNodes}), | |||
reactors: make(map[types.NodeID]*Reactor, numNodes), | |||
mempoolChnnels: make(map[types.NodeID]*p2p.Channel, numNodes), | |||
mempools: make(map[types.NodeID]*CListMempool, numNodes), | |||
kvstores: make(map[types.NodeID]*kvstore.Application, numNodes), | |||
peerChans: make(map[types.NodeID]chan p2p.PeerUpdate, numNodes), | |||
peerUpdates: make(map[types.NodeID]*p2p.PeerUpdates, numNodes), | |||
} | |||
chDesc := GetChannelDescriptor(config) | |||
chDesc.RecvBufferCapacity = int(chBuf) | |||
rts.mempoolChnnels = rts.network.MakeChannelsNoCleanup(t, chDesc) | |||
for nodeID := range rts.network.Nodes { | |||
rts.kvstores[nodeID] = kvstore.NewApplication() | |||
cc := abciclient.NewLocalCreator(rts.kvstores[nodeID]) | |||
mempool, memCleanup, err := newMempoolWithApp(cc) | |||
require.NoError(t, err) | |||
t.Cleanup(memCleanup) | |||
mempool.SetLogger(rts.logger) | |||
rts.mempools[nodeID] = mempool | |||
rts.peerChans[nodeID] = make(chan p2p.PeerUpdate) | |||
rts.peerUpdates[nodeID] = p2p.NewPeerUpdates(rts.peerChans[nodeID], 1) | |||
rts.network.Nodes[nodeID].PeerManager.Register(rts.peerUpdates[nodeID]) | |||
rts.reactors[nodeID] = NewReactor( | |||
rts.logger.With("nodeID", nodeID), | |||
config, | |||
rts.network.Nodes[nodeID].PeerManager, | |||
mempool, | |||
rts.mempoolChnnels[nodeID], | |||
rts.peerUpdates[nodeID], | |||
) | |||
rts.nodes = append(rts.nodes, nodeID) | |||
require.NoError(t, rts.reactors[nodeID].Start()) | |||
require.True(t, rts.reactors[nodeID].IsRunning()) | |||
} | |||
require.Len(t, rts.reactors, numNodes) | |||
t.Cleanup(func() { | |||
for nodeID := range rts.reactors { | |||
if rts.reactors[nodeID].IsRunning() { | |||
require.NoError(t, rts.reactors[nodeID].Stop()) | |||
require.False(t, rts.reactors[nodeID].IsRunning()) | |||
} | |||
} | |||
}) | |||
return rts | |||
} | |||
func (rts *reactorTestSuite) start(t *testing.T) { | |||
t.Helper() | |||
rts.network.Start(t) | |||
require.Len(t, | |||
rts.network.RandomNode().PeerManager.Peers(), | |||
len(rts.nodes)-1, | |||
"network does not have expected number of nodes") | |||
} | |||
func (rts *reactorTestSuite) assertMempoolChannelsDrained(t *testing.T) { | |||
t.Helper() | |||
for id, r := range rts.reactors { | |||
require.NoError(t, r.Stop(), "stopping reactor %s", id) | |||
r.Wait() | |||
require.False(t, r.IsRunning(), "reactor %s did not stop", id) | |||
} | |||
for _, mch := range rts.mempoolChnnels { | |||
require.Empty(t, mch.Out, "checking channel %q (len=%d)", mch.ID, len(mch.Out)) | |||
} | |||
} | |||
func (rts *reactorTestSuite) waitForTxns(t *testing.T, txs types.Txs, ids ...types.NodeID) { | |||
t.Helper() | |||
fn := func(pool *CListMempool) { | |||
for pool.Size() < len(txs) { | |||
time.Sleep(50 * time.Millisecond) | |||
} | |||
reapedTxs := pool.ReapMaxTxs(len(txs)) | |||
require.Equal(t, len(txs), len(reapedTxs)) | |||
for i, tx := range txs { | |||
require.Equalf(t, | |||
tx, | |||
reapedTxs[i], | |||
"txs at index %d in reactor mempool mismatch; got: %v, expected: %v", i, tx, reapedTxs[i], | |||
) | |||
} | |||
} | |||
if len(ids) == 1 { | |||
fn(rts.reactors[ids[0]].mempool) | |||
return | |||
} | |||
wg := &sync.WaitGroup{} | |||
for id := range rts.mempools { | |||
if len(ids) > 0 && !p2ptest.NodeInSlice(id, ids) { | |||
continue | |||
} | |||
wg.Add(1) | |||
func(nid types.NodeID) { defer wg.Done(); fn(rts.reactors[nid].mempool) }(id) | |||
} | |||
wg.Wait() | |||
} | |||
func TestReactorBroadcastTxs(t *testing.T) { | |||
numTxs := 1000 | |||
numNodes := 10 | |||
cfg := config.TestConfig() | |||
rts := setup(t, cfg.Mempool, numNodes, 0) | |||
primary := rts.nodes[0] | |||
secondaries := rts.nodes[1:] | |||
txs := checkTxs(t, rts.reactors[primary].mempool, numTxs, mempool.UnknownPeerID) | |||
// run the router | |||
rts.start(t) | |||
// Wait till all secondary suites (reactor) received all mempool txs from the | |||
// primary suite (node). | |||
rts.waitForTxns(t, txs, secondaries...) | |||
for _, pool := range rts.mempools { | |||
require.Equal(t, len(txs), pool.Size()) | |||
} | |||
rts.assertMempoolChannelsDrained(t) | |||
} | |||
// regression test for https://github.com/tendermint/tendermint/issues/5408 | |||
func TestReactorConcurrency(t *testing.T) { | |||
numTxs := 5 | |||
numNodes := 2 | |||
cfg := config.TestConfig() | |||
rts := setup(t, cfg.Mempool, numNodes, 0) | |||
primary := rts.nodes[0] | |||
secondary := rts.nodes[1] | |||
rts.start(t) | |||
var wg sync.WaitGroup | |||
for i := 0; i < 1000; i++ { | |||
wg.Add(2) | |||
// 1. submit a bunch of txs | |||
// 2. update the whole mempool | |||
txs := checkTxs(t, rts.reactors[primary].mempool, numTxs, mempool.UnknownPeerID) | |||
go func() { | |||
defer wg.Done() | |||
mempool := rts.mempools[primary] | |||
mempool.Lock() | |||
defer mempool.Unlock() | |||
deliverTxResponses := make([]*abci.ResponseDeliverTx, len(txs)) | |||
for i := range txs { | |||
deliverTxResponses[i] = &abci.ResponseDeliverTx{Code: 0} | |||
} | |||
require.NoError(t, mempool.Update(1, txs, deliverTxResponses, nil, nil)) | |||
}() | |||
// 1. submit a bunch of txs | |||
// 2. update none | |||
_ = checkTxs(t, rts.reactors[secondary].mempool, numTxs, mempool.UnknownPeerID) | |||
go func() { | |||
defer wg.Done() | |||
mempool := rts.mempools[secondary] | |||
mempool.Lock() | |||
defer mempool.Unlock() | |||
err := mempool.Update(1, []types.Tx{}, make([]*abci.ResponseDeliverTx, 0), nil, nil) | |||
require.NoError(t, err) | |||
}() | |||
// flush the mempool | |||
rts.mempools[secondary].Flush() | |||
} | |||
wg.Wait() | |||
} | |||
func TestReactorNoBroadcastToSender(t *testing.T) { | |||
numTxs := 1000 | |||
numNodes := 2 | |||
cfg := config.TestConfig() | |||
rts := setup(t, cfg.Mempool, numNodes, uint(numTxs)) | |||
primary := rts.nodes[0] | |||
secondary := rts.nodes[1] | |||
peerID := uint16(1) | |||
_ = checkTxs(t, rts.mempools[primary], numTxs, peerID) | |||
rts.start(t) | |||
time.Sleep(100 * time.Millisecond) | |||
require.Eventually(t, func() bool { | |||
return rts.mempools[secondary].Size() == 0 | |||
}, time.Minute, 100*time.Millisecond) | |||
rts.assertMempoolChannelsDrained(t) | |||
} | |||
func TestReactor_MaxTxBytes(t *testing.T) { | |||
numNodes := 2 | |||
cfg := config.TestConfig() | |||
rts := setup(t, cfg.Mempool, numNodes, 0) | |||
primary := rts.nodes[0] | |||
secondary := rts.nodes[1] | |||
// Broadcast a tx, which has the max size and ensure it's received by the | |||
// second reactor. | |||
tx1 := tmrand.Bytes(cfg.Mempool.MaxTxBytes) | |||
err := rts.reactors[primary].mempool.CheckTx( | |||
context.Background(), | |||
tx1, | |||
nil, | |||
mempool.TxInfo{ | |||
SenderID: mempool.UnknownPeerID, | |||
}, | |||
) | |||
require.NoError(t, err) | |||
rts.start(t) | |||
// Wait till all secondary suites (reactor) received all mempool txs from the | |||
// primary suite (node). | |||
rts.waitForTxns(t, []types.Tx{tx1}, secondary) | |||
rts.reactors[primary].mempool.Flush() | |||
rts.reactors[secondary].mempool.Flush() | |||
// broadcast a tx, which is beyond the max size and ensure it's not sent | |||
tx2 := tmrand.Bytes(cfg.Mempool.MaxTxBytes + 1) | |||
err = rts.mempools[primary].CheckTx(context.Background(), tx2, nil, mempool.TxInfo{SenderID: mempool.UnknownPeerID}) | |||
require.Error(t, err) | |||
rts.assertMempoolChannelsDrained(t) | |||
} | |||
func TestDontExhaustMaxActiveIDs(t *testing.T) { | |||
cfg := config.TestConfig() | |||
// we're creating a single node network, but not starting the | |||
// network. | |||
rts := setup(t, cfg.Mempool, 1, mempool.MaxActiveIDs+1) | |||
nodeID := rts.nodes[0] | |||
peerID, err := types.NewNodeID("0011223344556677889900112233445566778899") | |||
require.NoError(t, err) | |||
// ensure the reactor does not panic (i.e. exhaust active IDs) | |||
for i := 0; i < mempool.MaxActiveIDs+1; i++ { | |||
rts.peerChans[nodeID] <- p2p.PeerUpdate{ | |||
Status: p2p.PeerStatusUp, | |||
NodeID: peerID, | |||
} | |||
rts.mempoolChnnels[nodeID].Out <- p2p.Envelope{ | |||
To: peerID, | |||
Message: &protomem.Txs{ | |||
Txs: [][]byte{}, | |||
}, | |||
} | |||
} | |||
require.Eventually( | |||
t, | |||
func() bool { | |||
for _, mch := range rts.mempoolChnnels { | |||
if len(mch.Out) > 0 { | |||
return false | |||
} | |||
} | |||
return true | |||
}, | |||
time.Minute, | |||
10*time.Millisecond, | |||
) | |||
rts.assertMempoolChannelsDrained(t) | |||
} | |||
func TestMempoolIDsPanicsIfNodeRequestsOvermaxActiveIDs(t *testing.T) { | |||
if testing.Short() { | |||
t.Skip("skipping test in short mode") | |||
} | |||
// 0 is already reserved for UnknownPeerID | |||
ids := mempool.NewMempoolIDs() | |||
peerID, err := types.NewNodeID("0011223344556677889900112233445566778899") | |||
require.NoError(t, err) | |||
for i := 0; i < mempool.MaxActiveIDs-1; i++ { | |||
ids.ReserveForPeer(peerID) | |||
} | |||
require.Panics(t, func() { | |||
ids.ReserveForPeer(peerID) | |||
}) | |||
} | |||
func TestBroadcastTxForPeerStopsWhenPeerStops(t *testing.T) { | |||
if testing.Short() { | |||
t.Skip("skipping test in short mode") | |||
} | |||
cfg := config.TestConfig() | |||
rts := setup(t, cfg.Mempool, 2, 0) | |||
primary := rts.nodes[0] | |||
secondary := rts.nodes[1] | |||
rts.start(t) | |||
// disconnect peer | |||
rts.peerChans[primary] <- p2p.PeerUpdate{ | |||
Status: p2p.PeerStatusDown, | |||
NodeID: secondary, | |||
} | |||
} |
@ -1,887 +0,0 @@ | |||
package v1 | |||
import ( | |||
"bytes" | |||
"context" | |||
"errors" | |||
"fmt" | |||
"reflect" | |||
"sync/atomic" | |||
"time" | |||
abci "github.com/tendermint/tendermint/abci/types" | |||
"github.com/tendermint/tendermint/config" | |||
"github.com/tendermint/tendermint/internal/libs/clist" | |||
tmsync "github.com/tendermint/tendermint/internal/libs/sync" | |||
"github.com/tendermint/tendermint/internal/mempool" | |||
"github.com/tendermint/tendermint/internal/proxy" | |||
"github.com/tendermint/tendermint/libs/log" | |||
tmmath "github.com/tendermint/tendermint/libs/math" | |||
"github.com/tendermint/tendermint/types" | |||
) | |||
var _ mempool.Mempool = (*TxMempool)(nil) | |||
// TxMempoolOption sets an optional parameter on the TxMempool. | |||
type TxMempoolOption func(*TxMempool) | |||
// TxMempool defines a prioritized mempool data structure used by the v1 mempool | |||
// reactor. It keeps a thread-safe priority queue of transactions that is used | |||
// when a block proposer constructs a block and a thread-safe linked-list that | |||
// is used to gossip transactions to peers in a FIFO manner. | |||
type TxMempool struct { | |||
logger log.Logger | |||
metrics *mempool.Metrics | |||
config *config.MempoolConfig | |||
proxyAppConn proxy.AppConnMempool | |||
// txsAvailable fires once for each height when the mempool is not empty | |||
txsAvailable chan struct{} | |||
notifiedTxsAvailable bool | |||
// height defines the last block height process during Update() | |||
height int64 | |||
// sizeBytes defines the total size of the mempool (sum of all tx bytes) | |||
sizeBytes int64 | |||
// cache defines a fixed-size cache of already seen transactions as this | |||
// reduces pressure on the proxyApp. | |||
cache mempool.TxCache | |||
// txStore defines the main storage of valid transactions. Indexes are built | |||
// on top of this store. | |||
txStore *TxStore | |||
// gossipIndex defines the gossiping index of valid transactions via a | |||
// thread-safe linked-list. We also use the gossip index as a cursor for | |||
// rechecking transactions already in the mempool. | |||
gossipIndex *clist.CList | |||
// recheckCursor and recheckEnd are used as cursors based on the gossip index | |||
// to recheck transactions that are already in the mempool. Iteration is not | |||
// thread-safe and transaction may be mutated in serial order. | |||
// | |||
// XXX/TODO: It might be somewhat of a codesmell to use the gossip index for | |||
// iterator and cursor management when rechecking transactions. If the gossip | |||
// index changes or is removed in a future refactor, this will have to be | |||
// refactored. Instead, we should consider just keeping a slice of a snapshot | |||
// of the mempool's current transactions during Update and an integer cursor | |||
// into that slice. This, however, requires additional O(n) space complexity. | |||
recheckCursor *clist.CElement // next expected response | |||
recheckEnd *clist.CElement // re-checking stops here | |||
// priorityIndex defines the priority index of valid transactions via a | |||
// thread-safe priority queue. | |||
priorityIndex *TxPriorityQueue | |||
// heightIndex defines a height-based, in ascending order, transaction index. | |||
// i.e. older transactions are first. | |||
heightIndex *WrappedTxList | |||
// timestampIndex defines a timestamp-based, in ascending order, transaction | |||
// index. i.e. older transactions are first. | |||
timestampIndex *WrappedTxList | |||
// A read/write lock is used to safe guard updates, insertions and deletions | |||
// from the mempool. A read-lock is implicitly acquired when executing CheckTx, | |||
// however, a caller must explicitly grab a write-lock via Lock when updating | |||
// the mempool via Update(). | |||
mtx tmsync.RWMutex | |||
preCheck mempool.PreCheckFunc | |||
postCheck mempool.PostCheckFunc | |||
} | |||
func NewTxMempool( | |||
logger log.Logger, | |||
cfg *config.MempoolConfig, | |||
proxyAppConn proxy.AppConnMempool, | |||
height int64, | |||
options ...TxMempoolOption, | |||
) *TxMempool { | |||
txmp := &TxMempool{ | |||
logger: logger, | |||
config: cfg, | |||
proxyAppConn: proxyAppConn, | |||
height: height, | |||
cache: mempool.NopTxCache{}, | |||
metrics: mempool.NopMetrics(), | |||
txStore: NewTxStore(), | |||
gossipIndex: clist.New(), | |||
priorityIndex: NewTxPriorityQueue(), | |||
heightIndex: NewWrappedTxList(func(wtx1, wtx2 *WrappedTx) bool { | |||
return wtx1.height >= wtx2.height | |||
}), | |||
timestampIndex: NewWrappedTxList(func(wtx1, wtx2 *WrappedTx) bool { | |||
return wtx1.timestamp.After(wtx2.timestamp) || wtx1.timestamp.Equal(wtx2.timestamp) | |||
}), | |||
} | |||
if cfg.CacheSize > 0 { | |||
txmp.cache = mempool.NewLRUTxCache(cfg.CacheSize) | |||
} | |||
proxyAppConn.SetResponseCallback(txmp.defaultTxCallback) | |||
for _, opt := range options { | |||
opt(txmp) | |||
} | |||
return txmp | |||
} | |||
// WithPreCheck sets a filter for the mempool to reject a transaction if f(tx) | |||
// returns an error. This is executed before CheckTx. It only applies to the | |||
// first created block. After that, Update() overwrites the existing value. | |||
func WithPreCheck(f mempool.PreCheckFunc) TxMempoolOption { | |||
return func(txmp *TxMempool) { txmp.preCheck = f } | |||
} | |||
// WithPostCheck sets a filter for the mempool to reject a transaction if | |||
// f(tx, resp) returns an error. This is executed after CheckTx. It only applies | |||
// to the first created block. After that, Update overwrites the existing value. | |||
func WithPostCheck(f mempool.PostCheckFunc) TxMempoolOption { | |||
return func(txmp *TxMempool) { txmp.postCheck = f } | |||
} | |||
// WithMetrics sets the mempool's metrics collector. | |||
func WithMetrics(metrics *mempool.Metrics) TxMempoolOption { | |||
return func(txmp *TxMempool) { txmp.metrics = metrics } | |||
} | |||
// Lock obtains a write-lock on the mempool. A caller must be sure to explicitly | |||
// release the lock when finished. | |||
func (txmp *TxMempool) Lock() { | |||
txmp.mtx.Lock() | |||
} | |||
// Unlock releases a write-lock on the mempool. | |||
func (txmp *TxMempool) Unlock() { | |||
txmp.mtx.Unlock() | |||
} | |||
// Size returns the number of valid transactions in the mempool. It is | |||
// thread-safe. | |||
func (txmp *TxMempool) Size() int { | |||
return txmp.txStore.Size() | |||
} | |||
// SizeBytes return the total sum in bytes of all the valid transactions in the | |||
// mempool. It is thread-safe. | |||
func (txmp *TxMempool) SizeBytes() int64 { | |||
return atomic.LoadInt64(&txmp.sizeBytes) | |||
} | |||
// FlushAppConn executes FlushSync on the mempool's proxyAppConn. | |||
// | |||
// NOTE: The caller must obtain a write-lock via Lock() prior to execution. | |||
func (txmp *TxMempool) FlushAppConn() error { | |||
return txmp.proxyAppConn.FlushSync(context.Background()) | |||
} | |||
// WaitForNextTx returns a blocking channel that will be closed when the next | |||
// valid transaction is available to gossip. It is thread-safe. | |||
func (txmp *TxMempool) WaitForNextTx() <-chan struct{} { | |||
return txmp.gossipIndex.WaitChan() | |||
} | |||
// NextGossipTx returns the next valid transaction to gossip. A caller must wait | |||
// for WaitForNextTx to signal a transaction is available to gossip first. It is | |||
// thread-safe. | |||
func (txmp *TxMempool) NextGossipTx() *clist.CElement { | |||
return txmp.gossipIndex.Front() | |||
} | |||
// EnableTxsAvailable enables the mempool to trigger events when transactions | |||
// are available on a block by block basis. | |||
func (txmp *TxMempool) EnableTxsAvailable() { | |||
txmp.mtx.Lock() | |||
defer txmp.mtx.Unlock() | |||
txmp.txsAvailable = make(chan struct{}, 1) | |||
} | |||
// TxsAvailable returns a channel which fires once for every height, and only | |||
// when transactions are available in the mempool. It is thread-safe. | |||
func (txmp *TxMempool) TxsAvailable() <-chan struct{} { | |||
return txmp.txsAvailable | |||
} | |||
// CheckTx executes the ABCI CheckTx method for a given transaction. It acquires | |||
// a read-lock attempts to execute the application's CheckTx ABCI method via | |||
// CheckTxAsync. We return an error if any of the following happen: | |||
// | |||
// - The CheckTxAsync execution fails. | |||
// - The transaction already exists in the cache and we've already received the | |||
// transaction from the peer. Otherwise, if it solely exists in the cache, we | |||
// return nil. | |||
// - The transaction size exceeds the maximum transaction size as defined by the | |||
// configuration provided to the mempool. | |||
// - The transaction fails Pre-Check (if it is defined). | |||
// - The proxyAppConn fails, e.g. the buffer is full. | |||
// | |||
// If the mempool is full, we still execute CheckTx and attempt to find a lower | |||
// priority transaction to evict. If such a transaction exists, we remove the | |||
// lower priority transaction and add the new one with higher priority. | |||
// | |||
// NOTE: | |||
// - The applications' CheckTx implementation may panic. | |||
// - The caller is not to explicitly require any locks for executing CheckTx. | |||
func (txmp *TxMempool) CheckTx( | |||
ctx context.Context, | |||
tx types.Tx, | |||
cb func(*abci.Response), | |||
txInfo mempool.TxInfo, | |||
) error { | |||
txmp.mtx.RLock() | |||
defer txmp.mtx.RUnlock() | |||
txSize := len(tx) | |||
if txSize > txmp.config.MaxTxBytes { | |||
return types.ErrTxTooLarge{ | |||
Max: txmp.config.MaxTxBytes, | |||
Actual: txSize, | |||
} | |||
} | |||
if txmp.preCheck != nil { | |||
if err := txmp.preCheck(tx); err != nil { | |||
return types.ErrPreCheck{ | |||
Reason: err, | |||
} | |||
} | |||
} | |||
if err := txmp.proxyAppConn.Error(); err != nil { | |||
return err | |||
} | |||
txHash := tx.Key() | |||
// We add the transaction to the mempool's cache and if the transaction already | |||
// exists, i.e. false is returned, then we check if we've seen this transaction | |||
// from the same sender and error if we have. Otherwise, we return nil. | |||
if !txmp.cache.Push(tx) { | |||
wtx, ok := txmp.txStore.GetOrSetPeerByTxHash(txHash, txInfo.SenderID) | |||
if wtx != nil && ok { | |||
// We already have the transaction stored and the we've already seen this | |||
// transaction from txInfo.SenderID. | |||
return types.ErrTxInCache | |||
} | |||
txmp.logger.Debug("tx exists already in cache", "tx_hash", tx.Hash()) | |||
return nil | |||
} | |||
if ctx == nil { | |||
ctx = context.Background() | |||
} | |||
reqRes, err := txmp.proxyAppConn.CheckTxAsync(ctx, abci.RequestCheckTx{Tx: tx}) | |||
if err != nil { | |||
txmp.cache.Remove(tx) | |||
return err | |||
} | |||
reqRes.SetCallback(func(res *abci.Response) { | |||
if txmp.recheckCursor != nil { | |||
panic("recheck cursor is non-nil in CheckTx callback") | |||
} | |||
wtx := &WrappedTx{ | |||
tx: tx, | |||
hash: txHash, | |||
timestamp: time.Now().UTC(), | |||
height: txmp.height, | |||
} | |||
txmp.initTxCallback(wtx, res, txInfo) | |||
if cb != nil { | |||
cb(res) | |||
} | |||
}) | |||
return nil | |||
} | |||
func (txmp *TxMempool) RemoveTxByKey(txKey types.TxKey) error { | |||
txmp.Lock() | |||
defer txmp.Unlock() | |||
// remove the committed transaction from the transaction store and indexes | |||
if wtx := txmp.txStore.GetTxByHash(txKey); wtx != nil { | |||
txmp.removeTx(wtx, false) | |||
return nil | |||
} | |||
return errors.New("transaction not found") | |||
} | |||
// Flush flushes out the mempool. It acquires a read-lock, fetches all the | |||
// transactions currently in the transaction store and removes each transaction | |||
// from the store and all indexes and finally resets the cache. | |||
// | |||
// NOTE: | |||
// - Flushing the mempool may leave the mempool in an inconsistent state. | |||
func (txmp *TxMempool) Flush() { | |||
txmp.mtx.RLock() | |||
defer txmp.mtx.RUnlock() | |||
txmp.heightIndex.Reset() | |||
txmp.timestampIndex.Reset() | |||
for _, wtx := range txmp.txStore.GetAllTxs() { | |||
txmp.removeTx(wtx, false) | |||
} | |||
atomic.SwapInt64(&txmp.sizeBytes, 0) | |||
txmp.cache.Reset() | |||
} | |||
// ReapMaxBytesMaxGas returns a list of transactions within the provided size | |||
// and gas constraints. Transaction are retrieved in priority order. | |||
// | |||
// NOTE: | |||
// - A read-lock is acquired. | |||
// - Transactions returned are not actually removed from the mempool transaction | |||
// store or indexes. | |||
func (txmp *TxMempool) ReapMaxBytesMaxGas(maxBytes, maxGas int64) types.Txs { | |||
txmp.mtx.RLock() | |||
defer txmp.mtx.RUnlock() | |||
var ( | |||
totalGas int64 | |||
totalSize int64 | |||
) | |||
// wTxs contains a list of *WrappedTx retrieved from the priority queue that | |||
// need to be re-enqueued prior to returning. | |||
wTxs := make([]*WrappedTx, 0, txmp.priorityIndex.NumTxs()) | |||
defer func() { | |||
for _, wtx := range wTxs { | |||
txmp.priorityIndex.PushTx(wtx) | |||
} | |||
}() | |||
txs := make([]types.Tx, 0, txmp.priorityIndex.NumTxs()) | |||
for txmp.priorityIndex.NumTxs() > 0 { | |||
wtx := txmp.priorityIndex.PopTx() | |||
txs = append(txs, wtx.tx) | |||
wTxs = append(wTxs, wtx) | |||
size := types.ComputeProtoSizeForTxs([]types.Tx{wtx.tx}) | |||
// Ensure we have capacity for the transaction with respect to the | |||
// transaction size. | |||
if maxBytes > -1 && totalSize+size > maxBytes { | |||
return txs[:len(txs)-1] | |||
} | |||
totalSize += size | |||
// ensure we have capacity for the transaction with respect to total gas | |||
gas := totalGas + wtx.gasWanted | |||
if maxGas > -1 && gas > maxGas { | |||
return txs[:len(txs)-1] | |||
} | |||
totalGas = gas | |||
} | |||
return txs | |||
} | |||
// ReapMaxTxs returns a list of transactions within the provided number of | |||
// transactions bound. Transaction are retrieved in priority order. | |||
// | |||
// NOTE: | |||
// - A read-lock is acquired. | |||
// - Transactions returned are not actually removed from the mempool transaction | |||
// store or indexes. | |||
func (txmp *TxMempool) ReapMaxTxs(max int) types.Txs { | |||
txmp.mtx.RLock() | |||
defer txmp.mtx.RUnlock() | |||
numTxs := txmp.priorityIndex.NumTxs() | |||
if max < 0 { | |||
max = numTxs | |||
} | |||
cap := tmmath.MinInt(numTxs, max) | |||
// wTxs contains a list of *WrappedTx retrieved from the priority queue that | |||
// need to be re-enqueued prior to returning. | |||
wTxs := make([]*WrappedTx, 0, cap) | |||
defer func() { | |||
for _, wtx := range wTxs { | |||
txmp.priorityIndex.PushTx(wtx) | |||
} | |||
}() | |||
txs := make([]types.Tx, 0, cap) | |||
for txmp.priorityIndex.NumTxs() > 0 && len(txs) < max { | |||
wtx := txmp.priorityIndex.PopTx() | |||
txs = append(txs, wtx.tx) | |||
wTxs = append(wTxs, wtx) | |||
} | |||
return txs | |||
} | |||
// Update iterates over all the transactions provided by the caller, i.e. the | |||
// block producer, and removes them from the cache (if applicable) and removes | |||
// the transactions from the main transaction store and associated indexes. | |||
// Finally, if there are trainsactions remaining in the mempool, we initiate a | |||
// re-CheckTx for them (if applicable), otherwise, we notify the caller more | |||
// transactions are available. | |||
// | |||
// NOTE: | |||
// - The caller must explicitly acquire a write-lock via Lock(). | |||
func (txmp *TxMempool) Update( | |||
blockHeight int64, | |||
blockTxs types.Txs, | |||
deliverTxResponses []*abci.ResponseDeliverTx, | |||
newPreFn mempool.PreCheckFunc, | |||
newPostFn mempool.PostCheckFunc, | |||
) error { | |||
txmp.height = blockHeight | |||
txmp.notifiedTxsAvailable = false | |||
if newPreFn != nil { | |||
txmp.preCheck = newPreFn | |||
} | |||
if newPostFn != nil { | |||
txmp.postCheck = newPostFn | |||
} | |||
for i, tx := range blockTxs { | |||
if deliverTxResponses[i].Code == abci.CodeTypeOK { | |||
// add the valid committed transaction to the cache (if missing) | |||
_ = txmp.cache.Push(tx) | |||
} else if !txmp.config.KeepInvalidTxsInCache { | |||
// allow invalid transactions to be re-submitted | |||
txmp.cache.Remove(tx) | |||
} | |||
// remove the committed transaction from the transaction store and indexes | |||
if wtx := txmp.txStore.GetTxByHash(tx.Key()); wtx != nil { | |||
txmp.removeTx(wtx, false) | |||
} | |||
} | |||
txmp.purgeExpiredTxs(blockHeight) | |||
// If there any uncommitted transactions left in the mempool, we either | |||
// initiate re-CheckTx per remaining transaction or notify that remaining | |||
// transactions are left. | |||
if txmp.Size() > 0 { | |||
if txmp.config.Recheck { | |||
txmp.logger.Debug( | |||
"executing re-CheckTx for all remaining transactions", | |||
"num_txs", txmp.Size(), | |||
"height", blockHeight, | |||
) | |||
txmp.updateReCheckTxs() | |||
} else { | |||
txmp.notifyTxsAvailable() | |||
} | |||
} | |||
txmp.metrics.Size.Set(float64(txmp.Size())) | |||
return nil | |||
} | |||
// initTxCallback performs the initial, i.e. the first, callback after CheckTx | |||
// has been executed by the ABCI application. In other words, initTxCallback is | |||
// called after executing CheckTx when we see a unique transaction for the first | |||
// time. CheckTx can be called again for the same transaction at a later point | |||
// in time when re-checking, however, this callback will not be called. | |||
// | |||
// After the ABCI application executes CheckTx, initTxCallback is called with | |||
// the ABCI *Response object and TxInfo. If postCheck is defined on the mempool, | |||
// we execute that first. If there is no error from postCheck (if defined) and | |||
// the ABCI CheckTx response code is OK, we attempt to insert the transaction. | |||
// | |||
// When attempting to insert the transaction, we first check if there is | |||
// sufficient capacity. If there is sufficient capacity, the transaction is | |||
// inserted into the txStore and indexed across all indexes. Otherwise, if the | |||
// mempool is full, we attempt to find a lower priority transaction to evict in | |||
// place of the new incoming transaction. If no such transaction exists, the | |||
// new incoming transaction is rejected. | |||
// | |||
// If the new incoming transaction fails CheckTx or postCheck fails, we reject | |||
// the new incoming transaction. | |||
// | |||
// NOTE: | |||
// - An explicit lock is NOT required. | |||
func (txmp *TxMempool) initTxCallback(wtx *WrappedTx, res *abci.Response, txInfo mempool.TxInfo) { | |||
checkTxRes, ok := res.Value.(*abci.Response_CheckTx) | |||
if !ok { | |||
return | |||
} | |||
var err error | |||
if txmp.postCheck != nil { | |||
err = txmp.postCheck(wtx.tx, checkTxRes.CheckTx) | |||
} | |||
if err != nil || checkTxRes.CheckTx.Code != abci.CodeTypeOK { | |||
// ignore bad transactions | |||
txmp.logger.Info( | |||
"rejected bad transaction", | |||
"priority", wtx.priority, | |||
"tx", fmt.Sprintf("%X", wtx.tx.Hash()), | |||
"peer_id", txInfo.SenderNodeID, | |||
"code", checkTxRes.CheckTx.Code, | |||
"post_check_err", err, | |||
) | |||
txmp.metrics.FailedTxs.Add(1) | |||
if !txmp.config.KeepInvalidTxsInCache { | |||
txmp.cache.Remove(wtx.tx) | |||
} | |||
if err != nil { | |||
checkTxRes.CheckTx.MempoolError = err.Error() | |||
} | |||
return | |||
} | |||
sender := checkTxRes.CheckTx.Sender | |||
priority := checkTxRes.CheckTx.Priority | |||
if len(sender) > 0 { | |||
if wtx := txmp.txStore.GetTxBySender(sender); wtx != nil { | |||
txmp.logger.Error( | |||
"rejected incoming good transaction; tx already exists for sender", | |||
"tx", fmt.Sprintf("%X", wtx.tx.Hash()), | |||
"sender", sender, | |||
) | |||
txmp.metrics.RejectedTxs.Add(1) | |||
return | |||
} | |||
} | |||
if err := txmp.canAddTx(wtx); err != nil { | |||
evictTxs := txmp.priorityIndex.GetEvictableTxs( | |||
priority, | |||
int64(wtx.Size()), | |||
txmp.SizeBytes(), | |||
txmp.config.MaxTxsBytes, | |||
) | |||
if len(evictTxs) == 0 { | |||
// No room for the new incoming transaction so we just remove it from | |||
// the cache. | |||
txmp.cache.Remove(wtx.tx) | |||
txmp.logger.Error( | |||
"rejected incoming good transaction; mempool full", | |||
"tx", fmt.Sprintf("%X", wtx.tx.Hash()), | |||
"err", err.Error(), | |||
) | |||
txmp.metrics.RejectedTxs.Add(1) | |||
return | |||
} | |||
// evict an existing transaction(s) | |||
// | |||
// NOTE: | |||
// - The transaction, toEvict, can be removed while a concurrent | |||
// reCheckTx callback is being executed for the same transaction. | |||
for _, toEvict := range evictTxs { | |||
txmp.removeTx(toEvict, true) | |||
txmp.logger.Debug( | |||
"evicted existing good transaction; mempool full", | |||
"old_tx", fmt.Sprintf("%X", toEvict.tx.Hash()), | |||
"old_priority", toEvict.priority, | |||
"new_tx", fmt.Sprintf("%X", wtx.tx.Hash()), | |||
"new_priority", wtx.priority, | |||
) | |||
txmp.metrics.EvictedTxs.Add(1) | |||
} | |||
} | |||
wtx.gasWanted = checkTxRes.CheckTx.GasWanted | |||
wtx.priority = priority | |||
wtx.sender = sender | |||
wtx.peers = map[uint16]struct{}{ | |||
txInfo.SenderID: {}, | |||
} | |||
txmp.metrics.TxSizeBytes.Observe(float64(wtx.Size())) | |||
txmp.metrics.Size.Set(float64(txmp.Size())) | |||
txmp.insertTx(wtx) | |||
txmp.logger.Debug( | |||
"inserted good transaction", | |||
"priority", wtx.priority, | |||
"tx", fmt.Sprintf("%X", wtx.tx.Hash()), | |||
"height", txmp.height, | |||
"num_txs", txmp.Size(), | |||
) | |||
txmp.notifyTxsAvailable() | |||
} | |||
// defaultTxCallback performs the default CheckTx application callback. This is | |||
// NOT executed when a transaction is first seen/received. Instead, this callback | |||
// is executed during re-checking transactions (if enabled). A caller, i.e a | |||
// block proposer, acquires a mempool write-lock via Lock() and when executing | |||
// Update(), if the mempool is non-empty and Recheck is enabled, then all | |||
// remaining transactions will be rechecked via CheckTxAsync. The order in which | |||
// they are rechecked must be the same order in which this callback is called | |||
// per transaction. | |||
func (txmp *TxMempool) defaultTxCallback(req *abci.Request, res *abci.Response) { | |||
if txmp.recheckCursor == nil { | |||
return | |||
} | |||
txmp.metrics.RecheckTimes.Add(1) | |||
checkTxRes, ok := res.Value.(*abci.Response_CheckTx) | |||
if !ok { | |||
txmp.logger.Error("received incorrect type in mempool callback", | |||
"expected", reflect.TypeOf(&abci.Response_CheckTx{}).Name(), | |||
"got", reflect.TypeOf(res.Value).Name(), | |||
) | |||
return | |||
} | |||
tx := req.GetCheckTx().Tx | |||
wtx := txmp.recheckCursor.Value.(*WrappedTx) | |||
// Search through the remaining list of tx to recheck for a transaction that matches | |||
// the one we received from the ABCI application. | |||
for { | |||
if bytes.Equal(tx, wtx.tx) { | |||
// We've found a tx in the recheck list that matches the tx that we | |||
// received from the ABCI application. | |||
// Break, and use this transaction for further checks. | |||
break | |||
} | |||
txmp.logger.Error( | |||
"re-CheckTx transaction mismatch", | |||
"got", wtx.tx.Hash(), | |||
"expected", types.Tx(tx).Key(), | |||
) | |||
if txmp.recheckCursor == txmp.recheckEnd { | |||
// we reached the end of the recheckTx list without finding a tx | |||
// matching the one we received from the ABCI application. | |||
// Return without processing any tx. | |||
txmp.recheckCursor = nil | |||
return | |||
} | |||
txmp.recheckCursor = txmp.recheckCursor.Next() | |||
wtx = txmp.recheckCursor.Value.(*WrappedTx) | |||
} | |||
// Only evaluate transactions that have not been removed. This can happen | |||
// if an existing transaction is evicted during CheckTx and while this | |||
// callback is being executed for the same evicted transaction. | |||
if !txmp.txStore.IsTxRemoved(wtx.hash) { | |||
var err error | |||
if txmp.postCheck != nil { | |||
err = txmp.postCheck(tx, checkTxRes.CheckTx) | |||
} | |||
if checkTxRes.CheckTx.Code == abci.CodeTypeOK && err == nil { | |||
wtx.priority = checkTxRes.CheckTx.Priority | |||
} else { | |||
txmp.logger.Debug( | |||
"existing transaction no longer valid; failed re-CheckTx callback", | |||
"priority", wtx.priority, | |||
"tx", fmt.Sprintf("%X", wtx.tx.Hash()), | |||
"err", err, | |||
"code", checkTxRes.CheckTx.Code, | |||
) | |||
if wtx.gossipEl != txmp.recheckCursor { | |||
panic("corrupted reCheckTx cursor") | |||
} | |||
txmp.removeTx(wtx, !txmp.config.KeepInvalidTxsInCache) | |||
} | |||
} | |||
// move reCheckTx cursor to next element | |||
if txmp.recheckCursor == txmp.recheckEnd { | |||
txmp.recheckCursor = nil | |||
} else { | |||
txmp.recheckCursor = txmp.recheckCursor.Next() | |||
} | |||
if txmp.recheckCursor == nil { | |||
txmp.logger.Debug("finished rechecking transactions") | |||
if txmp.Size() > 0 { | |||
txmp.notifyTxsAvailable() | |||
} | |||
} | |||
txmp.metrics.Size.Set(float64(txmp.Size())) | |||
} | |||
// updateReCheckTxs updates the recheck cursors by using the gossipIndex. For | |||
// each transaction, it executes CheckTxAsync. The global callback defined on | |||
// the proxyAppConn will be executed for each transaction after CheckTx is | |||
// executed. | |||
// | |||
// NOTE: | |||
// - The caller must have a write-lock when executing updateReCheckTxs. | |||
func (txmp *TxMempool) updateReCheckTxs() { | |||
if txmp.Size() == 0 { | |||
panic("attempted to update re-CheckTx txs when mempool is empty") | |||
} | |||
txmp.recheckCursor = txmp.gossipIndex.Front() | |||
txmp.recheckEnd = txmp.gossipIndex.Back() | |||
ctx := context.Background() | |||
for e := txmp.gossipIndex.Front(); e != nil; e = e.Next() { | |||
wtx := e.Value.(*WrappedTx) | |||
// Only execute CheckTx if the transaction is not marked as removed which | |||
// could happen if the transaction was evicted. | |||
if !txmp.txStore.IsTxRemoved(wtx.hash) { | |||
_, err := txmp.proxyAppConn.CheckTxAsync(ctx, abci.RequestCheckTx{ | |||
Tx: wtx.tx, | |||
Type: abci.CheckTxType_Recheck, | |||
}) | |||
if err != nil { | |||
// no need in retrying since the tx will be rechecked after the next block | |||
txmp.logger.Error("failed to execute CheckTx during rechecking", "err", err) | |||
} | |||
} | |||
} | |||
if _, err := txmp.proxyAppConn.FlushAsync(ctx); err != nil { | |||
txmp.logger.Error("failed to flush transactions during rechecking", "err", err) | |||
} | |||
} | |||
// canAddTx returns an error if we cannot insert the provided *WrappedTx into | |||
// the mempool due to mempool configured constraints. Otherwise, nil is returned | |||
// and the transaction can be inserted into the mempool. | |||
func (txmp *TxMempool) canAddTx(wtx *WrappedTx) error { | |||
var ( | |||
numTxs = txmp.Size() | |||
sizeBytes = txmp.SizeBytes() | |||
) | |||
if numTxs >= txmp.config.Size || int64(wtx.Size())+sizeBytes > txmp.config.MaxTxsBytes { | |||
return types.ErrMempoolIsFull{ | |||
NumTxs: numTxs, | |||
MaxTxs: txmp.config.Size, | |||
TxsBytes: sizeBytes, | |||
MaxTxsBytes: txmp.config.MaxTxsBytes, | |||
} | |||
} | |||
return nil | |||
} | |||
func (txmp *TxMempool) insertTx(wtx *WrappedTx) { | |||
txmp.txStore.SetTx(wtx) | |||
txmp.priorityIndex.PushTx(wtx) | |||
txmp.heightIndex.Insert(wtx) | |||
txmp.timestampIndex.Insert(wtx) | |||
// Insert the transaction into the gossip index and mark the reference to the | |||
// linked-list element, which will be needed at a later point when the | |||
// transaction is removed. | |||
gossipEl := txmp.gossipIndex.PushBack(wtx) | |||
wtx.gossipEl = gossipEl | |||
atomic.AddInt64(&txmp.sizeBytes, int64(wtx.Size())) | |||
} | |||
func (txmp *TxMempool) removeTx(wtx *WrappedTx, removeFromCache bool) { | |||
if txmp.txStore.IsTxRemoved(wtx.hash) { | |||
return | |||
} | |||
txmp.txStore.RemoveTx(wtx) | |||
txmp.priorityIndex.RemoveTx(wtx) | |||
txmp.heightIndex.Remove(wtx) | |||
txmp.timestampIndex.Remove(wtx) | |||
// Remove the transaction from the gossip index and cleanup the linked-list | |||
// element so it can be garbage collected. | |||
txmp.gossipIndex.Remove(wtx.gossipEl) | |||
wtx.gossipEl.DetachPrev() | |||
atomic.AddInt64(&txmp.sizeBytes, int64(-wtx.Size())) | |||
if removeFromCache { | |||
txmp.cache.Remove(wtx.tx) | |||
} | |||
} | |||
// purgeExpiredTxs removes all transactions that have exceeded their respective | |||
// height and/or time based TTLs from their respective indexes. Every expired | |||
// transaction will be removed from the mempool entirely, except for the cache. | |||
// | |||
// NOTE: purgeExpiredTxs must only be called during TxMempool#Update in which | |||
// the caller has a write-lock on the mempool and so we can safely iterate over | |||
// the height and time based indexes. | |||
func (txmp *TxMempool) purgeExpiredTxs(blockHeight int64) { | |||
now := time.Now() | |||
expiredTxs := make(map[types.TxKey]*WrappedTx) | |||
if txmp.config.TTLNumBlocks > 0 { | |||
purgeIdx := -1 | |||
for i, wtx := range txmp.heightIndex.txs { | |||
if (blockHeight - wtx.height) > txmp.config.TTLNumBlocks { | |||
expiredTxs[wtx.tx.Key()] = wtx | |||
purgeIdx = i | |||
} else { | |||
// since the index is sorted, we know no other txs can be be purged | |||
break | |||
} | |||
} | |||
if purgeIdx >= 0 { | |||
txmp.heightIndex.txs = txmp.heightIndex.txs[purgeIdx+1:] | |||
} | |||
} | |||
if txmp.config.TTLDuration > 0 { | |||
purgeIdx := -1 | |||
for i, wtx := range txmp.timestampIndex.txs { | |||
if now.Sub(wtx.timestamp) > txmp.config.TTLDuration { | |||
expiredTxs[wtx.tx.Key()] = wtx | |||
purgeIdx = i | |||
} else { | |||
// since the index is sorted, we know no other txs can be be purged | |||
break | |||
} | |||
} | |||
if purgeIdx >= 0 { | |||
txmp.timestampIndex.txs = txmp.timestampIndex.txs[purgeIdx+1:] | |||
} | |||
} | |||
for _, wtx := range expiredTxs { | |||
txmp.removeTx(wtx, false) | |||
} | |||
} | |||
func (txmp *TxMempool) notifyTxsAvailable() { | |||
if txmp.Size() == 0 { | |||
panic("attempt to notify txs available but mempool is empty!") | |||
} | |||
if txmp.txsAvailable != nil && !txmp.notifiedTxsAvailable { | |||
// channel cap is 1, so this will send once | |||
txmp.notifiedTxsAvailable = true | |||
select { | |||
case txmp.txsAvailable <- struct{}{}: | |||
default: | |||
} | |||
} | |||
} |
@ -1,281 +0,0 @@ | |||
package v1 | |||
import ( | |||
"sort" | |||
"time" | |||
"github.com/tendermint/tendermint/internal/libs/clist" | |||
tmsync "github.com/tendermint/tendermint/internal/libs/sync" | |||
"github.com/tendermint/tendermint/types" | |||
) | |||
// WrappedTx defines a wrapper around a raw transaction with additional metadata | |||
// that is used for indexing. | |||
type WrappedTx struct { | |||
// tx represents the raw binary transaction data | |||
tx types.Tx | |||
// hash defines the transaction hash and the primary key used in the mempool | |||
hash types.TxKey | |||
// height defines the height at which the transaction was validated at | |||
height int64 | |||
// gasWanted defines the amount of gas the transaction sender requires | |||
gasWanted int64 | |||
// priority defines the transaction's priority as specified by the application | |||
// in the ResponseCheckTx response. | |||
priority int64 | |||
// sender defines the transaction's sender as specified by the application in | |||
// the ResponseCheckTx response. | |||
sender string | |||
// timestamp is the time at which the node first received the transaction from | |||
// a peer. It is used as a second dimension is prioritizing transactions when | |||
// two transactions have the same priority. | |||
timestamp time.Time | |||
// peers records a mapping of all peers that sent a given transaction | |||
peers map[uint16]struct{} | |||
// heapIndex defines the index of the item in the heap | |||
heapIndex int | |||
// gossipEl references the linked-list element in the gossip index | |||
gossipEl *clist.CElement | |||
// removed marks the transaction as removed from the mempool. This is set | |||
// during RemoveTx and is needed due to the fact that a given existing | |||
// transaction in the mempool can be evicted when it is simultaneously having | |||
// a reCheckTx callback executed. | |||
removed bool | |||
} | |||
func (wtx *WrappedTx) Size() int { | |||
return len(wtx.tx) | |||
} | |||
// TxStore implements a thread-safe mapping of valid transaction(s). | |||
// | |||
// NOTE: | |||
// - Concurrent read-only access to a *WrappedTx object is OK. However, mutative | |||
// access is not allowed. Regardless, it is not expected for the mempool to | |||
// need mutative access. | |||
type TxStore struct { | |||
mtx tmsync.RWMutex | |||
hashTxs map[types.TxKey]*WrappedTx // primary index | |||
senderTxs map[string]*WrappedTx // sender is defined by the ABCI application | |||
} | |||
func NewTxStore() *TxStore { | |||
return &TxStore{ | |||
senderTxs: make(map[string]*WrappedTx), | |||
hashTxs: make(map[types.TxKey]*WrappedTx), | |||
} | |||
} | |||
// Size returns the total number of transactions in the store. | |||
func (txs *TxStore) Size() int { | |||
txs.mtx.RLock() | |||
defer txs.mtx.RUnlock() | |||
return len(txs.hashTxs) | |||
} | |||
// GetAllTxs returns all the transactions currently in the store. | |||
func (txs *TxStore) GetAllTxs() []*WrappedTx { | |||
txs.mtx.RLock() | |||
defer txs.mtx.RUnlock() | |||
wTxs := make([]*WrappedTx, len(txs.hashTxs)) | |||
i := 0 | |||
for _, wtx := range txs.hashTxs { | |||
wTxs[i] = wtx | |||
i++ | |||
} | |||
return wTxs | |||
} | |||
// GetTxBySender returns a *WrappedTx by the transaction's sender property | |||
// defined by the ABCI application. | |||
func (txs *TxStore) GetTxBySender(sender string) *WrappedTx { | |||
txs.mtx.RLock() | |||
defer txs.mtx.RUnlock() | |||
return txs.senderTxs[sender] | |||
} | |||
// GetTxByHash returns a *WrappedTx by the transaction's hash. | |||
func (txs *TxStore) GetTxByHash(hash types.TxKey) *WrappedTx { | |||
txs.mtx.RLock() | |||
defer txs.mtx.RUnlock() | |||
return txs.hashTxs[hash] | |||
} | |||
// IsTxRemoved returns true if a transaction by hash is marked as removed and | |||
// false otherwise. | |||
func (txs *TxStore) IsTxRemoved(hash types.TxKey) bool { | |||
txs.mtx.RLock() | |||
defer txs.mtx.RUnlock() | |||
wtx, ok := txs.hashTxs[hash] | |||
if ok { | |||
return wtx.removed | |||
} | |||
return false | |||
} | |||
// SetTx stores a *WrappedTx by it's hash. If the transaction also contains a | |||
// non-empty sender, we additionally store the transaction by the sender as | |||
// defined by the ABCI application. | |||
func (txs *TxStore) SetTx(wtx *WrappedTx) { | |||
txs.mtx.Lock() | |||
defer txs.mtx.Unlock() | |||
if len(wtx.sender) > 0 { | |||
txs.senderTxs[wtx.sender] = wtx | |||
} | |||
txs.hashTxs[wtx.tx.Key()] = wtx | |||
} | |||
// RemoveTx removes a *WrappedTx from the transaction store. It deletes all | |||
// indexes of the transaction. | |||
func (txs *TxStore) RemoveTx(wtx *WrappedTx) { | |||
txs.mtx.Lock() | |||
defer txs.mtx.Unlock() | |||
if len(wtx.sender) > 0 { | |||
delete(txs.senderTxs, wtx.sender) | |||
} | |||
delete(txs.hashTxs, wtx.tx.Key()) | |||
wtx.removed = true | |||
} | |||
// TxHasPeer returns true if a transaction by hash has a given peer ID and false | |||
// otherwise. If the transaction does not exist, false is returned. | |||
func (txs *TxStore) TxHasPeer(hash types.TxKey, peerID uint16) bool { | |||
txs.mtx.RLock() | |||
defer txs.mtx.RUnlock() | |||
wtx := txs.hashTxs[hash] | |||
if wtx == nil { | |||
return false | |||
} | |||
_, ok := wtx.peers[peerID] | |||
return ok | |||
} | |||
// GetOrSetPeerByTxHash looks up a WrappedTx by transaction hash and adds the | |||
// given peerID to the WrappedTx's set of peers that sent us this transaction. | |||
// We return true if we've already recorded the given peer for this transaction | |||
// and false otherwise. If the transaction does not exist by hash, we return | |||
// (nil, false). | |||
func (txs *TxStore) GetOrSetPeerByTxHash(hash types.TxKey, peerID uint16) (*WrappedTx, bool) { | |||
txs.mtx.Lock() | |||
defer txs.mtx.Unlock() | |||
wtx := txs.hashTxs[hash] | |||
if wtx == nil { | |||
return nil, false | |||
} | |||
if wtx.peers == nil { | |||
wtx.peers = make(map[uint16]struct{}) | |||
} | |||
if _, ok := wtx.peers[peerID]; ok { | |||
return wtx, true | |||
} | |||
wtx.peers[peerID] = struct{}{} | |||
return wtx, false | |||
} | |||
// WrappedTxList implements a thread-safe list of *WrappedTx objects that can be | |||
// used to build generic transaction indexes in the mempool. It accepts a | |||
// comparator function, less(a, b *WrappedTx) bool, that compares two WrappedTx | |||
// references which is used during Insert in order to determine sorted order. If | |||
// less returns true, a <= b. | |||
type WrappedTxList struct { | |||
mtx tmsync.RWMutex | |||
txs []*WrappedTx | |||
less func(*WrappedTx, *WrappedTx) bool | |||
} | |||
func NewWrappedTxList(less func(*WrappedTx, *WrappedTx) bool) *WrappedTxList { | |||
return &WrappedTxList{ | |||
txs: make([]*WrappedTx, 0), | |||
less: less, | |||
} | |||
} | |||
// Size returns the number of WrappedTx objects in the list. | |||
func (wtl *WrappedTxList) Size() int { | |||
wtl.mtx.RLock() | |||
defer wtl.mtx.RUnlock() | |||
return len(wtl.txs) | |||
} | |||
// Reset resets the list of transactions to an empty list. | |||
func (wtl *WrappedTxList) Reset() { | |||
wtl.mtx.Lock() | |||
defer wtl.mtx.Unlock() | |||
wtl.txs = make([]*WrappedTx, 0) | |||
} | |||
// Insert inserts a WrappedTx reference into the sorted list based on the list's | |||
// comparator function. | |||
func (wtl *WrappedTxList) Insert(wtx *WrappedTx) { | |||
wtl.mtx.Lock() | |||
defer wtl.mtx.Unlock() | |||
i := sort.Search(len(wtl.txs), func(i int) bool { | |||
return wtl.less(wtl.txs[i], wtx) | |||
}) | |||
if i == len(wtl.txs) { | |||
// insert at the end | |||
wtl.txs = append(wtl.txs, wtx) | |||
return | |||
} | |||
// Make space for the inserted element by shifting values at the insertion | |||
// index up one index. | |||
// | |||
// NOTE: The call to append does not allocate memory when cap(wtl.txs) > len(wtl.txs). | |||
wtl.txs = append(wtl.txs[:i+1], wtl.txs[i:]...) | |||
wtl.txs[i] = wtx | |||
} | |||
// Remove attempts to remove a WrappedTx from the sorted list. | |||
func (wtl *WrappedTxList) Remove(wtx *WrappedTx) { | |||
wtl.mtx.Lock() | |||
defer wtl.mtx.Unlock() | |||
i := sort.Search(len(wtl.txs), func(i int) bool { | |||
return wtl.less(wtl.txs[i], wtx) | |||
}) | |||
// Since the list is sorted, we evaluate all elements starting at i. Note, if | |||
// the element does not exist, we may potentially evaluate the entire remainder | |||
// of the list. However, a caller should not be expected to call Remove with a | |||
// non-existing element. | |||
for i < len(wtl.txs) { | |||
if wtl.txs[i] == wtx { | |||
wtl.txs = append(wtl.txs[:i], wtl.txs[i+1:]...) | |||
return | |||
} | |||
i++ | |||
} | |||
} |
@ -1,37 +0,0 @@ | |||
package v0 | |||
import ( | |||
"context" | |||
abciclient "github.com/tendermint/tendermint/abci/client" | |||
"github.com/tendermint/tendermint/abci/example/kvstore" | |||
"github.com/tendermint/tendermint/config" | |||
"github.com/tendermint/tendermint/internal/mempool" | |||
mempoolv0 "github.com/tendermint/tendermint/internal/mempool/v0" | |||
) | |||
var mp mempool.Mempool | |||
func init() { | |||
app := kvstore.NewApplication() | |||
cc := abciclient.NewLocalCreator(app) | |||
appConnMem, _ := cc() | |||
err := appConnMem.Start() | |||
if err != nil { | |||
panic(err) | |||
} | |||
cfg := config.DefaultMempoolConfig() | |||
cfg.Broadcast = false | |||
mp = mempoolv0.NewCListMempool(cfg, appConnMem, 0) | |||
} | |||
func Fuzz(data []byte) int { | |||
err := mp.CheckTx(context.Background(), data, nil, mempool.TxInfo{}) | |||
if err != nil { | |||
return 0 | |||
} | |||
return 1 | |||
} |
@ -1,33 +0,0 @@ | |||
package v0_test | |||
import ( | |||
"io" | |||
"os" | |||
"path/filepath" | |||
"testing" | |||
"github.com/stretchr/testify/require" | |||
mempoolv0 "github.com/tendermint/tendermint/test/fuzz/mempool/v0" | |||
) | |||
const testdataCasesDir = "testdata/cases" | |||
func TestMempoolTestdataCases(t *testing.T) { | |||
entries, err := os.ReadDir(testdataCasesDir) | |||
require.NoError(t, err) | |||
for _, e := range entries { | |||
entry := e | |||
t.Run(entry.Name(), func(t *testing.T) { | |||
defer func() { | |||
r := recover() | |||
require.Nilf(t, r, "testdata/cases test panic") | |||
}() | |||
f, err := os.Open(filepath.Join(testdataCasesDir, entry.Name())) | |||
require.NoError(t, err) | |||
input, err := io.ReadAll(f) | |||
require.NoError(t, err) | |||
mempoolv0.Fuzz(input) | |||
}) | |||
} | |||
} |