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.

1673 lines
57 KiB

  1. package consensus
  2. import (
  3. "bytes"
  4. "errors"
  5. "fmt"
  6. "reflect"
  7. "runtime/debug"
  8. "sync"
  9. "time"
  10. fail "github.com/ebuchman/fail-test"
  11. cmn "github.com/tendermint/tmlibs/common"
  12. "github.com/tendermint/tmlibs/log"
  13. cfg "github.com/tendermint/tendermint/config"
  14. cstypes "github.com/tendermint/tendermint/consensus/types"
  15. tmevents "github.com/tendermint/tendermint/libs/events"
  16. "github.com/tendermint/tendermint/p2p"
  17. sm "github.com/tendermint/tendermint/state"
  18. "github.com/tendermint/tendermint/types"
  19. )
  20. //-----------------------------------------------------------------------------
  21. // Config
  22. const (
  23. proposalHeartbeatIntervalSeconds = 2
  24. )
  25. //-----------------------------------------------------------------------------
  26. // Errors
  27. var (
  28. ErrInvalidProposalSignature = errors.New("Error invalid proposal signature")
  29. ErrInvalidProposalPOLRound = errors.New("Error invalid proposal POL round")
  30. ErrAddingVote = errors.New("Error adding vote")
  31. ErrVoteHeightMismatch = errors.New("Error vote height mismatch")
  32. )
  33. //-----------------------------------------------------------------------------
  34. var (
  35. msgQueueSize = 1000
  36. )
  37. // msgs from the reactor which may update the state
  38. type msgInfo struct {
  39. Msg ConsensusMessage `json:"msg"`
  40. PeerID p2p.ID `json:"peer_key"`
  41. }
  42. // internally generated messages which may update the state
  43. type timeoutInfo struct {
  44. Duration time.Duration `json:"duration"`
  45. Height int64 `json:"height"`
  46. Round int `json:"round"`
  47. Step cstypes.RoundStepType `json:"step"`
  48. }
  49. func (ti *timeoutInfo) String() string {
  50. return fmt.Sprintf("%v ; %d/%d %v", ti.Duration, ti.Height, ti.Round, ti.Step)
  51. }
  52. // ConsensusState handles execution of the consensus algorithm.
  53. // It processes votes and proposals, and upon reaching agreement,
  54. // commits blocks to the chain and executes them against the application.
  55. // The internal state machine receives input from peers, the internal validator, and from a timer.
  56. type ConsensusState struct {
  57. cmn.BaseService
  58. // config details
  59. config *cfg.ConsensusConfig
  60. privValidator types.PrivValidator // for signing votes
  61. // services for creating and executing blocks
  62. // TODO: encapsulate all of this in one "BlockManager"
  63. blockExec *sm.BlockExecutor
  64. blockStore sm.BlockStore
  65. mempool sm.Mempool
  66. evpool sm.EvidencePool
  67. // internal state
  68. mtx sync.Mutex
  69. cstypes.RoundState
  70. state sm.State // State until height-1.
  71. // state changes may be triggered by: msgs from peers,
  72. // msgs from ourself, or by timeouts
  73. peerMsgQueue chan msgInfo
  74. internalMsgQueue chan msgInfo
  75. timeoutTicker TimeoutTicker
  76. // we use eventBus to trigger msg broadcasts in the reactor,
  77. // and to notify external subscribers, eg. through a websocket
  78. eventBus *types.EventBus
  79. // a Write-Ahead Log ensures we can recover from any kind of crash
  80. // and helps us avoid signing conflicting votes
  81. wal WAL
  82. replayMode bool // so we don't log signing errors during replay
  83. doWALCatchup bool // determines if we even try to do the catchup
  84. // for tests where we want to limit the number of transitions the state makes
  85. nSteps int
  86. // some functions can be overwritten for testing
  87. decideProposal func(height int64, round int)
  88. doPrevote func(height int64, round int)
  89. setProposal func(proposal *types.Proposal) error
  90. // closed when we finish shutting down
  91. done chan struct{}
  92. // synchronous pubsub between consensus state and reactor.
  93. // state only emits EventNewRoundStep, EventVote and EventProposalHeartbeat
  94. evsw tmevents.EventSwitch
  95. // for reporting metrics
  96. metrics *Metrics
  97. }
  98. // CSOption sets an optional parameter on the ConsensusState.
  99. type CSOption func(*ConsensusState)
  100. // NewConsensusState returns a new ConsensusState.
  101. func NewConsensusState(
  102. config *cfg.ConsensusConfig,
  103. state sm.State,
  104. blockExec *sm.BlockExecutor,
  105. blockStore sm.BlockStore,
  106. mempool sm.Mempool,
  107. evpool sm.EvidencePool,
  108. options ...CSOption,
  109. ) *ConsensusState {
  110. cs := &ConsensusState{
  111. config: config,
  112. blockExec: blockExec,
  113. blockStore: blockStore,
  114. mempool: mempool,
  115. peerMsgQueue: make(chan msgInfo, msgQueueSize),
  116. internalMsgQueue: make(chan msgInfo, msgQueueSize),
  117. timeoutTicker: NewTimeoutTicker(),
  118. done: make(chan struct{}),
  119. doWALCatchup: true,
  120. wal: nilWAL{},
  121. evpool: evpool,
  122. evsw: tmevents.NewEventSwitch(),
  123. metrics: NopMetrics(),
  124. }
  125. // set function defaults (may be overwritten before calling Start)
  126. cs.decideProposal = cs.defaultDecideProposal
  127. cs.doPrevote = cs.defaultDoPrevote
  128. cs.setProposal = cs.defaultSetProposal
  129. cs.updateToState(state)
  130. // Don't call scheduleRound0 yet.
  131. // We do that upon Start().
  132. cs.reconstructLastCommit(state)
  133. cs.BaseService = *cmn.NewBaseService(nil, "ConsensusState", cs)
  134. for _, option := range options {
  135. option(cs)
  136. }
  137. return cs
  138. }
  139. //----------------------------------------
  140. // Public interface
  141. // SetLogger implements Service.
  142. func (cs *ConsensusState) SetLogger(l log.Logger) {
  143. cs.BaseService.Logger = l
  144. cs.timeoutTicker.SetLogger(l)
  145. }
  146. // SetEventBus sets event bus.
  147. func (cs *ConsensusState) SetEventBus(b *types.EventBus) {
  148. cs.eventBus = b
  149. cs.blockExec.SetEventBus(b)
  150. }
  151. // WithMetrics sets the metrics.
  152. func WithMetrics(metrics *Metrics) CSOption {
  153. return func(cs *ConsensusState) { cs.metrics = metrics }
  154. }
  155. // String returns a string.
  156. func (cs *ConsensusState) String() string {
  157. // better not to access shared variables
  158. return cmn.Fmt("ConsensusState") //(H:%v R:%v S:%v", cs.Height, cs.Round, cs.Step)
  159. }
  160. // GetState returns a copy of the chain state.
  161. func (cs *ConsensusState) GetState() sm.State {
  162. cs.mtx.Lock()
  163. defer cs.mtx.Unlock()
  164. return cs.state.Copy()
  165. }
  166. // GetRoundState returns a shallow copy of the internal consensus state.
  167. func (cs *ConsensusState) GetRoundState() *cstypes.RoundState {
  168. cs.mtx.Lock()
  169. defer cs.mtx.Unlock()
  170. rs := cs.RoundState // copy
  171. return &rs
  172. }
  173. // GetRoundStateJSON returns a json of RoundState, marshalled using go-amino.
  174. func (cs *ConsensusState) GetRoundStateJSON() ([]byte, error) {
  175. cs.mtx.Lock()
  176. defer cs.mtx.Unlock()
  177. return cdc.MarshalJSON(cs.RoundState)
  178. }
  179. // GetRoundStateSimpleJSON returns a json of RoundStateSimple, marshalled using go-amino.
  180. func (cs *ConsensusState) GetRoundStateSimpleJSON() ([]byte, error) {
  181. cs.mtx.Lock()
  182. defer cs.mtx.Unlock()
  183. return cdc.MarshalJSON(cs.RoundState.RoundStateSimple())
  184. }
  185. // GetValidators returns a copy of the current validators.
  186. func (cs *ConsensusState) GetValidators() (int64, []*types.Validator) {
  187. cs.mtx.Lock()
  188. defer cs.mtx.Unlock()
  189. return cs.state.LastBlockHeight, cs.state.Validators.Copy().Validators
  190. }
  191. // SetPrivValidator sets the private validator account for signing votes.
  192. func (cs *ConsensusState) SetPrivValidator(priv types.PrivValidator) {
  193. cs.mtx.Lock()
  194. defer cs.mtx.Unlock()
  195. cs.privValidator = priv
  196. }
  197. // SetTimeoutTicker sets the local timer. It may be useful to overwrite for testing.
  198. func (cs *ConsensusState) SetTimeoutTicker(timeoutTicker TimeoutTicker) {
  199. cs.mtx.Lock()
  200. defer cs.mtx.Unlock()
  201. cs.timeoutTicker = timeoutTicker
  202. }
  203. // LoadCommit loads the commit for a given height.
  204. func (cs *ConsensusState) LoadCommit(height int64) *types.Commit {
  205. cs.mtx.Lock()
  206. defer cs.mtx.Unlock()
  207. if height == cs.blockStore.Height() {
  208. return cs.blockStore.LoadSeenCommit(height)
  209. }
  210. return cs.blockStore.LoadBlockCommit(height)
  211. }
  212. // OnStart implements cmn.Service.
  213. // It loads the latest state via the WAL, and starts the timeout and receive routines.
  214. func (cs *ConsensusState) OnStart() error {
  215. if err := cs.evsw.Start(); err != nil {
  216. return err
  217. }
  218. // we may set the WAL in testing before calling Start,
  219. // so only OpenWAL if its still the nilWAL
  220. if _, ok := cs.wal.(nilWAL); ok {
  221. walFile := cs.config.WalFile()
  222. wal, err := cs.OpenWAL(walFile)
  223. if err != nil {
  224. cs.Logger.Error("Error loading ConsensusState wal", "err", err.Error())
  225. return err
  226. }
  227. cs.wal = wal
  228. }
  229. // we need the timeoutRoutine for replay so
  230. // we don't block on the tick chan.
  231. // NOTE: we will get a build up of garbage go routines
  232. // firing on the tockChan until the receiveRoutine is started
  233. // to deal with them (by that point, at most one will be valid)
  234. if err := cs.timeoutTicker.Start(); err != nil {
  235. return err
  236. }
  237. // we may have lost some votes if the process crashed
  238. // reload from consensus log to catchup
  239. if cs.doWALCatchup {
  240. if err := cs.catchupReplay(cs.Height); err != nil {
  241. cs.Logger.Error("Error on catchup replay. Proceeding to start ConsensusState anyway", "err", err.Error())
  242. // NOTE: if we ever do return an error here,
  243. // make sure to stop the timeoutTicker
  244. }
  245. }
  246. // now start the receiveRoutine
  247. go cs.receiveRoutine(0)
  248. // schedule the first round!
  249. // use GetRoundState so we don't race the receiveRoutine for access
  250. cs.scheduleRound0(cs.GetRoundState())
  251. return nil
  252. }
  253. // timeoutRoutine: receive requests for timeouts on tickChan and fire timeouts on tockChan
  254. // receiveRoutine: serializes processing of proposoals, block parts, votes; coordinates state transitions
  255. func (cs *ConsensusState) startRoutines(maxSteps int) {
  256. err := cs.timeoutTicker.Start()
  257. if err != nil {
  258. cs.Logger.Error("Error starting timeout ticker", "err", err)
  259. return
  260. }
  261. go cs.receiveRoutine(maxSteps)
  262. }
  263. // OnStop implements cmn.Service. It stops all routines and waits for the WAL to finish.
  264. func (cs *ConsensusState) OnStop() {
  265. cs.BaseService.OnStop()
  266. cs.evsw.Stop()
  267. cs.timeoutTicker.Stop()
  268. // Make BaseService.Wait() wait until cs.wal.Wait()
  269. if cs.IsRunning() {
  270. cs.wal.Wait()
  271. }
  272. }
  273. // Wait waits for the the main routine to return.
  274. // NOTE: be sure to Stop() the event switch and drain
  275. // any event channels or this may deadlock
  276. func (cs *ConsensusState) Wait() {
  277. <-cs.done
  278. }
  279. // OpenWAL opens a file to log all consensus messages and timeouts for deterministic accountability
  280. func (cs *ConsensusState) OpenWAL(walFile string) (WAL, error) {
  281. wal, err := NewWAL(walFile)
  282. if err != nil {
  283. cs.Logger.Error("Failed to open WAL for consensus state", "wal", walFile, "err", err)
  284. return nil, err
  285. }
  286. wal.SetLogger(cs.Logger.With("wal", walFile))
  287. if err := wal.Start(); err != nil {
  288. return nil, err
  289. }
  290. return wal, nil
  291. }
  292. //------------------------------------------------------------
  293. // Public interface for passing messages into the consensus state, possibly causing a state transition.
  294. // If peerID == "", the msg is considered internal.
  295. // Messages are added to the appropriate queue (peer or internal).
  296. // If the queue is full, the function may block.
  297. // TODO: should these return anything or let callers just use events?
  298. // AddVote inputs a vote.
  299. func (cs *ConsensusState) AddVote(vote *types.Vote, peerID p2p.ID) (added bool, err error) {
  300. if peerID == "" {
  301. cs.internalMsgQueue <- msgInfo{&VoteMessage{vote}, ""}
  302. } else {
  303. cs.peerMsgQueue <- msgInfo{&VoteMessage{vote}, peerID}
  304. }
  305. // TODO: wait for event?!
  306. return false, nil
  307. }
  308. // SetProposal inputs a proposal.
  309. func (cs *ConsensusState) SetProposal(proposal *types.Proposal, peerID p2p.ID) error {
  310. if peerID == "" {
  311. cs.internalMsgQueue <- msgInfo{&ProposalMessage{proposal}, ""}
  312. } else {
  313. cs.peerMsgQueue <- msgInfo{&ProposalMessage{proposal}, peerID}
  314. }
  315. // TODO: wait for event?!
  316. return nil
  317. }
  318. // AddProposalBlockPart inputs a part of the proposal block.
  319. func (cs *ConsensusState) AddProposalBlockPart(height int64, round int, part *types.Part, peerID p2p.ID) error {
  320. if peerID == "" {
  321. cs.internalMsgQueue <- msgInfo{&BlockPartMessage{height, round, part}, ""}
  322. } else {
  323. cs.peerMsgQueue <- msgInfo{&BlockPartMessage{height, round, part}, peerID}
  324. }
  325. // TODO: wait for event?!
  326. return nil
  327. }
  328. // SetProposalAndBlock inputs the proposal and all block parts.
  329. func (cs *ConsensusState) SetProposalAndBlock(proposal *types.Proposal, block *types.Block, parts *types.PartSet, peerID p2p.ID) error {
  330. if err := cs.SetProposal(proposal, peerID); err != nil {
  331. return err
  332. }
  333. for i := 0; i < parts.Total(); i++ {
  334. part := parts.GetPart(i)
  335. if err := cs.AddProposalBlockPart(proposal.Height, proposal.Round, part, peerID); err != nil {
  336. return err
  337. }
  338. }
  339. return nil
  340. }
  341. //------------------------------------------------------------
  342. // internal functions for managing the state
  343. func (cs *ConsensusState) updateHeight(height int64) {
  344. cs.metrics.Height.Set(float64(height))
  345. cs.Height = height
  346. }
  347. func (cs *ConsensusState) updateRoundStep(round int, step cstypes.RoundStepType) {
  348. cs.Round = round
  349. cs.Step = step
  350. }
  351. // enterNewRound(height, 0) at cs.StartTime.
  352. func (cs *ConsensusState) scheduleRound0(rs *cstypes.RoundState) {
  353. //cs.Logger.Info("scheduleRound0", "now", time.Now(), "startTime", cs.StartTime)
  354. sleepDuration := rs.StartTime.Sub(time.Now()) // nolint: gotype, gosimple
  355. cs.scheduleTimeout(sleepDuration, rs.Height, 0, cstypes.RoundStepNewHeight)
  356. }
  357. // Attempt to schedule a timeout (by sending timeoutInfo on the tickChan)
  358. func (cs *ConsensusState) scheduleTimeout(duration time.Duration, height int64, round int, step cstypes.RoundStepType) {
  359. cs.timeoutTicker.ScheduleTimeout(timeoutInfo{duration, height, round, step})
  360. }
  361. // send a msg into the receiveRoutine regarding our own proposal, block part, or vote
  362. func (cs *ConsensusState) sendInternalMessage(mi msgInfo) {
  363. select {
  364. case cs.internalMsgQueue <- mi:
  365. default:
  366. // NOTE: using the go-routine means our votes can
  367. // be processed out of order.
  368. // TODO: use CList here for strict determinism and
  369. // attempt push to internalMsgQueue in receiveRoutine
  370. cs.Logger.Info("Internal msg queue is full. Using a go-routine")
  371. go func() { cs.internalMsgQueue <- mi }()
  372. }
  373. }
  374. // Reconstruct LastCommit from SeenCommit, which we saved along with the block,
  375. // (which happens even before saving the state)
  376. func (cs *ConsensusState) reconstructLastCommit(state sm.State) {
  377. if state.LastBlockHeight == 0 {
  378. return
  379. }
  380. seenCommit := cs.blockStore.LoadSeenCommit(state.LastBlockHeight)
  381. lastPrecommits := types.NewVoteSet(state.ChainID, state.LastBlockHeight, seenCommit.Round(), types.VoteTypePrecommit, state.LastValidators)
  382. for _, precommit := range seenCommit.Precommits {
  383. if precommit == nil {
  384. continue
  385. }
  386. added, err := lastPrecommits.AddVote(precommit)
  387. if !added || err != nil {
  388. cmn.PanicCrisis(cmn.Fmt("Failed to reconstruct LastCommit: %v", err))
  389. }
  390. }
  391. if !lastPrecommits.HasTwoThirdsMajority() {
  392. cmn.PanicSanity("Failed to reconstruct LastCommit: Does not have +2/3 maj")
  393. }
  394. cs.LastCommit = lastPrecommits
  395. }
  396. // Updates ConsensusState and increments height to match that of state.
  397. // The round becomes 0 and cs.Step becomes cstypes.RoundStepNewHeight.
  398. func (cs *ConsensusState) updateToState(state sm.State) {
  399. if cs.CommitRound > -1 && 0 < cs.Height && cs.Height != state.LastBlockHeight {
  400. cmn.PanicSanity(cmn.Fmt("updateToState() expected state height of %v but found %v",
  401. cs.Height, state.LastBlockHeight))
  402. }
  403. if !cs.state.IsEmpty() && cs.state.LastBlockHeight+1 != cs.Height {
  404. // This might happen when someone else is mutating cs.state.
  405. // Someone forgot to pass in state.Copy() somewhere?!
  406. cmn.PanicSanity(cmn.Fmt("Inconsistent cs.state.LastBlockHeight+1 %v vs cs.Height %v",
  407. cs.state.LastBlockHeight+1, cs.Height))
  408. }
  409. // If state isn't further out than cs.state, just ignore.
  410. // This happens when SwitchToConsensus() is called in the reactor.
  411. // We don't want to reset e.g. the Votes.
  412. if !cs.state.IsEmpty() && (state.LastBlockHeight <= cs.state.LastBlockHeight) {
  413. cs.Logger.Info("Ignoring updateToState()", "newHeight", state.LastBlockHeight+1, "oldHeight", cs.state.LastBlockHeight+1)
  414. return
  415. }
  416. // Reset fields based on state.
  417. validators := state.Validators
  418. lastPrecommits := (*types.VoteSet)(nil)
  419. if cs.CommitRound > -1 && cs.Votes != nil {
  420. if !cs.Votes.Precommits(cs.CommitRound).HasTwoThirdsMajority() {
  421. cmn.PanicSanity("updateToState(state) called but last Precommit round didn't have +2/3")
  422. }
  423. lastPrecommits = cs.Votes.Precommits(cs.CommitRound)
  424. }
  425. // Next desired block height
  426. height := state.LastBlockHeight + 1
  427. // RoundState fields
  428. cs.updateHeight(height)
  429. cs.updateRoundStep(0, cstypes.RoundStepNewHeight)
  430. if cs.CommitTime.IsZero() {
  431. // "Now" makes it easier to sync up dev nodes.
  432. // We add timeoutCommit to allow transactions
  433. // to be gathered for the first block.
  434. // And alternative solution that relies on clocks:
  435. // cs.StartTime = state.LastBlockTime.Add(timeoutCommit)
  436. cs.StartTime = cs.config.Commit(time.Now())
  437. } else {
  438. cs.StartTime = cs.config.Commit(cs.CommitTime)
  439. }
  440. cs.Validators = validators
  441. cs.Proposal = nil
  442. cs.ProposalBlock = nil
  443. cs.ProposalBlockParts = nil
  444. cs.LockedRound = 0
  445. cs.LockedBlock = nil
  446. cs.LockedBlockParts = nil
  447. cs.ValidRound = 0
  448. cs.ValidBlock = nil
  449. cs.ValidBlockParts = nil
  450. cs.Votes = cstypes.NewHeightVoteSet(state.ChainID, height, validators)
  451. cs.CommitRound = -1
  452. cs.LastCommit = lastPrecommits
  453. cs.LastValidators = state.LastValidators
  454. cs.state = state
  455. // Finally, broadcast RoundState
  456. cs.newStep()
  457. }
  458. func (cs *ConsensusState) newStep() {
  459. rs := cs.RoundStateEvent()
  460. cs.wal.Write(rs)
  461. cs.nSteps++
  462. // newStep is called by updateToStep in NewConsensusState before the eventBus is set!
  463. if cs.eventBus != nil {
  464. cs.eventBus.PublishEventNewRoundStep(rs)
  465. cs.evsw.FireEvent(types.EventNewRoundStep, &cs.RoundState)
  466. }
  467. }
  468. //-----------------------------------------
  469. // the main go routines
  470. // receiveRoutine handles messages which may cause state transitions.
  471. // it's argument (n) is the number of messages to process before exiting - use 0 to run forever
  472. // It keeps the RoundState and is the only thing that updates it.
  473. // Updates (state transitions) happen on timeouts, complete proposals, and 2/3 majorities.
  474. // ConsensusState must be locked before any internal state is updated.
  475. func (cs *ConsensusState) receiveRoutine(maxSteps int) {
  476. defer func() {
  477. if r := recover(); r != nil {
  478. cs.Logger.Error("CONSENSUS FAILURE!!!", "err", r, "stack", string(debug.Stack()))
  479. }
  480. }()
  481. for {
  482. if maxSteps > 0 {
  483. if cs.nSteps >= maxSteps {
  484. cs.Logger.Info("reached max steps. exiting receive routine")
  485. cs.nSteps = 0
  486. return
  487. }
  488. }
  489. rs := cs.RoundState
  490. var mi msgInfo
  491. select {
  492. case height := <-cs.mempool.TxsAvailable():
  493. cs.handleTxsAvailable(height)
  494. case mi = <-cs.peerMsgQueue:
  495. cs.wal.Write(mi)
  496. // handles proposals, block parts, votes
  497. // may generate internal events (votes, complete proposals, 2/3 majorities)
  498. cs.handleMsg(mi)
  499. case mi = <-cs.internalMsgQueue:
  500. cs.wal.WriteSync(mi) // NOTE: fsync
  501. // handles proposals, block parts, votes
  502. cs.handleMsg(mi)
  503. case ti := <-cs.timeoutTicker.Chan(): // tockChan:
  504. cs.wal.Write(ti)
  505. // if the timeout is relevant to the rs
  506. // go to the next step
  507. cs.handleTimeout(ti, rs)
  508. case <-cs.Quit():
  509. // NOTE: the internalMsgQueue may have signed messages from our
  510. // priv_val that haven't hit the WAL, but its ok because
  511. // priv_val tracks LastSig
  512. // close wal now that we're done writing to it
  513. cs.wal.Stop()
  514. close(cs.done)
  515. return
  516. }
  517. }
  518. }
  519. // state transitions on complete-proposal, 2/3-any, 2/3-one
  520. func (cs *ConsensusState) handleMsg(mi msgInfo) {
  521. cs.mtx.Lock()
  522. defer cs.mtx.Unlock()
  523. var err error
  524. msg, peerID := mi.Msg, mi.PeerID
  525. switch msg := msg.(type) {
  526. case *ProposalMessage:
  527. // will not cause transition.
  528. // once proposal is set, we can receive block parts
  529. err = cs.setProposal(msg.Proposal)
  530. case *BlockPartMessage:
  531. // if the proposal is complete, we'll enterPrevote or tryFinalizeCommit
  532. _, err = cs.addProposalBlockPart(msg.Height, msg.Part)
  533. if err != nil && msg.Round != cs.Round {
  534. cs.Logger.Debug("Received block part from wrong round", "height", cs.Height, "csRound", cs.Round, "blockRound", msg.Round)
  535. err = nil
  536. }
  537. case *VoteMessage:
  538. // attempt to add the vote and dupeout the validator if its a duplicate signature
  539. // if the vote gives us a 2/3-any or 2/3-one, we transition
  540. err := cs.tryAddVote(msg.Vote, peerID)
  541. if err == ErrAddingVote {
  542. // TODO: punish peer
  543. // We probably don't want to stop the peer here. The vote does not
  544. // necessarily comes from a malicious peer but can be just broadcasted by
  545. // a typical peer.
  546. // https://github.com/tendermint/tendermint/issues/1281
  547. }
  548. // NOTE: the vote is broadcast to peers by the reactor listening
  549. // for vote events
  550. // TODO: If rs.Height == vote.Height && rs.Round < vote.Round,
  551. // the peer is sending us CatchupCommit precommits.
  552. // We could make note of this and help filter in broadcastHasVoteMessage().
  553. default:
  554. cs.Logger.Error("Unknown msg type", reflect.TypeOf(msg))
  555. }
  556. if err != nil {
  557. cs.Logger.Error("Error with msg", "height", cs.Height, "round", cs.Round, "type", reflect.TypeOf(msg), "peer", peerID, "err", err, "msg", msg)
  558. }
  559. }
  560. func (cs *ConsensusState) handleTimeout(ti timeoutInfo, rs cstypes.RoundState) {
  561. cs.Logger.Debug("Received tock", "timeout", ti.Duration, "height", ti.Height, "round", ti.Round, "step", ti.Step)
  562. // timeouts must be for current height, round, step
  563. if ti.Height != rs.Height || ti.Round < rs.Round || (ti.Round == rs.Round && ti.Step < rs.Step) {
  564. cs.Logger.Debug("Ignoring tock because we're ahead", "height", rs.Height, "round", rs.Round, "step", rs.Step)
  565. return
  566. }
  567. // the timeout will now cause a state transition
  568. cs.mtx.Lock()
  569. defer cs.mtx.Unlock()
  570. switch ti.Step {
  571. case cstypes.RoundStepNewHeight:
  572. // NewRound event fired from enterNewRound.
  573. // XXX: should we fire timeout here (for timeout commit)?
  574. cs.enterNewRound(ti.Height, 0)
  575. case cstypes.RoundStepNewRound:
  576. cs.enterPropose(ti.Height, 0)
  577. case cstypes.RoundStepPropose:
  578. cs.eventBus.PublishEventTimeoutPropose(cs.RoundStateEvent())
  579. cs.enterPrevote(ti.Height, ti.Round)
  580. case cstypes.RoundStepPrevoteWait:
  581. cs.eventBus.PublishEventTimeoutWait(cs.RoundStateEvent())
  582. cs.enterPrecommit(ti.Height, ti.Round)
  583. case cstypes.RoundStepPrecommitWait:
  584. cs.eventBus.PublishEventTimeoutWait(cs.RoundStateEvent())
  585. cs.enterNewRound(ti.Height, ti.Round+1)
  586. default:
  587. panic(cmn.Fmt("Invalid timeout step: %v", ti.Step))
  588. }
  589. }
  590. func (cs *ConsensusState) handleTxsAvailable(height int64) {
  591. cs.mtx.Lock()
  592. defer cs.mtx.Unlock()
  593. // we only need to do this for round 0
  594. cs.enterPropose(height, 0)
  595. }
  596. //-----------------------------------------------------------------------------
  597. // State functions
  598. // Used internally by handleTimeout and handleMsg to make state transitions
  599. // Enter: `timeoutNewHeight` by startTime (commitTime+timeoutCommit),
  600. // or, if SkipTimeout==true, after receiving all precommits from (height,round-1)
  601. // Enter: `timeoutPrecommits` after any +2/3 precommits from (height,round-1)
  602. // Enter: +2/3 precommits for nil at (height,round-1)
  603. // Enter: +2/3 prevotes any or +2/3 precommits for block or any from (height, round)
  604. // NOTE: cs.StartTime was already set for height.
  605. func (cs *ConsensusState) enterNewRound(height int64, round int) {
  606. logger := cs.Logger.With("height", height, "round", round)
  607. if cs.Height != height || round < cs.Round || (cs.Round == round && cs.Step != cstypes.RoundStepNewHeight) {
  608. logger.Debug(cmn.Fmt("enterNewRound(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
  609. return
  610. }
  611. if now := time.Now(); cs.StartTime.After(now) {
  612. logger.Info("Need to set a buffer and log message here for sanity.", "startTime", cs.StartTime, "now", now)
  613. }
  614. logger.Info(cmn.Fmt("enterNewRound(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
  615. // Increment validators if necessary
  616. validators := cs.Validators
  617. if cs.Round < round {
  618. validators = validators.Copy()
  619. validators.IncrementAccum(round - cs.Round)
  620. }
  621. // Setup new round
  622. // we don't fire newStep for this step,
  623. // but we fire an event, so update the round step first
  624. cs.updateRoundStep(round, cstypes.RoundStepNewRound)
  625. cs.Validators = validators
  626. if round == 0 {
  627. // We've already reset these upon new height,
  628. // and meanwhile we might have received a proposal
  629. // for round 0.
  630. } else {
  631. logger.Info("Resetting Proposal info")
  632. cs.Proposal = nil
  633. cs.ProposalBlock = nil
  634. cs.ProposalBlockParts = nil
  635. }
  636. cs.Votes.SetRound(round + 1) // also track next round (round+1) to allow round-skipping
  637. cs.eventBus.PublishEventNewRound(cs.RoundStateEvent())
  638. cs.metrics.Rounds.Set(float64(round))
  639. // Wait for txs to be available in the mempool
  640. // before we enterPropose in round 0. If the last block changed the app hash,
  641. // we may need an empty "proof" block, and enterPropose immediately.
  642. waitForTxs := cs.config.WaitForTxs() && round == 0 && !cs.needProofBlock(height)
  643. if waitForTxs {
  644. if cs.config.CreateEmptyBlocksInterval > 0 {
  645. cs.scheduleTimeout(cs.config.EmptyBlocksInterval(), height, round, cstypes.RoundStepNewRound)
  646. }
  647. go cs.proposalHeartbeat(height, round)
  648. } else {
  649. cs.enterPropose(height, round)
  650. }
  651. }
  652. // needProofBlock returns true on the first height (so the genesis app hash is signed right away)
  653. // and where the last block (height-1) caused the app hash to change
  654. func (cs *ConsensusState) needProofBlock(height int64) bool {
  655. if height == 1 {
  656. return true
  657. }
  658. lastBlockMeta := cs.blockStore.LoadBlockMeta(height - 1)
  659. return !bytes.Equal(cs.state.AppHash, lastBlockMeta.Header.AppHash)
  660. }
  661. func (cs *ConsensusState) proposalHeartbeat(height int64, round int) {
  662. counter := 0
  663. addr := cs.privValidator.GetAddress()
  664. valIndex, _ := cs.Validators.GetByAddress(addr)
  665. chainID := cs.state.ChainID
  666. for {
  667. rs := cs.GetRoundState()
  668. // if we've already moved on, no need to send more heartbeats
  669. if rs.Step > cstypes.RoundStepNewRound || rs.Round > round || rs.Height > height {
  670. return
  671. }
  672. heartbeat := &types.Heartbeat{
  673. Height: rs.Height,
  674. Round: rs.Round,
  675. Sequence: counter,
  676. ValidatorAddress: addr,
  677. ValidatorIndex: valIndex,
  678. }
  679. cs.privValidator.SignHeartbeat(chainID, heartbeat)
  680. cs.eventBus.PublishEventProposalHeartbeat(types.EventDataProposalHeartbeat{heartbeat})
  681. cs.evsw.FireEvent(types.EventProposalHeartbeat, heartbeat)
  682. counter++
  683. time.Sleep(proposalHeartbeatIntervalSeconds * time.Second)
  684. }
  685. }
  686. // Enter (CreateEmptyBlocks): from enterNewRound(height,round)
  687. // Enter (CreateEmptyBlocks, CreateEmptyBlocksInterval > 0 ): after enterNewRound(height,round), after timeout of CreateEmptyBlocksInterval
  688. // Enter (!CreateEmptyBlocks) : after enterNewRound(height,round), once txs are in the mempool
  689. func (cs *ConsensusState) enterPropose(height int64, round int) {
  690. logger := cs.Logger.With("height", height, "round", round)
  691. if cs.Height != height || round < cs.Round || (cs.Round == round && cstypes.RoundStepPropose <= cs.Step) {
  692. logger.Debug(cmn.Fmt("enterPropose(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
  693. return
  694. }
  695. logger.Info(cmn.Fmt("enterPropose(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
  696. defer func() {
  697. // Done enterPropose:
  698. cs.updateRoundStep(round, cstypes.RoundStepPropose)
  699. cs.newStep()
  700. // If we have the whole proposal + POL, then goto Prevote now.
  701. // else, we'll enterPrevote when the rest of the proposal is received (in AddProposalBlockPart),
  702. // or else after timeoutPropose
  703. if cs.isProposalComplete() {
  704. cs.enterPrevote(height, cs.Round)
  705. }
  706. }()
  707. // If we don't get the proposal and all block parts quick enough, enterPrevote
  708. cs.scheduleTimeout(cs.config.Propose(round), height, round, cstypes.RoundStepPropose)
  709. // Nothing more to do if we're not a validator
  710. if cs.privValidator == nil {
  711. logger.Debug("This node is not a validator")
  712. return
  713. }
  714. // if not a validator, we're done
  715. if !cs.Validators.HasAddress(cs.privValidator.GetAddress()) {
  716. logger.Debug("This node is not a validator", "addr", cs.privValidator.GetAddress(), "vals", cs.Validators)
  717. return
  718. }
  719. logger.Debug("This node is a validator")
  720. if cs.isProposer() {
  721. logger.Info("enterPropose: Our turn to propose", "proposer", cs.Validators.GetProposer().Address, "privValidator", cs.privValidator)
  722. cs.decideProposal(height, round)
  723. } else {
  724. logger.Info("enterPropose: Not our turn to propose", "proposer", cs.Validators.GetProposer().Address, "privValidator", cs.privValidator)
  725. }
  726. }
  727. func (cs *ConsensusState) isProposer() bool {
  728. return bytes.Equal(cs.Validators.GetProposer().Address, cs.privValidator.GetAddress())
  729. }
  730. func (cs *ConsensusState) defaultDecideProposal(height int64, round int) {
  731. var block *types.Block
  732. var blockParts *types.PartSet
  733. // Decide on block
  734. if cs.LockedBlock != nil {
  735. // If we're locked onto a block, just choose that.
  736. block, blockParts = cs.LockedBlock, cs.LockedBlockParts
  737. } else if cs.ValidBlock != nil {
  738. // If there is valid block, choose that.
  739. block, blockParts = cs.ValidBlock, cs.ValidBlockParts
  740. } else {
  741. // Create a new proposal block from state/txs from the mempool.
  742. block, blockParts = cs.createProposalBlock()
  743. if block == nil { // on error
  744. return
  745. }
  746. }
  747. // Make proposal
  748. polRound, polBlockID := cs.Votes.POLInfo()
  749. proposal := types.NewProposal(height, round, blockParts.Header(), polRound, polBlockID)
  750. if err := cs.privValidator.SignProposal(cs.state.ChainID, proposal); err == nil {
  751. // Set fields
  752. /* fields set by setProposal and addBlockPart
  753. cs.Proposal = proposal
  754. cs.ProposalBlock = block
  755. cs.ProposalBlockParts = blockParts
  756. */
  757. // send proposal and block parts on internal msg queue
  758. cs.sendInternalMessage(msgInfo{&ProposalMessage{proposal}, ""})
  759. for i := 0; i < blockParts.Total(); i++ {
  760. part := blockParts.GetPart(i)
  761. cs.sendInternalMessage(msgInfo{&BlockPartMessage{cs.Height, cs.Round, part}, ""})
  762. }
  763. cs.Logger.Info("Signed proposal", "height", height, "round", round, "proposal", proposal)
  764. cs.Logger.Debug(cmn.Fmt("Signed proposal block: %v", block))
  765. } else {
  766. if !cs.replayMode {
  767. cs.Logger.Error("enterPropose: Error signing proposal", "height", height, "round", round, "err", err)
  768. }
  769. }
  770. }
  771. // Returns true if the proposal block is complete &&
  772. // (if POLRound was proposed, we have +2/3 prevotes from there).
  773. func (cs *ConsensusState) isProposalComplete() bool {
  774. if cs.Proposal == nil || cs.ProposalBlock == nil {
  775. return false
  776. }
  777. // we have the proposal. if there's a POLRound,
  778. // make sure we have the prevotes from it too
  779. if cs.Proposal.POLRound < 0 {
  780. return true
  781. }
  782. // if this is false the proposer is lying or we haven't received the POL yet
  783. return cs.Votes.Prevotes(cs.Proposal.POLRound).HasTwoThirdsMajority()
  784. }
  785. // Create the next block to propose and return it.
  786. // Returns nil block upon error.
  787. // NOTE: keep it side-effect free for clarity.
  788. func (cs *ConsensusState) createProposalBlock() (block *types.Block, blockParts *types.PartSet) {
  789. var commit *types.Commit
  790. if cs.Height == 1 {
  791. // We're creating a proposal for the first block.
  792. // The commit is empty, but not nil.
  793. commit = &types.Commit{}
  794. } else if cs.LastCommit.HasTwoThirdsMajority() {
  795. // Make the commit from LastCommit
  796. commit = cs.LastCommit.MakeCommit()
  797. } else {
  798. // This shouldn't happen.
  799. cs.Logger.Error("enterPropose: Cannot propose anything: No commit for the previous block.")
  800. return
  801. }
  802. // Mempool validated transactions
  803. txs := cs.mempool.Reap(cs.config.MaxBlockSizeTxs)
  804. block, parts := cs.state.MakeBlock(cs.Height, txs, commit)
  805. evidence := cs.evpool.PendingEvidence()
  806. block.AddEvidence(evidence)
  807. return block, parts
  808. }
  809. // Enter: `timeoutPropose` after entering Propose.
  810. // Enter: proposal block and POL is ready.
  811. // Enter: any +2/3 prevotes for future round.
  812. // Prevote for LockedBlock if we're locked, or ProposalBlock if valid.
  813. // Otherwise vote nil.
  814. func (cs *ConsensusState) enterPrevote(height int64, round int) {
  815. if cs.Height != height || round < cs.Round || (cs.Round == round && cstypes.RoundStepPrevote <= cs.Step) {
  816. cs.Logger.Debug(cmn.Fmt("enterPrevote(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
  817. return
  818. }
  819. defer func() {
  820. // Done enterPrevote:
  821. cs.updateRoundStep(round, cstypes.RoundStepPrevote)
  822. cs.newStep()
  823. }()
  824. // fire event for how we got here
  825. if cs.isProposalComplete() {
  826. cs.eventBus.PublishEventCompleteProposal(cs.RoundStateEvent())
  827. } else {
  828. // we received +2/3 prevotes for a future round
  829. // TODO: catchup event?
  830. }
  831. cs.Logger.Info(cmn.Fmt("enterPrevote(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
  832. // Sign and broadcast vote as necessary
  833. cs.doPrevote(height, round)
  834. // Once `addVote` hits any +2/3 prevotes, we will go to PrevoteWait
  835. // (so we have more time to try and collect +2/3 prevotes for a single block)
  836. }
  837. func (cs *ConsensusState) defaultDoPrevote(height int64, round int) {
  838. logger := cs.Logger.With("height", height, "round", round)
  839. // If a block is locked, prevote that.
  840. if cs.LockedBlock != nil {
  841. logger.Info("enterPrevote: Block was locked")
  842. cs.signAddVote(types.VoteTypePrevote, cs.LockedBlock.Hash(), cs.LockedBlockParts.Header())
  843. return
  844. }
  845. // If ProposalBlock is nil, prevote nil.
  846. if cs.ProposalBlock == nil {
  847. logger.Info("enterPrevote: ProposalBlock is nil")
  848. cs.signAddVote(types.VoteTypePrevote, nil, types.PartSetHeader{})
  849. return
  850. }
  851. // Validate proposal block
  852. err := cs.blockExec.ValidateBlock(cs.state, cs.ProposalBlock)
  853. if err != nil {
  854. // ProposalBlock is invalid, prevote nil.
  855. logger.Error("enterPrevote: ProposalBlock is invalid", "err", err)
  856. cs.signAddVote(types.VoteTypePrevote, nil, types.PartSetHeader{})
  857. return
  858. }
  859. // Prevote cs.ProposalBlock
  860. // NOTE: the proposal signature is validated when it is received,
  861. // and the proposal block parts are validated as they are received (against the merkle hash in the proposal)
  862. logger.Info("enterPrevote: ProposalBlock is valid")
  863. cs.signAddVote(types.VoteTypePrevote, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header())
  864. }
  865. // Enter: any +2/3 prevotes at next round.
  866. func (cs *ConsensusState) enterPrevoteWait(height int64, round int) {
  867. logger := cs.Logger.With("height", height, "round", round)
  868. if cs.Height != height || round < cs.Round || (cs.Round == round && cstypes.RoundStepPrevoteWait <= cs.Step) {
  869. logger.Debug(cmn.Fmt("enterPrevoteWait(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
  870. return
  871. }
  872. if !cs.Votes.Prevotes(round).HasTwoThirdsAny() {
  873. cmn.PanicSanity(cmn.Fmt("enterPrevoteWait(%v/%v), but Prevotes does not have any +2/3 votes", height, round))
  874. }
  875. logger.Info(cmn.Fmt("enterPrevoteWait(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
  876. defer func() {
  877. // Done enterPrevoteWait:
  878. cs.updateRoundStep(round, cstypes.RoundStepPrevoteWait)
  879. cs.newStep()
  880. }()
  881. // Wait for some more prevotes; enterPrecommit
  882. cs.scheduleTimeout(cs.config.Prevote(round), height, round, cstypes.RoundStepPrevoteWait)
  883. }
  884. // Enter: `timeoutPrevote` after any +2/3 prevotes.
  885. // Enter: +2/3 precomits for block or nil.
  886. // Enter: any +2/3 precommits for next round.
  887. // Lock & precommit the ProposalBlock if we have enough prevotes for it (a POL in this round)
  888. // else, unlock an existing lock and precommit nil if +2/3 of prevotes were nil,
  889. // else, precommit nil otherwise.
  890. func (cs *ConsensusState) enterPrecommit(height int64, round int) {
  891. logger := cs.Logger.With("height", height, "round", round)
  892. if cs.Height != height || round < cs.Round || (cs.Round == round && cstypes.RoundStepPrecommit <= cs.Step) {
  893. logger.Debug(cmn.Fmt("enterPrecommit(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
  894. return
  895. }
  896. logger.Info(cmn.Fmt("enterPrecommit(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
  897. defer func() {
  898. // Done enterPrecommit:
  899. cs.updateRoundStep(round, cstypes.RoundStepPrecommit)
  900. cs.newStep()
  901. }()
  902. // check for a polka
  903. blockID, ok := cs.Votes.Prevotes(round).TwoThirdsMajority()
  904. // If we don't have a polka, we must precommit nil.
  905. if !ok {
  906. if cs.LockedBlock != nil {
  907. logger.Info("enterPrecommit: No +2/3 prevotes during enterPrecommit while we're locked. Precommitting nil")
  908. } else {
  909. logger.Info("enterPrecommit: No +2/3 prevotes during enterPrecommit. Precommitting nil.")
  910. }
  911. cs.signAddVote(types.VoteTypePrecommit, nil, types.PartSetHeader{})
  912. return
  913. }
  914. // At this point +2/3 prevoted for a particular block or nil.
  915. cs.eventBus.PublishEventPolka(cs.RoundStateEvent())
  916. // the latest POLRound should be this round.
  917. polRound, _ := cs.Votes.POLInfo()
  918. if polRound < round {
  919. cmn.PanicSanity(cmn.Fmt("This POLRound should be %v but got %", round, polRound))
  920. }
  921. // +2/3 prevoted nil. Unlock and precommit nil.
  922. if len(blockID.Hash) == 0 {
  923. if cs.LockedBlock == nil {
  924. logger.Info("enterPrecommit: +2/3 prevoted for nil.")
  925. } else {
  926. logger.Info("enterPrecommit: +2/3 prevoted for nil. Unlocking")
  927. cs.LockedRound = 0
  928. cs.LockedBlock = nil
  929. cs.LockedBlockParts = nil
  930. cs.eventBus.PublishEventUnlock(cs.RoundStateEvent())
  931. }
  932. cs.signAddVote(types.VoteTypePrecommit, nil, types.PartSetHeader{})
  933. return
  934. }
  935. // At this point, +2/3 prevoted for a particular block.
  936. // If we're already locked on that block, precommit it, and update the LockedRound
  937. if cs.LockedBlock.HashesTo(blockID.Hash) {
  938. logger.Info("enterPrecommit: +2/3 prevoted locked block. Relocking")
  939. cs.LockedRound = round
  940. cs.eventBus.PublishEventRelock(cs.RoundStateEvent())
  941. cs.signAddVote(types.VoteTypePrecommit, blockID.Hash, blockID.PartsHeader)
  942. return
  943. }
  944. // If +2/3 prevoted for proposal block, stage and precommit it
  945. if cs.ProposalBlock.HashesTo(blockID.Hash) {
  946. logger.Info("enterPrecommit: +2/3 prevoted proposal block. Locking", "hash", blockID.Hash)
  947. // Validate the block.
  948. if err := cs.blockExec.ValidateBlock(cs.state, cs.ProposalBlock); err != nil {
  949. cmn.PanicConsensus(cmn.Fmt("enterPrecommit: +2/3 prevoted for an invalid block: %v", err))
  950. }
  951. cs.LockedRound = round
  952. cs.LockedBlock = cs.ProposalBlock
  953. cs.LockedBlockParts = cs.ProposalBlockParts
  954. cs.eventBus.PublishEventLock(cs.RoundStateEvent())
  955. cs.signAddVote(types.VoteTypePrecommit, blockID.Hash, blockID.PartsHeader)
  956. return
  957. }
  958. // There was a polka in this round for a block we don't have.
  959. // Fetch that block, unlock, and precommit nil.
  960. // The +2/3 prevotes for this round is the POL for our unlock.
  961. // TODO: In the future save the POL prevotes for justification.
  962. cs.LockedRound = 0
  963. cs.LockedBlock = nil
  964. cs.LockedBlockParts = nil
  965. if !cs.ProposalBlockParts.HasHeader(blockID.PartsHeader) {
  966. cs.ProposalBlock = nil
  967. cs.ProposalBlockParts = types.NewPartSetFromHeader(blockID.PartsHeader)
  968. }
  969. cs.eventBus.PublishEventUnlock(cs.RoundStateEvent())
  970. cs.signAddVote(types.VoteTypePrecommit, nil, types.PartSetHeader{})
  971. }
  972. // Enter: any +2/3 precommits for next round.
  973. func (cs *ConsensusState) enterPrecommitWait(height int64, round int) {
  974. logger := cs.Logger.With("height", height, "round", round)
  975. if cs.Height != height || round < cs.Round || (cs.Round == round && cstypes.RoundStepPrecommitWait <= cs.Step) {
  976. logger.Debug(cmn.Fmt("enterPrecommitWait(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
  977. return
  978. }
  979. if !cs.Votes.Precommits(round).HasTwoThirdsAny() {
  980. cmn.PanicSanity(cmn.Fmt("enterPrecommitWait(%v/%v), but Precommits does not have any +2/3 votes", height, round))
  981. }
  982. logger.Info(cmn.Fmt("enterPrecommitWait(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
  983. defer func() {
  984. // Done enterPrecommitWait:
  985. cs.updateRoundStep(round, cstypes.RoundStepPrecommitWait)
  986. cs.newStep()
  987. }()
  988. // Wait for some more precommits; enterNewRound
  989. cs.scheduleTimeout(cs.config.Precommit(round), height, round, cstypes.RoundStepPrecommitWait)
  990. }
  991. // Enter: +2/3 precommits for block
  992. func (cs *ConsensusState) enterCommit(height int64, commitRound int) {
  993. logger := cs.Logger.With("height", height, "commitRound", commitRound)
  994. if cs.Height != height || cstypes.RoundStepCommit <= cs.Step {
  995. logger.Debug(cmn.Fmt("enterCommit(%v/%v): Invalid args. Current step: %v/%v/%v", height, commitRound, cs.Height, cs.Round, cs.Step))
  996. return
  997. }
  998. logger.Info(cmn.Fmt("enterCommit(%v/%v). Current: %v/%v/%v", height, commitRound, cs.Height, cs.Round, cs.Step))
  999. defer func() {
  1000. // Done enterCommit:
  1001. // keep cs.Round the same, commitRound points to the right Precommits set.
  1002. cs.updateRoundStep(cs.Round, cstypes.RoundStepCommit)
  1003. cs.CommitRound = commitRound
  1004. cs.CommitTime = time.Now()
  1005. cs.newStep()
  1006. // Maybe finalize immediately.
  1007. cs.tryFinalizeCommit(height)
  1008. }()
  1009. blockID, ok := cs.Votes.Precommits(commitRound).TwoThirdsMajority()
  1010. if !ok {
  1011. cmn.PanicSanity("RunActionCommit() expects +2/3 precommits")
  1012. }
  1013. // The Locked* fields no longer matter.
  1014. // Move them over to ProposalBlock if they match the commit hash,
  1015. // otherwise they'll be cleared in updateToState.
  1016. if cs.LockedBlock.HashesTo(blockID.Hash) {
  1017. logger.Info("Commit is for locked block. Set ProposalBlock=LockedBlock", "blockHash", blockID.Hash)
  1018. cs.ProposalBlock = cs.LockedBlock
  1019. cs.ProposalBlockParts = cs.LockedBlockParts
  1020. }
  1021. // If we don't have the block being committed, set up to get it.
  1022. if !cs.ProposalBlock.HashesTo(blockID.Hash) {
  1023. if !cs.ProposalBlockParts.HasHeader(blockID.PartsHeader) {
  1024. logger.Info("Commit is for a block we don't know about. Set ProposalBlock=nil", "proposal", cs.ProposalBlock.Hash(), "commit", blockID.Hash)
  1025. // We're getting the wrong block.
  1026. // Set up ProposalBlockParts and keep waiting.
  1027. cs.ProposalBlock = nil
  1028. cs.ProposalBlockParts = types.NewPartSetFromHeader(blockID.PartsHeader)
  1029. } else {
  1030. // We just need to keep waiting.
  1031. }
  1032. }
  1033. }
  1034. // If we have the block AND +2/3 commits for it, finalize.
  1035. func (cs *ConsensusState) tryFinalizeCommit(height int64) {
  1036. logger := cs.Logger.With("height", height)
  1037. if cs.Height != height {
  1038. cmn.PanicSanity(cmn.Fmt("tryFinalizeCommit() cs.Height: %v vs height: %v", cs.Height, height))
  1039. }
  1040. blockID, ok := cs.Votes.Precommits(cs.CommitRound).TwoThirdsMajority()
  1041. if !ok || len(blockID.Hash) == 0 {
  1042. logger.Error("Attempt to finalize failed. There was no +2/3 majority, or +2/3 was for <nil>.")
  1043. return
  1044. }
  1045. if !cs.ProposalBlock.HashesTo(blockID.Hash) {
  1046. // TODO: this happens every time if we're not a validator (ugly logs)
  1047. // TODO: ^^ wait, why does it matter that we're a validator?
  1048. logger.Info("Attempt to finalize failed. We don't have the commit block.", "proposal-block", cs.ProposalBlock.Hash(), "commit-block", blockID.Hash)
  1049. return
  1050. }
  1051. // go
  1052. cs.finalizeCommit(height)
  1053. }
  1054. // Increment height and goto cstypes.RoundStepNewHeight
  1055. func (cs *ConsensusState) finalizeCommit(height int64) {
  1056. if cs.Height != height || cs.Step != cstypes.RoundStepCommit {
  1057. cs.Logger.Debug(cmn.Fmt("finalizeCommit(%v): Invalid args. Current step: %v/%v/%v", height, cs.Height, cs.Round, cs.Step))
  1058. return
  1059. }
  1060. blockID, ok := cs.Votes.Precommits(cs.CommitRound).TwoThirdsMajority()
  1061. block, blockParts := cs.ProposalBlock, cs.ProposalBlockParts
  1062. if !ok {
  1063. cmn.PanicSanity(cmn.Fmt("Cannot finalizeCommit, commit does not have two thirds majority"))
  1064. }
  1065. if !blockParts.HasHeader(blockID.PartsHeader) {
  1066. cmn.PanicSanity(cmn.Fmt("Expected ProposalBlockParts header to be commit header"))
  1067. }
  1068. if !block.HashesTo(blockID.Hash) {
  1069. cmn.PanicSanity(cmn.Fmt("Cannot finalizeCommit, ProposalBlock does not hash to commit hash"))
  1070. }
  1071. if err := cs.blockExec.ValidateBlock(cs.state, block); err != nil {
  1072. cmn.PanicConsensus(cmn.Fmt("+2/3 committed an invalid block: %v", err))
  1073. }
  1074. cs.Logger.Info(cmn.Fmt("Finalizing commit of block with %d txs", block.NumTxs),
  1075. "height", block.Height, "hash", block.Hash(), "root", block.AppHash)
  1076. cs.Logger.Info(cmn.Fmt("%v", block))
  1077. fail.Fail() // XXX
  1078. // Save to blockStore.
  1079. if cs.blockStore.Height() < block.Height {
  1080. // NOTE: the seenCommit is local justification to commit this block,
  1081. // but may differ from the LastCommit included in the next block
  1082. precommits := cs.Votes.Precommits(cs.CommitRound)
  1083. seenCommit := precommits.MakeCommit()
  1084. cs.blockStore.SaveBlock(block, blockParts, seenCommit)
  1085. } else {
  1086. // Happens during replay if we already saved the block but didn't commit
  1087. cs.Logger.Info("Calling finalizeCommit on already stored block", "height", block.Height)
  1088. }
  1089. fail.Fail() // XXX
  1090. // Write EndHeightMessage{} for this height, implying that the blockstore
  1091. // has saved the block.
  1092. //
  1093. // If we crash before writing this EndHeightMessage{}, we will recover by
  1094. // running ApplyBlock during the ABCI handshake when we restart. If we
  1095. // didn't save the block to the blockstore before writing
  1096. // EndHeightMessage{}, we'd have to change WAL replay -- currently it
  1097. // complains about replaying for heights where an #ENDHEIGHT entry already
  1098. // exists.
  1099. //
  1100. // Either way, the ConsensusState should not be resumed until we
  1101. // successfully call ApplyBlock (ie. later here, or in Handshake after
  1102. // restart).
  1103. cs.wal.WriteSync(EndHeightMessage{height}) // NOTE: fsync
  1104. fail.Fail() // XXX
  1105. // Create a copy of the state for staging and an event cache for txs.
  1106. stateCopy := cs.state.Copy()
  1107. // Execute and commit the block, update and save the state, and update the mempool.
  1108. // NOTE The block.AppHash wont reflect these txs until the next block.
  1109. var err error
  1110. stateCopy, err = cs.blockExec.ApplyBlock(stateCopy, types.BlockID{block.Hash(), blockParts.Header()}, block)
  1111. if err != nil {
  1112. cs.Logger.Error("Error on ApplyBlock. Did the application crash? Please restart tendermint", "err", err)
  1113. err := cmn.Kill()
  1114. if err != nil {
  1115. cs.Logger.Error("Failed to kill this process - please do so manually", "err", err)
  1116. }
  1117. return
  1118. }
  1119. fail.Fail() // XXX
  1120. // must be called before we update state
  1121. cs.recordMetrics(height, block)
  1122. // NewHeightStep!
  1123. cs.updateToState(stateCopy)
  1124. fail.Fail() // XXX
  1125. // cs.StartTime is already set.
  1126. // Schedule Round0 to start soon.
  1127. cs.scheduleRound0(&cs.RoundState)
  1128. // By here,
  1129. // * cs.Height has been increment to height+1
  1130. // * cs.Step is now cstypes.RoundStepNewHeight
  1131. // * cs.StartTime is set to when we will start round0.
  1132. }
  1133. func (cs *ConsensusState) recordMetrics(height int64, block *types.Block) {
  1134. cs.metrics.Validators.Set(float64(cs.Validators.Size()))
  1135. cs.metrics.ValidatorsPower.Set(float64(cs.Validators.TotalVotingPower()))
  1136. missingValidators := 0
  1137. missingValidatorsPower := int64(0)
  1138. for i, val := range cs.Validators.Validators {
  1139. var vote *types.Vote
  1140. if i < len(block.LastCommit.Precommits) {
  1141. vote = block.LastCommit.Precommits[i]
  1142. }
  1143. if vote == nil {
  1144. missingValidators++
  1145. missingValidatorsPower += val.VotingPower
  1146. }
  1147. }
  1148. cs.metrics.MissingValidators.Set(float64(missingValidators))
  1149. cs.metrics.MissingValidatorsPower.Set(float64(missingValidatorsPower))
  1150. cs.metrics.ByzantineValidators.Set(float64(len(block.Evidence.Evidence)))
  1151. byzantineValidatorsPower := int64(0)
  1152. for _, ev := range block.Evidence.Evidence {
  1153. if _, val := cs.Validators.GetByAddress(ev.Address()); val != nil {
  1154. byzantineValidatorsPower += val.VotingPower
  1155. }
  1156. }
  1157. cs.metrics.ByzantineValidatorsPower.Set(float64(byzantineValidatorsPower))
  1158. if height > 1 {
  1159. lastBlockMeta := cs.blockStore.LoadBlockMeta(height - 1)
  1160. cs.metrics.BlockIntervalSeconds.Observe(
  1161. block.Time.Sub(lastBlockMeta.Header.Time).Seconds(),
  1162. )
  1163. }
  1164. cs.metrics.NumTxs.Set(float64(block.NumTxs))
  1165. cs.metrics.BlockSizeBytes.Set(float64(block.Size()))
  1166. cs.metrics.TotalTxs.Set(float64(block.TotalTxs))
  1167. }
  1168. //-----------------------------------------------------------------------------
  1169. func (cs *ConsensusState) defaultSetProposal(proposal *types.Proposal) error {
  1170. // Already have one
  1171. // TODO: possibly catch double proposals
  1172. if cs.Proposal != nil {
  1173. return nil
  1174. }
  1175. // Does not apply
  1176. if proposal.Height != cs.Height || proposal.Round != cs.Round {
  1177. return nil
  1178. }
  1179. // We don't care about the proposal if we're already in cstypes.RoundStepCommit.
  1180. if cstypes.RoundStepCommit <= cs.Step {
  1181. return nil
  1182. }
  1183. // Verify POLRound, which must be -1 or between 0 and proposal.Round exclusive.
  1184. if proposal.POLRound != -1 &&
  1185. (proposal.POLRound < 0 || proposal.Round <= proposal.POLRound) {
  1186. return ErrInvalidProposalPOLRound
  1187. }
  1188. // Verify signature
  1189. if !cs.Validators.GetProposer().PubKey.VerifyBytes(proposal.SignBytes(cs.state.ChainID), proposal.Signature) {
  1190. return ErrInvalidProposalSignature
  1191. }
  1192. cs.Proposal = proposal
  1193. cs.ProposalBlockParts = types.NewPartSetFromHeader(proposal.BlockPartsHeader)
  1194. cs.Logger.Info("Received proposal", "proposal", proposal)
  1195. return nil
  1196. }
  1197. // NOTE: block is not necessarily valid.
  1198. // Asynchronously triggers either enterPrevote (before we timeout of propose) or tryFinalizeCommit, once we have the full block.
  1199. func (cs *ConsensusState) addProposalBlockPart(height int64, part *types.Part) (added bool, err error) {
  1200. // Blocks might be reused, so round mismatch is OK
  1201. if cs.Height != height {
  1202. cs.Logger.Debug("Received block part from wrong height", "height", height)
  1203. return false, nil
  1204. }
  1205. // We're not expecting a block part.
  1206. if cs.ProposalBlockParts == nil {
  1207. cs.Logger.Info("Received a block part when we're not expecting any", "height", height)
  1208. return false, nil // TODO: bad peer? Return error?
  1209. }
  1210. added, err = cs.ProposalBlockParts.AddPart(part)
  1211. if err != nil {
  1212. return added, err
  1213. }
  1214. if added && cs.ProposalBlockParts.IsComplete() {
  1215. // Added and completed!
  1216. _, err = cdc.UnmarshalBinaryReader(cs.ProposalBlockParts.GetReader(), &cs.ProposalBlock, int64(cs.state.ConsensusParams.BlockSize.MaxBytes))
  1217. if err != nil {
  1218. return true, err
  1219. }
  1220. // NOTE: it's possible to receive complete proposal blocks for future rounds without having the proposal
  1221. cs.Logger.Info("Received complete proposal block", "height", cs.ProposalBlock.Height, "hash", cs.ProposalBlock.Hash())
  1222. // Update Valid* if we can.
  1223. prevotes := cs.Votes.Prevotes(cs.Round)
  1224. blockID, hasTwoThirds := prevotes.TwoThirdsMajority()
  1225. if hasTwoThirds && !blockID.IsZero() && (cs.ValidRound < cs.Round) {
  1226. if cs.ProposalBlock.HashesTo(blockID.Hash) {
  1227. cs.Logger.Info("Updating valid block to new proposal block",
  1228. "valid-round", cs.Round, "valid-block-hash", cs.ProposalBlock.Hash())
  1229. cs.ValidRound = cs.Round
  1230. cs.ValidBlock = cs.ProposalBlock
  1231. cs.ValidBlockParts = cs.ProposalBlockParts
  1232. }
  1233. // TODO: In case there is +2/3 majority in Prevotes set for some
  1234. // block and cs.ProposalBlock contains different block, either
  1235. // proposer is faulty or voting power of faulty processes is more
  1236. // than 1/3. We should trigger in the future accountability
  1237. // procedure at this point.
  1238. }
  1239. if cs.Step == cstypes.RoundStepPropose && cs.isProposalComplete() {
  1240. // Move onto the next step
  1241. cs.enterPrevote(height, cs.Round)
  1242. } else if cs.Step == cstypes.RoundStepCommit {
  1243. // If we're waiting on the proposal block...
  1244. cs.tryFinalizeCommit(height)
  1245. }
  1246. return true, nil
  1247. }
  1248. return added, nil
  1249. }
  1250. // Attempt to add the vote. if its a duplicate signature, dupeout the validator
  1251. func (cs *ConsensusState) tryAddVote(vote *types.Vote, peerID p2p.ID) error {
  1252. _, err := cs.addVote(vote, peerID)
  1253. if err != nil {
  1254. // If the vote height is off, we'll just ignore it,
  1255. // But if it's a conflicting sig, add it to the cs.evpool.
  1256. // If it's otherwise invalid, punish peer.
  1257. if err == ErrVoteHeightMismatch {
  1258. return err
  1259. } else if voteErr, ok := err.(*types.ErrVoteConflictingVotes); ok {
  1260. if bytes.Equal(vote.ValidatorAddress, cs.privValidator.GetAddress()) {
  1261. cs.Logger.Error("Found conflicting vote from ourselves. Did you unsafe_reset a validator?", "height", vote.Height, "round", vote.Round, "type", vote.Type)
  1262. return err
  1263. }
  1264. cs.evpool.AddEvidence(voteErr.DuplicateVoteEvidence)
  1265. return err
  1266. } else {
  1267. // Probably an invalid signature / Bad peer.
  1268. // Seems this can also err sometimes with "Unexpected step" - perhaps not from a bad peer ?
  1269. cs.Logger.Error("Error attempting to add vote", "err", err)
  1270. return ErrAddingVote
  1271. }
  1272. }
  1273. return nil
  1274. }
  1275. //-----------------------------------------------------------------------------
  1276. func (cs *ConsensusState) addVote(vote *types.Vote, peerID p2p.ID) (added bool, err error) {
  1277. cs.Logger.Debug("addVote", "voteHeight", vote.Height, "voteType", vote.Type, "valIndex", vote.ValidatorIndex, "csHeight", cs.Height)
  1278. // A precommit for the previous height?
  1279. // These come in while we wait timeoutCommit
  1280. if vote.Height+1 == cs.Height {
  1281. if !(cs.Step == cstypes.RoundStepNewHeight && vote.Type == types.VoteTypePrecommit) {
  1282. // TODO: give the reason ..
  1283. // fmt.Errorf("tryAddVote: Wrong height, not a LastCommit straggler commit.")
  1284. return added, ErrVoteHeightMismatch
  1285. }
  1286. added, err = cs.LastCommit.AddVote(vote)
  1287. if !added {
  1288. return added, err
  1289. }
  1290. cs.Logger.Info(cmn.Fmt("Added to lastPrecommits: %v", cs.LastCommit.StringShort()))
  1291. cs.eventBus.PublishEventVote(types.EventDataVote{vote})
  1292. cs.evsw.FireEvent(types.EventVote, vote)
  1293. // if we can skip timeoutCommit and have all the votes now,
  1294. if cs.config.SkipTimeoutCommit && cs.LastCommit.HasAll() {
  1295. // go straight to new round (skip timeout commit)
  1296. // cs.scheduleTimeout(time.Duration(0), cs.Height, 0, cstypes.RoundStepNewHeight)
  1297. cs.enterNewRound(cs.Height, 0)
  1298. }
  1299. return
  1300. }
  1301. // Height mismatch is ignored.
  1302. // Not necessarily a bad peer, but not favourable behaviour.
  1303. if vote.Height != cs.Height {
  1304. err = ErrVoteHeightMismatch
  1305. cs.Logger.Info("Vote ignored and not added", "voteHeight", vote.Height, "csHeight", cs.Height, "err", err)
  1306. return
  1307. }
  1308. height := cs.Height
  1309. added, err = cs.Votes.AddVote(vote, peerID)
  1310. if !added {
  1311. // Either duplicate, or error upon cs.Votes.AddByIndex()
  1312. return
  1313. }
  1314. cs.eventBus.PublishEventVote(types.EventDataVote{vote})
  1315. cs.evsw.FireEvent(types.EventVote, vote)
  1316. switch vote.Type {
  1317. case types.VoteTypePrevote:
  1318. prevotes := cs.Votes.Prevotes(vote.Round)
  1319. cs.Logger.Info("Added to prevote", "vote", vote, "prevotes", prevotes.StringShort())
  1320. // If +2/3 prevotes for a block or nil for *any* round:
  1321. if blockID, ok := prevotes.TwoThirdsMajority(); ok {
  1322. // There was a polka!
  1323. // If we're locked but this is a recent polka, unlock.
  1324. // If it matches our ProposalBlock, update the ValidBlock
  1325. // Unlock if `cs.LockedRound < vote.Round <= cs.Round`
  1326. // NOTE: If vote.Round > cs.Round, we'll deal with it when we get to vote.Round
  1327. if (cs.LockedBlock != nil) &&
  1328. (cs.LockedRound < vote.Round) &&
  1329. (vote.Round <= cs.Round) &&
  1330. !cs.LockedBlock.HashesTo(blockID.Hash) {
  1331. cs.Logger.Info("Unlocking because of POL.", "lockedRound", cs.LockedRound, "POLRound", vote.Round)
  1332. cs.LockedRound = 0
  1333. cs.LockedBlock = nil
  1334. cs.LockedBlockParts = nil
  1335. cs.eventBus.PublishEventUnlock(cs.RoundStateEvent())
  1336. }
  1337. // Update Valid* if we can.
  1338. // NOTE: our proposal block may be nil or not what received a polka..
  1339. // TODO: we may want to still update the ValidBlock and obtain it via gossipping
  1340. if !blockID.IsZero() &&
  1341. (cs.ValidRound < vote.Round) &&
  1342. (vote.Round <= cs.Round) &&
  1343. cs.ProposalBlock.HashesTo(blockID.Hash) {
  1344. cs.Logger.Info("Updating ValidBlock because of POL.", "validRound", cs.ValidRound, "POLRound", vote.Round)
  1345. cs.ValidRound = vote.Round
  1346. cs.ValidBlock = cs.ProposalBlock
  1347. cs.ValidBlockParts = cs.ProposalBlockParts
  1348. }
  1349. }
  1350. // If +2/3 prevotes for *anything* for this or future round:
  1351. if cs.Round <= vote.Round && prevotes.HasTwoThirdsAny() {
  1352. // Round-skip over to PrevoteWait or goto Precommit.
  1353. cs.enterNewRound(height, vote.Round) // if the vote is ahead of us
  1354. if prevotes.HasTwoThirdsMajority() {
  1355. cs.enterPrecommit(height, vote.Round)
  1356. } else {
  1357. cs.enterPrevote(height, vote.Round) // if the vote is ahead of us
  1358. cs.enterPrevoteWait(height, vote.Round)
  1359. }
  1360. } else if cs.Proposal != nil && 0 <= cs.Proposal.POLRound && cs.Proposal.POLRound == vote.Round {
  1361. // If the proposal is now complete, enter prevote of cs.Round.
  1362. if cs.isProposalComplete() {
  1363. cs.enterPrevote(height, cs.Round)
  1364. }
  1365. }
  1366. case types.VoteTypePrecommit:
  1367. precommits := cs.Votes.Precommits(vote.Round)
  1368. cs.Logger.Info("Added to precommit", "vote", vote, "precommits", precommits.StringShort())
  1369. blockID, ok := precommits.TwoThirdsMajority()
  1370. if ok {
  1371. if len(blockID.Hash) == 0 {
  1372. cs.enterNewRound(height, vote.Round+1)
  1373. } else {
  1374. cs.enterNewRound(height, vote.Round)
  1375. cs.enterPrecommit(height, vote.Round)
  1376. cs.enterCommit(height, vote.Round)
  1377. if cs.config.SkipTimeoutCommit && precommits.HasAll() {
  1378. // if we have all the votes now,
  1379. // go straight to new round (skip timeout commit)
  1380. // cs.scheduleTimeout(time.Duration(0), cs.Height, 0, cstypes.RoundStepNewHeight)
  1381. cs.enterNewRound(cs.Height, 0)
  1382. }
  1383. }
  1384. } else if cs.Round <= vote.Round && precommits.HasTwoThirdsAny() {
  1385. cs.enterNewRound(height, vote.Round)
  1386. cs.enterPrecommit(height, vote.Round)
  1387. cs.enterPrecommitWait(height, vote.Round)
  1388. }
  1389. default:
  1390. panic(cmn.Fmt("Unexpected vote type %X", vote.Type)) // go-wire should prevent this.
  1391. }
  1392. return
  1393. }
  1394. func (cs *ConsensusState) signVote(type_ byte, hash []byte, header types.PartSetHeader) (*types.Vote, error) {
  1395. addr := cs.privValidator.GetAddress()
  1396. valIndex, _ := cs.Validators.GetByAddress(addr)
  1397. vote := &types.Vote{
  1398. ValidatorAddress: addr,
  1399. ValidatorIndex: valIndex,
  1400. Height: cs.Height,
  1401. Round: cs.Round,
  1402. Timestamp: time.Now().UTC(),
  1403. Type: type_,
  1404. BlockID: types.BlockID{hash, header},
  1405. }
  1406. err := cs.privValidator.SignVote(cs.state.ChainID, vote)
  1407. return vote, err
  1408. }
  1409. // sign the vote and publish on internalMsgQueue
  1410. func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header types.PartSetHeader) *types.Vote {
  1411. // if we don't have a key or we're not in the validator set, do nothing
  1412. if cs.privValidator == nil || !cs.Validators.HasAddress(cs.privValidator.GetAddress()) {
  1413. return nil
  1414. }
  1415. vote, err := cs.signVote(type_, hash, header)
  1416. if err == nil {
  1417. cs.sendInternalMessage(msgInfo{&VoteMessage{vote}, ""})
  1418. cs.Logger.Info("Signed and pushed vote", "height", cs.Height, "round", cs.Round, "vote", vote, "err", err)
  1419. return vote
  1420. }
  1421. //if !cs.replayMode {
  1422. cs.Logger.Error("Error signing vote", "height", cs.Height, "round", cs.Round, "vote", vote, "err", err)
  1423. //}
  1424. return nil
  1425. }
  1426. //---------------------------------------------------------
  1427. func CompareHRS(h1 int64, r1 int, s1 cstypes.RoundStepType, h2 int64, r2 int, s2 cstypes.RoundStepType) int {
  1428. if h1 < h2 {
  1429. return -1
  1430. } else if h1 > h2 {
  1431. return 1
  1432. }
  1433. if r1 < r2 {
  1434. return -1
  1435. } else if r1 > r2 {
  1436. return 1
  1437. }
  1438. if s1 < s2 {
  1439. return -1
  1440. } else if s1 > s2 {
  1441. return 1
  1442. }
  1443. return 0
  1444. }