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.

1238 lines
36 KiB

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