|
@ -1,16 +1,13 @@ |
|
|
package mempool |
|
|
package mempool |
|
|
|
|
|
|
|
|
import ( |
|
|
import ( |
|
|
"encoding/hex" |
|
|
|
|
|
"errors" |
|
|
|
|
|
"net" |
|
|
|
|
|
|
|
|
"fmt" |
|
|
|
|
|
"math/rand" |
|
|
"sync" |
|
|
"sync" |
|
|
"testing" |
|
|
"testing" |
|
|
"time" |
|
|
"time" |
|
|
|
|
|
|
|
|
"github.com/fortytw2/leaktest" |
|
|
"github.com/fortytw2/leaktest" |
|
|
"github.com/go-kit/kit/log/term" |
|
|
|
|
|
"github.com/stretchr/testify/assert" |
|
|
|
|
|
"github.com/stretchr/testify/require" |
|
|
"github.com/stretchr/testify/require" |
|
|
|
|
|
|
|
|
"github.com/tendermint/tendermint/abci/example/kvstore" |
|
|
"github.com/tendermint/tendermint/abci/example/kvstore" |
|
@ -19,370 +16,413 @@ import ( |
|
|
"github.com/tendermint/tendermint/libs/log" |
|
|
"github.com/tendermint/tendermint/libs/log" |
|
|
tmrand "github.com/tendermint/tendermint/libs/rand" |
|
|
tmrand "github.com/tendermint/tendermint/libs/rand" |
|
|
"github.com/tendermint/tendermint/p2p" |
|
|
"github.com/tendermint/tendermint/p2p" |
|
|
"github.com/tendermint/tendermint/p2p/mock" |
|
|
|
|
|
memproto "github.com/tendermint/tendermint/proto/tendermint/mempool" |
|
|
|
|
|
|
|
|
protomem "github.com/tendermint/tendermint/proto/tendermint/mempool" |
|
|
"github.com/tendermint/tendermint/proxy" |
|
|
"github.com/tendermint/tendermint/proxy" |
|
|
"github.com/tendermint/tendermint/types" |
|
|
"github.com/tendermint/tendermint/types" |
|
|
) |
|
|
) |
|
|
|
|
|
|
|
|
const ( |
|
|
|
|
|
numTxs = 1000 |
|
|
|
|
|
timeout = 120 * time.Second // ridiculously high because CircleCI is slow
|
|
|
|
|
|
) |
|
|
|
|
|
|
|
|
var rng = rand.New(rand.NewSource(time.Now().UnixNano())) |
|
|
|
|
|
|
|
|
|
|
|
type reactorTestSuite struct { |
|
|
|
|
|
reactor *Reactor |
|
|
|
|
|
|
|
|
|
|
|
peerID p2p.NodeID |
|
|
|
|
|
|
|
|
type peerState struct { |
|
|
|
|
|
height int64 |
|
|
|
|
|
|
|
|
mempoolChannel *p2p.Channel |
|
|
|
|
|
mempoolInCh chan p2p.Envelope |
|
|
|
|
|
mempoolOutCh chan p2p.Envelope |
|
|
|
|
|
mempoolPeerErrCh chan p2p.PeerError |
|
|
|
|
|
|
|
|
|
|
|
peerUpdatesCh chan p2p.PeerUpdate |
|
|
|
|
|
peerUpdates *p2p.PeerUpdatesCh |
|
|
} |
|
|
} |
|
|
|
|
|
|
|
|
func (ps peerState) GetHeight() int64 { |
|
|
|
|
|
return ps.height |
|
|
|
|
|
|
|
|
func setup(t *testing.T, cfg *cfg.MempoolConfig, logger log.Logger, chBuf uint) *reactorTestSuite { |
|
|
|
|
|
t.Helper() |
|
|
|
|
|
|
|
|
|
|
|
pID := make([]byte, 16) |
|
|
|
|
|
_, err := rng.Read(pID) |
|
|
|
|
|
require.NoError(t, err) |
|
|
|
|
|
|
|
|
|
|
|
peerID, err := p2p.NewNodeID(fmt.Sprintf("%x", pID)) |
|
|
|
|
|
require.NoError(t, err) |
|
|
|
|
|
|
|
|
|
|
|
peerUpdatesCh := make(chan p2p.PeerUpdate, chBuf) |
|
|
|
|
|
|
|
|
|
|
|
rts := &reactorTestSuite{ |
|
|
|
|
|
mempoolInCh: make(chan p2p.Envelope, chBuf), |
|
|
|
|
|
mempoolOutCh: make(chan p2p.Envelope, chBuf), |
|
|
|
|
|
mempoolPeerErrCh: make(chan p2p.PeerError, chBuf), |
|
|
|
|
|
peerUpdatesCh: peerUpdatesCh, |
|
|
|
|
|
peerUpdates: p2p.NewPeerUpdates(peerUpdatesCh), |
|
|
|
|
|
peerID: peerID, |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
rts.mempoolChannel = p2p.NewChannel( |
|
|
|
|
|
MempoolChannel, |
|
|
|
|
|
new(protomem.Message), |
|
|
|
|
|
rts.mempoolInCh, |
|
|
|
|
|
rts.mempoolOutCh, |
|
|
|
|
|
rts.mempoolPeerErrCh, |
|
|
|
|
|
) |
|
|
|
|
|
|
|
|
|
|
|
app := kvstore.NewApplication() |
|
|
|
|
|
cc := proxy.NewLocalClientCreator(app) |
|
|
|
|
|
mempool, memCleanup := newMempoolWithApp(cc) |
|
|
|
|
|
|
|
|
|
|
|
mempool.SetLogger(logger) |
|
|
|
|
|
|
|
|
|
|
|
rts.reactor = NewReactor( |
|
|
|
|
|
logger, |
|
|
|
|
|
cfg, |
|
|
|
|
|
nil, |
|
|
|
|
|
mempool, |
|
|
|
|
|
rts.mempoolChannel, |
|
|
|
|
|
rts.peerUpdates, |
|
|
|
|
|
) |
|
|
|
|
|
|
|
|
|
|
|
require.NoError(t, rts.reactor.Start()) |
|
|
|
|
|
require.True(t, rts.reactor.IsRunning()) |
|
|
|
|
|
|
|
|
|
|
|
t.Cleanup(func() { |
|
|
|
|
|
memCleanup() |
|
|
|
|
|
require.NoError(t, rts.reactor.Stop()) |
|
|
|
|
|
require.False(t, rts.reactor.IsRunning()) |
|
|
|
|
|
}) |
|
|
|
|
|
|
|
|
|
|
|
return rts |
|
|
} |
|
|
} |
|
|
|
|
|
|
|
|
// Send a bunch of txs to the first reactor's mempool and wait for them all to
|
|
|
|
|
|
// be received in the others.
|
|
|
|
|
|
func TestReactorBroadcastTxsMessage(t *testing.T) { |
|
|
|
|
|
config := cfg.TestConfig() |
|
|
|
|
|
// if there were more than two reactors, the order of transactions could not be
|
|
|
|
|
|
// asserted in waitForTxsOnReactors (due to transactions gossiping). If we
|
|
|
|
|
|
// replace Connect2Switches (full mesh) with a func, which connects first
|
|
|
|
|
|
// reactor to others and nothing else, this test should also pass with >2 reactors.
|
|
|
|
|
|
const N = 2 |
|
|
|
|
|
reactors := makeAndConnectReactors(config, N) |
|
|
|
|
|
defer func() { |
|
|
|
|
|
for _, r := range reactors { |
|
|
|
|
|
if err := r.Stop(); err != nil { |
|
|
|
|
|
assert.NoError(t, err) |
|
|
|
|
|
|
|
|
func simulateRouter(wg *sync.WaitGroup, primary *reactorTestSuite, suites []*reactorTestSuite, numOut int) { |
|
|
|
|
|
wg.Add(1) |
|
|
|
|
|
|
|
|
|
|
|
// create a mapping for efficient suite lookup by peer ID
|
|
|
|
|
|
suitesByPeerID := make(map[p2p.NodeID]*reactorTestSuite) |
|
|
|
|
|
for _, suite := range suites { |
|
|
|
|
|
suitesByPeerID[suite.peerID] = suite |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
// Simulate a router by listening for all outbound envelopes and proxying the
|
|
|
|
|
|
// envelope to the respective peer (suite).
|
|
|
|
|
|
go func() { |
|
|
|
|
|
for i := 0; i < numOut; i++ { |
|
|
|
|
|
envelope := <-primary.mempoolOutCh |
|
|
|
|
|
other := suitesByPeerID[envelope.To] |
|
|
|
|
|
|
|
|
|
|
|
other.mempoolInCh <- p2p.Envelope{ |
|
|
|
|
|
From: primary.peerID, |
|
|
|
|
|
To: envelope.To, |
|
|
|
|
|
Message: envelope.Message, |
|
|
} |
|
|
} |
|
|
} |
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
wg.Done() |
|
|
}() |
|
|
}() |
|
|
for _, r := range reactors { |
|
|
|
|
|
for _, peer := range r.Switch.Peers().List() { |
|
|
|
|
|
peer.Set(types.PeerStateKey, peerState{1}) |
|
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
func waitForTxs(t *testing.T, txs types.Txs, suites ...*reactorTestSuite) { |
|
|
|
|
|
t.Helper() |
|
|
|
|
|
|
|
|
|
|
|
wg := new(sync.WaitGroup) |
|
|
|
|
|
|
|
|
|
|
|
for _, suite := range suites { |
|
|
|
|
|
wg.Add(1) |
|
|
|
|
|
|
|
|
|
|
|
go func(s *reactorTestSuite) { |
|
|
|
|
|
mempool := s.reactor.mempool |
|
|
|
|
|
for mempool.Size() < len(txs) { |
|
|
|
|
|
time.Sleep(time.Millisecond * 100) |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
reapedTxs := mempool.ReapMaxTxs(len(txs)) |
|
|
|
|
|
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], |
|
|
|
|
|
) |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
wg.Done() |
|
|
|
|
|
}(suite) |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
wg.Wait() |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
func TestReactorBroadcastTxs(t *testing.T) { |
|
|
|
|
|
numTxs := 1000 |
|
|
|
|
|
numNodes := 10 |
|
|
|
|
|
config := cfg.TestConfig() |
|
|
|
|
|
|
|
|
|
|
|
testSuites := make([]*reactorTestSuite, numNodes) |
|
|
|
|
|
for i := 0; i < len(testSuites); i++ { |
|
|
|
|
|
logger := log.TestingLogger().With("node", i) |
|
|
|
|
|
testSuites[i] = setup(t, config.Mempool, logger, 0) |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
primary := testSuites[0] |
|
|
|
|
|
secondaries := testSuites[1:] |
|
|
|
|
|
|
|
|
|
|
|
// Simulate a router by listening for all outbound envelopes and proxying the
|
|
|
|
|
|
// envelopes to the respective peer (suite).
|
|
|
|
|
|
wg := new(sync.WaitGroup) |
|
|
|
|
|
simulateRouter(wg, primary, testSuites, numTxs*len(secondaries)) |
|
|
|
|
|
|
|
|
|
|
|
txs := checkTxs(t, primary.reactor.mempool, numTxs, UnknownPeerID) |
|
|
|
|
|
|
|
|
|
|
|
// Add each secondary suite (node) as a peer to the primary suite (node). This
|
|
|
|
|
|
// will cause the primary to gossip all mempool txs to the secondaries.
|
|
|
|
|
|
for _, suite := range secondaries { |
|
|
|
|
|
primary.peerUpdatesCh <- p2p.PeerUpdate{ |
|
|
|
|
|
Status: p2p.PeerStatusUp, |
|
|
|
|
|
PeerID: suite.peerID, |
|
|
} |
|
|
} |
|
|
} |
|
|
} |
|
|
|
|
|
|
|
|
txs := checkTxs(t, reactors[0].mempool, numTxs, UnknownPeerID) |
|
|
|
|
|
waitForTxsOnReactors(t, txs, reactors) |
|
|
|
|
|
|
|
|
// Wait till all secondary suites (reactor) received all mempool txs from the
|
|
|
|
|
|
// primary suite (node).
|
|
|
|
|
|
waitForTxs(t, txs, secondaries...) |
|
|
|
|
|
|
|
|
|
|
|
for _, suite := range testSuites { |
|
|
|
|
|
require.Equal(t, len(txs), suite.reactor.mempool.Size()) |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
wg.Wait() |
|
|
|
|
|
|
|
|
|
|
|
// ensure all channels are drained
|
|
|
|
|
|
for _, suite := range testSuites { |
|
|
|
|
|
require.Empty(t, suite.mempoolOutCh) |
|
|
|
|
|
} |
|
|
} |
|
|
} |
|
|
|
|
|
|
|
|
// regression test for https://github.com/tendermint/tendermint/issues/5408
|
|
|
// regression test for https://github.com/tendermint/tendermint/issues/5408
|
|
|
func TestReactorConcurrency(t *testing.T) { |
|
|
func TestReactorConcurrency(t *testing.T) { |
|
|
|
|
|
numTxs := 5 |
|
|
|
|
|
numNodes := 2 |
|
|
config := cfg.TestConfig() |
|
|
config := cfg.TestConfig() |
|
|
const N = 2 |
|
|
|
|
|
reactors := makeAndConnectReactors(config, N) |
|
|
|
|
|
defer func() { |
|
|
|
|
|
for _, r := range reactors { |
|
|
|
|
|
if err := r.Stop(); err != nil { |
|
|
|
|
|
assert.NoError(t, err) |
|
|
|
|
|
} |
|
|
|
|
|
} |
|
|
|
|
|
}() |
|
|
|
|
|
for _, r := range reactors { |
|
|
|
|
|
for _, peer := range r.Switch.Peers().List() { |
|
|
|
|
|
peer.Set(types.PeerStateKey, peerState{1}) |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
testSuites := make([]*reactorTestSuite, numNodes) |
|
|
|
|
|
for i := 0; i < len(testSuites); i++ { |
|
|
|
|
|
logger := log.TestingLogger().With("node", i) |
|
|
|
|
|
testSuites[i] = setup(t, config.Mempool, logger, 0) |
|
|
} |
|
|
} |
|
|
var wg sync.WaitGroup |
|
|
|
|
|
|
|
|
|
|
|
const numTxs = 5 |
|
|
|
|
|
|
|
|
primary := testSuites[0] |
|
|
|
|
|
secondary := testSuites[1] |
|
|
|
|
|
|
|
|
|
|
|
var wg sync.WaitGroup |
|
|
|
|
|
|
|
|
for i := 0; i < 1000; i++ { |
|
|
for i := 0; i < 1000; i++ { |
|
|
wg.Add(2) |
|
|
wg.Add(2) |
|
|
|
|
|
|
|
|
// 1. submit a bunch of txs
|
|
|
// 1. submit a bunch of txs
|
|
|
// 2. update the whole mempool
|
|
|
// 2. update the whole mempool
|
|
|
txs := checkTxs(t, reactors[0].mempool, numTxs, UnknownPeerID) |
|
|
|
|
|
|
|
|
txs := checkTxs(t, primary.reactor.mempool, numTxs, UnknownPeerID) |
|
|
go func() { |
|
|
go func() { |
|
|
defer wg.Done() |
|
|
defer wg.Done() |
|
|
|
|
|
|
|
|
reactors[0].mempool.Lock() |
|
|
|
|
|
defer reactors[0].mempool.Unlock() |
|
|
|
|
|
|
|
|
primary.reactor.mempool.Lock() |
|
|
|
|
|
defer primary.reactor.mempool.Unlock() |
|
|
|
|
|
|
|
|
deliverTxResponses := make([]*abci.ResponseDeliverTx, len(txs)) |
|
|
deliverTxResponses := make([]*abci.ResponseDeliverTx, len(txs)) |
|
|
for i := range txs { |
|
|
for i := range txs { |
|
|
deliverTxResponses[i] = &abci.ResponseDeliverTx{Code: 0} |
|
|
deliverTxResponses[i] = &abci.ResponseDeliverTx{Code: 0} |
|
|
} |
|
|
} |
|
|
err := reactors[0].mempool.Update(1, txs, deliverTxResponses, nil, nil) |
|
|
|
|
|
assert.NoError(t, err) |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
err := primary.reactor.mempool.Update(1, txs, deliverTxResponses, nil, nil) |
|
|
|
|
|
require.NoError(t, err) |
|
|
}() |
|
|
}() |
|
|
|
|
|
|
|
|
// 1. submit a bunch of txs
|
|
|
// 1. submit a bunch of txs
|
|
|
// 2. update none
|
|
|
// 2. update none
|
|
|
_ = checkTxs(t, reactors[1].mempool, numTxs, UnknownPeerID) |
|
|
|
|
|
|
|
|
_ = checkTxs(t, secondary.reactor.mempool, numTxs, UnknownPeerID) |
|
|
go func() { |
|
|
go func() { |
|
|
defer wg.Done() |
|
|
defer wg.Done() |
|
|
|
|
|
|
|
|
reactors[1].mempool.Lock() |
|
|
|
|
|
defer reactors[1].mempool.Unlock() |
|
|
|
|
|
err := reactors[1].mempool.Update(1, []types.Tx{}, make([]*abci.ResponseDeliverTx, 0), nil, nil) |
|
|
|
|
|
assert.NoError(t, err) |
|
|
|
|
|
|
|
|
secondary.reactor.mempool.Lock() |
|
|
|
|
|
defer secondary.reactor.mempool.Unlock() |
|
|
|
|
|
|
|
|
|
|
|
err := secondary.reactor.mempool.Update(1, []types.Tx{}, make([]*abci.ResponseDeliverTx, 0), nil, nil) |
|
|
|
|
|
require.NoError(t, err) |
|
|
}() |
|
|
}() |
|
|
|
|
|
|
|
|
// 1. flush the mempool
|
|
|
|
|
|
reactors[1].mempool.Flush() |
|
|
|
|
|
|
|
|
// flush the mempool
|
|
|
|
|
|
secondary.reactor.mempool.Flush() |
|
|
} |
|
|
} |
|
|
|
|
|
|
|
|
wg.Wait() |
|
|
wg.Wait() |
|
|
} |
|
|
} |
|
|
|
|
|
|
|
|
// Send a bunch of txs to the first reactor's mempool, claiming it came from peer
|
|
|
|
|
|
// ensure peer gets no txs.
|
|
|
|
|
|
func TestReactorNoBroadcastToSender(t *testing.T) { |
|
|
func TestReactorNoBroadcastToSender(t *testing.T) { |
|
|
|
|
|
numTxs := 1000 |
|
|
|
|
|
numNodes := 2 |
|
|
config := cfg.TestConfig() |
|
|
config := cfg.TestConfig() |
|
|
const N = 2 |
|
|
|
|
|
reactors := makeAndConnectReactors(config, N) |
|
|
|
|
|
defer func() { |
|
|
|
|
|
for _, r := range reactors { |
|
|
|
|
|
if err := r.Stop(); err != nil { |
|
|
|
|
|
assert.NoError(t, err) |
|
|
|
|
|
} |
|
|
|
|
|
} |
|
|
|
|
|
}() |
|
|
|
|
|
for _, r := range reactors { |
|
|
|
|
|
for _, peer := range r.Switch.Peers().List() { |
|
|
|
|
|
peer.Set(types.PeerStateKey, peerState{1}) |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
testSuites := make([]*reactorTestSuite, numNodes) |
|
|
|
|
|
for i := 0; i < len(testSuites); i++ { |
|
|
|
|
|
logger := log.TestingLogger().With("node", i) |
|
|
|
|
|
testSuites[i] = setup(t, config.Mempool, logger, uint(numTxs)) |
|
|
} |
|
|
} |
|
|
|
|
|
|
|
|
const peerID = 1 |
|
|
|
|
|
checkTxs(t, reactors[0].mempool, numTxs, peerID) |
|
|
|
|
|
ensureNoTxs(t, reactors[peerID], 100*time.Millisecond) |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
primary := testSuites[0] |
|
|
|
|
|
secondary := testSuites[1] |
|
|
|
|
|
|
|
|
func TestReactor_MaxTxBytes(t *testing.T) { |
|
|
|
|
|
config := cfg.TestConfig() |
|
|
|
|
|
|
|
|
peerID := uint16(1) |
|
|
|
|
|
_ = checkTxs(t, primary.reactor.mempool, numTxs, peerID) |
|
|
|
|
|
|
|
|
const N = 2 |
|
|
|
|
|
reactors := makeAndConnectReactors(config, N) |
|
|
|
|
|
defer func() { |
|
|
|
|
|
for _, r := range reactors { |
|
|
|
|
|
if err := r.Stop(); err != nil { |
|
|
|
|
|
assert.NoError(t, err) |
|
|
|
|
|
} |
|
|
|
|
|
} |
|
|
|
|
|
}() |
|
|
|
|
|
for _, r := range reactors { |
|
|
|
|
|
for _, peer := range r.Switch.Peers().List() { |
|
|
|
|
|
peer.Set(types.PeerStateKey, peerState{1}) |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
primary.peerUpdatesCh <- p2p.PeerUpdate{ |
|
|
|
|
|
Status: p2p.PeerStatusUp, |
|
|
|
|
|
PeerID: secondary.peerID, |
|
|
} |
|
|
} |
|
|
|
|
|
|
|
|
// Broadcast a tx, which has the max size
|
|
|
|
|
|
// => ensure it's received by the second reactor.
|
|
|
|
|
|
tx1 := tmrand.Bytes(config.Mempool.MaxTxBytes) |
|
|
|
|
|
err := reactors[0].mempool.CheckTx(tx1, nil, TxInfo{SenderID: UnknownPeerID}) |
|
|
|
|
|
require.NoError(t, err) |
|
|
|
|
|
waitForTxsOnReactors(t, []types.Tx{tx1}, reactors) |
|
|
|
|
|
|
|
|
time.Sleep(100 * time.Millisecond) |
|
|
|
|
|
|
|
|
reactors[0].mempool.Flush() |
|
|
|
|
|
reactors[1].mempool.Flush() |
|
|
|
|
|
|
|
|
require.Eventually(t, func() bool { |
|
|
|
|
|
return secondary.reactor.mempool.Size() == 0 |
|
|
|
|
|
}, time.Minute, 100*time.Millisecond) |
|
|
|
|
|
|
|
|
// Broadcast a tx, which is beyond the max size
|
|
|
|
|
|
// => ensure it's not sent
|
|
|
|
|
|
tx2 := tmrand.Bytes(config.Mempool.MaxTxBytes + 1) |
|
|
|
|
|
err = reactors[0].mempool.CheckTx(tx2, nil, TxInfo{SenderID: UnknownPeerID}) |
|
|
|
|
|
require.Error(t, err) |
|
|
|
|
|
|
|
|
// ensure all channels are drained
|
|
|
|
|
|
for _, suite := range testSuites { |
|
|
|
|
|
require.Empty(t, suite.mempoolOutCh) |
|
|
|
|
|
} |
|
|
} |
|
|
} |
|
|
|
|
|
|
|
|
func TestBroadcastTxForPeerStopsWhenPeerStops(t *testing.T) { |
|
|
|
|
|
if testing.Short() { |
|
|
|
|
|
t.Skip("skipping test in short mode.") |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
func TestMempoolIDsBasic(t *testing.T) { |
|
|
|
|
|
ids := newMempoolIDs() |
|
|
|
|
|
|
|
|
config := cfg.TestConfig() |
|
|
|
|
|
const N = 2 |
|
|
|
|
|
reactors := makeAndConnectReactors(config, N) |
|
|
|
|
|
defer func() { |
|
|
|
|
|
for _, r := range reactors { |
|
|
|
|
|
if err := r.Stop(); err != nil { |
|
|
|
|
|
assert.NoError(t, err) |
|
|
|
|
|
} |
|
|
|
|
|
} |
|
|
|
|
|
}() |
|
|
|
|
|
|
|
|
peerID, err := p2p.NewNodeID("00ffaa") |
|
|
|
|
|
require.NoError(t, err) |
|
|
|
|
|
|
|
|
// stop peer
|
|
|
|
|
|
sw := reactors[1].Switch |
|
|
|
|
|
sw.StopPeerForError(sw.Peers().List()[0], errors.New("some reason")) |
|
|
|
|
|
|
|
|
ids.ReserveForPeer(peerID) |
|
|
|
|
|
require.EqualValues(t, 1, ids.GetForPeer(peerID)) |
|
|
|
|
|
ids.Reclaim(peerID) |
|
|
|
|
|
|
|
|
// check that we are not leaking any go-routines
|
|
|
|
|
|
// i.e. broadcastTxRoutine finishes when peer is stopped
|
|
|
|
|
|
leaktest.CheckTimeout(t, 10*time.Second)() |
|
|
|
|
|
|
|
|
ids.ReserveForPeer(peerID) |
|
|
|
|
|
require.EqualValues(t, 2, ids.GetForPeer(peerID)) |
|
|
|
|
|
ids.Reclaim(peerID) |
|
|
} |
|
|
} |
|
|
|
|
|
|
|
|
func TestBroadcastTxForPeerStopsWhenReactorStops(t *testing.T) { |
|
|
|
|
|
if testing.Short() { |
|
|
|
|
|
t.Skip("skipping test in short mode.") |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
func TestReactor_MaxTxBytes(t *testing.T) { |
|
|
|
|
|
numNodes := 2 |
|
|
config := cfg.TestConfig() |
|
|
config := cfg.TestConfig() |
|
|
const N = 2 |
|
|
|
|
|
reactors := makeAndConnectReactors(config, N) |
|
|
|
|
|
|
|
|
|
|
|
// stop reactors
|
|
|
|
|
|
for _, r := range reactors { |
|
|
|
|
|
if err := r.Stop(); err != nil { |
|
|
|
|
|
assert.NoError(t, err) |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
testSuites := make([]*reactorTestSuite, numNodes) |
|
|
|
|
|
for i := 0; i < len(testSuites); i++ { |
|
|
|
|
|
logger := log.TestingLogger().With("node", i) |
|
|
|
|
|
testSuites[i] = setup(t, config.Mempool, logger, 0) |
|
|
} |
|
|
} |
|
|
|
|
|
|
|
|
// check that we are not leaking any go-routines
|
|
|
|
|
|
// i.e. broadcastTxRoutine finishes when reactor is stopped
|
|
|
|
|
|
leaktest.CheckTimeout(t, 10*time.Second)() |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
primary := testSuites[0] |
|
|
|
|
|
secondary := testSuites[1] |
|
|
|
|
|
|
|
|
func TestMempoolIDsBasic(t *testing.T) { |
|
|
|
|
|
ids := newMempoolIDs() |
|
|
|
|
|
|
|
|
// Simulate a router by listening for all outbound envelopes and proxying the
|
|
|
|
|
|
// envelopes to the respective peer (suite).
|
|
|
|
|
|
wg := new(sync.WaitGroup) |
|
|
|
|
|
simulateRouter(wg, primary, testSuites, 1) |
|
|
|
|
|
|
|
|
peer := mock.NewPeer(net.IP{127, 0, 0, 1}) |
|
|
|
|
|
|
|
|
// Broadcast a tx, which has the max size and ensure it's received by the
|
|
|
|
|
|
// second reactor.
|
|
|
|
|
|
tx1 := tmrand.Bytes(config.Mempool.MaxTxBytes) |
|
|
|
|
|
err := primary.reactor.mempool.CheckTx(tx1, nil, TxInfo{SenderID: UnknownPeerID}) |
|
|
|
|
|
require.NoError(t, err) |
|
|
|
|
|
|
|
|
ids.ReserveForPeer(peer) |
|
|
|
|
|
assert.EqualValues(t, 1, ids.GetForPeer(peer)) |
|
|
|
|
|
ids.Reclaim(peer) |
|
|
|
|
|
|
|
|
primary.peerUpdatesCh <- p2p.PeerUpdate{ |
|
|
|
|
|
Status: p2p.PeerStatusUp, |
|
|
|
|
|
PeerID: secondary.peerID, |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
ids.ReserveForPeer(peer) |
|
|
|
|
|
assert.EqualValues(t, 2, ids.GetForPeer(peer)) |
|
|
|
|
|
ids.Reclaim(peer) |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
// Wait till all secondary suites (reactor) received all mempool txs from the
|
|
|
|
|
|
// primary suite (node).
|
|
|
|
|
|
waitForTxs(t, []types.Tx{tx1}, secondary) |
|
|
|
|
|
|
|
|
func TestMempoolIDsPanicsIfNodeRequestsOvermaxActiveIDs(t *testing.T) { |
|
|
|
|
|
if testing.Short() { |
|
|
|
|
|
return |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
primary.reactor.mempool.Flush() |
|
|
|
|
|
secondary.reactor.mempool.Flush() |
|
|
|
|
|
|
|
|
// 0 is already reserved for UnknownPeerID
|
|
|
|
|
|
ids := newMempoolIDs() |
|
|
|
|
|
|
|
|
// broadcast a tx, which is beyond the max size and ensure it's not sent
|
|
|
|
|
|
tx2 := tmrand.Bytes(config.Mempool.MaxTxBytes + 1) |
|
|
|
|
|
err = primary.reactor.mempool.CheckTx(tx2, nil, TxInfo{SenderID: UnknownPeerID}) |
|
|
|
|
|
require.Error(t, err) |
|
|
|
|
|
|
|
|
for i := 0; i < maxActiveIDs-1; i++ { |
|
|
|
|
|
peer := mock.NewPeer(net.IP{127, 0, 0, 1}) |
|
|
|
|
|
ids.ReserveForPeer(peer) |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
wg.Wait() |
|
|
|
|
|
|
|
|
assert.Panics(t, func() { |
|
|
|
|
|
peer := mock.NewPeer(net.IP{127, 0, 0, 1}) |
|
|
|
|
|
ids.ReserveForPeer(peer) |
|
|
|
|
|
}) |
|
|
|
|
|
|
|
|
// ensure all channels are drained
|
|
|
|
|
|
for _, suite := range testSuites { |
|
|
|
|
|
require.Empty(t, suite.mempoolOutCh) |
|
|
|
|
|
} |
|
|
} |
|
|
} |
|
|
|
|
|
|
|
|
func TestDontExhaustMaxActiveIDs(t *testing.T) { |
|
|
func TestDontExhaustMaxActiveIDs(t *testing.T) { |
|
|
config := cfg.TestConfig() |
|
|
config := cfg.TestConfig() |
|
|
const N = 1 |
|
|
|
|
|
reactors := makeAndConnectReactors(config, N) |
|
|
|
|
|
defer func() { |
|
|
|
|
|
for _, r := range reactors { |
|
|
|
|
|
if err := r.Stop(); err != nil { |
|
|
|
|
|
assert.NoError(t, err) |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
reactor := setup(t, config.Mempool, log.TestingLogger().With("node", 0), 0) |
|
|
|
|
|
|
|
|
|
|
|
go func() { |
|
|
|
|
|
for range reactor.mempoolOutCh { |
|
|
} |
|
|
} |
|
|
}() |
|
|
}() |
|
|
reactor := reactors[0] |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
peerID, err := p2p.NewNodeID("00ffaa") |
|
|
|
|
|
require.NoError(t, err) |
|
|
|
|
|
|
|
|
|
|
|
// ensure the reactor does not panic (i.e. exhaust active IDs)
|
|
|
for i := 0; i < maxActiveIDs+1; i++ { |
|
|
for i := 0; i < maxActiveIDs+1; i++ { |
|
|
peer := mock.NewPeer(nil) |
|
|
|
|
|
reactor.Receive(MempoolChannel, peer, []byte{0x1, 0x2, 0x3}) |
|
|
|
|
|
reactor.AddPeer(peer) |
|
|
|
|
|
|
|
|
reactor.peerUpdatesCh <- p2p.PeerUpdate{ |
|
|
|
|
|
Status: p2p.PeerStatusUp, |
|
|
|
|
|
PeerID: peerID, |
|
|
|
|
|
} |
|
|
|
|
|
reactor.mempoolOutCh <- p2p.Envelope{ |
|
|
|
|
|
To: peerID, |
|
|
|
|
|
Message: &protomem.Txs{ |
|
|
|
|
|
Txs: [][]byte{}, |
|
|
|
|
|
}, |
|
|
|
|
|
} |
|
|
} |
|
|
} |
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
// mempoolLogger is a TestingLogger which uses a different
|
|
|
|
|
|
// color for each validator ("validator" key must exist).
|
|
|
|
|
|
func mempoolLogger() log.Logger { |
|
|
|
|
|
return log.TestingLoggerWithColorFn(func(keyvals ...interface{}) term.FgBgColor { |
|
|
|
|
|
for i := 0; i < len(keyvals)-1; i += 2 { |
|
|
|
|
|
if keyvals[i] == "validator" { |
|
|
|
|
|
return term.FgBgColor{Fg: term.Color(uint8(keyvals[i+1].(int) + 1))} |
|
|
|
|
|
} |
|
|
|
|
|
} |
|
|
|
|
|
return term.FgBgColor{} |
|
|
|
|
|
}) |
|
|
|
|
|
|
|
|
require.Empty(t, reactor.mempoolOutCh) |
|
|
} |
|
|
} |
|
|
|
|
|
|
|
|
// connect N mempool reactors through N switches
|
|
|
|
|
|
func makeAndConnectReactors(config *cfg.Config, n int) []*Reactor { |
|
|
|
|
|
reactors := make([]*Reactor, n) |
|
|
|
|
|
logger := mempoolLogger() |
|
|
|
|
|
for i := 0; i < n; i++ { |
|
|
|
|
|
app := kvstore.NewApplication() |
|
|
|
|
|
cc := proxy.NewLocalClientCreator(app) |
|
|
|
|
|
mempool, cleanup := newMempoolWithApp(cc) |
|
|
|
|
|
defer cleanup() |
|
|
|
|
|
|
|
|
|
|
|
reactors[i] = NewReactor(config.Mempool, mempool) // so we dont start the consensus states
|
|
|
|
|
|
reactors[i].SetLogger(logger.With("validator", i)) |
|
|
|
|
|
|
|
|
func TestMempoolIDsPanicsIfNodeRequestsOvermaxActiveIDs(t *testing.T) { |
|
|
|
|
|
if testing.Short() { |
|
|
|
|
|
t.Skip("skipping test in short mode") |
|
|
} |
|
|
} |
|
|
|
|
|
|
|
|
p2p.MakeConnectedSwitches(config.P2P, n, func(i int, s *p2p.Switch) *p2p.Switch { |
|
|
|
|
|
s.AddReactor("MEMPOOL", reactors[i]) |
|
|
|
|
|
return s |
|
|
|
|
|
|
|
|
// 0 is already reserved for UnknownPeerID
|
|
|
|
|
|
ids := newMempoolIDs() |
|
|
|
|
|
|
|
|
}, p2p.Connect2Switches) |
|
|
|
|
|
return reactors |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
peerID, err := p2p.NewNodeID("00ffaa") |
|
|
|
|
|
require.NoError(t, err) |
|
|
|
|
|
|
|
|
func waitForTxsOnReactors(t *testing.T, txs types.Txs, reactors []*Reactor) { |
|
|
|
|
|
// wait for the txs in all mempools
|
|
|
|
|
|
wg := new(sync.WaitGroup) |
|
|
|
|
|
for i, reactor := range reactors { |
|
|
|
|
|
wg.Add(1) |
|
|
|
|
|
go func(r *Reactor, reactorIndex int) { |
|
|
|
|
|
defer wg.Done() |
|
|
|
|
|
waitForTxsOnReactor(t, txs, r, reactorIndex) |
|
|
|
|
|
}(reactor, i) |
|
|
|
|
|
|
|
|
for i := 0; i < maxActiveIDs-1; i++ { |
|
|
|
|
|
ids.ReserveForPeer(peerID) |
|
|
} |
|
|
} |
|
|
|
|
|
|
|
|
done := make(chan struct{}) |
|
|
|
|
|
go func() { |
|
|
|
|
|
wg.Wait() |
|
|
|
|
|
close(done) |
|
|
|
|
|
}() |
|
|
|
|
|
|
|
|
|
|
|
timer := time.After(timeout) |
|
|
|
|
|
select { |
|
|
|
|
|
case <-timer: |
|
|
|
|
|
t.Fatal("Timed out waiting for txs") |
|
|
|
|
|
case <-done: |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
require.Panics(t, func() { |
|
|
|
|
|
ids.ReserveForPeer(peerID) |
|
|
|
|
|
}) |
|
|
} |
|
|
} |
|
|
|
|
|
|
|
|
func waitForTxsOnReactor(t *testing.T, txs types.Txs, reactor *Reactor, reactorIndex int) { |
|
|
|
|
|
mempool := reactor.mempool |
|
|
|
|
|
for mempool.Size() < len(txs) { |
|
|
|
|
|
time.Sleep(time.Millisecond * 100) |
|
|
|
|
|
|
|
|
func TestBroadcastTxForPeerStopsWhenPeerStops(t *testing.T) { |
|
|
|
|
|
if testing.Short() { |
|
|
|
|
|
t.Skip("skipping test in short mode") |
|
|
} |
|
|
} |
|
|
|
|
|
|
|
|
reapedTxs := mempool.ReapMaxTxs(len(txs)) |
|
|
|
|
|
for i, tx := range txs { |
|
|
|
|
|
assert.Equalf(t, tx, reapedTxs[i], |
|
|
|
|
|
"txs at index %d on reactor %d don't match: %v vs %v", i, reactorIndex, tx, reapedTxs[i]) |
|
|
|
|
|
} |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
config := cfg.TestConfig() |
|
|
|
|
|
|
|
|
// ensure no txs on reactor after some timeout
|
|
|
|
|
|
func ensureNoTxs(t *testing.T, reactor *Reactor, timeout time.Duration) { |
|
|
|
|
|
time.Sleep(timeout) // wait for the txs in all mempools
|
|
|
|
|
|
assert.Zero(t, reactor.mempool.Size()) |
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
primary := setup(t, config.Mempool, log.TestingLogger().With("node", 0), 0) |
|
|
|
|
|
secondary := setup(t, config.Mempool, log.TestingLogger().With("node", 1), 0) |
|
|
|
|
|
|
|
|
func TestMempoolVectors(t *testing.T) { |
|
|
|
|
|
testCases := []struct { |
|
|
|
|
|
testName string |
|
|
|
|
|
tx []byte |
|
|
|
|
|
expBytes string |
|
|
|
|
|
}{ |
|
|
|
|
|
{"tx 1", []byte{123}, "0a030a017b"}, |
|
|
|
|
|
{"tx 2", []byte("proto encoding in mempool"), "0a1b0a1970726f746f20656e636f64696e6720696e206d656d706f6f6c"}, |
|
|
|
|
|
|
|
|
// connect peer
|
|
|
|
|
|
primary.peerUpdatesCh <- p2p.PeerUpdate{ |
|
|
|
|
|
Status: p2p.PeerStatusUp, |
|
|
|
|
|
PeerID: secondary.peerID, |
|
|
} |
|
|
} |
|
|
|
|
|
|
|
|
for _, tc := range testCases { |
|
|
|
|
|
tc := tc |
|
|
|
|
|
|
|
|
|
|
|
msg := memproto.Message{ |
|
|
|
|
|
Sum: &memproto.Message_Txs{ |
|
|
|
|
|
Txs: &memproto.Txs{Txs: [][]byte{tc.tx}}, |
|
|
|
|
|
}, |
|
|
|
|
|
} |
|
|
|
|
|
bz, err := msg.Marshal() |
|
|
|
|
|
require.NoError(t, err, tc.testName) |
|
|
|
|
|
|
|
|
|
|
|
require.Equal(t, tc.expBytes, hex.EncodeToString(bz), tc.testName) |
|
|
|
|
|
|
|
|
// disconnect peer
|
|
|
|
|
|
primary.peerUpdatesCh <- p2p.PeerUpdate{ |
|
|
|
|
|
Status: p2p.PeerStatusDown, |
|
|
|
|
|
PeerID: secondary.peerID, |
|
|
} |
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
// check that we are not leaking any go-routines
|
|
|
|
|
|
// i.e. broadcastTxRoutine finishes when peer is stopped
|
|
|
|
|
|
leaktest.CheckTimeout(t, 10*time.Second)() |
|
|
} |
|
|
} |