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.

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