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.

528 lines
14 KiB

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