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.

1018 lines
31 KiB

10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
  1. /*
  2. Consensus State Machine Overview:
  3. * Propose, Prevote, Precommit represent state machine stages. (aka RoundStep, or step).
  4. Each take a predetermined amount of time depending on the round number.
  5. * The Commit step can be entered by two means:
  6. 1. After the Precommit step, +2/3 Precommits were found
  7. 2. At any time, +2/3 Commits were found
  8. * Once in the Commit stage, two conditions must both be satisfied
  9. before proceeding to the next height NewHeight.
  10. * The Propose step of the next height does not begin until
  11. at least Delta duration *after* +2/3 Commits were found.
  12. The step stays at NewHeight until this timeout occurs before
  13. proceeding to Propose.
  14. +-------------------------------------+
  15. | |
  16. v |(Wait til CommitTime + Delta)
  17. +-----------+ +-----+-----+
  18. +----------> | Propose +--------------+ | NewHeight |
  19. | +-----------+ | +-----------+
  20. | | ^
  21. | | |
  22. | | |
  23. |(Else) v |
  24. +-----+-----+ +-----------+ |
  25. | Precommit | <------------------------+ Prevote | |
  26. +-----+-----+ +-----------+ |
  27. |(If +2/3 Precommits found) |
  28. | |
  29. | + (When +2/3 Commits found) |
  30. | | |
  31. v v |
  32. +------------------------------------------------------------------------------+
  33. | Commit | |
  34. | | |
  35. | +----------------+ * Save Block | |
  36. | |Get Block Parts |---> * Stage Block +--+ + |
  37. | +----------------+ * Broadcast Commit | * Setup New Height |
  38. | | * Move Commits set to |
  39. | +--> LastCommits to continue |
  40. | | collecting commits |
  41. | +-----------------+ | * Broadcast New State |
  42. | |Get +2/3 Commits |--> * Set CommitTime +--+ |
  43. | +-----------------+ |
  44. | |
  45. +------------------------------------------------------------------------------+
  46. */
  47. package consensus
  48. import (
  49. "errors"
  50. "fmt"
  51. "math"
  52. "sync"
  53. "sync/atomic"
  54. "time"
  55. . "github.com/tendermint/tendermint/binary"
  56. . "github.com/tendermint/tendermint/blocks"
  57. . "github.com/tendermint/tendermint/common"
  58. . "github.com/tendermint/tendermint/config"
  59. "github.com/tendermint/tendermint/mempool"
  60. "github.com/tendermint/tendermint/state"
  61. )
  62. type RoundStep uint8
  63. type RoundActionType uint8
  64. const (
  65. RoundStepNewHeight = RoundStep(0x00) // Round0 for new height started, wait til CommitTime + Delta
  66. RoundStepNewRound = RoundStep(0x01) // Pseudostep, immediately goes to RoundStepPropose
  67. RoundStepPropose = RoundStep(0x10) // Did propose, gossip proposal
  68. RoundStepPrevote = RoundStep(0x11) // Did prevote, gossip prevotes
  69. RoundStepPrecommit = RoundStep(0x12) // Did precommit, gossip precommits
  70. RoundStepCommit = RoundStep(0x20) // Entered commit state machine
  71. RoundActionPropose = RoundActionType(0xA0) // Propose and goto RoundStepPropose
  72. RoundActionPrevote = RoundActionType(0xA1) // Prevote and goto RoundStepPrevote
  73. RoundActionPrecommit = RoundActionType(0xA2) // Precommit and goto RoundStepPrecommit
  74. RoundActionTryCommit = RoundActionType(0xC0) // Goto RoundStepCommit, or RoundStepPropose for next round.
  75. RoundActionCommit = RoundActionType(0xC1) // Goto RoundStepCommit
  76. RoundActionTryFinalize = RoundActionType(0xC2) // Maybe goto RoundStepPropose for next round.
  77. roundDuration0 = 60 * time.Second // The first round is 60 seconds long.
  78. roundDurationDelta = 15 * time.Second // Each successive round lasts 15 seconds longer.
  79. roundDeadlinePrevote = float64(1.0 / 3.0) // When the prevote is due.
  80. roundDeadlinePrecommit = float64(2.0 / 3.0) // When the precommit vote is due.
  81. newHeightDelta = roundDuration0 / 3 // The time to wait between commitTime and startTime of next consensus rounds.
  82. )
  83. var (
  84. ErrInvalidProposalSignature = errors.New("Error invalid proposal signature")
  85. )
  86. type RoundAction struct {
  87. Height uint32 // The block height for which consensus is reaching for.
  88. Round uint16 // The round number at given height.
  89. Action RoundActionType // Action to perform.
  90. }
  91. //-----------------------------------------------------------------------------
  92. // Immutable when returned from ConsensusState.GetRoundState()
  93. type RoundState struct {
  94. Height uint32 // Height we are working on
  95. Round uint16
  96. Step RoundStep
  97. StartTime time.Time
  98. CommitTime time.Time // Time when +2/3 commits were found
  99. Validators *state.ValidatorSet
  100. Proposal *Proposal
  101. ProposalBlock *Block
  102. ProposalBlockParts *PartSet
  103. ProposalPOL *POL
  104. ProposalPOLParts *PartSet
  105. LockedBlock *Block
  106. LockedBlockParts *PartSet
  107. LockedPOL *POL // Rarely needed, so no LockedPOLParts.
  108. Prevotes *VoteSet
  109. Precommits *VoteSet
  110. Commits *VoteSet
  111. LastCommits *VoteSet
  112. PrivValidator *PrivValidator
  113. }
  114. func (rs *RoundState) String() string {
  115. return rs.StringWithIndent("")
  116. }
  117. func (rs *RoundState) StringWithIndent(indent string) string {
  118. return fmt.Sprintf(`RoundState{
  119. %s H:%v R:%v S:%v
  120. %s StartTime: %v
  121. %s CommitTime: %v
  122. %s Validators: %v
  123. %s Proposal: %v
  124. %s ProposalBlock: %v %v
  125. %s ProposalPOL: %v %v
  126. %s LockedBlock: %v %v
  127. %s LockedPOL: %v
  128. %s Prevotes: %v
  129. %s Precommits: %v
  130. %s Commits: %v
  131. %s LastCommits: %v
  132. %s}`,
  133. indent, rs.Height, rs.Round, rs.Step,
  134. indent, rs.StartTime,
  135. indent, rs.CommitTime,
  136. indent, rs.Validators.StringWithIndent(indent+" "),
  137. indent, rs.Proposal,
  138. indent, rs.ProposalBlockParts.Description(), rs.ProposalBlock.Description(),
  139. indent, rs.ProposalPOLParts.Description(), rs.ProposalPOL.Description(),
  140. indent, rs.LockedBlockParts.Description(), rs.LockedBlock.Description(),
  141. indent, rs.LockedPOL.Description(),
  142. indent, rs.Prevotes.StringWithIndent(indent+" "),
  143. indent, rs.Precommits.StringWithIndent(indent+" "),
  144. indent, rs.Commits.StringWithIndent(indent+" "),
  145. indent, rs.LastCommits.Description(),
  146. indent)
  147. }
  148. func (rs *RoundState) Description() string {
  149. return fmt.Sprintf(`RS{%v/%v/%X %v}`,
  150. rs.Height, rs.Round, rs.Step, rs.StartTime)
  151. }
  152. //-----------------------------------------------------------------------------
  153. // Tracks consensus state across block heights and rounds.
  154. type ConsensusState struct {
  155. started uint32
  156. stopped uint32
  157. quit chan struct{}
  158. blockStore *BlockStore
  159. mempool *mempool.Mempool
  160. runActionCh chan RoundAction
  161. newStepCh chan *RoundState
  162. mtx sync.Mutex
  163. RoundState
  164. state *state.State // State until height-1.
  165. stagedBlock *Block // Cache last staged block.
  166. stagedState *state.State // Cache result of staged block.
  167. }
  168. func NewConsensusState(state *state.State, blockStore *BlockStore, mempool *mempool.Mempool) *ConsensusState {
  169. cs := &ConsensusState{
  170. quit: make(chan struct{}),
  171. blockStore: blockStore,
  172. mempool: mempool,
  173. runActionCh: make(chan RoundAction, 1),
  174. newStepCh: make(chan *RoundState, 1),
  175. }
  176. cs.updateToState(state)
  177. return cs
  178. }
  179. func (cs *ConsensusState) GetRoundState() *RoundState {
  180. cs.mtx.Lock()
  181. defer cs.mtx.Unlock()
  182. return cs.getRoundState()
  183. }
  184. func (cs *ConsensusState) getRoundState() *RoundState {
  185. rs := cs.RoundState // copy
  186. return &rs
  187. }
  188. func (cs *ConsensusState) NewStepCh() chan *RoundState {
  189. return cs.newStepCh
  190. }
  191. func (cs *ConsensusState) Start() {
  192. if atomic.CompareAndSwapUint32(&cs.started, 0, 1) {
  193. log.Info("Starting ConsensusState")
  194. go cs.stepTransitionRoutine()
  195. }
  196. }
  197. func (cs *ConsensusState) Stop() {
  198. if atomic.CompareAndSwapUint32(&cs.stopped, 0, 1) {
  199. log.Info("Stopping ConsensusState")
  200. close(cs.quit)
  201. }
  202. }
  203. func (cs *ConsensusState) IsStopped() bool {
  204. return atomic.LoadUint32(&cs.stopped) == 1
  205. }
  206. func (cs *ConsensusState) queueAction(ra RoundAction) {
  207. go func() {
  208. cs.runActionCh <- ra
  209. }()
  210. }
  211. // Source of all round state transitions (and votes).
  212. func (cs *ConsensusState) stepTransitionRoutine() {
  213. // For clarity, all state transitions that happen after some timeout are here.
  214. // Schedule the next action by pushing a RoundAction{} to cs.runActionCh.
  215. scheduleNextAction := func() {
  216. go func() {
  217. // NOTE: We can push directly to runActionCh because
  218. // we're running in a separate goroutine, which avoids deadlocks.
  219. rs := cs.getRoundState()
  220. round, roundStartTime, roundDuration, _, elapsedRatio := calcRoundInfo(rs.StartTime)
  221. log.Debug("Called scheduleNextAction. round:%v roundStartTime:%v elapsedRatio:%v", round, roundStartTime, elapsedRatio)
  222. switch rs.Step {
  223. case RoundStepNewHeight:
  224. // We should run RoundActionPropose when rs.StartTime passes.
  225. if elapsedRatio < 0 {
  226. // startTime is in the future.
  227. time.Sleep(time.Duration((-1.0 * elapsedRatio) * float64(roundDuration)))
  228. }
  229. cs.runActionCh <- RoundAction{rs.Height, rs.Round, RoundActionPropose}
  230. case RoundStepNewRound:
  231. // Pseudostep: Immediately goto propose.
  232. cs.runActionCh <- RoundAction{rs.Height, rs.Round, RoundActionPropose}
  233. case RoundStepPropose:
  234. // Wake up when it's time to vote.
  235. time.Sleep(time.Duration((roundDeadlinePrevote - elapsedRatio) * float64(roundDuration)))
  236. cs.runActionCh <- RoundAction{rs.Height, rs.Round, RoundActionPrevote}
  237. case RoundStepPrevote:
  238. // Wake up when it's time to precommit.
  239. time.Sleep(time.Duration((roundDeadlinePrecommit - elapsedRatio) * float64(roundDuration)))
  240. cs.runActionCh <- RoundAction{rs.Height, rs.Round, RoundActionPrecommit}
  241. case RoundStepPrecommit:
  242. // Wake up when the round is over.
  243. time.Sleep(time.Duration((1.0 - elapsedRatio) * float64(roundDuration)))
  244. cs.runActionCh <- RoundAction{rs.Height, rs.Round, RoundActionTryCommit}
  245. case RoundStepCommit:
  246. // There's nothing to scheudle, we're waiting for
  247. // ProposalBlockParts.IsComplete() &&
  248. // Commits.HasTwoThirdsMajority()
  249. panic("The next action from RoundStepCommit is not scheduled by time")
  250. default:
  251. panic("Should not happen")
  252. }
  253. }()
  254. }
  255. scheduleNextAction()
  256. // NOTE: All ConsensusState.RunAction*() calls come from here.
  257. // Since only one routine calls them, it is safe to assume that
  258. // the RoundState Height/Round/Step won't change concurrently.
  259. // However, other fields like Proposal could change concurrent
  260. // due to gossip routines.
  261. ACTION_LOOP:
  262. for {
  263. var roundAction RoundAction
  264. select {
  265. case roundAction = <-cs.runActionCh:
  266. case <-cs.quit:
  267. return
  268. }
  269. height, round, action := roundAction.Height, roundAction.Round, roundAction.Action
  270. rs := cs.GetRoundState()
  271. log.Info("Running round action A:%X %v", action, rs.Description())
  272. // Continue if action is not relevant
  273. if height != rs.Height {
  274. continue
  275. }
  276. // If action <= RoundActionPrecommit, the round must match too.
  277. if action <= RoundActionPrecommit && round != rs.Round {
  278. continue
  279. }
  280. // Run action
  281. switch action {
  282. case RoundActionPropose:
  283. if rs.Step != RoundStepNewHeight && rs.Step != RoundStepNewRound {
  284. continue ACTION_LOOP
  285. }
  286. cs.RunActionPropose(rs.Height, rs.Round)
  287. scheduleNextAction()
  288. continue ACTION_LOOP
  289. case RoundActionPrevote:
  290. if rs.Step >= RoundStepPrevote {
  291. continue ACTION_LOOP
  292. }
  293. cs.RunActionPrevote(rs.Height, rs.Round)
  294. scheduleNextAction()
  295. continue ACTION_LOOP
  296. case RoundActionPrecommit:
  297. if rs.Step >= RoundStepPrecommit {
  298. continue ACTION_LOOP
  299. }
  300. cs.RunActionPrecommit(rs.Height, rs.Round)
  301. scheduleNextAction()
  302. continue ACTION_LOOP
  303. case RoundActionTryCommit:
  304. if rs.Step >= RoundStepCommit {
  305. continue ACTION_LOOP
  306. }
  307. if rs.Precommits.HasTwoThirdsMajority() {
  308. // Enter RoundStepCommit and commit.
  309. cs.RunActionCommit(rs.Height)
  310. cs.queueAction(RoundAction{rs.Height, rs.Round, RoundActionTryFinalize})
  311. continue ACTION_LOOP
  312. } else {
  313. // Could not commit, move onto next round.
  314. cs.SetupNewRound(rs.Height, rs.Round+1)
  315. // cs.Step is now at RoundStepNewRound
  316. scheduleNextAction()
  317. continue ACTION_LOOP
  318. }
  319. case RoundActionCommit:
  320. if rs.Step >= RoundStepCommit {
  321. continue ACTION_LOOP
  322. }
  323. // Enter RoundStepCommit and commit.
  324. cs.RunActionCommit(rs.Height)
  325. cs.queueAction(RoundAction{rs.Height, rs.Round, RoundActionTryFinalize})
  326. continue ACTION_LOOP
  327. case RoundActionTryFinalize:
  328. if cs.TryFinalizeCommit(rs.Height) {
  329. // Now at new height
  330. // cs.Step is at RoundStepNewHeight or RoundStepNewRound.
  331. scheduleNextAction()
  332. continue ACTION_LOOP
  333. } else {
  334. // do not schedule next action.
  335. continue ACTION_LOOP
  336. }
  337. default:
  338. panic("Unknown action")
  339. }
  340. // For clarity, ensure that all switch cases call "continue"
  341. panic("Should not happen.")
  342. }
  343. }
  344. // Updates ConsensusState and increments height to match that of state.
  345. // If calculated round is greater than 0 (based on BlockTime or calculated StartTime)
  346. // then also sets up the appropriate round, and cs.Step becomes RoundStepNewRound.
  347. // Otherwise the round is 0 and cs.Step becomes RoundStepNewHeight.
  348. func (cs *ConsensusState) updateToState(state *state.State) {
  349. // Sanity check state.
  350. if cs.Height > 0 && cs.Height != state.LastBlockHeight {
  351. Panicf("updateToState() expected state height of %v but found %v",
  352. cs.Height, state.LastBlockHeight)
  353. }
  354. // Reset fields based on state.
  355. validators := state.BondedValidators
  356. height := state.LastBlockHeight + 1 // next desired block height
  357. cs.Height = height
  358. cs.Round = 0
  359. cs.Step = RoundStepNewHeight
  360. if cs.CommitTime.IsZero() {
  361. cs.StartTime = state.LastBlockTime.Add(newHeightDelta)
  362. } else {
  363. cs.StartTime = cs.CommitTime.Add(newHeightDelta)
  364. }
  365. cs.CommitTime = time.Time{}
  366. cs.Validators = validators
  367. cs.Proposal = nil
  368. cs.ProposalBlock = nil
  369. cs.ProposalBlockParts = nil
  370. cs.ProposalPOL = nil
  371. cs.ProposalPOLParts = nil
  372. cs.LockedBlock = nil
  373. cs.LockedBlockParts = nil
  374. cs.LockedPOL = nil
  375. cs.Prevotes = NewVoteSet(height, 0, VoteTypePrevote, validators)
  376. cs.Precommits = NewVoteSet(height, 0, VoteTypePrecommit, validators)
  377. cs.LastCommits = cs.Commits
  378. cs.Commits = NewVoteSet(height, 0, VoteTypeCommit, validators)
  379. cs.state = state
  380. cs.stagedBlock = nil
  381. cs.stagedState = nil
  382. // Update the round if we need to.
  383. round := calcRound(cs.StartTime)
  384. if round > 0 {
  385. cs.setupNewRound(round)
  386. }
  387. }
  388. // After the call cs.Step becomes RoundStepNewRound.
  389. func (cs *ConsensusState) setupNewRound(round uint16) {
  390. // Sanity check
  391. if round == 0 {
  392. panic("setupNewRound() should never be called for round 0")
  393. }
  394. // Increment all the way to round.
  395. validators := cs.Validators.Copy()
  396. for r := cs.Round; r < round; r++ {
  397. validators.IncrementAccum()
  398. }
  399. cs.Round = round
  400. cs.Step = RoundStepNewRound
  401. cs.Validators = validators
  402. cs.Proposal = nil
  403. cs.ProposalBlock = nil
  404. cs.ProposalBlockParts = nil
  405. cs.ProposalPOL = nil
  406. cs.ProposalPOLParts = nil
  407. cs.Prevotes = NewVoteSet(cs.Height, round, VoteTypePrevote, validators)
  408. cs.Prevotes.AddFromCommits(cs.Commits)
  409. cs.Precommits = NewVoteSet(cs.Height, round, VoteTypePrecommit, validators)
  410. cs.Precommits.AddFromCommits(cs.Commits)
  411. }
  412. func (cs *ConsensusState) SetPrivValidator(priv *PrivValidator) {
  413. cs.mtx.Lock()
  414. defer cs.mtx.Unlock()
  415. cs.PrivValidator = priv
  416. }
  417. //-----------------------------------------------------------------------------
  418. // Set up the round to desired round and set step to RoundStepNewRound
  419. func (cs *ConsensusState) SetupNewRound(height uint32, desiredRound uint16) bool {
  420. cs.mtx.Lock()
  421. defer cs.mtx.Unlock()
  422. if cs.Height != height {
  423. return false
  424. }
  425. if desiredRound <= cs.Round {
  426. return false
  427. }
  428. cs.setupNewRound(desiredRound)
  429. // c.Step is now RoundStepNewRound
  430. cs.newStepCh <- cs.getRoundState()
  431. return true
  432. }
  433. func (cs *ConsensusState) RunActionPropose(height uint32, round uint16) {
  434. cs.mtx.Lock()
  435. defer cs.mtx.Unlock()
  436. if cs.Height != height || cs.Round != round {
  437. return
  438. }
  439. defer func() {
  440. cs.Step = RoundStepPropose
  441. cs.newStepCh <- cs.getRoundState()
  442. }()
  443. // Nothing to do if it's not our turn.
  444. if cs.PrivValidator == nil || cs.Validators.Proposer().Id != cs.PrivValidator.Id {
  445. return
  446. }
  447. var block *Block
  448. var blockParts *PartSet
  449. var pol *POL
  450. var polParts *PartSet
  451. // Decide on block and POL
  452. if cs.LockedBlock != nil {
  453. // If we're locked onto a block, just choose that.
  454. block = cs.LockedBlock
  455. blockParts = cs.LockedBlockParts
  456. pol = cs.LockedPOL
  457. } else {
  458. // Otherwise we should create a new proposal.
  459. var validation *Validation
  460. if cs.Height == 1 {
  461. // We're creating a proposal for the first block.
  462. // The validation is empty.
  463. validation = &Validation{}
  464. } else {
  465. // We need to create a proposal.
  466. // If we don't have enough commits from the last height,
  467. // we can't do anything.
  468. if !cs.LastCommits.HasTwoThirdsMajority() {
  469. return
  470. } else {
  471. validation = cs.LastCommits.MakeValidation()
  472. }
  473. }
  474. txs := cs.mempool.GetProposalTxs()
  475. block = &Block{
  476. Header: &Header{
  477. Network: Config.Network,
  478. Height: cs.Height,
  479. Time: time.Now(),
  480. Fees: 0, // TODO fees
  481. LastBlockHash: cs.state.LastBlockHash,
  482. LastBlockParts: cs.state.LastBlockParts,
  483. StateHash: nil, // Will set afterwards.
  484. },
  485. Validation: validation,
  486. Data: &Data{
  487. Txs: txs,
  488. },
  489. }
  490. // Set the block.Header.StateHash.
  491. // TODO: we could cache the resulting state to cs.stagedState.
  492. cs.state.Copy().AppendBlock(block, PartSetHeader{}, false)
  493. blockParts = NewPartSetFromData(BinaryBytes(block))
  494. pol = cs.LockedPOL // If exists, is a PoUnlock.
  495. }
  496. if pol != nil {
  497. polParts = NewPartSetFromData(BinaryBytes(pol))
  498. }
  499. // Make proposal
  500. proposal := NewProposal(cs.Height, cs.Round, blockParts.Header(), polParts.Header())
  501. cs.PrivValidator.Sign(proposal)
  502. // Set fields
  503. cs.Proposal = proposal
  504. cs.ProposalBlock = block
  505. cs.ProposalBlockParts = blockParts
  506. cs.ProposalPOL = pol
  507. cs.ProposalPOLParts = polParts
  508. }
  509. // Prevote for LockedBlock if we're locked, or ProposealBlock if valid.
  510. // Otherwise vote nil.
  511. func (cs *ConsensusState) RunActionPrevote(height uint32, round uint16) {
  512. cs.mtx.Lock()
  513. defer cs.mtx.Unlock()
  514. if cs.Height != height || cs.Round != round {
  515. Panicf("RunActionPrevote(%v/%v), expected %v/%v", height, round, cs.Height, cs.Round)
  516. }
  517. defer func() {
  518. cs.Step = RoundStepPrevote
  519. cs.newStepCh <- cs.getRoundState()
  520. }()
  521. // If a block is locked, prevote that.
  522. if cs.LockedBlock != nil {
  523. cs.signAddVote(VoteTypePrevote, cs.LockedBlock.Hash(), cs.LockedBlockParts.Header())
  524. return
  525. }
  526. // If ProposalBlock is nil, prevote nil.
  527. if cs.ProposalBlock == nil {
  528. log.Warning("ProposalBlock is nil")
  529. cs.signAddVote(VoteTypePrevote, nil, PartSetHeader{})
  530. return
  531. }
  532. // Try staging cs.ProposalBlock
  533. err := cs.stageBlock(cs.ProposalBlock, cs.ProposalBlockParts)
  534. if err != nil {
  535. // ProposalBlock is invalid, prevote nil.
  536. log.Warning("ProposalBlock is invalid: %v", err)
  537. cs.signAddVote(VoteTypePrevote, nil, PartSetHeader{})
  538. return
  539. }
  540. // Prevote cs.ProposalBlock
  541. cs.signAddVote(VoteTypePrevote, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header())
  542. return
  543. }
  544. // Lock & Precommit the ProposalBlock if we have enough prevotes for it,
  545. // or unlock an existing lock if +2/3 of prevotes were nil.
  546. func (cs *ConsensusState) RunActionPrecommit(height uint32, round uint16) {
  547. cs.mtx.Lock()
  548. defer cs.mtx.Unlock()
  549. if cs.Height != height || cs.Round != round {
  550. Panicf("RunActionPrecommit(%v/%v), expected %v/%v", height, round, cs.Height, cs.Round)
  551. }
  552. defer func() {
  553. cs.Step = RoundStepPrecommit
  554. cs.newStepCh <- cs.getRoundState()
  555. }()
  556. hash, partsHeader, ok := cs.Prevotes.TwoThirdsMajority()
  557. if !ok {
  558. // If we don't have two thirds of prevotes,
  559. // don't do anything at all.
  560. return
  561. }
  562. // Remember this POL. (hash may be nil)
  563. cs.LockedPOL = cs.Prevotes.MakePOL()
  564. // If +2/3 prevoted nil. Just unlock.
  565. if len(hash) == 0 {
  566. cs.LockedBlock = nil
  567. cs.LockedBlockParts = nil
  568. return
  569. }
  570. // If +2/3 prevoted for already locked block, precommit it.
  571. if cs.LockedBlock.HashesTo(hash) {
  572. cs.signAddVote(VoteTypePrecommit, hash, partsHeader)
  573. return
  574. }
  575. // If +2/3 prevoted for cs.ProposalBlock, lock it and precommit it.
  576. if cs.ProposalBlock.HashesTo(hash) {
  577. // Validate the block.
  578. if err := cs.stageBlock(cs.ProposalBlock, cs.ProposalBlockParts); err != nil {
  579. // Prevent zombies.
  580. log.Warning("+2/3 prevoted for an invalid block: %v", err)
  581. return
  582. }
  583. cs.LockedBlock = cs.ProposalBlock
  584. cs.LockedBlockParts = cs.ProposalBlockParts
  585. cs.signAddVote(VoteTypePrecommit, hash, partsHeader)
  586. return
  587. }
  588. // We don't have the block that validators prevoted.
  589. // Unlock if we're locked.
  590. cs.LockedBlock = nil
  591. cs.LockedBlockParts = nil
  592. return
  593. }
  594. // Enter commit step. See the diagram for details.
  595. func (cs *ConsensusState) RunActionCommit(height uint32) {
  596. cs.mtx.Lock()
  597. defer cs.mtx.Unlock()
  598. if cs.Height != height {
  599. Panicf("RunActionCommit(%v), expected %v", height, cs.Height)
  600. }
  601. defer func() {
  602. cs.Step = RoundStepCommit
  603. cs.newStepCh <- cs.getRoundState()
  604. }()
  605. // There are two ways to enter:
  606. // 1. +2/3 precommits at the end of RoundStepPrecommit
  607. // 2. +2/3 commits at any time
  608. hash, partsHeader, ok := cs.Precommits.TwoThirdsMajority()
  609. if !ok {
  610. hash, partsHeader, ok = cs.Commits.TwoThirdsMajority()
  611. if !ok {
  612. panic("RunActionCommit() expects +2/3 precommits or commits")
  613. }
  614. }
  615. // Clear the Locked* fields and use cs.Proposed*
  616. if cs.LockedBlock.HashesTo(hash) {
  617. cs.ProposalBlock = cs.LockedBlock
  618. cs.ProposalBlockParts = cs.LockedBlockParts
  619. cs.LockedBlock = nil
  620. cs.LockedBlockParts = nil
  621. cs.LockedPOL = nil
  622. }
  623. // If we don't have the block being committed, set up to get it.
  624. if !cs.ProposalBlock.HashesTo(hash) {
  625. if !cs.ProposalBlockParts.HasHeader(partsHeader) {
  626. // We're getting the wrong block.
  627. // Set up ProposalBlockParts and keep waiting.
  628. cs.ProposalBlock = nil
  629. cs.ProposalBlockParts = NewPartSetFromHeader(partsHeader)
  630. } else {
  631. // We just need to keep waiting.
  632. }
  633. } else {
  634. // We have the block, so save/stage/sign-commit-vote.
  635. cs.processBlockForCommit(cs.ProposalBlock, cs.ProposalBlockParts)
  636. }
  637. }
  638. // Returns true if Finalize happened, which increments height && sets
  639. // the step to RoundStepNewHeight (or RoundStepNewRound, but probably not).
  640. func (cs *ConsensusState) TryFinalizeCommit(height uint32) bool {
  641. cs.mtx.Lock()
  642. defer cs.mtx.Unlock()
  643. if cs.Height != height {
  644. Panicf("TryFinalizeCommit(%v), expected %v", height, cs.Height)
  645. }
  646. if cs.Step == RoundStepCommit &&
  647. cs.Commits.HasTwoThirdsMajority() &&
  648. cs.ProposalBlockParts.IsComplete() {
  649. // Sanity check
  650. if cs.ProposalBlock == nil {
  651. Panicf("Expected ProposalBlock to exist")
  652. }
  653. hash, header, _ := cs.Commits.TwoThirdsMajority()
  654. if !cs.ProposalBlock.HashesTo(hash) {
  655. Panicf("Expected ProposalBlock to hash to commit hash")
  656. }
  657. if !cs.ProposalBlockParts.HasHeader(header) {
  658. Panicf("Expected ProposalBlockParts header to be commit header")
  659. }
  660. err := cs.stageBlock(cs.ProposalBlock, cs.ProposalBlockParts)
  661. if err == nil {
  662. log.Debug("Finalizing commit of block: %v", cs.ProposalBlock)
  663. // Increment height.
  664. cs.updateToState(cs.stagedState)
  665. // cs.Step is now RoundStepNewHeight or RoundStepNewRound
  666. cs.newStepCh <- cs.getRoundState()
  667. return true
  668. } else {
  669. // Prevent zombies.
  670. Panicf("+2/3 committed an invalid block: %v", err)
  671. }
  672. }
  673. return false
  674. }
  675. //-----------------------------------------------------------------------------
  676. func (cs *ConsensusState) SetProposal(proposal *Proposal) error {
  677. cs.mtx.Lock()
  678. defer cs.mtx.Unlock()
  679. // Already have one
  680. if cs.Proposal != nil {
  681. return nil
  682. }
  683. // Does not apply
  684. if proposal.Height != cs.Height || proposal.Round != cs.Round {
  685. return nil
  686. }
  687. // We don't care about the proposal if we're already in RoundStepCommit.
  688. if cs.Step == RoundStepCommit {
  689. return nil
  690. }
  691. // Verify signature
  692. if !cs.Validators.Proposer().Verify(proposal) {
  693. return ErrInvalidProposalSignature
  694. }
  695. cs.Proposal = proposal
  696. cs.ProposalBlockParts = NewPartSetFromHeader(proposal.BlockParts)
  697. cs.ProposalPOLParts = NewPartSetFromHeader(proposal.POLParts)
  698. return nil
  699. }
  700. // NOTE: block is not necessarily valid.
  701. // NOTE: This function may increment the height.
  702. func (cs *ConsensusState) AddProposalBlockPart(height uint32, round uint16, part *Part) (added bool, err error) {
  703. cs.mtx.Lock()
  704. defer cs.mtx.Unlock()
  705. // Blocks might be reused, so round mismatch is OK
  706. if cs.Height != height {
  707. return false, nil
  708. }
  709. // We're not expecting a block part.
  710. if cs.ProposalBlockParts == nil {
  711. return false, nil // TODO: bad peer? Return error?
  712. }
  713. added, err = cs.ProposalBlockParts.AddPart(part)
  714. if err != nil {
  715. return added, err
  716. }
  717. if added && cs.ProposalBlockParts.IsComplete() {
  718. var n int64
  719. var err error
  720. cs.ProposalBlock = ReadBlock(cs.ProposalBlockParts.GetReader(), &n, &err)
  721. cs.queueAction(RoundAction{cs.Height, cs.Round, RoundActionTryFinalize})
  722. return true, err
  723. }
  724. return true, nil
  725. }
  726. // NOTE: POL is not necessarily valid.
  727. func (cs *ConsensusState) AddProposalPOLPart(height uint32, round uint16, part *Part) (added bool, err error) {
  728. cs.mtx.Lock()
  729. defer cs.mtx.Unlock()
  730. if cs.Height != height || cs.Round != round {
  731. return false, nil
  732. }
  733. // We're not expecting a POL part.
  734. if cs.ProposalPOLParts == nil {
  735. return false, nil // TODO: bad peer? Return error?
  736. }
  737. added, err = cs.ProposalPOLParts.AddPart(part)
  738. if err != nil {
  739. return added, err
  740. }
  741. if added && cs.ProposalPOLParts.IsComplete() {
  742. var n int64
  743. var err error
  744. cs.ProposalPOL = ReadPOL(cs.ProposalPOLParts.GetReader(), &n, &err)
  745. return true, err
  746. }
  747. return true, nil
  748. }
  749. func (cs *ConsensusState) AddVote(vote *Vote) (added bool, index uint, err error) {
  750. cs.mtx.Lock()
  751. defer cs.mtx.Unlock()
  752. return cs.addVote(vote)
  753. }
  754. // TODO: Maybe move this out of here?
  755. func (cs *ConsensusState) LoadHeaderValidation(height uint32) (*Header, *Validation) {
  756. meta := cs.blockStore.LoadBlockMeta(height)
  757. if meta == nil {
  758. return nil, nil
  759. }
  760. validation := cs.blockStore.LoadBlockValidation(height)
  761. return meta.Header, validation
  762. }
  763. //-----------------------------------------------------------------------------
  764. func (cs *ConsensusState) addVote(vote *Vote) (added bool, index uint, err error) {
  765. switch vote.Type {
  766. case VoteTypePrevote:
  767. // Prevotes checks for height+round match.
  768. return cs.Prevotes.Add(vote)
  769. case VoteTypePrecommit:
  770. // Precommits checks for height+round match.
  771. return cs.Precommits.Add(vote)
  772. case VoteTypeCommit:
  773. if vote.Height == cs.Height {
  774. // No need to check if vote.Round < cs.Round ...
  775. // Prevotes && Precommits already checks that.
  776. cs.Prevotes.Add(vote)
  777. cs.Precommits.Add(vote)
  778. added, index, err = cs.Commits.Add(vote)
  779. if added && cs.Commits.HasTwoThirdsMajority() && cs.CommitTime.IsZero() {
  780. cs.CommitTime = time.Now()
  781. log.Debug("Set CommitTime to %v", cs.CommitTime)
  782. if cs.Step < RoundStepCommit {
  783. cs.queueAction(RoundAction{cs.Height, cs.Round, RoundActionCommit})
  784. } else {
  785. cs.queueAction(RoundAction{cs.Height, cs.Round, RoundActionTryFinalize})
  786. }
  787. }
  788. return added, index, err
  789. }
  790. if vote.Height+1 == cs.Height {
  791. return cs.LastCommits.Add(vote)
  792. }
  793. return false, 0, nil
  794. default:
  795. panic("Unknown vote type")
  796. }
  797. }
  798. func (cs *ConsensusState) stageBlock(block *Block, blockParts *PartSet) error {
  799. if block == nil {
  800. panic("Cannot stage nil block")
  801. }
  802. // Already staged?
  803. if cs.stagedBlock == block {
  804. return nil
  805. }
  806. // Create a copy of the state for staging
  807. stateCopy := cs.state.Copy()
  808. // Commit block onto the copied state.
  809. // NOTE: Basic validation is done in state.AppendBlock().
  810. err := stateCopy.AppendBlock(block, blockParts.Header(), true)
  811. if err != nil {
  812. return err
  813. } else {
  814. cs.stagedBlock = block
  815. cs.stagedState = stateCopy
  816. return nil
  817. }
  818. }
  819. func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header PartSetHeader) *Vote {
  820. if cs.PrivValidator == nil || !cs.Validators.HasId(cs.PrivValidator.Id) {
  821. return nil
  822. }
  823. vote := &Vote{
  824. Height: cs.Height,
  825. Round: cs.Round,
  826. Type: type_,
  827. BlockHash: hash,
  828. BlockParts: header,
  829. }
  830. cs.PrivValidator.Sign(vote)
  831. cs.addVote(vote)
  832. return vote
  833. }
  834. func (cs *ConsensusState) processBlockForCommit(block *Block, blockParts *PartSet) {
  835. // The proposal must be valid.
  836. if err := cs.stageBlock(block, blockParts); err != nil {
  837. // Prevent zombies.
  838. log.Warning("+2/3 precommitted an invalid block: %v", err)
  839. return
  840. }
  841. // Save to blockStore
  842. cs.blockStore.SaveBlock(block, blockParts)
  843. // Save the state
  844. cs.stagedState.Save()
  845. // Update mempool.
  846. cs.mempool.ResetForBlockAndState(block, cs.stagedState)
  847. cs.signAddVote(VoteTypeCommit, block.Hash(), blockParts.Header())
  848. }
  849. //-----------------------------------------------------------------------------
  850. // total duration of given round
  851. func calcRoundDuration(round uint16) time.Duration {
  852. return roundDuration0 + roundDurationDelta*time.Duration(round)
  853. }
  854. // startTime is when round zero started.
  855. func calcRoundStartTime(round uint16, startTime time.Time) time.Time {
  856. return startTime.Add(roundDuration0*time.Duration(round) +
  857. roundDurationDelta*(time.Duration((int64(round)*int64(round)-int64(round))/2)))
  858. }
  859. // calculates the current round given startTime of round zero.
  860. // NOTE: round is zero if startTime is in the future.
  861. func calcRound(startTime time.Time) uint16 {
  862. now := time.Now()
  863. if now.Before(startTime) {
  864. return 0
  865. }
  866. // Start + D_0 * R + D_delta * (R^2 - R)/2 <= Now; find largest integer R.
  867. // D_delta * R^2 + (2D_0 - D_delta) * R + 2(Start - Now) <= 0.
  868. // AR^2 + BR + C <= 0; A = D_delta, B = (2_D0 - D_delta), C = 2(Start - Now).
  869. // R = Floor((-B + Sqrt(B^2 - 4AC))/2A)
  870. A := float64(roundDurationDelta)
  871. B := 2.0*float64(roundDuration0) - float64(roundDurationDelta)
  872. C := 2.0 * float64(startTime.Sub(now))
  873. R := math.Floor((-B + math.Sqrt(B*B-4.0*A*C)) / (2 * A))
  874. if math.IsNaN(R) {
  875. panic("Could not calc round, should not happen")
  876. }
  877. if R > math.MaxInt16 {
  878. Panicf("Could not calc round, round overflow: %v", R)
  879. }
  880. if R < 0 {
  881. return 0
  882. }
  883. return uint16(R)
  884. }
  885. // convenience
  886. // NOTE: elapsedRatio can be negative if startTime is in the future.
  887. func calcRoundInfo(startTime time.Time) (round uint16, roundStartTime time.Time, roundDuration time.Duration,
  888. roundElapsed time.Duration, elapsedRatio float64) {
  889. round = calcRound(startTime)
  890. roundStartTime = calcRoundStartTime(round, startTime)
  891. roundDuration = calcRoundDuration(round)
  892. roundElapsed = time.Now().Sub(roundStartTime)
  893. elapsedRatio = float64(roundElapsed) / float64(roundDuration)
  894. return
  895. }