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.

456 lines
15 KiB

7 years ago
  1. package evidence_test
  2. import (
  3. "os"
  4. "testing"
  5. "time"
  6. "github.com/stretchr/testify/assert"
  7. "github.com/stretchr/testify/mock"
  8. "github.com/stretchr/testify/require"
  9. dbm "github.com/tendermint/tm-db"
  10. abci "github.com/tendermint/tendermint/abci/types"
  11. "github.com/tendermint/tendermint/evidence"
  12. "github.com/tendermint/tendermint/evidence/mocks"
  13. "github.com/tendermint/tendermint/libs/log"
  14. tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
  15. tmversion "github.com/tendermint/tendermint/proto/tendermint/version"
  16. sm "github.com/tendermint/tendermint/state"
  17. smmocks "github.com/tendermint/tendermint/state/mocks"
  18. "github.com/tendermint/tendermint/store"
  19. "github.com/tendermint/tendermint/types"
  20. "github.com/tendermint/tendermint/version"
  21. )
  22. func TestMain(m *testing.M) {
  23. code := m.Run()
  24. os.Exit(code)
  25. }
  26. const evidenceChainID = "test_chain"
  27. var (
  28. defaultEvidenceTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
  29. defaultEvidenceMaxBytes int64 = 1000
  30. )
  31. func TestEvidencePoolBasic(t *testing.T) {
  32. var (
  33. height = int64(1)
  34. stateStore = &smmocks.Store{}
  35. evidenceDB = dbm.NewMemDB()
  36. blockStore = &mocks.BlockStore{}
  37. )
  38. valSet, privVals := types.RandValidatorSet(3, 10)
  39. blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(
  40. &types.BlockMeta{Header: types.Header{Time: defaultEvidenceTime}},
  41. )
  42. stateStore.On("LoadValidators", mock.AnythingOfType("int64")).Return(valSet, nil)
  43. stateStore.On("Load").Return(createState(height+1, valSet), nil)
  44. pool, err := evidence.NewPool(evidenceDB, stateStore, blockStore)
  45. require.NoError(t, err)
  46. pool.SetLogger(log.TestingLogger())
  47. // evidence not seen yet:
  48. evs, size := pool.PendingEvidence(defaultEvidenceMaxBytes)
  49. assert.Equal(t, 0, len(evs))
  50. assert.Zero(t, size)
  51. ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime, privVals[0], evidenceChainID)
  52. // good evidence
  53. evAdded := make(chan struct{})
  54. go func() {
  55. <-pool.EvidenceWaitChan()
  56. close(evAdded)
  57. }()
  58. // evidence seen but not yet committed:
  59. assert.NoError(t, pool.AddEvidence(ev))
  60. select {
  61. case <-evAdded:
  62. case <-time.After(5 * time.Second):
  63. t.Fatal("evidence was not added to list after 5s")
  64. }
  65. next := pool.EvidenceFront()
  66. assert.Equal(t, ev, next.Value.(types.Evidence))
  67. evs, size = pool.PendingEvidence(defaultEvidenceMaxBytes)
  68. assert.Equal(t, 1, len(evs))
  69. assert.Equal(t, int64(357), size) // check that the size of the single evidence in bytes is correct
  70. // shouldn't be able to add evidence twice
  71. assert.Error(t, pool.AddEvidence(ev))
  72. evs, _ = pool.PendingEvidence(defaultEvidenceMaxBytes)
  73. assert.Equal(t, 1, len(evs))
  74. }
  75. // Tests inbound evidence for the right time and height
  76. func TestAddExpiredEvidence(t *testing.T) {
  77. var (
  78. val = types.NewMockPV()
  79. height = int64(30)
  80. stateStore = initializeValidatorState(val, height)
  81. evidenceDB = dbm.NewMemDB()
  82. blockStore = &mocks.BlockStore{}
  83. expiredEvidenceTime = time.Date(2018, 1, 1, 0, 0, 0, 0, time.UTC)
  84. expiredHeight = int64(2)
  85. )
  86. blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(func(h int64) *types.BlockMeta {
  87. if h == height || h == expiredHeight {
  88. return &types.BlockMeta{Header: types.Header{Time: defaultEvidenceTime.Add(time.Duration(height) * time.Minute)}}
  89. }
  90. return &types.BlockMeta{Header: types.Header{Time: expiredEvidenceTime}}
  91. })
  92. pool, err := evidence.NewPool(evidenceDB, stateStore, blockStore)
  93. require.NoError(t, err)
  94. testCases := []struct {
  95. evHeight int64
  96. evTime time.Time
  97. expErr bool
  98. evDescription string
  99. }{
  100. {height, defaultEvidenceTime, false, "valid evidence"},
  101. {expiredHeight, defaultEvidenceTime, false, "valid evidence (despite old height)"},
  102. {height - 1, expiredEvidenceTime, false, "valid evidence (despite old time)"},
  103. {expiredHeight - 1, expiredEvidenceTime, true,
  104. "evidence from height 1 (created at: 2019-01-01 00:00:00 +0000 UTC) is too old"},
  105. }
  106. for _, tc := range testCases {
  107. tc := tc
  108. t.Run(tc.evDescription, func(t *testing.T) {
  109. ev := types.NewMockDuplicateVoteEvidenceWithValidator(tc.evHeight, tc.evTime, val, evidenceChainID)
  110. err := pool.AddEvidence(ev)
  111. if tc.expErr {
  112. assert.Error(t, err)
  113. } else {
  114. assert.NoError(t, err)
  115. }
  116. })
  117. }
  118. }
  119. func TestAddEvidenceFromConsensus(t *testing.T) {
  120. var height int64 = 10
  121. pool, val := defaultTestPool(height)
  122. ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime, val, evidenceChainID)
  123. err := pool.AddEvidenceFromConsensus(ev, defaultEvidenceTime,
  124. types.NewValidatorSet([]*types.Validator{val.ExtractIntoValidator(2)}))
  125. assert.NoError(t, err)
  126. next := pool.EvidenceFront()
  127. assert.Equal(t, ev, next.Value.(types.Evidence))
  128. // shouldn't be able to submit the same evidence twice
  129. err = pool.AddEvidenceFromConsensus(ev, defaultEvidenceTime.Add(-1*time.Second),
  130. types.NewValidatorSet([]*types.Validator{val.ExtractIntoValidator(3)}))
  131. if assert.Error(t, err) {
  132. assert.Equal(t, "evidence already verified and added", err.Error())
  133. }
  134. }
  135. func TestEvidencePoolUpdate(t *testing.T) {
  136. height := int64(21)
  137. pool, val := defaultTestPool(height)
  138. state := pool.State()
  139. // create new block (no need to save it to blockStore)
  140. prunedEv := types.NewMockDuplicateVoteEvidenceWithValidator(1, defaultEvidenceTime,
  141. val, evidenceChainID)
  142. err := pool.AddEvidence(prunedEv)
  143. require.NoError(t, err)
  144. ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime, val, evidenceChainID)
  145. lastCommit := makeCommit(height, val.PrivKey.PubKey().Address())
  146. block := types.MakeBlock(height+1, []types.Tx{}, lastCommit, []types.Evidence{ev})
  147. // update state (partially)
  148. state.LastBlockHeight = height + 1
  149. state.LastBlockTime = defaultEvidenceTime.Add(22 * time.Minute)
  150. err = pool.CheckEvidence(types.EvidenceList{ev})
  151. require.NoError(t, err)
  152. byzVals := pool.ABCIEvidence(block.Height, block.Evidence.Evidence)
  153. expectedByzVals := []abci.Evidence{
  154. {
  155. Type: abci.EvidenceType_DUPLICATE_VOTE,
  156. Validator: types.TM2PB.Validator(val.ExtractIntoValidator(10)),
  157. Height: height,
  158. Time: defaultEvidenceTime.Add(time.Duration(height) * time.Minute),
  159. TotalVotingPower: 10,
  160. },
  161. }
  162. assert.Equal(t, expectedByzVals, byzVals)
  163. evList, _ := pool.PendingEvidence(defaultEvidenceMaxBytes)
  164. assert.Equal(t, 1, len(evList))
  165. pool.Update(state)
  166. // a) Update marks evidence as committed so pending evidence should be empty
  167. evList, evSize := pool.PendingEvidence(defaultEvidenceMaxBytes)
  168. assert.Empty(t, evList)
  169. assert.Zero(t, evSize)
  170. // b) If we try to check this evidence again it should fail because it has already been committed
  171. err = pool.CheckEvidence(types.EvidenceList{ev})
  172. if assert.Error(t, err) {
  173. assert.Equal(t, "evidence was already committed", err.(*types.ErrInvalidEvidence).Reason.Error())
  174. }
  175. assert.Empty(t, pool.ABCIEvidence(height, []types.Evidence{}))
  176. }
  177. func TestVerifyPendingEvidencePasses(t *testing.T) {
  178. var height int64 = 1
  179. pool, val := defaultTestPool(height)
  180. ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime, val, evidenceChainID)
  181. err := pool.AddEvidence(ev)
  182. require.NoError(t, err)
  183. err = pool.CheckEvidence(types.EvidenceList{ev})
  184. assert.NoError(t, err)
  185. }
  186. func TestVerifyDuplicatedEvidenceFails(t *testing.T) {
  187. var height int64 = 1
  188. pool, val := defaultTestPool(height)
  189. ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime, val, evidenceChainID)
  190. err := pool.CheckEvidence(types.EvidenceList{ev, ev})
  191. if assert.Error(t, err) {
  192. assert.Equal(t, "duplicate evidence", err.(*types.ErrInvalidEvidence).Reason.Error())
  193. }
  194. }
  195. // check that
  196. func TestCheckEvidenceWithLightClientAttack(t *testing.T) {
  197. nValidators := 5
  198. conflictingVals, conflictingPrivVals := types.RandValidatorSet(nValidators, 10)
  199. trustedHeader := makeHeaderRandom(10)
  200. conflictingHeader := makeHeaderRandom(10)
  201. conflictingHeader.ValidatorsHash = conflictingVals.Hash()
  202. trustedHeader.ValidatorsHash = conflictingHeader.ValidatorsHash
  203. trustedHeader.NextValidatorsHash = conflictingHeader.NextValidatorsHash
  204. trustedHeader.ConsensusHash = conflictingHeader.ConsensusHash
  205. trustedHeader.AppHash = conflictingHeader.AppHash
  206. trustedHeader.LastResultsHash = conflictingHeader.LastResultsHash
  207. // for simplicity we are simulating a duplicate vote attack where all the validators in the
  208. // conflictingVals set voted twice
  209. blockID := makeBlockID(conflictingHeader.Hash(), 1000, []byte("partshash"))
  210. voteSet := types.NewVoteSet(evidenceChainID, 10, 1, tmproto.SignedMsgType(2), conflictingVals)
  211. commit, err := types.MakeCommit(blockID, 10, 1, voteSet, conflictingPrivVals, defaultEvidenceTime)
  212. require.NoError(t, err)
  213. ev := &types.LightClientAttackEvidence{
  214. ConflictingBlock: &types.LightBlock{
  215. SignedHeader: &types.SignedHeader{
  216. Header: conflictingHeader,
  217. Commit: commit,
  218. },
  219. ValidatorSet: conflictingVals,
  220. },
  221. CommonHeight: 10,
  222. }
  223. trustedBlockID := makeBlockID(trustedHeader.Hash(), 1000, []byte("partshash"))
  224. trustedVoteSet := types.NewVoteSet(evidenceChainID, 10, 1, tmproto.SignedMsgType(2), conflictingVals)
  225. trustedCommit, err := types.MakeCommit(trustedBlockID, 10, 1, trustedVoteSet, conflictingPrivVals, defaultEvidenceTime)
  226. require.NoError(t, err)
  227. state := sm.State{
  228. LastBlockTime: defaultEvidenceTime.Add(1 * time.Minute),
  229. LastBlockHeight: 11,
  230. ConsensusParams: *types.DefaultConsensusParams(),
  231. }
  232. stateStore := &smmocks.Store{}
  233. stateStore.On("LoadValidators", int64(10)).Return(conflictingVals, nil)
  234. stateStore.On("Load").Return(state, nil)
  235. blockStore := &mocks.BlockStore{}
  236. blockStore.On("LoadBlockMeta", int64(10)).Return(&types.BlockMeta{Header: *trustedHeader})
  237. blockStore.On("LoadBlockCommit", int64(10)).Return(trustedCommit)
  238. pool, err := evidence.NewPool(dbm.NewMemDB(), stateStore, blockStore)
  239. require.NoError(t, err)
  240. pool.SetLogger(log.TestingLogger())
  241. err = pool.AddEvidence(ev)
  242. assert.NoError(t, err)
  243. err = pool.CheckEvidence(types.EvidenceList{ev})
  244. assert.NoError(t, err)
  245. // take away the last signature -> there are less validators then what we have detected,
  246. // hence we move to full verification where the evidence should still pass
  247. commit.Signatures = append(commit.Signatures[:nValidators-1], types.NewCommitSigAbsent())
  248. err = pool.CheckEvidence(types.EvidenceList{ev})
  249. assert.NoError(t, err)
  250. // take away the last two signatures -> should fail due to insufficient power
  251. commit.Signatures = append(commit.Signatures[:nValidators-2], types.NewCommitSigAbsent(), types.NewCommitSigAbsent())
  252. err = pool.CheckEvidence(types.EvidenceList{ev})
  253. assert.Error(t, err)
  254. }
  255. func TestRecoverPendingEvidence(t *testing.T) {
  256. height := int64(10)
  257. val := types.NewMockPV()
  258. valAddress := val.PrivKey.PubKey().Address()
  259. evidenceDB := dbm.NewMemDB()
  260. stateStore := initializeValidatorState(val, height)
  261. state, err := stateStore.Load()
  262. require.NoError(t, err)
  263. blockStore := initializeBlockStore(dbm.NewMemDB(), state, valAddress)
  264. // create previous pool and populate it
  265. pool, err := evidence.NewPool(evidenceDB, stateStore, blockStore)
  266. require.NoError(t, err)
  267. pool.SetLogger(log.TestingLogger())
  268. goodEvidence := types.NewMockDuplicateVoteEvidenceWithValidator(height,
  269. defaultEvidenceTime, val, evidenceChainID)
  270. expiredEvidence := types.NewMockDuplicateVoteEvidenceWithValidator(int64(1),
  271. defaultEvidenceTime, val, evidenceChainID)
  272. err = pool.AddEvidence(goodEvidence)
  273. require.NoError(t, err)
  274. err = pool.AddEvidence(expiredEvidence)
  275. require.NoError(t, err)
  276. // now recover from the previous pool at a different time
  277. newStateStore := &smmocks.Store{}
  278. newStateStore.On("Load").Return(sm.State{
  279. LastBlockTime: defaultEvidenceTime.Add(25 * time.Minute),
  280. LastBlockHeight: height + 15,
  281. ConsensusParams: tmproto.ConsensusParams{
  282. Block: tmproto.BlockParams{
  283. MaxBytes: 22020096,
  284. MaxGas: -1,
  285. },
  286. Evidence: tmproto.EvidenceParams{
  287. MaxAgeNumBlocks: 20,
  288. MaxAgeDuration: 20 * time.Minute,
  289. MaxBytes: 1000,
  290. },
  291. },
  292. }, nil)
  293. newPool, err := evidence.NewPool(evidenceDB, newStateStore, blockStore)
  294. assert.NoError(t, err)
  295. evList, _ := newPool.PendingEvidence(defaultEvidenceMaxBytes)
  296. assert.Equal(t, 1, len(evList))
  297. next := newPool.EvidenceFront()
  298. assert.Equal(t, goodEvidence, next.Value.(types.Evidence))
  299. }
  300. func initializeStateFromValidatorSet(valSet *types.ValidatorSet, height int64) sm.Store {
  301. stateDB := dbm.NewMemDB()
  302. stateStore := sm.NewStore(stateDB)
  303. state := sm.State{
  304. ChainID: evidenceChainID,
  305. InitialHeight: 1,
  306. LastBlockHeight: height,
  307. LastBlockTime: defaultEvidenceTime,
  308. Validators: valSet,
  309. NextValidators: valSet.CopyIncrementProposerPriority(1),
  310. LastValidators: valSet,
  311. LastHeightValidatorsChanged: 1,
  312. ConsensusParams: tmproto.ConsensusParams{
  313. Block: tmproto.BlockParams{
  314. MaxBytes: 22020096,
  315. MaxGas: -1,
  316. },
  317. Evidence: tmproto.EvidenceParams{
  318. MaxAgeNumBlocks: 20,
  319. MaxAgeDuration: 20 * time.Minute,
  320. MaxBytes: 1000,
  321. },
  322. },
  323. }
  324. // save all states up to height
  325. for i := int64(0); i <= height; i++ {
  326. state.LastBlockHeight = i
  327. if err := stateStore.Save(state); err != nil {
  328. panic(err)
  329. }
  330. }
  331. return stateStore
  332. }
  333. func initializeValidatorState(privVal types.PrivValidator, height int64) sm.Store {
  334. pubKey, _ := privVal.GetPubKey()
  335. validator := &types.Validator{Address: pubKey.Address(), VotingPower: 10, PubKey: pubKey}
  336. // create validator set and state
  337. valSet := &types.ValidatorSet{
  338. Validators: []*types.Validator{validator},
  339. Proposer: validator,
  340. }
  341. return initializeStateFromValidatorSet(valSet, height)
  342. }
  343. // initializeBlockStore creates a block storage and populates it w/ a dummy
  344. // block at +height+.
  345. func initializeBlockStore(db dbm.DB, state sm.State, valAddr []byte) *store.BlockStore {
  346. blockStore := store.NewBlockStore(db)
  347. for i := int64(1); i <= state.LastBlockHeight; i++ {
  348. lastCommit := makeCommit(i-1, valAddr)
  349. block, _ := state.MakeBlock(i, []types.Tx{}, lastCommit, nil,
  350. state.Validators.GetProposer().Address)
  351. block.Header.Time = defaultEvidenceTime.Add(time.Duration(i) * time.Minute)
  352. block.Header.Version = tmversion.Consensus{Block: version.BlockProtocol, App: 1}
  353. const parts = 1
  354. partSet := block.MakePartSet(parts)
  355. seenCommit := makeCommit(i, valAddr)
  356. blockStore.SaveBlock(block, partSet, seenCommit)
  357. }
  358. return blockStore
  359. }
  360. func makeCommit(height int64, valAddr []byte) *types.Commit {
  361. commitSigs := []types.CommitSig{{
  362. BlockIDFlag: types.BlockIDFlagCommit,
  363. ValidatorAddress: valAddr,
  364. Timestamp: defaultEvidenceTime,
  365. Signature: []byte("Signature"),
  366. }}
  367. return types.NewCommit(height, 0, types.BlockID{}, commitSigs)
  368. }
  369. func defaultTestPool(height int64) (*evidence.Pool, types.MockPV) {
  370. val := types.NewMockPV()
  371. valAddress := val.PrivKey.PubKey().Address()
  372. evidenceDB := dbm.NewMemDB()
  373. stateStore := initializeValidatorState(val, height)
  374. state, _ := stateStore.Load()
  375. blockStore := initializeBlockStore(dbm.NewMemDB(), state, valAddress)
  376. pool, err := evidence.NewPool(evidenceDB, stateStore, blockStore)
  377. if err != nil {
  378. panic("test evidence pool could not be created")
  379. }
  380. pool.SetLogger(log.TestingLogger())
  381. return pool, val
  382. }
  383. func createState(height int64, valSet *types.ValidatorSet) sm.State {
  384. return sm.State{
  385. ChainID: evidenceChainID,
  386. LastBlockHeight: height,
  387. LastBlockTime: defaultEvidenceTime,
  388. Validators: valSet,
  389. ConsensusParams: *types.DefaultConsensusParams(),
  390. }
  391. }