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.

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