You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.

425 lines
10 KiB

abci: Synchronize FinalizeBlock with the updated specification (#7983) This change set implements the most recent version of `FinalizeBlock`. # What does this change actually contain? * This change set is rather large but fear not! The majority of the files touched and changes are renaming `ResponseDeliverTx` to `ExecTxResult`. This should be a pretty inoffensive change since they're effectively the same type but with a different name. * The `execBlockOnProxyApp` was totally removed since it served as just a wrapper around the logic that is now mostly encapsulated within `FinalizeBlock` * The `updateState` helper function has been made a public method on `State`. It was being exposed as a shim through the testing infrastructure, so this seemed innocuous. * Tests already existed to ensure that the application received the `ByzantineValidators` and the `ValidatorUpdates`, but one was fixed up to ensure that `LastCommitInfo` was being sent across. * Tests were removed from the `psql` indexer that seemed to search for an event in the indexer that was not being created. # Questions for reviewers * We store this [ABCIResponses](https://github.com/tendermint/tendermint/blob/5721a13ab1f4479f9807f449f0bf5c536b9a05f2/proto/tendermint/state/types.pb.go#L37) type in the data base as the block results. This type has changed since v0.35 to contain the `FinalizeBlock` response. I'm wondering if we need to do any shimming to keep the old data retrieveable? * Similarly, this change is exposed via the RPC through [ResultBlockResults](https://github.com/tendermint/tendermint/blob/5721a13ab1f4479f9807f449f0bf5c536b9a05f2/rpc/coretypes/responses.go#L69) changing. Should we somehow shim or notify for this change? closes: #7658
2 years ago
abci: Synchronize FinalizeBlock with the updated specification (#7983) This change set implements the most recent version of `FinalizeBlock`. # What does this change actually contain? * This change set is rather large but fear not! The majority of the files touched and changes are renaming `ResponseDeliverTx` to `ExecTxResult`. This should be a pretty inoffensive change since they're effectively the same type but with a different name. * The `execBlockOnProxyApp` was totally removed since it served as just a wrapper around the logic that is now mostly encapsulated within `FinalizeBlock` * The `updateState` helper function has been made a public method on `State`. It was being exposed as a shim through the testing infrastructure, so this seemed innocuous. * Tests already existed to ensure that the application received the `ByzantineValidators` and the `ValidatorUpdates`, but one was fixed up to ensure that `LastCommitInfo` was being sent across. * Tests were removed from the `psql` indexer that seemed to search for an event in the indexer that was not being created. # Questions for reviewers * We store this [ABCIResponses](https://github.com/tendermint/tendermint/blob/5721a13ab1f4479f9807f449f0bf5c536b9a05f2/proto/tendermint/state/types.pb.go#L37) type in the data base as the block results. This type has changed since v0.35 to contain the `FinalizeBlock` response. I'm wondering if we need to do any shimming to keep the old data retrieveable? * Similarly, this change is exposed via the RPC through [ResultBlockResults](https://github.com/tendermint/tendermint/blob/5721a13ab1f4479f9807f449f0bf5c536b9a05f2/rpc/coretypes/responses.go#L69) changing. Should we somehow shim or notify for this change? closes: #7658
2 years ago
abci: Synchronize FinalizeBlock with the updated specification (#7983) This change set implements the most recent version of `FinalizeBlock`. # What does this change actually contain? * This change set is rather large but fear not! The majority of the files touched and changes are renaming `ResponseDeliverTx` to `ExecTxResult`. This should be a pretty inoffensive change since they're effectively the same type but with a different name. * The `execBlockOnProxyApp` was totally removed since it served as just a wrapper around the logic that is now mostly encapsulated within `FinalizeBlock` * The `updateState` helper function has been made a public method on `State`. It was being exposed as a shim through the testing infrastructure, so this seemed innocuous. * Tests already existed to ensure that the application received the `ByzantineValidators` and the `ValidatorUpdates`, but one was fixed up to ensure that `LastCommitInfo` was being sent across. * Tests were removed from the `psql` indexer that seemed to search for an event in the indexer that was not being created. # Questions for reviewers * We store this [ABCIResponses](https://github.com/tendermint/tendermint/blob/5721a13ab1f4479f9807f449f0bf5c536b9a05f2/proto/tendermint/state/types.pb.go#L37) type in the data base as the block results. This type has changed since v0.35 to contain the `FinalizeBlock` response. I'm wondering if we need to do any shimming to keep the old data retrieveable? * Similarly, this change is exposed via the RPC through [ResultBlockResults](https://github.com/tendermint/tendermint/blob/5721a13ab1f4479f9807f449f0bf5c536b9a05f2/rpc/coretypes/responses.go#L69) changing. Should we somehow shim or notify for this change? closes: #7658
2 years ago
  1. package mempool
  2. import (
  3. "context"
  4. "fmt"
  5. "os"
  6. "runtime"
  7. "strings"
  8. "sync"
  9. "testing"
  10. "time"
  11. "github.com/fortytw2/leaktest"
  12. "github.com/stretchr/testify/require"
  13. abciclient "github.com/tendermint/tendermint/abci/client"
  14. "github.com/tendermint/tendermint/abci/example/kvstore"
  15. abci "github.com/tendermint/tendermint/abci/types"
  16. "github.com/tendermint/tendermint/config"
  17. "github.com/tendermint/tendermint/internal/p2p"
  18. "github.com/tendermint/tendermint/internal/p2p/p2ptest"
  19. "github.com/tendermint/tendermint/libs/log"
  20. tmrand "github.com/tendermint/tendermint/libs/rand"
  21. protomem "github.com/tendermint/tendermint/proto/tendermint/mempool"
  22. "github.com/tendermint/tendermint/types"
  23. )
  24. type reactorTestSuite struct {
  25. network *p2ptest.Network
  26. logger log.Logger
  27. reactors map[types.NodeID]*Reactor
  28. mempoolChannels map[types.NodeID]*p2p.Channel
  29. mempools map[types.NodeID]*TxMempool
  30. kvstores map[types.NodeID]*kvstore.Application
  31. peerChans map[types.NodeID]chan p2p.PeerUpdate
  32. peerUpdates map[types.NodeID]*p2p.PeerUpdates
  33. nodes []types.NodeID
  34. }
  35. func setupReactors(ctx context.Context, t *testing.T, logger log.Logger, numNodes int, chBuf uint) *reactorTestSuite {
  36. t.Helper()
  37. cfg, err := config.ResetTestRoot(t.TempDir(), strings.ReplaceAll(t.Name(), "/", "|"))
  38. require.NoError(t, err)
  39. t.Cleanup(func() { os.RemoveAll(cfg.RootDir) })
  40. rts := &reactorTestSuite{
  41. logger: log.NewNopLogger().With("testCase", t.Name()),
  42. network: p2ptest.MakeNetwork(ctx, t, p2ptest.NetworkOptions{NumNodes: numNodes}),
  43. reactors: make(map[types.NodeID]*Reactor, numNodes),
  44. mempoolChannels: make(map[types.NodeID]*p2p.Channel, numNodes),
  45. mempools: make(map[types.NodeID]*TxMempool, numNodes),
  46. kvstores: make(map[types.NodeID]*kvstore.Application, numNodes),
  47. peerChans: make(map[types.NodeID]chan p2p.PeerUpdate, numNodes),
  48. peerUpdates: make(map[types.NodeID]*p2p.PeerUpdates, numNodes),
  49. }
  50. chDesc := getChannelDescriptor(cfg.Mempool)
  51. rts.mempoolChannels = rts.network.MakeChannelsNoCleanup(ctx, t, chDesc)
  52. for nodeID := range rts.network.Nodes {
  53. rts.kvstores[nodeID] = kvstore.NewApplication()
  54. client := abciclient.NewLocalClient(logger, rts.kvstores[nodeID])
  55. require.NoError(t, client.Start(ctx))
  56. t.Cleanup(client.Wait)
  57. mempool := setup(ctx, t, client, 0)
  58. rts.mempools[nodeID] = mempool
  59. rts.peerChans[nodeID] = make(chan p2p.PeerUpdate, chBuf)
  60. rts.peerUpdates[nodeID] = p2p.NewPeerUpdates(rts.peerChans[nodeID], 1)
  61. rts.network.Nodes[nodeID].PeerManager.Register(ctx, rts.peerUpdates[nodeID])
  62. chCreator := func(ctx context.Context, chDesc *p2p.ChannelDescriptor) (*p2p.Channel, error) {
  63. return rts.mempoolChannels[nodeID], nil
  64. }
  65. rts.reactors[nodeID], err = NewReactor(
  66. ctx,
  67. rts.logger.With("nodeID", nodeID),
  68. cfg.Mempool,
  69. rts.network.Nodes[nodeID].PeerManager,
  70. mempool,
  71. chCreator,
  72. rts.peerUpdates[nodeID],
  73. )
  74. require.NoError(t, err)
  75. rts.nodes = append(rts.nodes, nodeID)
  76. require.NoError(t, rts.reactors[nodeID].Start(ctx))
  77. require.True(t, rts.reactors[nodeID].IsRunning())
  78. }
  79. require.Len(t, rts.reactors, numNodes)
  80. t.Cleanup(func() {
  81. for nodeID := range rts.reactors {
  82. if rts.reactors[nodeID].IsRunning() {
  83. rts.reactors[nodeID].Stop()
  84. rts.reactors[nodeID].Wait()
  85. require.False(t, rts.reactors[nodeID].IsRunning())
  86. }
  87. }
  88. })
  89. t.Cleanup(leaktest.Check(t))
  90. return rts
  91. }
  92. func (rts *reactorTestSuite) start(ctx context.Context, t *testing.T) {
  93. t.Helper()
  94. rts.network.Start(ctx, t)
  95. require.Len(t,
  96. rts.network.RandomNode().PeerManager.Peers(),
  97. len(rts.nodes)-1,
  98. "network does not have expected number of nodes")
  99. }
  100. func (rts *reactorTestSuite) waitForTxns(t *testing.T, txs []types.Tx, ids ...types.NodeID) {
  101. t.Helper()
  102. // ensure that the transactions get fully broadcast to the
  103. // rest of the network
  104. wg := &sync.WaitGroup{}
  105. for name, pool := range rts.mempools {
  106. if !p2ptest.NodeInSlice(name, ids) {
  107. continue
  108. }
  109. if len(txs) == pool.Size() {
  110. continue
  111. }
  112. wg.Add(1)
  113. go func(pool *TxMempool) {
  114. defer wg.Done()
  115. require.Eventually(t, func() bool { return len(txs) == pool.Size() },
  116. time.Minute,
  117. 250*time.Millisecond,
  118. "ntx=%d, size=%d", len(txs), pool.Size(),
  119. )
  120. }(pool)
  121. }
  122. wg.Wait()
  123. }
  124. func TestReactorBroadcastDoesNotPanic(t *testing.T) {
  125. ctx, cancel := context.WithCancel(context.Background())
  126. defer cancel()
  127. const numNodes = 2
  128. logger := log.NewNopLogger()
  129. rts := setupReactors(ctx, t, logger, numNodes, 0)
  130. observePanic := func(r interface{}) {
  131. t.Fatal("panic detected in reactor")
  132. }
  133. primary := rts.nodes[0]
  134. secondary := rts.nodes[1]
  135. primaryReactor := rts.reactors[primary]
  136. primaryMempool := primaryReactor.mempool
  137. secondaryReactor := rts.reactors[secondary]
  138. primaryReactor.observePanic = observePanic
  139. secondaryReactor.observePanic = observePanic
  140. firstTx := &WrappedTx{}
  141. primaryMempool.insertTx(firstTx)
  142. // run the router
  143. rts.start(ctx, t)
  144. go primaryReactor.broadcastTxRoutine(ctx, secondary)
  145. wg := &sync.WaitGroup{}
  146. for i := 0; i < 50; i++ {
  147. next := &WrappedTx{}
  148. wg.Add(1)
  149. go func() {
  150. defer wg.Done()
  151. primaryMempool.insertTx(next)
  152. }()
  153. }
  154. primaryReactor.Stop()
  155. wg.Wait()
  156. }
  157. func TestReactorBroadcastTxs(t *testing.T) {
  158. numTxs := 1000
  159. numNodes := 10
  160. ctx, cancel := context.WithCancel(context.Background())
  161. defer cancel()
  162. logger := log.NewNopLogger()
  163. rts := setupReactors(ctx, t, logger, numNodes, uint(numTxs))
  164. primary := rts.nodes[0]
  165. secondaries := rts.nodes[1:]
  166. txs := checkTxs(ctx, t, rts.reactors[primary].mempool, numTxs, UnknownPeerID)
  167. require.Equal(t, numTxs, rts.reactors[primary].mempool.Size())
  168. rts.start(ctx, t)
  169. // Wait till all secondary suites (reactor) received all mempool txs from the
  170. // primary suite (node).
  171. rts.waitForTxns(t, convertTex(txs), secondaries...)
  172. }
  173. // regression test for https://github.com/tendermint/tendermint/issues/5408
  174. func TestReactorConcurrency(t *testing.T) {
  175. numTxs := 10
  176. numNodes := 2
  177. ctx, cancel := context.WithCancel(context.Background())
  178. defer cancel()
  179. logger := log.NewNopLogger()
  180. rts := setupReactors(ctx, t, logger, numNodes, 0)
  181. primary := rts.nodes[0]
  182. secondary := rts.nodes[1]
  183. rts.start(ctx, t)
  184. var wg sync.WaitGroup
  185. for i := 0; i < runtime.NumCPU()*2; i++ {
  186. wg.Add(2)
  187. // 1. submit a bunch of txs
  188. // 2. update the whole mempool
  189. txs := checkTxs(ctx, t, rts.reactors[primary].mempool, numTxs, UnknownPeerID)
  190. go func() {
  191. defer wg.Done()
  192. mempool := rts.mempools[primary]
  193. mempool.Lock()
  194. defer mempool.Unlock()
  195. deliverTxResponses := make([]*abci.ExecTxResult, len(txs))
  196. for i := range txs {
  197. deliverTxResponses[i] = &abci.ExecTxResult{Code: 0}
  198. }
  199. require.NoError(t, mempool.Update(ctx, 1, convertTex(txs), deliverTxResponses, nil, nil))
  200. }()
  201. // 1. submit a bunch of txs
  202. // 2. update none
  203. _ = checkTxs(ctx, t, rts.reactors[secondary].mempool, numTxs, UnknownPeerID)
  204. go func() {
  205. defer wg.Done()
  206. mempool := rts.mempools[secondary]
  207. mempool.Lock()
  208. defer mempool.Unlock()
  209. err := mempool.Update(ctx, 1, []types.Tx{}, make([]*abci.ExecTxResult, 0), nil, nil)
  210. require.NoError(t, err)
  211. }()
  212. }
  213. wg.Wait()
  214. }
  215. func TestReactorNoBroadcastToSender(t *testing.T) {
  216. numTxs := 1000
  217. numNodes := 2
  218. ctx, cancel := context.WithCancel(context.Background())
  219. defer cancel()
  220. logger := log.NewNopLogger()
  221. rts := setupReactors(ctx, t, logger, numNodes, uint(numTxs))
  222. primary := rts.nodes[0]
  223. secondary := rts.nodes[1]
  224. peerID := uint16(1)
  225. _ = checkTxs(ctx, t, rts.mempools[primary], numTxs, peerID)
  226. rts.start(ctx, t)
  227. time.Sleep(100 * time.Millisecond)
  228. require.Eventually(t, func() bool {
  229. return rts.mempools[secondary].Size() == 0
  230. }, time.Minute, 100*time.Millisecond)
  231. }
  232. func TestReactor_MaxTxBytes(t *testing.T) {
  233. numNodes := 2
  234. cfg := config.TestConfig()
  235. ctx, cancel := context.WithCancel(context.Background())
  236. defer cancel()
  237. logger := log.NewNopLogger()
  238. rts := setupReactors(ctx, t, logger, numNodes, 0)
  239. primary := rts.nodes[0]
  240. secondary := rts.nodes[1]
  241. // Broadcast a tx, which has the max size and ensure it's received by the
  242. // second reactor.
  243. tx1 := tmrand.Bytes(cfg.Mempool.MaxTxBytes)
  244. err := rts.reactors[primary].mempool.CheckTx(
  245. ctx,
  246. tx1,
  247. nil,
  248. TxInfo{
  249. SenderID: UnknownPeerID,
  250. },
  251. )
  252. require.NoError(t, err)
  253. rts.start(ctx, t)
  254. rts.reactors[primary].mempool.Flush()
  255. rts.reactors[secondary].mempool.Flush()
  256. // broadcast a tx, which is beyond the max size and ensure it's not sent
  257. tx2 := tmrand.Bytes(cfg.Mempool.MaxTxBytes + 1)
  258. err = rts.mempools[primary].CheckTx(ctx, tx2, nil, TxInfo{SenderID: UnknownPeerID})
  259. require.Error(t, err)
  260. }
  261. func TestDontExhaustMaxActiveIDs(t *testing.T) {
  262. // we're creating a single node network, but not starting the
  263. // network.
  264. ctx, cancel := context.WithCancel(context.Background())
  265. defer cancel()
  266. logger := log.NewNopLogger()
  267. rts := setupReactors(ctx, t, logger, 1, MaxActiveIDs+1)
  268. nodeID := rts.nodes[0]
  269. peerID, err := types.NewNodeID("0011223344556677889900112233445566778899")
  270. require.NoError(t, err)
  271. // ensure the reactor does not panic (i.e. exhaust active IDs)
  272. for i := 0; i < MaxActiveIDs+1; i++ {
  273. rts.peerChans[nodeID] <- p2p.PeerUpdate{
  274. Status: p2p.PeerStatusUp,
  275. NodeID: peerID,
  276. }
  277. require.NoError(t, rts.mempoolChannels[nodeID].Send(ctx, p2p.Envelope{
  278. To: peerID,
  279. Message: &protomem.Txs{
  280. Txs: [][]byte{},
  281. },
  282. }))
  283. }
  284. }
  285. func TestMempoolIDsPanicsIfNodeRequestsOvermaxActiveIDs(t *testing.T) {
  286. if testing.Short() {
  287. t.Skip("skipping test in short mode")
  288. }
  289. // 0 is already reserved for UnknownPeerID
  290. ids := NewMempoolIDs()
  291. for i := 0; i < MaxActiveIDs-1; i++ {
  292. peerID, err := types.NewNodeID(fmt.Sprintf("%040d", i))
  293. require.NoError(t, err)
  294. ids.ReserveForPeer(peerID)
  295. }
  296. peerID, err := types.NewNodeID(fmt.Sprintf("%040d", MaxActiveIDs-1))
  297. require.NoError(t, err)
  298. require.Panics(t, func() {
  299. ids.ReserveForPeer(peerID)
  300. })
  301. }
  302. func TestBroadcastTxForPeerStopsWhenPeerStops(t *testing.T) {
  303. if testing.Short() {
  304. t.Skip("skipping test in short mode")
  305. }
  306. ctx, cancel := context.WithCancel(context.Background())
  307. defer cancel()
  308. logger := log.NewNopLogger()
  309. rts := setupReactors(ctx, t, logger, 2, 2)
  310. primary := rts.nodes[0]
  311. secondary := rts.nodes[1]
  312. rts.start(ctx, t)
  313. // disconnect peer
  314. rts.peerChans[primary] <- p2p.PeerUpdate{
  315. Status: p2p.PeerStatusDown,
  316. NodeID: secondary,
  317. }
  318. time.Sleep(500 * time.Millisecond)
  319. txs := checkTxs(ctx, t, rts.reactors[primary].mempool, 4, UnknownPeerID)
  320. require.Equal(t, 4, len(txs))
  321. require.Equal(t, 4, rts.mempools[primary].Size())
  322. require.Equal(t, 0, rts.mempools[secondary].Size())
  323. }