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.

1720 lines
50 KiB

  1. package consensus
  2. import (
  3. "errors"
  4. "fmt"
  5. "reflect"
  6. "sync"
  7. "time"
  8. "github.com/gogo/protobuf/proto"
  9. cstypes "github.com/tendermint/tendermint/consensus/types"
  10. "github.com/tendermint/tendermint/libs/bits"
  11. tmevents "github.com/tendermint/tendermint/libs/events"
  12. tmjson "github.com/tendermint/tendermint/libs/json"
  13. "github.com/tendermint/tendermint/libs/log"
  14. tmsync "github.com/tendermint/tendermint/libs/sync"
  15. "github.com/tendermint/tendermint/p2p"
  16. tmcons "github.com/tendermint/tendermint/proto/tendermint/consensus"
  17. tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
  18. sm "github.com/tendermint/tendermint/state"
  19. "github.com/tendermint/tendermint/types"
  20. tmtime "github.com/tendermint/tendermint/types/time"
  21. )
  22. const (
  23. StateChannel = byte(0x20)
  24. DataChannel = byte(0x21)
  25. VoteChannel = byte(0x22)
  26. VoteSetBitsChannel = byte(0x23)
  27. maxMsgSize = 1048576 // 1MB; NOTE/TODO: keep in sync with types.PartSet sizes.
  28. blocksToContributeToBecomeGoodPeer = 10000
  29. votesToContributeToBecomeGoodPeer = 10000
  30. )
  31. //-----------------------------------------------------------------------------
  32. // Reactor defines a reactor for the consensus service.
  33. type Reactor struct {
  34. p2p.BaseReactor // BaseService + p2p.Switch
  35. conS *State
  36. mtx tmsync.RWMutex
  37. waitSync bool
  38. eventBus *types.EventBus
  39. Metrics *Metrics
  40. }
  41. type ReactorOption func(*Reactor)
  42. // NewReactor returns a new Reactor with the given
  43. // consensusState.
  44. func NewReactor(consensusState *State, waitSync bool, options ...ReactorOption) *Reactor {
  45. conR := &Reactor{
  46. conS: consensusState,
  47. waitSync: waitSync,
  48. Metrics: NopMetrics(),
  49. }
  50. conR.BaseReactor = *p2p.NewBaseReactor("Consensus", conR)
  51. for _, option := range options {
  52. option(conR)
  53. }
  54. return conR
  55. }
  56. // OnStart implements BaseService by subscribing to events, which later will be
  57. // broadcasted to other peers and starting state if we're not in fast sync.
  58. func (conR *Reactor) OnStart() error {
  59. conR.Logger.Info("Reactor ", "waitSync", conR.WaitSync())
  60. // start routine that computes peer statistics for evaluating peer quality
  61. go conR.peerStatsRoutine()
  62. conR.subscribeToBroadcastEvents()
  63. if !conR.WaitSync() {
  64. conR.conS.SetSwitch(conR.Switch)
  65. err := conR.conS.Start()
  66. if err != nil {
  67. return err
  68. }
  69. }
  70. return nil
  71. }
  72. // OnStop implements BaseService by unsubscribing from events and stopping
  73. // state.
  74. func (conR *Reactor) OnStop() {
  75. conR.unsubscribeFromBroadcastEvents()
  76. if err := conR.conS.Stop(); err != nil {
  77. conR.Logger.Error("Error stopping consensus state", "err", err)
  78. }
  79. if !conR.WaitSync() {
  80. conR.conS.Wait()
  81. }
  82. }
  83. // SwitchToConsensus switches from fast_sync mode to consensus mode.
  84. // It resets the state, turns off fast_sync, and starts the consensus state-machine
  85. func (conR *Reactor) SwitchToConsensus(state sm.State, skipWAL bool) {
  86. conR.Logger.Info("SwitchToConsensus")
  87. // We have no votes, so reconstruct LastCommit from SeenCommit.
  88. if state.LastBlockHeight > 0 {
  89. conR.conS.reconstructLastCommit(state)
  90. }
  91. // NOTE: The line below causes broadcastNewRoundStepRoutine() to broadcast a
  92. // NewRoundStepMessage.
  93. conR.conS.updateToState(state)
  94. conR.mtx.Lock()
  95. conR.waitSync = false
  96. conR.mtx.Unlock()
  97. conR.Metrics.FastSyncing.Set(0)
  98. conR.Metrics.StateSyncing.Set(0)
  99. if skipWAL {
  100. conR.conS.doWALCatchup = false
  101. }
  102. conR.conS.SetSwitch(conR.Switch)
  103. err := conR.conS.Start()
  104. if err != nil {
  105. panic(fmt.Sprintf(`Failed to start consensus state: %v
  106. conS:
  107. %+v
  108. conR:
  109. %+v`, err, conR.conS, conR))
  110. }
  111. }
  112. // GetChannels implements Reactor
  113. func (conR *Reactor) GetChannels() []*p2p.ChannelDescriptor {
  114. // TODO optimize
  115. return []*p2p.ChannelDescriptor{
  116. {
  117. ID: StateChannel,
  118. Priority: 5,
  119. SendQueueCapacity: 100,
  120. RecvMessageCapacity: maxMsgSize,
  121. },
  122. {
  123. ID: DataChannel, // maybe split between gossiping current block and catchup stuff
  124. // once we gossip the whole block there's nothing left to send until next height or round
  125. Priority: 10,
  126. SendQueueCapacity: 100,
  127. RecvBufferCapacity: 50 * 4096,
  128. RecvMessageCapacity: maxMsgSize,
  129. },
  130. {
  131. ID: VoteChannel,
  132. Priority: 5,
  133. SendQueueCapacity: 100,
  134. RecvBufferCapacity: 100 * 100,
  135. RecvMessageCapacity: maxMsgSize,
  136. },
  137. {
  138. ID: VoteSetBitsChannel,
  139. Priority: 1,
  140. SendQueueCapacity: 2,
  141. RecvBufferCapacity: 1024,
  142. RecvMessageCapacity: maxMsgSize,
  143. },
  144. }
  145. }
  146. // InitPeer implements Reactor by creating a state for the peer.
  147. func (conR *Reactor) InitPeer(peer p2p.Peer) p2p.Peer {
  148. peerState := NewPeerState(peer).SetLogger(conR.Logger)
  149. peer.Set(types.PeerStateKey, peerState)
  150. return peer
  151. }
  152. // AddPeer implements Reactor by spawning multiple gossiping goroutines for the
  153. // peer.
  154. func (conR *Reactor) AddPeer(peer p2p.Peer) {
  155. if !conR.IsRunning() {
  156. return
  157. }
  158. peerState, ok := peer.Get(types.PeerStateKey).(*PeerState)
  159. if !ok {
  160. panic(fmt.Sprintf("peer %v has no state", peer))
  161. }
  162. // Begin routines for this peer.
  163. go conR.gossipDataRoutine(peer, peerState)
  164. go conR.gossipVotesRoutine(peer, peerState)
  165. go conR.queryMaj23Routine(peer, peerState)
  166. // Send our state to peer.
  167. // If we're fast_syncing, broadcast a RoundStepMessage later upon SwitchToConsensus().
  168. if !conR.WaitSync() {
  169. conR.sendNewRoundStepMessage(peer)
  170. }
  171. }
  172. // RemovePeer is a noop.
  173. func (conR *Reactor) RemovePeer(peer p2p.Peer, reason interface{}) {
  174. if !conR.IsRunning() {
  175. return
  176. }
  177. // TODO
  178. // ps, ok := peer.Get(PeerStateKey).(*PeerState)
  179. // if !ok {
  180. // panic(fmt.Sprintf("Peer %v has no state", peer))
  181. // }
  182. // ps.Disconnect()
  183. }
  184. // Receive implements Reactor
  185. // NOTE: We process these messages even when we're fast_syncing.
  186. // Messages affect either a peer state or the consensus state.
  187. // Peer state updates can happen in parallel, but processing of
  188. // proposals, block parts, and votes are ordered by the receiveRoutine
  189. // NOTE: blocks on consensus state for proposals, block parts, and votes
  190. func (conR *Reactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
  191. if !conR.IsRunning() {
  192. conR.Logger.Debug("Receive", "src", src, "chId", chID, "bytes", msgBytes)
  193. return
  194. }
  195. msg, err := decodeMsg(msgBytes)
  196. if err != nil {
  197. conR.Logger.Error("Error decoding message", "src", src, "chId", chID, "err", err)
  198. conR.Switch.StopPeerForError(src, err)
  199. return
  200. }
  201. if err = msg.ValidateBasic(); err != nil {
  202. conR.Logger.Error("Peer sent us invalid msg", "peer", src, "msg", msg, "err", err)
  203. conR.Switch.StopPeerForError(src, err)
  204. return
  205. }
  206. conR.Logger.Debug("Receive", "src", src, "chId", chID, "msg", msg)
  207. // Get peer states
  208. ps, ok := src.Get(types.PeerStateKey).(*PeerState)
  209. if !ok {
  210. panic(fmt.Sprintf("Peer %v has no state", src))
  211. }
  212. switch chID {
  213. case StateChannel:
  214. switch msg := msg.(type) {
  215. case *NewRoundStepMessage:
  216. conR.conS.mtx.Lock()
  217. initialHeight := conR.conS.state.InitialHeight
  218. conR.conS.mtx.Unlock()
  219. if err = msg.ValidateHeight(initialHeight); err != nil {
  220. conR.Logger.Error("Peer sent us invalid msg", "peer", src, "msg", msg, "err", err)
  221. conR.Switch.StopPeerForError(src, err)
  222. return
  223. }
  224. ps.ApplyNewRoundStepMessage(msg)
  225. case *NewValidBlockMessage:
  226. ps.ApplyNewValidBlockMessage(msg)
  227. case *HasVoteMessage:
  228. ps.ApplyHasVoteMessage(msg)
  229. case *VoteSetMaj23Message:
  230. cs := conR.conS
  231. cs.mtx.Lock()
  232. height, votes := cs.Height, cs.Votes
  233. cs.mtx.Unlock()
  234. if height != msg.Height {
  235. return
  236. }
  237. // Peer claims to have a maj23 for some BlockID at H,R,S,
  238. err := votes.SetPeerMaj23(msg.Round, msg.Type, ps.peer.ID(), msg.BlockID)
  239. if err != nil {
  240. conR.Switch.StopPeerForError(src, err)
  241. return
  242. }
  243. // Respond with a VoteSetBitsMessage showing which votes we have.
  244. // (and consequently shows which we don't have)
  245. var ourVotes *bits.BitArray
  246. switch msg.Type {
  247. case tmproto.PrevoteType:
  248. ourVotes = votes.Prevotes(msg.Round).BitArrayByBlockID(msg.BlockID)
  249. case tmproto.PrecommitType:
  250. ourVotes = votes.Precommits(msg.Round).BitArrayByBlockID(msg.BlockID)
  251. default:
  252. panic("Bad VoteSetBitsMessage field Type. Forgot to add a check in ValidateBasic?")
  253. }
  254. src.TrySend(VoteSetBitsChannel, MustEncode(&VoteSetBitsMessage{
  255. Height: msg.Height,
  256. Round: msg.Round,
  257. Type: msg.Type,
  258. BlockID: msg.BlockID,
  259. Votes: ourVotes,
  260. }))
  261. default:
  262. conR.Logger.Error(fmt.Sprintf("Unknown message type %v", reflect.TypeOf(msg)))
  263. }
  264. case DataChannel:
  265. if conR.WaitSync() {
  266. conR.Logger.Info("Ignoring message received during sync", "msg", msg)
  267. return
  268. }
  269. switch msg := msg.(type) {
  270. case *ProposalMessage:
  271. ps.SetHasProposal(msg.Proposal)
  272. conR.conS.peerMsgQueue <- msgInfo{msg, src.ID()}
  273. case *ProposalPOLMessage:
  274. ps.ApplyProposalPOLMessage(msg)
  275. case *BlockPartMessage:
  276. ps.SetHasProposalBlockPart(msg.Height, msg.Round, int(msg.Part.Index))
  277. conR.Metrics.BlockParts.With("peer_id", string(src.ID())).Add(1)
  278. conR.conS.peerMsgQueue <- msgInfo{msg, src.ID()}
  279. default:
  280. conR.Logger.Error(fmt.Sprintf("Unknown message type %v", reflect.TypeOf(msg)))
  281. }
  282. case VoteChannel:
  283. if conR.WaitSync() {
  284. conR.Logger.Info("Ignoring message received during sync", "msg", msg)
  285. return
  286. }
  287. switch msg := msg.(type) {
  288. case *VoteMessage:
  289. cs := conR.conS
  290. cs.mtx.RLock()
  291. height, valSize, lastCommitSize := cs.Height, cs.Validators.Size(), cs.LastCommit.Size()
  292. cs.mtx.RUnlock()
  293. ps.EnsureVoteBitArrays(height, valSize)
  294. ps.EnsureVoteBitArrays(height-1, lastCommitSize)
  295. ps.SetHasVote(msg.Vote)
  296. cs.peerMsgQueue <- msgInfo{msg, src.ID()}
  297. default:
  298. // don't punish (leave room for soft upgrades)
  299. conR.Logger.Error(fmt.Sprintf("Unknown message type %v", reflect.TypeOf(msg)))
  300. }
  301. case VoteSetBitsChannel:
  302. if conR.WaitSync() {
  303. conR.Logger.Info("Ignoring message received during sync", "msg", msg)
  304. return
  305. }
  306. switch msg := msg.(type) {
  307. case *VoteSetBitsMessage:
  308. cs := conR.conS
  309. cs.mtx.Lock()
  310. height, votes := cs.Height, cs.Votes
  311. cs.mtx.Unlock()
  312. if height == msg.Height {
  313. var ourVotes *bits.BitArray
  314. switch msg.Type {
  315. case tmproto.PrevoteType:
  316. ourVotes = votes.Prevotes(msg.Round).BitArrayByBlockID(msg.BlockID)
  317. case tmproto.PrecommitType:
  318. ourVotes = votes.Precommits(msg.Round).BitArrayByBlockID(msg.BlockID)
  319. default:
  320. panic("Bad VoteSetBitsMessage field Type. Forgot to add a check in ValidateBasic?")
  321. }
  322. ps.ApplyVoteSetBitsMessage(msg, ourVotes)
  323. } else {
  324. ps.ApplyVoteSetBitsMessage(msg, nil)
  325. }
  326. default:
  327. // don't punish (leave room for soft upgrades)
  328. conR.Logger.Error(fmt.Sprintf("Unknown message type %v", reflect.TypeOf(msg)))
  329. }
  330. default:
  331. conR.Logger.Error(fmt.Sprintf("Unknown chId %X", chID))
  332. }
  333. }
  334. // SetEventBus sets event bus.
  335. func (conR *Reactor) SetEventBus(b *types.EventBus) {
  336. conR.eventBus = b
  337. conR.conS.SetEventBus(b)
  338. }
  339. // WaitSync returns whether the consensus reactor is waiting for state/fast sync.
  340. func (conR *Reactor) WaitSync() bool {
  341. conR.mtx.RLock()
  342. defer conR.mtx.RUnlock()
  343. return conR.waitSync
  344. }
  345. //--------------------------------------
  346. // subscribeToBroadcastEvents subscribes for new round steps and votes
  347. // using internal pubsub defined on state to broadcast
  348. // them to peers upon receiving.
  349. func (conR *Reactor) subscribeToBroadcastEvents() {
  350. const subscriber = "consensus-reactor"
  351. if err := conR.conS.evsw.AddListenerForEvent(subscriber, types.EventNewRoundStep,
  352. func(data tmevents.EventData) {
  353. conR.broadcastNewRoundStepMessage(data.(*cstypes.RoundState))
  354. }); err != nil {
  355. conR.Logger.Error("Error adding listener for events", "err", err)
  356. }
  357. if err := conR.conS.evsw.AddListenerForEvent(subscriber, types.EventValidBlock,
  358. func(data tmevents.EventData) {
  359. conR.broadcastNewValidBlockMessage(data.(*cstypes.RoundState))
  360. }); err != nil {
  361. conR.Logger.Error("Error adding listener for events", "err", err)
  362. }
  363. if err := conR.conS.evsw.AddListenerForEvent(subscriber, types.EventVote,
  364. func(data tmevents.EventData) {
  365. conR.broadcastHasVoteMessage(data.(*types.Vote))
  366. }); err != nil {
  367. conR.Logger.Error("Error adding listener for events", "err", err)
  368. }
  369. }
  370. func (conR *Reactor) unsubscribeFromBroadcastEvents() {
  371. const subscriber = "consensus-reactor"
  372. conR.conS.evsw.RemoveListener(subscriber)
  373. }
  374. func (conR *Reactor) broadcastNewRoundStepMessage(rs *cstypes.RoundState) {
  375. nrsMsg := makeRoundStepMessage(rs)
  376. conR.Switch.Broadcast(StateChannel, MustEncode(nrsMsg))
  377. }
  378. func (conR *Reactor) broadcastNewValidBlockMessage(rs *cstypes.RoundState) {
  379. csMsg := &NewValidBlockMessage{
  380. Height: rs.Height,
  381. Round: rs.Round,
  382. BlockPartSetHeader: rs.ProposalBlockParts.Header(),
  383. BlockParts: rs.ProposalBlockParts.BitArray(),
  384. IsCommit: rs.Step == cstypes.RoundStepCommit,
  385. }
  386. conR.Switch.Broadcast(StateChannel, MustEncode(csMsg))
  387. }
  388. // Broadcasts HasVoteMessage to peers that care.
  389. func (conR *Reactor) broadcastHasVoteMessage(vote *types.Vote) {
  390. msg := &HasVoteMessage{
  391. Height: vote.Height,
  392. Round: vote.Round,
  393. Type: vote.Type,
  394. Index: vote.ValidatorIndex,
  395. }
  396. conR.Switch.Broadcast(StateChannel, MustEncode(msg))
  397. /*
  398. // TODO: Make this broadcast more selective.
  399. for _, peer := range conR.Switch.Peers().List() {
  400. ps, ok := peer.Get(PeerStateKey).(*PeerState)
  401. if !ok {
  402. panic(fmt.Sprintf("Peer %v has no state", peer))
  403. }
  404. prs := ps.GetRoundState()
  405. if prs.Height == vote.Height {
  406. // TODO: Also filter on round?
  407. peer.TrySend(StateChannel, struct{ ConsensusMessage }{msg})
  408. } else {
  409. // Height doesn't match
  410. // TODO: check a field, maybe CatchupCommitRound?
  411. // TODO: But that requires changing the struct field comment.
  412. }
  413. }
  414. */
  415. }
  416. func makeRoundStepMessage(rs *cstypes.RoundState) (nrsMsg *NewRoundStepMessage) {
  417. nrsMsg = &NewRoundStepMessage{
  418. Height: rs.Height,
  419. Round: rs.Round,
  420. Step: rs.Step,
  421. SecondsSinceStartTime: int64(time.Since(rs.StartTime).Seconds()),
  422. LastCommitRound: rs.LastCommit.GetRound(),
  423. }
  424. return
  425. }
  426. func (conR *Reactor) sendNewRoundStepMessage(peer p2p.Peer) {
  427. rs := conR.conS.GetRoundState()
  428. nrsMsg := makeRoundStepMessage(rs)
  429. peer.Send(StateChannel, MustEncode(nrsMsg))
  430. }
  431. func (conR *Reactor) gossipDataRoutine(peer p2p.Peer, ps *PeerState) {
  432. logger := conR.Logger.With("peer", peer)
  433. OUTER_LOOP:
  434. for {
  435. // Manage disconnects from self or peer.
  436. if !peer.IsRunning() || !conR.IsRunning() {
  437. logger.Info("Stopping gossipDataRoutine for peer")
  438. return
  439. }
  440. rs := conR.conS.GetRoundState()
  441. prs := ps.GetRoundState()
  442. // Send proposal Block parts?
  443. if rs.ProposalBlockParts.HasHeader(prs.ProposalBlockPartSetHeader) {
  444. if index, ok := rs.ProposalBlockParts.BitArray().Sub(prs.ProposalBlockParts.Copy()).PickRandom(); ok {
  445. part := rs.ProposalBlockParts.GetPart(index)
  446. msg := &BlockPartMessage{
  447. Height: rs.Height, // This tells peer that this part applies to us.
  448. Round: rs.Round, // This tells peer that this part applies to us.
  449. Part: part,
  450. }
  451. logger.Debug("Sending block part", "height", prs.Height, "round", prs.Round)
  452. if peer.Send(DataChannel, MustEncode(msg)) {
  453. ps.SetHasProposalBlockPart(prs.Height, prs.Round, index)
  454. }
  455. continue OUTER_LOOP
  456. }
  457. }
  458. // If the peer is on a previous height that we have, help catch up.
  459. if (0 < prs.Height) && (prs.Height < rs.Height) && (prs.Height >= conR.conS.blockStore.Base()) {
  460. heightLogger := logger.With("height", prs.Height)
  461. // if we never received the commit message from the peer, the block parts wont be initialized
  462. if prs.ProposalBlockParts == nil {
  463. blockMeta := conR.conS.blockStore.LoadBlockMeta(prs.Height)
  464. if blockMeta == nil {
  465. heightLogger.Error("Failed to load block meta",
  466. "blockstoreBase", conR.conS.blockStore.Base(), "blockstoreHeight", conR.conS.blockStore.Height())
  467. time.Sleep(conR.conS.config.PeerGossipSleepDuration)
  468. } else {
  469. ps.InitProposalBlockParts(blockMeta.BlockID.PartSetHeader)
  470. }
  471. // continue the loop since prs is a copy and not effected by this initialization
  472. continue OUTER_LOOP
  473. }
  474. conR.gossipDataForCatchup(heightLogger, rs, prs, ps, peer)
  475. continue OUTER_LOOP
  476. }
  477. // If height and round don't match, sleep.
  478. if (rs.Height != prs.Height) || (rs.Round != prs.Round) {
  479. time.Sleep(conR.conS.config.PeerGossipSleepDuration)
  480. continue OUTER_LOOP
  481. }
  482. // By here, height and round match.
  483. // Proposal block parts were already matched and sent if any were wanted.
  484. // (These can match on hash so the round doesn't matter)
  485. // Now consider sending other things, like the Proposal itself.
  486. // Send Proposal && ProposalPOL BitArray?
  487. if rs.Proposal != nil && !prs.Proposal {
  488. // Proposal: share the proposal metadata with peer.
  489. {
  490. msg := &ProposalMessage{Proposal: rs.Proposal}
  491. logger.Debug("Sending proposal", "height", prs.Height, "round", prs.Round)
  492. if peer.Send(DataChannel, MustEncode(msg)) {
  493. // NOTE[ZM]: A peer might have received different proposal msg so this Proposal msg will be rejected!
  494. ps.SetHasProposal(rs.Proposal)
  495. }
  496. }
  497. // ProposalPOL: lets peer know which POL votes we have so far.
  498. // Peer must receive ProposalMessage first.
  499. // rs.Proposal was validated, so rs.Proposal.POLRound <= rs.Round,
  500. // so we definitely have rs.Votes.Prevotes(rs.Proposal.POLRound).
  501. if 0 <= rs.Proposal.POLRound {
  502. msg := &ProposalPOLMessage{
  503. Height: rs.Height,
  504. ProposalPOLRound: rs.Proposal.POLRound,
  505. ProposalPOL: rs.Votes.Prevotes(rs.Proposal.POLRound).BitArray(),
  506. }
  507. logger.Debug("Sending POL", "height", prs.Height, "round", prs.Round)
  508. peer.Send(DataChannel, MustEncode(msg))
  509. }
  510. continue OUTER_LOOP
  511. }
  512. // Nothing to do. Sleep.
  513. time.Sleep(conR.conS.config.PeerGossipSleepDuration)
  514. continue OUTER_LOOP
  515. }
  516. }
  517. func (conR *Reactor) gossipDataForCatchup(logger log.Logger, rs *cstypes.RoundState,
  518. prs *cstypes.PeerRoundState, ps *PeerState, peer p2p.Peer) {
  519. if index, ok := prs.ProposalBlockParts.Not().PickRandom(); ok {
  520. // Ensure that the peer's PartSetHeader is correct
  521. blockMeta := conR.conS.blockStore.LoadBlockMeta(prs.Height)
  522. if blockMeta == nil {
  523. logger.Error("Failed to load block meta", "ourHeight", rs.Height,
  524. "blockstoreBase", conR.conS.blockStore.Base(), "blockstoreHeight", conR.conS.blockStore.Height())
  525. time.Sleep(conR.conS.config.PeerGossipSleepDuration)
  526. return
  527. } else if !blockMeta.BlockID.PartSetHeader.Equals(prs.ProposalBlockPartSetHeader) {
  528. logger.Info("Peer ProposalBlockPartSetHeader mismatch, sleeping",
  529. "blockPartSetHeader", blockMeta.BlockID.PartSetHeader, "peerBlockPartSetHeader", prs.ProposalBlockPartSetHeader)
  530. time.Sleep(conR.conS.config.PeerGossipSleepDuration)
  531. return
  532. }
  533. // Load the part
  534. part := conR.conS.blockStore.LoadBlockPart(prs.Height, index)
  535. if part == nil {
  536. logger.Error("Could not load part", "index", index,
  537. "blockPartSetHeader", blockMeta.BlockID.PartSetHeader, "peerBlockPartSetHeader", prs.ProposalBlockPartSetHeader)
  538. time.Sleep(conR.conS.config.PeerGossipSleepDuration)
  539. return
  540. }
  541. // Send the part
  542. msg := &BlockPartMessage{
  543. Height: prs.Height, // Not our height, so it doesn't matter.
  544. Round: prs.Round, // Not our height, so it doesn't matter.
  545. Part: part,
  546. }
  547. logger.Debug("Sending block part for catchup", "round", prs.Round, "index", index)
  548. if peer.Send(DataChannel, MustEncode(msg)) {
  549. ps.SetHasProposalBlockPart(prs.Height, prs.Round, index)
  550. } else {
  551. logger.Debug("Sending block part for catchup failed")
  552. }
  553. return
  554. }
  555. time.Sleep(conR.conS.config.PeerGossipSleepDuration)
  556. }
  557. func (conR *Reactor) gossipVotesRoutine(peer p2p.Peer, ps *PeerState) {
  558. logger := conR.Logger.With("peer", peer)
  559. // Simple hack to throttle logs upon sleep.
  560. var sleeping = 0
  561. OUTER_LOOP:
  562. for {
  563. // Manage disconnects from self or peer.
  564. if !peer.IsRunning() || !conR.IsRunning() {
  565. logger.Info("Stopping gossipVotesRoutine for peer")
  566. return
  567. }
  568. rs := conR.conS.GetRoundState()
  569. prs := ps.GetRoundState()
  570. switch sleeping {
  571. case 1: // First sleep
  572. sleeping = 2
  573. case 2: // No more sleep
  574. sleeping = 0
  575. }
  576. // If height matches, then send LastCommit, Prevotes, Precommits.
  577. if rs.Height == prs.Height {
  578. heightLogger := logger.With("height", prs.Height)
  579. if conR.gossipVotesForHeight(heightLogger, rs, prs, ps) {
  580. continue OUTER_LOOP
  581. }
  582. }
  583. // Special catchup logic.
  584. // If peer is lagging by height 1, send LastCommit.
  585. if prs.Height != 0 && rs.Height == prs.Height+1 {
  586. if ps.PickSendVote(rs.LastCommit) {
  587. logger.Debug("Picked rs.LastCommit to send", "height", prs.Height)
  588. continue OUTER_LOOP
  589. }
  590. }
  591. // Catchup logic
  592. // If peer is lagging by more than 1, send Commit.
  593. if prs.Height != 0 && rs.Height >= prs.Height+2 && prs.Height >= conR.conS.blockStore.Base() {
  594. // Load the block commit for prs.Height,
  595. // which contains precommit signatures for prs.Height.
  596. if commit := conR.conS.blockStore.LoadBlockCommit(prs.Height); commit != nil {
  597. if ps.PickSendVote(commit) {
  598. logger.Debug("Picked Catchup commit to send", "height", prs.Height)
  599. continue OUTER_LOOP
  600. }
  601. }
  602. }
  603. if sleeping == 0 {
  604. // We sent nothing. Sleep...
  605. sleeping = 1
  606. logger.Debug("No votes to send, sleeping", "rs.Height", rs.Height, "prs.Height", prs.Height,
  607. "localPV", rs.Votes.Prevotes(rs.Round).BitArray(), "peerPV", prs.Prevotes,
  608. "localPC", rs.Votes.Precommits(rs.Round).BitArray(), "peerPC", prs.Precommits)
  609. } else if sleeping == 2 {
  610. // Continued sleep...
  611. sleeping = 1
  612. }
  613. time.Sleep(conR.conS.config.PeerGossipSleepDuration)
  614. continue OUTER_LOOP
  615. }
  616. }
  617. func (conR *Reactor) gossipVotesForHeight(
  618. logger log.Logger,
  619. rs *cstypes.RoundState,
  620. prs *cstypes.PeerRoundState,
  621. ps *PeerState,
  622. ) bool {
  623. // If there are lastCommits to send...
  624. if prs.Step == cstypes.RoundStepNewHeight {
  625. if ps.PickSendVote(rs.LastCommit) {
  626. logger.Debug("Picked rs.LastCommit to send")
  627. return true
  628. }
  629. }
  630. // If there are POL prevotes to send...
  631. if prs.Step <= cstypes.RoundStepPropose && prs.Round != -1 && prs.Round <= rs.Round && prs.ProposalPOLRound != -1 {
  632. if polPrevotes := rs.Votes.Prevotes(prs.ProposalPOLRound); polPrevotes != nil {
  633. if ps.PickSendVote(polPrevotes) {
  634. logger.Debug("Picked rs.Prevotes(prs.ProposalPOLRound) to send",
  635. "round", prs.ProposalPOLRound)
  636. return true
  637. }
  638. }
  639. }
  640. // If there are prevotes to send...
  641. if prs.Step <= cstypes.RoundStepPrevoteWait && prs.Round != -1 && prs.Round <= rs.Round {
  642. if ps.PickSendVote(rs.Votes.Prevotes(prs.Round)) {
  643. logger.Debug("Picked rs.Prevotes(prs.Round) to send", "round", prs.Round)
  644. return true
  645. }
  646. }
  647. // If there are precommits to send...
  648. if prs.Step <= cstypes.RoundStepPrecommitWait && prs.Round != -1 && prs.Round <= rs.Round {
  649. if ps.PickSendVote(rs.Votes.Precommits(prs.Round)) {
  650. logger.Debug("Picked rs.Precommits(prs.Round) to send", "round", prs.Round)
  651. return true
  652. }
  653. }
  654. // If there are prevotes to send...Needed because of validBlock mechanism
  655. if prs.Round != -1 && prs.Round <= rs.Round {
  656. if ps.PickSendVote(rs.Votes.Prevotes(prs.Round)) {
  657. logger.Debug("Picked rs.Prevotes(prs.Round) to send", "round", prs.Round)
  658. return true
  659. }
  660. }
  661. // If there are POLPrevotes to send...
  662. if prs.ProposalPOLRound != -1 {
  663. if polPrevotes := rs.Votes.Prevotes(prs.ProposalPOLRound); polPrevotes != nil {
  664. if ps.PickSendVote(polPrevotes) {
  665. logger.Debug("Picked rs.Prevotes(prs.ProposalPOLRound) to send",
  666. "round", prs.ProposalPOLRound)
  667. return true
  668. }
  669. }
  670. }
  671. return false
  672. }
  673. // NOTE: `queryMaj23Routine` has a simple crude design since it only comes
  674. // into play for liveness when there's a signature DDoS attack happening.
  675. func (conR *Reactor) queryMaj23Routine(peer p2p.Peer, ps *PeerState) {
  676. logger := conR.Logger.With("peer", peer)
  677. OUTER_LOOP:
  678. for {
  679. // Manage disconnects from self or peer.
  680. if !peer.IsRunning() || !conR.IsRunning() {
  681. logger.Info("Stopping queryMaj23Routine for peer")
  682. return
  683. }
  684. // Maybe send Height/Round/Prevotes
  685. {
  686. rs := conR.conS.GetRoundState()
  687. prs := ps.GetRoundState()
  688. if rs.Height == prs.Height {
  689. if maj23, ok := rs.Votes.Prevotes(prs.Round).TwoThirdsMajority(); ok {
  690. peer.TrySend(StateChannel, MustEncode(&VoteSetMaj23Message{
  691. Height: prs.Height,
  692. Round: prs.Round,
  693. Type: tmproto.PrevoteType,
  694. BlockID: maj23,
  695. }))
  696. time.Sleep(conR.conS.config.PeerQueryMaj23SleepDuration)
  697. }
  698. }
  699. }
  700. // Maybe send Height/Round/Precommits
  701. {
  702. rs := conR.conS.GetRoundState()
  703. prs := ps.GetRoundState()
  704. if rs.Height == prs.Height {
  705. if maj23, ok := rs.Votes.Precommits(prs.Round).TwoThirdsMajority(); ok {
  706. peer.TrySend(StateChannel, MustEncode(&VoteSetMaj23Message{
  707. Height: prs.Height,
  708. Round: prs.Round,
  709. Type: tmproto.PrecommitType,
  710. BlockID: maj23,
  711. }))
  712. time.Sleep(conR.conS.config.PeerQueryMaj23SleepDuration)
  713. }
  714. }
  715. }
  716. // Maybe send Height/Round/ProposalPOL
  717. {
  718. rs := conR.conS.GetRoundState()
  719. prs := ps.GetRoundState()
  720. if rs.Height == prs.Height && prs.ProposalPOLRound >= 0 {
  721. if maj23, ok := rs.Votes.Prevotes(prs.ProposalPOLRound).TwoThirdsMajority(); ok {
  722. peer.TrySend(StateChannel, MustEncode(&VoteSetMaj23Message{
  723. Height: prs.Height,
  724. Round: prs.ProposalPOLRound,
  725. Type: tmproto.PrevoteType,
  726. BlockID: maj23,
  727. }))
  728. time.Sleep(conR.conS.config.PeerQueryMaj23SleepDuration)
  729. }
  730. }
  731. }
  732. // Little point sending LastCommitRound/LastCommit,
  733. // These are fleeting and non-blocking.
  734. // Maybe send Height/CatchupCommitRound/CatchupCommit.
  735. {
  736. prs := ps.GetRoundState()
  737. if prs.CatchupCommitRound != -1 && prs.Height > 0 && prs.Height <= conR.conS.blockStore.Height() &&
  738. prs.Height >= conR.conS.blockStore.Base() {
  739. if commit := conR.conS.LoadCommit(prs.Height); commit != nil {
  740. peer.TrySend(StateChannel, MustEncode(&VoteSetMaj23Message{
  741. Height: prs.Height,
  742. Round: commit.Round,
  743. Type: tmproto.PrecommitType,
  744. BlockID: commit.BlockID,
  745. }))
  746. time.Sleep(conR.conS.config.PeerQueryMaj23SleepDuration)
  747. }
  748. }
  749. }
  750. time.Sleep(conR.conS.config.PeerQueryMaj23SleepDuration)
  751. continue OUTER_LOOP
  752. }
  753. }
  754. func (conR *Reactor) peerStatsRoutine() {
  755. for {
  756. if !conR.IsRunning() {
  757. conR.Logger.Info("Stopping peerStatsRoutine")
  758. return
  759. }
  760. select {
  761. case msg := <-conR.conS.statsMsgQueue:
  762. // Get peer
  763. peer := conR.Switch.Peers().Get(msg.PeerID)
  764. if peer == nil {
  765. conR.Logger.Debug("Attempt to update stats for non-existent peer",
  766. "peer", msg.PeerID)
  767. continue
  768. }
  769. // Get peer state
  770. ps, ok := peer.Get(types.PeerStateKey).(*PeerState)
  771. if !ok {
  772. panic(fmt.Sprintf("Peer %v has no state", peer))
  773. }
  774. switch msg.Msg.(type) {
  775. case *VoteMessage:
  776. if numVotes := ps.RecordVote(); numVotes%votesToContributeToBecomeGoodPeer == 0 {
  777. conR.Switch.MarkPeerAsGood(peer)
  778. }
  779. case *BlockPartMessage:
  780. if numParts := ps.RecordBlockPart(); numParts%blocksToContributeToBecomeGoodPeer == 0 {
  781. conR.Switch.MarkPeerAsGood(peer)
  782. }
  783. }
  784. case <-conR.conS.Quit():
  785. return
  786. case <-conR.Quit():
  787. return
  788. }
  789. }
  790. }
  791. // String returns a string representation of the Reactor.
  792. // NOTE: For now, it is just a hard-coded string to avoid accessing unprotected shared variables.
  793. // TODO: improve!
  794. func (conR *Reactor) String() string {
  795. // better not to access shared variables
  796. return "ConsensusReactor" // conR.StringIndented("")
  797. }
  798. // StringIndented returns an indented string representation of the Reactor
  799. func (conR *Reactor) StringIndented(indent string) string {
  800. s := "ConsensusReactor{\n"
  801. s += indent + " " + conR.conS.StringIndented(indent+" ") + "\n"
  802. for _, peer := range conR.Switch.Peers().List() {
  803. ps, ok := peer.Get(types.PeerStateKey).(*PeerState)
  804. if !ok {
  805. panic(fmt.Sprintf("Peer %v has no state", peer))
  806. }
  807. s += indent + " " + ps.StringIndented(indent+" ") + "\n"
  808. }
  809. s += indent + "}"
  810. return s
  811. }
  812. // ReactorMetrics sets the metrics
  813. func ReactorMetrics(metrics *Metrics) ReactorOption {
  814. return func(conR *Reactor) { conR.Metrics = metrics }
  815. }
  816. //-----------------------------------------------------------------------------
  817. var (
  818. ErrPeerStateHeightRegression = errors.New("error peer state height regression")
  819. ErrPeerStateInvalidStartTime = errors.New("error peer state invalid startTime")
  820. )
  821. // PeerState contains the known state of a peer, including its connection and
  822. // threadsafe access to its PeerRoundState.
  823. // NOTE: THIS GETS DUMPED WITH rpc/core/consensus.go.
  824. // Be mindful of what you Expose.
  825. type PeerState struct {
  826. peer p2p.Peer
  827. logger log.Logger
  828. mtx sync.Mutex // NOTE: Modify below using setters, never directly.
  829. PRS cstypes.PeerRoundState `json:"round_state"` // Exposed.
  830. Stats *peerStateStats `json:"stats"` // Exposed.
  831. }
  832. // peerStateStats holds internal statistics for a peer.
  833. type peerStateStats struct {
  834. Votes int `json:"votes"`
  835. BlockParts int `json:"block_parts"`
  836. }
  837. func (pss peerStateStats) String() string {
  838. return fmt.Sprintf("peerStateStats{votes: %d, blockParts: %d}",
  839. pss.Votes, pss.BlockParts)
  840. }
  841. // NewPeerState returns a new PeerState for the given Peer
  842. func NewPeerState(peer p2p.Peer) *PeerState {
  843. return &PeerState{
  844. peer: peer,
  845. logger: log.NewNopLogger(),
  846. PRS: cstypes.PeerRoundState{
  847. Round: -1,
  848. ProposalPOLRound: -1,
  849. LastCommitRound: -1,
  850. CatchupCommitRound: -1,
  851. },
  852. Stats: &peerStateStats{},
  853. }
  854. }
  855. // SetLogger allows to set a logger on the peer state. Returns the peer state
  856. // itself.
  857. func (ps *PeerState) SetLogger(logger log.Logger) *PeerState {
  858. ps.logger = logger
  859. return ps
  860. }
  861. // GetRoundState returns an shallow copy of the PeerRoundState.
  862. // There's no point in mutating it since it won't change PeerState.
  863. func (ps *PeerState) GetRoundState() *cstypes.PeerRoundState {
  864. ps.mtx.Lock()
  865. defer ps.mtx.Unlock()
  866. prs := ps.PRS // copy
  867. return &prs
  868. }
  869. // ToJSON returns a json of PeerState.
  870. func (ps *PeerState) ToJSON() ([]byte, error) {
  871. ps.mtx.Lock()
  872. defer ps.mtx.Unlock()
  873. return tmjson.Marshal(ps)
  874. }
  875. // GetHeight returns an atomic snapshot of the PeerRoundState's height
  876. // used by the mempool to ensure peers are caught up before broadcasting new txs
  877. func (ps *PeerState) GetHeight() int64 {
  878. ps.mtx.Lock()
  879. defer ps.mtx.Unlock()
  880. return ps.PRS.Height
  881. }
  882. // SetHasProposal sets the given proposal as known for the peer.
  883. func (ps *PeerState) SetHasProposal(proposal *types.Proposal) {
  884. ps.mtx.Lock()
  885. defer ps.mtx.Unlock()
  886. if ps.PRS.Height != proposal.Height || ps.PRS.Round != proposal.Round {
  887. return
  888. }
  889. if ps.PRS.Proposal {
  890. return
  891. }
  892. ps.PRS.Proposal = true
  893. // ps.PRS.ProposalBlockParts is set due to NewValidBlockMessage
  894. if ps.PRS.ProposalBlockParts != nil {
  895. return
  896. }
  897. ps.PRS.ProposalBlockPartSetHeader = proposal.BlockID.PartSetHeader
  898. ps.PRS.ProposalBlockParts = bits.NewBitArray(int(proposal.BlockID.PartSetHeader.Total))
  899. ps.PRS.ProposalPOLRound = proposal.POLRound
  900. ps.PRS.ProposalPOL = nil // Nil until ProposalPOLMessage received.
  901. }
  902. // InitProposalBlockParts initializes the peer's proposal block parts header and bit array.
  903. func (ps *PeerState) InitProposalBlockParts(partSetHeader types.PartSetHeader) {
  904. ps.mtx.Lock()
  905. defer ps.mtx.Unlock()
  906. if ps.PRS.ProposalBlockParts != nil {
  907. return
  908. }
  909. ps.PRS.ProposalBlockPartSetHeader = partSetHeader
  910. ps.PRS.ProposalBlockParts = bits.NewBitArray(int(partSetHeader.Total))
  911. }
  912. // SetHasProposalBlockPart sets the given block part index as known for the peer.
  913. func (ps *PeerState) SetHasProposalBlockPart(height int64, round int32, index int) {
  914. ps.mtx.Lock()
  915. defer ps.mtx.Unlock()
  916. if ps.PRS.Height != height || ps.PRS.Round != round {
  917. return
  918. }
  919. ps.PRS.ProposalBlockParts.SetIndex(index, true)
  920. }
  921. // PickSendVote picks a vote and sends it to the peer.
  922. // Returns true if vote was sent.
  923. func (ps *PeerState) PickSendVote(votes types.VoteSetReader) bool {
  924. if vote, ok := ps.PickVoteToSend(votes); ok {
  925. msg := &VoteMessage{vote}
  926. ps.logger.Debug("Sending vote message", "ps", ps, "vote", vote)
  927. if ps.peer.Send(VoteChannel, MustEncode(msg)) {
  928. ps.SetHasVote(vote)
  929. return true
  930. }
  931. return false
  932. }
  933. return false
  934. }
  935. // PickVoteToSend picks a vote to send to the peer.
  936. // Returns true if a vote was picked.
  937. // NOTE: `votes` must be the correct Size() for the Height().
  938. func (ps *PeerState) PickVoteToSend(votes types.VoteSetReader) (vote *types.Vote, ok bool) {
  939. ps.mtx.Lock()
  940. defer ps.mtx.Unlock()
  941. if votes.Size() == 0 {
  942. return nil, false
  943. }
  944. height, round, votesType, size :=
  945. votes.GetHeight(), votes.GetRound(), tmproto.SignedMsgType(votes.Type()), votes.Size()
  946. // Lazily set data using 'votes'.
  947. if votes.IsCommit() {
  948. ps.ensureCatchupCommitRound(height, round, size)
  949. }
  950. ps.ensureVoteBitArrays(height, size)
  951. psVotes := ps.getVoteBitArray(height, round, votesType)
  952. if psVotes == nil {
  953. return nil, false // Not something worth sending
  954. }
  955. if index, ok := votes.BitArray().Sub(psVotes).PickRandom(); ok {
  956. return votes.GetByIndex(int32(index)), true
  957. }
  958. return nil, false
  959. }
  960. func (ps *PeerState) getVoteBitArray(height int64, round int32, votesType tmproto.SignedMsgType) *bits.BitArray {
  961. if !types.IsVoteTypeValid(votesType) {
  962. return nil
  963. }
  964. if ps.PRS.Height == height {
  965. if ps.PRS.Round == round {
  966. switch votesType {
  967. case tmproto.PrevoteType:
  968. return ps.PRS.Prevotes
  969. case tmproto.PrecommitType:
  970. return ps.PRS.Precommits
  971. }
  972. }
  973. if ps.PRS.CatchupCommitRound == round {
  974. switch votesType {
  975. case tmproto.PrevoteType:
  976. return nil
  977. case tmproto.PrecommitType:
  978. return ps.PRS.CatchupCommit
  979. }
  980. }
  981. if ps.PRS.ProposalPOLRound == round {
  982. switch votesType {
  983. case tmproto.PrevoteType:
  984. return ps.PRS.ProposalPOL
  985. case tmproto.PrecommitType:
  986. return nil
  987. }
  988. }
  989. return nil
  990. }
  991. if ps.PRS.Height == height+1 {
  992. if ps.PRS.LastCommitRound == round {
  993. switch votesType {
  994. case tmproto.PrevoteType:
  995. return nil
  996. case tmproto.PrecommitType:
  997. return ps.PRS.LastCommit
  998. }
  999. }
  1000. return nil
  1001. }
  1002. return nil
  1003. }
  1004. // 'round': A round for which we have a +2/3 commit.
  1005. func (ps *PeerState) ensureCatchupCommitRound(height int64, round int32, numValidators int) {
  1006. if ps.PRS.Height != height {
  1007. return
  1008. }
  1009. /*
  1010. NOTE: This is wrong, 'round' could change.
  1011. e.g. if orig round is not the same as block LastCommit round.
  1012. if ps.CatchupCommitRound != -1 && ps.CatchupCommitRound != round {
  1013. panic(fmt.Sprintf(
  1014. "Conflicting CatchupCommitRound. Height: %v,
  1015. Orig: %v,
  1016. New: %v",
  1017. height,
  1018. ps.CatchupCommitRound,
  1019. round))
  1020. }
  1021. */
  1022. if ps.PRS.CatchupCommitRound == round {
  1023. return // Nothing to do!
  1024. }
  1025. ps.PRS.CatchupCommitRound = round
  1026. if round == ps.PRS.Round {
  1027. ps.PRS.CatchupCommit = ps.PRS.Precommits
  1028. } else {
  1029. ps.PRS.CatchupCommit = bits.NewBitArray(numValidators)
  1030. }
  1031. }
  1032. // EnsureVoteBitArrays ensures the bit-arrays have been allocated for tracking
  1033. // what votes this peer has received.
  1034. // NOTE: It's important to make sure that numValidators actually matches
  1035. // what the node sees as the number of validators for height.
  1036. func (ps *PeerState) EnsureVoteBitArrays(height int64, numValidators int) {
  1037. ps.mtx.Lock()
  1038. defer ps.mtx.Unlock()
  1039. ps.ensureVoteBitArrays(height, numValidators)
  1040. }
  1041. func (ps *PeerState) ensureVoteBitArrays(height int64, numValidators int) {
  1042. if ps.PRS.Height == height {
  1043. if ps.PRS.Prevotes == nil {
  1044. ps.PRS.Prevotes = bits.NewBitArray(numValidators)
  1045. }
  1046. if ps.PRS.Precommits == nil {
  1047. ps.PRS.Precommits = bits.NewBitArray(numValidators)
  1048. }
  1049. if ps.PRS.CatchupCommit == nil {
  1050. ps.PRS.CatchupCommit = bits.NewBitArray(numValidators)
  1051. }
  1052. if ps.PRS.ProposalPOL == nil {
  1053. ps.PRS.ProposalPOL = bits.NewBitArray(numValidators)
  1054. }
  1055. } else if ps.PRS.Height == height+1 {
  1056. if ps.PRS.LastCommit == nil {
  1057. ps.PRS.LastCommit = bits.NewBitArray(numValidators)
  1058. }
  1059. }
  1060. }
  1061. // RecordVote increments internal votes related statistics for this peer.
  1062. // It returns the total number of added votes.
  1063. func (ps *PeerState) RecordVote() int {
  1064. ps.mtx.Lock()
  1065. defer ps.mtx.Unlock()
  1066. ps.Stats.Votes++
  1067. return ps.Stats.Votes
  1068. }
  1069. // VotesSent returns the number of blocks for which peer has been sending us
  1070. // votes.
  1071. func (ps *PeerState) VotesSent() int {
  1072. ps.mtx.Lock()
  1073. defer ps.mtx.Unlock()
  1074. return ps.Stats.Votes
  1075. }
  1076. // RecordBlockPart increments internal block part related statistics for this peer.
  1077. // It returns the total number of added block parts.
  1078. func (ps *PeerState) RecordBlockPart() int {
  1079. ps.mtx.Lock()
  1080. defer ps.mtx.Unlock()
  1081. ps.Stats.BlockParts++
  1082. return ps.Stats.BlockParts
  1083. }
  1084. // BlockPartsSent returns the number of useful block parts the peer has sent us.
  1085. func (ps *PeerState) BlockPartsSent() int {
  1086. ps.mtx.Lock()
  1087. defer ps.mtx.Unlock()
  1088. return ps.Stats.BlockParts
  1089. }
  1090. // SetHasVote sets the given vote as known by the peer
  1091. func (ps *PeerState) SetHasVote(vote *types.Vote) {
  1092. ps.mtx.Lock()
  1093. defer ps.mtx.Unlock()
  1094. ps.setHasVote(vote.Height, vote.Round, vote.Type, vote.ValidatorIndex)
  1095. }
  1096. func (ps *PeerState) setHasVote(height int64, round int32, voteType tmproto.SignedMsgType, index int32) {
  1097. logger := ps.logger.With(
  1098. "peerH/R",
  1099. fmt.Sprintf("%d/%d", ps.PRS.Height, ps.PRS.Round),
  1100. "H/R",
  1101. fmt.Sprintf("%d/%d", height, round))
  1102. logger.Debug("setHasVote", "type", voteType, "index", index)
  1103. // NOTE: some may be nil BitArrays -> no side effects.
  1104. psVotes := ps.getVoteBitArray(height, round, voteType)
  1105. if psVotes != nil {
  1106. psVotes.SetIndex(int(index), true)
  1107. }
  1108. }
  1109. // ApplyNewRoundStepMessage updates the peer state for the new round.
  1110. func (ps *PeerState) ApplyNewRoundStepMessage(msg *NewRoundStepMessage) {
  1111. ps.mtx.Lock()
  1112. defer ps.mtx.Unlock()
  1113. // Ignore duplicates or decreases
  1114. if CompareHRS(msg.Height, msg.Round, msg.Step, ps.PRS.Height, ps.PRS.Round, ps.PRS.Step) <= 0 {
  1115. return
  1116. }
  1117. // Just remember these values.
  1118. psHeight := ps.PRS.Height
  1119. psRound := ps.PRS.Round
  1120. psCatchupCommitRound := ps.PRS.CatchupCommitRound
  1121. psCatchupCommit := ps.PRS.CatchupCommit
  1122. startTime := tmtime.Now().Add(-1 * time.Duration(msg.SecondsSinceStartTime) * time.Second)
  1123. ps.PRS.Height = msg.Height
  1124. ps.PRS.Round = msg.Round
  1125. ps.PRS.Step = msg.Step
  1126. ps.PRS.StartTime = startTime
  1127. if psHeight != msg.Height || psRound != msg.Round {
  1128. ps.PRS.Proposal = false
  1129. ps.PRS.ProposalBlockPartSetHeader = types.PartSetHeader{}
  1130. ps.PRS.ProposalBlockParts = nil
  1131. ps.PRS.ProposalPOLRound = -1
  1132. ps.PRS.ProposalPOL = nil
  1133. // We'll update the BitArray capacity later.
  1134. ps.PRS.Prevotes = nil
  1135. ps.PRS.Precommits = nil
  1136. }
  1137. if psHeight == msg.Height && psRound != msg.Round && msg.Round == psCatchupCommitRound {
  1138. // Peer caught up to CatchupCommitRound.
  1139. // Preserve psCatchupCommit!
  1140. // NOTE: We prefer to use prs.Precommits if
  1141. // pr.Round matches pr.CatchupCommitRound.
  1142. ps.PRS.Precommits = psCatchupCommit
  1143. }
  1144. if psHeight != msg.Height {
  1145. // Shift Precommits to LastCommit.
  1146. if psHeight+1 == msg.Height && psRound == msg.LastCommitRound {
  1147. ps.PRS.LastCommitRound = msg.LastCommitRound
  1148. ps.PRS.LastCommit = ps.PRS.Precommits
  1149. } else {
  1150. ps.PRS.LastCommitRound = msg.LastCommitRound
  1151. ps.PRS.LastCommit = nil
  1152. }
  1153. // We'll update the BitArray capacity later.
  1154. ps.PRS.CatchupCommitRound = -1
  1155. ps.PRS.CatchupCommit = nil
  1156. }
  1157. }
  1158. // ApplyNewValidBlockMessage updates the peer state for the new valid block.
  1159. func (ps *PeerState) ApplyNewValidBlockMessage(msg *NewValidBlockMessage) {
  1160. ps.mtx.Lock()
  1161. defer ps.mtx.Unlock()
  1162. if ps.PRS.Height != msg.Height {
  1163. return
  1164. }
  1165. if ps.PRS.Round != msg.Round && !msg.IsCommit {
  1166. return
  1167. }
  1168. ps.PRS.ProposalBlockPartSetHeader = msg.BlockPartSetHeader
  1169. ps.PRS.ProposalBlockParts = msg.BlockParts
  1170. }
  1171. // ApplyProposalPOLMessage updates the peer state for the new proposal POL.
  1172. func (ps *PeerState) ApplyProposalPOLMessage(msg *ProposalPOLMessage) {
  1173. ps.mtx.Lock()
  1174. defer ps.mtx.Unlock()
  1175. if ps.PRS.Height != msg.Height {
  1176. return
  1177. }
  1178. if ps.PRS.ProposalPOLRound != msg.ProposalPOLRound {
  1179. return
  1180. }
  1181. // TODO: Merge onto existing ps.PRS.ProposalPOL?
  1182. // We might have sent some prevotes in the meantime.
  1183. ps.PRS.ProposalPOL = msg.ProposalPOL
  1184. }
  1185. // ApplyHasVoteMessage updates the peer state for the new vote.
  1186. func (ps *PeerState) ApplyHasVoteMessage(msg *HasVoteMessage) {
  1187. ps.mtx.Lock()
  1188. defer ps.mtx.Unlock()
  1189. if ps.PRS.Height != msg.Height {
  1190. return
  1191. }
  1192. ps.setHasVote(msg.Height, msg.Round, msg.Type, msg.Index)
  1193. }
  1194. // ApplyVoteSetBitsMessage updates the peer state for the bit-array of votes
  1195. // it claims to have for the corresponding BlockID.
  1196. // `ourVotes` is a BitArray of votes we have for msg.BlockID
  1197. // NOTE: if ourVotes is nil (e.g. msg.Height < rs.Height),
  1198. // we conservatively overwrite ps's votes w/ msg.Votes.
  1199. func (ps *PeerState) ApplyVoteSetBitsMessage(msg *VoteSetBitsMessage, ourVotes *bits.BitArray) {
  1200. ps.mtx.Lock()
  1201. defer ps.mtx.Unlock()
  1202. votes := ps.getVoteBitArray(msg.Height, msg.Round, msg.Type)
  1203. if votes != nil {
  1204. if ourVotes == nil {
  1205. votes.Update(msg.Votes)
  1206. } else {
  1207. otherVotes := votes.Sub(ourVotes)
  1208. hasVotes := otherVotes.Or(msg.Votes)
  1209. votes.Update(hasVotes)
  1210. }
  1211. }
  1212. }
  1213. // String returns a string representation of the PeerState
  1214. func (ps *PeerState) String() string {
  1215. return ps.StringIndented("")
  1216. }
  1217. // StringIndented returns a string representation of the PeerState
  1218. func (ps *PeerState) StringIndented(indent string) string {
  1219. ps.mtx.Lock()
  1220. defer ps.mtx.Unlock()
  1221. return fmt.Sprintf(`PeerState{
  1222. %s Key %v
  1223. %s RoundState %v
  1224. %s Stats %v
  1225. %s}`,
  1226. indent, ps.peer.ID(),
  1227. indent, ps.PRS.StringIndented(indent+" "),
  1228. indent, ps.Stats,
  1229. indent)
  1230. }
  1231. //-----------------------------------------------------------------------------
  1232. // Messages
  1233. // Message is a message that can be sent and received on the Reactor
  1234. type Message interface {
  1235. ValidateBasic() error
  1236. }
  1237. // func init() {
  1238. // tmjson.RegisterType(&NewRoundStepMessage{}, "tendermint/NewRoundStepMessage")
  1239. // tmjson.RegisterType(&NewValidBlockMessage{}, "tendermint/NewValidBlockMessage")
  1240. // tmjson.RegisterType(&ProposalMessage{}, "tendermint/Proposal")
  1241. // tmjson.RegisterType(&ProposalPOLMessage{}, "tendermint/ProposalPOL")
  1242. // tmjson.RegisterType(&BlockPartMessage{}, "tendermint/BlockPart")
  1243. // tmjson.RegisterType(&VoteMessage{}, "tendermint/Vote")
  1244. // tmjson.RegisterType(&HasVoteMessage{}, "tendermint/HasVote")
  1245. // tmjson.RegisterType(&VoteSetMaj23Message{}, "tendermint/VoteSetMaj23")
  1246. // tmjson.RegisterType(&VoteSetBitsMessage{}, "tendermint/VoteSetBits")
  1247. // }
  1248. func decodeMsg(bz []byte) (msg Message, err error) {
  1249. pb := &tmcons.Message{}
  1250. if err = proto.Unmarshal(bz, pb); err != nil {
  1251. return msg, err
  1252. }
  1253. return MsgFromProto(pb)
  1254. }
  1255. //-------------------------------------
  1256. // NewRoundStepMessage is sent for every step taken in the ConsensusState.
  1257. // For every height/round/step transition
  1258. type NewRoundStepMessage struct {
  1259. Height int64
  1260. Round int32
  1261. Step cstypes.RoundStepType
  1262. SecondsSinceStartTime int64
  1263. LastCommitRound int32
  1264. }
  1265. // ValidateBasic performs basic validation.
  1266. func (m *NewRoundStepMessage) ValidateBasic() error {
  1267. if m.Height < 0 {
  1268. return errors.New("negative Height")
  1269. }
  1270. if m.Round < 0 {
  1271. return errors.New("negative Round")
  1272. }
  1273. if !m.Step.IsValid() {
  1274. return errors.New("invalid Step")
  1275. }
  1276. // NOTE: SecondsSinceStartTime may be negative
  1277. // LastCommitRound will be -1 for the initial height, but we don't know what height this is
  1278. // since it can be specified in genesis. The reactor will have to validate this via
  1279. // ValidateHeight().
  1280. if m.LastCommitRound < -1 {
  1281. return errors.New("invalid LastCommitRound (cannot be < -1)")
  1282. }
  1283. return nil
  1284. }
  1285. // ValidateHeight validates the height given the chain's initial height.
  1286. func (m *NewRoundStepMessage) ValidateHeight(initialHeight int64) error {
  1287. if m.Height < initialHeight {
  1288. return fmt.Errorf("invalid Height %v (lower than initial height %v)",
  1289. m.Height, initialHeight)
  1290. }
  1291. if m.Height == initialHeight && m.LastCommitRound != -1 {
  1292. return fmt.Errorf("invalid LastCommitRound %v (must be -1 for initial height %v)",
  1293. m.LastCommitRound, initialHeight)
  1294. }
  1295. if m.Height > initialHeight && m.LastCommitRound < 0 {
  1296. return fmt.Errorf("LastCommitRound can only be negative for initial height %v", // nolint
  1297. initialHeight)
  1298. }
  1299. return nil
  1300. }
  1301. // String returns a string representation.
  1302. func (m *NewRoundStepMessage) String() string {
  1303. return fmt.Sprintf("[NewRoundStep H:%v R:%v S:%v LCR:%v]",
  1304. m.Height, m.Round, m.Step, m.LastCommitRound)
  1305. }
  1306. //-------------------------------------
  1307. // NewValidBlockMessage is sent when a validator observes a valid block B in some round r,
  1308. // i.e., there is a Proposal for block B and 2/3+ prevotes for the block B in the round r.
  1309. // In case the block is also committed, then IsCommit flag is set to true.
  1310. type NewValidBlockMessage struct {
  1311. Height int64
  1312. Round int32
  1313. BlockPartSetHeader types.PartSetHeader
  1314. BlockParts *bits.BitArray
  1315. IsCommit bool
  1316. }
  1317. // ValidateBasic performs basic validation.
  1318. func (m *NewValidBlockMessage) ValidateBasic() error {
  1319. if m.Height < 0 {
  1320. return errors.New("negative Height")
  1321. }
  1322. if m.Round < 0 {
  1323. return errors.New("negative Round")
  1324. }
  1325. if err := m.BlockPartSetHeader.ValidateBasic(); err != nil {
  1326. return fmt.Errorf("wrong BlockPartSetHeader: %v", err)
  1327. }
  1328. if m.BlockParts.Size() == 0 {
  1329. return errors.New("empty blockParts")
  1330. }
  1331. if m.BlockParts.Size() != int(m.BlockPartSetHeader.Total) {
  1332. return fmt.Errorf("blockParts bit array size %d not equal to BlockPartSetHeader.Total %d",
  1333. m.BlockParts.Size(),
  1334. m.BlockPartSetHeader.Total)
  1335. }
  1336. if m.BlockParts.Size() > int(types.MaxBlockPartsCount) {
  1337. return fmt.Errorf("blockParts bit array is too big: %d, max: %d", m.BlockParts.Size(), types.MaxBlockPartsCount)
  1338. }
  1339. return nil
  1340. }
  1341. // String returns a string representation.
  1342. func (m *NewValidBlockMessage) String() string {
  1343. return fmt.Sprintf("[ValidBlockMessage H:%v R:%v BP:%v BA:%v IsCommit:%v]",
  1344. m.Height, m.Round, m.BlockPartSetHeader, m.BlockParts, m.IsCommit)
  1345. }
  1346. //-------------------------------------
  1347. // ProposalMessage is sent when a new block is proposed.
  1348. type ProposalMessage struct {
  1349. Proposal *types.Proposal
  1350. }
  1351. // ValidateBasic performs basic validation.
  1352. func (m *ProposalMessage) ValidateBasic() error {
  1353. return m.Proposal.ValidateBasic()
  1354. }
  1355. // String returns a string representation.
  1356. func (m *ProposalMessage) String() string {
  1357. return fmt.Sprintf("[Proposal %v]", m.Proposal)
  1358. }
  1359. //-------------------------------------
  1360. // ProposalPOLMessage is sent when a previous proposal is re-proposed.
  1361. type ProposalPOLMessage struct {
  1362. Height int64
  1363. ProposalPOLRound int32
  1364. ProposalPOL *bits.BitArray
  1365. }
  1366. // ValidateBasic performs basic validation.
  1367. func (m *ProposalPOLMessage) ValidateBasic() error {
  1368. if m.Height < 0 {
  1369. return errors.New("negative Height")
  1370. }
  1371. if m.ProposalPOLRound < 0 {
  1372. return errors.New("negative ProposalPOLRound")
  1373. }
  1374. if m.ProposalPOL.Size() == 0 {
  1375. return errors.New("empty ProposalPOL bit array")
  1376. }
  1377. if m.ProposalPOL.Size() > types.MaxVotesCount {
  1378. return fmt.Errorf("proposalPOL bit array is too big: %d, max: %d", m.ProposalPOL.Size(), types.MaxVotesCount)
  1379. }
  1380. return nil
  1381. }
  1382. // String returns a string representation.
  1383. func (m *ProposalPOLMessage) String() string {
  1384. return fmt.Sprintf("[ProposalPOL H:%v POLR:%v POL:%v]", m.Height, m.ProposalPOLRound, m.ProposalPOL)
  1385. }
  1386. //-------------------------------------
  1387. // BlockPartMessage is sent when gossipping a piece of the proposed block.
  1388. type BlockPartMessage struct {
  1389. Height int64
  1390. Round int32
  1391. Part *types.Part
  1392. }
  1393. // ValidateBasic performs basic validation.
  1394. func (m *BlockPartMessage) ValidateBasic() error {
  1395. if m.Height < 0 {
  1396. return errors.New("negative Height")
  1397. }
  1398. if m.Round < 0 {
  1399. return errors.New("negative Round")
  1400. }
  1401. if err := m.Part.ValidateBasic(); err != nil {
  1402. return fmt.Errorf("wrong Part: %v", err)
  1403. }
  1404. return nil
  1405. }
  1406. // String returns a string representation.
  1407. func (m *BlockPartMessage) String() string {
  1408. return fmt.Sprintf("[BlockPart H:%v R:%v P:%v]", m.Height, m.Round, m.Part)
  1409. }
  1410. //-------------------------------------
  1411. // VoteMessage is sent when voting for a proposal (or lack thereof).
  1412. type VoteMessage struct {
  1413. Vote *types.Vote
  1414. }
  1415. // ValidateBasic performs basic validation.
  1416. func (m *VoteMessage) ValidateBasic() error {
  1417. return m.Vote.ValidateBasic()
  1418. }
  1419. // String returns a string representation.
  1420. func (m *VoteMessage) String() string {
  1421. return fmt.Sprintf("[Vote %v]", m.Vote)
  1422. }
  1423. //-------------------------------------
  1424. // HasVoteMessage is sent to indicate that a particular vote has been received.
  1425. type HasVoteMessage struct {
  1426. Height int64
  1427. Round int32
  1428. Type tmproto.SignedMsgType
  1429. Index int32
  1430. }
  1431. // ValidateBasic performs basic validation.
  1432. func (m *HasVoteMessage) ValidateBasic() error {
  1433. if m.Height < 0 {
  1434. return errors.New("negative Height")
  1435. }
  1436. if m.Round < 0 {
  1437. return errors.New("negative Round")
  1438. }
  1439. if !types.IsVoteTypeValid(m.Type) {
  1440. return errors.New("invalid Type")
  1441. }
  1442. if m.Index < 0 {
  1443. return errors.New("negative Index")
  1444. }
  1445. return nil
  1446. }
  1447. // String returns a string representation.
  1448. func (m *HasVoteMessage) String() string {
  1449. return fmt.Sprintf("[HasVote VI:%v V:{%v/%02d/%v}]", m.Index, m.Height, m.Round, m.Type)
  1450. }
  1451. //-------------------------------------
  1452. // VoteSetMaj23Message is sent to indicate that a given BlockID has seen +2/3 votes.
  1453. type VoteSetMaj23Message struct {
  1454. Height int64
  1455. Round int32
  1456. Type tmproto.SignedMsgType
  1457. BlockID types.BlockID
  1458. }
  1459. // ValidateBasic performs basic validation.
  1460. func (m *VoteSetMaj23Message) ValidateBasic() error {
  1461. if m.Height < 0 {
  1462. return errors.New("negative Height")
  1463. }
  1464. if m.Round < 0 {
  1465. return errors.New("negative Round")
  1466. }
  1467. if !types.IsVoteTypeValid(m.Type) {
  1468. return errors.New("invalid Type")
  1469. }
  1470. if err := m.BlockID.ValidateBasic(); err != nil {
  1471. return fmt.Errorf("wrong BlockID: %v", err)
  1472. }
  1473. return nil
  1474. }
  1475. // String returns a string representation.
  1476. func (m *VoteSetMaj23Message) String() string {
  1477. return fmt.Sprintf("[VSM23 %v/%02d/%v %v]", m.Height, m.Round, m.Type, m.BlockID)
  1478. }
  1479. //-------------------------------------
  1480. // VoteSetBitsMessage is sent to communicate the bit-array of votes seen for the BlockID.
  1481. type VoteSetBitsMessage struct {
  1482. Height int64
  1483. Round int32
  1484. Type tmproto.SignedMsgType
  1485. BlockID types.BlockID
  1486. Votes *bits.BitArray
  1487. }
  1488. // ValidateBasic performs basic validation.
  1489. func (m *VoteSetBitsMessage) ValidateBasic() error {
  1490. if m.Height < 0 {
  1491. return errors.New("negative Height")
  1492. }
  1493. if !types.IsVoteTypeValid(m.Type) {
  1494. return errors.New("invalid Type")
  1495. }
  1496. if err := m.BlockID.ValidateBasic(); err != nil {
  1497. return fmt.Errorf("wrong BlockID: %v", err)
  1498. }
  1499. // NOTE: Votes.Size() can be zero if the node does not have any
  1500. if m.Votes.Size() > types.MaxVotesCount {
  1501. return fmt.Errorf("votes bit array is too big: %d, max: %d", m.Votes.Size(), types.MaxVotesCount)
  1502. }
  1503. return nil
  1504. }
  1505. // String returns a string representation.
  1506. func (m *VoteSetBitsMessage) String() string {
  1507. return fmt.Sprintf("[VSB %v/%02d/%v %v %v]", m.Height, m.Round, m.Type, m.BlockID, m.Votes)
  1508. }
  1509. //-------------------------------------