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.

663 lines
20 KiB

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