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.

563 lines
17 KiB

  1. package evidence_test
  2. import (
  3. "context"
  4. "encoding/hex"
  5. "math/rand"
  6. "sync"
  7. "testing"
  8. "time"
  9. "github.com/fortytw2/leaktest"
  10. "github.com/stretchr/testify/assert"
  11. "github.com/stretchr/testify/mock"
  12. "github.com/stretchr/testify/require"
  13. dbm "github.com/tendermint/tm-db"
  14. "github.com/tendermint/tendermint/crypto"
  15. "github.com/tendermint/tendermint/crypto/tmhash"
  16. "github.com/tendermint/tendermint/internal/evidence"
  17. "github.com/tendermint/tendermint/internal/evidence/mocks"
  18. "github.com/tendermint/tendermint/internal/p2p"
  19. "github.com/tendermint/tendermint/internal/p2p/p2ptest"
  20. sm "github.com/tendermint/tendermint/internal/state"
  21. "github.com/tendermint/tendermint/libs/log"
  22. tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
  23. "github.com/tendermint/tendermint/types"
  24. )
  25. var (
  26. numEvidence = 10
  27. rng = rand.New(rand.NewSource(time.Now().UnixNano()))
  28. )
  29. type reactorTestSuite struct {
  30. network *p2ptest.Network
  31. logger log.Logger
  32. reactors map[types.NodeID]*evidence.Reactor
  33. pools map[types.NodeID]*evidence.Pool
  34. evidenceChannels map[types.NodeID]*p2p.Channel
  35. peerUpdates map[types.NodeID]*p2p.PeerUpdates
  36. peerChans map[types.NodeID]chan p2p.PeerUpdate
  37. nodes []*p2ptest.Node
  38. numStateStores int
  39. }
  40. func setup(ctx context.Context, t *testing.T, stateStores []sm.Store, chBuf uint) *reactorTestSuite {
  41. t.Helper()
  42. pID := make([]byte, 16)
  43. _, err := rng.Read(pID)
  44. require.NoError(t, err)
  45. numStateStores := len(stateStores)
  46. rts := &reactorTestSuite{
  47. numStateStores: numStateStores,
  48. logger: log.TestingLogger().With("testCase", t.Name()),
  49. network: p2ptest.MakeNetwork(ctx, t, p2ptest.NetworkOptions{NumNodes: numStateStores}),
  50. reactors: make(map[types.NodeID]*evidence.Reactor, numStateStores),
  51. pools: make(map[types.NodeID]*evidence.Pool, numStateStores),
  52. peerUpdates: make(map[types.NodeID]*p2p.PeerUpdates, numStateStores),
  53. peerChans: make(map[types.NodeID]chan p2p.PeerUpdate, numStateStores),
  54. }
  55. chDesc := &p2p.ChannelDescriptor{ID: evidence.EvidenceChannel, MessageType: new(tmproto.EvidenceList)}
  56. rts.evidenceChannels = rts.network.MakeChannelsNoCleanup(ctx, t, chDesc)
  57. require.Len(t, rts.network.RandomNode().PeerManager.Peers(), 0)
  58. idx := 0
  59. evidenceTime := time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
  60. for nodeID := range rts.network.Nodes {
  61. logger := rts.logger.With("validator", idx)
  62. evidenceDB := dbm.NewMemDB()
  63. blockStore := &mocks.BlockStore{}
  64. state, _ := stateStores[idx].Load()
  65. blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(func(h int64) *types.BlockMeta {
  66. if h <= state.LastBlockHeight {
  67. return &types.BlockMeta{Header: types.Header{Time: evidenceTime}}
  68. }
  69. return nil
  70. })
  71. rts.pools[nodeID], err = evidence.NewPool(logger, evidenceDB, stateStores[idx], blockStore)
  72. require.NoError(t, err)
  73. rts.peerChans[nodeID] = make(chan p2p.PeerUpdate)
  74. rts.peerUpdates[nodeID] = p2p.NewPeerUpdates(rts.peerChans[nodeID], 1)
  75. rts.network.Nodes[nodeID].PeerManager.Register(ctx, rts.peerUpdates[nodeID])
  76. rts.nodes = append(rts.nodes, rts.network.Nodes[nodeID])
  77. rts.reactors[nodeID] = evidence.NewReactor(logger,
  78. rts.evidenceChannels[nodeID],
  79. rts.peerUpdates[nodeID],
  80. rts.pools[nodeID])
  81. require.NoError(t, rts.reactors[nodeID].Start(ctx))
  82. require.True(t, rts.reactors[nodeID].IsRunning())
  83. idx++
  84. }
  85. t.Cleanup(func() {
  86. for _, r := range rts.reactors {
  87. if r.IsRunning() {
  88. require.NoError(t, r.Stop())
  89. require.False(t, r.IsRunning())
  90. }
  91. }
  92. })
  93. t.Cleanup(leaktest.Check(t))
  94. return rts
  95. }
  96. func (rts *reactorTestSuite) start(ctx context.Context, t *testing.T) {
  97. rts.network.Start(ctx, t)
  98. require.Len(t,
  99. rts.network.RandomNode().PeerManager.Peers(),
  100. rts.numStateStores-1,
  101. "network does not have expected number of nodes")
  102. }
  103. func (rts *reactorTestSuite) waitForEvidence(t *testing.T, evList types.EvidenceList, ids ...types.NodeID) {
  104. t.Helper()
  105. fn := func(pool *evidence.Pool) {
  106. var (
  107. localEvList []types.Evidence
  108. size int64
  109. loops int
  110. )
  111. // wait till we have at least the amount of evidence
  112. // that we expect. if there's more local evidence then
  113. // it doesn't make sense to wait longer and a
  114. // different assertion should catch the resulting error
  115. for len(localEvList) < len(evList) {
  116. // each evidence should not be more than 500 bytes
  117. localEvList, size = pool.PendingEvidence(int64(len(evList) * 500))
  118. if loops == 100 {
  119. t.Log("current wait status:", "|",
  120. "local", len(localEvList), "|",
  121. "waitlist", len(evList), "|",
  122. "size", size)
  123. }
  124. loops++
  125. }
  126. // put the reaped evidence in a map so we can quickly check we got everything
  127. evMap := make(map[string]types.Evidence)
  128. for _, e := range localEvList {
  129. evMap[string(e.Hash())] = e
  130. }
  131. for i, expectedEv := range evList {
  132. gotEv := evMap[string(expectedEv.Hash())]
  133. require.Equalf(
  134. t,
  135. expectedEv,
  136. gotEv,
  137. "evidence for pool %d in pool does not match; got: %v, expected: %v", i, gotEv, expectedEv,
  138. )
  139. }
  140. }
  141. if len(ids) == 1 {
  142. // special case waiting once, just to avoid the extra
  143. // goroutine, in the case that this hits a timeout,
  144. // the stack will be clearer.
  145. fn(rts.pools[ids[0]])
  146. return
  147. }
  148. wg := sync.WaitGroup{}
  149. for id := range rts.pools {
  150. if len(ids) > 0 && !p2ptest.NodeInSlice(id, ids) {
  151. // if an ID list is specified, then we only
  152. // want to wait for those pools that are
  153. // specified in the list, otherwise, wait for
  154. // all pools.
  155. continue
  156. }
  157. wg.Add(1)
  158. go func(id types.NodeID) { defer wg.Done(); fn(rts.pools[id]) }(id)
  159. }
  160. wg.Wait()
  161. }
  162. func createEvidenceList(
  163. ctx context.Context,
  164. t *testing.T,
  165. pool *evidence.Pool,
  166. val types.PrivValidator,
  167. numEvidence int,
  168. ) types.EvidenceList {
  169. t.Helper()
  170. evList := make([]types.Evidence, numEvidence)
  171. for i := 0; i < numEvidence; i++ {
  172. ev, err := types.NewMockDuplicateVoteEvidenceWithValidator(
  173. ctx,
  174. int64(i+1),
  175. time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC),
  176. val,
  177. evidenceChainID,
  178. )
  179. require.NoError(t, err)
  180. require.NoError(t, pool.AddEvidence(ev),
  181. "adding evidence it#%d of %d to pool with height %d",
  182. i, numEvidence, pool.State().LastBlockHeight)
  183. evList[i] = ev
  184. }
  185. return evList
  186. }
  187. func TestReactorMultiDisconnect(t *testing.T) {
  188. ctx, cancel := context.WithCancel(context.Background())
  189. defer cancel()
  190. val := types.NewMockPV()
  191. height := int64(numEvidence) + 10
  192. stateDB1 := initializeValidatorState(ctx, t, val, height)
  193. stateDB2 := initializeValidatorState(ctx, t, val, height)
  194. rts := setup(ctx, t, []sm.Store{stateDB1, stateDB2}, 20)
  195. primary := rts.nodes[0]
  196. secondary := rts.nodes[1]
  197. _ = createEvidenceList(ctx, t, rts.pools[primary.NodeID], val, numEvidence)
  198. require.Equal(t, primary.PeerManager.Status(secondary.NodeID), p2p.PeerStatusDown)
  199. rts.start(ctx, t)
  200. require.Equal(t, primary.PeerManager.Status(secondary.NodeID), p2p.PeerStatusUp)
  201. // Ensure "disconnecting" the secondary peer from the primary more than once
  202. // is handled gracefully.
  203. primary.PeerManager.Disconnected(ctx, secondary.NodeID)
  204. require.Equal(t, primary.PeerManager.Status(secondary.NodeID), p2p.PeerStatusDown)
  205. _, err := primary.PeerManager.TryEvictNext()
  206. require.NoError(t, err)
  207. primary.PeerManager.Disconnected(ctx, secondary.NodeID)
  208. require.Equal(t, primary.PeerManager.Status(secondary.NodeID), p2p.PeerStatusDown)
  209. require.Equal(t, secondary.PeerManager.Status(primary.NodeID), p2p.PeerStatusUp)
  210. }
  211. // TestReactorBroadcastEvidence creates an environment of multiple peers that
  212. // are all at the same height. One peer, designated as a primary, gossips all
  213. // evidence to the remaining peers.
  214. func TestReactorBroadcastEvidence(t *testing.T) {
  215. numPeers := 7
  216. ctx, cancel := context.WithCancel(context.Background())
  217. defer cancel()
  218. // create a stateDB for all test suites (nodes)
  219. stateDBs := make([]sm.Store, numPeers)
  220. val := types.NewMockPV()
  221. // We need all validators saved for heights at least as high as we have
  222. // evidence for.
  223. height := int64(numEvidence) + 10
  224. for i := 0; i < numPeers; i++ {
  225. stateDBs[i] = initializeValidatorState(ctx, t, val, height)
  226. }
  227. rts := setup(ctx, t, stateDBs, 0)
  228. rts.start(ctx, t)
  229. // Create a series of fixtures where each suite contains a reactor and
  230. // evidence pool. In addition, we mark a primary suite and the rest are
  231. // secondaries where each secondary is added as a peer via a PeerUpdate to the
  232. // primary. As a result, the primary will gossip all evidence to each secondary.
  233. primary := rts.network.RandomNode()
  234. secondaries := make([]*p2ptest.Node, 0, len(rts.network.NodeIDs())-1)
  235. secondaryIDs := make([]types.NodeID, 0, cap(secondaries))
  236. for id := range rts.network.Nodes {
  237. if id == primary.NodeID {
  238. continue
  239. }
  240. secondaries = append(secondaries, rts.network.Nodes[id])
  241. secondaryIDs = append(secondaryIDs, id)
  242. }
  243. evList := createEvidenceList(ctx, t, rts.pools[primary.NodeID], val, numEvidence)
  244. // Add each secondary suite (node) as a peer to the primary suite (node). This
  245. // will cause the primary to gossip all evidence to the secondaries.
  246. for _, suite := range secondaries {
  247. rts.peerChans[primary.NodeID] <- p2p.PeerUpdate{
  248. Status: p2p.PeerStatusUp,
  249. NodeID: suite.NodeID,
  250. }
  251. }
  252. // Wait till all secondary suites (reactor) received all evidence from the
  253. // primary suite (node).
  254. rts.waitForEvidence(t, evList, secondaryIDs...)
  255. for _, pool := range rts.pools {
  256. require.Equal(t, numEvidence, int(pool.Size()))
  257. }
  258. }
  259. // TestReactorSelectiveBroadcast tests a context where we have two reactors
  260. // connected to one another but are at different heights. Reactor 1 which is
  261. // ahead receives a list of evidence.
  262. func TestReactorBroadcastEvidence_Lagging(t *testing.T) {
  263. val := types.NewMockPV()
  264. height1 := int64(numEvidence) + 10
  265. height2 := int64(numEvidence) / 2
  266. ctx, cancel := context.WithCancel(context.Background())
  267. defer cancel()
  268. // stateDB1 is ahead of stateDB2, where stateDB1 has all heights (1-20) and
  269. // stateDB2 only has heights 1-5.
  270. stateDB1 := initializeValidatorState(ctx, t, val, height1)
  271. stateDB2 := initializeValidatorState(ctx, t, val, height2)
  272. rts := setup(ctx, t, []sm.Store{stateDB1, stateDB2}, 100)
  273. rts.start(ctx, t)
  274. primary := rts.nodes[0]
  275. secondary := rts.nodes[1]
  276. // Send a list of valid evidence to the first reactor's, the one that is ahead,
  277. // evidence pool.
  278. evList := createEvidenceList(ctx, t, rts.pools[primary.NodeID], val, numEvidence)
  279. // Add each secondary suite (node) as a peer to the primary suite (node). This
  280. // will cause the primary to gossip all evidence to the secondaries.
  281. rts.peerChans[primary.NodeID] <- p2p.PeerUpdate{
  282. Status: p2p.PeerStatusUp,
  283. NodeID: secondary.NodeID,
  284. }
  285. // only ones less than the peers height should make it through
  286. rts.waitForEvidence(t, evList[:height2], secondary.NodeID)
  287. require.Equal(t, numEvidence, int(rts.pools[primary.NodeID].Size()))
  288. require.Equal(t, int(height2), int(rts.pools[secondary.NodeID].Size()))
  289. }
  290. func TestReactorBroadcastEvidence_Pending(t *testing.T) {
  291. val := types.NewMockPV()
  292. height := int64(10)
  293. ctx, cancel := context.WithCancel(context.Background())
  294. defer cancel()
  295. stateDB1 := initializeValidatorState(ctx, t, val, height)
  296. stateDB2 := initializeValidatorState(ctx, t, val, height)
  297. rts := setup(ctx, t, []sm.Store{stateDB1, stateDB2}, 100)
  298. primary := rts.nodes[0]
  299. secondary := rts.nodes[1]
  300. evList := createEvidenceList(ctx, t, rts.pools[primary.NodeID], val, numEvidence)
  301. // Manually add half the evidence to the secondary which will mark them as
  302. // pending.
  303. for i := 0; i < numEvidence/2; i++ {
  304. require.NoError(t, rts.pools[secondary.NodeID].AddEvidence(evList[i]))
  305. }
  306. // the secondary should have half the evidence as pending
  307. require.Equal(t, numEvidence/2, int(rts.pools[secondary.NodeID].Size()))
  308. rts.start(ctx, t)
  309. // The secondary reactor should have received all the evidence ignoring the
  310. // already pending evidence.
  311. rts.waitForEvidence(t, evList, secondary.NodeID)
  312. // check to make sure that all of the evidence has
  313. // propogated
  314. require.Len(t, rts.pools, 2)
  315. assert.EqualValues(t, numEvidence, rts.pools[primary.NodeID].Size(),
  316. "primary node should have all the evidence")
  317. assert.EqualValues(t, numEvidence, rts.pools[secondary.NodeID].Size(),
  318. "secondary nodes should have caught up")
  319. }
  320. func TestReactorBroadcastEvidence_Committed(t *testing.T) {
  321. val := types.NewMockPV()
  322. height := int64(10)
  323. ctx, cancel := context.WithCancel(context.Background())
  324. defer cancel()
  325. stateDB1 := initializeValidatorState(ctx, t, val, height)
  326. stateDB2 := initializeValidatorState(ctx, t, val, height)
  327. rts := setup(ctx, t, []sm.Store{stateDB1, stateDB2}, 0)
  328. primary := rts.nodes[0]
  329. secondary := rts.nodes[1]
  330. // add all evidence to the primary reactor
  331. evList := createEvidenceList(ctx, t, rts.pools[primary.NodeID], val, numEvidence)
  332. // Manually add half the evidence to the secondary which will mark them as
  333. // pending.
  334. for i := 0; i < numEvidence/2; i++ {
  335. require.NoError(t, rts.pools[secondary.NodeID].AddEvidence(evList[i]))
  336. }
  337. // the secondary should have half the evidence as pending
  338. require.Equal(t, numEvidence/2, int(rts.pools[secondary.NodeID].Size()))
  339. state, err := stateDB2.Load()
  340. require.NoError(t, err)
  341. // update the secondary's pool such that all pending evidence is committed
  342. state.LastBlockHeight++
  343. rts.pools[secondary.NodeID].Update(state, evList[:numEvidence/2])
  344. // the secondary should have half the evidence as committed
  345. require.Equal(t, 0, int(rts.pools[secondary.NodeID].Size()))
  346. // start the network and ensure it's configured
  347. rts.start(ctx, t)
  348. // The secondary reactor should have received all the evidence ignoring the
  349. // already committed evidence.
  350. rts.waitForEvidence(t, evList[numEvidence/2:], secondary.NodeID)
  351. require.Len(t, rts.pools, 2)
  352. assert.EqualValues(t, numEvidence, rts.pools[primary.NodeID].Size(),
  353. "primary node should have all the evidence")
  354. assert.EqualValues(t, numEvidence/2, rts.pools[secondary.NodeID].Size(),
  355. "secondary nodes should have caught up")
  356. }
  357. func TestReactorBroadcastEvidence_FullyConnected(t *testing.T) {
  358. numPeers := 7
  359. // create a stateDB for all test suites (nodes)
  360. stateDBs := make([]sm.Store, numPeers)
  361. val := types.NewMockPV()
  362. ctx, cancel := context.WithCancel(context.Background())
  363. defer cancel()
  364. // We need all validators saved for heights at least as high as we have
  365. // evidence for.
  366. height := int64(numEvidence) + 10
  367. for i := 0; i < numPeers; i++ {
  368. stateDBs[i] = initializeValidatorState(ctx, t, val, height)
  369. }
  370. rts := setup(ctx, t, stateDBs, 0)
  371. rts.start(ctx, t)
  372. evList := createEvidenceList(ctx, t, rts.pools[rts.network.RandomNode().NodeID], val, numEvidence)
  373. // every suite (reactor) connects to every other suite (reactor)
  374. for outerID, outerChan := range rts.peerChans {
  375. for innerID := range rts.peerChans {
  376. if outerID != innerID {
  377. outerChan <- p2p.PeerUpdate{
  378. Status: p2p.PeerStatusUp,
  379. NodeID: innerID,
  380. }
  381. }
  382. }
  383. }
  384. // wait till all suites (reactors) received all evidence from other suites (reactors)
  385. rts.waitForEvidence(t, evList)
  386. for _, pool := range rts.pools {
  387. require.Equal(t, numEvidence, int(pool.Size()))
  388. // commit state so we do not continue to repeat gossiping the same evidence
  389. state := pool.State()
  390. state.LastBlockHeight++
  391. pool.Update(state, evList)
  392. }
  393. }
  394. func TestEvidenceListSerialization(t *testing.T) {
  395. exampleVote := func(msgType byte) *types.Vote {
  396. var stamp, err = time.Parse(types.TimeFormat, "2017-12-25T03:00:01.234Z")
  397. require.NoError(t, err)
  398. return &types.Vote{
  399. Type: tmproto.SignedMsgType(msgType),
  400. Height: 3,
  401. Round: 2,
  402. Timestamp: stamp,
  403. BlockID: types.BlockID{
  404. Hash: tmhash.Sum([]byte("blockID_hash")),
  405. PartSetHeader: types.PartSetHeader{
  406. Total: 1000000,
  407. Hash: tmhash.Sum([]byte("blockID_part_set_header_hash")),
  408. },
  409. },
  410. ValidatorAddress: crypto.AddressHash([]byte("validator_address")),
  411. ValidatorIndex: 56789,
  412. }
  413. }
  414. val := &types.Validator{
  415. Address: crypto.AddressHash([]byte("validator_address")),
  416. VotingPower: 10,
  417. }
  418. valSet := types.NewValidatorSet([]*types.Validator{val})
  419. dupl, err := types.NewDuplicateVoteEvidence(
  420. exampleVote(1),
  421. exampleVote(2),
  422. defaultEvidenceTime,
  423. valSet,
  424. )
  425. require.NoError(t, err)
  426. testCases := map[string]struct {
  427. evidenceList []types.Evidence
  428. expBytes string
  429. }{
  430. "DuplicateVoteEvidence": {
  431. []types.Evidence{dupl},
  432. "0a85020a82020a79080210031802224a0a208b01023386c371778ecb6368573e539afc3cc860ec3a2f614e54fe5652f4fc80122608c0843d122072db3d959635dff1bb567bedaa70573392c5159666a3f8caf11e413aac52207a2a0b08b1d381d20510809dca6f32146af1f4111082efb388211bc72c55bcd61e9ac3d538d5bb031279080110031802224a0a208b01023386c371778ecb6368573e539afc3cc860ec3a2f614e54fe5652f4fc80122608c0843d122072db3d959635dff1bb567bedaa70573392c5159666a3f8caf11e413aac52207a2a0b08b1d381d20510809dca6f32146af1f4111082efb388211bc72c55bcd61e9ac3d538d5bb03180a200a2a060880dbaae105",
  433. },
  434. }
  435. for name, tc := range testCases {
  436. tc := tc
  437. t.Run(name, func(t *testing.T) {
  438. protoEv := make([]tmproto.Evidence, len(tc.evidenceList))
  439. for i := 0; i < len(tc.evidenceList); i++ {
  440. ev, err := types.EvidenceToProto(tc.evidenceList[i])
  441. require.NoError(t, err)
  442. protoEv[i] = *ev
  443. }
  444. epl := tmproto.EvidenceList{
  445. Evidence: protoEv,
  446. }
  447. bz, err := epl.Marshal()
  448. require.NoError(t, err)
  449. require.Equal(t, tc.expBytes, hex.EncodeToString(bz))
  450. })
  451. }
  452. }