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.

697 lines
21 KiB

7 years ago
7 years ago
7 years ago
7 years ago
lint: Enable Golint (#4212) * Fix many golint errors * Fix golint errors in the 'lite' package * Don't export Pool.store * Fix typo * Revert unwanted changes * Fix errors in counter package * Fix linter errors in kvstore package * Fix linter error in example package * Fix error in tests package * Fix linter errors in v2 package * Fix linter errors in consensus package * Fix linter errors in evidence package * Fix linter error in fail package * Fix linter errors in query package * Fix linter errors in core package * Fix linter errors in node package * Fix linter errors in mempool package * Fix linter error in conn package * Fix linter errors in pex package * Rename PEXReactor export to Reactor * Fix linter errors in trust package * Fix linter errors in upnp package * Fix linter errors in p2p package * Fix linter errors in proxy package * Fix linter errors in mock_test package * Fix linter error in client_test package * Fix linter errors in coretypes package * Fix linter errors in coregrpc package * Fix linter errors in rpcserver package * Fix linter errors in rpctypes package * Fix linter errors in rpctest package * Fix linter error in json2wal script * Fix linter error in wal2json script * Fix linter errors in kv package * Fix linter error in state package * Fix linter error in grpc_client * Fix linter errors in types package * Fix linter error in version package * Fix remaining errors * Address review comments * Fix broken tests * Reconcile package coregrpc * Fix golangci bot error * Fix new golint errors * Fix broken reference * Enable golint linter * minor changes to bring golint into line * fix failing test * fix pex reactor naming * address PR comments
5 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
  1. package evidence
  2. import (
  3. "bytes"
  4. "errors"
  5. "fmt"
  6. "sort"
  7. "sync"
  8. "sync/atomic"
  9. "time"
  10. "github.com/gogo/protobuf/proto"
  11. gogotypes "github.com/gogo/protobuf/types"
  12. "github.com/google/orderedcode"
  13. dbm "github.com/tendermint/tm-db"
  14. clist "github.com/tendermint/tendermint/libs/clist"
  15. "github.com/tendermint/tendermint/libs/log"
  16. tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
  17. sm "github.com/tendermint/tendermint/state"
  18. "github.com/tendermint/tendermint/types"
  19. )
  20. const (
  21. // prefixes are unique across all tm db's
  22. prefixCommitted = int64(9)
  23. prefixPending = int64(10)
  24. )
  25. // Pool maintains a pool of valid evidence to be broadcasted and committed
  26. type Pool struct {
  27. logger log.Logger
  28. evidenceStore dbm.DB
  29. evidenceList *clist.CList // concurrent linked-list of evidence
  30. evidenceSize uint32 // amount of pending evidence
  31. // needed to load validators to verify evidence
  32. stateDB sm.Store
  33. // needed to load headers and commits to verify evidence
  34. blockStore BlockStore
  35. mtx sync.Mutex
  36. // latest state
  37. state sm.State
  38. // evidence from consensus is buffered to this slice, awaiting until the next height
  39. // before being flushed to the pool. This prevents broadcasting and proposing of
  40. // evidence before the height with which the evidence happened is finished.
  41. consensusBuffer []duplicateVoteSet
  42. pruningHeight int64
  43. pruningTime time.Time
  44. }
  45. // NewPool creates an evidence pool. If using an existing evidence store,
  46. // it will add all pending evidence to the concurrent list.
  47. func NewPool(logger log.Logger, evidenceDB dbm.DB, stateDB sm.Store, blockStore BlockStore) (*Pool, error) {
  48. state, err := stateDB.Load()
  49. if err != nil {
  50. return nil, fmt.Errorf("failed to load state: %w", err)
  51. }
  52. pool := &Pool{
  53. stateDB: stateDB,
  54. blockStore: blockStore,
  55. state: state,
  56. logger: logger,
  57. evidenceStore: evidenceDB,
  58. evidenceList: clist.New(),
  59. consensusBuffer: make([]duplicateVoteSet, 0),
  60. }
  61. // If pending evidence already in db, in event of prior failure, then check
  62. // for expiration, update the size and load it back to the evidenceList.
  63. pool.pruningHeight, pool.pruningTime = pool.removeExpiredPendingEvidence()
  64. evList, _, err := pool.listEvidence(prefixPending, -1)
  65. if err != nil {
  66. return nil, err
  67. }
  68. atomic.StoreUint32(&pool.evidenceSize, uint32(len(evList)))
  69. for _, ev := range evList {
  70. pool.evidenceList.PushBack(ev)
  71. }
  72. return pool, nil
  73. }
  74. // PendingEvidence is used primarily as part of block proposal and returns up to
  75. // maxNum of uncommitted evidence.
  76. func (evpool *Pool) PendingEvidence(maxBytes int64) ([]types.Evidence, int64) {
  77. if evpool.Size() == 0 {
  78. return []types.Evidence{}, 0
  79. }
  80. evidence, size, err := evpool.listEvidence(prefixPending, maxBytes)
  81. if err != nil {
  82. evpool.logger.Error("failed to retrieve pending evidence", "err", err)
  83. }
  84. return evidence, size
  85. }
  86. // Update takes both the new state and the evidence committed at that height and performs
  87. // the following operations:
  88. // 1. Take any conflicting votes from consensus and use the state's LastBlockTime to form
  89. // DuplicateVoteEvidence and add it to the pool.
  90. // 2. Update the pool's state which contains evidence params relating to expiry.
  91. // 3. Moves pending evidence that has now been committed into the committed pool.
  92. // 4. Removes any expired evidence based on both height and time.
  93. func (evpool *Pool) Update(state sm.State, ev types.EvidenceList) {
  94. // sanity check
  95. if state.LastBlockHeight <= evpool.state.LastBlockHeight {
  96. panic(fmt.Sprintf(
  97. "failed EvidencePool.Update new state height is less than or equal to previous state height: %d <= %d",
  98. state.LastBlockHeight,
  99. evpool.state.LastBlockHeight,
  100. ))
  101. }
  102. evpool.logger.Debug(
  103. "updating evidence pool",
  104. "last_block_height", state.LastBlockHeight,
  105. "last_block_time", state.LastBlockTime,
  106. )
  107. // flush conflicting vote pairs from the buffer, producing DuplicateVoteEvidence and
  108. // adding it to the pool
  109. evpool.processConsensusBuffer(state)
  110. // update state
  111. evpool.updateState(state)
  112. // move committed evidence out from the pending pool and into the committed pool
  113. evpool.markEvidenceAsCommitted(ev, state.LastBlockHeight)
  114. // Prune pending evidence when it has expired. This also updates when the next
  115. // evidence will expire.
  116. if evpool.Size() > 0 && state.LastBlockHeight > evpool.pruningHeight &&
  117. state.LastBlockTime.After(evpool.pruningTime) {
  118. evpool.pruningHeight, evpool.pruningTime = evpool.removeExpiredPendingEvidence()
  119. }
  120. }
  121. // AddEvidence checks the evidence is valid and adds it to the pool.
  122. func (evpool *Pool) AddEvidence(ev types.Evidence) error {
  123. evpool.logger.Debug("attempting to add evidence", "evidence", ev)
  124. // We have already verified this piece of evidence - no need to do it again
  125. if evpool.isPending(ev) {
  126. evpool.logger.Debug("evidence already pending; ignoring", "evidence", ev)
  127. return nil
  128. }
  129. // check that the evidence isn't already committed
  130. if evpool.isCommitted(ev) {
  131. // This can happen if the peer that sent us the evidence is behind so we
  132. // shouldn't punish the peer.
  133. evpool.logger.Debug("evidence was already committed; ignoring", "evidence", ev)
  134. return nil
  135. }
  136. // 1) Verify against state.
  137. if err := evpool.verify(ev); err != nil {
  138. return err
  139. }
  140. // 2) Save to store.
  141. if err := evpool.addPendingEvidence(ev); err != nil {
  142. return fmt.Errorf("failed to add evidence to pending list: %w", err)
  143. }
  144. // 3) Add evidence to clist.
  145. evpool.evidenceList.PushBack(ev)
  146. evpool.logger.Info("verified new evidence of byzantine behavior", "evidence", ev)
  147. return nil
  148. }
  149. // ReportConflictingVotes takes two conflicting votes and forms duplicate vote evidence,
  150. // adding it eventually to the evidence pool.
  151. //
  152. // Duplicate vote attacks happen before the block is committed and the timestamp is
  153. // finalized, thus the evidence pool holds these votes in a buffer, forming the
  154. // evidence from them once consensus at that height has been reached and `Update()` with
  155. // the new state called.
  156. //
  157. // Votes are not verified.
  158. func (evpool *Pool) ReportConflictingVotes(voteA, voteB *types.Vote) {
  159. evpool.mtx.Lock()
  160. defer evpool.mtx.Unlock()
  161. evpool.consensusBuffer = append(evpool.consensusBuffer, duplicateVoteSet{
  162. VoteA: voteA,
  163. VoteB: voteB,
  164. })
  165. }
  166. // CheckEvidence takes an array of evidence from a block and verifies all the evidence there.
  167. // If it has already verified the evidence then it jumps to the next one. It ensures that no
  168. // evidence has already been committed or is being proposed twice. It also adds any
  169. // evidence that it doesn't currently have so that it can quickly form ABCI Evidence later.
  170. func (evpool *Pool) CheckEvidence(evList types.EvidenceList) error {
  171. hashes := make([][]byte, len(evList))
  172. for idx, ev := range evList {
  173. ok := evpool.fastCheck(ev)
  174. if !ok {
  175. // check that the evidence isn't already committed
  176. if evpool.isCommitted(ev) {
  177. return &types.ErrInvalidEvidence{Evidence: ev, Reason: errors.New("evidence was already committed")}
  178. }
  179. err := evpool.verify(ev)
  180. if err != nil {
  181. return err
  182. }
  183. if err := evpool.addPendingEvidence(ev); err != nil {
  184. // Something went wrong with adding the evidence but we already know it is valid
  185. // hence we log an error and continue
  186. evpool.logger.Error("failed to add evidence to pending list", "err", err, "evidence", ev)
  187. }
  188. evpool.logger.Info("verified new evidence of byzantine behavior", "evidence", ev)
  189. }
  190. // check for duplicate evidence. We cache hashes so we don't have to work them out again.
  191. hashes[idx] = ev.Hash()
  192. for i := idx - 1; i >= 0; i-- {
  193. if bytes.Equal(hashes[i], hashes[idx]) {
  194. return &types.ErrInvalidEvidence{Evidence: ev, Reason: errors.New("duplicate evidence")}
  195. }
  196. }
  197. }
  198. return nil
  199. }
  200. // EvidenceFront goes to the first evidence in the clist
  201. func (evpool *Pool) EvidenceFront() *clist.CElement {
  202. return evpool.evidenceList.Front()
  203. }
  204. // EvidenceWaitChan is a channel that closes once the first evidence in the list
  205. // is there. i.e Front is not nil.
  206. func (evpool *Pool) EvidenceWaitChan() <-chan struct{} {
  207. return evpool.evidenceList.WaitChan()
  208. }
  209. // Size returns the number of evidence in the pool.
  210. func (evpool *Pool) Size() uint32 {
  211. return atomic.LoadUint32(&evpool.evidenceSize)
  212. }
  213. // State returns the current state of the evpool.
  214. func (evpool *Pool) State() sm.State {
  215. evpool.mtx.Lock()
  216. defer evpool.mtx.Unlock()
  217. return evpool.state
  218. }
  219. // fastCheck leverages the fact that the evidence pool may have already verified
  220. // the evidence to see if it can quickly conclude that the evidence is already
  221. // valid.
  222. func (evpool *Pool) fastCheck(ev types.Evidence) bool {
  223. if lcae, ok := ev.(*types.LightClientAttackEvidence); ok {
  224. key := keyPending(ev)
  225. evBytes, err := evpool.evidenceStore.Get(key)
  226. if evBytes == nil { // the evidence is not in the nodes pending list
  227. return false
  228. }
  229. if err != nil {
  230. evpool.logger.Error("failed to load light client attack evidence", "err", err, "key(height/hash)", key)
  231. return false
  232. }
  233. var trustedPb tmproto.LightClientAttackEvidence
  234. if err = trustedPb.Unmarshal(evBytes); err != nil {
  235. evpool.logger.Error(
  236. "failed to convert light client attack evidence from bytes",
  237. "key(height/hash)", key,
  238. "err", err,
  239. )
  240. return false
  241. }
  242. trustedEv, err := types.LightClientAttackEvidenceFromProto(&trustedPb)
  243. if err != nil {
  244. evpool.logger.Error(
  245. "failed to convert light client attack evidence from protobuf",
  246. "key(height/hash)", key,
  247. "err", err,
  248. )
  249. return false
  250. }
  251. // Ensure that all the byzantine validators that the evidence pool has match
  252. // the byzantine validators in this evidence.
  253. if trustedEv.ByzantineValidators == nil && lcae.ByzantineValidators != nil {
  254. return false
  255. }
  256. if len(trustedEv.ByzantineValidators) != len(lcae.ByzantineValidators) {
  257. return false
  258. }
  259. byzValsCopy := make([]*types.Validator, len(lcae.ByzantineValidators))
  260. for i, v := range lcae.ByzantineValidators {
  261. byzValsCopy[i] = v.Copy()
  262. }
  263. // ensure that both validator arrays are in the same order
  264. sort.Sort(types.ValidatorsByVotingPower(byzValsCopy))
  265. for idx, val := range trustedEv.ByzantineValidators {
  266. if !bytes.Equal(byzValsCopy[idx].Address, val.Address) {
  267. return false
  268. }
  269. if byzValsCopy[idx].VotingPower != val.VotingPower {
  270. return false
  271. }
  272. }
  273. return true
  274. }
  275. // For all other evidence the evidence pool just checks if it is already in
  276. // the pending db.
  277. return evpool.isPending(ev)
  278. }
  279. // IsExpired checks whether evidence or a polc is expired by checking whether a height and time is older
  280. // than set by the evidence consensus parameters
  281. func (evpool *Pool) isExpired(height int64, time time.Time) bool {
  282. var (
  283. params = evpool.State().ConsensusParams.Evidence
  284. ageDuration = evpool.State().LastBlockTime.Sub(time)
  285. ageNumBlocks = evpool.State().LastBlockHeight - height
  286. )
  287. return ageNumBlocks > params.MaxAgeNumBlocks &&
  288. ageDuration > params.MaxAgeDuration
  289. }
  290. // IsCommitted returns true if we have already seen this exact evidence and it is already marked as committed.
  291. func (evpool *Pool) isCommitted(evidence types.Evidence) bool {
  292. key := keyCommitted(evidence)
  293. ok, err := evpool.evidenceStore.Has(key)
  294. if err != nil {
  295. evpool.logger.Error("failed to find committed evidence", "err", err)
  296. }
  297. return ok
  298. }
  299. // IsPending checks whether the evidence is already pending. DB errors are passed to the logger.
  300. func (evpool *Pool) isPending(evidence types.Evidence) bool {
  301. key := keyPending(evidence)
  302. ok, err := evpool.evidenceStore.Has(key)
  303. if err != nil {
  304. evpool.logger.Error("failed to find pending evidence", "err", err)
  305. }
  306. return ok
  307. }
  308. func (evpool *Pool) addPendingEvidence(ev types.Evidence) error {
  309. evpb, err := types.EvidenceToProto(ev)
  310. if err != nil {
  311. return fmt.Errorf("failed to convert to proto: %w", err)
  312. }
  313. evBytes, err := evpb.Marshal()
  314. if err != nil {
  315. return fmt.Errorf("failed to marshal evidence: %w", err)
  316. }
  317. key := keyPending(ev)
  318. err = evpool.evidenceStore.Set(key, evBytes)
  319. if err != nil {
  320. return fmt.Errorf("failed to persist evidence: %w", err)
  321. }
  322. atomic.AddUint32(&evpool.evidenceSize, 1)
  323. return nil
  324. }
  325. // markEvidenceAsCommitted processes all the evidence in the block, marking it as
  326. // committed and removing it from the pending database.
  327. func (evpool *Pool) markEvidenceAsCommitted(evidence types.EvidenceList, height int64) {
  328. blockEvidenceMap := make(map[string]struct{}, len(evidence))
  329. batch := evpool.evidenceStore.NewBatch()
  330. defer batch.Close()
  331. for _, ev := range evidence {
  332. if evpool.isPending(ev) {
  333. if err := batch.Delete(keyPending(ev)); err != nil {
  334. evpool.logger.Error("failed to batch pending evidence", "err", err)
  335. }
  336. blockEvidenceMap[evMapKey(ev)] = struct{}{}
  337. }
  338. // Add evidence to the committed list. As the evidence is stored in the block store
  339. // we only need to record the height that it was saved at.
  340. key := keyCommitted(ev)
  341. h := gogotypes.Int64Value{Value: height}
  342. evBytes, err := proto.Marshal(&h)
  343. if err != nil {
  344. evpool.logger.Error("failed to marshal committed evidence", "key(height/hash)", key, "err", err)
  345. continue
  346. }
  347. if err := evpool.evidenceStore.Set(key, evBytes); err != nil {
  348. evpool.logger.Error("failed to save committed evidence", "key(height/hash)", key, "err", err)
  349. }
  350. evpool.logger.Debug("marked evidence as committed", "evidence", ev)
  351. }
  352. // check if we need to remove any pending evidence
  353. if len(blockEvidenceMap) == 0 {
  354. return
  355. }
  356. // remove committed evidence from pending bucket
  357. if err := batch.WriteSync(); err != nil {
  358. evpool.logger.Error("failed to batch delete pending evidence", "err", err)
  359. return
  360. }
  361. // remove committed evidence from the clist
  362. evpool.removeEvidenceFromList(blockEvidenceMap)
  363. // update the evidence size
  364. atomic.AddUint32(&evpool.evidenceSize, ^uint32(len(blockEvidenceMap)-1))
  365. }
  366. // listEvidence retrieves lists evidence from oldest to newest within maxBytes.
  367. // If maxBytes is -1, there's no cap on the size of returned evidence.
  368. func (evpool *Pool) listEvidence(prefixKey int64, maxBytes int64) ([]types.Evidence, int64, error) {
  369. var (
  370. evSize int64
  371. totalSize int64
  372. evidence []types.Evidence
  373. evList tmproto.EvidenceList // used for calculating the bytes size
  374. )
  375. iter, err := dbm.IteratePrefix(evpool.evidenceStore, prefixToBytes(prefixKey))
  376. if err != nil {
  377. return nil, totalSize, fmt.Errorf("database error: %v", err)
  378. }
  379. defer iter.Close()
  380. for ; iter.Valid(); iter.Next() {
  381. var evpb tmproto.Evidence
  382. if err := evpb.Unmarshal(iter.Value()); err != nil {
  383. return evidence, totalSize, err
  384. }
  385. evList.Evidence = append(evList.Evidence, evpb)
  386. evSize = int64(evList.Size())
  387. if maxBytes != -1 && evSize > maxBytes {
  388. if err := iter.Error(); err != nil {
  389. return evidence, totalSize, err
  390. }
  391. return evidence, totalSize, nil
  392. }
  393. ev, err := types.EvidenceFromProto(&evpb)
  394. if err != nil {
  395. return nil, totalSize, err
  396. }
  397. totalSize = evSize
  398. evidence = append(evidence, ev)
  399. }
  400. if err := iter.Error(); err != nil {
  401. return evidence, totalSize, err
  402. }
  403. return evidence, totalSize, nil
  404. }
  405. func (evpool *Pool) removeExpiredPendingEvidence() (int64, time.Time) {
  406. batch := evpool.evidenceStore.NewBatch()
  407. defer batch.Close()
  408. height, time, blockEvidenceMap := evpool.batchExpiredPendingEvidence(batch)
  409. // if we haven't removed any evidence then return early
  410. if len(blockEvidenceMap) == 0 {
  411. return height, time
  412. }
  413. evpool.logger.Debug("removing expired evidence",
  414. "height", evpool.State().LastBlockHeight,
  415. "time", evpool.State().LastBlockTime,
  416. "expired evidence", len(blockEvidenceMap),
  417. )
  418. // remove expired evidence from pending bucket
  419. if err := batch.WriteSync(); err != nil {
  420. evpool.logger.Error("failed to batch delete pending evidence", "err", err)
  421. return evpool.State().LastBlockHeight, evpool.State().LastBlockTime
  422. }
  423. // remove evidence from the clist
  424. evpool.removeEvidenceFromList(blockEvidenceMap)
  425. // update the evidence size
  426. atomic.AddUint32(&evpool.evidenceSize, ^uint32(len(blockEvidenceMap)-1))
  427. return height, time
  428. }
  429. func (evpool *Pool) batchExpiredPendingEvidence(batch dbm.Batch) (int64, time.Time, map[string]struct{}) {
  430. blockEvidenceMap := make(map[string]struct{})
  431. iter, err := dbm.IteratePrefix(evpool.evidenceStore, prefixToBytes(prefixPending))
  432. if err != nil {
  433. evpool.logger.Error("failed to iterate over pending evidence", "err", err)
  434. return evpool.State().LastBlockHeight, evpool.State().LastBlockTime, blockEvidenceMap
  435. }
  436. defer iter.Close()
  437. for ; iter.Valid(); iter.Next() {
  438. ev, err := bytesToEv(iter.Value())
  439. if err != nil {
  440. evpool.logger.Error("failed to transition evidence from protobuf", "err", err, "ev", ev)
  441. continue
  442. }
  443. // if true, we have looped through all expired evidence
  444. if !evpool.isExpired(ev.Height(), ev.Time()) {
  445. // Return the height and time with which this evidence will have expired
  446. // so we know when to prune next.
  447. return ev.Height() + evpool.State().ConsensusParams.Evidence.MaxAgeNumBlocks + 1,
  448. ev.Time().Add(evpool.State().ConsensusParams.Evidence.MaxAgeDuration).Add(time.Second),
  449. blockEvidenceMap
  450. }
  451. // else add to the batch
  452. if err := batch.Delete(iter.Key()); err != nil {
  453. evpool.logger.Error("failed to batch evidence", "err", err, "ev", ev)
  454. continue
  455. }
  456. // and add to the map to remove the evidence from the clist
  457. blockEvidenceMap[evMapKey(ev)] = struct{}{}
  458. }
  459. return evpool.State().LastBlockHeight, evpool.State().LastBlockTime, blockEvidenceMap
  460. }
  461. func (evpool *Pool) removeEvidenceFromList(
  462. blockEvidenceMap map[string]struct{}) {
  463. for e := evpool.evidenceList.Front(); e != nil; e = e.Next() {
  464. // Remove from clist
  465. ev := e.Value.(types.Evidence)
  466. if _, ok := blockEvidenceMap[evMapKey(ev)]; ok {
  467. evpool.evidenceList.Remove(e)
  468. e.DetachPrev()
  469. }
  470. }
  471. }
  472. func (evpool *Pool) updateState(state sm.State) {
  473. evpool.mtx.Lock()
  474. defer evpool.mtx.Unlock()
  475. evpool.state = state
  476. }
  477. // processConsensusBuffer converts all the duplicate votes witnessed from consensus
  478. // into DuplicateVoteEvidence. It sets the evidence timestamp to the block height
  479. // from the most recently committed block.
  480. // Evidence is then added to the pool so as to be ready to be broadcasted and proposed.
  481. func (evpool *Pool) processConsensusBuffer(state sm.State) {
  482. evpool.mtx.Lock()
  483. defer evpool.mtx.Unlock()
  484. for _, voteSet := range evpool.consensusBuffer {
  485. // Check the height of the conflicting votes and fetch the corresponding time and validator set
  486. // to produce the valid evidence
  487. var dve *types.DuplicateVoteEvidence
  488. switch {
  489. case voteSet.VoteA.Height == state.LastBlockHeight:
  490. dve = types.NewDuplicateVoteEvidence(
  491. voteSet.VoteA,
  492. voteSet.VoteB,
  493. state.LastBlockTime,
  494. state.LastValidators,
  495. )
  496. case voteSet.VoteA.Height < state.LastBlockHeight:
  497. valSet, err := evpool.stateDB.LoadValidators(voteSet.VoteA.Height)
  498. if err != nil {
  499. evpool.logger.Error("failed to load validator set for conflicting votes",
  500. "height", voteSet.VoteA.Height, "err", err)
  501. continue
  502. }
  503. blockMeta := evpool.blockStore.LoadBlockMeta(voteSet.VoteA.Height)
  504. if blockMeta == nil {
  505. evpool.logger.Error("failed to load block time for conflicting votes", "height", voteSet.VoteA.Height)
  506. continue
  507. }
  508. dve = types.NewDuplicateVoteEvidence(
  509. voteSet.VoteA,
  510. voteSet.VoteB,
  511. blockMeta.Header.Time,
  512. valSet,
  513. )
  514. default:
  515. // evidence pool shouldn't expect to get votes from consensus of a height that is above the current
  516. // state. If this error is seen then perhaps consider keeping the votes in the buffer and retry
  517. // in following heights
  518. evpool.logger.Error("inbound duplicate votes from consensus are of a greater height than current state",
  519. "duplicate vote height", voteSet.VoteA.Height,
  520. "state.LastBlockHeight", state.LastBlockHeight)
  521. continue
  522. }
  523. // check if we already have this evidence
  524. if evpool.isPending(dve) {
  525. evpool.logger.Debug("evidence already pending; ignoring", "evidence", dve)
  526. continue
  527. }
  528. // check that the evidence is not already committed on chain
  529. if evpool.isCommitted(dve) {
  530. evpool.logger.Debug("evidence already committed; ignoring", "evidence", dve)
  531. continue
  532. }
  533. if err := evpool.addPendingEvidence(dve); err != nil {
  534. evpool.logger.Error("failed to flush evidence from consensus buffer to pending list: %w", err)
  535. continue
  536. }
  537. evpool.evidenceList.PushBack(dve)
  538. evpool.logger.Info("verified new evidence of byzantine behavior", "evidence", dve)
  539. }
  540. // reset consensus buffer
  541. evpool.consensusBuffer = make([]duplicateVoteSet, 0)
  542. }
  543. type duplicateVoteSet struct {
  544. VoteA *types.Vote
  545. VoteB *types.Vote
  546. }
  547. func bytesToEv(evBytes []byte) (types.Evidence, error) {
  548. var evpb tmproto.Evidence
  549. err := evpb.Unmarshal(evBytes)
  550. if err != nil {
  551. return &types.DuplicateVoteEvidence{}, err
  552. }
  553. return types.EvidenceFromProto(&evpb)
  554. }
  555. func evMapKey(ev types.Evidence) string {
  556. return string(ev.Hash())
  557. }
  558. func prefixToBytes(prefix int64) []byte {
  559. key, err := orderedcode.Append(nil, prefix)
  560. if err != nil {
  561. panic(err)
  562. }
  563. return key
  564. }
  565. func keyCommitted(evidence types.Evidence) []byte {
  566. var height int64 = evidence.Height()
  567. key, err := orderedcode.Append(nil, prefixCommitted, height, string(evidence.Hash()))
  568. if err != nil {
  569. panic(err)
  570. }
  571. return key
  572. }
  573. func keyPending(evidence types.Evidence) []byte {
  574. var height int64 = evidence.Height()
  575. key, err := orderedcode.Append(nil, prefixPending, height, string(evidence.Hash()))
  576. if err != nil {
  577. panic(err)
  578. }
  579. return key
  580. }