tendermint/consensus/state.go

1172 lines
37 KiB
Go
Raw Normal View History

2014-10-30 03:32:09 -07:00
/*
Consensus State Machine Overview:
* Propose, Prevote, Precommit represent state machine stages. (aka RoundStep, or step).
Each take a predetermined amount of time depending on the round number.
* The Commit step can be entered by two means:
1. After the Precommit step, +2/3 Precommits were found
2. At any time, +2/3 Commits were found
* Once in the Commit stage, two conditions must both be satisfied
before proceeding to the next height NewHeight.
* The Propose step of the next height does not begin until
at least Delta duration *after* +2/3 Commits were found.
The step stays at NewHeight until this timeout occurs before
proceeding to Propose.
+-------------------------------------+
| |
v |(Wait til CommitTime + Delta)
+-----------+ +-----+-----+
+----------> | Propose +--------------+ | NewHeight |
| +-----------+ | +-----------+
| | ^
| | |
| | |
|(Else) v |
+-----+-----+ +-----------+ |
| Precommit | <------------------------+ Prevote | |
+-----+-----+ +-----------+ |
|(If +2/3 Precommits found) |
| |
| + (When +2/3 Commits found) |
| | |
v v |
+------------------------------------------------------------------------------+
| Commit | |
| | |
| +----------------+ * Save Block | |
| |Get Block Parts |---> * Stage Block +--+ + |
| +----------------+ * Broadcast Commit | * Setup New Height |
| | * Move Commits set to |
| +--> LastCommits to continue |
| | collecting commits |
| +-----------------+ | * Broadcast New State |
| |Get +2/3 Commits |--> * Set CommitTime +--+ |
| +-----------------+ |
| |
+------------------------------------------------------------------------------+
*/
2014-08-10 16:35:08 -07:00
package consensus
import (
"bytes"
2014-09-14 15:37:32 -07:00
"errors"
2014-10-18 01:42:33 -07:00
"fmt"
2014-10-30 03:32:09 -07:00
"math"
2014-08-10 16:35:08 -07:00
"sync"
2014-10-30 03:32:09 -07:00
"sync/atomic"
2014-08-10 16:35:08 -07:00
"time"
2015-04-01 17:30:16 -07:00
"github.com/tendermint/tendermint/account"
"github.com/tendermint/tendermint/binary"
bc "github.com/tendermint/tendermint/blockchain"
. "github.com/tendermint/tendermint/common"
"github.com/tendermint/tendermint/config"
. "github.com/tendermint/tendermint/consensus/types"
"github.com/tendermint/tendermint/events"
2015-04-01 17:30:16 -07:00
mempl "github.com/tendermint/tendermint/mempool"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
2014-08-10 16:35:08 -07:00
)
2014-12-30 17:14:54 -08:00
const (
2015-03-18 01:02:04 -07:00
roundDuration0 = 10 * time.Second // The first round is 60 seconds long.
roundDurationDelta = 3 * time.Second // Each successive round lasts 15 seconds longer.
2014-12-30 17:14:54 -08:00
roundDeadlinePrevote = float64(1.0 / 3.0) // When the prevote is due.
roundDeadlinePrecommit = float64(2.0 / 3.0) // When the precommit vote is due.
newHeightDelta = roundDuration0 / 3 // The time to wait between commitTime and startTime of next consensus rounds.
)
var (
ErrInvalidProposalSignature = errors.New("Error invalid proposal signature")
)
//-----------------------------------------------------------------------------
// RoundStep enum type
type RoundStep uint8
2014-09-14 15:37:32 -07:00
const (
2014-10-30 03:32:09 -07:00
RoundStepNewHeight = RoundStep(0x00) // Round0 for new height started, wait til CommitTime + Delta
RoundStepNewRound = RoundStep(0x01) // Pseudostep, immediately goes to RoundStepPropose
RoundStepPropose = RoundStep(0x10) // Did propose, gossip proposal
RoundStepPrevote = RoundStep(0x11) // Did prevote, gossip prevotes
RoundStepPrecommit = RoundStep(0x12) // Did precommit, gossip precommits
RoundStepCommit = RoundStep(0x20) // Entered commit state machine
2014-12-30 17:14:54 -08:00
)
func (rs RoundStep) String() string {
switch rs {
case RoundStepNewHeight:
return "RoundStepNewHeight"
case RoundStepNewRound:
return "RoundStepNewRound"
case RoundStepPropose:
return "RoundStepPropose"
case RoundStepPrevote:
return "RoundStepPrevote"
case RoundStepPrecommit:
return "RoundStepPrecommit"
case RoundStepCommit:
return "RoundStepCommit"
default:
panic(Fmt("Unknown RoundStep %X", rs))
}
}
2014-10-30 03:32:09 -07:00
2014-12-30 17:14:54 -08:00
//-----------------------------------------------------------------------------
// RoundAction enum type
type RoundActionType uint8
const (
2014-10-30 03:32:09 -07:00
RoundActionPropose = RoundActionType(0xA0) // Propose and goto RoundStepPropose
RoundActionPrevote = RoundActionType(0xA1) // Prevote and goto RoundStepPrevote
RoundActionPrecommit = RoundActionType(0xA2) // Precommit and goto RoundStepPrecommit
RoundActionTryCommit = RoundActionType(0xC0) // Goto RoundStepCommit, or RoundStepPropose for next round.
RoundActionCommit = RoundActionType(0xC1) // Goto RoundStepCommit upon +2/3 commits
RoundActionTryFinalize = RoundActionType(0xC2) // Maybe goto RoundStepPropose for next round.
2014-09-14 15:37:32 -07:00
)
func (rat RoundActionType) String() string {
switch rat {
2014-12-30 17:14:54 -08:00
case RoundActionPropose:
return "RoundActionPropose"
case RoundActionPrevote:
return "RoundActionPrevote"
case RoundActionPrecommit:
return "RoundActionPrecommit"
case RoundActionTryCommit:
return "RoundActionTryCommit"
case RoundActionCommit:
return "RoundActionCommit"
case RoundActionTryFinalize:
return "RoundActionTryFinalize"
default:
panic(Fmt("Unknown RoundAction %X", rat))
2014-12-30 17:14:54 -08:00
}
}
//-----------------------------------------------------------------------------
2014-08-10 16:35:08 -07:00
2014-10-30 03:32:09 -07:00
type RoundAction struct {
Height uint // The block height for which consensus is reaching for.
Round uint // The round number at given height.
2014-10-30 03:32:09 -07:00
Action RoundActionType // Action to perform.
}
func (ra RoundAction) String() string {
return Fmt("RoundAction{H:%v R:%v A:%v}", ra.Height, ra.Round, ra.Action)
}
2014-10-30 03:32:09 -07:00
//-----------------------------------------------------------------------------
2014-09-14 15:37:32 -07:00
// Immutable when returned from ConsensusState.GetRoundState()
type RoundState struct {
Height uint // Height we are working on
Round uint
2014-10-26 13:26:27 -07:00
Step RoundStep
StartTime time.Time
CommitTime time.Time // Time when +2/3 commits were found
Validators *sm.ValidatorSet
2014-10-26 13:26:27 -07:00
Proposal *Proposal
ProposalBlock *types.Block
ProposalBlockParts *types.PartSet
2014-10-26 13:26:27 -07:00
ProposalPOL *POL
ProposalPOLParts *types.PartSet
LockedBlock *types.Block
LockedBlockParts *types.PartSet
2014-10-26 13:26:27 -07:00
LockedPOL *POL // Rarely needed, so no LockedPOLParts.
Prevotes *VoteSet
Precommits *VoteSet
Commits *VoteSet
LastCommits *VoteSet
PrivValidator *sm.PrivValidator
2014-09-14 15:37:32 -07:00
}
2014-10-18 01:42:33 -07:00
func (rs *RoundState) String() string {
2014-12-23 01:35:54 -08:00
return rs.StringIndented("")
2014-10-18 01:42:33 -07:00
}
2014-12-23 01:35:54 -08:00
func (rs *RoundState) StringIndented(indent string) string {
2014-10-18 01:42:33 -07:00
return fmt.Sprintf(`RoundState{
%s H:%v R:%v S:%v
%s StartTime: %v
%s CommitTime: %v
2014-10-18 01:42:33 -07:00
%s Validators: %v
%s Proposal: %v
%s ProposalBlock: %v %v
%s ProposalPOL: %v %v
%s LockedBlock: %v %v
2014-10-18 01:42:33 -07:00
%s LockedPOL: %v
%s Prevotes: %v
%s Precommits: %v
%s Commits: %v
%s LastCommits: %v
2014-10-18 01:42:33 -07:00
%s}`,
indent, rs.Height, rs.Round, rs.Step,
indent, rs.StartTime,
indent, rs.CommitTime,
2014-12-23 01:35:54 -08:00
indent, rs.Validators.StringIndented(indent+" "),
2014-10-18 01:42:33 -07:00
indent, rs.Proposal,
2014-12-23 01:35:54 -08:00
indent, rs.ProposalBlockParts.StringShort(), rs.ProposalBlock.StringShort(),
indent, rs.ProposalPOLParts.StringShort(), rs.ProposalPOL.StringShort(),
indent, rs.LockedBlockParts.StringShort(), rs.LockedBlock.StringShort(),
indent, rs.LockedPOL.StringShort(),
indent, rs.Prevotes.StringIndented(indent+" "),
indent, rs.Precommits.StringIndented(indent+" "),
indent, rs.Commits.StringIndented(indent+" "),
indent, rs.LastCommits.StringShort(),
2014-10-18 01:42:33 -07:00
indent)
}
2014-12-23 01:35:54 -08:00
func (rs *RoundState) StringShort() string {
2014-12-30 17:14:54 -08:00
return fmt.Sprintf(`RoundState{H:%v R:%v S:%v ST:%v}`,
rs.Height, rs.Round, rs.Step, rs.StartTime)
}
2014-10-30 03:32:09 -07:00
//-----------------------------------------------------------------------------
2014-09-14 15:37:32 -07:00
// Tracks consensus state across block heights and rounds.
type ConsensusState struct {
2014-10-30 03:32:09 -07:00
started uint32
stopped uint32
quit chan struct{}
blockStore *bc.BlockStore
mempoolReactor *mempl.MempoolReactor
runActionCh chan RoundAction
newStepCh chan *RoundState
2014-09-14 15:37:32 -07:00
2014-10-07 01:05:54 -07:00
mtx sync.Mutex
RoundState
state *sm.State // State until height-1.
stagedBlock *types.Block // Cache last staged block.
stagedState *sm.State // Cache result of staged block.
lastCommitVoteHeight uint // Last called commitVoteBlock() or saveCommitVoteBlock() on.
evsw *events.EventSwitch
2014-09-14 15:37:32 -07:00
}
func NewConsensusState(state *sm.State, blockStore *bc.BlockStore, mempoolReactor *mempl.MempoolReactor) *ConsensusState {
2014-09-14 15:37:32 -07:00
cs := &ConsensusState{
quit: make(chan struct{}),
blockStore: blockStore,
mempoolReactor: mempoolReactor,
runActionCh: make(chan RoundAction, 1),
newStepCh: make(chan *RoundState, 1),
2014-09-14 15:37:32 -07:00
}
cs.updateToState(state, true)
return cs
2014-08-10 16:35:08 -07:00
}
func (cs *ConsensusState) GetState() *sm.State {
2015-01-11 14:27:46 -08:00
cs.mtx.Lock()
defer cs.mtx.Unlock()
return cs.state.Copy()
}
2014-09-14 15:37:32 -07:00
func (cs *ConsensusState) GetRoundState() *RoundState {
cs.mtx.Lock()
defer cs.mtx.Unlock()
2014-10-30 03:32:09 -07:00
return cs.getRoundState()
}
func (cs *ConsensusState) getRoundState() *RoundState {
2014-09-14 15:37:32 -07:00
rs := cs.RoundState // copy
return &rs
2014-08-10 16:35:08 -07:00
}
2014-10-30 03:32:09 -07:00
func (cs *ConsensusState) NewStepCh() chan *RoundState {
return cs.newStepCh
}
func (cs *ConsensusState) Start() {
if atomic.CompareAndSwapUint32(&cs.started, 0, 1) {
log.Info("Starting ConsensusState")
go cs.stepTransitionRoutine()
}
}
func (cs *ConsensusState) Stop() {
if atomic.CompareAndSwapUint32(&cs.stopped, 0, 1) {
log.Info("Stopping ConsensusState")
close(cs.quit)
}
}
func (cs *ConsensusState) IsStopped() bool {
return atomic.LoadUint32(&cs.stopped) == 1
}
func (cs *ConsensusState) queueAction(ra RoundAction) {
go func() {
cs.runActionCh <- ra
}()
}
2014-10-30 03:32:09 -07:00
// Source of all round state transitions (and votes).
func (cs *ConsensusState) stepTransitionRoutine() {
// For clarity, all state transitions that happen after some timeout are here.
// Schedule the next action by pushing a RoundAction{} to cs.runActionCh.
scheduleNextAction := func() {
go func() {
// NOTE: We can push directly to runActionCh because
// we're running in a separate goroutine, which avoids deadlocks.
2014-10-30 03:32:09 -07:00
rs := cs.getRoundState()
round, roundStartTime, roundDuration, _, elapsedRatio := calcRoundInfo(rs.StartTime)
2015-01-20 15:31:31 -08:00
log.Debug("Scheduling next action", "height", rs.Height, "round", round, "step", rs.Step, "roundStartTime", roundStartTime, "elapsedRatio", elapsedRatio)
2014-10-30 03:32:09 -07:00
switch rs.Step {
case RoundStepNewHeight:
// We should run RoundActionPropose when rs.StartTime passes.
if elapsedRatio < 0 {
// startTime is in the future.
time.Sleep(time.Duration((-1.0 * elapsedRatio) * float64(roundDuration)))
}
cs.runActionCh <- RoundAction{rs.Height, rs.Round, RoundActionPropose}
case RoundStepNewRound:
// Pseudostep: Immediately goto propose.
cs.runActionCh <- RoundAction{rs.Height, rs.Round, RoundActionPropose}
case RoundStepPropose:
// Wake up when it's time to vote.
time.Sleep(time.Duration((roundDeadlinePrevote - elapsedRatio) * float64(roundDuration)))
cs.runActionCh <- RoundAction{rs.Height, rs.Round, RoundActionPrevote}
case RoundStepPrevote:
// Wake up when it's time to precommit.
time.Sleep(time.Duration((roundDeadlinePrecommit - elapsedRatio) * float64(roundDuration)))
cs.runActionCh <- RoundAction{rs.Height, rs.Round, RoundActionPrecommit}
case RoundStepPrecommit:
// Wake up when the round is over.
time.Sleep(time.Duration((1.0 - elapsedRatio) * float64(roundDuration)))
cs.runActionCh <- RoundAction{rs.Height, rs.Round, RoundActionTryCommit}
case RoundStepCommit:
// There's nothing to scheudle, we're waiting for
// ProposalBlockParts.IsComplete() &&
// Commits.HasTwoThirdsMajority()
panic("The next action from RoundStepCommit is not scheduled by time")
default:
panic("Should not happen")
}
}()
}
scheduleNextAction()
// NOTE: All ConsensusState.RunAction*() calls come from here.
// Since only one routine calls them, it is safe to assume that
// the RoundState Height/Round/Step won't change concurrently.
// However, other fields like Proposal could change concurrent
// due to gossip routines.
2014-10-30 03:32:09 -07:00
ACTION_LOOP:
for {
var roundAction RoundAction
select {
case roundAction = <-cs.runActionCh:
case <-cs.quit:
return
}
height, round, action := roundAction.Height, roundAction.Round, roundAction.Action
rs := cs.GetRoundState()
// Continue if action is not relevant
if height != rs.Height {
log.Debug("Discarding round action: Height mismatch", "height", rs.Height, "roundAction", roundAction)
2014-10-30 03:32:09 -07:00
continue
}
// If action <= RoundActionPrecommit, the round must match too.
if action <= RoundActionPrecommit && round != rs.Round {
log.Debug("Discarding round action: Round mismatch", "round", rs.Round, "roundAction", roundAction)
2014-10-30 03:32:09 -07:00
continue
}
log.Info("Running round action", "height", rs.Height, "round", rs.Round, "step", rs.Step, "roundAction", roundAction, "startTime", rs.StartTime)
2014-10-30 03:32:09 -07:00
// Run action
switch action {
case RoundActionPropose:
if rs.Step != RoundStepNewHeight && rs.Step != RoundStepNewRound {
continue ACTION_LOOP
}
cs.RunActionPropose(rs.Height, rs.Round)
scheduleNextAction()
continue ACTION_LOOP
case RoundActionPrevote:
if rs.Step >= RoundStepPrevote {
continue ACTION_LOOP
}
cs.RunActionPrevote(rs.Height, rs.Round)
scheduleNextAction()
continue ACTION_LOOP
case RoundActionPrecommit:
if rs.Step >= RoundStepPrecommit {
continue ACTION_LOOP
}
cs.RunActionPrecommit(rs.Height, rs.Round)
scheduleNextAction()
continue ACTION_LOOP
case RoundActionTryCommit:
if rs.Step >= RoundStepCommit {
continue ACTION_LOOP
}
if rs.Precommits.HasTwoThirdsMajority() {
// Enter RoundStepCommit and commit.
cs.RunActionCommit(rs.Height)
continue ACTION_LOOP
} else {
// Could not commit, move onto next round.
cs.SetupNewRound(rs.Height, rs.Round+1)
// cs.Step is now at RoundStepNewRound
scheduleNextAction()
continue ACTION_LOOP
}
case RoundActionCommit:
if rs.Step >= RoundStepCommit {
continue ACTION_LOOP
}
// Enter RoundStepCommit and commit.
cs.RunActionCommit(rs.Height)
continue ACTION_LOOP
2014-10-30 03:32:09 -07:00
case RoundActionTryFinalize:
if cs.TryFinalizeCommit(rs.Height) {
// Now at new height
// cs.Step is at RoundStepNewHeight or RoundStepNewRound.
2015-04-13 18:26:41 -07:00
newBlock := cs.blockStore.LoadBlock(cs.state.LastBlockHeight)
cs.evsw.FireEvent(types.EventStringNewBlock(), newBlock)
2014-10-30 03:32:09 -07:00
scheduleNextAction()
continue ACTION_LOOP
} else {
// do not schedule next action.
continue ACTION_LOOP
}
default:
panic("Unknown action")
}
// For clarity, ensure that all switch cases call "continue"
panic("Should not happen.")
}
}
// Updates ConsensusState and increments height to match that of state.
// If calculated round is greater than 0 (based on BlockTime or calculated StartTime)
// then also sets up the appropriate round, and cs.Step becomes RoundStepNewRound.
// Otherwise the round is 0 and cs.Step becomes RoundStepNewHeight.
func (cs *ConsensusState) updateToState(state *sm.State, contiguous bool) {
2014-09-14 15:37:32 -07:00
// Sanity check state.
if contiguous && cs.Height > 0 && cs.Height != state.LastBlockHeight {
2014-12-29 18:09:06 -08:00
panic(Fmt("updateToState() expected state height of %v but found %v",
cs.Height, state.LastBlockHeight))
2014-09-14 15:37:32 -07:00
}
// Reset fields based on state.
2014-10-12 21:14:10 -07:00
validators := state.BondedValidators
height := state.LastBlockHeight + 1 // next desired block height
// RoundState fields
2014-09-14 15:37:32 -07:00
cs.Height = height
cs.Round = 0
2014-10-30 03:32:09 -07:00
cs.Step = RoundStepNewHeight
if cs.CommitTime.IsZero() {
cs.StartTime = state.LastBlockTime.Add(newHeightDelta)
} else {
2014-10-30 03:32:09 -07:00
cs.StartTime = cs.CommitTime.Add(newHeightDelta)
}
cs.CommitTime = time.Time{}
2014-09-14 15:37:32 -07:00
cs.Validators = validators
cs.Proposal = nil
cs.ProposalBlock = nil
2014-10-26 13:26:27 -07:00
cs.ProposalBlockParts = nil
2014-09-14 15:37:32 -07:00
cs.ProposalPOL = nil
2014-10-26 13:26:27 -07:00
cs.ProposalPOLParts = nil
2014-09-14 15:37:32 -07:00
cs.LockedBlock = nil
2014-10-26 13:26:27 -07:00
cs.LockedBlockParts = nil
2014-09-14 15:37:32 -07:00
cs.LockedPOL = nil
cs.Prevotes = NewVoteSet(height, 0, types.VoteTypePrevote, validators)
cs.Precommits = NewVoteSet(height, 0, types.VoteTypePrecommit, validators)
cs.LastCommits = cs.Commits
cs.Commits = NewVoteSet(height, 0, types.VoteTypeCommit, validators)
2014-09-14 15:37:32 -07:00
2014-10-07 01:05:54 -07:00
cs.state = state
2014-09-14 15:37:32 -07:00
cs.stagedBlock = nil
cs.stagedState = nil
// Update the round if we need to.
round := calcRound(cs.StartTime)
if round > 0 {
2014-10-30 03:32:09 -07:00
cs.setupNewRound(round)
2014-09-14 15:37:32 -07:00
}
// If we've timed out, then send rebond tx.
if cs.PrivValidator != nil && cs.state.UnbondingValidators.HasAddress(cs.PrivValidator.Address) {
rebondTx := &types.RebondTx{
Address: cs.PrivValidator.Address,
Height: cs.Height + 1,
}
err := cs.PrivValidator.SignRebondTx(rebondTx)
if err == nil {
log.Info("Signed and broadcast RebondTx", "height", cs.Height, "round", cs.Round, "tx", rebondTx)
cs.mempoolReactor.BroadcastTx(rebondTx)
} else {
log.Warn("Error signing RebondTx", "height", cs.Height, "round", cs.Round, "tx", rebondTx, "error", err)
}
}
2014-09-14 15:37:32 -07:00
}
// After the call cs.Step becomes RoundStepNewRound.
func (cs *ConsensusState) setupNewRound(round uint) {
2014-10-30 03:32:09 -07:00
// Sanity check
if round == 0 {
panic("setupNewRound() should never be called for round 0")
2014-09-14 15:37:32 -07:00
}
// Increment all the way to round.
validators := cs.Validators.Copy()
validators.IncrementAccum(round - cs.Round)
2014-09-14 15:37:32 -07:00
cs.Round = round
2014-10-30 03:32:09 -07:00
cs.Step = RoundStepNewRound
2014-09-14 15:37:32 -07:00
cs.Validators = validators
cs.Proposal = nil
cs.ProposalBlock = nil
2014-10-26 13:26:27 -07:00
cs.ProposalBlockParts = nil
2014-09-14 15:37:32 -07:00
cs.ProposalPOL = nil
2014-10-26 13:26:27 -07:00
cs.ProposalPOLParts = nil
cs.Prevotes = NewVoteSet(cs.Height, round, types.VoteTypePrevote, validators)
cs.Prevotes.AddFromCommits(cs.Commits)
cs.Precommits = NewVoteSet(cs.Height, round, types.VoteTypePrecommit, validators)
2014-10-15 20:15:38 -07:00
cs.Precommits.AddFromCommits(cs.Commits)
2014-09-14 15:37:32 -07:00
}
func (cs *ConsensusState) SetPrivValidator(priv *sm.PrivValidator) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
2014-09-14 15:37:32 -07:00
cs.PrivValidator = priv
2014-08-10 16:35:08 -07:00
}
2014-10-21 23:30:18 -07:00
//-----------------------------------------------------------------------------
2014-08-10 16:35:08 -07:00
2014-10-30 03:32:09 -07:00
// Set up the round to desired round and set step to RoundStepNewRound
func (cs *ConsensusState) SetupNewRound(height uint, desiredRound uint) bool {
2014-10-30 03:32:09 -07:00
cs.mtx.Lock()
defer cs.mtx.Unlock()
if cs.Height != height {
return false
}
if desiredRound <= cs.Round {
return false
}
cs.setupNewRound(desiredRound)
// c.Step is now RoundStepNewRound
2014-10-30 03:32:09 -07:00
cs.newStepCh <- cs.getRoundState()
return true
}
func (cs *ConsensusState) RunActionPropose(height uint, round uint) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
2014-10-21 01:18:46 -07:00
if cs.Height != height || cs.Round != round {
return
}
defer func() {
cs.Step = RoundStepPropose
cs.newStepCh <- cs.getRoundState()
}()
2014-09-14 15:37:32 -07:00
2014-10-30 03:32:09 -07:00
// Nothing to do if it's not our turn.
if cs.PrivValidator == nil {
return
}
if !bytes.Equal(cs.Validators.Proposer().Address, cs.PrivValidator.Address) {
log.Debug("Not our turn to propose", "proposer", cs.Validators.Proposer().Address, "privValidator", cs.PrivValidator)
2014-08-10 16:35:08 -07:00
return
} else {
log.Debug("Our turn to propose", "proposer", cs.Validators.Proposer().Address, "privValidator", cs.PrivValidator)
2014-08-10 16:35:08 -07:00
}
var block *types.Block
var blockParts *types.PartSet
2014-09-14 15:37:32 -07:00
var pol *POL
var polParts *types.PartSet
2014-09-14 15:37:32 -07:00
// Decide on block and POL
if cs.LockedBlock != nil {
// If we're locked onto a block, just choose that.
2015-01-15 22:43:15 -08:00
block = cs.LockedBlock
2014-10-26 13:26:27 -07:00
blockParts = cs.LockedBlockParts
2014-09-14 15:37:32 -07:00
pol = cs.LockedPOL
2014-08-10 16:35:08 -07:00
} else {
2014-10-30 03:32:09 -07:00
// Otherwise we should create a new proposal.
var validation *types.Validation
2014-10-21 01:18:46 -07:00
if cs.Height == 1 {
// We're creating a proposal for the first block.
// The validation is empty.
validation = &types.Validation{}
} else if cs.LastCommits.HasTwoThirdsMajority() {
// Make the validation from LastCommits
validation = cs.LastCommits.MakeValidation()
2014-10-21 01:18:46 -07:00
} else {
// Upon reboot, we may have to use SeenValidation
validation = cs.blockStore.LoadSeenValidation(height - 1)
if validation == nil {
// We just don't have any validation for the previous block
log.Debug("Cannot propose anything: No validation for the previous block.")
2014-10-21 01:18:46 -07:00
return
}
}
txs := cs.mempoolReactor.Mempool.GetProposalTxs()
block = &types.Block{
Header: &types.Header{
2015-03-16 23:16:26 -07:00
Network: config.App().GetString("Network"),
Height: cs.Height,
Time: time.Now(),
Fees: 0, // TODO fees
2014-12-23 01:35:54 -08:00
NumTxs: uint(len(txs)),
LastBlockHash: cs.state.LastBlockHash,
LastBlockParts: cs.state.LastBlockParts,
StateHash: nil, // Will set afterwards.
},
2014-10-21 01:18:46 -07:00
Validation: validation,
Data: &types.Data{
Txs: txs,
},
}
// Set the types.Header.StateHash.
err := cs.state.SetBlockStateHash(block)
if err != nil {
log.Error("Error setting state hash", "error", err)
return
}
blockParts = block.MakePartSet()
2014-09-14 15:37:32 -07:00
pol = cs.LockedPOL // If exists, is a PoUnlock.
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
if pol != nil {
polParts = pol.MakePartSet()
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
// Make proposal
2014-10-30 03:32:09 -07:00
proposal := NewProposal(cs.Height, cs.Round, blockParts.Header(), polParts.Header())
err := cs.PrivValidator.SignProposal(proposal)
if err == nil {
log.Info("Signed and set proposal", "height", cs.Height, "round", cs.Round, "proposal", proposal)
2015-01-15 22:43:15 -08:00
log.Debug(Fmt("Signed and set proposal block: %v", block))
// Set fields
cs.Proposal = proposal
2015-01-15 22:43:15 -08:00
cs.ProposalBlock = block
cs.ProposalBlockParts = blockParts
cs.ProposalPOL = pol
cs.ProposalPOLParts = polParts
} else {
log.Warn("Error signing proposal", "height", cs.Height, "round", cs.Round, "error", err)
}
2014-08-10 16:35:08 -07:00
}
2014-10-30 03:32:09 -07:00
// Prevote for LockedBlock if we're locked, or ProposealBlock if valid.
// Otherwise vote nil.
func (cs *ConsensusState) RunActionPrevote(height uint, round uint) {
2014-10-21 01:18:46 -07:00
cs.mtx.Lock()
defer cs.mtx.Unlock()
if cs.Height != height || cs.Round != round {
2014-12-29 18:09:06 -08:00
panic(Fmt("RunActionPrevote(%v/%v), expected %v/%v", height, round, cs.Height, cs.Round))
2014-10-21 01:18:46 -07:00
}
defer func() {
cs.Step = RoundStepPrevote
cs.newStepCh <- cs.getRoundState()
}()
2014-10-21 01:18:46 -07:00
// If a block is locked, prevote that.
if cs.LockedBlock != nil {
cs.signAddVote(types.VoteTypePrevote, cs.LockedBlock.Hash(), cs.LockedBlockParts.Header())
2014-10-30 03:32:09 -07:00
return
}
2014-10-30 03:32:09 -07:00
// If ProposalBlock is nil, prevote nil.
if cs.ProposalBlock == nil {
2014-12-29 18:09:06 -08:00
log.Warn("ProposalBlock is nil")
cs.signAddVote(types.VoteTypePrevote, nil, types.PartSetHeader{})
2014-10-30 03:32:09 -07:00
return
2014-10-21 01:18:46 -07:00
}
2014-10-30 03:32:09 -07:00
// Try staging cs.ProposalBlock
err := cs.stageBlock(cs.ProposalBlock, cs.ProposalBlockParts)
2014-10-21 01:18:46 -07:00
if err != nil {
2014-10-30 03:32:09 -07:00
// ProposalBlock is invalid, prevote nil.
2014-12-29 18:39:19 -08:00
log.Warn("ProposalBlock is invalid", "error", err)
cs.signAddVote(types.VoteTypePrevote, nil, types.PartSetHeader{})
2014-10-30 03:32:09 -07:00
return
2014-10-21 01:18:46 -07:00
}
2014-10-30 03:32:09 -07:00
// Prevote cs.ProposalBlock
cs.signAddVote(types.VoteTypePrevote, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header())
2014-10-30 03:32:09 -07:00
return
2014-10-21 01:18:46 -07:00
}
2014-10-30 03:32:09 -07:00
// Lock & Precommit the ProposalBlock if we have enough prevotes for it,
2014-10-21 01:18:46 -07:00
// or unlock an existing lock if +2/3 of prevotes were nil.
func (cs *ConsensusState) RunActionPrecommit(height uint, round uint) {
2014-09-14 15:37:32 -07:00
cs.mtx.Lock()
defer cs.mtx.Unlock()
if cs.Height != height || cs.Round != round {
2014-12-29 18:09:06 -08:00
panic(Fmt("RunActionPrecommit(%v/%v), expected %v/%v", height, round, cs.Height, cs.Round))
2014-09-14 15:37:32 -07:00
}
defer func() {
cs.Step = RoundStepPrecommit
cs.newStepCh <- cs.getRoundState()
}()
2014-09-14 15:37:32 -07:00
2014-10-30 03:32:09 -07:00
hash, partsHeader, ok := cs.Prevotes.TwoThirdsMajority()
if !ok {
// If we don't have two thirds of prevotes,
// don't do anything at all.
return
2014-09-14 15:37:32 -07:00
}
2014-08-10 16:35:08 -07:00
2014-10-30 03:32:09 -07:00
// Remember this POL. (hash may be nil)
cs.LockedPOL = cs.Prevotes.MakePOL()
2014-09-14 15:37:32 -07:00
2014-10-30 03:32:09 -07:00
// If +2/3 prevoted nil. Just unlock.
if len(hash) == 0 {
cs.LockedBlock = nil
cs.LockedBlockParts = nil
return
}
2014-09-14 15:37:32 -07:00
2014-10-30 03:32:09 -07:00
// If +2/3 prevoted for already locked block, precommit it.
if cs.LockedBlock.HashesTo(hash) {
cs.signAddVote(types.VoteTypePrecommit, hash, partsHeader)
2014-10-30 03:32:09 -07:00
return
}
2014-09-14 15:37:32 -07:00
2014-10-30 03:32:09 -07:00
// If +2/3 prevoted for cs.ProposalBlock, lock it and precommit it.
if cs.ProposalBlock.HashesTo(hash) {
// Validate the block.
if err := cs.stageBlock(cs.ProposalBlock, cs.ProposalBlockParts); err != nil {
2014-10-30 03:32:09 -07:00
// Prevent zombies.
2014-12-29 18:39:19 -08:00
log.Warn("+2/3 prevoted for an invalid block", "error", err)
2014-10-30 03:32:09 -07:00
return
}
2014-10-30 03:32:09 -07:00
cs.LockedBlock = cs.ProposalBlock
cs.LockedBlockParts = cs.ProposalBlockParts
cs.signAddVote(types.VoteTypePrecommit, hash, partsHeader)
2014-10-30 03:32:09 -07:00
return
2014-09-14 15:37:32 -07:00
}
2014-10-30 03:32:09 -07:00
// We don't have the block that validators prevoted.
// Unlock if we're locked.
cs.LockedBlock = nil
cs.LockedBlockParts = nil
return
2014-08-10 16:35:08 -07:00
}
2014-10-30 03:32:09 -07:00
// Enter commit step. See the diagram for details.
// There are two ways to enter this step:
// * After the Precommit step with +2/3 precommits, or,
// * Upon +2/3 commits regardless of current step
// Either way this action is run at most once per round.
func (cs *ConsensusState) RunActionCommit(height uint) {
2014-10-21 01:18:46 -07:00
cs.mtx.Lock()
defer cs.mtx.Unlock()
2014-10-30 03:32:09 -07:00
if cs.Height != height {
2014-12-29 18:09:06 -08:00
panic(Fmt("RunActionCommit(%v), expected %v", height, cs.Height))
2014-10-30 03:32:09 -07:00
}
defer func() {
cs.Step = RoundStepCommit
cs.newStepCh <- cs.getRoundState()
}()
// Sanity check.
2014-10-30 03:32:09 -07:00
// There are two ways to enter:
// 1. +2/3 precommits at the end of RoundStepPrecommit
// 2. +2/3 commits at any time
hash, partsHeader, ok := cs.Precommits.TwoThirdsMajority()
if !ok {
hash, partsHeader, ok = cs.Commits.TwoThirdsMajority()
if !ok {
panic("RunActionCommit() expects +2/3 precommits or commits")
}
}
// Clear the Locked* fields and use cs.Proposed*
if cs.LockedBlock.HashesTo(hash) {
cs.ProposalBlock = cs.LockedBlock
cs.ProposalBlockParts = cs.LockedBlockParts
cs.LockedBlock = nil
cs.LockedBlockParts = nil
cs.LockedPOL = nil
}
// If we don't have the block being committed, set up to get it.
if !cs.ProposalBlock.HashesTo(hash) {
2014-11-03 15:50:23 -08:00
if !cs.ProposalBlockParts.HasHeader(partsHeader) {
2014-10-30 03:32:09 -07:00
// We're getting the wrong block.
// Set up ProposalBlockParts and keep waiting.
cs.ProposalBlock = nil
cs.ProposalBlockParts = types.NewPartSetFromHeader(partsHeader)
2014-10-30 03:32:09 -07:00
} else {
// We just need to keep waiting.
}
} else {
// We have the block, so sign a Commit-vote.
cs.commitVoteBlock(cs.ProposalBlock, cs.ProposalBlockParts)
}
// If we have the block AND +2/3 commits, queue RoundActionTryFinalize.
// Round will immediately become finalized.
if cs.ProposalBlock.HashesTo(hash) && cs.Commits.HasTwoThirdsMajority() {
cs.queueAction(RoundAction{cs.Height, cs.Round, RoundActionTryFinalize})
}
}
2014-10-30 03:32:09 -07:00
// Returns true if Finalize happened, which increments height && sets
// the step to RoundStepNewHeight (or RoundStepNewRound, but probably not).
func (cs *ConsensusState) TryFinalizeCommit(height uint) bool {
cs.mtx.Lock()
defer cs.mtx.Unlock()
2014-10-30 03:32:09 -07:00
if cs.Height != height {
2014-12-29 18:09:06 -08:00
panic(Fmt("TryFinalizeCommit(%v), expected %v", height, cs.Height))
2014-10-21 01:18:46 -07:00
}
2014-10-30 03:32:09 -07:00
if cs.Step == RoundStepCommit &&
cs.Commits.HasTwoThirdsMajority() &&
cs.ProposalBlockParts.IsComplete() {
// Sanity check
if cs.ProposalBlock == nil {
2014-12-29 18:09:06 -08:00
panic(Fmt("Expected ProposalBlock to exist"))
2014-10-30 03:32:09 -07:00
}
hash, header, _ := cs.Commits.TwoThirdsMajority()
if !cs.ProposalBlock.HashesTo(hash) {
// XXX See: https://github.com/tendermint/tendermint/issues/44
2015-01-06 15:51:41 -08:00
panic(Fmt("Expected ProposalBlock to hash to commit hash. Expected %X, got %X", hash, cs.ProposalBlock.Hash()))
2014-10-30 03:32:09 -07:00
}
2014-11-03 15:50:23 -08:00
if !cs.ProposalBlockParts.HasHeader(header) {
2014-12-29 18:09:06 -08:00
panic(Fmt("Expected ProposalBlockParts header to be commit header"))
2014-10-30 03:32:09 -07:00
}
err := cs.stageBlock(cs.ProposalBlock, cs.ProposalBlockParts)
2014-10-30 03:32:09 -07:00
if err == nil {
2014-12-29 18:09:06 -08:00
log.Debug(Fmt("Finalizing commit of block: %v", cs.ProposalBlock))
2014-12-31 16:46:49 -08:00
// We have the block, so save/stage/sign-commit-vote.
cs.saveCommitVoteBlock(cs.ProposalBlock, cs.ProposalBlockParts, cs.Commits)
2014-10-30 03:32:09 -07:00
// Increment height.
cs.updateToState(cs.stagedState, true)
2014-10-30 03:32:09 -07:00
// cs.Step is now RoundStepNewHeight or RoundStepNewRound
cs.newStepCh <- cs.getRoundState()
return true
} else {
// Prevent zombies.
// TODO: Does this ever happen?
2014-12-29 18:09:06 -08:00
panic(Fmt("+2/3 committed an invalid block: %v", err))
2014-10-30 03:32:09 -07:00
}
}
return false
}
2014-10-21 23:30:18 -07:00
//-----------------------------------------------------------------------------
func (cs *ConsensusState) SetProposal(proposal *Proposal) error {
cs.mtx.Lock()
defer cs.mtx.Unlock()
// Already have one
if cs.Proposal != nil {
return nil
}
2014-10-30 03:32:09 -07:00
// Does not apply
2014-10-21 23:30:18 -07:00
if proposal.Height != cs.Height || proposal.Round != cs.Round {
return nil
}
2014-10-30 03:32:09 -07:00
// We don't care about the proposal if we're already in RoundStepCommit.
if cs.Step == RoundStepCommit {
return nil
}
2014-10-21 23:30:18 -07:00
// Verify signature
if !cs.Validators.Proposer().PubKey.VerifyBytes(account.SignBytes(proposal), proposal.Signature) {
2014-10-21 23:30:18 -07:00
return ErrInvalidProposalSignature
}
cs.Proposal = proposal
cs.ProposalBlockParts = types.NewPartSetFromHeader(proposal.BlockParts)
cs.ProposalPOLParts = types.NewPartSetFromHeader(proposal.POLParts)
2014-10-21 23:30:18 -07:00
return nil
}
// NOTE: block is not necessarily valid.
2014-10-30 03:32:09 -07:00
// NOTE: This function may increment the height.
func (cs *ConsensusState) AddProposalBlockPart(height uint, round uint, part *types.Part) (added bool, err error) {
2014-10-21 23:30:18 -07:00
cs.mtx.Lock()
defer cs.mtx.Unlock()
// Blocks might be reused, so round mismatch is OK
if cs.Height != height {
return false, nil
}
// We're not expecting a block part.
2014-10-30 03:32:09 -07:00
if cs.ProposalBlockParts == nil {
2014-10-21 23:30:18 -07:00
return false, nil // TODO: bad peer? Return error?
}
2014-10-26 13:26:27 -07:00
added, err = cs.ProposalBlockParts.AddPart(part)
2014-10-21 23:30:18 -07:00
if err != nil {
return added, err
}
2014-10-26 13:26:27 -07:00
if added && cs.ProposalBlockParts.IsComplete() {
2014-10-21 23:30:18 -07:00
var n int64
var err error
cs.ProposalBlock = binary.ReadBinary(&types.Block{}, cs.ProposalBlockParts.GetReader(), &n, &err).(*types.Block)
// If we're already in the commit step, try to finalize round.
if cs.Step == RoundStepCommit {
cs.queueAction(RoundAction{cs.Height, cs.Round, RoundActionTryFinalize})
}
2014-11-30 18:31:44 -08:00
// XXX If POL is valid, consider unlocking.
2014-10-21 23:30:18 -07:00
return true, err
}
return true, nil
}
// NOTE: POL is not necessarily valid.
func (cs *ConsensusState) AddProposalPOLPart(height uint, round uint, part *types.Part) (added bool, err error) {
2014-10-21 23:30:18 -07:00
cs.mtx.Lock()
defer cs.mtx.Unlock()
if cs.Height != height || cs.Round != round {
return false, nil
}
// We're not expecting a POL part.
2014-10-30 03:32:09 -07:00
if cs.ProposalPOLParts == nil {
2014-10-21 23:30:18 -07:00
return false, nil // TODO: bad peer? Return error?
}
2014-10-26 13:26:27 -07:00
added, err = cs.ProposalPOLParts.AddPart(part)
2014-10-21 23:30:18 -07:00
if err != nil {
return added, err
}
2014-10-26 13:26:27 -07:00
if added && cs.ProposalPOLParts.IsComplete() {
2014-10-21 23:30:18 -07:00
var n int64
var err error
cs.ProposalPOL = binary.ReadBinary(&POL{}, cs.ProposalPOLParts.GetReader(), &n, &err).(*POL)
2014-10-21 23:30:18 -07:00
return true, err
}
return true, nil
}
func (cs *ConsensusState) AddVote(address []byte, vote *types.Vote) (added bool, index uint, err error) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
return cs.addVote(address, vote)
}
//-----------------------------------------------------------------------------
func (cs *ConsensusState) addVote(address []byte, vote *types.Vote) (added bool, index uint, err error) {
2014-10-21 23:30:18 -07:00
switch vote.Type {
case types.VoteTypePrevote:
2014-10-21 23:30:18 -07:00
// Prevotes checks for height+round match.
return cs.Prevotes.Add(address, vote)
case types.VoteTypePrecommit:
2014-10-21 23:30:18 -07:00
// Precommits checks for height+round match.
return cs.Precommits.Add(address, vote)
case types.VoteTypeCommit:
if vote.Height == cs.Height {
// No need to check if vote.Round < cs.Round ...
// Prevotes && Precommits already checks that.
cs.Prevotes.Add(address, vote)
cs.Precommits.Add(address, vote)
added, index, err = cs.Commits.Add(address, vote)
if added && cs.Commits.HasTwoThirdsMajority() && cs.CommitTime.IsZero() {
cs.CommitTime = time.Now()
2014-12-29 18:09:06 -08:00
log.Debug(Fmt("Set CommitTime to %v", cs.CommitTime))
if cs.Step < RoundStepCommit {
cs.queueAction(RoundAction{cs.Height, cs.Round, RoundActionCommit})
} else {
cs.queueAction(RoundAction{cs.Height, cs.Round, RoundActionTryFinalize})
}
}
return added, index, err
2014-10-30 03:32:09 -07:00
}
if vote.Height+1 == cs.Height {
return cs.LastCommits.Add(address, vote)
}
return false, 0, nil
2014-10-21 23:30:18 -07:00
default:
panic("Unknown vote type")
}
}
func (cs *ConsensusState) stageBlock(block *types.Block, blockParts *types.PartSet) error {
2015-01-15 22:43:15 -08:00
if block == nil {
panic("Cannot stage nil block")
}
2014-09-14 15:37:32 -07:00
// Already staged?
2015-03-26 10:58:20 -07:00
blockHash := block.Hash()
if cs.stagedBlock != nil && len(blockHash) != 0 && bytes.Equal(cs.stagedBlock.Hash(), blockHash) {
2014-09-14 15:37:32 -07:00
return nil
}
// Create a copy of the state for staging
2014-10-14 13:11:54 -07:00
stateCopy := cs.state.Copy()
2014-09-14 15:37:32 -07:00
// Commit block onto the copied state.
2014-10-14 13:11:54 -07:00
// NOTE: Basic validation is done in state.AppendBlock().
err := sm.ExecBlock(stateCopy, block, blockParts.Header())
2014-09-14 15:37:32 -07:00
if err != nil {
return err
2014-08-10 16:35:08 -07:00
} else {
2015-01-15 22:43:15 -08:00
cs.stagedBlock = block
2014-09-14 15:37:32 -07:00
cs.stagedState = stateCopy
return nil
2014-08-10 16:35:08 -07:00
}
}
func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header types.PartSetHeader) *types.Vote {
if cs.PrivValidator == nil || !cs.Validators.HasAddress(cs.PrivValidator.Address) {
return nil
}
vote := &types.Vote{
2014-10-30 03:32:09 -07:00
Height: cs.Height,
Round: cs.Round,
Type: type_,
BlockHash: hash,
BlockParts: header,
}
err := cs.PrivValidator.SignVote(vote)
if err == nil {
log.Info("Signed and added vote", "height", cs.Height, "round", cs.Round, "vote", vote)
cs.addVote(cs.PrivValidator.Address, vote)
return vote
} else {
log.Warn("Error signing vote", "height", cs.Height, "round", cs.Round, "vote", vote, "error", err)
return nil
}
}
2014-10-30 03:32:09 -07:00
// sign a Commit-Vote
func (cs *ConsensusState) commitVoteBlock(block *types.Block, blockParts *types.PartSet) {
2014-10-30 03:32:09 -07:00
// The proposal must be valid.
2015-01-15 22:43:15 -08:00
if err := cs.stageBlock(block, blockParts); err != nil {
// Prevent zombies.
log.Warn("commitVoteBlock() an invalid block", "error", err)
2014-12-31 16:46:49 -08:00
return
}
// Commit-vote.
2015-01-15 22:43:15 -08:00
if cs.lastCommitVoteHeight < block.Height {
cs.signAddVote(types.VoteTypeCommit, block.Hash(), blockParts.Header())
2015-01-15 22:43:15 -08:00
cs.lastCommitVoteHeight = block.Height
2014-12-31 16:46:49 -08:00
} else {
log.Error("Duplicate commitVoteBlock() attempt", "lastCommitVoteHeight", cs.lastCommitVoteHeight, "types.Height", block.Height)
2014-12-31 16:46:49 -08:00
}
}
// Save Block, save the +2/3 Commits we've seen,
// and sign a Commit-Vote if we haven't already
func (cs *ConsensusState) saveCommitVoteBlock(block *types.Block, blockParts *types.PartSet, commits *VoteSet) {
2014-12-31 16:46:49 -08:00
2014-10-30 03:32:09 -07:00
// The proposal must be valid.
2015-01-15 22:43:15 -08:00
if err := cs.stageBlock(block, blockParts); err != nil {
2014-10-30 03:32:09 -07:00
// Prevent zombies.
log.Warn("saveCommitVoteBlock() an invalid block", "error", err)
2014-10-30 03:32:09 -07:00
return
}
// Save to blockStore.
2015-01-15 22:43:15 -08:00
if cs.blockStore.Height() < block.Height {
seenValidation := commits.MakeValidation()
2015-01-15 22:43:15 -08:00
cs.blockStore.SaveBlock(block, blockParts, seenValidation)
}
2014-10-30 03:32:09 -07:00
// Save the state.
2014-10-30 03:32:09 -07:00
cs.stagedState.Save()
// Update mempool.
2015-01-15 22:43:15 -08:00
cs.mempoolReactor.Mempool.ResetForBlockAndState(block, cs.stagedState)
2014-10-30 03:32:09 -07:00
// Commit-vote if we haven't already.
2015-01-15 22:43:15 -08:00
if cs.lastCommitVoteHeight < block.Height {
cs.signAddVote(types.VoteTypeCommit, block.Hash(), blockParts.Header())
2015-01-15 22:43:15 -08:00
cs.lastCommitVoteHeight = block.Height
}
2014-10-30 03:32:09 -07:00
}
// implements events.Eventable
func (cs *ConsensusState) SetEventSwitch(evsw *events.EventSwitch) {
cs.evsw = evsw
2015-04-13 18:26:41 -07:00
cs.state.SetEventSwitch(evsw)
}
2014-10-30 03:32:09 -07:00
//-----------------------------------------------------------------------------
// total duration of given round
func calcRoundDuration(round uint) time.Duration {
2014-10-30 03:32:09 -07:00
return roundDuration0 + roundDurationDelta*time.Duration(round)
}
// startTime is when round zero started.
func calcRoundStartTime(round uint, startTime time.Time) time.Time {
2014-10-30 03:32:09 -07:00
return startTime.Add(roundDuration0*time.Duration(round) +
roundDurationDelta*(time.Duration((int64(round)*int64(round)-int64(round))/2)))
}
// calculates the current round given startTime of round zero.
// NOTE: round is zero if startTime is in the future.
func calcRound(startTime time.Time) uint {
2014-10-30 03:32:09 -07:00
now := time.Now()
if now.Before(startTime) {
return 0
}
// Start + D_0 * R + D_delta * (R^2 - R)/2 <= Now; find largest integer R.
// D_delta * R^2 + (2D_0 - D_delta) * R + 2(Start - Now) <= 0.
// AR^2 + BR + C <= 0; A = D_delta, B = (2_D0 - D_delta), C = 2(Start - Now).
// R = Floor((-B + Sqrt(B^2 - 4AC))/2A)
A := float64(roundDurationDelta)
B := 2.0*float64(roundDuration0) - float64(roundDurationDelta)
C := 2.0 * float64(startTime.Sub(now))
R := math.Floor((-B + math.Sqrt(B*B-4.0*A*C)) / (2 * A))
if math.IsNaN(R) {
panic("Could not calc round, should not happen")
}
if R > math.MaxInt32 {
2014-12-29 18:09:06 -08:00
panic(Fmt("Could not calc round, round overflow: %v", R))
2014-10-30 03:32:09 -07:00
}
if R < 0 {
return 0
}
return uint(R)
2014-10-30 03:32:09 -07:00
}
// convenience
// NOTE: elapsedRatio can be negative if startTime is in the future.
func calcRoundInfo(startTime time.Time) (round uint, roundStartTime time.Time, roundDuration time.Duration,
2014-10-30 03:32:09 -07:00
roundElapsed time.Duration, elapsedRatio float64) {
round = calcRound(startTime)
roundStartTime = calcRoundStartTime(round, startTime)
roundDuration = calcRoundDuration(round)
roundElapsed = time.Now().Sub(roundStartTime)
elapsedRatio = float64(roundElapsed) / float64(roundDuration)
return
}