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.

1014 lines
29 KiB

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
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
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
10 years ago
10 years ago
10 years ago
10 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
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
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. "github.com/tendermint/tendermint/binary"
  10. bc "github.com/tendermint/tendermint/blockchain"
  11. . "github.com/tendermint/tendermint/common"
  12. . "github.com/tendermint/tendermint/consensus/types"
  13. "github.com/tendermint/tendermint/events"
  14. "github.com/tendermint/tendermint/p2p"
  15. sm "github.com/tendermint/tendermint/state"
  16. "github.com/tendermint/tendermint/types"
  17. )
  18. const (
  19. StateChannel = byte(0x20)
  20. DataChannel = byte(0x21)
  21. VoteChannel = byte(0x22)
  22. PeerStateKey = "ConsensusReactor.peerState"
  23. peerGossipSleepDuration = 100 * time.Millisecond // Time to sleep if there's nothing to send.
  24. )
  25. //-----------------------------------------------------------------------------
  26. type ConsensusReactor struct {
  27. p2p.BaseReactor
  28. blockStore *bc.BlockStore
  29. conS *ConsensusState
  30. fastSync bool
  31. evsw events.Fireable
  32. }
  33. func NewConsensusReactor(consensusState *ConsensusState, blockStore *bc.BlockStore, fastSync bool) *ConsensusReactor {
  34. conR := &ConsensusReactor{
  35. blockStore: blockStore,
  36. conS: consensusState,
  37. fastSync: fastSync,
  38. }
  39. conR.BaseReactor = *p2p.NewBaseReactor(log, "ConsensusReactor", conR)
  40. return conR
  41. }
  42. func (conR *ConsensusReactor) AfterStart() {
  43. log.Notice("ConsensusReactor ", "fastSync", conR.fastSync)
  44. if !conR.fastSync {
  45. conR.conS.Start()
  46. }
  47. go conR.broadcastNewRoundStepRoutine()
  48. }
  49. func (conR *ConsensusReactor) AfterStop() {
  50. conR.conS.Stop()
  51. }
  52. // Switch from the fast_sync to the consensus:
  53. // reset the state, turn off fast_sync, start the consensus-state-machine
  54. func (conR *ConsensusReactor) SwitchToConsensus(state *sm.State) {
  55. log.Notice("SwitchToConsensus")
  56. // NOTE: The line below causes broadcastNewRoundStepRoutine() to
  57. // broadcast a NewRoundStepMessage.
  58. conR.conS.updateToState(state, false)
  59. conR.fastSync = false
  60. conR.conS.Start()
  61. }
  62. // Implements Reactor
  63. func (conR *ConsensusReactor) GetChannels() []*p2p.ChannelDescriptor {
  64. // TODO optimize
  65. return []*p2p.ChannelDescriptor{
  66. &p2p.ChannelDescriptor{
  67. Id: StateChannel,
  68. Priority: 5,
  69. SendQueueCapacity: 100,
  70. },
  71. &p2p.ChannelDescriptor{
  72. Id: DataChannel,
  73. Priority: 5,
  74. SendQueueCapacity: 2,
  75. },
  76. &p2p.ChannelDescriptor{
  77. Id: VoteChannel,
  78. Priority: 5,
  79. SendQueueCapacity: 40,
  80. },
  81. }
  82. }
  83. // Implements Reactor
  84. func (conR *ConsensusReactor) AddPeer(peer *p2p.Peer) {
  85. if !conR.IsRunning() {
  86. return
  87. }
  88. // Create peerState for peer
  89. peerState := NewPeerState(peer)
  90. peer.Data.Set(PeerStateKey, peerState)
  91. // Begin gossip routines for this peer.
  92. go conR.gossipDataRoutine(peer, peerState)
  93. go conR.gossipVotesRoutine(peer, peerState)
  94. // Send our state to peer.
  95. // If we're fast_syncing, broadcast a RoundStepMessage later upon SwitchToConsensus().
  96. if !conR.fastSync {
  97. conR.sendNewRoundStepMessage(peer)
  98. }
  99. }
  100. // Implements Reactor
  101. func (conR *ConsensusReactor) RemovePeer(peer *p2p.Peer, reason interface{}) {
  102. if !conR.IsRunning() {
  103. return
  104. }
  105. // TODO
  106. //peer.Data.Get(PeerStateKey).(*PeerState).Disconnect()
  107. }
  108. // Implements Reactor
  109. // NOTE: We process these messages even when we're fast_syncing.
  110. func (conR *ConsensusReactor) Receive(chId byte, peer *p2p.Peer, msgBytes []byte) {
  111. if !conR.IsRunning() {
  112. log.Debug("Receive", "channel", chId, "peer", peer, "bytes", msgBytes)
  113. return
  114. }
  115. // Get round state
  116. rs := conR.conS.GetRoundState()
  117. ps := peer.Data.Get(PeerStateKey).(*PeerState)
  118. _, msg, err := DecodeMessage(msgBytes)
  119. if err != nil {
  120. log.Warn("Error decoding message", "channel", chId, "peer", peer, "msg", msg, "error", err, "bytes", msgBytes)
  121. return
  122. }
  123. log.Debug("Receive", "channel", chId, "peer", peer, "msg", msg, "rsHeight", rs.Height)
  124. switch chId {
  125. case StateChannel:
  126. switch msg := msg.(type) {
  127. case *NewRoundStepMessage:
  128. ps.ApplyNewRoundStepMessage(msg, rs)
  129. case *CommitStepMessage:
  130. ps.ApplyCommitStepMessage(msg)
  131. case *HasVoteMessage:
  132. ps.ApplyHasVoteMessage(msg)
  133. default:
  134. log.Warn(Fmt("Unknown message type %v", reflect.TypeOf(msg)))
  135. }
  136. case DataChannel:
  137. if conR.fastSync {
  138. log.Warn("Ignoring message received during fastSync", "msg", msg)
  139. return
  140. }
  141. switch msg := msg.(type) {
  142. case *ProposalMessage:
  143. ps.SetHasProposal(msg.Proposal)
  144. err = conR.conS.SetProposal(msg.Proposal)
  145. case *ProposalPOLMessage:
  146. ps.ApplyProposalPOLMessage(msg)
  147. case *BlockPartMessage:
  148. ps.SetHasProposalBlockPart(msg.Height, msg.Round, msg.Part.Proof.Index)
  149. _, err = conR.conS.AddProposalBlockPart(msg.Height, msg.Part)
  150. default:
  151. log.Warn(Fmt("Unknown message type %v", reflect.TypeOf(msg)))
  152. }
  153. case VoteChannel:
  154. if conR.fastSync {
  155. log.Warn("Ignoring message received during fastSync", "msg", msg)
  156. return
  157. }
  158. switch msg := msg.(type) {
  159. case *VoteMessage:
  160. vote := msg.Vote
  161. var validators *sm.ValidatorSet
  162. if rs.Height == vote.Height {
  163. validators = rs.Validators
  164. } else if rs.Height == vote.Height+1 {
  165. if !(rs.Step == RoundStepNewHeight && vote.Type == types.VoteTypePrecommit) {
  166. return // Wrong height, not a LastCommit straggler commit.
  167. }
  168. validators = rs.LastValidators
  169. } else {
  170. return // Wrong height. Not necessarily a bad peer.
  171. }
  172. // We have vote/validators. Height may not be rs.Height
  173. address, _ := validators.GetByIndex(msg.ValidatorIndex)
  174. added, index, err := conR.conS.AddVote(address, vote, peer.Key)
  175. if err != nil {
  176. // If conflicting sig, broadcast evidence tx for slashing. Else punish peer.
  177. if errDupe, ok := err.(*types.ErrVoteConflictingSignature); ok {
  178. log.Warn("Found conflicting vote. Publish evidence")
  179. evidenceTx := &types.DupeoutTx{
  180. Address: address,
  181. VoteA: *errDupe.VoteA,
  182. VoteB: *errDupe.VoteB,
  183. }
  184. conR.conS.mempoolReactor.BroadcastTx(evidenceTx) // shouldn't need to check returned err
  185. } else {
  186. // Probably an invalid signature. Bad peer.
  187. log.Warn("Error attempting to add vote", "error", err)
  188. // TODO: punish peer
  189. }
  190. }
  191. ps.EnsureVoteBitArrays(rs.Height, rs.Validators.Size())
  192. ps.EnsureVoteBitArrays(rs.Height-1, rs.LastCommit.Size())
  193. ps.SetHasVote(vote, index)
  194. if added {
  195. // If rs.Height == vote.Height && rs.Round < vote.Round,
  196. // the peer is sending us CatchupCommit precommits.
  197. // We could make note of this and help filter in broadcastHasVoteMessage().
  198. conR.broadcastHasVoteMessage(vote, index)
  199. }
  200. default:
  201. log.Warn(Fmt("Unknown message type %v", reflect.TypeOf(msg)))
  202. }
  203. default:
  204. log.Warn(Fmt("Unknown channel %X", chId))
  205. }
  206. if err != nil {
  207. log.Warn("Error in Receive()", "error", err)
  208. }
  209. }
  210. // Broadcasts HasVoteMessage to peers that care.
  211. func (conR *ConsensusReactor) broadcastHasVoteMessage(vote *types.Vote, index int) {
  212. msg := &HasVoteMessage{
  213. Height: vote.Height,
  214. Round: vote.Round,
  215. Type: vote.Type,
  216. Index: index,
  217. }
  218. conR.Switch.Broadcast(StateChannel, msg)
  219. /*
  220. // TODO: Make this broadcast more selective.
  221. for _, peer := range conR.Switch.Peers().List() {
  222. ps := peer.Data.Get(PeerStateKey).(*PeerState)
  223. prs := ps.GetRoundState()
  224. if prs.Height == vote.Height {
  225. // TODO: Also filter on round?
  226. peer.TrySend(StateChannel, msg)
  227. } else {
  228. // Height doesn't match
  229. // TODO: check a field, maybe CatchupCommitRound?
  230. // TODO: But that requires changing the struct field comment.
  231. }
  232. }
  233. */
  234. }
  235. // Sets our private validator account for signing votes.
  236. func (conR *ConsensusReactor) SetPrivValidator(priv *sm.PrivValidator) {
  237. conR.conS.SetPrivValidator(priv)
  238. }
  239. // implements events.Eventable
  240. func (conR *ConsensusReactor) SetFireable(evsw events.Fireable) {
  241. conR.evsw = evsw
  242. conR.conS.SetFireable(evsw)
  243. }
  244. //--------------------------------------
  245. func makeRoundStepMessages(rs *RoundState) (nrsMsg *NewRoundStepMessage, csMsg *CommitStepMessage) {
  246. nrsMsg = &NewRoundStepMessage{
  247. Height: rs.Height,
  248. Round: rs.Round,
  249. Step: rs.Step,
  250. SecondsSinceStartTime: int(time.Now().Sub(rs.StartTime).Seconds()),
  251. LastCommitRound: rs.LastCommit.Round(),
  252. }
  253. if rs.Step == RoundStepCommit {
  254. csMsg = &CommitStepMessage{
  255. Height: rs.Height,
  256. BlockPartsHeader: rs.ProposalBlockParts.Header(),
  257. BlockParts: rs.ProposalBlockParts.BitArray(),
  258. }
  259. }
  260. return
  261. }
  262. // Listens for changes to the ConsensusState.Step by pulling
  263. // on conR.conS.NewStepCh().
  264. func (conR *ConsensusReactor) broadcastNewRoundStepRoutine() {
  265. for {
  266. // Get RoundState with new Step or quit.
  267. var rs *RoundState
  268. select {
  269. case rs = <-conR.conS.NewStepCh():
  270. case <-conR.Quit:
  271. return
  272. }
  273. nrsMsg, csMsg := makeRoundStepMessages(rs)
  274. if nrsMsg != nil {
  275. conR.Switch.Broadcast(StateChannel, nrsMsg)
  276. }
  277. if csMsg != nil {
  278. conR.Switch.Broadcast(StateChannel, csMsg)
  279. }
  280. }
  281. }
  282. func (conR *ConsensusReactor) sendNewRoundStepMessage(peer *p2p.Peer) {
  283. rs := conR.conS.GetRoundState()
  284. nrsMsg, csMsg := makeRoundStepMessages(rs)
  285. if nrsMsg != nil {
  286. peer.Send(StateChannel, nrsMsg)
  287. }
  288. if csMsg != nil {
  289. peer.Send(StateChannel, csMsg)
  290. }
  291. }
  292. func (conR *ConsensusReactor) gossipDataRoutine(peer *p2p.Peer, ps *PeerState) {
  293. log := log.New("peer", peer.Key)
  294. OUTER_LOOP:
  295. for {
  296. // Manage disconnects from self or peer.
  297. if !peer.IsRunning() || !conR.IsRunning() {
  298. log.Notice(Fmt("Stopping gossipDataRoutine for %v.", peer))
  299. return
  300. }
  301. rs := conR.conS.GetRoundState()
  302. prs := ps.GetRoundState()
  303. // Send proposal Block parts?
  304. if rs.ProposalBlockParts.HasHeader(prs.ProposalBlockPartsHeader) {
  305. //log.Info("ProposalBlockParts matched", "blockParts", prs.ProposalBlockParts)
  306. if index, ok := rs.ProposalBlockParts.BitArray().Sub(prs.ProposalBlockParts.Copy()).PickRandom(); ok {
  307. part := rs.ProposalBlockParts.GetPart(index)
  308. msg := &BlockPartMessage{
  309. Height: rs.Height, // This tells peer that this part applies to us.
  310. Round: rs.Round, // This tells peer that this part applies to us.
  311. Part: part,
  312. }
  313. peer.Send(DataChannel, msg)
  314. ps.SetHasProposalBlockPart(prs.Height, prs.Round, index)
  315. continue OUTER_LOOP
  316. }
  317. }
  318. // If the peer is on a previous height, help catch up.
  319. if (0 < prs.Height) && (prs.Height < rs.Height) {
  320. //log.Info("Data catchup", "height", rs.Height, "peerHeight", prs.Height, "peerProposalBlockParts", prs.ProposalBlockParts)
  321. if index, ok := prs.ProposalBlockParts.Not().PickRandom(); ok {
  322. // Ensure that the peer's PartSetHeader is correct
  323. blockMeta := conR.blockStore.LoadBlockMeta(prs.Height)
  324. if !blockMeta.PartsHeader.Equals(prs.ProposalBlockPartsHeader) {
  325. log.Info("Peer ProposalBlockPartsHeader mismatch, sleeping",
  326. "peerHeight", prs.Height, "blockPartsHeader", blockMeta.PartsHeader, "peerBlockPartsHeader", prs.ProposalBlockPartsHeader)
  327. time.Sleep(peerGossipSleepDuration)
  328. continue OUTER_LOOP
  329. }
  330. // Load the part
  331. part := conR.blockStore.LoadBlockPart(prs.Height, index)
  332. if part == nil {
  333. log.Warn("Could not load part", "index", index,
  334. "peerHeight", prs.Height, "blockPartsHeader", blockMeta.PartsHeader, "peerBlockPartsHeader", prs.ProposalBlockPartsHeader)
  335. time.Sleep(peerGossipSleepDuration)
  336. continue OUTER_LOOP
  337. }
  338. // Send the part
  339. msg := &BlockPartMessage{
  340. Height: prs.Height, // Not our height, so it doesn't matter.
  341. Round: prs.Round, // Not our height, so it doesn't matter.
  342. Part: part,
  343. }
  344. peer.Send(DataChannel, msg)
  345. ps.SetHasProposalBlockPart(prs.Height, prs.Round, index)
  346. continue OUTER_LOOP
  347. } else {
  348. //log.Info("No parts to send in catch-up, sleeping")
  349. time.Sleep(peerGossipSleepDuration)
  350. continue OUTER_LOOP
  351. }
  352. }
  353. // If height and round don't match, sleep.
  354. if (rs.Height != prs.Height) || (rs.Round != prs.Round) {
  355. //log.Info("Peer Height|Round mismatch, sleeping", "peerHeight", prs.Height, "peerRound", prs.Round, "peer", peer)
  356. time.Sleep(peerGossipSleepDuration)
  357. continue OUTER_LOOP
  358. }
  359. // By here, height and round match.
  360. // Proposal block parts were already matched and sent if any were wanted.
  361. // (These can match on hash so the round doesn't matter)
  362. // Now consider sending other things, like the Proposal itself.
  363. // Send Proposal && ProposalPOL BitArray?
  364. if rs.Proposal != nil && !prs.Proposal {
  365. // Proposal
  366. {
  367. msg := &ProposalMessage{Proposal: rs.Proposal}
  368. peer.Send(DataChannel, msg)
  369. ps.SetHasProposal(rs.Proposal)
  370. }
  371. // ProposalPOL.
  372. // Peer must receive ProposalMessage first.
  373. // rs.Proposal was validated, so rs.Proposal.POLRound <= rs.Round,
  374. // so we definitely have rs.Votes.Prevotes(rs.Proposal.POLRound).
  375. if 0 <= rs.Proposal.POLRound {
  376. msg := &ProposalPOLMessage{
  377. Height: rs.Height,
  378. ProposalPOLRound: rs.Proposal.POLRound,
  379. ProposalPOL: rs.Votes.Prevotes(rs.Proposal.POLRound).BitArray(),
  380. }
  381. peer.Send(DataChannel, msg)
  382. }
  383. continue OUTER_LOOP
  384. }
  385. // Nothing to do. Sleep.
  386. time.Sleep(peerGossipSleepDuration)
  387. continue OUTER_LOOP
  388. }
  389. }
  390. func (conR *ConsensusReactor) gossipVotesRoutine(peer *p2p.Peer, ps *PeerState) {
  391. log := log.New("peer", peer.Key)
  392. // Simple hack to throttle logs upon sleep.
  393. var sleeping = 0
  394. OUTER_LOOP:
  395. for {
  396. // Manage disconnects from self or peer.
  397. if !peer.IsRunning() || !conR.IsRunning() {
  398. log.Notice(Fmt("Stopping gossipVotesRoutine for %v.", peer))
  399. return
  400. }
  401. rs := conR.conS.GetRoundState()
  402. prs := ps.GetRoundState()
  403. switch sleeping {
  404. case 1: // First sleep
  405. sleeping = 2
  406. case 2: // No more sleep
  407. sleeping = 0
  408. }
  409. log.Debug("gossipVotesRoutine", "rsHeight", rs.Height, "rsRound", rs.Round,
  410. "prsHeight", prs.Height, "prsRound", prs.Round, "prsStep", prs.Step)
  411. // If height matches, then send LastCommit, Prevotes, Precommits.
  412. if rs.Height == prs.Height {
  413. // If there are lastCommits to send...
  414. if prs.Step == RoundStepNewHeight {
  415. if ps.PickSendVote(rs.LastCommit) {
  416. log.Info("Picked rs.LastCommit to send")
  417. continue OUTER_LOOP
  418. }
  419. }
  420. // If there are prevotes to send...
  421. if rs.Round == prs.Round && prs.Step <= RoundStepPrevote {
  422. if ps.PickSendVote(rs.Votes.Prevotes(rs.Round)) {
  423. log.Info("Picked rs.Prevotes(rs.Round) to send")
  424. continue OUTER_LOOP
  425. }
  426. }
  427. // If there are precommits to send...
  428. if rs.Round == prs.Round && prs.Step <= RoundStepPrecommit {
  429. if ps.PickSendVote(rs.Votes.Precommits(rs.Round)) {
  430. log.Info("Picked rs.Precommits(rs.Round) to send")
  431. continue OUTER_LOOP
  432. }
  433. }
  434. // If there are prevotes to send for the last round...
  435. if rs.Round == prs.Round+1 && prs.Step <= RoundStepPrevote {
  436. if ps.PickSendVote(rs.Votes.Prevotes(prs.Round)) {
  437. log.Info("Picked rs.Prevotes(prs.Round) to send")
  438. continue OUTER_LOOP
  439. }
  440. }
  441. // If there are precommits to send for the last round...
  442. if rs.Round == prs.Round+1 && prs.Step <= RoundStepPrecommit {
  443. if ps.PickSendVote(rs.Votes.Precommits(prs.Round)) {
  444. log.Info("Picked rs.Precommits(prs.Round) to send")
  445. continue OUTER_LOOP
  446. }
  447. }
  448. // If there are POLPrevotes to send...
  449. if 0 <= prs.ProposalPOLRound {
  450. if polPrevotes := rs.Votes.Prevotes(prs.ProposalPOLRound); polPrevotes != nil {
  451. if ps.PickSendVote(polPrevotes) {
  452. log.Info("Picked rs.Prevotes(prs.ProposalPOLRound) to send")
  453. continue OUTER_LOOP
  454. }
  455. }
  456. }
  457. }
  458. // Special catchup logic.
  459. // If peer is lagging by height 1, send LastCommit.
  460. if prs.Height != 0 && rs.Height == prs.Height+1 {
  461. if ps.PickSendVote(rs.LastCommit) {
  462. log.Info("Picked rs.LastCommit to send")
  463. continue OUTER_LOOP
  464. }
  465. }
  466. // Catchup logic
  467. // If peer is lagging by more than 1, send Validation.
  468. if prs.Height != 0 && rs.Height >= prs.Height+2 {
  469. // Load the block validation for prs.Height,
  470. // which contains precommit signatures for prs.Height.
  471. validation := conR.blockStore.LoadBlockValidation(prs.Height)
  472. log.Info("Loaded BlockValidation for catch-up", "height", prs.Height, "validation", validation)
  473. if ps.PickSendVote(validation) {
  474. log.Info("Picked Catchup validation to send")
  475. continue OUTER_LOOP
  476. }
  477. }
  478. if sleeping == 0 {
  479. // We sent nothing. Sleep...
  480. sleeping = 1
  481. log.Info("No votes to send, sleeping", "peer", peer,
  482. "localPV", rs.Votes.Prevotes(rs.Round).BitArray(), "peerPV", prs.Prevotes,
  483. "localPC", rs.Votes.Precommits(rs.Round).BitArray(), "peerPC", prs.Precommits)
  484. } else if sleeping == 2 {
  485. // Continued sleep...
  486. sleeping = 1
  487. }
  488. time.Sleep(peerGossipSleepDuration)
  489. continue OUTER_LOOP
  490. }
  491. }
  492. //-----------------------------------------------------------------------------
  493. // Read only when returned by PeerState.GetRoundState().
  494. type PeerRoundState struct {
  495. Height int // Height peer is at
  496. Round int // Round peer is at
  497. Step RoundStepType // Step peer is at
  498. StartTime time.Time // Estimated start of round 0 at this height
  499. Proposal bool // True if peer has proposal for this round
  500. ProposalBlockPartsHeader types.PartSetHeader //
  501. ProposalBlockParts *BitArray //
  502. ProposalPOLRound int // -1 if none
  503. ProposalPOL *BitArray // nil until ProposalPOLMessage received.
  504. Prevotes *BitArray // All votes peer has for this round
  505. Precommits *BitArray // All precommits peer has for this round
  506. LastCommitRound int // Round of commit for last height.
  507. LastCommit *BitArray // All commit precommits of commit for last height.
  508. CatchupCommitRound int // Round that we believe commit round is.
  509. CatchupCommit *BitArray // All commit precommits peer has for this height
  510. }
  511. //-----------------------------------------------------------------------------
  512. var (
  513. ErrPeerStateHeightRegression = errors.New("Error peer state height regression")
  514. ErrPeerStateInvalidStartTime = errors.New("Error peer state invalid startTime")
  515. )
  516. type PeerState struct {
  517. Peer *p2p.Peer
  518. mtx sync.Mutex
  519. PeerRoundState
  520. }
  521. func NewPeerState(peer *p2p.Peer) *PeerState {
  522. return &PeerState{Peer: peer}
  523. }
  524. // Returns an atomic snapshot of the PeerRoundState.
  525. // There's no point in mutating it since it won't change PeerState.
  526. func (ps *PeerState) GetRoundState() *PeerRoundState {
  527. ps.mtx.Lock()
  528. defer ps.mtx.Unlock()
  529. prs := ps.PeerRoundState // copy
  530. return &prs
  531. }
  532. func (ps *PeerState) SetHasProposal(proposal *Proposal) {
  533. ps.mtx.Lock()
  534. defer ps.mtx.Unlock()
  535. if ps.Height != proposal.Height || ps.Round != proposal.Round {
  536. return
  537. }
  538. if ps.Proposal {
  539. return
  540. }
  541. ps.Proposal = true
  542. ps.ProposalBlockPartsHeader = proposal.BlockPartsHeader
  543. ps.ProposalBlockParts = NewBitArray(proposal.BlockPartsHeader.Total)
  544. ps.ProposalPOLRound = proposal.POLRound
  545. ps.ProposalPOL = nil // Nil until ProposalPOLMessage received.
  546. }
  547. func (ps *PeerState) SetHasProposalBlockPart(height int, round int, index int) {
  548. ps.mtx.Lock()
  549. defer ps.mtx.Unlock()
  550. if ps.Height != height || ps.Round != round {
  551. return
  552. }
  553. ps.ProposalBlockParts.SetIndex(index, true)
  554. }
  555. // Convenience function to send vote to peer.
  556. // Returns true if vote was sent.
  557. func (ps *PeerState) PickSendVote(votes types.VoteSetReader) (ok bool) {
  558. if index, vote, ok := ps.PickVoteToSend(votes); ok {
  559. msg := &VoteMessage{index, vote}
  560. ps.Peer.Send(VoteChannel, msg)
  561. return true
  562. }
  563. return false
  564. }
  565. // votes: Must be the correct Size() for the Height().
  566. func (ps *PeerState) PickVoteToSend(votes types.VoteSetReader) (index int, vote *types.Vote, ok bool) {
  567. ps.mtx.Lock()
  568. defer ps.mtx.Unlock()
  569. if votes.Size() == 0 {
  570. return 0, nil, false
  571. }
  572. height, round, type_, size := votes.Height(), votes.Round(), votes.Type(), votes.Size()
  573. // Lazily set data using 'votes'.
  574. if votes.IsCommit() {
  575. ps.ensureCatchupCommitRound(height, round, size)
  576. }
  577. ps.ensureVoteBitArrays(height, size)
  578. psVotes := ps.getVoteBitArray(height, round, type_)
  579. if psVotes == nil {
  580. return 0, nil, false // Not something worth sending
  581. }
  582. if index, ok := votes.BitArray().Sub(psVotes).PickRandom(); ok {
  583. ps.setHasVote(height, round, type_, index)
  584. return index, votes.GetByIndex(index), true
  585. }
  586. return 0, nil, false
  587. }
  588. func (ps *PeerState) getVoteBitArray(height, round int, type_ byte) *BitArray {
  589. if ps.Height == height {
  590. if ps.Round == round {
  591. switch type_ {
  592. case types.VoteTypePrevote:
  593. return ps.Prevotes
  594. case types.VoteTypePrecommit:
  595. return ps.Precommits
  596. default:
  597. PanicSanity(Fmt("Unexpected vote type %X", type_))
  598. }
  599. }
  600. if ps.CatchupCommitRound == round {
  601. switch type_ {
  602. case types.VoteTypePrevote:
  603. return nil
  604. case types.VoteTypePrecommit:
  605. return ps.CatchupCommit
  606. default:
  607. PanicSanity(Fmt("Unexpected vote type %X", type_))
  608. }
  609. }
  610. return nil
  611. }
  612. if ps.Height == height+1 {
  613. if ps.LastCommitRound == round {
  614. switch type_ {
  615. case types.VoteTypePrevote:
  616. return nil
  617. case types.VoteTypePrecommit:
  618. return ps.LastCommit
  619. default:
  620. PanicSanity(Fmt("Unexpected vote type %X", type_))
  621. }
  622. }
  623. return nil
  624. }
  625. return nil
  626. }
  627. // NOTE: 'round' is what we know to be the commit round for height.
  628. func (ps *PeerState) ensureCatchupCommitRound(height, round int, numValidators int) {
  629. if ps.Height != height {
  630. return
  631. }
  632. if ps.CatchupCommitRound != -1 && ps.CatchupCommitRound != round {
  633. PanicSanity(Fmt("Conflicting CatchupCommitRound. Height: %v, Orig: %v, New: %v", height, ps.CatchupCommitRound, round))
  634. }
  635. if ps.CatchupCommitRound == round {
  636. return // Nothing to do!
  637. }
  638. ps.CatchupCommitRound = round
  639. if round == ps.Round {
  640. ps.CatchupCommit = ps.Precommits
  641. } else {
  642. ps.CatchupCommit = NewBitArray(numValidators)
  643. }
  644. }
  645. // NOTE: It's important to make sure that numValidators actually matches
  646. // what the node sees as the number of validators for height.
  647. func (ps *PeerState) EnsureVoteBitArrays(height int, numValidators int) {
  648. ps.mtx.Lock()
  649. defer ps.mtx.Unlock()
  650. ps.ensureVoteBitArrays(height, numValidators)
  651. }
  652. func (ps *PeerState) ensureVoteBitArrays(height int, numValidators int) {
  653. if ps.Height == height {
  654. if ps.Prevotes == nil {
  655. ps.Prevotes = NewBitArray(numValidators)
  656. }
  657. if ps.Precommits == nil {
  658. ps.Precommits = NewBitArray(numValidators)
  659. }
  660. if ps.CatchupCommit == nil {
  661. ps.CatchupCommit = NewBitArray(numValidators)
  662. }
  663. if ps.ProposalPOL == nil {
  664. ps.ProposalPOL = NewBitArray(numValidators)
  665. }
  666. } else if ps.Height == height+1 {
  667. if ps.LastCommit == nil {
  668. ps.LastCommit = NewBitArray(numValidators)
  669. }
  670. }
  671. }
  672. func (ps *PeerState) SetHasVote(vote *types.Vote, index int) {
  673. ps.mtx.Lock()
  674. defer ps.mtx.Unlock()
  675. ps.setHasVote(vote.Height, vote.Round, vote.Type, index)
  676. }
  677. func (ps *PeerState) setHasVote(height int, round int, type_ byte, index int) {
  678. log := log.New("peer", ps.Peer.Key, "peerRound", ps.Round, "height", height, "round", round)
  679. if type_ != types.VoteTypePrevote && type_ != types.VoteTypePrecommit {
  680. PanicSanity("Invalid vote type")
  681. }
  682. if ps.Height == height {
  683. if ps.Round == round {
  684. switch type_ {
  685. case types.VoteTypePrevote:
  686. ps.Prevotes.SetIndex(index, true)
  687. log.Info("SetHasVote(round-match)", "prevotes", ps.Prevotes, "index", index)
  688. case types.VoteTypePrecommit:
  689. ps.Precommits.SetIndex(index, true)
  690. log.Info("SetHasVote(round-match)", "precommits", ps.Precommits, "index", index)
  691. }
  692. } else if ps.CatchupCommitRound == round {
  693. switch type_ {
  694. case types.VoteTypePrevote:
  695. case types.VoteTypePrecommit:
  696. ps.CatchupCommit.SetIndex(index, true)
  697. log.Info("SetHasVote(CatchupCommit)", "precommits", ps.Precommits, "index", index)
  698. }
  699. } else if ps.ProposalPOLRound == round {
  700. switch type_ {
  701. case types.VoteTypePrevote:
  702. ps.ProposalPOL.SetIndex(index, true)
  703. log.Info("SetHasVote(ProposalPOL)", "prevotes", ps.Prevotes, "index", index)
  704. case types.VoteTypePrecommit:
  705. }
  706. }
  707. } else if ps.Height == height+1 {
  708. if ps.LastCommitRound == round {
  709. switch type_ {
  710. case types.VoteTypePrevote:
  711. case types.VoteTypePrecommit:
  712. ps.LastCommit.SetIndex(index, true)
  713. log.Info("setHasVote(LastCommit)", "lastCommit", ps.LastCommit, "index", index)
  714. }
  715. }
  716. } else {
  717. // Does not apply.
  718. }
  719. }
  720. func (ps *PeerState) ApplyNewRoundStepMessage(msg *NewRoundStepMessage, rs *RoundState) {
  721. ps.mtx.Lock()
  722. defer ps.mtx.Unlock()
  723. // Ignore duplicate messages.
  724. if ps.Height == msg.Height && ps.Round == msg.Round && ps.Step == msg.Step {
  725. return
  726. }
  727. // Just remember these values.
  728. psHeight := ps.Height
  729. psRound := ps.Round
  730. //psStep := ps.Step
  731. psCatchupCommitRound := ps.CatchupCommitRound
  732. psCatchupCommit := ps.CatchupCommit
  733. startTime := time.Now().Add(-1 * time.Duration(msg.SecondsSinceStartTime) * time.Second)
  734. ps.Height = msg.Height
  735. ps.Round = msg.Round
  736. ps.Step = msg.Step
  737. ps.StartTime = startTime
  738. if psHeight != msg.Height || psRound != msg.Round {
  739. ps.Proposal = false
  740. ps.ProposalBlockPartsHeader = types.PartSetHeader{}
  741. ps.ProposalBlockParts = nil
  742. ps.ProposalPOLRound = -1
  743. ps.ProposalPOL = nil
  744. // We'll update the BitArray capacity later.
  745. ps.Prevotes = nil
  746. ps.Precommits = nil
  747. }
  748. if psHeight == msg.Height && psRound != msg.Round && msg.Round == psCatchupCommitRound {
  749. // Peer caught up to CatchupCommitRound.
  750. // Preserve psCatchupCommit!
  751. // NOTE: We prefer to use prs.Precommits if
  752. // pr.Round matches pr.CatchupCommitRound.
  753. ps.Precommits = psCatchupCommit
  754. }
  755. if psHeight != msg.Height {
  756. // Shift Precommits to LastCommit.
  757. if psHeight+1 == msg.Height && psRound == msg.LastCommitRound {
  758. ps.LastCommitRound = msg.LastCommitRound
  759. ps.LastCommit = ps.Precommits
  760. } else {
  761. ps.LastCommitRound = msg.LastCommitRound
  762. ps.LastCommit = nil
  763. }
  764. // We'll update the BitArray capacity later.
  765. ps.CatchupCommitRound = -1
  766. ps.CatchupCommit = nil
  767. }
  768. }
  769. func (ps *PeerState) ApplyCommitStepMessage(msg *CommitStepMessage) {
  770. ps.mtx.Lock()
  771. defer ps.mtx.Unlock()
  772. if ps.Height != msg.Height {
  773. return
  774. }
  775. ps.ProposalBlockPartsHeader = msg.BlockPartsHeader
  776. ps.ProposalBlockParts = msg.BlockParts
  777. }
  778. func (ps *PeerState) ApplyHasVoteMessage(msg *HasVoteMessage) {
  779. ps.mtx.Lock()
  780. defer ps.mtx.Unlock()
  781. if ps.Height != msg.Height {
  782. return
  783. }
  784. ps.setHasVote(msg.Height, msg.Round, msg.Type, msg.Index)
  785. }
  786. func (ps *PeerState) ApplyProposalPOLMessage(msg *ProposalPOLMessage) {
  787. ps.mtx.Lock()
  788. defer ps.mtx.Unlock()
  789. if ps.Height != msg.Height {
  790. return
  791. }
  792. if ps.ProposalPOLRound != msg.ProposalPOLRound {
  793. return
  794. }
  795. // TODO: Merge onto existing ps.ProposalPOL?
  796. // We might have sent some prevotes in the meantime.
  797. ps.ProposalPOL = msg.ProposalPOL
  798. }
  799. //-----------------------------------------------------------------------------
  800. // Messages
  801. const (
  802. msgTypeNewRoundStep = byte(0x01)
  803. msgTypeCommitStep = byte(0x02)
  804. msgTypeProposal = byte(0x11)
  805. msgTypeProposalPOL = byte(0x12)
  806. msgTypeBlockPart = byte(0x13) // both block & POL
  807. msgTypeVote = byte(0x14)
  808. msgTypeHasVote = byte(0x15)
  809. )
  810. type ConsensusMessage interface{}
  811. var _ = binary.RegisterInterface(
  812. struct{ ConsensusMessage }{},
  813. binary.ConcreteType{&NewRoundStepMessage{}, msgTypeNewRoundStep},
  814. binary.ConcreteType{&CommitStepMessage{}, msgTypeCommitStep},
  815. binary.ConcreteType{&ProposalMessage{}, msgTypeProposal},
  816. binary.ConcreteType{&ProposalPOLMessage{}, msgTypeProposalPOL},
  817. binary.ConcreteType{&BlockPartMessage{}, msgTypeBlockPart},
  818. binary.ConcreteType{&VoteMessage{}, msgTypeVote},
  819. binary.ConcreteType{&HasVoteMessage{}, msgTypeHasVote},
  820. )
  821. // TODO: check for unnecessary extra bytes at the end.
  822. func DecodeMessage(bz []byte) (msgType byte, msg ConsensusMessage, err error) {
  823. msgType = bz[0]
  824. n := new(int64)
  825. r := bytes.NewReader(bz)
  826. msg = binary.ReadBinary(struct{ ConsensusMessage }{}, r, n, &err).(struct{ ConsensusMessage }).ConsensusMessage
  827. return
  828. }
  829. //-------------------------------------
  830. // For every height/round/step transition
  831. type NewRoundStepMessage struct {
  832. Height int
  833. Round int
  834. Step RoundStepType
  835. SecondsSinceStartTime int
  836. LastCommitRound int
  837. }
  838. func (m *NewRoundStepMessage) String() string {
  839. return fmt.Sprintf("[NewRoundStep H:%v R:%v S:%v LCR:%v]",
  840. m.Height, m.Round, m.Step, m.LastCommitRound)
  841. }
  842. //-------------------------------------
  843. type CommitStepMessage struct {
  844. Height int
  845. BlockPartsHeader types.PartSetHeader
  846. BlockParts *BitArray
  847. }
  848. func (m *CommitStepMessage) String() string {
  849. return fmt.Sprintf("[CommitStep H:%v BP:%v BA:%v]", m.Height, m.BlockPartsHeader, m.BlockParts)
  850. }
  851. //-------------------------------------
  852. type ProposalMessage struct {
  853. Proposal *Proposal
  854. }
  855. func (m *ProposalMessage) String() string {
  856. return fmt.Sprintf("[Proposal %v]", m.Proposal)
  857. }
  858. //-------------------------------------
  859. type ProposalPOLMessage struct {
  860. Height int
  861. ProposalPOLRound int
  862. ProposalPOL *BitArray
  863. }
  864. func (m *ProposalPOLMessage) String() string {
  865. return fmt.Sprintf("[ProposalPOL H:%v POLR:%v POL:%v]", m.Height, m.ProposalPOLRound, m.ProposalPOL)
  866. }
  867. //-------------------------------------
  868. type BlockPartMessage struct {
  869. Height int
  870. Round int
  871. Part *types.Part
  872. }
  873. func (m *BlockPartMessage) String() string {
  874. return fmt.Sprintf("[BlockPart H:%v R:%v P:%v]", m.Height, m.Round, m.Part)
  875. }
  876. //-------------------------------------
  877. type VoteMessage struct {
  878. ValidatorIndex int
  879. Vote *types.Vote
  880. }
  881. func (m *VoteMessage) String() string {
  882. return fmt.Sprintf("[Vote VI:%v V:%v VI:%v]", m.ValidatorIndex, m.Vote, m.ValidatorIndex)
  883. }
  884. //-------------------------------------
  885. type HasVoteMessage struct {
  886. Height int
  887. Round int
  888. Type byte
  889. Index int
  890. }
  891. func (m *HasVoteMessage) String() string {
  892. return fmt.Sprintf("[HasVote VI:%v V:{%v/%02d/%v} VI:%v]", m.Index, m.Height, m.Round, m.Type, m.Index)
  893. }