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.

1307 lines
40 KiB

7 years ago
10 years ago
10 years ago
7 years ago
10 years ago
7 years ago
10 years ago
7 years ago
9 years ago
9 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
7 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
7 years ago
7 years ago
10 years ago
10 years ago
7 years ago
10 years ago
10 years ago
10 years ago
10 years ago
7 years ago
7 years ago
7 years ago
10 years ago
10 years ago
10 years ago
10 years ago
7 years ago
10 years ago
10 years ago
7 years ago
10 years ago
10 years ago
9 years ago
7 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
7 years ago
10 years ago
7 years ago
10 years ago
10 years ago
7 years ago
9 years ago
10 years ago
10 years ago
7 years ago
10 years ago
10 years ago
8 years ago
7 years ago
7 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
7 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
9 years ago
  1. package consensus
  2. import (
  3. "bytes"
  4. "errors"
  5. "fmt"
  6. "reflect"
  7. "sync"
  8. "time"
  9. wire "github.com/tendermint/go-wire"
  10. cmn "github.com/tendermint/tmlibs/common"
  11. "github.com/tendermint/tmlibs/log"
  12. "github.com/tendermint/tendermint/p2p"
  13. sm "github.com/tendermint/tendermint/state"
  14. "github.com/tendermint/tendermint/types"
  15. )
  16. const (
  17. StateChannel = byte(0x20)
  18. DataChannel = byte(0x21)
  19. VoteChannel = byte(0x22)
  20. VoteSetBitsChannel = byte(0x23)
  21. maxConsensusMessageSize = 1048576 // 1MB; NOTE/TODO: keep in sync with types.PartSet sizes.
  22. )
  23. //-----------------------------------------------------------------------------
  24. // ConsensusReactor defines a reactor for the consensus service.
  25. type ConsensusReactor struct {
  26. p2p.BaseReactor // BaseService + p2p.Switch
  27. conS *ConsensusState
  28. fastSync bool
  29. evsw types.EventSwitch
  30. }
  31. // NewConsensusReactor returns a new ConsensusReactor with the given consensusState.
  32. func NewConsensusReactor(consensusState *ConsensusState, fastSync bool) *ConsensusReactor {
  33. conR := &ConsensusReactor{
  34. conS: consensusState,
  35. fastSync: fastSync,
  36. }
  37. conR.BaseReactor = *p2p.NewBaseReactor("ConsensusReactor", conR)
  38. return conR
  39. }
  40. // OnStart implements BaseService.
  41. func (conR *ConsensusReactor) OnStart() error {
  42. conR.Logger.Info("ConsensusReactor ", "fastSync", conR.fastSync)
  43. conR.BaseReactor.OnStart()
  44. // callbacks for broadcasting new steps and votes to peers
  45. // upon their respective events (ie. uses evsw)
  46. conR.registerEventCallbacks()
  47. if !conR.fastSync {
  48. _, err := conR.conS.Start()
  49. if err != nil {
  50. return err
  51. }
  52. }
  53. return nil
  54. }
  55. // OnStop implements BaseService
  56. func (conR *ConsensusReactor) OnStop() {
  57. conR.BaseReactor.OnStop()
  58. conR.conS.Stop()
  59. }
  60. // SwitchToConsensus switches from fast_sync mode to consensus mode.
  61. // It resets the state, turns off fast_sync, and starts the consensus state-machine
  62. func (conR *ConsensusReactor) SwitchToConsensus(state *sm.State) {
  63. conR.Logger.Info("SwitchToConsensus")
  64. conR.conS.reconstructLastCommit(state)
  65. // NOTE: The line below causes broadcastNewRoundStepRoutine() to
  66. // broadcast a NewRoundStepMessage.
  67. conR.conS.updateToState(state)
  68. conR.fastSync = false
  69. conR.conS.Start()
  70. }
  71. // GetChannels implements Reactor
  72. func (conR *ConsensusReactor) GetChannels() []*p2p.ChannelDescriptor {
  73. // TODO optimize
  74. return []*p2p.ChannelDescriptor{
  75. &p2p.ChannelDescriptor{
  76. ID: StateChannel,
  77. Priority: 5,
  78. SendQueueCapacity: 100,
  79. },
  80. &p2p.ChannelDescriptor{
  81. ID: DataChannel, // maybe split between gossiping current block and catchup stuff
  82. Priority: 10, // once we gossip the whole block there's nothing left to send until next height or round
  83. SendQueueCapacity: 100,
  84. RecvBufferCapacity: 50 * 4096,
  85. },
  86. &p2p.ChannelDescriptor{
  87. ID: VoteChannel,
  88. Priority: 5,
  89. SendQueueCapacity: 100,
  90. RecvBufferCapacity: 100 * 100,
  91. },
  92. &p2p.ChannelDescriptor{
  93. ID: VoteSetBitsChannel,
  94. Priority: 1,
  95. SendQueueCapacity: 2,
  96. RecvBufferCapacity: 1024,
  97. },
  98. }
  99. }
  100. // AddPeer implements Reactor
  101. func (conR *ConsensusReactor) AddPeer(peer *p2p.Peer) {
  102. if !conR.IsRunning() {
  103. return
  104. }
  105. // Create peerState for peer
  106. peerState := NewPeerState(peer)
  107. peer.Data.Set(types.PeerStateKey, peerState)
  108. // Begin routines for this peer.
  109. go conR.gossipDataRoutine(peer, peerState)
  110. go conR.gossipVotesRoutine(peer, peerState)
  111. go conR.queryMaj23Routine(peer, peerState)
  112. // Send our state to peer.
  113. // If we're fast_syncing, broadcast a RoundStepMessage later upon SwitchToConsensus().
  114. if !conR.fastSync {
  115. conR.sendNewRoundStepMessages(peer)
  116. }
  117. }
  118. // RemovePeer implements Reactor
  119. func (conR *ConsensusReactor) RemovePeer(peer *p2p.Peer, reason interface{}) {
  120. if !conR.IsRunning() {
  121. return
  122. }
  123. // TODO
  124. //peer.Data.Get(PeerStateKey).(*PeerState).Disconnect()
  125. }
  126. // Receive implements Reactor
  127. // NOTE: We process these messages even when we're fast_syncing.
  128. // Messages affect either a peer state or the consensus state.
  129. // Peer state updates can happen in parallel, but processing of
  130. // proposals, block parts, and votes are ordered by the receiveRoutine
  131. // NOTE: blocks on consensus state for proposals, block parts, and votes
  132. func (conR *ConsensusReactor) Receive(chID byte, src *p2p.Peer, msgBytes []byte) {
  133. if !conR.IsRunning() {
  134. conR.Logger.Debug("Receive", "src", src, "chId", chID, "bytes", msgBytes)
  135. return
  136. }
  137. _, msg, err := DecodeMessage(msgBytes)
  138. if err != nil {
  139. conR.Logger.Error("Error decoding message", "src", src, "chId", chID, "msg", msg, "err", err, "bytes", msgBytes)
  140. // TODO punish peer?
  141. return
  142. }
  143. conR.Logger.Debug("Receive", "src", src, "chId", chID, "msg", msg)
  144. // Get peer states
  145. ps := src.Data.Get(types.PeerStateKey).(*PeerState)
  146. switch chID {
  147. case StateChannel:
  148. switch msg := msg.(type) {
  149. case *NewRoundStepMessage:
  150. ps.ApplyNewRoundStepMessage(msg)
  151. case *CommitStepMessage:
  152. ps.ApplyCommitStepMessage(msg)
  153. case *HasVoteMessage:
  154. ps.ApplyHasVoteMessage(msg)
  155. case *VoteSetMaj23Message:
  156. cs := conR.conS
  157. cs.mtx.Lock()
  158. height, votes := cs.Height, cs.Votes
  159. cs.mtx.Unlock()
  160. if height != msg.Height {
  161. return
  162. }
  163. // Peer claims to have a maj23 for some BlockID at H,R,S,
  164. votes.SetPeerMaj23(msg.Round, msg.Type, ps.Peer.Key, msg.BlockID)
  165. // Respond with a VoteSetBitsMessage showing which votes we have.
  166. // (and consequently shows which we don't have)
  167. var ourVotes *cmn.BitArray
  168. switch msg.Type {
  169. case types.VoteTypePrevote:
  170. ourVotes = votes.Prevotes(msg.Round).BitArrayByBlockID(msg.BlockID)
  171. case types.VoteTypePrecommit:
  172. ourVotes = votes.Precommits(msg.Round).BitArrayByBlockID(msg.BlockID)
  173. default:
  174. conR.Logger.Error("Bad VoteSetBitsMessage field Type")
  175. return
  176. }
  177. src.TrySend(VoteSetBitsChannel, struct{ ConsensusMessage }{&VoteSetBitsMessage{
  178. Height: msg.Height,
  179. Round: msg.Round,
  180. Type: msg.Type,
  181. BlockID: msg.BlockID,
  182. Votes: ourVotes,
  183. }})
  184. default:
  185. conR.Logger.Error(cmn.Fmt("Unknown message type %v", reflect.TypeOf(msg)))
  186. }
  187. case DataChannel:
  188. if conR.fastSync {
  189. conR.Logger.Info("Ignoring message received during fastSync", "msg", msg)
  190. return
  191. }
  192. switch msg := msg.(type) {
  193. case *ProposalMessage:
  194. ps.SetHasProposal(msg.Proposal)
  195. conR.conS.peerMsgQueue <- msgInfo{msg, src.Key}
  196. case *ProposalPOLMessage:
  197. ps.ApplyProposalPOLMessage(msg)
  198. case *BlockPartMessage:
  199. ps.SetHasProposalBlockPart(msg.Height, msg.Round, msg.Part.Index)
  200. conR.conS.peerMsgQueue <- msgInfo{msg, src.Key}
  201. default:
  202. conR.Logger.Error(cmn.Fmt("Unknown message type %v", reflect.TypeOf(msg)))
  203. }
  204. case VoteChannel:
  205. if conR.fastSync {
  206. conR.Logger.Info("Ignoring message received during fastSync", "msg", msg)
  207. return
  208. }
  209. switch msg := msg.(type) {
  210. case *VoteMessage:
  211. cs := conR.conS
  212. cs.mtx.Lock()
  213. height, valSize, lastCommitSize := cs.Height, cs.Validators.Size(), cs.LastCommit.Size()
  214. cs.mtx.Unlock()
  215. ps.EnsureVoteBitArrays(height, valSize)
  216. ps.EnsureVoteBitArrays(height-1, lastCommitSize)
  217. ps.SetHasVote(msg.Vote)
  218. conR.conS.peerMsgQueue <- msgInfo{msg, src.Key}
  219. default:
  220. // don't punish (leave room for soft upgrades)
  221. conR.Logger.Error(cmn.Fmt("Unknown message type %v", reflect.TypeOf(msg)))
  222. }
  223. case VoteSetBitsChannel:
  224. if conR.fastSync {
  225. conR.Logger.Info("Ignoring message received during fastSync", "msg", msg)
  226. return
  227. }
  228. switch msg := msg.(type) {
  229. case *VoteSetBitsMessage:
  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. var ourVotes *cmn.BitArray
  236. switch msg.Type {
  237. case types.VoteTypePrevote:
  238. ourVotes = votes.Prevotes(msg.Round).BitArrayByBlockID(msg.BlockID)
  239. case types.VoteTypePrecommit:
  240. ourVotes = votes.Precommits(msg.Round).BitArrayByBlockID(msg.BlockID)
  241. default:
  242. conR.Logger.Error("Bad VoteSetBitsMessage field Type")
  243. return
  244. }
  245. ps.ApplyVoteSetBitsMessage(msg, ourVotes)
  246. } else {
  247. ps.ApplyVoteSetBitsMessage(msg, nil)
  248. }
  249. default:
  250. // don't punish (leave room for soft upgrades)
  251. conR.Logger.Error(cmn.Fmt("Unknown message type %v", reflect.TypeOf(msg)))
  252. }
  253. default:
  254. conR.Logger.Error(cmn.Fmt("Unknown chId %X", chID))
  255. }
  256. if err != nil {
  257. conR.Logger.Error("Error in Receive()", "err", err)
  258. }
  259. }
  260. // SetEventSwitch implements events.Eventable
  261. func (conR *ConsensusReactor) SetEventSwitch(evsw types.EventSwitch) {
  262. conR.evsw = evsw
  263. conR.conS.SetEventSwitch(evsw)
  264. }
  265. //--------------------------------------
  266. // Listens for new steps and votes,
  267. // broadcasting the result to peers
  268. func (conR *ConsensusReactor) registerEventCallbacks() {
  269. types.AddListenerForEvent(conR.evsw, "conR", types.EventStringNewRoundStep(), func(data types.TMEventData) {
  270. rs := data.Unwrap().(types.EventDataRoundState).RoundState.(*RoundState)
  271. conR.broadcastNewRoundStep(rs)
  272. })
  273. types.AddListenerForEvent(conR.evsw, "conR", types.EventStringVote(), func(data types.TMEventData) {
  274. edv := data.Unwrap().(types.EventDataVote)
  275. conR.broadcastHasVoteMessage(edv.Vote)
  276. })
  277. }
  278. func (conR *ConsensusReactor) broadcastNewRoundStep(rs *RoundState) {
  279. nrsMsg, csMsg := makeRoundStepMessages(rs)
  280. if nrsMsg != nil {
  281. conR.Switch.Broadcast(StateChannel, struct{ ConsensusMessage }{nrsMsg})
  282. }
  283. if csMsg != nil {
  284. conR.Switch.Broadcast(StateChannel, struct{ ConsensusMessage }{csMsg})
  285. }
  286. }
  287. // Broadcasts HasVoteMessage to peers that care.
  288. func (conR *ConsensusReactor) broadcastHasVoteMessage(vote *types.Vote) {
  289. msg := &HasVoteMessage{
  290. Height: vote.Height,
  291. Round: vote.Round,
  292. Type: vote.Type,
  293. Index: vote.ValidatorIndex,
  294. }
  295. conR.Switch.Broadcast(StateChannel, struct{ ConsensusMessage }{msg})
  296. /*
  297. // TODO: Make this broadcast more selective.
  298. for _, peer := range conR.Switch.Peers().List() {
  299. ps := peer.Data.Get(PeerStateKey).(*PeerState)
  300. prs := ps.GetRoundState()
  301. if prs.Height == vote.Height {
  302. // TODO: Also filter on round?
  303. peer.TrySend(StateChannel, struct{ ConsensusMessage }{msg})
  304. } else {
  305. // Height doesn't match
  306. // TODO: check a field, maybe CatchupCommitRound?
  307. // TODO: But that requires changing the struct field comment.
  308. }
  309. }
  310. */
  311. }
  312. func makeRoundStepMessages(rs *RoundState) (nrsMsg *NewRoundStepMessage, csMsg *CommitStepMessage) {
  313. nrsMsg = &NewRoundStepMessage{
  314. Height: rs.Height,
  315. Round: rs.Round,
  316. Step: rs.Step,
  317. SecondsSinceStartTime: int(time.Since(rs.StartTime).Seconds()),
  318. LastCommitRound: rs.LastCommit.Round(),
  319. }
  320. if rs.Step == RoundStepCommit {
  321. csMsg = &CommitStepMessage{
  322. Height: rs.Height,
  323. BlockPartsHeader: rs.ProposalBlockParts.Header(),
  324. BlockParts: rs.ProposalBlockParts.BitArray(),
  325. }
  326. }
  327. return
  328. }
  329. func (conR *ConsensusReactor) sendNewRoundStepMessages(peer *p2p.Peer) {
  330. rs := conR.conS.GetRoundState()
  331. nrsMsg, csMsg := makeRoundStepMessages(rs)
  332. if nrsMsg != nil {
  333. peer.Send(StateChannel, struct{ ConsensusMessage }{nrsMsg})
  334. }
  335. if csMsg != nil {
  336. peer.Send(StateChannel, struct{ ConsensusMessage }{csMsg})
  337. }
  338. }
  339. func (conR *ConsensusReactor) gossipDataRoutine(peer *p2p.Peer, ps *PeerState) {
  340. logger := conR.Logger.With("peer", peer)
  341. OUTER_LOOP:
  342. for {
  343. // Manage disconnects from self or peer.
  344. if !peer.IsRunning() || !conR.IsRunning() {
  345. logger.Info("Stopping gossipDataRoutine for peer")
  346. return
  347. }
  348. rs := conR.conS.GetRoundState()
  349. prs := ps.GetRoundState()
  350. // Send proposal Block parts?
  351. if rs.ProposalBlockParts.HasHeader(prs.ProposalBlockPartsHeader) {
  352. if index, ok := rs.ProposalBlockParts.BitArray().Sub(prs.ProposalBlockParts.Copy()).PickRandom(); ok {
  353. part := rs.ProposalBlockParts.GetPart(index)
  354. msg := &BlockPartMessage{
  355. Height: rs.Height, // This tells peer that this part applies to us.
  356. Round: rs.Round, // This tells peer that this part applies to us.
  357. Part: part,
  358. }
  359. logger.Debug("Sending block part", "height", prs.Height, "round", prs.Round)
  360. if peer.Send(DataChannel, struct{ ConsensusMessage }{msg}) {
  361. ps.SetHasProposalBlockPart(prs.Height, prs.Round, index)
  362. }
  363. continue OUTER_LOOP
  364. }
  365. }
  366. // If the peer is on a previous height, help catch up.
  367. if (0 < prs.Height) && (prs.Height < rs.Height) {
  368. heightLogger := logger.With("height", prs.Height)
  369. conR.gossipDataForCatchup(heightLogger, rs, prs, ps, peer)
  370. continue OUTER_LOOP
  371. }
  372. // If height and round don't match, sleep.
  373. if (rs.Height != prs.Height) || (rs.Round != prs.Round) {
  374. //logger.Info("Peer Height|Round mismatch, sleeping", "peerHeight", prs.Height, "peerRound", prs.Round, "peer", peer)
  375. time.Sleep(conR.conS.config.PeerGossipSleep())
  376. continue OUTER_LOOP
  377. }
  378. // By here, height and round match.
  379. // Proposal block parts were already matched and sent if any were wanted.
  380. // (These can match on hash so the round doesn't matter)
  381. // Now consider sending other things, like the Proposal itself.
  382. // Send Proposal && ProposalPOL BitArray?
  383. if rs.Proposal != nil && !prs.Proposal {
  384. // Proposal: share the proposal metadata with peer.
  385. {
  386. msg := &ProposalMessage{Proposal: rs.Proposal}
  387. logger.Debug("Sending proposal", "height", prs.Height, "round", prs.Round)
  388. if peer.Send(DataChannel, struct{ ConsensusMessage }{msg}) {
  389. ps.SetHasProposal(rs.Proposal)
  390. }
  391. }
  392. // ProposalPOL: lets peer know which POL votes we have so far.
  393. // Peer must receive ProposalMessage first.
  394. // rs.Proposal was validated, so rs.Proposal.POLRound <= rs.Round,
  395. // so we definitely have rs.Votes.Prevotes(rs.Proposal.POLRound).
  396. if 0 <= rs.Proposal.POLRound {
  397. msg := &ProposalPOLMessage{
  398. Height: rs.Height,
  399. ProposalPOLRound: rs.Proposal.POLRound,
  400. ProposalPOL: rs.Votes.Prevotes(rs.Proposal.POLRound).BitArray(),
  401. }
  402. logger.Debug("Sending POL", "height", prs.Height, "round", prs.Round)
  403. peer.Send(DataChannel, struct{ ConsensusMessage }{msg})
  404. }
  405. continue OUTER_LOOP
  406. }
  407. // Nothing to do. Sleep.
  408. time.Sleep(conR.conS.config.PeerGossipSleep())
  409. continue OUTER_LOOP
  410. }
  411. }
  412. func (conR *ConsensusReactor) gossipDataForCatchup(logger log.Logger, rs *RoundState,
  413. prs *PeerRoundState, ps *PeerState, peer *p2p.Peer) {
  414. if index, ok := prs.ProposalBlockParts.Not().PickRandom(); ok {
  415. // Ensure that the peer's PartSetHeader is correct
  416. blockMeta := conR.conS.blockStore.LoadBlockMeta(prs.Height)
  417. if blockMeta == nil {
  418. logger.Error("Failed to load block meta",
  419. "ourHeight", rs.Height, "blockstoreHeight", conR.conS.blockStore.Height())
  420. time.Sleep(conR.conS.config.PeerGossipSleep())
  421. return
  422. } else if !blockMeta.BlockID.PartsHeader.Equals(prs.ProposalBlockPartsHeader) {
  423. logger.Info("Peer ProposalBlockPartsHeader mismatch, sleeping",
  424. "blockPartsHeader", blockMeta.BlockID.PartsHeader, "peerBlockPartsHeader", prs.ProposalBlockPartsHeader)
  425. time.Sleep(conR.conS.config.PeerGossipSleep())
  426. return
  427. }
  428. // Load the part
  429. part := conR.conS.blockStore.LoadBlockPart(prs.Height, index)
  430. if part == nil {
  431. logger.Error("Could not load part", "index", index,
  432. "blockPartsHeader", blockMeta.BlockID.PartsHeader, "peerBlockPartsHeader", prs.ProposalBlockPartsHeader)
  433. time.Sleep(conR.conS.config.PeerGossipSleep())
  434. return
  435. }
  436. // Send the part
  437. msg := &BlockPartMessage{
  438. Height: prs.Height, // Not our height, so it doesn't matter.
  439. Round: prs.Round, // Not our height, so it doesn't matter.
  440. Part: part,
  441. }
  442. logger.Debug("Sending block part for catchup", "round", prs.Round)
  443. if peer.Send(DataChannel, struct{ ConsensusMessage }{msg}) {
  444. ps.SetHasProposalBlockPart(prs.Height, prs.Round, index)
  445. }
  446. return
  447. } else {
  448. //logger.Info("No parts to send in catch-up, sleeping")
  449. time.Sleep(conR.conS.config.PeerGossipSleep())
  450. return
  451. }
  452. }
  453. func (conR *ConsensusReactor) gossipVotesRoutine(peer *p2p.Peer, ps *PeerState) {
  454. logger := conR.Logger.With("peer", peer)
  455. // Simple hack to throttle logs upon sleep.
  456. var sleeping = 0
  457. OUTER_LOOP:
  458. for {
  459. // Manage disconnects from self or peer.
  460. if !peer.IsRunning() || !conR.IsRunning() {
  461. logger.Info("Stopping gossipVotesRoutine for peer")
  462. return
  463. }
  464. rs := conR.conS.GetRoundState()
  465. prs := ps.GetRoundState()
  466. switch sleeping {
  467. case 1: // First sleep
  468. sleeping = 2
  469. case 2: // No more sleep
  470. sleeping = 0
  471. }
  472. //logger.Debug("gossipVotesRoutine", "rsHeight", rs.Height, "rsRound", rs.Round,
  473. // "prsHeight", prs.Height, "prsRound", prs.Round, "prsStep", prs.Step)
  474. // If height matches, then send LastCommit, Prevotes, Precommits.
  475. if rs.Height == prs.Height {
  476. heightLogger := logger.With("height", prs.Height)
  477. if conR.gossipVotesForHeight(heightLogger, rs, prs, ps) {
  478. continue OUTER_LOOP
  479. }
  480. }
  481. // Special catchup logic.
  482. // If peer is lagging by height 1, send LastCommit.
  483. if prs.Height != 0 && rs.Height == prs.Height+1 {
  484. if ps.PickSendVote(rs.LastCommit) {
  485. logger.Debug("Picked rs.LastCommit to send", "height", prs.Height)
  486. continue OUTER_LOOP
  487. }
  488. }
  489. // Catchup logic
  490. // If peer is lagging by more than 1, send Commit.
  491. if prs.Height != 0 && rs.Height >= prs.Height+2 {
  492. // Load the block commit for prs.Height,
  493. // which contains precommit signatures for prs.Height.
  494. commit := conR.conS.blockStore.LoadBlockCommit(prs.Height)
  495. logger.Info("Loaded BlockCommit for catch-up", "height", prs.Height, "commit", commit)
  496. if ps.PickSendVote(commit) {
  497. logger.Debug("Picked Catchup commit to send", "height", prs.Height)
  498. continue OUTER_LOOP
  499. }
  500. }
  501. if sleeping == 0 {
  502. // We sent nothing. Sleep...
  503. sleeping = 1
  504. logger.Debug("No votes to send, sleeping", "rs.Height", rs.Height, "prs.Height", prs.Height,
  505. "localPV", rs.Votes.Prevotes(rs.Round).BitArray(), "peerPV", prs.Prevotes,
  506. "localPC", rs.Votes.Precommits(rs.Round).BitArray(), "peerPC", prs.Precommits)
  507. } else if sleeping == 2 {
  508. // Continued sleep...
  509. sleeping = 1
  510. }
  511. time.Sleep(conR.conS.config.PeerGossipSleep())
  512. continue OUTER_LOOP
  513. }
  514. }
  515. func (conR *ConsensusReactor) gossipVotesForHeight(logger log.Logger, rs *RoundState, prs *PeerRoundState, ps *PeerState) bool {
  516. // If there are lastCommits to send...
  517. if prs.Step == RoundStepNewHeight {
  518. if ps.PickSendVote(rs.LastCommit) {
  519. logger.Debug("Picked rs.LastCommit to send")
  520. return true
  521. }
  522. }
  523. // If there are prevotes to send...
  524. if prs.Step <= RoundStepPrevote && prs.Round != -1 && prs.Round <= rs.Round {
  525. if ps.PickSendVote(rs.Votes.Prevotes(prs.Round)) {
  526. logger.Debug("Picked rs.Prevotes(prs.Round) to send", "round", prs.Round)
  527. return true
  528. }
  529. }
  530. // If there are precommits to send...
  531. if prs.Step <= RoundStepPrecommit && prs.Round != -1 && prs.Round <= rs.Round {
  532. if ps.PickSendVote(rs.Votes.Precommits(prs.Round)) {
  533. logger.Debug("Picked rs.Precommits(prs.Round) to send", "round", prs.Round)
  534. return true
  535. }
  536. }
  537. // If there are POLPrevotes to send...
  538. if prs.ProposalPOLRound != -1 {
  539. if polPrevotes := rs.Votes.Prevotes(prs.ProposalPOLRound); polPrevotes != nil {
  540. if ps.PickSendVote(polPrevotes) {
  541. logger.Debug("Picked rs.Prevotes(prs.ProposalPOLRound) to send",
  542. "round", prs.ProposalPOLRound)
  543. return true
  544. }
  545. }
  546. }
  547. return false
  548. }
  549. // NOTE: `queryMaj23Routine` has a simple crude design since it only comes
  550. // into play for liveness when there's a signature DDoS attack happening.
  551. func (conR *ConsensusReactor) queryMaj23Routine(peer *p2p.Peer, ps *PeerState) {
  552. logger := conR.Logger.With("peer", peer)
  553. OUTER_LOOP:
  554. for {
  555. // Manage disconnects from self or peer.
  556. if !peer.IsRunning() || !conR.IsRunning() {
  557. logger.Info("Stopping queryMaj23Routine for peer")
  558. return
  559. }
  560. // Maybe send Height/Round/Prevotes
  561. {
  562. rs := conR.conS.GetRoundState()
  563. prs := ps.GetRoundState()
  564. if rs.Height == prs.Height {
  565. if maj23, ok := rs.Votes.Prevotes(prs.Round).TwoThirdsMajority(); ok {
  566. peer.TrySend(StateChannel, struct{ ConsensusMessage }{&VoteSetMaj23Message{
  567. Height: prs.Height,
  568. Round: prs.Round,
  569. Type: types.VoteTypePrevote,
  570. BlockID: maj23,
  571. }})
  572. time.Sleep(conR.conS.config.PeerQueryMaj23Sleep())
  573. }
  574. }
  575. }
  576. // Maybe send Height/Round/Precommits
  577. {
  578. rs := conR.conS.GetRoundState()
  579. prs := ps.GetRoundState()
  580. if rs.Height == prs.Height {
  581. if maj23, ok := rs.Votes.Precommits(prs.Round).TwoThirdsMajority(); ok {
  582. peer.TrySend(StateChannel, struct{ ConsensusMessage }{&VoteSetMaj23Message{
  583. Height: prs.Height,
  584. Round: prs.Round,
  585. Type: types.VoteTypePrecommit,
  586. BlockID: maj23,
  587. }})
  588. time.Sleep(conR.conS.config.PeerQueryMaj23Sleep())
  589. }
  590. }
  591. }
  592. // Maybe send Height/Round/ProposalPOL
  593. {
  594. rs := conR.conS.GetRoundState()
  595. prs := ps.GetRoundState()
  596. if rs.Height == prs.Height && prs.ProposalPOLRound >= 0 {
  597. if maj23, ok := rs.Votes.Prevotes(prs.ProposalPOLRound).TwoThirdsMajority(); ok {
  598. peer.TrySend(StateChannel, struct{ ConsensusMessage }{&VoteSetMaj23Message{
  599. Height: prs.Height,
  600. Round: prs.ProposalPOLRound,
  601. Type: types.VoteTypePrevote,
  602. BlockID: maj23,
  603. }})
  604. time.Sleep(conR.conS.config.PeerQueryMaj23Sleep())
  605. }
  606. }
  607. }
  608. // Little point sending LastCommitRound/LastCommit,
  609. // These are fleeting and non-blocking.
  610. // Maybe send Height/CatchupCommitRound/CatchupCommit.
  611. {
  612. prs := ps.GetRoundState()
  613. if prs.CatchupCommitRound != -1 && 0 < prs.Height && prs.Height <= conR.conS.blockStore.Height() {
  614. commit := conR.conS.LoadCommit(prs.Height)
  615. peer.TrySend(StateChannel, struct{ ConsensusMessage }{&VoteSetMaj23Message{
  616. Height: prs.Height,
  617. Round: commit.Round(),
  618. Type: types.VoteTypePrecommit,
  619. BlockID: commit.BlockID,
  620. }})
  621. time.Sleep(conR.conS.config.PeerQueryMaj23Sleep())
  622. }
  623. }
  624. time.Sleep(conR.conS.config.PeerQueryMaj23Sleep())
  625. continue OUTER_LOOP
  626. }
  627. }
  628. // String returns a string representation of the ConsensusReactor.
  629. // NOTE: For now, it is just a hard-coded string to avoid accessing unprotected shared variables.
  630. // TODO: improve!
  631. func (conR *ConsensusReactor) String() string {
  632. // better not to access shared variables
  633. return "ConsensusReactor" // conR.StringIndented("")
  634. }
  635. // StringIndented returns an indented string representation of the ConsensusReactor
  636. func (conR *ConsensusReactor) StringIndented(indent string) string {
  637. s := "ConsensusReactor{\n"
  638. s += indent + " " + conR.conS.StringIndented(indent+" ") + "\n"
  639. for _, peer := range conR.Switch.Peers().List() {
  640. ps := peer.Data.Get(types.PeerStateKey).(*PeerState)
  641. s += indent + " " + ps.StringIndented(indent+" ") + "\n"
  642. }
  643. s += indent + "}"
  644. return s
  645. }
  646. //-----------------------------------------------------------------------------
  647. // PeerRoundState contains the known state of a peer.
  648. // NOTE: Read-only when returned by PeerState.GetRoundState().
  649. type PeerRoundState struct {
  650. Height int // Height peer is at
  651. Round int // Round peer is at, -1 if unknown.
  652. Step RoundStepType // Step peer is at
  653. StartTime time.Time // Estimated start of round 0 at this height
  654. Proposal bool // True if peer has proposal for this round
  655. ProposalBlockPartsHeader types.PartSetHeader //
  656. ProposalBlockParts *cmn.BitArray //
  657. ProposalPOLRound int // Proposal's POL round. -1 if none.
  658. ProposalPOL *cmn.BitArray // nil until ProposalPOLMessage received.
  659. Prevotes *cmn.BitArray // All votes peer has for this round
  660. Precommits *cmn.BitArray // All precommits peer has for this round
  661. LastCommitRound int // Round of commit for last height. -1 if none.
  662. LastCommit *cmn.BitArray // All commit precommits of commit for last height.
  663. CatchupCommitRound int // Round that we have commit for. Not necessarily unique. -1 if none.
  664. CatchupCommit *cmn.BitArray // All commit precommits peer has for this height & CatchupCommitRound
  665. }
  666. // String returns a string representation of the PeerRoundState
  667. func (prs PeerRoundState) String() string {
  668. return prs.StringIndented("")
  669. }
  670. // StringIndented returns a string representation of the PeerRoundState
  671. func (prs PeerRoundState) StringIndented(indent string) string {
  672. return fmt.Sprintf(`PeerRoundState{
  673. %s %v/%v/%v @%v
  674. %s Proposal %v -> %v
  675. %s POL %v (round %v)
  676. %s Prevotes %v
  677. %s Precommits %v
  678. %s LastCommit %v (round %v)
  679. %s Catchup %v (round %v)
  680. %s}`,
  681. indent, prs.Height, prs.Round, prs.Step, prs.StartTime,
  682. indent, prs.ProposalBlockPartsHeader, prs.ProposalBlockParts,
  683. indent, prs.ProposalPOL, prs.ProposalPOLRound,
  684. indent, prs.Prevotes,
  685. indent, prs.Precommits,
  686. indent, prs.LastCommit, prs.LastCommitRound,
  687. indent, prs.CatchupCommit, prs.CatchupCommitRound,
  688. indent)
  689. }
  690. //-----------------------------------------------------------------------------
  691. var (
  692. ErrPeerStateHeightRegression = errors.New("Error peer state height regression")
  693. ErrPeerStateInvalidStartTime = errors.New("Error peer state invalid startTime")
  694. )
  695. // PeerState contains the known state of a peer, including its connection
  696. // and threadsafe access to its PeerRoundState.
  697. type PeerState struct {
  698. Peer *p2p.Peer
  699. mtx sync.Mutex
  700. PeerRoundState
  701. }
  702. // NewPeerState returns a new PeerState for the given Peer
  703. func NewPeerState(peer *p2p.Peer) *PeerState {
  704. return &PeerState{
  705. Peer: peer,
  706. PeerRoundState: PeerRoundState{
  707. Round: -1,
  708. ProposalPOLRound: -1,
  709. LastCommitRound: -1,
  710. CatchupCommitRound: -1,
  711. },
  712. }
  713. }
  714. // GetRoundState returns an atomic snapshot of the PeerRoundState.
  715. // There's no point in mutating it since it won't change PeerState.
  716. func (ps *PeerState) GetRoundState() *PeerRoundState {
  717. ps.mtx.Lock()
  718. defer ps.mtx.Unlock()
  719. prs := ps.PeerRoundState // copy
  720. return &prs
  721. }
  722. // GetHeight returns an atomic snapshot of the PeerRoundState's height
  723. // used by the mempool to ensure peers are caught up before broadcasting new txs
  724. func (ps *PeerState) GetHeight() int {
  725. ps.mtx.Lock()
  726. defer ps.mtx.Unlock()
  727. return ps.PeerRoundState.Height
  728. }
  729. // SetHasProposal sets the given proposal as known for the peer.
  730. func (ps *PeerState) SetHasProposal(proposal *types.Proposal) {
  731. ps.mtx.Lock()
  732. defer ps.mtx.Unlock()
  733. if ps.Height != proposal.Height || ps.Round != proposal.Round {
  734. return
  735. }
  736. if ps.Proposal {
  737. return
  738. }
  739. ps.Proposal = true
  740. ps.ProposalBlockPartsHeader = proposal.BlockPartsHeader
  741. ps.ProposalBlockParts = cmn.NewBitArray(proposal.BlockPartsHeader.Total)
  742. ps.ProposalPOLRound = proposal.POLRound
  743. ps.ProposalPOL = nil // Nil until ProposalPOLMessage received.
  744. }
  745. // SetHasProposalBlockPart sets the given block part index as known for the peer.
  746. func (ps *PeerState) SetHasProposalBlockPart(height int, round int, index int) {
  747. ps.mtx.Lock()
  748. defer ps.mtx.Unlock()
  749. if ps.Height != height || ps.Round != round {
  750. return
  751. }
  752. ps.ProposalBlockParts.SetIndex(index, true)
  753. }
  754. // PickSendVote picks a vote and sends it to the peer.
  755. // Returns true if vote was sent.
  756. func (ps *PeerState) PickSendVote(votes types.VoteSetReader) bool {
  757. if vote, ok := ps.PickVoteToSend(votes); ok {
  758. msg := &VoteMessage{vote}
  759. return ps.Peer.Send(VoteChannel, struct{ ConsensusMessage }{msg})
  760. }
  761. return false
  762. }
  763. // PickVoteToSend picks a vote to send to the peer.
  764. // Returns true if a vote was picked.
  765. // NOTE: `votes` must be the correct Size() for the Height().
  766. func (ps *PeerState) PickVoteToSend(votes types.VoteSetReader) (vote *types.Vote, ok bool) {
  767. ps.mtx.Lock()
  768. defer ps.mtx.Unlock()
  769. if votes.Size() == 0 {
  770. return nil, false
  771. }
  772. height, round, type_, size := votes.Height(), votes.Round(), votes.Type(), votes.Size()
  773. // Lazily set data using 'votes'.
  774. if votes.IsCommit() {
  775. ps.ensureCatchupCommitRound(height, round, size)
  776. }
  777. ps.ensureVoteBitArrays(height, size)
  778. psVotes := ps.getVoteBitArray(height, round, type_)
  779. if psVotes == nil {
  780. return nil, false // Not something worth sending
  781. }
  782. if index, ok := votes.BitArray().Sub(psVotes).PickRandom(); ok {
  783. ps.setHasVote(height, round, type_, index)
  784. return votes.GetByIndex(index), true
  785. }
  786. return nil, false
  787. }
  788. func (ps *PeerState) getVoteBitArray(height, round int, type_ byte) *cmn.BitArray {
  789. if !types.IsVoteTypeValid(type_) {
  790. cmn.PanicSanity("Invalid vote type")
  791. }
  792. if ps.Height == height {
  793. if ps.Round == round {
  794. switch type_ {
  795. case types.VoteTypePrevote:
  796. return ps.Prevotes
  797. case types.VoteTypePrecommit:
  798. return ps.Precommits
  799. }
  800. }
  801. if ps.CatchupCommitRound == round {
  802. switch type_ {
  803. case types.VoteTypePrevote:
  804. return nil
  805. case types.VoteTypePrecommit:
  806. return ps.CatchupCommit
  807. }
  808. }
  809. if ps.ProposalPOLRound == round {
  810. switch type_ {
  811. case types.VoteTypePrevote:
  812. return ps.ProposalPOL
  813. case types.VoteTypePrecommit:
  814. return nil
  815. }
  816. }
  817. return nil
  818. }
  819. if ps.Height == height+1 {
  820. if ps.LastCommitRound == round {
  821. switch type_ {
  822. case types.VoteTypePrevote:
  823. return nil
  824. case types.VoteTypePrecommit:
  825. return ps.LastCommit
  826. }
  827. }
  828. return nil
  829. }
  830. return nil
  831. }
  832. // 'round': A round for which we have a +2/3 commit.
  833. func (ps *PeerState) ensureCatchupCommitRound(height, round int, numValidators int) {
  834. if ps.Height != height {
  835. return
  836. }
  837. /*
  838. NOTE: This is wrong, 'round' could change.
  839. e.g. if orig round is not the same as block LastCommit round.
  840. if ps.CatchupCommitRound != -1 && ps.CatchupCommitRound != round {
  841. cmn.PanicSanity(cmn.Fmt("Conflicting CatchupCommitRound. Height: %v, Orig: %v, New: %v", height, ps.CatchupCommitRound, round))
  842. }
  843. */
  844. if ps.CatchupCommitRound == round {
  845. return // Nothing to do!
  846. }
  847. ps.CatchupCommitRound = round
  848. if round == ps.Round {
  849. ps.CatchupCommit = ps.Precommits
  850. } else {
  851. ps.CatchupCommit = cmn.NewBitArray(numValidators)
  852. }
  853. }
  854. // EnsureVoteVitArrays ensures the bit-arrays have been allocated for tracking
  855. // what votes this peer has received.
  856. // NOTE: It's important to make sure that numValidators actually matches
  857. // what the node sees as the number of validators for height.
  858. func (ps *PeerState) EnsureVoteBitArrays(height int, numValidators int) {
  859. ps.mtx.Lock()
  860. defer ps.mtx.Unlock()
  861. ps.ensureVoteBitArrays(height, numValidators)
  862. }
  863. func (ps *PeerState) ensureVoteBitArrays(height int, numValidators int) {
  864. if ps.Height == height {
  865. if ps.Prevotes == nil {
  866. ps.Prevotes = cmn.NewBitArray(numValidators)
  867. }
  868. if ps.Precommits == nil {
  869. ps.Precommits = cmn.NewBitArray(numValidators)
  870. }
  871. if ps.CatchupCommit == nil {
  872. ps.CatchupCommit = cmn.NewBitArray(numValidators)
  873. }
  874. if ps.ProposalPOL == nil {
  875. ps.ProposalPOL = cmn.NewBitArray(numValidators)
  876. }
  877. } else if ps.Height == height+1 {
  878. if ps.LastCommit == nil {
  879. ps.LastCommit = cmn.NewBitArray(numValidators)
  880. }
  881. }
  882. }
  883. // SetHasVote sets the given vote as known by the peer
  884. func (ps *PeerState) SetHasVote(vote *types.Vote) {
  885. ps.mtx.Lock()
  886. defer ps.mtx.Unlock()
  887. ps.setHasVote(vote.Height, vote.Round, vote.Type, vote.ValidatorIndex)
  888. }
  889. func (ps *PeerState) setHasVote(height int, round int, type_ byte, index int) {
  890. logger := ps.Peer.Logger.With("peerRound", ps.Round, "height", height, "round", round)
  891. logger.Debug("setHasVote(LastCommit)", "lastCommit", ps.LastCommit, "index", index)
  892. // NOTE: some may be nil BitArrays -> no side effects.
  893. ps.getVoteBitArray(height, round, type_).SetIndex(index, true)
  894. }
  895. // ApplyNewRoundStepMessage updates the peer state for the new round.
  896. func (ps *PeerState) ApplyNewRoundStepMessage(msg *NewRoundStepMessage) {
  897. ps.mtx.Lock()
  898. defer ps.mtx.Unlock()
  899. // Ignore duplicates or decreases
  900. if CompareHRS(msg.Height, msg.Round, msg.Step, ps.Height, ps.Round, ps.Step) <= 0 {
  901. return
  902. }
  903. // Just remember these values.
  904. psHeight := ps.Height
  905. psRound := ps.Round
  906. //psStep := ps.Step
  907. psCatchupCommitRound := ps.CatchupCommitRound
  908. psCatchupCommit := ps.CatchupCommit
  909. startTime := time.Now().Add(-1 * time.Duration(msg.SecondsSinceStartTime) * time.Second)
  910. ps.Height = msg.Height
  911. ps.Round = msg.Round
  912. ps.Step = msg.Step
  913. ps.StartTime = startTime
  914. if psHeight != msg.Height || psRound != msg.Round {
  915. ps.Proposal = false
  916. ps.ProposalBlockPartsHeader = types.PartSetHeader{}
  917. ps.ProposalBlockParts = nil
  918. ps.ProposalPOLRound = -1
  919. ps.ProposalPOL = nil
  920. // We'll update the BitArray capacity later.
  921. ps.Prevotes = nil
  922. ps.Precommits = nil
  923. }
  924. if psHeight == msg.Height && psRound != msg.Round && msg.Round == psCatchupCommitRound {
  925. // Peer caught up to CatchupCommitRound.
  926. // Preserve psCatchupCommit!
  927. // NOTE: We prefer to use prs.Precommits if
  928. // pr.Round matches pr.CatchupCommitRound.
  929. ps.Precommits = psCatchupCommit
  930. }
  931. if psHeight != msg.Height {
  932. // Shift Precommits to LastCommit.
  933. if psHeight+1 == msg.Height && psRound == msg.LastCommitRound {
  934. ps.LastCommitRound = msg.LastCommitRound
  935. ps.LastCommit = ps.Precommits
  936. } else {
  937. ps.LastCommitRound = msg.LastCommitRound
  938. ps.LastCommit = nil
  939. }
  940. // We'll update the BitArray capacity later.
  941. ps.CatchupCommitRound = -1
  942. ps.CatchupCommit = nil
  943. }
  944. }
  945. // ApplyCommitStepMessage updates the peer state for the new commit.
  946. func (ps *PeerState) ApplyCommitStepMessage(msg *CommitStepMessage) {
  947. ps.mtx.Lock()
  948. defer ps.mtx.Unlock()
  949. if ps.Height != msg.Height {
  950. return
  951. }
  952. ps.ProposalBlockPartsHeader = msg.BlockPartsHeader
  953. ps.ProposalBlockParts = msg.BlockParts
  954. }
  955. // ApplyProposalPOLMessage updates the peer state for the new proposal POL.
  956. func (ps *PeerState) ApplyProposalPOLMessage(msg *ProposalPOLMessage) {
  957. ps.mtx.Lock()
  958. defer ps.mtx.Unlock()
  959. if ps.Height != msg.Height {
  960. return
  961. }
  962. if ps.ProposalPOLRound != msg.ProposalPOLRound {
  963. return
  964. }
  965. // TODO: Merge onto existing ps.ProposalPOL?
  966. // We might have sent some prevotes in the meantime.
  967. ps.ProposalPOL = msg.ProposalPOL
  968. }
  969. // ApplyHasVoteMessage updates the peer state for the new vote.
  970. func (ps *PeerState) ApplyHasVoteMessage(msg *HasVoteMessage) {
  971. ps.mtx.Lock()
  972. defer ps.mtx.Unlock()
  973. if ps.Height != msg.Height {
  974. return
  975. }
  976. ps.setHasVote(msg.Height, msg.Round, msg.Type, msg.Index)
  977. }
  978. // ApplyVoteSetBitsMessage updates the peer state for the bit-array of votes
  979. // it claims to have for the corresponding BlockID.
  980. // `ourVotes` is a BitArray of votes we have for msg.BlockID
  981. // NOTE: if ourVotes is nil (e.g. msg.Height < rs.Height),
  982. // we conservatively overwrite ps's votes w/ msg.Votes.
  983. func (ps *PeerState) ApplyVoteSetBitsMessage(msg *VoteSetBitsMessage, ourVotes *cmn.BitArray) {
  984. ps.mtx.Lock()
  985. defer ps.mtx.Unlock()
  986. votes := ps.getVoteBitArray(msg.Height, msg.Round, msg.Type)
  987. if votes != nil {
  988. if ourVotes == nil {
  989. votes.Update(msg.Votes)
  990. } else {
  991. otherVotes := votes.Sub(ourVotes)
  992. hasVotes := otherVotes.Or(msg.Votes)
  993. votes.Update(hasVotes)
  994. }
  995. }
  996. }
  997. // String returns a string representation of the PeerState
  998. func (ps *PeerState) String() string {
  999. return ps.StringIndented("")
  1000. }
  1001. // StringIndented returns a string representation of the PeerState
  1002. func (ps *PeerState) StringIndented(indent string) string {
  1003. return fmt.Sprintf(`PeerState{
  1004. %s Key %v
  1005. %s PRS %v
  1006. %s}`,
  1007. indent, ps.Peer.Key,
  1008. indent, ps.PeerRoundState.StringIndented(indent+" "),
  1009. indent)
  1010. }
  1011. //-----------------------------------------------------------------------------
  1012. // Messages
  1013. const (
  1014. msgTypeNewRoundStep = byte(0x01)
  1015. msgTypeCommitStep = byte(0x02)
  1016. msgTypeProposal = byte(0x11)
  1017. msgTypeProposalPOL = byte(0x12)
  1018. msgTypeBlockPart = byte(0x13) // both block & POL
  1019. msgTypeVote = byte(0x14)
  1020. msgTypeHasVote = byte(0x15)
  1021. msgTypeVoteSetMaj23 = byte(0x16)
  1022. msgTypeVoteSetBits = byte(0x17)
  1023. )
  1024. // ConsensusMessage is a message that can be sent and received on the ConsensusReactor
  1025. type ConsensusMessage interface{}
  1026. var _ = wire.RegisterInterface(
  1027. struct{ ConsensusMessage }{},
  1028. wire.ConcreteType{&NewRoundStepMessage{}, msgTypeNewRoundStep},
  1029. wire.ConcreteType{&CommitStepMessage{}, msgTypeCommitStep},
  1030. wire.ConcreteType{&ProposalMessage{}, msgTypeProposal},
  1031. wire.ConcreteType{&ProposalPOLMessage{}, msgTypeProposalPOL},
  1032. wire.ConcreteType{&BlockPartMessage{}, msgTypeBlockPart},
  1033. wire.ConcreteType{&VoteMessage{}, msgTypeVote},
  1034. wire.ConcreteType{&HasVoteMessage{}, msgTypeHasVote},
  1035. wire.ConcreteType{&VoteSetMaj23Message{}, msgTypeVoteSetMaj23},
  1036. wire.ConcreteType{&VoteSetBitsMessage{}, msgTypeVoteSetBits},
  1037. )
  1038. // DecodeMessage decodes the given bytes into a ConsensusMessage.
  1039. // TODO: check for unnecessary extra bytes at the end.
  1040. func DecodeMessage(bz []byte) (msgType byte, msg ConsensusMessage, err error) {
  1041. msgType = bz[0]
  1042. n := new(int)
  1043. r := bytes.NewReader(bz)
  1044. msgI := wire.ReadBinary(struct{ ConsensusMessage }{}, r, maxConsensusMessageSize, n, &err)
  1045. msg = msgI.(struct{ ConsensusMessage }).ConsensusMessage
  1046. return
  1047. }
  1048. //-------------------------------------
  1049. // NewRoundStepMessage is sent for every step taken in the ConsensusState.
  1050. // For every height/round/step transition
  1051. type NewRoundStepMessage struct {
  1052. Height int
  1053. Round int
  1054. Step RoundStepType
  1055. SecondsSinceStartTime int
  1056. LastCommitRound int
  1057. }
  1058. // String returns a string representation.
  1059. func (m *NewRoundStepMessage) String() string {
  1060. return fmt.Sprintf("[NewRoundStep H:%v R:%v S:%v LCR:%v]",
  1061. m.Height, m.Round, m.Step, m.LastCommitRound)
  1062. }
  1063. //-------------------------------------
  1064. // CommitStepMessage is sent when a block is committed.
  1065. type CommitStepMessage struct {
  1066. Height int
  1067. BlockPartsHeader types.PartSetHeader
  1068. BlockParts *cmn.BitArray
  1069. }
  1070. // String returns a string representation.
  1071. func (m *CommitStepMessage) String() string {
  1072. return fmt.Sprintf("[CommitStep H:%v BP:%v BA:%v]", m.Height, m.BlockPartsHeader, m.BlockParts)
  1073. }
  1074. //-------------------------------------
  1075. // ProposalMessage is sent when a new block is proposed.
  1076. type ProposalMessage struct {
  1077. Proposal *types.Proposal
  1078. }
  1079. // String returns a string representation.
  1080. func (m *ProposalMessage) String() string {
  1081. return fmt.Sprintf("[Proposal %v]", m.Proposal)
  1082. }
  1083. //-------------------------------------
  1084. // ProposalPOLMessage is sent when a previous proposal is re-proposed.
  1085. type ProposalPOLMessage struct {
  1086. Height int
  1087. ProposalPOLRound int
  1088. ProposalPOL *cmn.BitArray
  1089. }
  1090. // String returns a string representation.
  1091. func (m *ProposalPOLMessage) String() string {
  1092. return fmt.Sprintf("[ProposalPOL H:%v POLR:%v POL:%v]", m.Height, m.ProposalPOLRound, m.ProposalPOL)
  1093. }
  1094. //-------------------------------------
  1095. // BlockPartMessage is sent when gossipping a piece of the proposed block.
  1096. type BlockPartMessage struct {
  1097. Height int
  1098. Round int
  1099. Part *types.Part
  1100. }
  1101. // String returns a string representation.
  1102. func (m *BlockPartMessage) String() string {
  1103. return fmt.Sprintf("[BlockPart H:%v R:%v P:%v]", m.Height, m.Round, m.Part)
  1104. }
  1105. //-------------------------------------
  1106. // VoteMessage is sent when voting for a proposal (or lack thereof).
  1107. type VoteMessage struct {
  1108. Vote *types.Vote
  1109. }
  1110. // String returns a string representation.
  1111. func (m *VoteMessage) String() string {
  1112. return fmt.Sprintf("[Vote %v]", m.Vote)
  1113. }
  1114. //-------------------------------------
  1115. // HasVoteMessage is sent to indicate that a particular vote has been received.
  1116. type HasVoteMessage struct {
  1117. Height int
  1118. Round int
  1119. Type byte
  1120. Index int
  1121. }
  1122. // String returns a string representation.
  1123. func (m *HasVoteMessage) String() string {
  1124. return fmt.Sprintf("[HasVote VI:%v V:{%v/%02d/%v} VI:%v]", m.Index, m.Height, m.Round, m.Type, m.Index)
  1125. }
  1126. //-------------------------------------
  1127. // VoteSetMaj23Message is sent to indicate that a given BlockID has seen +2/3 votes.
  1128. type VoteSetMaj23Message struct {
  1129. Height int
  1130. Round int
  1131. Type byte
  1132. BlockID types.BlockID
  1133. }
  1134. // String returns a string representation.
  1135. func (m *VoteSetMaj23Message) String() string {
  1136. return fmt.Sprintf("[VSM23 %v/%02d/%v %v]", m.Height, m.Round, m.Type, m.BlockID)
  1137. }
  1138. //-------------------------------------
  1139. // VoteSetBitsMessage is sent to communicate the bit-array of votes seen for the BlockID.
  1140. type VoteSetBitsMessage struct {
  1141. Height int
  1142. Round int
  1143. Type byte
  1144. BlockID types.BlockID
  1145. Votes *cmn.BitArray
  1146. }
  1147. // String returns a string representation.
  1148. func (m *VoteSetBitsMessage) String() string {
  1149. return fmt.Sprintf("[VSB %v/%02d/%v %v %v]", m.Height, m.Round, m.Type, m.BlockID, m.Votes)
  1150. }