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.

365 lines
11 KiB

7 years ago
7 years ago
  1. package evidence_test
  2. import (
  3. "encoding/hex"
  4. "fmt"
  5. "sync"
  6. "testing"
  7. "time"
  8. "github.com/go-kit/kit/log/term"
  9. "github.com/stretchr/testify/assert"
  10. "github.com/stretchr/testify/mock"
  11. "github.com/stretchr/testify/require"
  12. dbm "github.com/tendermint/tm-db"
  13. cfg "github.com/tendermint/tendermint/config"
  14. "github.com/tendermint/tendermint/crypto"
  15. "github.com/tendermint/tendermint/crypto/tmhash"
  16. "github.com/tendermint/tendermint/evidence"
  17. "github.com/tendermint/tendermint/evidence/mocks"
  18. "github.com/tendermint/tendermint/libs/log"
  19. "github.com/tendermint/tendermint/p2p"
  20. ep "github.com/tendermint/tendermint/proto/tendermint/evidence"
  21. tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
  22. sm "github.com/tendermint/tendermint/state"
  23. "github.com/tendermint/tendermint/types"
  24. )
  25. var (
  26. numEvidence = 10
  27. timeout = 120 * time.Second // ridiculously high because CircleCI is slow
  28. )
  29. // We have N evidence reactors connected to one another. The first reactor
  30. // receives a number of evidence at varying heights. We test that all
  31. // other reactors receive the evidence and add it to their own respective
  32. // evidence pools.
  33. func TestReactorBroadcastEvidence(t *testing.T) {
  34. config := cfg.TestConfig()
  35. N := 7
  36. // create statedb for everyone
  37. stateDBs := make([]sm.Store, N)
  38. val := types.NewMockPV()
  39. // we need validators saved for heights at least as high as we have evidence for
  40. height := int64(numEvidence) + 10
  41. for i := 0; i < N; i++ {
  42. stateDBs[i] = initializeValidatorState(val, height)
  43. }
  44. // make reactors from statedb
  45. reactors, pools := makeAndConnectReactorsAndPools(config, stateDBs)
  46. // set the peer height on each reactor
  47. for _, r := range reactors {
  48. for _, peer := range r.Switch.Peers().List() {
  49. ps := peerState{height}
  50. peer.Set(types.PeerStateKey, ps)
  51. }
  52. }
  53. // send a bunch of valid evidence to the first reactor's evpool
  54. // and wait for them all to be received in the others
  55. evList := sendEvidence(t, pools[0], val, numEvidence)
  56. waitForEvidence(t, evList, pools)
  57. }
  58. // We have two evidence reactors connected to one another but are at different heights.
  59. // Reactor 1 which is ahead receives a number of evidence. It should only send the evidence
  60. // that is below the height of the peer to that peer.
  61. func TestReactorSelectiveBroadcast(t *testing.T) {
  62. config := cfg.TestConfig()
  63. val := types.NewMockPV()
  64. height1 := int64(numEvidence) + 10
  65. height2 := int64(numEvidence) / 2
  66. // DB1 is ahead of DB2
  67. stateDB1 := initializeValidatorState(val, height1)
  68. stateDB2 := initializeValidatorState(val, height2)
  69. // make reactors from statedb
  70. reactors, pools := makeAndConnectReactorsAndPools(config, []sm.Store{stateDB1, stateDB2})
  71. // set the peer height on each reactor
  72. for _, r := range reactors {
  73. for _, peer := range r.Switch.Peers().List() {
  74. ps := peerState{height1}
  75. peer.Set(types.PeerStateKey, ps)
  76. }
  77. }
  78. // update the first reactor peer's height to be very small
  79. peer := reactors[0].Switch.Peers().List()[0]
  80. ps := peerState{height2}
  81. peer.Set(types.PeerStateKey, ps)
  82. // send a bunch of valid evidence to the first reactor's evpool
  83. evList := sendEvidence(t, pools[0], val, numEvidence)
  84. // only ones less than the peers height should make it through
  85. waitForEvidence(t, evList[:numEvidence/2-1], []*evidence.Pool{pools[1]})
  86. // peers should still be connected
  87. peers := reactors[1].Switch.Peers().List()
  88. assert.Equal(t, 1, len(peers))
  89. }
  90. // This tests aims to ensure that reactors don't send evidence that they have committed or that ar
  91. // not ready for the peer through three scenarios.
  92. // First, committed evidence to a newly connected peer
  93. // Second, evidence to a peer that is behind
  94. // Third, evidence that was pending and became committed just before the peer caught up
  95. func TestReactorsGossipNoCommittedEvidence(t *testing.T) {
  96. config := cfg.TestConfig()
  97. val := types.NewMockPV()
  98. var height int64 = 10
  99. // DB1 is ahead of DB2
  100. stateDB1 := initializeValidatorState(val, height)
  101. stateDB2 := initializeValidatorState(val, height-2)
  102. // make reactors from statedb
  103. reactors, pools := makeAndConnectReactorsAndPools(config, []sm.Store{stateDB1, stateDB2})
  104. evList := sendEvidence(t, pools[0], val, 2)
  105. abciEvs := pools[0].ABCIEvidence(height, evList)
  106. require.EqualValues(t, 2, len(abciEvs))
  107. require.EqualValues(t, uint32(0), pools[0].Size())
  108. time.Sleep(100 * time.Millisecond)
  109. peer := reactors[0].Switch.Peers().List()[0]
  110. ps := peerState{height - 2}
  111. peer.Set(types.PeerStateKey, ps)
  112. peer = reactors[1].Switch.Peers().List()[0]
  113. ps = peerState{height}
  114. peer.Set(types.PeerStateKey, ps)
  115. // wait to see that no evidence comes through
  116. time.Sleep(300 * time.Millisecond)
  117. // the second pool should not have received any evidence because it has already been committed
  118. assert.Equal(t, uint32(0), pools[1].Size(), "second reactor should not have received evidence")
  119. // the first reactor receives three more evidence
  120. evList = make([]types.Evidence, 3)
  121. for i := 0; i < 3; i++ {
  122. ev := types.NewMockDuplicateVoteEvidenceWithValidator(height-3+int64(i),
  123. time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC), val, evidenceChainID)
  124. err := pools[0].AddEvidence(ev)
  125. require.NoError(t, err)
  126. evList[i] = ev
  127. }
  128. // wait to see that only one evidence is sent
  129. time.Sleep(300 * time.Millisecond)
  130. // the second pool should only have received the first evidence because it is behind
  131. peerEv, _ := pools[1].PendingEvidence(1000)
  132. assert.EqualValues(t, []types.Evidence{evList[0]}, peerEv)
  133. // the last evidence is committed and the second reactor catches up in state to the first
  134. // reactor. We therefore expect that the second reactor only receives one more evidence, the
  135. // one that is still pending and not the evidence that has already been committed.
  136. _ = pools[0].ABCIEvidence(height, []types.Evidence{evList[2]})
  137. // the first reactor should have the two remaining pending evidence
  138. require.EqualValues(t, uint32(2), pools[0].Size())
  139. // now update the state of the second reactor
  140. pools[1].Update(sm.State{LastBlockHeight: height})
  141. peer = reactors[0].Switch.Peers().List()[0]
  142. ps = peerState{height}
  143. peer.Set(types.PeerStateKey, ps)
  144. // wait to see that only two evidence is sent
  145. time.Sleep(300 * time.Millisecond)
  146. peerEv, _ = pools[1].PendingEvidence(1000)
  147. assert.EqualValues(t, evList[0:1], peerEv)
  148. }
  149. // evidenceLogger is a TestingLogger which uses a different
  150. // color for each validator ("validator" key must exist).
  151. func evidenceLogger() log.Logger {
  152. return log.TestingLoggerWithColorFn(func(keyvals ...interface{}) term.FgBgColor {
  153. for i := 0; i < len(keyvals)-1; i += 2 {
  154. if keyvals[i] == "validator" {
  155. return term.FgBgColor{Fg: term.Color(uint8(keyvals[i+1].(int) + 1))}
  156. }
  157. }
  158. return term.FgBgColor{}
  159. })
  160. }
  161. // connect N evidence reactors through N switches
  162. func makeAndConnectReactorsAndPools(config *cfg.Config, stateStores []sm.Store) ([]*evidence.Reactor,
  163. []*evidence.Pool) {
  164. N := len(stateStores)
  165. reactors := make([]*evidence.Reactor, N)
  166. pools := make([]*evidence.Pool, N)
  167. logger := evidenceLogger()
  168. evidenceTime := time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
  169. for i := 0; i < N; i++ {
  170. evidenceDB := dbm.NewMemDB()
  171. blockStore := &mocks.BlockStore{}
  172. blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(
  173. &types.BlockMeta{Header: types.Header{Time: evidenceTime}},
  174. )
  175. pool, err := evidence.NewPool(evidenceDB, stateStores[i], blockStore)
  176. if err != nil {
  177. panic(err)
  178. }
  179. pools[i] = pool
  180. reactors[i] = evidence.NewReactor(pool)
  181. reactors[i].SetLogger(logger.With("validator", i))
  182. }
  183. p2p.MakeConnectedSwitches(config.P2P, N, func(i int, s *p2p.Switch) *p2p.Switch {
  184. s.AddReactor("EVIDENCE", reactors[i])
  185. return s
  186. }, p2p.Connect2Switches)
  187. return reactors, pools
  188. }
  189. // wait for all evidence on all reactors
  190. func waitForEvidence(t *testing.T, evs types.EvidenceList, pools []*evidence.Pool) {
  191. // wait for the evidence in all evpools
  192. wg := new(sync.WaitGroup)
  193. for i := 0; i < len(pools); i++ {
  194. wg.Add(1)
  195. go _waitForEvidence(t, wg, evs, i, pools)
  196. }
  197. done := make(chan struct{})
  198. go func() {
  199. wg.Wait()
  200. close(done)
  201. }()
  202. timer := time.After(timeout)
  203. select {
  204. case <-timer:
  205. t.Fatal("Timed out waiting for evidence")
  206. case <-done:
  207. }
  208. }
  209. // wait for all evidence on a single evpool
  210. func _waitForEvidence(
  211. t *testing.T,
  212. wg *sync.WaitGroup,
  213. evs types.EvidenceList,
  214. poolIdx int,
  215. pools []*evidence.Pool,
  216. ) {
  217. evpool := pools[poolIdx]
  218. var evList []types.Evidence
  219. currentPoolSize := 0
  220. for currentPoolSize != len(evs) {
  221. evList, _ = evpool.PendingEvidence(int64(len(evs) * 500)) // each evidence should not be more than 500 bytes
  222. currentPoolSize = len(evList)
  223. time.Sleep(time.Millisecond * 100)
  224. }
  225. // put the reaped evidence in a map so we can quickly check we got everything
  226. evMap := make(map[string]types.Evidence)
  227. for _, e := range evList {
  228. evMap[string(e.Hash())] = e
  229. }
  230. for i, expectedEv := range evs {
  231. gotEv := evMap[string(expectedEv.Hash())]
  232. assert.Equal(t, expectedEv, gotEv,
  233. fmt.Sprintf("evidence at index %d on pool %d don't match: %v vs %v",
  234. i, poolIdx, expectedEv, gotEv))
  235. }
  236. wg.Done()
  237. }
  238. func sendEvidence(t *testing.T, evpool *evidence.Pool, val types.PrivValidator, n int) types.EvidenceList {
  239. evList := make([]types.Evidence, n)
  240. for i := 0; i < n; i++ {
  241. ev := types.NewMockDuplicateVoteEvidenceWithValidator(int64(i+1),
  242. time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC), val, evidenceChainID)
  243. err := evpool.AddEvidence(ev)
  244. require.NoError(t, err)
  245. evList[i] = ev
  246. }
  247. return evList
  248. }
  249. type peerState struct {
  250. height int64
  251. }
  252. func (ps peerState) GetHeight() int64 {
  253. return ps.height
  254. }
  255. func exampleVote(t byte) *types.Vote {
  256. var stamp, err = time.Parse(types.TimeFormat, "2017-12-25T03:00:01.234Z")
  257. if err != nil {
  258. panic(err)
  259. }
  260. return &types.Vote{
  261. Type: tmproto.SignedMsgType(t),
  262. Height: 3,
  263. Round: 2,
  264. Timestamp: stamp,
  265. BlockID: types.BlockID{
  266. Hash: tmhash.Sum([]byte("blockID_hash")),
  267. PartSetHeader: types.PartSetHeader{
  268. Total: 1000000,
  269. Hash: tmhash.Sum([]byte("blockID_part_set_header_hash")),
  270. },
  271. },
  272. ValidatorAddress: crypto.AddressHash([]byte("validator_address")),
  273. ValidatorIndex: 56789,
  274. }
  275. }
  276. // nolint:lll //ignore line length for tests
  277. func TestEvidenceVectors(t *testing.T) {
  278. dupl := types.NewDuplicateVoteEvidence(exampleVote(1), exampleVote(2))
  279. testCases := []struct {
  280. testName string
  281. evidenceList []types.Evidence
  282. expBytes string
  283. }{
  284. {"DuplicateVoteEvidence", []types.Evidence{dupl}, "0af9010af6010a79080210031802224a0a208b01023386c371778ecb6368573e539afc3cc860ec3a2f614e54fe5652f4fc80122608c0843d122072db3d959635dff1bb567bedaa70573392c5159666a3f8caf11e413aac52207a2a0b08b1d381d20510809dca6f32146af1f4111082efb388211bc72c55bcd61e9ac3d538d5bb031279080110031802224a0a208b01023386c371778ecb6368573e539afc3cc860ec3a2f614e54fe5652f4fc80122608c0843d122072db3d959635dff1bb567bedaa70573392c5159666a3f8caf11e413aac52207a2a0b08b1d381d20510809dca6f32146af1f4111082efb388211bc72c55bcd61e9ac3d538d5bb03"},
  285. }
  286. for _, tc := range testCases {
  287. tc := tc
  288. evi := make([]*tmproto.Evidence, len(tc.evidenceList))
  289. for i := 0; i < len(tc.evidenceList); i++ {
  290. ev, err := types.EvidenceToProto(tc.evidenceList[i])
  291. require.NoError(t, err, tc.testName)
  292. evi[i] = ev
  293. }
  294. epl := ep.List{
  295. Evidence: evi,
  296. }
  297. bz, err := epl.Marshal()
  298. require.NoError(t, err, tc.testName)
  299. require.Equal(t, tc.expBytes, hex.EncodeToString(bz), tc.testName)
  300. }
  301. }