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.

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