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.

379 lines
12 KiB

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