mirror of
https://github.com/tendermint/tendermint.git
synced 2026-01-03 11:45:18 +00:00
mempool: consoldate implementations (#7171)
* 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>
This commit is contained in:
@@ -14,6 +14,7 @@ Special thanks to external contributors on this release:
|
||||
|
||||
- [rpc] Remove the deprecated gRPC interface to the RPC service. (@creachadair)
|
||||
- [blocksync] \#7159 Remove support for disabling blocksync in any circumstance. (@tychoish)
|
||||
- [mempool] \#7171 Remove legacy mempool implementation. (@tychoish)
|
||||
|
||||
- Apps
|
||||
|
||||
|
||||
@@ -27,9 +27,6 @@ const (
|
||||
ModeFull = "full"
|
||||
ModeValidator = "validator"
|
||||
ModeSeed = "seed"
|
||||
|
||||
MempoolV0 = "v0"
|
||||
MempoolV1 = "v1"
|
||||
)
|
||||
|
||||
// NOTE: Most of the structs & relevant comments + the
|
||||
@@ -693,7 +690,6 @@ func TestP2PConfig() *P2PConfig {
|
||||
|
||||
// MempoolConfig defines the configuration options for the Tendermint mempool.
|
||||
type MempoolConfig struct {
|
||||
Version string `mapstructure:"version"`
|
||||
RootDir string `mapstructure:"home"`
|
||||
Recheck bool `mapstructure:"recheck"`
|
||||
Broadcast bool `mapstructure:"broadcast"`
|
||||
@@ -743,7 +739,6 @@ type MempoolConfig struct {
|
||||
// DefaultMempoolConfig returns a default configuration for the Tendermint mempool.
|
||||
func DefaultMempoolConfig() *MempoolConfig {
|
||||
return &MempoolConfig{
|
||||
Version: MempoolV1,
|
||||
Recheck: true,
|
||||
Broadcast: true,
|
||||
// Each signature verification takes .5ms, Size reduced until we implement
|
||||
|
||||
@@ -327,11 +327,6 @@ recv-rate = {{ .P2P.RecvRate }}
|
||||
#######################################################
|
||||
[mempool]
|
||||
|
||||
# Mempool version to use:
|
||||
# 1) "v0" - The legacy non-prioritized mempool reactor.
|
||||
# 2) "v1" (default) - The prioritized mempool reactor.
|
||||
version = "{{ .Mempool.Version }}"
|
||||
|
||||
recheck = {{ .Mempool.Recheck }}
|
||||
broadcast = {{ .Mempool.Broadcast }}
|
||||
|
||||
|
||||
@@ -16,7 +16,7 @@ import (
|
||||
abci "github.com/tendermint/tendermint/abci/types"
|
||||
"github.com/tendermint/tendermint/internal/evidence"
|
||||
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
|
||||
mempoolv0 "github.com/tendermint/tendermint/internal/mempool/v0"
|
||||
"github.com/tendermint/tendermint/internal/mempool"
|
||||
"github.com/tendermint/tendermint/internal/p2p"
|
||||
sm "github.com/tendermint/tendermint/internal/state"
|
||||
"github.com/tendermint/tendermint/internal/store"
|
||||
@@ -69,8 +69,12 @@ func TestByzantinePrevoteEquivocation(t *testing.T) {
|
||||
proxyAppConnCon := abciclient.NewLocalClient(mtx, app)
|
||||
|
||||
// Make Mempool
|
||||
mempool := mempoolv0.NewCListMempool(thisConfig.Mempool, proxyAppConnMem, 0)
|
||||
mempool.SetLogger(log.TestingLogger().With("module", "mempool"))
|
||||
mempool := mempool.NewTxMempool(
|
||||
log.TestingLogger().With("module", "mempool"),
|
||||
thisConfig.Mempool,
|
||||
proxyAppConnMem,
|
||||
0,
|
||||
)
|
||||
if thisConfig.Consensus.WaitForTxs() {
|
||||
mempool.EnableTxsAvailable()
|
||||
}
|
||||
|
||||
@@ -21,7 +21,7 @@ import (
|
||||
"github.com/tendermint/tendermint/config"
|
||||
cstypes "github.com/tendermint/tendermint/internal/consensus/types"
|
||||
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
|
||||
mempoolv0 "github.com/tendermint/tendermint/internal/mempool/v0"
|
||||
"github.com/tendermint/tendermint/internal/mempool"
|
||||
sm "github.com/tendermint/tendermint/internal/state"
|
||||
"github.com/tendermint/tendermint/internal/store"
|
||||
"github.com/tendermint/tendermint/internal/test/factory"
|
||||
@@ -420,8 +420,14 @@ func newStateWithConfigAndBlockStore(
|
||||
proxyAppConnCon := abciclient.NewLocalClient(mtx, app)
|
||||
|
||||
// Make Mempool
|
||||
mempool := mempoolv0.NewCListMempool(thisConfig.Mempool, proxyAppConnMem, 0)
|
||||
mempool.SetLogger(log.TestingLogger().With("module", "mempool"))
|
||||
|
||||
mempool := mempool.NewTxMempool(
|
||||
log.TestingLogger().With("module", "mempool"),
|
||||
thisConfig.Mempool,
|
||||
proxyAppConnMem,
|
||||
0,
|
||||
)
|
||||
|
||||
if thisConfig.Consensus.WaitForTxs() {
|
||||
mempool.EnableTxsAvailable()
|
||||
}
|
||||
|
||||
@@ -21,7 +21,6 @@ import (
|
||||
"github.com/tendermint/tendermint/crypto/encoding"
|
||||
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
|
||||
"github.com/tendermint/tendermint/internal/mempool"
|
||||
mempoolv0 "github.com/tendermint/tendermint/internal/mempool/v0"
|
||||
"github.com/tendermint/tendermint/internal/p2p"
|
||||
"github.com/tendermint/tendermint/internal/p2p/p2ptest"
|
||||
sm "github.com/tendermint/tendermint/internal/state"
|
||||
@@ -353,8 +352,13 @@ func TestReactorWithEvidence(t *testing.T) {
|
||||
proxyAppConnMem := abciclient.NewLocalClient(mtx, app)
|
||||
proxyAppConnCon := abciclient.NewLocalClient(mtx, app)
|
||||
|
||||
mempool := mempoolv0.NewCListMempool(thisConfig.Mempool, proxyAppConnMem, 0)
|
||||
mempool.SetLogger(log.TestingLogger().With("module", "mempool"))
|
||||
mempool := mempool.NewTxMempool(
|
||||
log.TestingLogger().With("module", "mempool"),
|
||||
thisConfig.Mempool,
|
||||
proxyAppConnMem,
|
||||
0,
|
||||
)
|
||||
|
||||
if thisConfig.Consensus.WaitForTxs() {
|
||||
mempool.EnableTxsAvailable()
|
||||
}
|
||||
|
||||
@@ -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.
|
||||
// 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 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 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()
|
||||
}
|
||||
|
||||
// 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.
|
||||
//
|
||||
// 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
|
||||
// NOTE: The caller must obtain a write-lock prior to execution.
|
||||
func (txmp *TxMempool) FlushAppConn() error {
|
||||
return txmp.proxyAppConn.FlushSync(context.Background())
|
||||
}
|
||||
|
||||
// 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
|
||||
// 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()
|
||||
}
|
||||
|
||||
// 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
|
||||
// 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()
|
||||
}
|
||||
|
||||
// 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})
|
||||
// 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()
|
||||
|
||||
if txSize > maxBytes {
|
||||
return fmt.Errorf("tx size is too big: %d, max: %d", txSize, maxBytes)
|
||||
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 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
|
||||
}
|
||||
|
||||
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 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)
|
||||
}
|
||||
}
|
||||
|
||||
// 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)
|
||||
}
|
||||
// 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()
|
||||
)
|
||||
|
||||
return nil
|
||||
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:
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
package v1
|
||||
package mempool
|
||||
|
||||
import (
|
||||
"context"
|
||||
@@ -8,7 +8,6 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
"github.com/tendermint/tendermint/internal/mempool"
|
||||
)
|
||||
|
||||
func BenchmarkTxMempool_CheckTx(b *testing.B) {
|
||||
@@ -27,6 +26,6 @@ func BenchmarkTxMempool_CheckTx(b *testing.B) {
|
||||
tx := []byte(fmt.Sprintf("%X=%d", prefix, priority))
|
||||
b.StartTimer()
|
||||
|
||||
require.NoError(b, txmp.CheckTx(context.Background(), tx, nil, mempool.TxInfo{}))
|
||||
require.NoError(b, txmp.CheckTx(context.Background(), tx, nil, TxInfo{}))
|
||||
}
|
||||
}
|
||||
@@ -1,4 +1,4 @@
|
||||
package v1
|
||||
package mempool
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
@@ -21,7 +21,6 @@ import (
|
||||
"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/libs/log"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
@@ -97,7 +96,7 @@ func setup(t testing.TB, cacheSize int, options ...TxMempoolOption) *TxMempool {
|
||||
|
||||
func checkTxs(t *testing.T, txmp *TxMempool, numTxs int, peerID uint16) []testTx {
|
||||
txs := make([]testTx, numTxs)
|
||||
txInfo := mempool.TxInfo{SenderID: peerID}
|
||||
txInfo := TxInfo{SenderID: peerID}
|
||||
|
||||
rng := rand.New(rand.NewSource(time.Now().UnixNano()))
|
||||
|
||||
@@ -338,13 +337,13 @@ func TestTxMempool_CheckTxExceedsMaxSize(t *testing.T) {
|
||||
_, err := rng.Read(tx)
|
||||
require.NoError(t, err)
|
||||
|
||||
require.Error(t, txmp.CheckTx(context.Background(), tx, nil, mempool.TxInfo{SenderID: 0}))
|
||||
require.Error(t, txmp.CheckTx(context.Background(), tx, nil, TxInfo{SenderID: 0}))
|
||||
|
||||
tx = make([]byte, txmp.config.MaxTxBytes-1)
|
||||
_, err = rng.Read(tx)
|
||||
require.NoError(t, err)
|
||||
|
||||
require.NoError(t, txmp.CheckTx(context.Background(), tx, nil, mempool.TxInfo{SenderID: 0}))
|
||||
require.NoError(t, txmp.CheckTx(context.Background(), tx, nil, TxInfo{SenderID: 0}))
|
||||
}
|
||||
|
||||
func TestTxMempool_CheckTxSamePeer(t *testing.T) {
|
||||
@@ -358,8 +357,8 @@ func TestTxMempool_CheckTxSamePeer(t *testing.T) {
|
||||
|
||||
tx := []byte(fmt.Sprintf("sender-0=%X=%d", prefix, 50))
|
||||
|
||||
require.NoError(t, txmp.CheckTx(context.Background(), tx, nil, mempool.TxInfo{SenderID: peerID}))
|
||||
require.Error(t, txmp.CheckTx(context.Background(), tx, nil, mempool.TxInfo{SenderID: peerID}))
|
||||
require.NoError(t, txmp.CheckTx(context.Background(), tx, nil, TxInfo{SenderID: peerID}))
|
||||
require.Error(t, txmp.CheckTx(context.Background(), tx, nil, TxInfo{SenderID: peerID}))
|
||||
}
|
||||
|
||||
func TestTxMempool_CheckTxSameSender(t *testing.T) {
|
||||
@@ -378,9 +377,9 @@ func TestTxMempool_CheckTxSameSender(t *testing.T) {
|
||||
tx1 := []byte(fmt.Sprintf("sender-0=%X=%d", prefix1, 50))
|
||||
tx2 := []byte(fmt.Sprintf("sender-0=%X=%d", prefix2, 50))
|
||||
|
||||
require.NoError(t, txmp.CheckTx(context.Background(), tx1, nil, mempool.TxInfo{SenderID: peerID}))
|
||||
require.NoError(t, txmp.CheckTx(context.Background(), tx1, nil, TxInfo{SenderID: peerID}))
|
||||
require.Equal(t, 1, txmp.Size())
|
||||
require.NoError(t, txmp.CheckTx(context.Background(), tx2, nil, mempool.TxInfo{SenderID: peerID}))
|
||||
require.NoError(t, txmp.CheckTx(context.Background(), tx2, nil, TxInfo{SenderID: peerID}))
|
||||
require.Equal(t, 1, txmp.Size())
|
||||
}
|
||||
|
||||
@@ -533,7 +532,7 @@ func TestTxMempool_CheckTxPostCheckError(t *testing.T) {
|
||||
}
|
||||
require.Equal(t, expectedErrString, checkTxRes.CheckTx.MempoolError)
|
||||
}
|
||||
require.NoError(t, txmp.CheckTx(context.Background(), tx, callback, mempool.TxInfo{SenderID: 0}))
|
||||
require.NoError(t, txmp.CheckTx(context.Background(), tx, callback, TxInfo{SenderID: 0}))
|
||||
})
|
||||
}
|
||||
}
|
||||
@@ -12,7 +12,7 @@ import (
|
||||
// Mempool is an empty implementation of a Mempool, useful for testing.
|
||||
type Mempool struct{}
|
||||
|
||||
var _ mempool.Mempool = Mempool{}
|
||||
var _ Mempool = Mempool{}
|
||||
|
||||
func (Mempool) Lock() {}
|
||||
func (Mempool) Unlock() {}
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
package v1
|
||||
package mempool
|
||||
|
||||
import (
|
||||
"container/heap"
|
||||
@@ -1,4 +1,4 @@
|
||||
package v1
|
||||
package mempool
|
||||
|
||||
import (
|
||||
"math/rand"
|
||||
@@ -1,4 +1,4 @@
|
||||
package v1
|
||||
package mempool
|
||||
|
||||
import (
|
||||
"context"
|
||||
@@ -11,7 +11,6 @@ import (
|
||||
"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"
|
||||
@@ -39,7 +38,7 @@ type Reactor struct {
|
||||
|
||||
cfg *config.MempoolConfig
|
||||
mempool *TxMempool
|
||||
ids *mempool.IDs
|
||||
ids *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.
|
||||
@@ -76,7 +75,7 @@ func NewReactor(
|
||||
cfg: cfg,
|
||||
peerMgr: peerMgr,
|
||||
mempool: txmp,
|
||||
ids: mempool.NewMempoolIDs(),
|
||||
ids: NewMempoolIDs(),
|
||||
mempoolCh: mempoolCh,
|
||||
peerUpdates: peerUpdates,
|
||||
closeCh: make(chan struct{}),
|
||||
@@ -101,7 +100,7 @@ func GetChannelDescriptor(cfg *config.MempoolConfig) *p2p.ChannelDescriptor {
|
||||
}
|
||||
|
||||
return &p2p.ChannelDescriptor{
|
||||
ID: mempool.MempoolChannel,
|
||||
ID: MempoolChannel,
|
||||
MessageType: new(protomem.Message),
|
||||
Priority: 5,
|
||||
RecvMessageCapacity: batchMsg.Size(),
|
||||
@@ -161,7 +160,7 @@ func (r *Reactor) handleMempoolMessage(envelope p2p.Envelope) error {
|
||||
return errors.New("empty txs received from peer")
|
||||
}
|
||||
|
||||
txInfo := mempool.TxInfo{SenderID: r.ids.GetForPeer(envelope.From)}
|
||||
txInfo := TxInfo{SenderID: r.ids.GetForPeer(envelope.From)}
|
||||
if len(envelope.From) != 0 {
|
||||
txInfo.SenderNodeID = envelope.From
|
||||
}
|
||||
@@ -198,7 +197,7 @@ func (r *Reactor) handleMessage(chID p2p.ChannelID, envelope p2p.Envelope) (err
|
||||
r.Logger.Debug("received message", "peer", envelope.From)
|
||||
|
||||
switch chID {
|
||||
case mempool.MempoolChannel:
|
||||
case MempoolChannel:
|
||||
err = r.handleMempoolMessage(envelope)
|
||||
|
||||
default:
|
||||
@@ -358,7 +357,7 @@ func (r *Reactor) broadcastTxRoutine(peerID types.NodeID, closer *tmsync.Closer)
|
||||
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)
|
||||
time.Sleep(PeerCatchupSleepIntervalMS * time.Millisecond)
|
||||
continue
|
||||
}
|
||||
}
|
||||
@@ -1,4 +1,4 @@
|
||||
package v1
|
||||
package mempool
|
||||
|
||||
import (
|
||||
"context"
|
||||
@@ -13,7 +13,6 @@ import (
|
||||
abci "github.com/tendermint/tendermint/abci/types"
|
||||
"github.com/tendermint/tendermint/config"
|
||||
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
|
||||
"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"
|
||||
@@ -199,7 +198,7 @@ func TestReactorBroadcastTxs(t *testing.T) {
|
||||
primary := rts.nodes[0]
|
||||
secondaries := rts.nodes[1:]
|
||||
|
||||
txs := checkTxs(t, rts.reactors[primary].mempool, numTxs, mempool.UnknownPeerID)
|
||||
txs := checkTxs(t, rts.reactors[primary].mempool, numTxs, UnknownPeerID)
|
||||
|
||||
// run the router
|
||||
rts.start(t)
|
||||
@@ -231,7 +230,7 @@ func TestReactorConcurrency(t *testing.T) {
|
||||
// 1. submit a bunch of txs
|
||||
// 2. update the whole mempool
|
||||
|
||||
txs := checkTxs(t, rts.reactors[primary].mempool, numTxs, mempool.UnknownPeerID)
|
||||
txs := checkTxs(t, rts.reactors[primary].mempool, numTxs, UnknownPeerID)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
|
||||
@@ -250,7 +249,7 @@ func TestReactorConcurrency(t *testing.T) {
|
||||
|
||||
// 1. submit a bunch of txs
|
||||
// 2. update none
|
||||
_ = checkTxs(t, rts.reactors[secondary].mempool, numTxs, mempool.UnknownPeerID)
|
||||
_ = checkTxs(t, rts.reactors[secondary].mempool, numTxs, UnknownPeerID)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
|
||||
@@ -309,8 +308,8 @@ func TestReactor_MaxTxBytes(t *testing.T) {
|
||||
context.Background(),
|
||||
tx1,
|
||||
nil,
|
||||
mempool.TxInfo{
|
||||
SenderID: mempool.UnknownPeerID,
|
||||
TxInfo{
|
||||
SenderID: UnknownPeerID,
|
||||
},
|
||||
)
|
||||
require.NoError(t, err)
|
||||
@@ -322,7 +321,7 @@ func TestReactor_MaxTxBytes(t *testing.T) {
|
||||
|
||||
// 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})
|
||||
err = rts.mempools[primary].CheckTx(context.Background(), tx2, nil, TxInfo{SenderID: UnknownPeerID})
|
||||
require.Error(t, err)
|
||||
|
||||
rts.assertMempoolChannelsDrained(t)
|
||||
@@ -331,7 +330,7 @@ func TestReactor_MaxTxBytes(t *testing.T) {
|
||||
func TestDontExhaustMaxActiveIDs(t *testing.T) {
|
||||
// we're creating a single node network, but not starting the
|
||||
// network.
|
||||
rts := setupReactors(t, 1, mempool.MaxActiveIDs+1)
|
||||
rts := setupReactors(t, 1, MaxActiveIDs+1)
|
||||
|
||||
nodeID := rts.nodes[0]
|
||||
|
||||
@@ -339,7 +338,7 @@ func TestDontExhaustMaxActiveIDs(t *testing.T) {
|
||||
require.NoError(t, err)
|
||||
|
||||
// ensure the reactor does not panic (i.e. exhaust active IDs)
|
||||
for i := 0; i < mempool.MaxActiveIDs+1; i++ {
|
||||
for i := 0; i < MaxActiveIDs+1; i++ {
|
||||
rts.peerChans[nodeID] <- p2p.PeerUpdate{
|
||||
Status: p2p.PeerStatusUp,
|
||||
NodeID: peerID,
|
||||
@@ -377,12 +376,12 @@ func TestMempoolIDsPanicsIfNodeRequestsOvermaxActiveIDs(t *testing.T) {
|
||||
}
|
||||
|
||||
// 0 is already reserved for UnknownPeerID
|
||||
ids := mempool.NewMempoolIDs()
|
||||
ids := NewMempoolIDs()
|
||||
|
||||
peerID, err := types.NewNodeID("0011223344556677889900112233445566778899")
|
||||
require.NoError(t, err)
|
||||
|
||||
for i := 0; i < mempool.MaxActiveIDs-1; i++ {
|
||||
for i := 0; i < MaxActiveIDs-1; i++ {
|
||||
ids.ReserveForPeer(peerID)
|
||||
}
|
||||
|
||||
@@ -410,7 +409,7 @@ func TestBroadcastTxForPeerStopsWhenPeerStops(t *testing.T) {
|
||||
}
|
||||
time.Sleep(500 * time.Millisecond)
|
||||
|
||||
txs := checkTxs(t, rts.reactors[primary].mempool, 4, mempool.UnknownPeerID)
|
||||
txs := checkTxs(t, rts.reactors[primary].mempool, 4, UnknownPeerID)
|
||||
require.Equal(t, 4, len(txs))
|
||||
require.Equal(t, 4, rts.mempools[primary].Size())
|
||||
require.Equal(t, 0, rts.mempools[secondary].Size())
|
||||
@@ -1,6 +1,11 @@
|
||||
package mempool
|
||||
|
||||
import (
|
||||
"sort"
|
||||
"time"
|
||||
|
||||
"github.com/tendermint/tendermint/internal/libs/clist"
|
||||
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
@@ -15,3 +20,274 @@ type TxInfo struct {
|
||||
// SenderNodeID is the actual types.NodeID of the sender.
|
||||
SenderNodeID types.NodeID
|
||||
}
|
||||
|
||||
// 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,4 +1,4 @@
|
||||
package v1
|
||||
package mempool
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
143
internal/mempool/types.go
Normal file
143
internal/mempool/types.go
Normal file
@@ -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++
|
||||
}
|
||||
}
|
||||
@@ -23,7 +23,6 @@ import (
|
||||
"github.com/tendermint/tendermint/crypto/tmhash"
|
||||
"github.com/tendermint/tendermint/internal/evidence"
|
||||
"github.com/tendermint/tendermint/internal/mempool"
|
||||
mempoolv0 "github.com/tendermint/tendermint/internal/mempool/v0"
|
||||
"github.com/tendermint/tendermint/internal/proxy"
|
||||
sm "github.com/tendermint/tendermint/internal/state"
|
||||
"github.com/tendermint/tendermint/internal/state/indexer"
|
||||
@@ -265,15 +264,12 @@ func TestCreateProposalBlock(t *testing.T) {
|
||||
state.ConsensusParams.Evidence.MaxBytes = maxEvidenceBytes
|
||||
proposerAddr, _ := state.Validators.GetByIndex(0)
|
||||
|
||||
mp := mempoolv0.NewCListMempool(
|
||||
mp := mempool.NewTxMempool(
|
||||
logger.With("module", "mempool"),
|
||||
cfg.Mempool,
|
||||
proxyApp.Mempool(),
|
||||
state.LastBlockHeight,
|
||||
mempoolv0.WithMetrics(mempool.NopMetrics()),
|
||||
mempoolv0.WithPreCheck(sm.TxPreCheck(state)),
|
||||
mempoolv0.WithPostCheck(sm.TxPostCheck(state)),
|
||||
)
|
||||
mp.SetLogger(logger)
|
||||
|
||||
// Make EvidencePool
|
||||
evidenceDB := dbm.NewMemDB()
|
||||
@@ -362,15 +358,13 @@ func TestMaxTxsProposalBlockSize(t *testing.T) {
|
||||
proposerAddr, _ := state.Validators.GetByIndex(0)
|
||||
|
||||
// Make Mempool
|
||||
mp := mempoolv0.NewCListMempool(
|
||||
|
||||
mp := mempool.NewTxMempool(
|
||||
logger.With("module", "mempool"),
|
||||
cfg.Mempool,
|
||||
proxyApp.Mempool(),
|
||||
state.LastBlockHeight,
|
||||
mempoolv0.WithMetrics(mempool.NopMetrics()),
|
||||
mempoolv0.WithPreCheck(sm.TxPreCheck(state)),
|
||||
mempoolv0.WithPostCheck(sm.TxPostCheck(state)),
|
||||
)
|
||||
mp.SetLogger(logger)
|
||||
|
||||
// fill the mempool with one txs just below the maximum size
|
||||
txLength := int(types.MaxDataBytesNoEvidence(maxBytes, 1))
|
||||
@@ -426,15 +420,12 @@ func TestMaxProposalBlockSize(t *testing.T) {
|
||||
proposerAddr, _ := state.Validators.GetByIndex(0)
|
||||
|
||||
// Make Mempool
|
||||
mp := mempoolv0.NewCListMempool(
|
||||
mp := mempool.NewTxMempool(
|
||||
logger.With("module", "mempool"),
|
||||
cfg.Mempool,
|
||||
proxyApp.Mempool(),
|
||||
state.LastBlockHeight,
|
||||
mempoolv0.WithMetrics(mempool.NopMetrics()),
|
||||
mempoolv0.WithPreCheck(sm.TxPreCheck(state)),
|
||||
mempoolv0.WithPostCheck(sm.TxPostCheck(state)),
|
||||
)
|
||||
mp.SetLogger(logger)
|
||||
|
||||
// fill the mempool with one txs just below the maximum size
|
||||
txLength := int(types.MaxDataBytesNoEvidence(maxBytes, types.MaxVotesCount))
|
||||
|
||||
@@ -16,8 +16,6 @@ import (
|
||||
"github.com/tendermint/tendermint/internal/consensus"
|
||||
"github.com/tendermint/tendermint/internal/evidence"
|
||||
"github.com/tendermint/tendermint/internal/mempool"
|
||||
mempoolv0 "github.com/tendermint/tendermint/internal/mempool/v0"
|
||||
mempoolv1 "github.com/tendermint/tendermint/internal/mempool/v1"
|
||||
"github.com/tendermint/tendermint/internal/p2p"
|
||||
"github.com/tendermint/tendermint/internal/p2p/conn"
|
||||
"github.com/tendermint/tendermint/internal/p2p/pex"
|
||||
@@ -197,76 +195,37 @@ func createMempoolReactor(
|
||||
logger log.Logger,
|
||||
) (service.Service, mempool.Mempool, error) {
|
||||
|
||||
logger = logger.With("module", "mempool", "version", cfg.Mempool.Version)
|
||||
peerUpdates := peerManager.Subscribe()
|
||||
logger = logger.With("module", "mempool")
|
||||
|
||||
switch cfg.Mempool.Version {
|
||||
case config.MempoolV0:
|
||||
ch, err := router.OpenChannel(mempoolv0.GetChannelDescriptor(cfg.Mempool))
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
mp := mempoolv0.NewCListMempool(
|
||||
cfg.Mempool,
|
||||
proxyApp.Mempool(),
|
||||
state.LastBlockHeight,
|
||||
mempoolv0.WithMetrics(memplMetrics),
|
||||
mempoolv0.WithPreCheck(sm.TxPreCheck(state)),
|
||||
mempoolv0.WithPostCheck(sm.TxPostCheck(state)),
|
||||
)
|
||||
|
||||
mp.SetLogger(logger)
|
||||
|
||||
reactor := mempoolv0.NewReactor(
|
||||
logger,
|
||||
cfg.Mempool,
|
||||
peerManager,
|
||||
mp,
|
||||
ch,
|
||||
peerUpdates,
|
||||
)
|
||||
|
||||
if cfg.Consensus.WaitForTxs() {
|
||||
mp.EnableTxsAvailable()
|
||||
}
|
||||
|
||||
return reactor, mp, nil
|
||||
|
||||
case config.MempoolV1:
|
||||
ch, err := router.OpenChannel(mempoolv1.GetChannelDescriptor(cfg.Mempool))
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
mp := mempoolv1.NewTxMempool(
|
||||
logger,
|
||||
cfg.Mempool,
|
||||
proxyApp.Mempool(),
|
||||
state.LastBlockHeight,
|
||||
mempoolv1.WithMetrics(memplMetrics),
|
||||
mempoolv1.WithPreCheck(sm.TxPreCheck(state)),
|
||||
mempoolv1.WithPostCheck(sm.TxPostCheck(state)),
|
||||
)
|
||||
|
||||
reactor := mempoolv1.NewReactor(
|
||||
logger,
|
||||
cfg.Mempool,
|
||||
peerManager,
|
||||
mp,
|
||||
ch,
|
||||
peerUpdates,
|
||||
)
|
||||
|
||||
if cfg.Consensus.WaitForTxs() {
|
||||
mp.EnableTxsAvailable()
|
||||
}
|
||||
|
||||
return reactor, mp, nil
|
||||
|
||||
default:
|
||||
return nil, nil, fmt.Errorf("unknown mempool version: %s", cfg.Mempool.Version)
|
||||
ch, err := router.OpenChannel(mempool.GetChannelDescriptor(cfg.Mempool))
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
mp := mempool.NewTxMempool(
|
||||
logger,
|
||||
cfg.Mempool,
|
||||
proxyApp.Mempool(),
|
||||
state.LastBlockHeight,
|
||||
mempool.WithMetrics(memplMetrics),
|
||||
mempool.WithPreCheck(sm.TxPreCheck(state)),
|
||||
mempool.WithPostCheck(sm.TxPostCheck(state)),
|
||||
)
|
||||
|
||||
reactor := mempool.NewReactor(
|
||||
logger,
|
||||
cfg.Mempool,
|
||||
peerManager,
|
||||
mp,
|
||||
ch,
|
||||
peerManager.Subscribe(),
|
||||
)
|
||||
|
||||
if cfg.Consensus.WaitForTxs() {
|
||||
mp.EnableTxsAvailable()
|
||||
}
|
||||
|
||||
return reactor, mp, nil
|
||||
}
|
||||
|
||||
func createEvidenceReactor(
|
||||
|
||||
@@ -44,10 +44,6 @@ var (
|
||||
"tcp": 20,
|
||||
"unix": 10,
|
||||
}
|
||||
nodeMempools = weightedChoice{
|
||||
"v0": 20,
|
||||
"v1": 80,
|
||||
}
|
||||
nodeStateSyncs = weightedChoice{
|
||||
e2e.StateSyncDisabled: 10,
|
||||
e2e.StateSyncP2P: 45,
|
||||
@@ -277,7 +273,6 @@ func generateNode(
|
||||
StartAt: startAt,
|
||||
Database: nodeDatabases.Choose(r),
|
||||
PrivvalProtocol: nodePrivvalProtocols.Choose(r),
|
||||
Mempool: nodeMempools.Choose(r),
|
||||
StateSync: e2e.StateSyncDisabled,
|
||||
PersistInterval: ptrUint64(uint64(nodePersistIntervals.Choose(r).(int))),
|
||||
SnapshotInterval: uint64(nodeSnapshotIntervals.Choose(r).(int)),
|
||||
@@ -285,9 +280,6 @@ func generateNode(
|
||||
Perturb: nodePerturbations.Choose(r),
|
||||
}
|
||||
|
||||
if node.Mempool == "" {
|
||||
node.Mempool = "v1"
|
||||
}
|
||||
if node.PrivvalProtocol == "" {
|
||||
node.PrivvalProtocol = "file"
|
||||
}
|
||||
|
||||
@@ -37,9 +37,6 @@ func TestGenerator(t *testing.T) {
|
||||
}
|
||||
})
|
||||
if e2e.Mode(node.Mode) != e2e.ModeLight {
|
||||
t.Run("Mempool", func(t *testing.T) {
|
||||
require.NotZero(t, node.Mempool)
|
||||
})
|
||||
t.Run("PrivvalProtocol", func(t *testing.T) {
|
||||
require.NotZero(t, node.PrivvalProtocol)
|
||||
})
|
||||
|
||||
@@ -291,10 +291,6 @@ func MakeConfig(node *e2e.Node) (*config.Config, error) {
|
||||
return nil, fmt.Errorf("unexpected mode %q", node.Mode)
|
||||
}
|
||||
|
||||
if node.Mempool != "" {
|
||||
cfg.Mempool.Version = node.Mempool
|
||||
}
|
||||
|
||||
switch node.StateSync {
|
||||
case e2e.StateSyncP2P:
|
||||
cfg.StateSync.Enable = true
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
package v0
|
||||
package mempool
|
||||
|
||||
import (
|
||||
"context"
|
||||
@@ -7,10 +7,11 @@ import (
|
||||
"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"
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
)
|
||||
|
||||
var mp mempool.Mempool
|
||||
var mp *mempool.TxMempool
|
||||
var getMp func() mempool.Mempool
|
||||
|
||||
func init() {
|
||||
app := kvstore.NewApplication()
|
||||
@@ -24,11 +25,22 @@ func init() {
|
||||
cfg := config.DefaultMempoolConfig()
|
||||
cfg.Broadcast = false
|
||||
|
||||
mp = mempoolv0.NewCListMempool(cfg, appConnMem, 0)
|
||||
getMp = func() mempool.Mempool {
|
||||
if mp == nil {
|
||||
mp = mempool.NewTxMempool(
|
||||
log.TestingLogger().With("module", "mempool"),
|
||||
cfg,
|
||||
appConnMem,
|
||||
0,
|
||||
)
|
||||
|
||||
}
|
||||
return mp
|
||||
}
|
||||
}
|
||||
|
||||
func Fuzz(data []byte) int {
|
||||
err := mp.CheckTx(context.Background(), data, nil, mempool.TxInfo{})
|
||||
err := getMp().CheckTx(context.Background(), data, nil, mempool.TxInfo{})
|
||||
if err != nil {
|
||||
return 0
|
||||
}
|
||||
@@ -1,4 +1,4 @@
|
||||
package v1_test
|
||||
package mempool_test
|
||||
|
||||
import (
|
||||
"io"
|
||||
@@ -7,7 +7,7 @@ import (
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
mempoolv1 "github.com/tendermint/tendermint/test/fuzz/mempool/v1"
|
||||
mempool "github.com/tendermint/tendermint/test/fuzz/mempool"
|
||||
)
|
||||
|
||||
const testdataCasesDir = "testdata/cases"
|
||||
@@ -27,7 +27,7 @@ func TestMempoolTestdataCases(t *testing.T) {
|
||||
require.NoError(t, err)
|
||||
input, err := io.ReadAll(f)
|
||||
require.NoError(t, err)
|
||||
mempoolv1.Fuzz(input)
|
||||
mempool.Fuzz(input)
|
||||
})
|
||||
}
|
||||
}
|
||||
@@ -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)
|
||||
})
|
||||
}
|
||||
}
|
||||
@@ -1,37 +0,0 @@
|
||||
package v1
|
||||
|
||||
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"
|
||||
mempoolv1 "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 = mempoolv1.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
|
||||
}
|
||||
Reference in New Issue
Block a user