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.

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