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.

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