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.

1363 lines
41 KiB

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