Browse Source

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>
pull/7187/head
Sam Kleinman 3 years ago
committed by GitHub
parent
commit
5cc980698a
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
37 changed files with 1358 additions and 3890 deletions
  1. +1
    -0
      CHANGELOG_PENDING.md
  2. +0
    -5
      config/config.go
  3. +0
    -5
      config/toml.go
  4. +7
    -3
      internal/consensus/byzantine_test.go
  5. +9
    -3
      internal/consensus/common_test.go
  6. +7
    -3
      internal/consensus/reactor_test.go
  7. +828
    -102
      internal/mempool/mempool.go
  8. +2
    -3
      internal/mempool/mempool_bench_test.go
  9. +9
    -10
      internal/mempool/mempool_test.go
  10. +1
    -1
      internal/mempool/mock/mempool.go
  11. +1
    -1
      internal/mempool/priority_queue.go
  12. +1
    -1
      internal/mempool/priority_queue_test.go
  13. +7
    -8
      internal/mempool/reactor.go
  14. +12
    -13
      internal/mempool/reactor_test.go
  15. +276
    -0
      internal/mempool/tx.go
  16. +1
    -1
      internal/mempool/tx_test.go
  17. +143
    -0
      internal/mempool/types.go
  18. +0
    -107
      internal/mempool/v0/bench_test.go
  19. +0
    -83
      internal/mempool/v0/cache_test.go
  20. +0
    -698
      internal/mempool/v0/clist_mempool.go
  21. +0
    -691
      internal/mempool/v0/clist_mempool_test.go
  22. +0
    -23
      internal/mempool/v0/doc.go
  23. +0
    -392
      internal/mempool/v0/reactor.go
  24. +0
    -393
      internal/mempool/v0/reactor_test.go
  25. +0
    -887
      internal/mempool/v1/mempool.go
  26. +0
    -281
      internal/mempool/v1/tx.go
  27. +7
    -16
      node/node_test.go
  28. +26
    -67
      node/setup.go
  29. +0
    -8
      test/e2e/generator/generate.go
  30. +0
    -3
      test/e2e/generator/generate_test.go
  31. +0
    -4
      test/e2e/runner/setup.go
  32. +17
    -5
      test/fuzz/mempool/checktx.go
  33. +3
    -3
      test/fuzz/mempool/fuzz_test.go
  34. +0
    -0
      test/fuzz/mempool/testdata/cases/empty
  35. +0
    -37
      test/fuzz/mempool/v0/checktx.go
  36. +0
    -33
      test/fuzz/mempool/v0/fuzz_test.go
  37. +0
    -0
      test/fuzz/mempool/v1/testdata/cases/empty

+ 1
- 0
CHANGELOG_PENDING.md View File

@ -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


+ 0
- 5
config/config.go View File

@ -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


+ 0
- 5
config/toml.go View File

@ -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 }}


+ 7
- 3
internal/consensus/byzantine_test.go View File

@ -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()
}


+ 9
- 3
internal/consensus/common_test.go View File

@ -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()
}


+ 7
- 3
internal/consensus/reactor_test.go View File

@ -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()
}


+ 828
- 102
internal/mempool/mempool.go View File

@ -1,143 +1,869 @@
package mempool
import (
"bytes"
"context"
"errors"
"fmt"
"math"
"reflect"
"sync/atomic"
"time"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/internal/p2p"
"github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/internal/libs/clist"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/internal/proxy"
"github.com/tendermint/tendermint/libs/log"
tmmath "github.com/tendermint/tendermint/libs/math"
"github.com/tendermint/tendermint/types"
)
const (
MempoolChannel = p2p.ChannelID(0x30)
var _ Mempool = (*TxMempool)(nil)
// PeerCatchupSleepIntervalMS defines how much time to sleep if a peer is behind
PeerCatchupSleepIntervalMS = 100
// TxMempoolOption sets an optional parameter on the TxMempool.
type TxMempoolOption func(*TxMempool)
// UnknownPeerID is the peer ID to use when running CheckTx when there is
// no peer (e.g. RPC)
UnknownPeerID uint16 = 0
// TxMempool defines a prioritized mempool data structure used by the v1 mempool
// reactor. It keeps a thread-safe priority queue of transactions that is used
// when a block proposer constructs a block and a thread-safe linked-list that
// is used to gossip transactions to peers in a FIFO manner.
type TxMempool struct {
logger log.Logger
metrics *Metrics
config *config.MempoolConfig
proxyAppConn proxy.AppConnMempool
MaxActiveIDs = math.MaxUint16
)
// txsAvailable fires once for each height when the mempool is not empty
txsAvailable chan struct{}
notifiedTxsAvailable bool
// Mempool defines the mempool interface.
//
// Updates to the mempool need to be synchronized with committing a block so
// applications can reset their transient state on Commit.
type Mempool interface {
// CheckTx executes a new transaction against the application to determine
// its validity and whether it should be added to the mempool.
CheckTx(ctx context.Context, tx types.Tx, callback func(*abci.Response), txInfo TxInfo) error
// RemoveTxByKey removes a transaction, identified by its key,
// from the mempool.
RemoveTxByKey(txKey types.TxKey) error
// ReapMaxBytesMaxGas reaps transactions from the mempool up to maxBytes
// bytes total with the condition that the total gasWanted must be less than
// maxGas.
//
// If both maxes are negative, there is no cap on the size of all returned
// transactions (~ all available transactions).
ReapMaxBytesMaxGas(maxBytes, maxGas int64) types.Txs
// height defines the last block height process during Update()
height int64
// ReapMaxTxs reaps up to max transactions from the mempool. If max is
// negative, there is no cap on the size of all returned transactions
// (~ all available transactions).
ReapMaxTxs(max int) types.Txs
// sizeBytes defines the total size of the mempool (sum of all tx bytes)
sizeBytes int64
// Lock locks the mempool. The consensus must be able to hold lock to safely
// update.
Lock()
// cache defines a fixed-size cache of already seen transactions as this
// reduces pressure on the proxyApp.
cache TxCache
// Unlock unlocks the mempool.
Unlock()
// txStore defines the main storage of valid transactions. Indexes are built
// on top of this store.
txStore *TxStore
// Update informs the mempool that the given txs were committed and can be
// discarded.
//
// NOTE:
// 1. This should be called *after* block is committed by consensus.
// 2. Lock/Unlock must be managed by the caller.
Update(
blockHeight int64,
blockTxs types.Txs,
deliverTxResponses []*abci.ResponseDeliverTx,
newPreFn PreCheckFunc,
newPostFn PostCheckFunc,
) error
// FlushAppConn flushes the mempool connection to ensure async callback calls
// are done, e.g. from CheckTx.
// gossipIndex defines the gossiping index of valid transactions via a
// thread-safe linked-list. We also use the gossip index as a cursor for
// rechecking transactions already in the mempool.
gossipIndex *clist.CList
// recheckCursor and recheckEnd are used as cursors based on the gossip index
// to recheck transactions that are already in the mempool. Iteration is not
// thread-safe and transaction may be mutated in serial order.
//
// NOTE:
// 1. Lock/Unlock must be managed by caller.
FlushAppConn() error
// XXX/TODO: It might be somewhat of a codesmell to use the gossip index for
// iterator and cursor management when rechecking transactions. If the gossip
// index changes or is removed in a future refactor, this will have to be
// refactored. Instead, we should consider just keeping a slice of a snapshot
// of the mempool's current transactions during Update and an integer cursor
// into that slice. This, however, requires additional O(n) space complexity.
recheckCursor *clist.CElement // next expected response
recheckEnd *clist.CElement // re-checking stops here
// Flush removes all transactions from the mempool and caches.
Flush()
// priorityIndex defines the priority index of valid transactions via a
// thread-safe priority queue.
priorityIndex *TxPriorityQueue
// TxsAvailable returns a channel which fires once for every height, and only
// when transactions are available in the mempool.
//
// NOTE:
// 1. The returned channel may be nil if EnableTxsAvailable was not called.
TxsAvailable() <-chan struct{}
// heightIndex defines a height-based, in ascending order, transaction index.
// i.e. older transactions are first.
heightIndex *WrappedTxList
// timestampIndex defines a timestamp-based, in ascending order, transaction
// index. i.e. older transactions are first.
timestampIndex *WrappedTxList
// A read/write lock is used to safe guard updates, insertions and deletions
// from the mempool. A read-lock is implicitly acquired when executing CheckTx,
// however, a caller must explicitly grab a write-lock via Lock when updating
// the mempool via Update().
mtx tmsync.RWMutex
preCheck PreCheckFunc
postCheck PostCheckFunc
}
func NewTxMempool(
logger log.Logger,
cfg *config.MempoolConfig,
proxyAppConn proxy.AppConnMempool,
height int64,
options ...TxMempoolOption,
) *TxMempool {
txmp := &TxMempool{
logger: logger,
config: cfg,
proxyAppConn: proxyAppConn,
height: height,
cache: NopTxCache{},
metrics: NopMetrics(),
txStore: NewTxStore(),
gossipIndex: clist.New(),
priorityIndex: NewTxPriorityQueue(),
heightIndex: NewWrappedTxList(func(wtx1, wtx2 *WrappedTx) bool {
return wtx1.height >= wtx2.height
}),
timestampIndex: NewWrappedTxList(func(wtx1, wtx2 *WrappedTx) bool {
return wtx1.timestamp.After(wtx2.timestamp) || wtx1.timestamp.Equal(wtx2.timestamp)
}),
}
if cfg.CacheSize > 0 {
txmp.cache = NewLRUTxCache(cfg.CacheSize)
}
proxyAppConn.SetResponseCallback(txmp.defaultTxCallback)
for _, opt := range options {
opt(txmp)
}
return txmp
}
// WithPreCheck sets a filter for the mempool to reject a transaction if f(tx)
// returns an error. This is executed before CheckTx. It only applies to the
// first created block. After that, Update() overwrites the existing value.
func WithPreCheck(f PreCheckFunc) TxMempoolOption {
return func(txmp *TxMempool) { txmp.preCheck = f }
}
// WithPostCheck sets a filter for the mempool to reject a transaction if
// f(tx, resp) returns an error. This is executed after CheckTx. It only applies
// to the first created block. After that, Update overwrites the existing value.
func WithPostCheck(f PostCheckFunc) TxMempoolOption {
return func(txmp *TxMempool) { txmp.postCheck = f }
}
// WithMetrics sets the mempool's metrics collector.
func WithMetrics(metrics *Metrics) TxMempoolOption {
return func(txmp *TxMempool) { txmp.metrics = metrics }
}
// Lock obtains a write-lock on the mempool. A caller must be sure to explicitly
// release the lock when finished.
func (txmp *TxMempool) Lock() {
txmp.mtx.Lock()
}
// EnableTxsAvailable initializes the TxsAvailable channel, ensuring it will
// trigger once every height when transactions are available.
EnableTxsAvailable()
// Unlock releases a write-lock on the mempool.
func (txmp *TxMempool) Unlock() {
txmp.mtx.Unlock()
}
// Size returns the number of valid transactions in the mempool. It is
// thread-safe.
func (txmp *TxMempool) Size() int {
return txmp.txStore.Size()
}
// SizeBytes return the total sum in bytes of all the valid transactions in the
// mempool. It is thread-safe.
func (txmp *TxMempool) SizeBytes() int64 {
return atomic.LoadInt64(&txmp.sizeBytes)
}
// FlushAppConn executes FlushSync on the mempool's proxyAppConn.
//
// NOTE: The caller must obtain a write-lock prior to execution.
func (txmp *TxMempool) FlushAppConn() error {
return txmp.proxyAppConn.FlushSync(context.Background())
}
// Size returns the number of transactions in the mempool.
Size() int
// WaitForNextTx returns a blocking channel that will be closed when the next
// valid transaction is available to gossip. It is thread-safe.
func (txmp *TxMempool) WaitForNextTx() <-chan struct{} {
return txmp.gossipIndex.WaitChan()
}
// SizeBytes returns the total size of all txs in the mempool.
SizeBytes() int64
// NextGossipTx returns the next valid transaction to gossip. A caller must wait
// for WaitForNextTx to signal a transaction is available to gossip first. It is
// thread-safe.
func (txmp *TxMempool) NextGossipTx() *clist.CElement {
return txmp.gossipIndex.Front()
}
// PreCheckFunc is an optional filter executed before CheckTx and rejects
// transaction if false is returned. An example would be to ensure that a
// transaction doesn't exceeded the block size.
type PreCheckFunc func(types.Tx) error
// EnableTxsAvailable enables the mempool to trigger events when transactions
// are available on a block by block basis.
func (txmp *TxMempool) EnableTxsAvailable() {
txmp.mtx.Lock()
defer txmp.mtx.Unlock()
// PostCheckFunc is an optional filter executed after CheckTx and rejects
// transaction if false is returned. An example would be to ensure a
// transaction doesn't require more gas than available for the block.
type PostCheckFunc func(types.Tx, *abci.ResponseCheckTx) error
txmp.txsAvailable = make(chan struct{}, 1)
}
// PreCheckMaxBytes checks that the size of the transaction is smaller or equal
// to the expected maxBytes.
func PreCheckMaxBytes(maxBytes int64) PreCheckFunc {
return func(tx types.Tx) error {
txSize := types.ComputeProtoSizeForTxs([]types.Tx{tx})
// TxsAvailable returns a channel which fires once for every height, and only
// when transactions are available in the mempool. It is thread-safe.
func (txmp *TxMempool) TxsAvailable() <-chan struct{} {
return txmp.txsAvailable
}
if txSize > maxBytes {
return fmt.Errorf("tx size is too big: %d, max: %d", txSize, maxBytes)
// CheckTx executes the ABCI CheckTx method for a given transaction. It acquires
// a read-lock attempts to execute the application's CheckTx ABCI method via
// CheckTxAsync. We return an error if any of the following happen:
//
// - The CheckTxAsync execution fails.
// - The transaction already exists in the cache and we've already received the
// transaction from the peer. Otherwise, if it solely exists in the cache, we
// return nil.
// - The transaction size exceeds the maximum transaction size as defined by the
// configuration provided to the mempool.
// - The transaction fails Pre-Check (if it is defined).
// - The proxyAppConn fails, e.g. the buffer is full.
//
// If the mempool is full, we still execute CheckTx and attempt to find a lower
// priority transaction to evict. If such a transaction exists, we remove the
// lower priority transaction and add the new one with higher priority.
//
// NOTE:
// - The applications' CheckTx implementation may panic.
// - The caller is not to explicitly require any locks for executing CheckTx.
func (txmp *TxMempool) CheckTx(
ctx context.Context,
tx types.Tx,
cb func(*abci.Response),
txInfo TxInfo,
) error {
if ctx == nil {
ctx = context.TODO()
}
txmp.mtx.RLock()
defer txmp.mtx.RUnlock()
if txSize := len(tx); txSize > txmp.config.MaxTxBytes {
return types.ErrTxTooLarge{
Max: txmp.config.MaxTxBytes,
Actual: txSize,
}
}
if txmp.preCheck != nil {
if err := txmp.preCheck(tx); err != nil {
return types.ErrPreCheck{Reason: err}
}
}
if err := txmp.proxyAppConn.Error(); err != nil {
return err
}
txHash := tx.Key()
// We add the transaction to the mempool's cache and if the transaction already
// exists, i.e. false is returned, then we check if we've seen this transaction
// from the same sender and error if we have. Otherwise, we return nil.
if !txmp.cache.Push(tx) {
wtx, ok := txmp.txStore.GetOrSetPeerByTxHash(txHash, txInfo.SenderID)
if wtx != nil && ok {
// We already have the transaction stored and the we've already seen this
// transaction from txInfo.SenderID.
return types.ErrTxInCache
}
txmp.logger.Debug("tx exists already in cache", "tx_hash", tx.Hash())
return nil
}
}
// PostCheckMaxGas checks that the wanted gas is smaller or equal to the passed
// maxGas. Returns nil if maxGas is -1.
func PostCheckMaxGas(maxGas int64) PostCheckFunc {
return func(tx types.Tx, res *abci.ResponseCheckTx) error {
if maxGas == -1 {
return nil
reqRes, err := txmp.proxyAppConn.CheckTxAsync(ctx, abci.RequestCheckTx{Tx: tx})
if err != nil {
txmp.cache.Remove(tx)
return err
}
reqRes.SetCallback(func(res *abci.Response) {
if txmp.recheckCursor != nil {
panic("recheck cursor is non-nil in CheckTx callback")
}
if res.GasWanted < 0 {
return fmt.Errorf("gas wanted %d is negative",
res.GasWanted)
wtx := &WrappedTx{
tx: tx,
hash: txHash,
timestamp: time.Now().UTC(),
height: txmp.height,
}
if res.GasWanted > maxGas {
return fmt.Errorf("gas wanted %d is greater than max gas %d",
res.GasWanted, maxGas)
txmp.initTxCallback(wtx, res, txInfo)
if cb != nil {
cb(res)
}
})
return nil
}
func (txmp *TxMempool) RemoveTxByKey(txKey types.TxKey) error {
txmp.Lock()
defer txmp.Unlock()
// remove the committed transaction from the transaction store and indexes
if wtx := txmp.txStore.GetTxByHash(txKey); wtx != nil {
txmp.removeTx(wtx, false)
return nil
}
return errors.New("transaction not found")
}
// Flush empties the mempool. It acquires a read-lock, fetches all the
// transactions currently in the transaction store and removes each transaction
// from the store and all indexes and finally resets the cache.
//
// NOTE:
// - Flushing the mempool may leave the mempool in an inconsistent state.
func (txmp *TxMempool) Flush() {
txmp.mtx.RLock()
defer txmp.mtx.RUnlock()
txmp.heightIndex.Reset()
txmp.timestampIndex.Reset()
for _, wtx := range txmp.txStore.GetAllTxs() {
txmp.removeTx(wtx, false)
}
atomic.SwapInt64(&txmp.sizeBytes, 0)
txmp.cache.Reset()
}
// ReapMaxBytesMaxGas returns a list of transactions within the provided size
// and gas constraints. Transaction are retrieved in priority order.
//
// NOTE:
// - Transactions returned are not removed from the mempool transaction
// store or indexes.
func (txmp *TxMempool) ReapMaxBytesMaxGas(maxBytes, maxGas int64) types.Txs {
txmp.mtx.RLock()
defer txmp.mtx.RUnlock()
var (
totalGas int64
totalSize int64
)
// wTxs contains a list of *WrappedTx retrieved from the priority queue that
// need to be re-enqueued prior to returning.
wTxs := make([]*WrappedTx, 0, txmp.priorityIndex.NumTxs())
defer func() {
for _, wtx := range wTxs {
txmp.priorityIndex.PushTx(wtx)
}
}()
txs := make([]types.Tx, 0, txmp.priorityIndex.NumTxs())
for txmp.priorityIndex.NumTxs() > 0 {
wtx := txmp.priorityIndex.PopTx()
txs = append(txs, wtx.tx)
wTxs = append(wTxs, wtx)
size := types.ComputeProtoSizeForTxs([]types.Tx{wtx.tx})
// Ensure we have capacity for the transaction with respect to the
// transaction size.
if maxBytes > -1 && totalSize+size > maxBytes {
return txs[:len(txs)-1]
}
totalSize += size
// ensure we have capacity for the transaction with respect to total gas
gas := totalGas + wtx.gasWanted
if maxGas > -1 && gas > maxGas {
return txs[:len(txs)-1]
}
totalGas = gas
}
return txs
}
// ReapMaxTxs returns a list of transactions within the provided number of
// transactions bound. Transaction are retrieved in priority order.
//
// NOTE:
// - Transactions returned are not removed from the mempool transaction
// store or indexes.
func (txmp *TxMempool) ReapMaxTxs(max int) types.Txs {
txmp.mtx.RLock()
defer txmp.mtx.RUnlock()
numTxs := txmp.priorityIndex.NumTxs()
if max < 0 {
max = numTxs
}
cap := tmmath.MinInt(numTxs, max)
// wTxs contains a list of *WrappedTx retrieved from the priority queue that
// need to be re-enqueued prior to returning.
wTxs := make([]*WrappedTx, 0, cap)
txs := make([]types.Tx, 0, cap)
for txmp.priorityIndex.NumTxs() > 0 && len(txs) < max {
wtx := txmp.priorityIndex.PopTx()
txs = append(txs, wtx.tx)
wTxs = append(wTxs, wtx)
}
for _, wtx := range wTxs {
txmp.priorityIndex.PushTx(wtx)
}
return txs
}
// Update iterates over all the transactions provided by the block producer,
// removes them from the cache (if applicable), and removes
// the transactions from the main transaction store and associated indexes.
// If there are transactions remaining in the mempool, we initiate a
// re-CheckTx for them (if applicable), otherwise, we notify the caller more
// transactions are available.
//
// NOTE:
// - The caller must explicitly acquire a write-lock.
func (txmp *TxMempool) Update(
blockHeight int64,
blockTxs types.Txs,
deliverTxResponses []*abci.ResponseDeliverTx,
newPreFn PreCheckFunc,
newPostFn PostCheckFunc,
) error {
txmp.height = blockHeight
txmp.notifiedTxsAvailable = false
if newPreFn != nil {
txmp.preCheck = newPreFn
}
if newPostFn != nil {
txmp.postCheck = newPostFn
}
for i, tx := range blockTxs {
if deliverTxResponses[i].Code == abci.CodeTypeOK {
// add the valid committed transaction to the cache (if missing)
_ = txmp.cache.Push(tx)
} else if !txmp.config.KeepInvalidTxsInCache {
// allow invalid transactions to be re-submitted
txmp.cache.Remove(tx)
}
// remove the committed transaction from the transaction store and indexes
if wtx := txmp.txStore.GetTxByHash(tx.Key()); wtx != nil {
txmp.removeTx(wtx, false)
}
}
txmp.purgeExpiredTxs(blockHeight)
// If there any uncommitted transactions left in the mempool, we either
// initiate re-CheckTx per remaining transaction or notify that remaining
// transactions are left.
if txmp.Size() > 0 {
if txmp.config.Recheck {
txmp.logger.Debug(
"executing re-CheckTx for all remaining transactions",
"num_txs", txmp.Size(),
"height", blockHeight,
)
txmp.updateReCheckTxs()
} else {
txmp.notifyTxsAvailable()
}
}
txmp.metrics.Size.Set(float64(txmp.Size()))
return nil
}
// initTxCallback is the callback invoked for a new unique transaction after CheckTx
// has been executed by the ABCI application for the first time on that transaction.
// CheckTx can be called again for the same transaction later when re-checking;
// however, this callback will not be called.
//
// initTxCallback runs after the ABCI application executes CheckTx.
// It runs the postCheck hook if one is defined on the mempool.
// If the CheckTx response response code is not OK, or if the postCheck hook
// reports an error, the transaction is rejected. Otherwise, we attempt to insert
// the transaction into the mempool.
//
// When inserting a transaction, we first check if there is sufficient capacity.
// If there is, the transaction is added to the txStore and all indexes.
// Otherwise, if the mempool is full, we attempt to find a lower priority transaction
// to evict in place of the new incoming transaction. If no such transaction exists,
// the new incoming transaction is rejected.
//
// NOTE:
// - An explicit lock is NOT required.
func (txmp *TxMempool) initTxCallback(wtx *WrappedTx, res *abci.Response, txInfo TxInfo) {
checkTxRes, ok := res.Value.(*abci.Response_CheckTx)
if !ok {
return
}
var err error
if txmp.postCheck != nil {
err = txmp.postCheck(wtx.tx, checkTxRes.CheckTx)
}
if err != nil || checkTxRes.CheckTx.Code != abci.CodeTypeOK {
// ignore bad transactions
txmp.logger.Info(
"rejected bad transaction",
"priority", wtx.priority,
"tx", fmt.Sprintf("%X", wtx.tx.Hash()),
"peer_id", txInfo.SenderNodeID,
"code", checkTxRes.CheckTx.Code,
"post_check_err", err,
)
txmp.metrics.FailedTxs.Add(1)
if !txmp.config.KeepInvalidTxsInCache {
txmp.cache.Remove(wtx.tx)
}
if err != nil {
checkTxRes.CheckTx.MempoolError = err.Error()
}
return
}
sender := checkTxRes.CheckTx.Sender
priority := checkTxRes.CheckTx.Priority
if len(sender) > 0 {
if wtx := txmp.txStore.GetTxBySender(sender); wtx != nil {
txmp.logger.Error(
"rejected incoming good transaction; tx already exists for sender",
"tx", fmt.Sprintf("%X", wtx.tx.Hash()),
"sender", sender,
)
txmp.metrics.RejectedTxs.Add(1)
return
}
}
if err := txmp.canAddTx(wtx); err != nil {
evictTxs := txmp.priorityIndex.GetEvictableTxs(
priority,
int64(wtx.Size()),
txmp.SizeBytes(),
txmp.config.MaxTxsBytes,
)
if len(evictTxs) == 0 {
// No room for the new incoming transaction so we just remove it from
// the cache.
txmp.cache.Remove(wtx.tx)
txmp.logger.Error(
"rejected incoming good transaction; mempool full",
"tx", fmt.Sprintf("%X", wtx.tx.Hash()),
"err", err.Error(),
)
txmp.metrics.RejectedTxs.Add(1)
return
}
// evict an existing transaction(s)
//
// NOTE:
// - The transaction, toEvict, can be removed while a concurrent
// reCheckTx callback is being executed for the same transaction.
for _, toEvict := range evictTxs {
txmp.removeTx(toEvict, true)
txmp.logger.Debug(
"evicted existing good transaction; mempool full",
"old_tx", fmt.Sprintf("%X", toEvict.tx.Hash()),
"old_priority", toEvict.priority,
"new_tx", fmt.Sprintf("%X", wtx.tx.Hash()),
"new_priority", wtx.priority,
)
txmp.metrics.EvictedTxs.Add(1)
}
}
wtx.gasWanted = checkTxRes.CheckTx.GasWanted
wtx.priority = priority
wtx.sender = sender
wtx.peers = map[uint16]struct{}{
txInfo.SenderID: {},
}
txmp.metrics.TxSizeBytes.Observe(float64(wtx.Size()))
txmp.metrics.Size.Set(float64(txmp.Size()))
txmp.insertTx(wtx)
txmp.logger.Debug(
"inserted good transaction",
"priority", wtx.priority,
"tx", fmt.Sprintf("%X", wtx.tx.Hash()),
"height", txmp.height,
"num_txs", txmp.Size(),
)
txmp.notifyTxsAvailable()
}
// defaultTxCallback is the CheckTx application callback used when a transaction
// is being re-checked (if re-checking is enabled). The caller must hold a mempool
// write-lock (via Lock()) and when executing Update(), if the mempool is non-empty
// and Recheck is enabled, then all remaining transactions will be rechecked via
// CheckTxAsync. The order transactions are rechecked must be the same as the
// order in which this callback is called.
func (txmp *TxMempool) defaultTxCallback(req *abci.Request, res *abci.Response) {
if txmp.recheckCursor == nil {
return
}
txmp.metrics.RecheckTimes.Add(1)
checkTxRes, ok := res.Value.(*abci.Response_CheckTx)
if !ok {
txmp.logger.Error("received incorrect type in mempool callback",
"expected", reflect.TypeOf(&abci.Response_CheckTx{}).Name(),
"got", reflect.TypeOf(res.Value).Name(),
)
return
}
tx := req.GetCheckTx().Tx
wtx := txmp.recheckCursor.Value.(*WrappedTx)
// Search through the remaining list of tx to recheck for a transaction that matches
// the one we received from the ABCI application.
for {
if bytes.Equal(tx, wtx.tx) {
// We've found a tx in the recheck list that matches the tx that we
// received from the ABCI application.
// Break, and use this transaction for further checks.
break
}
txmp.logger.Error(
"re-CheckTx transaction mismatch",
"got", wtx.tx.Hash(),
"expected", types.Tx(tx).Key(),
)
if txmp.recheckCursor == txmp.recheckEnd {
// we reached the end of the recheckTx list without finding a tx
// matching the one we received from the ABCI application.
// Return without processing any tx.
txmp.recheckCursor = nil
return
}
txmp.recheckCursor = txmp.recheckCursor.Next()
wtx = txmp.recheckCursor.Value.(*WrappedTx)
}
// Only evaluate transactions that have not been removed. This can happen
// if an existing transaction is evicted during CheckTx and while this
// callback is being executed for the same evicted transaction.
if !txmp.txStore.IsTxRemoved(wtx.hash) {
var err error
if txmp.postCheck != nil {
err = txmp.postCheck(tx, checkTxRes.CheckTx)
}
if checkTxRes.CheckTx.Code == abci.CodeTypeOK && err == nil {
wtx.priority = checkTxRes.CheckTx.Priority
} else {
txmp.logger.Debug(
"existing transaction no longer valid; failed re-CheckTx callback",
"priority", wtx.priority,
"tx", fmt.Sprintf("%X", wtx.tx.Hash()),
"err", err,
"code", checkTxRes.CheckTx.Code,
)
if wtx.gossipEl != txmp.recheckCursor {
panic("corrupted reCheckTx cursor")
}
txmp.removeTx(wtx, !txmp.config.KeepInvalidTxsInCache)
}
}
// move reCheckTx cursor to next element
if txmp.recheckCursor == txmp.recheckEnd {
txmp.recheckCursor = nil
} else {
txmp.recheckCursor = txmp.recheckCursor.Next()
}
if txmp.recheckCursor == nil {
txmp.logger.Debug("finished rechecking transactions")
if txmp.Size() > 0 {
txmp.notifyTxsAvailable()
}
}
txmp.metrics.Size.Set(float64(txmp.Size()))
}
// updateReCheckTxs updates the recheck cursors using the gossipIndex. For
// each transaction, it executes CheckTxAsync. The global callback defined on
// the proxyAppConn will be executed for each transaction after CheckTx is
// executed.
//
// NOTE:
// - The caller must have a write-lock when executing updateReCheckTxs.
func (txmp *TxMempool) updateReCheckTxs() {
if txmp.Size() == 0 {
panic("attempted to update re-CheckTx txs when mempool is empty")
}
txmp.recheckCursor = txmp.gossipIndex.Front()
txmp.recheckEnd = txmp.gossipIndex.Back()
ctx := context.Background()
for e := txmp.gossipIndex.Front(); e != nil; e = e.Next() {
wtx := e.Value.(*WrappedTx)
// Only execute CheckTx if the transaction is not marked as removed which
// could happen if the transaction was evicted.
if !txmp.txStore.IsTxRemoved(wtx.hash) {
_, err := txmp.proxyAppConn.CheckTxAsync(ctx, abci.RequestCheckTx{
Tx: wtx.tx,
Type: abci.CheckTxType_Recheck,
})
if err != nil {
// no need in retrying since the tx will be rechecked after the next block
txmp.logger.Error("failed to execute CheckTx during rechecking", "err", err)
}
}
}
if _, err := txmp.proxyAppConn.FlushAsync(ctx); err != nil {
txmp.logger.Error("failed to flush transactions during rechecking", "err", err)
}
}
// canAddTx returns an error if we cannot insert the provided *WrappedTx into
// the mempool due to mempool configured constraints. If it returns nil,
// the transaction can be inserted into the mempool.
func (txmp *TxMempool) canAddTx(wtx *WrappedTx) error {
var (
numTxs = txmp.Size()
sizeBytes = txmp.SizeBytes()
)
if numTxs >= txmp.config.Size || int64(wtx.Size())+sizeBytes > txmp.config.MaxTxsBytes {
return types.ErrMempoolIsFull{
NumTxs: numTxs,
MaxTxs: txmp.config.Size,
TxsBytes: sizeBytes,
MaxTxsBytes: txmp.config.MaxTxsBytes,
}
}
return nil
}
func (txmp *TxMempool) insertTx(wtx *WrappedTx) {
txmp.txStore.SetTx(wtx)
txmp.priorityIndex.PushTx(wtx)
txmp.heightIndex.Insert(wtx)
txmp.timestampIndex.Insert(wtx)
// Insert the transaction into the gossip index and mark the reference to the
// linked-list element, which will be needed at a later point when the
// transaction is removed.
gossipEl := txmp.gossipIndex.PushBack(wtx)
wtx.gossipEl = gossipEl
atomic.AddInt64(&txmp.sizeBytes, int64(wtx.Size()))
}
func (txmp *TxMempool) removeTx(wtx *WrappedTx, removeFromCache bool) {
if txmp.txStore.IsTxRemoved(wtx.hash) {
return
}
txmp.txStore.RemoveTx(wtx)
txmp.priorityIndex.RemoveTx(wtx)
txmp.heightIndex.Remove(wtx)
txmp.timestampIndex.Remove(wtx)
// Remove the transaction from the gossip index and cleanup the linked-list
// element so it can be garbage collected.
txmp.gossipIndex.Remove(wtx.gossipEl)
wtx.gossipEl.DetachPrev()
atomic.AddInt64(&txmp.sizeBytes, int64(-wtx.Size()))
if removeFromCache {
txmp.cache.Remove(wtx.tx)
}
}
// purgeExpiredTxs removes all transactions that have exceeded their respective
// height- and/or time-based TTLs from their respective indexes. Every expired
// transaction will be removed from the mempool, but preserved in the cache.
//
// NOTE: purgeExpiredTxs must only be called during TxMempool#Update in which
// the caller has a write-lock on the mempool and so we can safely iterate over
// the height and time based indexes.
func (txmp *TxMempool) purgeExpiredTxs(blockHeight int64) {
now := time.Now()
expiredTxs := make(map[types.TxKey]*WrappedTx)
if txmp.config.TTLNumBlocks > 0 {
purgeIdx := -1
for i, wtx := range txmp.heightIndex.txs {
if (blockHeight - wtx.height) > txmp.config.TTLNumBlocks {
expiredTxs[wtx.tx.Key()] = wtx
purgeIdx = i
} else {
// since the index is sorted, we know no other txs can be be purged
break
}
}
if purgeIdx >= 0 {
txmp.heightIndex.txs = txmp.heightIndex.txs[purgeIdx+1:]
}
}
if txmp.config.TTLDuration > 0 {
purgeIdx := -1
for i, wtx := range txmp.timestampIndex.txs {
if now.Sub(wtx.timestamp) > txmp.config.TTLDuration {
expiredTxs[wtx.tx.Key()] = wtx
purgeIdx = i
} else {
// since the index is sorted, we know no other txs can be be purged
break
}
}
if purgeIdx >= 0 {
txmp.timestampIndex.txs = txmp.timestampIndex.txs[purgeIdx+1:]
}
}
for _, wtx := range expiredTxs {
txmp.removeTx(wtx, false)
}
}
func (txmp *TxMempool) notifyTxsAvailable() {
if txmp.Size() == 0 {
panic("attempt to notify txs available but mempool is empty!")
}
if txmp.txsAvailable != nil && !txmp.notifiedTxsAvailable {
// channel cap is 1, so this will send once
txmp.notifiedTxsAvailable = true
select {
case txmp.txsAvailable <- struct{}{}:
default:
}
}
}

internal/mempool/v1/mempool_bench_test.go → internal/mempool/mempool_bench_test.go View File


internal/mempool/v1/mempool_test.go → internal/mempool/mempool_test.go View File


+ 1
- 1
internal/mempool/mock/mempool.go View File

@ -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() {}


internal/mempool/v1/priority_queue.go → internal/mempool/priority_queue.go View File


internal/mempool/v1/priority_queue_test.go → internal/mempool/priority_queue_test.go View File


internal/mempool/v1/reactor.go → internal/mempool/reactor.go View File


internal/mempool/v1/reactor_test.go → internal/mempool/reactor_test.go View File


+ 276
- 0
internal/mempool/tx.go View File

@ -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++
}
}

internal/mempool/v1/tx_test.go → internal/mempool/tx_test.go View File


+ 143
- 0
internal/mempool/types.go View 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
}
}

+ 0
- 107
internal/mempool/v0/bench_test.go View File

@ -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")
}
}
}

+ 0
- 83
internal/mempool/v0/cache_test.go View File

@ -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()
}
}

+ 0
- 698
internal/mempool/v0/clist_mempool.go View File

@ -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)
}

+ 0
- 691
internal/mempool/v0/clist_mempool_test.go View File

@ -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
}

+ 0
- 23
internal/mempool/v0/doc.go View File

@ -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

+ 0
- 392
internal/mempool/v0/reactor.go View File

@ -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
}
}
}

+ 0
- 393
internal/mempool/v0/reactor_test.go View File

@ -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,
}
}

+ 0
- 887
internal/mempool/v1/mempool.go View File

@ -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:
}
}
}

+ 0
- 281
internal/mempool/v1/tx.go View File

@ -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++
}
}

+ 7
- 16
node/node_test.go View File

@ -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))


+ 26
- 67
node/setup.go View File

@ -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()
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
}
logger = logger.With("module", "mempool")
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,
)
ch, err := router.OpenChannel(mempool.GetChannelDescriptor(cfg.Mempool))
if err != nil {
return nil, nil, err
}
if cfg.Consensus.WaitForTxs() {
mp.EnableTxsAvailable()
}
mp := mempool.NewTxMempool(
logger,
cfg.Mempool,
proxyApp.Mempool(),
state.LastBlockHeight,
mempool.WithMetrics(memplMetrics),
mempool.WithPreCheck(sm.TxPreCheck(state)),
mempool.WithPostCheck(sm.TxPostCheck(state)),
)
return reactor, mp, nil
reactor := mempool.NewReactor(
logger,
cfg.Mempool,
peerManager,
mp,
ch,
peerManager.Subscribe(),
)
default:
return nil, nil, fmt.Errorf("unknown mempool version: %s", cfg.Mempool.Version)
if cfg.Consensus.WaitForTxs() {
mp.EnableTxsAvailable()
}
return reactor, mp, nil
}
func createEvidenceReactor(


+ 0
- 8
test/e2e/generator/generate.go View File

@ -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"
}


+ 0
- 3
test/e2e/generator/generate_test.go View 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)
})


+ 0
- 4
test/e2e/runner/setup.go View File

@ -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


test/fuzz/mempool/v1/checktx.go → test/fuzz/mempool/checktx.go View File


test/fuzz/mempool/v1/fuzz_test.go → test/fuzz/mempool/fuzz_test.go View File


test/fuzz/mempool/v0/testdata/cases/empty → test/fuzz/mempool/testdata/cases/empty View File


+ 0
- 37
test/fuzz/mempool/v0/checktx.go View File

@ -1,37 +0,0 @@
package v0
import (
"context"
abciclient "github.com/tendermint/tendermint/abci/client"
"github.com/tendermint/tendermint/abci/example/kvstore"
"github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/internal/mempool"
mempoolv0 "github.com/tendermint/tendermint/internal/mempool/v0"
)
var mp mempool.Mempool
func init() {
app := kvstore.NewApplication()
cc := abciclient.NewLocalCreator(app)
appConnMem, _ := cc()
err := appConnMem.Start()
if err != nil {
panic(err)
}
cfg := config.DefaultMempoolConfig()
cfg.Broadcast = false
mp = mempoolv0.NewCListMempool(cfg, appConnMem, 0)
}
func Fuzz(data []byte) int {
err := mp.CheckTx(context.Background(), data, nil, mempool.TxInfo{})
if err != nil {
return 0
}
return 1
}

+ 0
- 33
test/fuzz/mempool/v0/fuzz_test.go View File

@ -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)
})
}
}

+ 0
- 0
test/fuzz/mempool/v1/testdata/cases/empty View File


Loading…
Cancel
Save