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.

475 lines
14 KiB

8 years ago
8 years ago
8 years ago
  1. package mempool
  2. import (
  3. "crypto/rand"
  4. "crypto/sha256"
  5. "encoding/binary"
  6. "fmt"
  7. "io/ioutil"
  8. "os"
  9. "path/filepath"
  10. "testing"
  11. "time"
  12. "github.com/stretchr/testify/assert"
  13. "github.com/stretchr/testify/require"
  14. amino "github.com/tendermint/go-amino"
  15. "github.com/tendermint/tendermint/abci/example/counter"
  16. "github.com/tendermint/tendermint/abci/example/kvstore"
  17. abci "github.com/tendermint/tendermint/abci/types"
  18. cfg "github.com/tendermint/tendermint/config"
  19. cmn "github.com/tendermint/tendermint/libs/common"
  20. "github.com/tendermint/tendermint/libs/log"
  21. "github.com/tendermint/tendermint/proxy"
  22. "github.com/tendermint/tendermint/types"
  23. )
  24. // A cleanupFunc cleans up any config / test files created for a particular
  25. // test.
  26. type cleanupFunc func()
  27. func newMempoolWithApp(cc proxy.ClientCreator) (*Mempool, cleanupFunc) {
  28. config := cfg.ResetTestRoot("mempool_test")
  29. appConnMem, _ := cc.NewABCIClient()
  30. appConnMem.SetLogger(log.TestingLogger().With("module", "abci-client", "connection", "mempool"))
  31. err := appConnMem.Start()
  32. if err != nil {
  33. panic(err)
  34. }
  35. mempool := NewMempool(config.Mempool, appConnMem, 0)
  36. mempool.SetLogger(log.TestingLogger())
  37. return mempool, func() { os.RemoveAll(config.RootDir) }
  38. }
  39. func ensureNoFire(t *testing.T, ch <-chan struct{}, timeoutMS int) {
  40. timer := time.NewTimer(time.Duration(timeoutMS) * time.Millisecond)
  41. select {
  42. case <-ch:
  43. t.Fatal("Expected not to fire")
  44. case <-timer.C:
  45. }
  46. }
  47. func ensureFire(t *testing.T, ch <-chan struct{}, timeoutMS int) {
  48. timer := time.NewTimer(time.Duration(timeoutMS) * time.Millisecond)
  49. select {
  50. case <-ch:
  51. case <-timer.C:
  52. t.Fatal("Expected to fire")
  53. }
  54. }
  55. func checkTxs(t *testing.T, mempool *Mempool, count int) types.Txs {
  56. txs := make(types.Txs, count)
  57. for i := 0; i < count; i++ {
  58. txBytes := make([]byte, 20)
  59. txs[i] = txBytes
  60. _, err := rand.Read(txBytes)
  61. if err != nil {
  62. t.Error(err)
  63. }
  64. if err := mempool.CheckTx(txBytes, nil); err != nil {
  65. // Skip invalid txs.
  66. // TestMempoolFilters will fail otherwise. It asserts a number of txs
  67. // returned.
  68. if IsPreCheckError(err) {
  69. continue
  70. }
  71. t.Fatalf("CheckTx failed: %v while checking #%d tx", err, i)
  72. }
  73. }
  74. return txs
  75. }
  76. func TestReapMaxBytesMaxGas(t *testing.T) {
  77. app := kvstore.NewKVStoreApplication()
  78. cc := proxy.NewLocalClientCreator(app)
  79. mempool, cleanup := newMempoolWithApp(cc)
  80. defer cleanup()
  81. // Ensure gas calculation behaves as expected
  82. checkTxs(t, mempool, 1)
  83. tx0 := mempool.TxsFront().Value.(*mempoolTx)
  84. // assert that kv store has gas wanted = 1.
  85. require.Equal(t, app.CheckTx(tx0.tx).GasWanted, int64(1), "KVStore had a gas value neq to 1")
  86. require.Equal(t, tx0.gasWanted, int64(1), "transactions gas was set incorrectly")
  87. // ensure each tx is 20 bytes long
  88. require.Equal(t, len(tx0.tx), 20, "Tx is longer than 20 bytes")
  89. mempool.Flush()
  90. // each table driven test creates numTxsToCreate txs with checkTx, and at the end clears all remaining txs.
  91. // each tx has 20 bytes + amino overhead = 21 bytes, 1 gas
  92. tests := []struct {
  93. numTxsToCreate int
  94. maxBytes int64
  95. maxGas int64
  96. expectedNumTxs int
  97. }{
  98. {20, -1, -1, 20},
  99. {20, -1, 0, 0},
  100. {20, -1, 10, 10},
  101. {20, -1, 30, 20},
  102. {20, 0, -1, 0},
  103. {20, 0, 10, 0},
  104. {20, 10, 10, 0},
  105. {20, 22, 10, 1},
  106. {20, 220, -1, 10},
  107. {20, 220, 5, 5},
  108. {20, 220, 10, 10},
  109. {20, 220, 15, 10},
  110. {20, 20000, -1, 20},
  111. {20, 20000, 5, 5},
  112. {20, 20000, 30, 20},
  113. }
  114. for tcIndex, tt := range tests {
  115. checkTxs(t, mempool, tt.numTxsToCreate)
  116. got := mempool.ReapMaxBytesMaxGas(tt.maxBytes, tt.maxGas)
  117. assert.Equal(t, tt.expectedNumTxs, len(got), "Got %d txs, expected %d, tc #%d",
  118. len(got), tt.expectedNumTxs, tcIndex)
  119. mempool.Flush()
  120. }
  121. }
  122. func TestMempoolFilters(t *testing.T) {
  123. app := kvstore.NewKVStoreApplication()
  124. cc := proxy.NewLocalClientCreator(app)
  125. mempool, cleanup := newMempoolWithApp(cc)
  126. defer cleanup()
  127. emptyTxArr := []types.Tx{[]byte{}}
  128. nopPreFilter := func(tx types.Tx) error { return nil }
  129. nopPostFilter := func(tx types.Tx, res *abci.ResponseCheckTx) error { return nil }
  130. // each table driven test creates numTxsToCreate txs with checkTx, and at the end clears all remaining txs.
  131. // each tx has 20 bytes + amino overhead = 21 bytes, 1 gas
  132. tests := []struct {
  133. numTxsToCreate int
  134. preFilter PreCheckFunc
  135. postFilter PostCheckFunc
  136. expectedNumTxs int
  137. }{
  138. {10, nopPreFilter, nopPostFilter, 10},
  139. {10, PreCheckAminoMaxBytes(10), nopPostFilter, 0},
  140. {10, PreCheckAminoMaxBytes(20), nopPostFilter, 0},
  141. {10, PreCheckAminoMaxBytes(22), nopPostFilter, 10},
  142. {10, nopPreFilter, PostCheckMaxGas(-1), 10},
  143. {10, nopPreFilter, PostCheckMaxGas(0), 0},
  144. {10, nopPreFilter, PostCheckMaxGas(1), 10},
  145. {10, nopPreFilter, PostCheckMaxGas(3000), 10},
  146. {10, PreCheckAminoMaxBytes(10), PostCheckMaxGas(20), 0},
  147. {10, PreCheckAminoMaxBytes(30), PostCheckMaxGas(20), 10},
  148. {10, PreCheckAminoMaxBytes(22), PostCheckMaxGas(1), 10},
  149. {10, PreCheckAminoMaxBytes(22), PostCheckMaxGas(0), 0},
  150. }
  151. for tcIndex, tt := range tests {
  152. mempool.Update(1, emptyTxArr, tt.preFilter, tt.postFilter)
  153. checkTxs(t, mempool, tt.numTxsToCreate)
  154. require.Equal(t, tt.expectedNumTxs, mempool.Size(), "mempool had the incorrect size, on test case %d", tcIndex)
  155. mempool.Flush()
  156. }
  157. }
  158. func TestMempoolUpdateAddsTxsToCache(t *testing.T) {
  159. app := kvstore.NewKVStoreApplication()
  160. cc := proxy.NewLocalClientCreator(app)
  161. mempool, cleanup := newMempoolWithApp(cc)
  162. defer cleanup()
  163. mempool.Update(1, []types.Tx{[]byte{0x01}}, nil, nil)
  164. err := mempool.CheckTx([]byte{0x01}, nil)
  165. if assert.Error(t, err) {
  166. assert.Equal(t, ErrTxInCache, err)
  167. }
  168. }
  169. func TestTxsAvailable(t *testing.T) {
  170. app := kvstore.NewKVStoreApplication()
  171. cc := proxy.NewLocalClientCreator(app)
  172. mempool, cleanup := newMempoolWithApp(cc)
  173. defer cleanup()
  174. mempool.EnableTxsAvailable()
  175. timeoutMS := 500
  176. // with no txs, it shouldnt fire
  177. ensureNoFire(t, mempool.TxsAvailable(), timeoutMS)
  178. // send a bunch of txs, it should only fire once
  179. txs := checkTxs(t, mempool, 100)
  180. ensureFire(t, mempool.TxsAvailable(), timeoutMS)
  181. ensureNoFire(t, mempool.TxsAvailable(), timeoutMS)
  182. // call update with half the txs.
  183. // it should fire once now for the new height
  184. // since there are still txs left
  185. committedTxs, txs := txs[:50], txs[50:]
  186. if err := mempool.Update(1, committedTxs, nil, nil); err != nil {
  187. t.Error(err)
  188. }
  189. ensureFire(t, mempool.TxsAvailable(), timeoutMS)
  190. ensureNoFire(t, mempool.TxsAvailable(), timeoutMS)
  191. // send a bunch more txs. we already fired for this height so it shouldnt fire again
  192. moreTxs := checkTxs(t, mempool, 50)
  193. ensureNoFire(t, mempool.TxsAvailable(), timeoutMS)
  194. // now call update with all the txs. it should not fire as there are no txs left
  195. committedTxs = append(txs, moreTxs...)
  196. if err := mempool.Update(2, committedTxs, nil, nil); err != nil {
  197. t.Error(err)
  198. }
  199. ensureNoFire(t, mempool.TxsAvailable(), timeoutMS)
  200. // send a bunch more txs, it should only fire once
  201. checkTxs(t, mempool, 100)
  202. ensureFire(t, mempool.TxsAvailable(), timeoutMS)
  203. ensureNoFire(t, mempool.TxsAvailable(), timeoutMS)
  204. }
  205. func TestSerialReap(t *testing.T) {
  206. app := counter.NewCounterApplication(true)
  207. app.SetOption(abci.RequestSetOption{Key: "serial", Value: "on"})
  208. cc := proxy.NewLocalClientCreator(app)
  209. mempool, cleanup := newMempoolWithApp(cc)
  210. defer cleanup()
  211. appConnCon, _ := cc.NewABCIClient()
  212. appConnCon.SetLogger(log.TestingLogger().With("module", "abci-client", "connection", "consensus"))
  213. err := appConnCon.Start()
  214. require.Nil(t, err)
  215. cacheMap := make(map[string]struct{})
  216. deliverTxsRange := func(start, end int) {
  217. // Deliver some txs.
  218. for i := start; i < end; i++ {
  219. // This will succeed
  220. txBytes := make([]byte, 8)
  221. binary.BigEndian.PutUint64(txBytes, uint64(i))
  222. err := mempool.CheckTx(txBytes, nil)
  223. _, cached := cacheMap[string(txBytes)]
  224. if cached {
  225. require.NotNil(t, err, "expected error for cached tx")
  226. } else {
  227. require.Nil(t, err, "expected no err for uncached tx")
  228. }
  229. cacheMap[string(txBytes)] = struct{}{}
  230. // Duplicates are cached and should return error
  231. err = mempool.CheckTx(txBytes, nil)
  232. require.NotNil(t, err, "Expected error after CheckTx on duplicated tx")
  233. }
  234. }
  235. reapCheck := func(exp int) {
  236. txs := mempool.ReapMaxBytesMaxGas(-1, -1)
  237. require.Equal(t, len(txs), exp, fmt.Sprintf("Expected to reap %v txs but got %v", exp, len(txs)))
  238. }
  239. updateRange := func(start, end int) {
  240. txs := make([]types.Tx, 0)
  241. for i := start; i < end; i++ {
  242. txBytes := make([]byte, 8)
  243. binary.BigEndian.PutUint64(txBytes, uint64(i))
  244. txs = append(txs, txBytes)
  245. }
  246. if err := mempool.Update(0, txs, nil, nil); err != nil {
  247. t.Error(err)
  248. }
  249. }
  250. commitRange := func(start, end int) {
  251. // Deliver some txs.
  252. for i := start; i < end; i++ {
  253. txBytes := make([]byte, 8)
  254. binary.BigEndian.PutUint64(txBytes, uint64(i))
  255. res, err := appConnCon.DeliverTxSync(txBytes)
  256. if err != nil {
  257. t.Errorf("Client error committing tx: %v", err)
  258. }
  259. if res.IsErr() {
  260. t.Errorf("Error committing tx. Code:%v result:%X log:%v",
  261. res.Code, res.Data, res.Log)
  262. }
  263. }
  264. res, err := appConnCon.CommitSync()
  265. if err != nil {
  266. t.Errorf("Client error committing: %v", err)
  267. }
  268. if len(res.Data) != 8 {
  269. t.Errorf("Error committing. Hash:%X", res.Data)
  270. }
  271. }
  272. //----------------------------------------
  273. // Deliver some txs.
  274. deliverTxsRange(0, 100)
  275. // Reap the txs.
  276. reapCheck(100)
  277. // Reap again. We should get the same amount
  278. reapCheck(100)
  279. // Deliver 0 to 999, we should reap 900 new txs
  280. // because 100 were already counted.
  281. deliverTxsRange(0, 1000)
  282. // Reap the txs.
  283. reapCheck(1000)
  284. // Reap again. We should get the same amount
  285. reapCheck(1000)
  286. // Commit from the conensus AppConn
  287. commitRange(0, 500)
  288. updateRange(0, 500)
  289. // We should have 500 left.
  290. reapCheck(500)
  291. // Deliver 100 invalid txs and 100 valid txs
  292. deliverTxsRange(900, 1100)
  293. // We should have 600 now.
  294. reapCheck(600)
  295. }
  296. func TestCacheRemove(t *testing.T) {
  297. cache := newMapTxCache(100)
  298. numTxs := 10
  299. txs := make([][]byte, numTxs)
  300. for i := 0; i < numTxs; i++ {
  301. // probability of collision is 2**-256
  302. txBytes := make([]byte, 32)
  303. rand.Read(txBytes)
  304. txs[i] = txBytes
  305. cache.Push(txBytes)
  306. // make sure its added to both the linked list and the map
  307. require.Equal(t, i+1, len(cache.map_))
  308. require.Equal(t, i+1, cache.list.Len())
  309. }
  310. for i := 0; i < numTxs; i++ {
  311. cache.Remove(txs[i])
  312. // make sure its removed from both the map and the linked list
  313. require.Equal(t, numTxs-(i+1), len(cache.map_))
  314. require.Equal(t, numTxs-(i+1), cache.list.Len())
  315. }
  316. }
  317. func TestMempoolCloseWAL(t *testing.T) {
  318. // 1. Create the temporary directory for mempool and WAL testing.
  319. rootDir, err := ioutil.TempDir("", "mempool-test")
  320. require.Nil(t, err, "expecting successful tmpdir creation")
  321. defer os.RemoveAll(rootDir)
  322. // 2. Ensure that it doesn't contain any elements -- Sanity check
  323. m1, err := filepath.Glob(filepath.Join(rootDir, "*"))
  324. require.Nil(t, err, "successful globbing expected")
  325. require.Equal(t, 0, len(m1), "no matches yet")
  326. // 3. Create the mempool
  327. wcfg := cfg.DefaultMempoolConfig()
  328. wcfg.RootDir = rootDir
  329. defer os.RemoveAll(wcfg.RootDir)
  330. app := kvstore.NewKVStoreApplication()
  331. cc := proxy.NewLocalClientCreator(app)
  332. appConnMem, _ := cc.NewABCIClient()
  333. mempool := NewMempool(wcfg, appConnMem, 10)
  334. mempool.InitWAL()
  335. // 4. Ensure that the directory contains the WAL file
  336. m2, err := filepath.Glob(filepath.Join(rootDir, "*"))
  337. require.Nil(t, err, "successful globbing expected")
  338. require.Equal(t, 1, len(m2), "expecting the wal match in")
  339. // 5. Write some contents to the WAL
  340. mempool.CheckTx(types.Tx([]byte("foo")), nil)
  341. walFilepath := mempool.wal.Path
  342. sum1 := checksumFile(walFilepath, t)
  343. // 6. Sanity check to ensure that the written TX matches the expectation.
  344. require.Equal(t, sum1, checksumIt([]byte("foo\n")), "foo with a newline should be written")
  345. // 7. Invoke CloseWAL() and ensure it discards the
  346. // WAL thus any other write won't go through.
  347. mempool.CloseWAL()
  348. mempool.CheckTx(types.Tx([]byte("bar")), nil)
  349. sum2 := checksumFile(walFilepath, t)
  350. require.Equal(t, sum1, sum2, "expected no change to the WAL after invoking CloseWAL() since it was discarded")
  351. // 8. Sanity check to ensure that the WAL file still exists
  352. m3, err := filepath.Glob(filepath.Join(rootDir, "*"))
  353. require.Nil(t, err, "successful globbing expected")
  354. require.Equal(t, 1, len(m3), "expecting the wal match in")
  355. }
  356. // Size of the amino encoded TxMessage is the length of the
  357. // encoded byte array, plus 1 for the struct field, plus 4
  358. // for the amino prefix.
  359. func txMessageSize(tx types.Tx) int {
  360. return amino.ByteSliceSize(tx) + 1 + 4
  361. }
  362. func TestMempoolMaxMsgSize(t *testing.T) {
  363. app := kvstore.NewKVStoreApplication()
  364. cc := proxy.NewLocalClientCreator(app)
  365. mempl, cleanup := newMempoolWithApp(cc)
  366. defer cleanup()
  367. testCases := []struct {
  368. len int
  369. err bool
  370. }{
  371. // check small txs. no error
  372. {10, false},
  373. {1000, false},
  374. {1000000, false},
  375. // check around maxTxSize
  376. // changes from no error to error
  377. {maxTxSize - 2, false},
  378. {maxTxSize - 1, false},
  379. {maxTxSize, false},
  380. {maxTxSize + 1, true},
  381. {maxTxSize + 2, true},
  382. // check around maxMsgSize. all error
  383. {maxMsgSize - 1, true},
  384. {maxMsgSize, true},
  385. {maxMsgSize + 1, true},
  386. }
  387. for i, testCase := range testCases {
  388. caseString := fmt.Sprintf("case %d, len %d", i, testCase.len)
  389. tx := cmn.RandBytes(testCase.len)
  390. err := mempl.CheckTx(tx, nil)
  391. msg := &TxMessage{tx}
  392. encoded := cdc.MustMarshalBinaryBare(msg)
  393. require.Equal(t, len(encoded), txMessageSize(tx), caseString)
  394. if !testCase.err {
  395. require.True(t, len(encoded) <= maxMsgSize, caseString)
  396. require.NoError(t, err, caseString)
  397. } else {
  398. require.True(t, len(encoded) > maxMsgSize, caseString)
  399. require.Equal(t, err, ErrTxTooLarge, caseString)
  400. }
  401. }
  402. }
  403. func checksumIt(data []byte) string {
  404. h := sha256.New()
  405. h.Write(data)
  406. return fmt.Sprintf("%x", h.Sum(nil))
  407. }
  408. func checksumFile(p string, t *testing.T) string {
  409. data, err := ioutil.ReadFile(p)
  410. require.Nil(t, err, "expecting successful read of %q", p)
  411. return checksumIt(data)
  412. }