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.

1628 lines
55 KiB

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