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.

537 lines
14 KiB

  1. package consensus
  2. import (
  3. "context"
  4. "encoding/json"
  5. "errors"
  6. "fmt"
  7. "sync"
  8. "time"
  9. cstypes "github.com/tendermint/tendermint/internal/consensus/types"
  10. "github.com/tendermint/tendermint/libs/bits"
  11. "github.com/tendermint/tendermint/libs/log"
  12. tmtime "github.com/tendermint/tendermint/libs/time"
  13. tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
  14. "github.com/tendermint/tendermint/types"
  15. )
  16. var (
  17. ErrPeerStateHeightRegression = errors.New("peer state height regression")
  18. ErrPeerStateInvalidStartTime = errors.New("peer state invalid startTime")
  19. ErrPeerStateSetNilVote = errors.New("peer state set a nil vote")
  20. ErrPeerStateInvalidVoteIndex = errors.New("peer sent a vote with an invalid vote index")
  21. )
  22. // peerStateStats holds internal statistics for a peer.
  23. type peerStateStats struct {
  24. Votes int `json:"votes,string"`
  25. BlockParts int `json:"block_parts,string"`
  26. }
  27. func (pss peerStateStats) String() string {
  28. return fmt.Sprintf("peerStateStats{votes: %d, blockParts: %d}", pss.Votes, pss.BlockParts)
  29. }
  30. // PeerState contains the known state of a peer, including its connection and
  31. // threadsafe access to its PeerRoundState.
  32. // NOTE: THIS GETS DUMPED WITH rpc/core/consensus.go.
  33. // Be mindful of what you Expose.
  34. type PeerState struct {
  35. peerID types.NodeID
  36. logger log.Logger
  37. // NOTE: Modify below using setters, never directly.
  38. mtx sync.RWMutex
  39. cancel context.CancelFunc
  40. running bool
  41. PRS cstypes.PeerRoundState `json:"round_state"`
  42. Stats *peerStateStats `json:"stats"`
  43. }
  44. // NewPeerState returns a new PeerState for the given node ID.
  45. func NewPeerState(logger log.Logger, peerID types.NodeID) *PeerState {
  46. return &PeerState{
  47. peerID: peerID,
  48. logger: logger,
  49. PRS: cstypes.PeerRoundState{
  50. Round: -1,
  51. ProposalPOLRound: -1,
  52. LastCommitRound: -1,
  53. CatchupCommitRound: -1,
  54. },
  55. Stats: &peerStateStats{},
  56. }
  57. }
  58. // SetRunning sets the running state of the peer.
  59. func (ps *PeerState) SetRunning(v bool) {
  60. ps.mtx.Lock()
  61. defer ps.mtx.Unlock()
  62. ps.running = v
  63. }
  64. // IsRunning returns true if a PeerState is considered running where multiple
  65. // broadcasting goroutines exist for the peer.
  66. func (ps *PeerState) IsRunning() bool {
  67. ps.mtx.RLock()
  68. defer ps.mtx.RUnlock()
  69. return ps.running
  70. }
  71. // GetRoundState returns a shallow copy of the PeerRoundState. There's no point
  72. // in mutating it since it won't change PeerState.
  73. func (ps *PeerState) GetRoundState() *cstypes.PeerRoundState {
  74. ps.mtx.Lock()
  75. defer ps.mtx.Unlock()
  76. prs := ps.PRS.Copy()
  77. return &prs
  78. }
  79. // ToJSON returns a json of PeerState.
  80. func (ps *PeerState) ToJSON() ([]byte, error) {
  81. ps.mtx.Lock()
  82. defer ps.mtx.Unlock()
  83. return json.Marshal(ps)
  84. }
  85. // GetHeight returns an atomic snapshot of the PeerRoundState's height used by
  86. // the mempool to ensure peers are caught up before broadcasting new txs.
  87. func (ps *PeerState) GetHeight() int64 {
  88. ps.mtx.Lock()
  89. defer ps.mtx.Unlock()
  90. return ps.PRS.Height
  91. }
  92. // SetHasProposal sets the given proposal as known for the peer.
  93. func (ps *PeerState) SetHasProposal(proposal *types.Proposal) {
  94. ps.mtx.Lock()
  95. defer ps.mtx.Unlock()
  96. if ps.PRS.Height != proposal.Height || ps.PRS.Round != proposal.Round {
  97. return
  98. }
  99. if ps.PRS.Proposal {
  100. return
  101. }
  102. ps.PRS.Proposal = true
  103. // ps.PRS.ProposalBlockParts is set due to NewValidBlockMessage
  104. if ps.PRS.ProposalBlockParts != nil {
  105. return
  106. }
  107. ps.PRS.ProposalBlockPartSetHeader = proposal.BlockID.PartSetHeader
  108. ps.PRS.ProposalBlockParts = bits.NewBitArray(int(proposal.BlockID.PartSetHeader.Total))
  109. ps.PRS.ProposalPOLRound = proposal.POLRound
  110. ps.PRS.ProposalPOL = nil // Nil until ProposalPOLMessage received.
  111. }
  112. // InitProposalBlockParts initializes the peer's proposal block parts header
  113. // and bit array.
  114. func (ps *PeerState) InitProposalBlockParts(partSetHeader types.PartSetHeader) {
  115. ps.mtx.Lock()
  116. defer ps.mtx.Unlock()
  117. if ps.PRS.ProposalBlockParts != nil {
  118. return
  119. }
  120. ps.PRS.ProposalBlockPartSetHeader = partSetHeader
  121. ps.PRS.ProposalBlockParts = bits.NewBitArray(int(partSetHeader.Total))
  122. }
  123. // SetHasProposalBlockPart sets the given block part index as known for the peer.
  124. func (ps *PeerState) SetHasProposalBlockPart(height int64, round int32, index int) {
  125. ps.mtx.Lock()
  126. defer ps.mtx.Unlock()
  127. if ps.PRS.Height != height || ps.PRS.Round != round {
  128. return
  129. }
  130. ps.PRS.ProposalBlockParts.SetIndex(index, true)
  131. }
  132. // PickVoteToSend picks a vote to send to the peer. It will return true if a
  133. // vote was picked.
  134. //
  135. // NOTE: `votes` must be the correct Size() for the Height().
  136. func (ps *PeerState) PickVoteToSend(votes types.VoteSetReader) (*types.Vote, bool) {
  137. ps.mtx.Lock()
  138. defer ps.mtx.Unlock()
  139. if votes.Size() == 0 {
  140. return nil, false
  141. }
  142. var (
  143. height = votes.GetHeight()
  144. round = votes.GetRound()
  145. votesType = tmproto.SignedMsgType(votes.Type())
  146. size = votes.Size()
  147. )
  148. // lazily set data using 'votes'
  149. if votes.IsCommit() {
  150. ps.ensureCatchupCommitRound(height, round, size)
  151. }
  152. ps.ensureVoteBitArrays(height, size)
  153. psVotes := ps.getVoteBitArray(height, round, votesType)
  154. if psVotes == nil {
  155. return nil, false // not something worth sending
  156. }
  157. if index, ok := votes.BitArray().Sub(psVotes).PickRandom(); ok {
  158. vote := votes.GetByIndex(int32(index))
  159. if vote != nil {
  160. return vote, true
  161. }
  162. }
  163. return nil, false
  164. }
  165. func (ps *PeerState) getVoteBitArray(height int64, round int32, votesType tmproto.SignedMsgType) *bits.BitArray {
  166. if !types.IsVoteTypeValid(votesType) {
  167. return nil
  168. }
  169. if ps.PRS.Height == height {
  170. if ps.PRS.Round == round {
  171. switch votesType {
  172. case tmproto.PrevoteType:
  173. return ps.PRS.Prevotes
  174. case tmproto.PrecommitType:
  175. return ps.PRS.Precommits
  176. }
  177. }
  178. if ps.PRS.CatchupCommitRound == round {
  179. switch votesType {
  180. case tmproto.PrevoteType:
  181. return nil
  182. case tmproto.PrecommitType:
  183. return ps.PRS.CatchupCommit
  184. }
  185. }
  186. if ps.PRS.ProposalPOLRound == round {
  187. switch votesType {
  188. case tmproto.PrevoteType:
  189. return ps.PRS.ProposalPOL
  190. case tmproto.PrecommitType:
  191. return nil
  192. }
  193. }
  194. return nil
  195. }
  196. if ps.PRS.Height == height+1 {
  197. if ps.PRS.LastCommitRound == round {
  198. switch votesType {
  199. case tmproto.PrevoteType:
  200. return nil
  201. case tmproto.PrecommitType:
  202. return ps.PRS.LastCommit
  203. }
  204. }
  205. return nil
  206. }
  207. return nil
  208. }
  209. // 'round': A round for which we have a +2/3 commit.
  210. func (ps *PeerState) ensureCatchupCommitRound(height int64, round int32, numValidators int) {
  211. if ps.PRS.Height != height {
  212. return
  213. }
  214. /*
  215. NOTE: This is wrong, 'round' could change.
  216. e.g. if orig round is not the same as block LastCommit round.
  217. if ps.CatchupCommitRound != -1 && ps.CatchupCommitRound != round {
  218. panic(fmt.Sprintf(
  219. "Conflicting CatchupCommitRound. Height: %v,
  220. Orig: %v,
  221. New: %v",
  222. height,
  223. ps.CatchupCommitRound,
  224. round))
  225. }
  226. */
  227. if ps.PRS.CatchupCommitRound == round {
  228. return // Nothing to do!
  229. }
  230. ps.PRS.CatchupCommitRound = round
  231. if round == ps.PRS.Round {
  232. ps.PRS.CatchupCommit = ps.PRS.Precommits
  233. } else {
  234. ps.PRS.CatchupCommit = bits.NewBitArray(numValidators)
  235. }
  236. }
  237. // EnsureVoteBitArrays ensures the bit-arrays have been allocated for tracking
  238. // what votes this peer has received.
  239. // NOTE: It's important to make sure that numValidators actually matches
  240. // what the node sees as the number of validators for height.
  241. func (ps *PeerState) EnsureVoteBitArrays(height int64, numValidators int) {
  242. ps.mtx.Lock()
  243. defer ps.mtx.Unlock()
  244. ps.ensureVoteBitArrays(height, numValidators)
  245. }
  246. func (ps *PeerState) ensureVoteBitArrays(height int64, numValidators int) {
  247. if ps.PRS.Height == height {
  248. if ps.PRS.Prevotes == nil {
  249. ps.PRS.Prevotes = bits.NewBitArray(numValidators)
  250. }
  251. if ps.PRS.Precommits == nil {
  252. ps.PRS.Precommits = bits.NewBitArray(numValidators)
  253. }
  254. if ps.PRS.CatchupCommit == nil {
  255. ps.PRS.CatchupCommit = bits.NewBitArray(numValidators)
  256. }
  257. if ps.PRS.ProposalPOL == nil {
  258. ps.PRS.ProposalPOL = bits.NewBitArray(numValidators)
  259. }
  260. } else if ps.PRS.Height == height+1 {
  261. if ps.PRS.LastCommit == nil {
  262. ps.PRS.LastCommit = bits.NewBitArray(numValidators)
  263. }
  264. }
  265. }
  266. // RecordVote increments internal votes related statistics for this peer.
  267. // It returns the total number of added votes.
  268. func (ps *PeerState) RecordVote() int {
  269. ps.mtx.Lock()
  270. defer ps.mtx.Unlock()
  271. ps.Stats.Votes++
  272. return ps.Stats.Votes
  273. }
  274. // VotesSent returns the number of blocks for which peer has been sending us
  275. // votes.
  276. func (ps *PeerState) VotesSent() int {
  277. ps.mtx.Lock()
  278. defer ps.mtx.Unlock()
  279. return ps.Stats.Votes
  280. }
  281. // RecordBlockPart increments internal block part related statistics for this peer.
  282. // It returns the total number of added block parts.
  283. func (ps *PeerState) RecordBlockPart() int {
  284. ps.mtx.Lock()
  285. defer ps.mtx.Unlock()
  286. ps.Stats.BlockParts++
  287. return ps.Stats.BlockParts
  288. }
  289. // BlockPartsSent returns the number of useful block parts the peer has sent us.
  290. func (ps *PeerState) BlockPartsSent() int {
  291. ps.mtx.Lock()
  292. defer ps.mtx.Unlock()
  293. return ps.Stats.BlockParts
  294. }
  295. // SetHasVote sets the given vote as known by the peer
  296. func (ps *PeerState) SetHasVote(vote *types.Vote) error {
  297. // sanity check
  298. if vote == nil {
  299. return ErrPeerStateSetNilVote
  300. }
  301. ps.mtx.Lock()
  302. defer ps.mtx.Unlock()
  303. return ps.setHasVote(vote.Height, vote.Round, vote.Type, vote.ValidatorIndex)
  304. }
  305. // setHasVote will return an error when the index exceeds the bitArray length
  306. func (ps *PeerState) setHasVote(height int64, round int32, voteType tmproto.SignedMsgType, index int32) error {
  307. logger := ps.logger.With(
  308. "peerH/R", fmt.Sprintf("%d/%d", ps.PRS.Height, ps.PRS.Round),
  309. "H/R", fmt.Sprintf("%d/%d", height, round),
  310. )
  311. logger.Debug("setHasVote", "type", voteType, "index", index)
  312. // NOTE: some may be nil BitArrays -> no side effects
  313. psVotes := ps.getVoteBitArray(height, round, voteType)
  314. if psVotes != nil {
  315. if ok := psVotes.SetIndex(int(index), true); !ok {
  316. // https://github.com/tendermint/tendermint/issues/2871
  317. return ErrPeerStateInvalidVoteIndex
  318. }
  319. }
  320. return nil
  321. }
  322. // ApplyNewRoundStepMessage updates the peer state for the new round.
  323. func (ps *PeerState) ApplyNewRoundStepMessage(msg *NewRoundStepMessage) {
  324. ps.mtx.Lock()
  325. defer ps.mtx.Unlock()
  326. // ignore duplicates or decreases
  327. if CompareHRS(msg.Height, msg.Round, msg.Step, ps.PRS.Height, ps.PRS.Round, ps.PRS.Step) <= 0 {
  328. return
  329. }
  330. var (
  331. psHeight = ps.PRS.Height
  332. psRound = ps.PRS.Round
  333. psCatchupCommitRound = ps.PRS.CatchupCommitRound
  334. psCatchupCommit = ps.PRS.CatchupCommit
  335. startTime = tmtime.Now().Add(-1 * time.Duration(msg.SecondsSinceStartTime) * time.Second)
  336. )
  337. ps.PRS.Height = msg.Height
  338. ps.PRS.Round = msg.Round
  339. ps.PRS.Step = msg.Step
  340. ps.PRS.StartTime = startTime
  341. if psHeight != msg.Height || psRound != msg.Round {
  342. ps.PRS.Proposal = false
  343. ps.PRS.ProposalBlockPartSetHeader = types.PartSetHeader{}
  344. ps.PRS.ProposalBlockParts = nil
  345. ps.PRS.ProposalPOLRound = -1
  346. ps.PRS.ProposalPOL = nil
  347. // we'll update the BitArray capacity later
  348. ps.PRS.Prevotes = nil
  349. ps.PRS.Precommits = nil
  350. }
  351. if psHeight == msg.Height && psRound != msg.Round && msg.Round == psCatchupCommitRound {
  352. // Peer caught up to CatchupCommitRound.
  353. // Preserve psCatchupCommit!
  354. // NOTE: We prefer to use prs.Precommits if
  355. // pr.Round matches pr.CatchupCommitRound.
  356. ps.PRS.Precommits = psCatchupCommit
  357. }
  358. if psHeight != msg.Height {
  359. // shift Precommits to LastCommit
  360. if psHeight+1 == msg.Height && psRound == msg.LastCommitRound {
  361. ps.PRS.LastCommitRound = msg.LastCommitRound
  362. ps.PRS.LastCommit = ps.PRS.Precommits
  363. } else {
  364. ps.PRS.LastCommitRound = msg.LastCommitRound
  365. ps.PRS.LastCommit = nil
  366. }
  367. // we'll update the BitArray capacity later
  368. ps.PRS.CatchupCommitRound = -1
  369. ps.PRS.CatchupCommit = nil
  370. }
  371. }
  372. // ApplyNewValidBlockMessage updates the peer state for the new valid block.
  373. func (ps *PeerState) ApplyNewValidBlockMessage(msg *NewValidBlockMessage) {
  374. ps.mtx.Lock()
  375. defer ps.mtx.Unlock()
  376. if ps.PRS.Height != msg.Height {
  377. return
  378. }
  379. if ps.PRS.Round != msg.Round && !msg.IsCommit {
  380. return
  381. }
  382. ps.PRS.ProposalBlockPartSetHeader = msg.BlockPartSetHeader
  383. ps.PRS.ProposalBlockParts = msg.BlockParts
  384. }
  385. // ApplyProposalPOLMessage updates the peer state for the new proposal POL.
  386. func (ps *PeerState) ApplyProposalPOLMessage(msg *ProposalPOLMessage) {
  387. ps.mtx.Lock()
  388. defer ps.mtx.Unlock()
  389. if ps.PRS.Height != msg.Height {
  390. return
  391. }
  392. if ps.PRS.ProposalPOLRound != msg.ProposalPOLRound {
  393. return
  394. }
  395. // TODO: Merge onto existing ps.PRS.ProposalPOL?
  396. // We might have sent some prevotes in the meantime.
  397. ps.PRS.ProposalPOL = msg.ProposalPOL
  398. }
  399. // ApplyHasVoteMessage updates the peer state for the new vote.
  400. func (ps *PeerState) ApplyHasVoteMessage(msg *HasVoteMessage) error {
  401. ps.mtx.Lock()
  402. defer ps.mtx.Unlock()
  403. if ps.PRS.Height != msg.Height {
  404. return nil
  405. }
  406. return ps.setHasVote(msg.Height, msg.Round, msg.Type, msg.Index)
  407. }
  408. // ApplyVoteSetBitsMessage updates the peer state for the bit-array of votes
  409. // it claims to have for the corresponding BlockID.
  410. // `ourVotes` is a BitArray of votes we have for msg.BlockID
  411. // NOTE: if ourVotes is nil (e.g. msg.Height < rs.Height),
  412. // we conservatively overwrite ps's votes w/ msg.Votes.
  413. func (ps *PeerState) ApplyVoteSetBitsMessage(msg *VoteSetBitsMessage, ourVotes *bits.BitArray) {
  414. ps.mtx.Lock()
  415. defer ps.mtx.Unlock()
  416. votes := ps.getVoteBitArray(msg.Height, msg.Round, msg.Type)
  417. if votes != nil {
  418. if ourVotes == nil {
  419. votes.Update(msg.Votes)
  420. } else {
  421. otherVotes := votes.Sub(ourVotes)
  422. hasVotes := otherVotes.Or(msg.Votes)
  423. votes.Update(hasVotes)
  424. }
  425. }
  426. }
  427. // String returns a string representation of the PeerState
  428. func (ps *PeerState) String() string {
  429. return ps.StringIndented("")
  430. }
  431. // StringIndented returns a string representation of the PeerState
  432. func (ps *PeerState) StringIndented(indent string) string {
  433. ps.mtx.Lock()
  434. defer ps.mtx.Unlock()
  435. return fmt.Sprintf(`PeerState{
  436. %s Key %v
  437. %s RoundState %v
  438. %s Stats %v
  439. %s}`,
  440. indent, ps.peerID,
  441. indent, ps.PRS.StringIndented(indent+" "),
  442. indent, ps.Stats,
  443. indent,
  444. )
  445. }